From 436dbd76989cb5096f50fb6b0e3f03fe2708b081 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 06:35:27 +0000 Subject: [PATCH 001/144] feat(storage): introduce storage proof worker pool - Added configuration for maximum and minimum storage proof workers. - Implemented a worker pool for processing storage proof tasks, improving efficiency by reusing transactions. - Updated `ProofTaskManager` to handle storage proof tasks via a dedicated channel. - Enhanced metrics to track storage proof requests and fallback scenarios. - Adjusted existing tests to accommodate the new storage worker functionality. --- Cargo.lock | 1 + crates/engine/primitives/src/config.rs | 42 ++ .../tree/src/tree/payload_processor/mod.rs | 2 + .../src/tree/payload_processor/multiproof.rs | 1 + crates/trie/parallel/Cargo.toml | 1 + crates/trie/parallel/src/proof.rs | 2 +- crates/trie/parallel/src/proof_task.rs | 627 ++++++++++++++++-- .../trie/parallel/src/proof_task_metrics.rs | 12 + 8 files changed, 629 insertions(+), 59 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8350347b6b4..fde6f2dc3aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10739,6 +10739,7 @@ dependencies = [ "alloy-primitives", "alloy-rlp", "codspeed-criterion-compat", + "crossbeam-channel", "dashmap 6.1.0", "derive_more", "itertools 0.14.0", diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index e5f58523d03..34cffbec2b8 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -20,6 +20,15 @@ pub const DEFAULT_RESERVED_CPU_CORES: usize = 1; /// Default maximum concurrency for prewarm task. pub const DEFAULT_PREWARM_MAX_CONCURRENCY: usize = 16; +/// Maximum number of storage proof workers +const MAX_STORAGE_PROOF_WORKERS: usize = 12; + +/// Minimum number of storage proof workers +const MIN_STORAGE_PROOF_WORKERS: usize = 2; + +/// Default ratio of storage proof workers to max_proof_task_concurrency +const DEFAULT_STORAGE_PROOF_WORKER_RATIO: f32 = 0.5; + const DEFAULT_BLOCK_BUFFER_LIMIT: u32 = 256; const DEFAULT_MAX_INVALID_HEADER_CACHE_LENGTH: u32 = 256; const DEFAULT_MAX_EXECUTE_BLOCK_BATCH_SIZE: usize = 4; @@ -109,6 +118,9 @@ pub struct TreeConfig { prewarm_max_concurrency: usize, /// Whether to unwind canonical header to ancestor during forkchoice updates. allow_unwind_canonical_header: bool, + /// Number of dedicated storage proof workers. + /// If None, defaults to half of max_proof_task_concurrency. + storage_proof_workers: Option, } impl Default for TreeConfig { @@ -135,6 +147,7 @@ impl Default for TreeConfig { always_process_payload_attributes_on_canonical_head: false, prewarm_max_concurrency: DEFAULT_PREWARM_MAX_CONCURRENCY, allow_unwind_canonical_header: false, + storage_proof_workers: None, } } } @@ -164,6 +177,7 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head: bool, prewarm_max_concurrency: usize, allow_unwind_canonical_header: bool, + storage_proof_workers: Option, ) -> Self { Self { persistence_threshold, @@ -187,6 +201,7 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head, prewarm_max_concurrency, allow_unwind_canonical_header, + storage_proof_workers, } } @@ -452,4 +467,31 @@ impl TreeConfig { pub const fn prewarm_max_concurrency(&self) -> usize { self.prewarm_max_concurrency } + + /// Get the number of storage proof workers. + /// + /// Defaults to half of max_proof_task_concurrency, clamped to valid range. + pub fn storage_proof_workers(&self) -> usize { + self.storage_proof_workers.unwrap_or_else(|| { + let derived = (self.max_proof_task_concurrency as f32 * + DEFAULT_STORAGE_PROOF_WORKER_RATIO) as usize; + derived.clamp(MIN_STORAGE_PROOF_WORKERS, MAX_STORAGE_PROOF_WORKERS) + }) + } + + /// Set the number of storage proof workers explicitly. + /// + /// Value is clamped to [MIN_STORAGE_PROOF_WORKERS, MAX_STORAGE_PROOF_WORKERS]. + pub const fn with_storage_proof_workers(mut self, workers: usize) -> Self { + // Note: Can't use clamp in const fn, so we'll do manual clamping + let clamped = if workers < MIN_STORAGE_PROOF_WORKERS { + MIN_STORAGE_PROOF_WORKERS + } else if workers > MAX_STORAGE_PROOF_WORKERS { + MAX_STORAGE_PROOF_WORKERS + } else { + workers + }; + self.storage_proof_workers = Some(clamped); + self + } } diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 8d9bd1ba2e0..8aa1f0b4bfe 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -196,11 +196,13 @@ where state_root_config.prefix_sets.clone(), ); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; + let storage_worker_count = config.storage_proof_workers(); let proof_task = ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), task_ctx, max_proof_task_concurrency, + storage_worker_count, ); // We set it to half of the proof task concurrency, because often for each multiproof we diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 6c7f5de40a3..e435e914622 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1236,6 +1236,7 @@ mod tests { config.consistent_view.clone(), task_ctx, 1, + 1, // storage_worker_count: 1 for tests ); let channel = channel(); diff --git a/crates/trie/parallel/Cargo.toml b/crates/trie/parallel/Cargo.toml index c9f625a1500..b4463d9ede3 100644 --- a/crates/trie/parallel/Cargo.toml +++ b/crates/trie/parallel/Cargo.toml @@ -36,6 +36,7 @@ derive_more.workspace = true rayon.workspace = true itertools.workspace = true tokio = { workspace = true, features = ["rt-multi-thread"] } +crossbeam-channel.workspace = true # `metrics` feature reth-metrics = { workspace = true, optional = true } diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index d6e1b57ed9b..4c9a3a57cf8 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -448,7 +448,7 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1); + ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1); let proof_task_handle = proof_task.handle(); // keep the join handle around to make sure it does not return any errors diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 9bb96d4b19e..4f4f0e746df 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -10,6 +10,7 @@ use crate::root::ParallelStateRootError; use alloy_primitives::{map::B256Set, B256}; +use crossbeam_channel::{bounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use reth_db_api::transaction::DbTx; use reth_execution_errors::SparseTrieError; use reth_provider::{ @@ -48,68 +49,403 @@ use crate::proof_task_metrics::ProofTaskMetrics; type StorageProofResult = Result; type TrieNodeProviderResult = Result, SparseTrieError>; -/// A task that manages sending multiproof requests to a number of tasks that have longer-running -/// database transactions +/// Internal message for storage proof workers. +/// +/// This is NOT exposed publicly. External callers still use `ProofTaskKind::StorageProof` +/// which is routed through the manager's std::mpsc channel. +#[derive(Debug)] +struct StorageProofJob { + /// Storage proof input parameters + input: StorageProofInput, + /// Channel to send result back to original caller + /// + /// This is the same std::mpsc::Sender that the external caller provided in + /// ProofTaskKind::StorageProof(input, sender). + result_sender: Sender, +} + +/// Internal message for on-demand task execution. +/// +/// These tasks are executed with lazily-created transactions that are +/// returned to the pool after use (same as current behavior). +#[derive(Debug)] +enum OnDemandTask { + /// Fetch a blinded account node by path + BlindedAccountNode(Nibbles, Sender), + /// Fetch a blinded storage node by account and path + BlindedStorageNode(B256, Nibbles, Sender), +} + +/// A task that manages sending proof requests to worker pools and on-demand tasks. +/// +/// # Architecture (PR1: Storage Workers Only) +/// +/// This manager maintains two execution paths: +/// +/// 1. **Storage Worker Pool** (NEW): +/// - Pre-spawned workers with dedicated long-lived transactions +/// - Tasks queued via crossbeam bounded channel +/// - Workers continuously process without transaction return overhead +/// +/// 2. **On-Demand Execution** (EXISTING): +/// - Lazy transaction creation for blinded node fetches +/// - Transactions returned to pool after use (original behavior) +/// - Same message-passing mechanism as before +/// +/// # External API +/// +/// The external API via `ProofTaskManagerHandle` is COMPLETELY UNCHANGED: +/// - `queue_task(ProofTaskKind)` signature identical +/// - Same std::mpsc message passing +/// - Same return types and error handling +/// +/// All changes are internal routing optimizations. #[derive(Debug)] pub struct ProofTaskManager { - /// Max number of database transactions to create - max_concurrency: usize, - /// Number of database transactions created - total_transactions: usize, - /// Consistent view provider used for creating transactions on-demand + // ==================== STORAGE WORKER POOL (NEW) ==================== + /// Sender for storage proof tasks to worker pool. + /// + /// Queue capacity = storage_worker_count * 2 (for 2x buffering) + storage_work_tx: CrossbeamSender, + + /// Number of storage workers successfully spawned. + /// + /// May be less than requested if transaction creation fails. + storage_worker_count: usize, + + // ==================== ON-DEMAND TRANSACTION POOL (REFACTORED) ==================== + /// Maximum number of on-demand transactions for blinded node fetches. + /// + /// Calculated as: max_concurrency - storage_worker_count + max_on_demand_txs: usize, + + /// Currently available on-demand transactions (reused after return). + /// + /// Same lifecycle as before PR1. + on_demand_txs: Vec>>, + + /// Total on-demand transactions created (for ID assignment). + on_demand_tx_count: usize, + + /// Queue of pending on-demand tasks waiting for available transaction. + /// + /// Replaces the old `pending_tasks` VecDeque which held all task types. + /// TODO: Change to VecDeque in Phase 8 when implementing proper task routing + pending_on_demand: VecDeque, + + // ==================== SHARED RESOURCES ==================== + /// Consistent view provider used for creating transactions on-demand. view: ConsistentDbView, - /// Proof task context shared across all proof tasks + + /// Proof task context shared across all proof tasks. task_ctx: ProofTaskCtx, - /// Proof tasks pending execution - pending_tasks: VecDeque, - /// The underlying handle from which to spawn proof tasks + + /// The underlying handle from which to spawn proof tasks. executor: Handle, - /// The proof task transactions, containing owned cursor factories that are reused for proof - /// calculation. - proof_task_txs: Vec>>, - /// A receiver for new proof tasks. + + /// A receiver for new proof task messages from external callers. + /// + /// This is the std::mpsc channel connected to ProofTaskManagerHandle. + /// UNCHANGED - maintains interface compatibility. proof_task_rx: Receiver>>, - /// A sender for sending back transactions. + + /// A sender for internal messaging (transaction returns). + /// + /// Used by on-demand tasks to return transactions to pool. tx_sender: Sender>>, + /// The number of active handles. /// /// Incremented in [`ProofTaskManagerHandle::new`] and decremented in /// [`ProofTaskManagerHandle::drop`]. active_handles: Arc, - /// Metrics tracking blinded node fetches. + + /// Metrics tracking proof task operations. #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, } -impl ProofTaskManager { - /// Creates a new [`ProofTaskManager`] with the given max concurrency, creating that number of - /// cursor factories. +/// Worker loop for storage proof computation. +/// +/// # Lifecycle +/// +/// Each worker: +/// 1. Receives `StorageProofJob` from crossbeam bounded channel +/// 2. Computes proof using its dedicated long-lived transaction +/// 3. Sends result directly to original caller via std::mpsc +/// 4. Repeats until channel closes (graceful shutdown) +/// +/// # Transaction Reuse +/// +/// The key optimization: the worker reuses the same `proof_tx` across ALL proofs, +/// avoiding the overhead of: +/// - Creating new database transactions +/// - Setting up cursor factories +/// - Returning transactions to a pool +/// +/// # Panic Safety +/// +/// If this function panics, the worker thread terminates but: +/// - Other workers continue operating +/// - The manager detects disconnection when trying to send +/// - System degrades gracefully rather than failing completely +/// +/// # Shutdown +/// +/// Worker shuts down when: +/// - Crossbeam channel closes (all senders dropped) +/// - `ProofTaskManager::run()` drops `storage_work_tx` on terminate +fn storage_worker_loop( + proof_tx: ProofTaskTx, + work_rx: CrossbeamReceiver, + worker_id: usize, +) where + Tx: DbTx, +{ + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Storage proof worker started" + ); + + let mut proofs_processed = 0u64; + let start_time = Instant::now(); + + // Main worker loop: process jobs until channel closes + while let Ok(StorageProofJob { input, result_sender }) = work_rx.recv() { + let proof_start = Instant::now(); + + trace!( + target: "trie::proof_task", + worker_id, + hashed_address = ?input.hashed_address, + prefix_set_len = input.prefix_set.len(), + target_slots = input.target_slots.len(), + "Processing storage proof" + ); + + // ==================== CORE COMPUTATION ==================== + // Compute storage proof using reused transaction + // This is the key difference from on-demand execution: + // - No transaction creation overhead + // - No transaction return message + // - Cursor factories reused across proofs + let result = proof_tx.compute_storage_proof(&input); + + let proof_elapsed = proof_start.elapsed(); + proofs_processed += 1; + + // ==================== RESULT DELIVERY ==================== + // Send result directly to original caller's std::mpsc::Receiver + // If receiver is dropped (caller cancelled), log and continue + if let Err(_) = result_sender.send(result) { + tracing::debug!( + target: "trie::proof_task", + worker_id, + hashed_address = ?input.hashed_address, + proofs_processed, + "Storage proof receiver dropped, discarding result" + ); + } + + trace!( + target: "trie::proof_task", + worker_id, + hashed_address = ?input.hashed_address, + proof_time_us = proof_elapsed.as_micros(), + total_processed = proofs_processed, + "Storage proof completed" + ); + } + + // Channel closed - graceful shutdown + let total_elapsed = start_time.elapsed(); + + tracing::info!( + target: "trie::proof_task", + worker_id, + proofs_processed, + uptime_secs = total_elapsed.as_secs(), + avg_proof_time_ms = if proofs_processed > 0 { + total_elapsed.as_millis() / proofs_processed as u128 + } else { + 0 + }, + "Storage proof worker shutting down" + ); +} + +impl ProofTaskManager +where + Factory: DatabaseProviderFactory, +{ + /// Creates a new [`ProofTaskManager`] with the given configuration. + /// + /// # Arguments + /// + /// * `executor` - Tokio runtime handle for spawning workers and tasks + /// * `view` - Consistent database view for creating read-only transactions + /// * `task_ctx` - Shared context (trie updates, hashed state, prefix sets) + /// * `max_concurrency` - Total transaction budget across all execution paths + /// * `storage_worker_count` - Number of storage proof workers to pre-spawn + /// + /// # Transaction Budget Allocation + /// + /// The total `max_concurrency` is split between two pools: + /// + /// 1. **Storage Workers**: `storage_worker_count` transactions (pre-allocated) + /// 2. **On-Demand Pool**: `max_concurrency - storage_worker_count` (lazy) + /// + /// Example: + /// ```text + /// max_concurrency = 8, storage_worker_count = 4 + /// → 4 storage workers (pre-spawned) + /// → 4 on-demand transactions (created lazily for blinded nodes) + /// Total: 8 transactions max (same capacity as before) + /// ``` /// - /// Returns an error if the consistent view provider fails to create a read-only transaction. + /// # Worker Spawn Resilience + /// + /// If some workers fail to spawn (e.g., transaction creation error): + /// - Failed workers are logged and skipped + /// - On-demand pool is adjusted: `max_concurrency - actual_spawned_workers` + /// - System continues with fewer workers rather than failing entirely + /// + /// # Panics + /// + /// Does not panic. All errors are logged and handled gracefully. pub fn new( executor: Handle, view: ConsistentDbView, task_ctx: ProofTaskCtx, max_concurrency: usize, + storage_worker_count: usize, ) -> Self { + // Create message channel for external callers (UNCHANGED) let (tx_sender, proof_task_rx) = channel(); - Self { + + // ==================== STORAGE WORKER POOL SETUP ==================== + + // Queue capacity: 2x buffering to reduce contention + // If workers = 4, queue holds 8 tasks maximum + let queue_capacity = storage_worker_count.saturating_mul(2).max(1); + let (storage_work_tx, storage_work_rx) = bounded::(queue_capacity); + + tracing::info!( + target: "trie::proof_task", + storage_worker_count, + queue_capacity, max_concurrency, - total_transactions: 0, + "Initializing storage proof worker pool" + ); + + // Spawn storage workers - each gets its own long-lived transaction + let mut spawned_workers = 0; + for worker_id in 0..storage_worker_count { + // Try to create transaction for this worker + match view.provider_ro() { + Ok(provider_ro) => { + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx = storage_work_rx.clone(); + + // Spawn worker on tokio blocking pool + executor.spawn_blocking(move || { + storage_worker_loop(proof_task_tx, work_rx, worker_id) + }); + + spawned_workers += 1; + + tracing::debug!( + target: "trie::proof_task", + worker_id, + spawned_workers, + "Storage worker spawned successfully" + ); + } + Err(err) => { + // Non-fatal: log and continue with fewer workers + tracing::warn!( + target: "trie::proof_task", + worker_id, + ?err, + requested = storage_worker_count, + spawned_workers, + "Failed to create transaction for storage worker, continuing with fewer workers" + ); + } + } + } + + // Verify we spawned at least some workers + if spawned_workers == 0 { + tracing::error!( + target: "trie::proof_task", + requested = storage_worker_count, + "Failed to spawn any storage workers - all will use on-demand pool" + ); + } else if spawned_workers < storage_worker_count { + tracing::warn!( + target: "trie::proof_task", + requested = storage_worker_count, + spawned = spawned_workers, + "Spawned fewer storage workers than requested" + ); + } else { + tracing::info!( + target: "trie::proof_task", + spawned_workers, + queue_capacity, + "Storage worker pool initialized successfully" + ); + } + + // ==================== ON-DEMAND POOL SETUP ==================== + + // Calculate on-demand budget: remaining capacity after storage workers + // Ensure at least 1 on-demand transaction even if storage workers consume all budget + let max_on_demand_txs = max_concurrency.saturating_sub(spawned_workers).max(1); + + tracing::debug!( + target: "trie::proof_task", + max_on_demand_txs, + storage_workers = spawned_workers, + total_capacity = max_concurrency, + "Configured on-demand transaction pool for blinded nodes" + ); + + // ==================== CONSTRUCT MANAGER ==================== + + Self { + // Storage worker pool + storage_work_tx, + storage_worker_count: spawned_workers, + + // On-demand pool + max_on_demand_txs, + on_demand_txs: Vec::with_capacity(max_on_demand_txs), + on_demand_tx_count: 0, + pending_on_demand: VecDeque::new(), + + // Shared resources view, task_ctx, - pending_tasks: VecDeque::new(), executor, - proof_task_txs: Vec::new(), proof_task_rx, tx_sender, active_handles: Arc::new(AtomicUsize::new(0)), + #[cfg(feature = "metrics")] metrics: ProofTaskMetrics::default(), } } - /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. + /// Returns a handle for sending new proof tasks to the manager. + /// + /// # Interface Compatibility + /// + /// This method is UNCHANGED from the original implementation. The returned + /// `ProofTaskManagerHandle` has the exact same public API as before PR1. pub fn handle(&self) -> ProofTaskManagerHandle> { ProofTaskManagerHandle::new(self.tx_sender.clone(), self.active_handles.clone()) } @@ -121,22 +457,22 @@ where { /// Inserts the task into the pending tasks queue. pub fn queue_proof_task(&mut self, task: ProofTaskKind) { - self.pending_tasks.push_back(task); + self.pending_on_demand.push_back(task); } /// Gets either the next available transaction, or creates a new one if all are in use and the /// total number of transactions created is less than the max concurrency. pub fn get_or_create_tx(&mut self) -> ProviderResult>>> { - if let Some(proof_task_tx) = self.proof_task_txs.pop() { + if let Some(proof_task_tx) = self.on_demand_txs.pop() { return Ok(Some(proof_task_tx)); } // if we can create a new tx within our concurrency limits, create one on-demand - if self.total_transactions < self.max_concurrency { + if self.on_demand_tx_count < self.max_on_demand_txs { let provider_ro = self.view.provider_ro()?; let tx = provider_ro.into_tx(); - self.total_transactions += 1; - return Ok(Some(ProofTaskTx::new(tx, self.task_ctx.clone(), self.total_transactions))); + self.on_demand_tx_count += 1; + return Ok(Some(ProofTaskTx::new(tx, self.task_ctx.clone(), self.on_demand_tx_count))); } Ok(None) @@ -148,11 +484,11 @@ where /// This will return an error if a transaction must be created on-demand and the consistent view /// provider fails. pub fn try_spawn_next(&mut self) -> ProviderResult<()> { - let Some(task) = self.pending_tasks.pop_front() else { return Ok(()) }; + let Some(task) = self.pending_on_demand.pop_front() else { return Ok(()) }; let Some(proof_task_tx) = self.get_or_create_tx()? else { // if there are no txs available, requeue the proof task - self.pending_tasks.push_front(task); + self.pending_on_demand.push_front(task); return Ok(()) }; @@ -173,42 +509,121 @@ where } /// Loops, managing the proof tasks, and sending new tasks to the executor. + /// + /// # Task Routing + /// + /// - **Storage Proofs**: Routed to pre-spawned worker pool via bounded channel + /// - If channel is full, falls back to on-demand spawn + /// - **Blinded Nodes**: Queued for on-demand execution (original behavior) + /// + /// # Worker Pool Lifecycle + /// + /// On termination, `storage_work_tx` is dropped, closing the channel and signaling + /// all workers to shut down gracefully. pub fn run(mut self) -> ProviderResult<()> { loop { match self.proof_task_rx.recv() { - Ok(message) => match message { - ProofTaskMessage::QueueTask(task) => { - // Track metrics for blinded node requests - #[cfg(feature = "metrics")] - match &task { - ProofTaskKind::BlindedAccountNode(_, _) => { - self.metrics.account_nodes += 1; + Ok(message) => { + match message { + ProofTaskMessage::QueueTask(task) => { + match task { + // ==================== STORAGE PROOF ROUTING ==================== + ProofTaskKind::StorageProof(input, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_proofs += 1; + } + + // Try to send to worker pool first + match self + .storage_work_tx + .try_send(StorageProofJob { input, result_sender: sender }) + { + Ok(_) => { + // Successfully queued to worker pool + tracing::trace!( + target: "trie::proof_task", + "Storage proof dispatched to worker pool" + ); + } + Err(crossbeam_channel::TrySendError::Full(job)) => { + // Channel full - fall back to on-demand spawn + tracing::debug!( + target: "trie::proof_task", + "Worker pool queue full, spawning on-demand" + ); + + #[cfg(feature = "metrics")] + { + self.metrics.on_demand_fallback += 1; + } + + // Queue for on-demand execution + self.pending_on_demand.push_back( + ProofTaskKind::StorageProof( + job.input, + job.result_sender, + ), + ); + } + Err(crossbeam_channel::TrySendError::Disconnected(_)) => { + // Workers shut down - this should not happen + tracing::error!( + target: "trie::proof_task", + "Worker pool disconnected unexpectedly" + ); + return Err(reth_storage_errors::provider::ProviderError::Database( + reth_db_api::DatabaseError::Other("Worker pool disconnected".into()) + )) + } + } + } + + // ==================== BLINDED NODE ROUTING ==================== + ProofTaskKind::BlindedAccountNode(_, _) => { + #[cfg(feature = "metrics")] + { + self.metrics.account_nodes += 1; + } + self.queue_proof_task(task); + } + ProofTaskKind::BlindedStorageNode(_, _, _) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_nodes += 1; + } + self.queue_proof_task(task); + } } - ProofTaskKind::BlindedStorageNode(_, _, _) => { - self.metrics.storage_nodes += 1; - } - _ => {} } - // queue the task - self.queue_proof_task(task) - } - ProofTaskMessage::Transaction(tx) => { - // return the transaction to the pool - self.proof_task_txs.push(tx); - } - ProofTaskMessage::Terminate => { - // Record metrics before terminating - #[cfg(feature = "metrics")] - self.metrics.record(); - return Ok(()) + ProofTaskMessage::Transaction(tx) => { + // Return transaction to on-demand pool + self.on_demand_txs.push(tx); + } + ProofTaskMessage::Terminate => { + // Drop storage_work_tx to signal workers to shut down + drop(self.storage_work_tx); + + tracing::info!( + target: "trie::proof_task", + storage_worker_count = self.storage_worker_count, + "Shutting down proof task manager, signaling workers to terminate" + ); + + // Record metrics before terminating + #[cfg(feature = "metrics")] + self.metrics.record(); + + return Ok(()) + } } - }, + } // All senders are disconnected, so we can terminate // However this should never happen, as this struct stores a sender Err(_) => return Ok(()), }; - // try spawning the next task + // Try spawning on-demand tasks only (storage proofs handled by worker pool) self.try_spawn_next()?; } } @@ -260,7 +675,103 @@ where (trie_cursor_factory, hashed_cursor_factory) } + /// Compute storage proof without consuming self (for worker pool reuse). + /// + /// # Purpose + /// + /// This method enables transaction reuse in the storage worker pool. Unlike the + /// original `storage_proof(self, ...)` which consumes self and returns the + /// transaction to a pool, this method: + /// + /// 1. Borrows self immutably + /// 2. Computes the proof using the owned transaction + /// 3. Returns only the result (transaction remains owned) + /// 4. Can be called repeatedly on the same ProofTaskTx instance + /// + /// # Usage + /// + /// This is called exclusively by storage workers in the worker pool. On-demand + /// execution still uses the original `storage_proof(self, ...)` method which + /// consumes self and returns the transaction. + /// + /// # Performance + /// + /// By reusing the same transaction and cursor factories across multiple proofs: + /// - Eliminates per-proof transaction creation overhead + /// - Avoids message passing to return transactions + /// - Reduces memory allocations for cursor factories + fn compute_storage_proof(&self, input: &StorageProofInput) -> StorageProofResult { + // ==================== SETUP ==================== + + // Create cursor factories (same as original implementation) + let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); + + // Get or create added/removed keys context + let multi_added_removed_keys = input + .multi_added_removed_keys + .clone() + .unwrap_or_else(|| Arc::new(MultiAddedRemovedKeys::new())); + let added_removed_keys = multi_added_removed_keys.get_storage(&input.hashed_address); + + let span = tracing::trace_span!( + target: "trie::proof_task", + "Storage proof calculation", + hashed_address = ?input.hashed_address, + // Worker ID embedded in ProofTaskTx for trace correlation + worker_id = self.id, + ); + let _guard = span.enter(); + + let target_slots_len = input.target_slots.len(); + let proof_start = Instant::now(); + + // Compute raw storage multiproof (identical to original) + let raw_proof_result = StorageProof::new_hashed( + trie_cursor_factory, + hashed_cursor_factory, + input.hashed_address, + ) + .with_prefix_set_mut(PrefixSetMut::from(input.prefix_set.iter().copied())) + .with_branch_node_masks(input.with_branch_node_masks) + .with_added_removed_keys(added_removed_keys) + .storage_multiproof(input.target_slots.clone()) + .map_err(|e| ParallelStateRootError::Other(e.to_string())); + + // Decode proof into DecodedStorageMultiProof + let decoded_result = raw_proof_result.and_then(|raw_proof| { + raw_proof.try_into().map_err(|e: alloy_rlp::Error| { + ParallelStateRootError::Other(format!( + "Failed to decode storage proof for {}: {}", + input.hashed_address, e + )) + }) + }); + + trace!( + target: "trie::proof_task", + hashed_address = ?input.hashed_address, + prefix_set_len = input.prefix_set.len(), + target_slots = target_slots_len, + proof_time_us = proof_start.elapsed().as_micros(), + worker_id = self.id, + "Completed storage proof calculation" + ); + + decoded_result + + // NOTE: self is NOT consumed - transaction remains owned by worker + // No ProofTaskMessage::Transaction sent + } + /// Calculates a storage proof for the given hashed address, and desired prefix set. + /// + /// **ON-DEMAND VARIANT** - Consumes self, returns transaction to pool. + /// + /// This method is NO LONGER CALLED for storage proofs from the worker pool, + /// but is kept for: + /// 1. Backward compatibility with any direct callers + /// 2. Future use cases that need one-off storage proofs + /// 3. Tests that rely on the transaction return mechanism fn storage_proof( self, input: StorageProofInput, diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index cdb59d078d8..97ec8e6f113 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -9,6 +9,10 @@ pub struct ProofTaskMetrics { pub account_nodes: usize, /// Count of blinded storage node requests. pub storage_nodes: usize, + /// Count of storage proof requests routed to worker pool. + pub storage_proofs: usize, + /// Count of times worker pool was full and fell back to on-demand execution. + pub on_demand_fallback: usize, } impl ProofTaskMetrics { @@ -16,6 +20,7 @@ impl ProofTaskMetrics { pub fn record(&self) { self.task_metrics.record_account_nodes(self.account_nodes); self.task_metrics.record_storage_nodes(self.storage_nodes); + self.task_metrics.record_storage_proofs(self.storage_proofs); } } @@ -27,6 +32,8 @@ pub struct ProofTaskTrieMetrics { blinded_account_nodes: Histogram, /// A histogram for the number of blinded storage nodes fetched. blinded_storage_nodes: Histogram, + /// A histogram for the number of storage proofs computed via worker pool. + storage_proofs: Histogram, } impl ProofTaskTrieMetrics { @@ -39,4 +46,9 @@ impl ProofTaskTrieMetrics { pub fn record_storage_nodes(&self, count: usize) { self.blinded_storage_nodes.record(count as f64); } + + /// Record storage proofs computed via worker pool. + pub fn record_storage_proofs(&self, count: usize) { + self.storage_proofs.record(count as f64); + } } From fbeec5063d0a031e1da1c8cedbdae27ec58cceab Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 06:42:13 +0000 Subject: [PATCH 002/144] fmt, clippy --- crates/engine/primitives/src/config.rs | 8 ++++---- crates/trie/parallel/src/proof_task.rs | 25 +++++++++++++------------ 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 34cffbec2b8..491583699b7 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -26,7 +26,7 @@ const MAX_STORAGE_PROOF_WORKERS: usize = 12; /// Minimum number of storage proof workers const MIN_STORAGE_PROOF_WORKERS: usize = 2; -/// Default ratio of storage proof workers to max_proof_task_concurrency +/// Default ratio of storage proof workers to `max_proof_task_concurrency` const DEFAULT_STORAGE_PROOF_WORKER_RATIO: f32 = 0.5; const DEFAULT_BLOCK_BUFFER_LIMIT: u32 = 256; @@ -119,7 +119,7 @@ pub struct TreeConfig { /// Whether to unwind canonical header to ancestor during forkchoice updates. allow_unwind_canonical_header: bool, /// Number of dedicated storage proof workers. - /// If None, defaults to half of max_proof_task_concurrency. + /// If None, defaults to half of `max_proof_task_concurrency`. storage_proof_workers: Option, } @@ -470,7 +470,7 @@ impl TreeConfig { /// Get the number of storage proof workers. /// - /// Defaults to half of max_proof_task_concurrency, clamped to valid range. + /// Defaults to half of `max_proof_task_concurrency`, clamped to valid range. pub fn storage_proof_workers(&self) -> usize { self.storage_proof_workers.unwrap_or_else(|| { let derived = (self.max_proof_task_concurrency as f32 * @@ -481,7 +481,7 @@ impl TreeConfig { /// Set the number of storage proof workers explicitly. /// - /// Value is clamped to [MIN_STORAGE_PROOF_WORKERS, MAX_STORAGE_PROOF_WORKERS]. + /// Value is clamped to [`MIN_STORAGE_PROOF_WORKERS`, `MAX_STORAGE_PROOF_WORKERS`]. pub const fn with_storage_proof_workers(mut self, workers: usize) -> Self { // Note: Can't use clamp in const fn, so we'll do manual clamping let clamped = if workers < MIN_STORAGE_PROOF_WORKERS { diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 4f4f0e746df..82584916ceb 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -52,15 +52,15 @@ type TrieNodeProviderResult = Result, SparseTrieError>; /// Internal message for storage proof workers. /// /// This is NOT exposed publicly. External callers still use `ProofTaskKind::StorageProof` -/// which is routed through the manager's std::mpsc channel. +/// which is routed through the manager's `std::mpsc` channel. #[derive(Debug)] struct StorageProofJob { /// Storage proof input parameters input: StorageProofInput, /// Channel to send result back to original caller /// - /// This is the same std::mpsc::Sender that the external caller provided in - /// ProofTaskKind::StorageProof(input, sender). + /// This is the same `std::mpsc::Sender` that the external caller provided in + /// `ProofTaskKind::StorageProof(input`, sender). result_sender: Sender, } @@ -69,6 +69,7 @@ struct StorageProofJob { /// These tasks are executed with lazily-created transactions that are /// returned to the pool after use (same as current behavior). #[derive(Debug)] +#[allow(dead_code)] enum OnDemandTask { /// Fetch a blinded account node by path BlindedAccountNode(Nibbles, Sender), @@ -96,7 +97,7 @@ enum OnDemandTask { /// /// The external API via `ProofTaskManagerHandle` is COMPLETELY UNCHANGED: /// - `queue_task(ProofTaskKind)` signature identical -/// - Same std::mpsc message passing +/// - Same `std::mpsc` message passing /// - Same return types and error handling /// /// All changes are internal routing optimizations. @@ -105,7 +106,7 @@ pub struct ProofTaskManager { // ==================== STORAGE WORKER POOL (NEW) ==================== /// Sender for storage proof tasks to worker pool. /// - /// Queue capacity = storage_worker_count * 2 (for 2x buffering) + /// Queue capacity = `storage_worker_count` * 2 (for 2x buffering) storage_work_tx: CrossbeamSender, /// Number of storage workers successfully spawned. @@ -116,7 +117,7 @@ pub struct ProofTaskManager { // ==================== ON-DEMAND TRANSACTION POOL (REFACTORED) ==================== /// Maximum number of on-demand transactions for blinded node fetches. /// - /// Calculated as: max_concurrency - storage_worker_count + /// Calculated as: `max_concurrency` - `storage_worker_count` max_on_demand_txs: usize, /// Currently available on-demand transactions (reused after return). @@ -129,8 +130,8 @@ pub struct ProofTaskManager { /// Queue of pending on-demand tasks waiting for available transaction. /// - /// Replaces the old `pending_tasks` VecDeque which held all task types. - /// TODO: Change to VecDeque in Phase 8 when implementing proper task routing + /// Replaces the old `pending_tasks` `VecDeque` which held all task types. + /// TODO: Change to `VecDeque` in Phase 8 when implementing proper task routing pending_on_demand: VecDeque, // ==================== SHARED RESOURCES ==================== @@ -145,7 +146,7 @@ pub struct ProofTaskManager { /// A receiver for new proof task messages from external callers. /// - /// This is the std::mpsc channel connected to ProofTaskManagerHandle. + /// This is the `std::mpsc` channel connected to [`ProofTaskManagerHandle`]. /// UNCHANGED - maintains interface compatibility. proof_task_rx: Receiver>>, @@ -172,7 +173,7 @@ pub struct ProofTaskManager { /// Each worker: /// 1. Receives `StorageProofJob` from crossbeam bounded channel /// 2. Computes proof using its dedicated long-lived transaction -/// 3. Sends result directly to original caller via std::mpsc +/// 3. Sends result directly to original caller via `std::mpsc` /// 4. Repeats until channel closes (graceful shutdown) /// /// # Transaction Reuse @@ -238,7 +239,7 @@ fn storage_worker_loop( // ==================== RESULT DELIVERY ==================== // Send result directly to original caller's std::mpsc::Receiver // If receiver is dropped (caller cancelled), log and continue - if let Err(_) = result_sender.send(result) { + if result_sender.send(result).is_err() { tracing::debug!( target: "trie::proof_task", worker_id, @@ -686,7 +687,7 @@ where /// 1. Borrows self immutably /// 2. Computes the proof using the owned transaction /// 3. Returns only the result (transaction remains owned) - /// 4. Can be called repeatedly on the same ProofTaskTx instance + /// 4. Can be called repeatedly on the same [`ProofTaskTx`] instance /// /// # Usage /// From ab823866fbff3ea82ed0e76c4998c7739ddee9da Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:00:34 +0000 Subject: [PATCH 003/144] add fallback - Enhanced documentation for `StorageProofJob` to clarify its current unused status and potential for future type-safe design. - Updated comments in `ProofTaskManager` regarding the handling of on-demand tasks and the possibility of refactoring to a more type-safe enum. - Improved logging for worker pool disconnection scenarios, emphasizing fallback to on-demand execution. --- crates/trie/parallel/src/proof_task.rs | 37 +++++++++++++++++++------- 1 file changed, 28 insertions(+), 9 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 82584916ceb..1d8019171ab 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -66,7 +66,12 @@ struct StorageProofJob { /// Internal message for on-demand task execution. /// -/// These tasks are executed with lazily-created transactions that are +/// **Note**: Currently unused in favor of [`ProofTaskKind`] for simplicity and backwards +/// compatibility. This enum represents a more type-safe design where on-demand tasks +/// (blinded nodes) are strictly separated from worker pool tasks (storage proofs). +/// Available for future refactoring if stricter type safety is desired. +/// +/// These tasks would be executed with lazily-created transactions that are /// returned to the pool after use (same as current behavior). #[derive(Debug)] #[allow(dead_code)] @@ -131,7 +136,9 @@ pub struct ProofTaskManager { /// Queue of pending on-demand tasks waiting for available transaction. /// /// Replaces the old `pending_tasks` `VecDeque` which held all task types. - /// TODO: Change to `VecDeque` in Phase 8 when implementing proper task routing + /// Currently holds `ProofTaskKind` for both blinded node fetches and storage proof + /// fallbacks (when worker pool is full/unavailable). Could be refactored to use + /// the more type-safe `OnDemandTask` enum if strict separation is desired. pending_on_demand: VecDeque, // ==================== SHARED RESOURCES ==================== @@ -567,15 +574,27 @@ where ), ); } - Err(crossbeam_channel::TrySendError::Disconnected(_)) => { - // Workers shut down - this should not happen - tracing::error!( + Err(crossbeam_channel::TrySendError::Disconnected(job)) => { + // No workers available (likely all spawns failed) - + // fall back to on-demand + tracing::warn!( target: "trie::proof_task", - "Worker pool disconnected unexpectedly" + storage_worker_count = self.storage_worker_count, + "Worker pool disconnected (no workers available), falling back to on-demand" + ); + + #[cfg(feature = "metrics")] + { + self.metrics.on_demand_fallback += 1; + } + + // Queue for on-demand execution instead of failing + self.pending_on_demand.push_back( + ProofTaskKind::StorageProof( + job.input, + job.result_sender, + ), ); - return Err(reth_storage_errors::provider::ProviderError::Database( - reth_db_api::DatabaseError::Other("Worker pool disconnected".into()) - )) } } } From 13891ad1ead85ffc44c1e5ad52b35a475f48cdf9 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:09:51 +0000 Subject: [PATCH 004/144] fix comments --- crates/engine/primitives/src/config.rs | 25 +- crates/trie/parallel/src/proof_task.rs | 385 ++++++++++--------------- 2 files changed, 173 insertions(+), 237 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 491583699b7..10d93676f5e 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -23,9 +23,6 @@ pub const DEFAULT_PREWARM_MAX_CONCURRENCY: usize = 16; /// Maximum number of storage proof workers const MAX_STORAGE_PROOF_WORKERS: usize = 12; -/// Minimum number of storage proof workers -const MIN_STORAGE_PROOF_WORKERS: usize = 2; - /// Default ratio of storage proof workers to `max_proof_task_concurrency` const DEFAULT_STORAGE_PROOF_WORKER_RATIO: f32 = 0.5; @@ -470,24 +467,32 @@ impl TreeConfig { /// Get the number of storage proof workers. /// - /// Defaults to half of `max_proof_task_concurrency`, clamped to valid range. + /// Defaults to half of `max_proof_task_concurrency`, clamped to valid range and leaving at + /// least one slot for on-demand work. pub fn storage_proof_workers(&self) -> usize { + let max_allowed = self.max_proof_task_concurrency.saturating_sub(1) as usize; + if max_allowed == 0 { + return 0; + } + self.storage_proof_workers.unwrap_or_else(|| { let derived = (self.max_proof_task_concurrency as f32 * DEFAULT_STORAGE_PROOF_WORKER_RATIO) as usize; - derived.clamp(MIN_STORAGE_PROOF_WORKERS, MAX_STORAGE_PROOF_WORKERS) + let capped = derived.min(MAX_STORAGE_PROOF_WORKERS); + + capped.clamp(1, max_allowed) }) } /// Set the number of storage proof workers explicitly. /// - /// Value is clamped to [`MIN_STORAGE_PROOF_WORKERS`, `MAX_STORAGE_PROOF_WORKERS`]. + /// Value is clamped to the remaining concurrency budget (leaving one on-demand slot). pub const fn with_storage_proof_workers(mut self, workers: usize) -> Self { - // Note: Can't use clamp in const fn, so we'll do manual clamping - let clamped = if workers < MIN_STORAGE_PROOF_WORKERS { - MIN_STORAGE_PROOF_WORKERS - } else if workers > MAX_STORAGE_PROOF_WORKERS { + let max_allowed = self.max_proof_task_concurrency.saturating_sub(1) as usize; + let clamped = if workers > MAX_STORAGE_PROOF_WORKERS { MAX_STORAGE_PROOF_WORKERS + } else if workers > max_allowed { + max_allowed } else { workers }; diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 1d8019171ab..fcab2787ff1 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -84,31 +84,27 @@ enum OnDemandTask { /// A task that manages sending proof requests to worker pools and on-demand tasks. /// -/// # Architecture (PR1: Storage Workers Only) +/// # Architecture /// /// This manager maintains two execution paths: /// -/// 1. **Storage Worker Pool** (NEW): +/// 1. **Storage Worker Pool**: /// - Pre-spawned workers with dedicated long-lived transactions /// - Tasks queued via crossbeam bounded channel /// - Workers continuously process without transaction return overhead /// -/// 2. **On-Demand Execution** (EXISTING): +/// 2. **On-Demand Execution**: /// - Lazy transaction creation for blinded node fetches -/// - Transactions returned to pool after use (original behavior) -/// - Same message-passing mechanism as before +/// - Transactions returned to pool after use /// /// # External API /// -/// The external API via `ProofTaskManagerHandle` is COMPLETELY UNCHANGED: -/// - `queue_task(ProofTaskKind)` signature identical -/// - Same `std::mpsc` message passing -/// - Same return types and error handling -/// -/// All changes are internal routing optimizations. +/// The external API via `ProofTaskManagerHandle`: +/// - `queue_task(ProofTaskKind)` for submitting tasks +/// - `std::mpsc` message passing +/// - Consistent return types and error handling #[derive(Debug)] pub struct ProofTaskManager { - // ==================== STORAGE WORKER POOL (NEW) ==================== /// Sender for storage proof tasks to worker pool. /// /// Queue capacity = `storage_worker_count` * 2 (for 2x buffering) @@ -119,15 +115,12 @@ pub struct ProofTaskManager { /// May be less than requested if transaction creation fails. storage_worker_count: usize, - // ==================== ON-DEMAND TRANSACTION POOL (REFACTORED) ==================== /// Maximum number of on-demand transactions for blinded node fetches. /// /// Calculated as: `max_concurrency` - `storage_worker_count` max_on_demand_txs: usize, /// Currently available on-demand transactions (reused after return). - /// - /// Same lifecycle as before PR1. on_demand_txs: Vec>>, /// Total on-demand transactions created (for ID assignment). @@ -135,13 +128,10 @@ pub struct ProofTaskManager { /// Queue of pending on-demand tasks waiting for available transaction. /// - /// Replaces the old `pending_tasks` `VecDeque` which held all task types. - /// Currently holds `ProofTaskKind` for both blinded node fetches and storage proof - /// fallbacks (when worker pool is full/unavailable). Could be refactored to use - /// the more type-safe `OnDemandTask` enum if strict separation is desired. + /// Holds `ProofTaskKind` for both blinded node fetches and storage proof + /// fallbacks (when worker pool is full/unavailable). pending_on_demand: VecDeque, - // ==================== SHARED RESOURCES ==================== /// Consistent view provider used for creating transactions on-demand. view: ConsistentDbView, @@ -154,7 +144,6 @@ pub struct ProofTaskManager { /// A receiver for new proof task messages from external callers. /// /// This is the `std::mpsc` channel connected to [`ProofTaskManagerHandle`]. - /// UNCHANGED - maintains interface compatibility. proof_task_rx: Receiver>>, /// A sender for internal messaging (transaction returns). @@ -185,24 +174,17 @@ pub struct ProofTaskManager { /// /// # Transaction Reuse /// -/// The key optimization: the worker reuses the same `proof_tx` across ALL proofs, -/// avoiding the overhead of: -/// - Creating new database transactions -/// - Setting up cursor factories -/// - Returning transactions to a pool +/// Reuses the same transaction across multiple proofs to avoid transaction +/// creation and cursor factory setup overhead. /// /// # Panic Safety /// -/// If this function panics, the worker thread terminates but: -/// - Other workers continue operating -/// - The manager detects disconnection when trying to send -/// - System degrades gracefully rather than failing completely +/// If this function panics, the worker thread terminates but other workers +/// continue operating and the system degrades gracefully. /// /// # Shutdown /// -/// Worker shuts down when: -/// - Crossbeam channel closes (all senders dropped) -/// - `ProofTaskManager::run()` drops `storage_work_tx` on terminate +/// Worker shuts down when the crossbeam channel closes (all senders dropped). fn storage_worker_loop( proof_tx: ProofTaskTx, work_rx: CrossbeamReceiver, @@ -219,38 +201,31 @@ fn storage_worker_loop( let mut proofs_processed = 0u64; let start_time = Instant::now(); - // Main worker loop: process jobs until channel closes while let Ok(StorageProofJob { input, result_sender }) = work_rx.recv() { - let proof_start = Instant::now(); + let hashed_address = input.hashed_address; + let prefix_set_len = input.prefix_set.len(); + let target_slots_len = input.target_slots.len(); trace!( target: "trie::proof_task", worker_id, - hashed_address = ?input.hashed_address, - prefix_set_len = input.prefix_set.len(), - target_slots = input.target_slots.len(), + hashed_address = ?hashed_address, + prefix_set_len, + target_slots = target_slots_len, "Processing storage proof" ); - // ==================== CORE COMPUTATION ==================== - // Compute storage proof using reused transaction - // This is the key difference from on-demand execution: - // - No transaction creation overhead - // - No transaction return message - // - Cursor factories reused across proofs - let result = proof_tx.compute_storage_proof(&input); + let proof_start = Instant::now(); + let result = proof_tx.compute_storage_proof(input); let proof_elapsed = proof_start.elapsed(); proofs_processed += 1; - // ==================== RESULT DELIVERY ==================== - // Send result directly to original caller's std::mpsc::Receiver - // If receiver is dropped (caller cancelled), log and continue if result_sender.send(result).is_err() { tracing::debug!( target: "trie::proof_task", worker_id, - hashed_address = ?input.hashed_address, + hashed_address = ?hashed_address, proofs_processed, "Storage proof receiver dropped, discarding result" ); @@ -259,8 +234,10 @@ fn storage_worker_loop( trace!( target: "trie::proof_task", worker_id, - hashed_address = ?input.hashed_address, + hashed_address = ?hashed_address, proof_time_us = proof_elapsed.as_micros(), + prefix_set_len, + target_slots = target_slots_len, total_processed = proofs_processed, "Storage proof completed" ); @@ -299,25 +276,20 @@ where /// /// # Transaction Budget Allocation /// - /// The total `max_concurrency` is split between two pools: + /// The total `max_concurrency` is split between storage workers (pre-allocated) + /// and the on-demand pool (lazy). We always reserve at least one slot for the + /// on-demand path, so the number of workers actually spawned is capped at + /// `max_concurrency - 1`. /// - /// 1. **Storage Workers**: `storage_worker_count` transactions (pre-allocated) - /// 2. **On-Demand Pool**: `max_concurrency - storage_worker_count` (lazy) - /// - /// Example: - /// ```text - /// max_concurrency = 8, storage_worker_count = 4 - /// → 4 storage workers (pre-spawned) - /// → 4 on-demand transactions (created lazily for blinded nodes) - /// Total: 8 transactions max (same capacity as before) - /// ``` + /// For example, if `max_concurrency = 8` and `storage_worker_count = 8`, then + /// 8 workers are requested but only 7 can be accommodated while leaving one + /// on-demand slot, so 7 workers are spawned and the remaining slot is reserved + /// for on-demand transactions (e.g. blinded nodes). /// /// # Worker Spawn Resilience /// - /// If some workers fail to spawn (e.g., transaction creation error): - /// - Failed workers are logged and skipped - /// - On-demand pool is adjusted: `max_concurrency - actual_spawned_workers` - /// - System continues with fewer workers rather than failing entirely + /// If some workers fail to spawn, the on-demand pool is adjusted accordingly + /// and the system continues with fewer workers. /// /// # Panics /// @@ -329,35 +301,40 @@ where max_concurrency: usize, storage_worker_count: usize, ) -> Self { - // Create message channel for external callers (UNCHANGED) let (tx_sender, proof_task_rx) = channel(); - // ==================== STORAGE WORKER POOL SETUP ==================== + let worker_budget = max_concurrency.saturating_sub(1); + let planned_workers = storage_worker_count.min(worker_budget); + + if planned_workers < storage_worker_count { + tracing::debug!( + target: "trie::proof_task", + requested = storage_worker_count, + capped = planned_workers, + max_concurrency, + "Adjusted storage worker count to fit concurrency budget" + ); + } - // Queue capacity: 2x buffering to reduce contention - // If workers = 4, queue holds 8 tasks maximum - let queue_capacity = storage_worker_count.saturating_mul(2).max(1); + let queue_capacity = planned_workers.saturating_mul(2).max(1); let (storage_work_tx, storage_work_rx) = bounded::(queue_capacity); tracing::info!( target: "trie::proof_task", - storage_worker_count, + storage_worker_count = planned_workers, queue_capacity, max_concurrency, "Initializing storage proof worker pool" ); - // Spawn storage workers - each gets its own long-lived transaction let mut spawned_workers = 0; - for worker_id in 0..storage_worker_count { - // Try to create transaction for this worker + for worker_id in 0..planned_workers { match view.provider_ro() { Ok(provider_ro) => { let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); let work_rx = storage_work_rx.clone(); - // Spawn worker on tokio blocking pool executor.spawn_blocking(move || { storage_worker_loop(proof_task_tx, work_rx, worker_id) }); @@ -372,12 +349,11 @@ where ); } Err(err) => { - // Non-fatal: log and continue with fewer workers tracing::warn!( target: "trie::proof_task", worker_id, ?err, - requested = storage_worker_count, + requested = planned_workers, spawned_workers, "Failed to create transaction for storage worker, continuing with fewer workers" ); @@ -385,17 +361,16 @@ where } } - // Verify we spawned at least some workers if spawned_workers == 0 { tracing::error!( target: "trie::proof_task", - requested = storage_worker_count, - "Failed to spawn any storage workers - all will use on-demand pool" + requested = planned_workers, + "Failed to spawn any storage workers - all work will execute on-demand" ); - } else if spawned_workers < storage_worker_count { + } else if spawned_workers < planned_workers { tracing::warn!( target: "trie::proof_task", - requested = storage_worker_count, + requested = planned_workers, spawned = spawned_workers, "Spawned fewer storage workers than requested" ); @@ -408,11 +383,8 @@ where ); } - // ==================== ON-DEMAND POOL SETUP ==================== - - // Calculate on-demand budget: remaining capacity after storage workers - // Ensure at least 1 on-demand transaction even if storage workers consume all budget - let max_on_demand_txs = max_concurrency.saturating_sub(spawned_workers).max(1); + // Allocate remaining capacity to on-demand pool. + let max_on_demand_txs = max_concurrency.saturating_sub(spawned_workers); tracing::debug!( target: "trie::proof_task", @@ -422,20 +394,13 @@ where "Configured on-demand transaction pool for blinded nodes" ); - // ==================== CONSTRUCT MANAGER ==================== - Self { - // Storage worker pool storage_work_tx, storage_worker_count: spawned_workers, - - // On-demand pool max_on_demand_txs, on_demand_txs: Vec::with_capacity(max_on_demand_txs), on_demand_tx_count: 0, pending_on_demand: VecDeque::new(), - - // Shared resources view, task_ctx, executor, @@ -449,11 +414,6 @@ where } /// Returns a handle for sending new proof tasks to the manager. - /// - /// # Interface Compatibility - /// - /// This method is UNCHANGED from the original implementation. The returned - /// `ProofTaskManagerHandle` has the exact same public API as before PR1. pub fn handle(&self) -> ProofTaskManagerHandle> { ProofTaskManagerHandle::new(self.tx_sender.clone(), self.active_handles.clone()) } @@ -520,102 +480,91 @@ where /// /// # Task Routing /// - /// - **Storage Proofs**: Routed to pre-spawned worker pool via bounded channel - /// - If channel is full, falls back to on-demand spawn - /// - **Blinded Nodes**: Queued for on-demand execution (original behavior) + /// - **Storage Proofs**: Routed to pre-spawned worker pool via bounded channel. Falls back to + /// on-demand spawn if channel is full or disconnected. + /// - **Blinded Nodes**: Queued for on-demand execution. /// - /// # Worker Pool Lifecycle + /// # Shutdown /// - /// On termination, `storage_work_tx` is dropped, closing the channel and signaling - /// all workers to shut down gracefully. + /// On termination, `storage_work_tx` is dropped, closing the channel and + /// signaling all workers to shut down gracefully. pub fn run(mut self) -> ProviderResult<()> { loop { match self.proof_task_rx.recv() { Ok(message) => { match message { - ProofTaskMessage::QueueTask(task) => { - match task { - // ==================== STORAGE PROOF ROUTING ==================== - ProofTaskKind::StorageProof(input, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_proofs += 1; - } + ProofTaskMessage::QueueTask(task) => match task { + ProofTaskKind::StorageProof(input, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_proofs += 1; + } - // Try to send to worker pool first - match self - .storage_work_tx - .try_send(StorageProofJob { input, result_sender: sender }) - { - Ok(_) => { - // Successfully queued to worker pool - tracing::trace!( - target: "trie::proof_task", - "Storage proof dispatched to worker pool" - ); - } - Err(crossbeam_channel::TrySendError::Full(job)) => { - // Channel full - fall back to on-demand spawn - tracing::debug!( - target: "trie::proof_task", - "Worker pool queue full, spawning on-demand" - ); - - #[cfg(feature = "metrics")] - { - self.metrics.on_demand_fallback += 1; - } - - // Queue for on-demand execution - self.pending_on_demand.push_back( - ProofTaskKind::StorageProof( - job.input, - job.result_sender, - ), - ); + match self + .storage_work_tx + .try_send(StorageProofJob { input, result_sender: sender }) + { + Ok(_) => { + tracing::trace!( + target: "trie::proof_task", + "Storage proof dispatched to worker pool" + ); + } + Err(crossbeam_channel::TrySendError::Full(job)) => { + tracing::debug!( + target: "trie::proof_task", + "Worker pool queue full, spawning on-demand" + ); + + #[cfg(feature = "metrics")] + { + self.metrics.on_demand_fallback += 1; } - Err(crossbeam_channel::TrySendError::Disconnected(job)) => { - // No workers available (likely all spawns failed) - - // fall back to on-demand - tracing::warn!( - target: "trie::proof_task", - storage_worker_count = self.storage_worker_count, - "Worker pool disconnected (no workers available), falling back to on-demand" - ); - - #[cfg(feature = "metrics")] - { - self.metrics.on_demand_fallback += 1; - } - - // Queue for on-demand execution instead of failing - self.pending_on_demand.push_back( - ProofTaskKind::StorageProof( - job.input, - job.result_sender, - ), - ); + + self.pending_on_demand.push_back( + ProofTaskKind::StorageProof( + job.input, + job.result_sender, + ), + ); + } + Err(crossbeam_channel::TrySendError::Disconnected(job)) => { + tracing::warn!( + target: "trie::proof_task", + storage_worker_count = self.storage_worker_count, + "Worker pool disconnected (no workers available), falling back to on-demand" + ); + + #[cfg(feature = "metrics")] + { + self.metrics.on_demand_fallback += 1; } + + self.pending_on_demand.push_back( + ProofTaskKind::StorageProof( + job.input, + job.result_sender, + ), + ); } } + } - // ==================== BLINDED NODE ROUTING ==================== - ProofTaskKind::BlindedAccountNode(_, _) => { - #[cfg(feature = "metrics")] - { - self.metrics.account_nodes += 1; - } - self.queue_proof_task(task); + ProofTaskKind::BlindedAccountNode(_, _) => { + #[cfg(feature = "metrics")] + { + self.metrics.account_nodes += 1; } - ProofTaskKind::BlindedStorageNode(_, _, _) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_nodes += 1; - } - self.queue_proof_task(task); + self.queue_proof_task(task); + } + ProofTaskKind::BlindedStorageNode(_, _, _) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_nodes += 1; } + self.queue_proof_task(task); } - } + }, ProofTaskMessage::Transaction(tx) => { // Return transaction to on-demand pool self.on_demand_txs.push(tx); @@ -695,82 +644,64 @@ where (trie_cursor_factory, hashed_cursor_factory) } - /// Compute storage proof without consuming self (for worker pool reuse). - /// - /// # Purpose - /// - /// This method enables transaction reuse in the storage worker pool. Unlike the - /// original `storage_proof(self, ...)` which consumes self and returns the - /// transaction to a pool, this method: - /// - /// 1. Borrows self immutably - /// 2. Computes the proof using the owned transaction - /// 3. Returns only the result (transaction remains owned) - /// 4. Can be called repeatedly on the same [`ProofTaskTx`] instance - /// - /// # Usage + /// Compute storage proof without consuming self. /// - /// This is called exclusively by storage workers in the worker pool. On-demand - /// execution still uses the original `storage_proof(self, ...)` method which - /// consumes self and returns the transaction. - /// - /// # Performance - /// - /// By reusing the same transaction and cursor factories across multiple proofs: - /// - Eliminates per-proof transaction creation overhead - /// - Avoids message passing to return transactions - /// - Reduces memory allocations for cursor factories - fn compute_storage_proof(&self, input: &StorageProofInput) -> StorageProofResult { - // ==================== SETUP ==================== - - // Create cursor factories (same as original implementation) + /// Borrows self immutably to allow transaction reuse across multiple calls. + /// Used by storage workers in the worker pool to avoid transaction creation + /// overhead on each proof computation. + fn compute_storage_proof(&self, input: StorageProofInput) -> StorageProofResult { let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); + // Consume the input so we can move large collections (e.g. target slots) without cloning. + let StorageProofInput { + hashed_address, + prefix_set, + target_slots, + with_branch_node_masks, + multi_added_removed_keys, + } = input; + let prefix_set_len = prefix_set.len(); + let target_slots_len = target_slots.len(); + // Get or create added/removed keys context - let multi_added_removed_keys = input - .multi_added_removed_keys - .clone() - .unwrap_or_else(|| Arc::new(MultiAddedRemovedKeys::new())); - let added_removed_keys = multi_added_removed_keys.get_storage(&input.hashed_address); + let multi_added_removed_keys = + multi_added_removed_keys.unwrap_or_else(|| Arc::new(MultiAddedRemovedKeys::new())); + let added_removed_keys = multi_added_removed_keys.get_storage(&hashed_address); let span = tracing::trace_span!( target: "trie::proof_task", "Storage proof calculation", - hashed_address = ?input.hashed_address, + hashed_address = ?hashed_address, // Worker ID embedded in ProofTaskTx for trace correlation worker_id = self.id, ); let _guard = span.enter(); - let target_slots_len = input.target_slots.len(); let proof_start = Instant::now(); // Compute raw storage multiproof (identical to original) - let raw_proof_result = StorageProof::new_hashed( - trie_cursor_factory, - hashed_cursor_factory, - input.hashed_address, - ) - .with_prefix_set_mut(PrefixSetMut::from(input.prefix_set.iter().copied())) - .with_branch_node_masks(input.with_branch_node_masks) - .with_added_removed_keys(added_removed_keys) - .storage_multiproof(input.target_slots.clone()) - .map_err(|e| ParallelStateRootError::Other(e.to_string())); + let raw_proof_result = + StorageProof::new_hashed(trie_cursor_factory, hashed_cursor_factory, hashed_address) + .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) + .with_branch_node_masks(with_branch_node_masks) + .with_added_removed_keys(added_removed_keys) + .storage_multiproof(target_slots) + .map_err(|e| ParallelStateRootError::Other(e.to_string())); // Decode proof into DecodedStorageMultiProof let decoded_result = raw_proof_result.and_then(|raw_proof| { raw_proof.try_into().map_err(|e: alloy_rlp::Error| { ParallelStateRootError::Other(format!( "Failed to decode storage proof for {}: {}", - input.hashed_address, e + hashed_address, e )) }) }); trace!( target: "trie::proof_task", - hashed_address = ?input.hashed_address, - prefix_set_len = input.prefix_set.len(), + hashed_address = ?hashed_address, + prefix_set_len, target_slots = target_slots_len, proof_time_us = proof_start.elapsed().as_micros(), worker_id = self.id, From d4e0adb75ceb64f56e44bdb600966f4b8f05a4d4 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:16:08 +0000 Subject: [PATCH 005/144] refactor(metrics): remove unused storage proof metrics from ProofTaskMetrics and ProofTaskTrieMetrics --- crates/trie/parallel/src/proof_task_metrics.rs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index 97ec8e6f113..cdb59d078d8 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -9,10 +9,6 @@ pub struct ProofTaskMetrics { pub account_nodes: usize, /// Count of blinded storage node requests. pub storage_nodes: usize, - /// Count of storage proof requests routed to worker pool. - pub storage_proofs: usize, - /// Count of times worker pool was full and fell back to on-demand execution. - pub on_demand_fallback: usize, } impl ProofTaskMetrics { @@ -20,7 +16,6 @@ impl ProofTaskMetrics { pub fn record(&self) { self.task_metrics.record_account_nodes(self.account_nodes); self.task_metrics.record_storage_nodes(self.storage_nodes); - self.task_metrics.record_storage_proofs(self.storage_proofs); } } @@ -32,8 +27,6 @@ pub struct ProofTaskTrieMetrics { blinded_account_nodes: Histogram, /// A histogram for the number of blinded storage nodes fetched. blinded_storage_nodes: Histogram, - /// A histogram for the number of storage proofs computed via worker pool. - storage_proofs: Histogram, } impl ProofTaskTrieMetrics { @@ -46,9 +39,4 @@ impl ProofTaskTrieMetrics { pub fn record_storage_nodes(&self, count: usize) { self.blinded_storage_nodes.record(count as f64); } - - /// Record storage proofs computed via worker pool. - pub fn record_storage_proofs(&self, count: usize) { - self.storage_proofs.record(count as f64); - } } From 2957afaef5534b70f0c5f295d600e849be719125 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:16:23 +0000 Subject: [PATCH 006/144] refactor(proof_task): improve documentation and rename variables for clarity - Updated comments in `ProofTaskManager` to enhance clarity regarding on-demand transaction handling and queue management. - Renamed `pending_on_demand` to `on_demand_queue` for better understanding of its purpose. - Adjusted the `new` function documentation to reflect the correct allocation of concurrency budget between storage workers and on-demand transactions. - Improved the `queue_proof_task` method to use the new queue name. --- crates/trie/parallel/src/proof_task.rs | 146 ++++++++++++++----------- 1 file changed, 83 insertions(+), 63 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index fcab2787ff1..ba60ae81a4a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -120,17 +120,17 @@ pub struct ProofTaskManager { /// Calculated as: `max_concurrency` - `storage_worker_count` max_on_demand_txs: usize, - /// Currently available on-demand transactions (reused after return). + /// On-demand transaction pool for blinded node fetches. on_demand_txs: Vec>>, /// Total on-demand transactions created (for ID assignment). on_demand_tx_count: usize, - /// Queue of pending on-demand tasks waiting for available transaction. + /// Queue of tasks waiting for on-demand transaction assignment. /// /// Holds `ProofTaskKind` for both blinded node fetches and storage proof /// fallbacks (when worker pool is full/unavailable). - pending_on_demand: VecDeque, + on_demand_queue: VecDeque, /// Consistent view provider used for creating transactions on-demand. view: ConsistentDbView, @@ -141,14 +141,10 @@ pub struct ProofTaskManager { /// The underlying handle from which to spawn proof tasks. executor: Handle, - /// A receiver for new proof task messages from external callers. - /// - /// This is the `std::mpsc` channel connected to [`ProofTaskManagerHandle`]. + /// Receives proof task requests from [`ProofTaskManagerHandle`]. proof_task_rx: Receiver>>, - /// A sender for internal messaging (transaction returns). - /// - /// Used by on-demand tasks to return transactions to pool. + /// Internal channel for on-demand tasks to return transactions after use. tx_sender: Sender>>, /// The number of active handles. @@ -264,36 +260,12 @@ impl ProofTaskManager where Factory: DatabaseProviderFactory, { - /// Creates a new [`ProofTaskManager`] with the given configuration. - /// - /// # Arguments - /// - /// * `executor` - Tokio runtime handle for spawning workers and tasks - /// * `view` - Consistent database view for creating read-only transactions - /// * `task_ctx` - Shared context (trie updates, hashed state, prefix sets) - /// * `max_concurrency` - Total transaction budget across all execution paths - /// * `storage_worker_count` - Number of storage proof workers to pre-spawn - /// - /// # Transaction Budget Allocation - /// - /// The total `max_concurrency` is split between storage workers (pre-allocated) - /// and the on-demand pool (lazy). We always reserve at least one slot for the - /// on-demand path, so the number of workers actually spawned is capped at - /// `max_concurrency - 1`. - /// - /// For example, if `max_concurrency = 8` and `storage_worker_count = 8`, then - /// 8 workers are requested but only 7 can be accommodated while leaving one - /// on-demand slot, so 7 workers are spawned and the remaining slot is reserved - /// for on-demand transactions (e.g. blinded nodes). + /// Creates a new [`ProofTaskManager`] with pre-spawned storage proof workers. /// - /// # Worker Spawn Resilience - /// - /// If some workers fail to spawn, the on-demand pool is adjusted accordingly - /// and the system continues with fewer workers. - /// - /// # Panics - /// - /// Does not panic. All errors are logged and handled gracefully. + /// The `max_concurrency` budget is split between pre-spawned storage workers and an + /// on-demand pool. At least one slot is always reserved for on-demand, so the actual + /// number of workers spawned is `min(storage_worker_count, max_concurrency - 1)`. + /// If workers fail to spawn, the system continues with fewer workers. pub fn new( executor: Handle, view: ConsistentDbView, @@ -400,7 +372,7 @@ where max_on_demand_txs, on_demand_txs: Vec::with_capacity(max_on_demand_txs), on_demand_tx_count: 0, - pending_on_demand: VecDeque::new(), + on_demand_queue: VecDeque::new(), view, task_ctx, executor, @@ -413,7 +385,7 @@ where } } - /// Returns a handle for sending new proof tasks to the manager. + /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. pub fn handle(&self) -> ProofTaskManagerHandle> { ProofTaskManagerHandle::new(self.tx_sender.clone(), self.active_handles.clone()) } @@ -425,7 +397,7 @@ where { /// Inserts the task into the pending tasks queue. pub fn queue_proof_task(&mut self, task: ProofTaskKind) { - self.pending_on_demand.push_back(task); + self.on_demand_queue.push_back(task); } /// Gets either the next available transaction, or creates a new one if all are in use and the @@ -452,11 +424,11 @@ where /// This will return an error if a transaction must be created on-demand and the consistent view /// provider fails. pub fn try_spawn_next(&mut self) -> ProviderResult<()> { - let Some(task) = self.pending_on_demand.pop_front() else { return Ok(()) }; + let Some(task) = self.on_demand_queue.pop_front() else { return Ok(()) }; let Some(proof_task_tx) = self.get_or_create_tx()? else { // if there are no txs available, requeue the proof task - self.pending_on_demand.push_front(task); + self.on_demand_queue.push_front(task); return Ok(()) }; @@ -495,11 +467,6 @@ where match message { ProofTaskMessage::QueueTask(task) => match task { ProofTaskKind::StorageProof(input, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_proofs += 1; - } - match self .storage_work_tx .try_send(StorageProofJob { input, result_sender: sender }) @@ -516,12 +483,7 @@ where "Worker pool queue full, spawning on-demand" ); - #[cfg(feature = "metrics")] - { - self.metrics.on_demand_fallback += 1; - } - - self.pending_on_demand.push_back( + self.on_demand_queue.push_back( ProofTaskKind::StorageProof( job.input, job.result_sender, @@ -535,12 +497,7 @@ where "Worker pool disconnected (no workers available), falling back to on-demand" ); - #[cfg(feature = "metrics")] - { - self.metrics.on_demand_fallback += 1; - } - - self.pending_on_demand.push_back( + self.on_demand_queue.push_back( ProofTaskKind::StorageProof( job.input, job.result_sender, @@ -709,9 +666,6 @@ where ); decoded_result - - // NOTE: self is NOT consumed - transaction remains owned by worker - // No ProofTaskMessage::Transaction sent } /// Calculates a storage proof for the given hashed address, and desired prefix set. @@ -1069,3 +1023,69 @@ impl TrieNodeProvider for ProofTaskTrieNodeProvider { rx.recv().unwrap() } } + +#[cfg(test)] +mod tests { + use super::*; + use alloy_primitives::map::B256Map; + use reth_provider::{providers::ConsistentDbView, test_utils::create_test_provider_factory}; + use reth_trie_common::{ + prefix_set::TriePrefixSetsMut, updates::TrieUpdatesSorted, HashedAccountsSorted, + HashedPostStateSorted, + }; + use std::sync::Arc; + use tokio::{runtime::Builder, task}; + + fn test_ctx() -> ProofTaskCtx { + ProofTaskCtx::new( + Arc::new(TrieUpdatesSorted::default()), + Arc::new(HashedPostStateSorted::new( + HashedAccountsSorted::default(), + B256Map::default(), + )), + Arc::new(TriePrefixSetsMut::default()), + ) + } + + /// Ensures the storage worker pool plus on-demand pool never exceed the requested concurrency + /// when the storage worker count saturates the budget. + #[test] + fn proof_task_manager_respects_concurrency_budget() { + let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); + runtime.block_on(async { + let handle = tokio::runtime::Handle::current(); + let factory = create_test_provider_factory(); + let view = ConsistentDbView::new(factory, None); + let ctx = test_ctx(); + + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 2, 2); + assert_eq!(manager.storage_worker_count, 1); + assert_eq!(manager.max_on_demand_txs, 1); + assert!(manager.storage_worker_count + manager.max_on_demand_txs <= 2); + + drop(manager); + task::yield_now().await; + }); + } + + /// Ensures the manager falls back to on-demand transactions when the budget only allows a + /// single concurrent transaction. + #[test] + fn proof_task_manager_handles_single_concurrency() { + let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); + runtime.block_on(async { + let handle = tokio::runtime::Handle::current(); + let factory = create_test_provider_factory(); + let view = ConsistentDbView::new(factory, None); + let ctx = test_ctx(); + + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5); + assert_eq!(manager.storage_worker_count, 0); + assert_eq!(manager.max_on_demand_txs, 1); + assert!(manager.storage_worker_count + manager.max_on_demand_txs <= 1); + + drop(manager); + task::yield_now().await; + }); + } +} From 800dcf6d52264e6eb9c8e133edb378d4c0d2fa6c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:33:04 +0000 Subject: [PATCH 007/144] refactor(proof_task): streamline documentation and clarify task management - Removed the unused `OnDemandTask` enum and updated comments in `ProofTaskManager` to clarify the distinction between storage worker pool and on-demand execution. - Enhanced documentation to better describe the public interface and task submission process. - Improved clarity regarding transaction handling and execution paths for proof requests. --- crates/trie/parallel/src/proof_task.rs | 36 +++++++------------------- 1 file changed, 9 insertions(+), 27 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index ba60ae81a4a..b8f117d824e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -64,45 +64,27 @@ struct StorageProofJob { result_sender: Sender, } -/// Internal message for on-demand task execution. -/// -/// **Note**: Currently unused in favor of [`ProofTaskKind`] for simplicity and backwards -/// compatibility. This enum represents a more type-safe design where on-demand tasks -/// (blinded nodes) are strictly separated from worker pool tasks (storage proofs). -/// Available for future refactoring if stricter type safety is desired. -/// -/// These tasks would be executed with lazily-created transactions that are -/// returned to the pool after use (same as current behavior). -#[derive(Debug)] -#[allow(dead_code)] -enum OnDemandTask { - /// Fetch a blinded account node by path - BlindedAccountNode(Nibbles, Sender), - /// Fetch a blinded storage node by account and path - BlindedStorageNode(B256, Nibbles, Sender), -} - -/// A task that manages sending proof requests to worker pools and on-demand tasks. +/// Manager for coordinating proof request execution across different task types. /// /// # Architecture /// -/// This manager maintains two execution paths: +/// This manager handles two distinct execution paths: /// /// 1. **Storage Worker Pool**: /// - Pre-spawned workers with dedicated long-lived transactions /// - Tasks queued via crossbeam bounded channel -/// - Workers continuously process without transaction return overhead +/// - Workers continuously process without transaction overhead /// /// 2. **On-Demand Execution**: /// - Lazy transaction creation for blinded node fetches -/// - Transactions returned to pool after use +/// - Transactions returned to pool after use for reuse /// -/// # External API +/// # Public Interface /// -/// The external API via `ProofTaskManagerHandle`: -/// - `queue_task(ProofTaskKind)` for submitting tasks -/// - `std::mpsc` message passing -/// - Consistent return types and error handling +/// The public interface through `ProofTaskManagerHandle` allows external callers to: +/// - Submit tasks via `queue_task(ProofTaskKind)` +/// - Use standard `std::mpsc` message passing +/// - Receive consistent return types and error handling #[derive(Debug)] pub struct ProofTaskManager { /// Sender for storage proof tasks to worker pool. From 29d48d4e1ecf9a835e59f43a737dd3bb3f139ead Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:36:02 +0000 Subject: [PATCH 008/144] refactor(config): remove storage proof worker configuration - Eliminated the `storage_proof_workers` field and related constants from `TreeConfig`. - Updated the default implementation and related methods to reflect the removal, streamlining the configuration structure. --- crates/engine/primitives/src/config.rs | 47 ------------------- .../tree/src/tree/payload_processor/mod.rs | 5 +- 2 files changed, 4 insertions(+), 48 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 10d93676f5e..e5f58523d03 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -20,12 +20,6 @@ pub const DEFAULT_RESERVED_CPU_CORES: usize = 1; /// Default maximum concurrency for prewarm task. pub const DEFAULT_PREWARM_MAX_CONCURRENCY: usize = 16; -/// Maximum number of storage proof workers -const MAX_STORAGE_PROOF_WORKERS: usize = 12; - -/// Default ratio of storage proof workers to `max_proof_task_concurrency` -const DEFAULT_STORAGE_PROOF_WORKER_RATIO: f32 = 0.5; - const DEFAULT_BLOCK_BUFFER_LIMIT: u32 = 256; const DEFAULT_MAX_INVALID_HEADER_CACHE_LENGTH: u32 = 256; const DEFAULT_MAX_EXECUTE_BLOCK_BATCH_SIZE: usize = 4; @@ -115,9 +109,6 @@ pub struct TreeConfig { prewarm_max_concurrency: usize, /// Whether to unwind canonical header to ancestor during forkchoice updates. allow_unwind_canonical_header: bool, - /// Number of dedicated storage proof workers. - /// If None, defaults to half of `max_proof_task_concurrency`. - storage_proof_workers: Option, } impl Default for TreeConfig { @@ -144,7 +135,6 @@ impl Default for TreeConfig { always_process_payload_attributes_on_canonical_head: false, prewarm_max_concurrency: DEFAULT_PREWARM_MAX_CONCURRENCY, allow_unwind_canonical_header: false, - storage_proof_workers: None, } } } @@ -174,7 +164,6 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head: bool, prewarm_max_concurrency: usize, allow_unwind_canonical_header: bool, - storage_proof_workers: Option, ) -> Self { Self { persistence_threshold, @@ -198,7 +187,6 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head, prewarm_max_concurrency, allow_unwind_canonical_header, - storage_proof_workers, } } @@ -464,39 +452,4 @@ impl TreeConfig { pub const fn prewarm_max_concurrency(&self) -> usize { self.prewarm_max_concurrency } - - /// Get the number of storage proof workers. - /// - /// Defaults to half of `max_proof_task_concurrency`, clamped to valid range and leaving at - /// least one slot for on-demand work. - pub fn storage_proof_workers(&self) -> usize { - let max_allowed = self.max_proof_task_concurrency.saturating_sub(1) as usize; - if max_allowed == 0 { - return 0; - } - - self.storage_proof_workers.unwrap_or_else(|| { - let derived = (self.max_proof_task_concurrency as f32 * - DEFAULT_STORAGE_PROOF_WORKER_RATIO) as usize; - let capped = derived.min(MAX_STORAGE_PROOF_WORKERS); - - capped.clamp(1, max_allowed) - }) - } - - /// Set the number of storage proof workers explicitly. - /// - /// Value is clamped to the remaining concurrency budget (leaving one on-demand slot). - pub const fn with_storage_proof_workers(mut self, workers: usize) -> Self { - let max_allowed = self.max_proof_task_concurrency.saturating_sub(1) as usize; - let clamped = if workers > MAX_STORAGE_PROOF_WORKERS { - MAX_STORAGE_PROOF_WORKERS - } else if workers > max_allowed { - max_allowed - } else { - workers - }; - self.storage_proof_workers = Some(clamped); - self - } } diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 8aa1f0b4bfe..37785d30c2e 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -196,7 +196,10 @@ where state_root_config.prefix_sets.clone(), ); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - let storage_worker_count = config.storage_proof_workers(); + // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and blinded nodes) + let storage_worker_count = (max_proof_task_concurrency / 2) + .max(1) + .min(max_proof_task_concurrency.saturating_sub(1)); let proof_task = ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), From 3fb97c6977bb672f519a25355ab9e2e40edb045b Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 07:53:38 +0000 Subject: [PATCH 009/144] refactor(proof_task): enhance comments and adjust queue capacity logic - Improved comments in `ProofTaskManager` and related functions for better clarity on task management and processing. - Updated queue capacity calculation to use 4x buffering, reducing fallback to slower on-demand execution during burst loads. - Removed redundant variable assignments to streamline the code. --- .../tree/src/tree/payload_processor/mod.rs | 3 +- crates/trie/parallel/src/proof_task.rs | 47 ++++--------------- 2 files changed, 10 insertions(+), 40 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 37785d30c2e..bf17d3a1164 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -196,7 +196,8 @@ where state_root_config.prefix_sets.clone(), ); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and blinded nodes) + // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and + // blinded nodes) let storage_worker_count = (max_proof_task_concurrency / 2) .max(1) .min(max_proof_task_concurrency.saturating_sub(1)); diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index b8f117d824e..384951d621f 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -88,8 +88,6 @@ struct StorageProofJob { #[derive(Debug)] pub struct ProofTaskManager { /// Sender for storage proof tasks to worker pool. - /// - /// Queue capacity = `storage_worker_count` * 2 (for 2x buffering) storage_work_tx: CrossbeamSender, /// Number of storage workers successfully spawned. @@ -177,19 +175,16 @@ fn storage_worker_loop( ); let mut proofs_processed = 0u64; - let start_time = Instant::now(); while let Ok(StorageProofJob { input, result_sender }) = work_rx.recv() { let hashed_address = input.hashed_address; - let prefix_set_len = input.prefix_set.len(); - let target_slots_len = input.target_slots.len(); trace!( target: "trie::proof_task", worker_id, hashed_address = ?hashed_address, - prefix_set_len, - target_slots = target_slots_len, + prefix_set_len = input.prefix_set.len(), + target_slots = input.target_slots.len(), "Processing storage proof" ); @@ -214,26 +209,16 @@ fn storage_worker_loop( worker_id, hashed_address = ?hashed_address, proof_time_us = proof_elapsed.as_micros(), - prefix_set_len, - target_slots = target_slots_len, total_processed = proofs_processed, "Storage proof completed" ); } // Channel closed - graceful shutdown - let total_elapsed = start_time.elapsed(); - tracing::info!( target: "trie::proof_task", worker_id, proofs_processed, - uptime_secs = total_elapsed.as_secs(), - avg_proof_time_ms = if proofs_processed > 0 { - total_elapsed.as_millis() / proofs_processed as u128 - } else { - 0 - }, "Storage proof worker shutting down" ); } @@ -270,7 +255,9 @@ where ); } - let queue_capacity = planned_workers.saturating_mul(2).max(1); + // Use 4x buffering to prevent queue saturation under burst load. + // Deeper queue reduces fallback to slower on-demand execution when workers are busy. + let queue_capacity = planned_workers.saturating_mul(4).max(1); let (storage_work_tx, storage_work_rx) = bounded::(queue_capacity); tracing::info!( @@ -328,26 +315,11 @@ where spawned = spawned_workers, "Spawned fewer storage workers than requested" ); - } else { - tracing::info!( - target: "trie::proof_task", - spawned_workers, - queue_capacity, - "Storage worker pool initialized successfully" - ); } // Allocate remaining capacity to on-demand pool. let max_on_demand_txs = max_concurrency.saturating_sub(spawned_workers); - tracing::debug!( - target: "trie::proof_task", - max_on_demand_txs, - storage_workers = spawned_workers, - total_capacity = max_concurrency, - "Configured on-demand transaction pool for blinded nodes" - ); - Self { storage_work_tx, storage_worker_count: spawned_workers, @@ -569,6 +541,7 @@ impl ProofTaskTx where Tx: DbTx, { + #[inline] fn create_factories(&self) -> ProofFactories<'_, Tx> { let trie_cursor_factory = InMemoryTrieCursorFactory::new( DatabaseTrieCursorFactory::new(&self.tx), @@ -588,6 +561,7 @@ where /// Borrows self immutably to allow transaction reuse across multiple calls. /// Used by storage workers in the worker pool to avoid transaction creation /// overhead on each proof computation. + #[inline] fn compute_storage_proof(&self, input: StorageProofInput) -> StorageProofResult { let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); @@ -599,8 +573,6 @@ where with_branch_node_masks, multi_added_removed_keys, } = input; - let prefix_set_len = prefix_set.len(); - let target_slots_len = target_slots.len(); // Get or create added/removed keys context let multi_added_removed_keys = @@ -611,14 +583,13 @@ where target: "trie::proof_task", "Storage proof calculation", hashed_address = ?hashed_address, - // Worker ID embedded in ProofTaskTx for trace correlation worker_id = self.id, ); let _guard = span.enter(); let proof_start = Instant::now(); - // Compute raw storage multiproof (identical to original) + // Compute raw storage multiproof let raw_proof_result = StorageProof::new_hashed(trie_cursor_factory, hashed_cursor_factory, hashed_address) .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) @@ -640,8 +611,6 @@ where trace!( target: "trie::proof_task", hashed_address = ?hashed_address, - prefix_set_len, - target_slots = target_slots_len, proof_time_us = proof_start.elapsed().as_micros(), worker_id = self.id, "Completed storage proof calculation" From 5779b869e98407bf900e00feaf604c9f13db0f36 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 08:24:23 +0000 Subject: [PATCH 010/144] disable max concurrency --- crates/engine/primitives/src/config.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index e5f58523d03..f622f32c893 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -165,6 +165,7 @@ impl TreeConfig { prewarm_max_concurrency: usize, allow_unwind_canonical_header: bool, ) -> Self { + assert!(max_proof_task_concurrency > 0, "max_proof_task_concurrency must be at least 1"); Self { persistence_threshold, memory_block_buffer_target, @@ -394,6 +395,7 @@ impl TreeConfig { mut self, max_proof_task_concurrency: u64, ) -> Self { + assert!(max_proof_task_concurrency > 0, "max_proof_task_concurrency must be at least 1"); self.max_proof_task_concurrency = max_proof_task_concurrency; self } From 0e33837041f784cc9b5ec693f79801c89f625f56 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 09:02:34 +0000 Subject: [PATCH 011/144] nits --- crates/trie/parallel/src/proof_task.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 384951d621f..6060d6aca1a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -96,8 +96,6 @@ pub struct ProofTaskManager { storage_worker_count: usize, /// Maximum number of on-demand transactions for blinded node fetches. - /// - /// Calculated as: `max_concurrency` - `storage_worker_count` max_on_demand_txs: usize, /// On-demand transaction pool for blinded node fetches. @@ -1001,7 +999,7 @@ mod tests { /// Ensures the storage worker pool plus on-demand pool never exceed the requested concurrency /// when the storage worker count saturates the budget. #[test] - fn proof_task_manager_respects_concurrency_budget() { + fn proof_task_manager_within_concurrency_limit() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); runtime.block_on(async { let handle = tokio::runtime::Handle::current(); From 3bcbc71356c27667b8b9e3f10d9c22f7213a90c9 Mon Sep 17 00:00:00 2001 From: YK Date: Tue, 7 Oct 2025 18:39:50 +0800 Subject: [PATCH 012/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Brian Picciano --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 6060d6aca1a..b03fcbc71ff 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -213,7 +213,7 @@ fn storage_worker_loop( } // Channel closed - graceful shutdown - tracing::info!( + tracing::debug!( target: "trie::proof_task", worker_id, proofs_processed, From 4a67076900d7a98c8e53925018750c730cb4d711 Mon Sep 17 00:00:00 2001 From: YK Date: Tue, 7 Oct 2025 18:39:58 +0800 Subject: [PATCH 013/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Brian Picciano --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index b03fcbc71ff..e9fdccaba9a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -482,7 +482,7 @@ where // Drop storage_work_tx to signal workers to shut down drop(self.storage_work_tx); - tracing::info!( + tracing::debug!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, "Shutting down proof task manager, signaling workers to terminate" From b2d5bcc7a7a36e9987533159e5fc5e096518aa77 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 10:48:12 +0000 Subject: [PATCH 014/144] using unbounded queue --- crates/trie/parallel/src/proof_task.rs | 36 ++++++++------------------ 1 file changed, 11 insertions(+), 25 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index e9fdccaba9a..88cbcda8cdd 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -10,7 +10,7 @@ use crate::root::ParallelStateRootError; use alloy_primitives::{map::B256Set, B256}; -use crossbeam_channel::{bounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; +use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use reth_db_api::transaction::DbTx; use reth_execution_errors::SparseTrieError; use reth_provider::{ @@ -72,8 +72,9 @@ struct StorageProofJob { /// /// 1. **Storage Worker Pool**: /// - Pre-spawned workers with dedicated long-lived transactions -/// - Tasks queued via crossbeam bounded channel +/// - Tasks queued via crossbeam unbounded channel /// - Workers continuously process without transaction overhead +/// - Unbounded queue ensures all storage proofs benefit from transaction reuse /// /// 2. **On-Demand Execution**: /// - Lazy transaction creation for blinded node fetches @@ -253,17 +254,15 @@ where ); } - // Use 4x buffering to prevent queue saturation under burst load. - // Deeper queue reduces fallback to slower on-demand execution when workers are busy. - let queue_capacity = planned_workers.saturating_mul(4).max(1); - let (storage_work_tx, storage_work_rx) = bounded::(queue_capacity); + // Use unbounded channel to ensure all storage proofs are queued to workers. + // This maintains transaction reuse benefits and avoids fallback to on-demand execution. + let (storage_work_tx, storage_work_rx) = unbounded::(); tracing::info!( target: "trie::proof_task", storage_worker_count = planned_workers, - queue_capacity, max_concurrency, - "Initializing storage proof worker pool" + "Initializing storage proof worker pool with unbounded queue" ); let mut spawned_workers = 0; @@ -404,8 +403,8 @@ where /// /// # Task Routing /// - /// - **Storage Proofs**: Routed to pre-spawned worker pool via bounded channel. Falls back to - /// on-demand spawn if channel is full or disconnected. + /// - **Storage Proofs**: Routed to pre-spawned worker pool via unbounded channel. Only falls + /// back to on-demand if workers are disconnected (e.g., all workers panicked). /// - **Blinded Nodes**: Queued for on-demand execution. /// /// # Shutdown @@ -421,7 +420,7 @@ where ProofTaskKind::StorageProof(input, sender) => { match self .storage_work_tx - .try_send(StorageProofJob { input, result_sender: sender }) + .send(StorageProofJob { input, result_sender: sender }) { Ok(_) => { tracing::trace!( @@ -429,20 +428,7 @@ where "Storage proof dispatched to worker pool" ); } - Err(crossbeam_channel::TrySendError::Full(job)) => { - tracing::debug!( - target: "trie::proof_task", - "Worker pool queue full, spawning on-demand" - ); - - self.on_demand_queue.push_back( - ProofTaskKind::StorageProof( - job.input, - job.result_sender, - ), - ); - } - Err(crossbeam_channel::TrySendError::Disconnected(job)) => { + Err(crossbeam_channel::SendError(job)) => { tracing::warn!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, From 8f4e3a1a24ad221c96d9d74e8e28e70c3c92e7cb Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 11:53:01 +0000 Subject: [PATCH 015/144] rm comment --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index e435e914622..1f7acbf4d75 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1236,7 +1236,7 @@ mod tests { config.consistent_view.clone(), task_ctx, 1, - 1, // storage_worker_count: 1 for tests + 1, ); let channel = channel(); From b4bf19399484ce1a5c85b456656af2bc2c7c595a Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 11:58:55 +0000 Subject: [PATCH 016/144] refactor(proof_task): optimize storage proof computation by reusing cursor factories - Introduced pre-created cursor factories in `storage_worker_loop` to reduce overhead during proof computation. - Updated `compute_storage_proof` to accept cursor factories as parameters, enhancing efficiency and clarity. - Improved logging to provide better insights into proof task calculations. --- crates/trie/parallel/src/proof_task.rs | 106 +++++++++++-------------- 1 file changed, 47 insertions(+), 59 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 88cbcda8cdd..f60732cfbee 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -173,6 +173,9 @@ fn storage_worker_loop( "Storage proof worker started" ); + // Create factories once at worker startup to avoid recreation overhead. + let (trie_cursor_factory, hashed_cursor_factory) = proof_tx.create_factories(); + let mut proofs_processed = 0u64; while let Ok(StorageProofJob { input, result_sender }) = work_rx.recv() { @@ -188,7 +191,8 @@ fn storage_worker_loop( ); let proof_start = Instant::now(); - let result = proof_tx.compute_storage_proof(input); + let result = + proof_tx.compute_storage_proof(input, &trie_cursor_factory, &hashed_cursor_factory); let proof_elapsed = proof_start.elapsed(); proofs_processed += 1; @@ -540,15 +544,24 @@ where (trie_cursor_factory, hashed_cursor_factory) } - /// Compute storage proof without consuming self. + /// Compute storage proof with pre-created factories. /// - /// Borrows self immutably to allow transaction reuse across multiple calls. - /// Used by storage workers in the worker pool to avoid transaction creation + /// Accepts cursor factories as parameters to allow reuse across multiple proofs. + /// Used by storage workers in the worker pool to avoid factory recreation /// overhead on each proof computation. #[inline] - fn compute_storage_proof(&self, input: StorageProofInput) -> StorageProofResult { - let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); - + fn compute_storage_proof( + &self, + input: StorageProofInput, + trie_cursor_factory: &InMemoryTrieCursorFactory< + DatabaseTrieCursorFactory<&Tx>, + &TrieUpdatesSorted, + >, + hashed_cursor_factory: &HashedPostStateCursorFactory< + DatabaseHashedCursorFactory<&Tx>, + &HashedPostStateSorted, + >, + ) -> StorageProofResult { // Consume the input so we can move large collections (e.g. target slots) without cloning. let StorageProofInput { hashed_address, @@ -574,13 +587,16 @@ where let proof_start = Instant::now(); // Compute raw storage multiproof - let raw_proof_result = - StorageProof::new_hashed(trie_cursor_factory, hashed_cursor_factory, hashed_address) - .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) - .with_branch_node_masks(with_branch_node_masks) - .with_added_removed_keys(added_removed_keys) - .storage_multiproof(target_slots) - .map_err(|e| ParallelStateRootError::Other(e.to_string())); + let raw_proof_result = StorageProof::new_hashed( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + hashed_address, + ) + .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) + .with_branch_node_masks(with_branch_node_masks) + .with_added_removed_keys(added_removed_keys) + .storage_multiproof(target_slots) + .map_err(|e| ParallelStateRootError::Other(e.to_string())); // Decode proof into DecodedStorageMultiProof let decoded_result = raw_proof_result.and_then(|raw_proof| { @@ -620,67 +636,39 @@ where ) { trace!( target: "trie::proof_task", - hashed_address=?input.hashed_address, + hashed_address = ?input.hashed_address, "Starting storage proof task calculation" ); - let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); - let multi_added_removed_keys = input - .multi_added_removed_keys - .unwrap_or_else(|| Arc::new(MultiAddedRemovedKeys::new())); - let added_removed_keys = multi_added_removed_keys.get_storage(&input.hashed_address); - - let span = tracing::trace_span!( - target: "trie::proof_task", - "Storage proof calculation", - hashed_address=?input.hashed_address, - // Add a unique id because we often have parallel storage proof calculations for the - // same hashed address, and we want to differentiate them during trace analysis. - span_id=self.id, - ); - let span_guard = span.enter(); - + let hashed_address = input.hashed_address; + let prefix_set_len = input.prefix_set.len(); let target_slots_len = input.target_slots.len(); let proof_start = Instant::now(); - let raw_proof_result = StorageProof::new_hashed( - trie_cursor_factory, - hashed_cursor_factory, - input.hashed_address, - ) - .with_prefix_set_mut(PrefixSetMut::from(input.prefix_set.iter().copied())) - .with_branch_node_masks(input.with_branch_node_masks) - .with_added_removed_keys(added_removed_keys) - .storage_multiproof(input.target_slots) - .map_err(|e| ParallelStateRootError::Other(e.to_string())); - - drop(span_guard); - - let decoded_result = raw_proof_result.and_then(|raw_proof| { - raw_proof.try_into().map_err(|e: alloy_rlp::Error| { - ParallelStateRootError::Other(format!( - "Failed to decode storage proof for {}: {}", - input.hashed_address, e - )) - }) - }); - + // Create factories + let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); + let proof_result = + self.compute_storage_proof(input, &trie_cursor_factory, &hashed_cursor_factory); + let proof_time = proof_start.elapsed(); + let success = proof_result.is_ok(); trace!( target: "trie::proof_task", - hashed_address=?input.hashed_address, - prefix_set = ?input.prefix_set.len(), + hashed_address = ?hashed_address, + prefix_set = ?prefix_set_len, target_slots = ?target_slots_len, - proof_time = ?proof_start.elapsed(), + proof_time = ?proof_time, + worker_id = self.id, + success, "Completed storage proof task calculation" ); // send the result back - if let Err(error) = result_sender.send(decoded_result) { + if let Err(error) = result_sender.send(proof_result) { debug!( target: "trie::proof_task", - hashed_address = ?input.hashed_address, + hashed_address = ?hashed_address, ?error, - task_time = ?proof_start.elapsed(), + task_time = ?proof_time, "Storage proof receiver is dropped, discarding the result" ); } From 6282d2ed87967a2525e66b030dd34faec9db4ea3 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:39:10 +0000 Subject: [PATCH 017/144] propogate error up --- crates/trie/parallel/src/proof_task.rs | 60 ++++++++++---------------- 1 file changed, 22 insertions(+), 38 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index f60732cfbee..58669642fac 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -271,15 +271,26 @@ where let mut spawned_workers = 0; for worker_id in 0..planned_workers { - match view.provider_ro() { - Ok(provider_ro) => { + let provider_ro = match view.provider_ro() { + Ok(provider_ro) => provider_ro, + Err(err) => { + tracing::error!( + target: "trie::proof_task", + worker_id, + ?err, + requested = planned_workers, + spawned_workers, + "Failed to create transaction for storage worker, falling back to on-demand execution" + ); + return Err(err); + } + }; + let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); let work_rx = storage_work_rx.clone(); - executor.spawn_blocking(move || { - storage_worker_loop(proof_task_tx, work_rx, worker_id) - }); + executor.spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx, worker_id)); spawned_workers += 1; @@ -290,44 +301,17 @@ where "Storage worker spawned successfully" ); } - Err(err) => { - tracing::warn!( - target: "trie::proof_task", - worker_id, - ?err, - requested = planned_workers, - spawned_workers, - "Failed to create transaction for storage worker, continuing with fewer workers" - ); - } - } - } - - if spawned_workers == 0 { - tracing::error!( - target: "trie::proof_task", - requested = planned_workers, - "Failed to spawn any storage workers - all work will execute on-demand" - ); - } else if spawned_workers < planned_workers { - tracing::warn!( - target: "trie::proof_task", - requested = planned_workers, - spawned = spawned_workers, - "Spawned fewer storage workers than requested" - ); - } - // Allocate remaining capacity to on-demand pool. - let max_on_demand_txs = max_concurrency.saturating_sub(spawned_workers); + // Allocate remaining capacity to on-demand pool for account trie operations. + let remaining_concurrency = max_concurrency.saturating_sub(spawned_workers); Self { storage_work_tx, storage_worker_count: spawned_workers, - max_on_demand_txs, - on_demand_txs: Vec::with_capacity(max_on_demand_txs), - on_demand_tx_count: 0, - on_demand_queue: VecDeque::new(), + max_concurrency: remaining_concurrency, + total_transactions: 0, + pending_tasks: VecDeque::new(), + proof_task_txs: Vec::with_capacity(remaining_concurrency), view, task_ctx, executor, From 838dc6700e7938cbb2330c6f393a66521cee2b4f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:43:21 +0000 Subject: [PATCH 018/144] reduce scope of pr - exclude all accs - not change the logic for pending_tasks and proof_tasks_txs (on-demand proofs) and just continue using it for the BlindedAccountNode requests, but start using dedicated storage workers for StorageProof and BlindedStorageNode requests --- .../tree/src/tree/payload_processor/mod.rs | 26 ++- .../src/tree/payload_processor/multiproof.rs | 3 +- .../engine/tree/src/tree/payload_validator.rs | 30 ++- crates/trie/parallel/src/proof.rs | 3 +- crates/trie/parallel/src/proof_task.rs | 215 +++++++----------- 5 files changed, 130 insertions(+), 147 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index bf17d3a1164..18a1978a131 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -45,7 +45,7 @@ use std::sync::{ mpsc::{self, channel, Sender}, Arc, }; -use tracing::{debug, instrument}; +use tracing::{debug, instrument, warn}; mod configured_sparse_trie; pub mod executor; @@ -201,13 +201,28 @@ where let storage_worker_count = (max_proof_task_concurrency / 2) .max(1) .min(max_proof_task_concurrency.saturating_sub(1)); - let proof_task = ProofTaskManager::new( + let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), task_ctx, max_proof_task_concurrency, storage_worker_count, - ); + ) { + Ok(proof_task) => proof_task, + Err(error) => { + // If we cannot bootstrap the proof task manager, continue with cache prewarming + // only; the caller will detect the missing state root channel and fall back to the + // parallel state root algorithm. + tracing::warn!( + target: "engine::tree", + ?error, + max_concurrency = max_proof_task_concurrency, + requested_workers = storage_worker_count, + "Failed to initialize proof task manager, falling back to cache-only mode" + ); + return self.spawn_cache_exclusive(env, transactions, provider_builder); + } + }; // We set it to half of the proof task concurrency, because often for each multiproof we // spawn one Tokio task for the account proof, and one Tokio task for the storage proof. @@ -472,6 +487,11 @@ impl PayloadHandle { .map_err(|_| ParallelStateRootError::Other("sparse trie task dropped".to_string()))? } + /// Returns `true` if the handle is connected to a background state root task. + pub fn supports_state_root(&self) -> bool { + self.state_root.is_some() + } + /// Returns a state hook to be used to send state updates to this task. /// /// If a multiproof task is spawned the hook will notify it about new states. diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 1f7acbf4d75..b36fcae4f46 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1237,7 +1237,8 @@ mod tests { task_ctx, 1, 1, - ); + ) + .unwrap(); let channel = channel(); MultiProofTask::new(config, executor, proof_task.handle(), channel.0, 1, None) diff --git a/crates/engine/tree/src/tree/payload_validator.rs b/crates/engine/tree/src/tree/payload_validator.rs index cd2c37d1e91..4842f67cd71 100644 --- a/crates/engine/tree/src/tree/payload_validator.rs +++ b/crates/engine/tree/src/tree/payload_validator.rs @@ -877,17 +877,25 @@ where // too expensive because it requires walking all paths in every proof. let spawn_start = Instant::now(); let (handle, strategy) = if trie_input.prefix_sets.is_empty() { - ( - self.payload_processor.spawn( - env, - txs, - provider_builder, - consistent_view, - trie_input, - &self.config, - ), - StateRootStrategy::StateRootTask, - ) + let handle = self.payload_processor.spawn( + env, + txs, + provider_builder, + consistent_view, + trie_input, + &self.config, + ); + let strategy = if handle.supports_state_root() { + StateRootStrategy::StateRootTask + } else { + debug!( + target: "engine::tree", + block=?block_num_hash, + "Proof task initialization failed, falling back to parallel state root" + ); + StateRootStrategy::Parallel + }; + (handle, strategy) // if prefix sets are not empty, we spawn a task that exclusively handles cache // prewarming for transaction execution } else { diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 4c9a3a57cf8..4a2738fd38e 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -448,7 +448,8 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1); + ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1) + .unwrap(); let proof_task_handle = proof_task.handle(); // keep the join handle around to make sure it does not return any errors diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 58669642fac..1073fb16ea1 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -41,7 +41,7 @@ use std::{ time::Instant, }; use tokio::runtime::Handle; -use tracing::{debug, trace}; +use tracing::trace; #[cfg(feature = "metrics")] use crate::proof_task_metrics::ProofTaskMetrics; @@ -70,14 +70,15 @@ struct StorageProofJob { /// /// This manager handles two distinct execution paths: /// -/// 1. **Storage Worker Pool**: +/// 1. **Storage Worker Pool** (for storage trie operations): /// - Pre-spawned workers with dedicated long-lived transactions +/// - Handles `StorageProof` and `BlindedStorageNode` requests /// - Tasks queued via crossbeam unbounded channel /// - Workers continuously process without transaction overhead /// - Unbounded queue ensures all storage proofs benefit from transaction reuse /// -/// 2. **On-Demand Execution**: -/// - Lazy transaction creation for blinded node fetches +/// 2. **On-Demand Execution** (for account trie operations): +/// - Lazy transaction creation for `BlindedAccountNode` requests /// - Transactions returned to pool after use for reuse /// /// # Public Interface @@ -93,23 +94,21 @@ pub struct ProofTaskManager { /// Number of storage workers successfully spawned. /// - /// May be less than requested if transaction creation fails. + /// May be less than requested if concurrency limits reduce the worker budget. storage_worker_count: usize, - /// Maximum number of on-demand transactions for blinded node fetches. - max_on_demand_txs: usize, + /// Max number of database transactions to create for on-demand account trie operations. + max_concurrency: usize, - /// On-demand transaction pool for blinded node fetches. - on_demand_txs: Vec>>, + /// Number of database transactions created for on-demand operations. + total_transactions: usize, - /// Total on-demand transactions created (for ID assignment). - on_demand_tx_count: usize, + /// Proof tasks pending execution (account trie operations only). + pending_tasks: VecDeque, - /// Queue of tasks waiting for on-demand transaction assignment. - /// - /// Holds `ProofTaskKind` for both blinded node fetches and storage proof - /// fallbacks (when worker pool is full/unavailable). - on_demand_queue: VecDeque, + /// The proof task transactions, containing owned cursor factories that are reused for proof + /// calculation (account trie operations only). + proof_task_txs: Vec>>, /// Consistent view provider used for creating transactions on-demand. view: ConsistentDbView, @@ -235,18 +234,21 @@ where /// The `max_concurrency` budget is split between pre-spawned storage workers and an /// on-demand pool. At least one slot is always reserved for on-demand, so the actual /// number of workers spawned is `min(storage_worker_count, max_concurrency - 1)`. - /// If workers fail to spawn, the system continues with fewer workers. + /// Returns an error if the underlying provider fails to create the transactions required for + /// spawning workers. pub fn new( executor: Handle, view: ConsistentDbView, task_ctx: ProofTaskCtx, max_concurrency: usize, storage_worker_count: usize, - ) -> Self { + ) -> ProviderResult { let (tx_sender, proof_task_rx) = channel(); - let worker_budget = max_concurrency.saturating_sub(1); - let planned_workers = storage_worker_count.min(worker_budget); + // Calculate how many workers to spawn. + // We use the full max_concurrency for workers if needed, since on-demand operations + // (BlindedAccountNode) are less frequent and can share the budget. + let planned_workers = storage_worker_count.min(max_concurrency); if planned_workers < storage_worker_count { tracing::debug!( @@ -286,26 +288,26 @@ where } }; - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx = storage_work_rx.clone(); + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx = storage_work_rx.clone(); executor.spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx, worker_id)); - spawned_workers += 1; + spawned_workers += 1; - tracing::debug!( - target: "trie::proof_task", - worker_id, - spawned_workers, - "Storage worker spawned successfully" - ); - } + tracing::debug!( + target: "trie::proof_task", + worker_id, + spawned_workers, + "Storage worker spawned successfully" + ); + } // Allocate remaining capacity to on-demand pool for account trie operations. let remaining_concurrency = max_concurrency.saturating_sub(spawned_workers); - Self { + Ok(Self { storage_work_tx, storage_worker_count: spawned_workers, max_concurrency: remaining_concurrency, @@ -321,7 +323,7 @@ where #[cfg(feature = "metrics")] metrics: ProofTaskMetrics::default(), - } + }) } /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. @@ -336,22 +338,22 @@ where { /// Inserts the task into the pending tasks queue. pub fn queue_proof_task(&mut self, task: ProofTaskKind) { - self.on_demand_queue.push_back(task); + self.pending_tasks.push_back(task); } /// Gets either the next available transaction, or creates a new one if all are in use and the /// total number of transactions created is less than the max concurrency. pub fn get_or_create_tx(&mut self) -> ProviderResult>>> { - if let Some(proof_task_tx) = self.on_demand_txs.pop() { + if let Some(proof_task_tx) = self.proof_task_txs.pop() { return Ok(Some(proof_task_tx)); } // if we can create a new tx within our concurrency limits, create one on-demand - if self.on_demand_tx_count < self.max_on_demand_txs { + if self.total_transactions < self.max_concurrency { let provider_ro = self.view.provider_ro()?; let tx = provider_ro.into_tx(); - self.on_demand_tx_count += 1; - return Ok(Some(ProofTaskTx::new(tx, self.task_ctx.clone(), self.on_demand_tx_count))); + self.total_transactions += 1; + return Ok(Some(ProofTaskTx::new(tx, self.task_ctx.clone(), self.total_transactions))); } Ok(None) @@ -363,25 +365,26 @@ where /// This will return an error if a transaction must be created on-demand and the consistent view /// provider fails. pub fn try_spawn_next(&mut self) -> ProviderResult<()> { - let Some(task) = self.on_demand_queue.pop_front() else { return Ok(()) }; + let Some(task) = self.pending_tasks.pop_front() else { return Ok(()) }; let Some(proof_task_tx) = self.get_or_create_tx()? else { // if there are no txs available, requeue the proof task - self.on_demand_queue.push_front(task); + self.pending_tasks.push_front(task); return Ok(()) }; let tx_sender = self.tx_sender.clone(); self.executor.spawn_blocking(move || match task { - ProofTaskKind::StorageProof(input, sender) => { - proof_task_tx.storage_proof(input, sender, tx_sender); - } ProofTaskKind::BlindedAccountNode(path, sender) => { proof_task_tx.blinded_account_node(path, sender, tx_sender); } ProofTaskKind::BlindedStorageNode(account, path, sender) => { proof_task_tx.blinded_storage_node(account, path, sender, tx_sender); } + // StorageProof should never reach here as it's routed to worker pool + ProofTaskKind::StorageProof(_, _) => { + unreachable!("StorageProof should be routed to worker pool") + } }); Ok(()) @@ -391,9 +394,11 @@ where /// /// # Task Routing /// - /// - **Storage Proofs**: Routed to pre-spawned worker pool via unbounded channel. Only falls - /// back to on-demand if workers are disconnected (e.g., all workers panicked). - /// - **Blinded Nodes**: Queued for on-demand execution. + /// - **Storage Trie Operations** (`StorageProof` and `BlindedStorageNode`): Routed to + /// pre-spawned worker pool via unbounded channel. Only falls back to `pending_tasks` if + /// workers are disconnected (e.g., all workers panicked). + /// - **Account Trie Operations** (`BlindedAccountNode`): Queued for on-demand execution via + /// `pending_tasks`. /// /// # Shutdown /// @@ -420,37 +425,43 @@ where tracing::warn!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, - "Worker pool disconnected (no workers available), falling back to on-demand" + "Worker pool disconnected, cannot process storage proof" ); - self.on_demand_queue.push_back( - ProofTaskKind::StorageProof( - job.input, - job.result_sender, + // Send error back to caller + let _ = job.result_sender.send(Err( + ParallelStateRootError::Other( + "Storage proof worker pool unavailable".to_string(), ), - ); + )); } } } - ProofTaskKind::BlindedAccountNode(_, _) => { + ProofTaskKind::BlindedStorageNode(account, path, sender) => { + // Route storage trie operations to worker pool + // For now, queue to pending_tasks until we add worker pool support #[cfg(feature = "metrics")] { - self.metrics.account_nodes += 1; + self.metrics.storage_nodes += 1; } - self.queue_proof_task(task); + self.queue_proof_task(ProofTaskKind::BlindedStorageNode( + account, path, sender, + )); } - ProofTaskKind::BlindedStorageNode(_, _, _) => { + + ProofTaskKind::BlindedAccountNode(_, _) => { + // Route account trie operations to pending_tasks #[cfg(feature = "metrics")] { - self.metrics.storage_nodes += 1; + self.metrics.account_nodes += 1; } self.queue_proof_task(task); } }, ProofTaskMessage::Transaction(tx) => { - // Return transaction to on-demand pool - self.on_demand_txs.push(tx); + // Return transaction to pending_tasks pool + self.proof_task_txs.push(tx); } ProofTaskMessage::Terminate => { // Drop storage_work_tx to signal workers to shut down @@ -475,7 +486,7 @@ where Err(_) => return Ok(()), }; - // Try spawning on-demand tasks only (storage proofs handled by worker pool) + // Try spawning pending account trie tasks self.try_spawn_next()?; } } @@ -566,7 +577,7 @@ where hashed_address = ?hashed_address, worker_id = self.id, ); - let _guard = span.enter(); + let _span_guard = span.enter(); let proof_start = Instant::now(); @@ -603,64 +614,6 @@ where decoded_result } - /// Calculates a storage proof for the given hashed address, and desired prefix set. - /// - /// **ON-DEMAND VARIANT** - Consumes self, returns transaction to pool. - /// - /// This method is NO LONGER CALLED for storage proofs from the worker pool, - /// but is kept for: - /// 1. Backward compatibility with any direct callers - /// 2. Future use cases that need one-off storage proofs - /// 3. Tests that rely on the transaction return mechanism - fn storage_proof( - self, - input: StorageProofInput, - result_sender: Sender, - tx_sender: Sender>, - ) { - trace!( - target: "trie::proof_task", - hashed_address = ?input.hashed_address, - "Starting storage proof task calculation" - ); - - let hashed_address = input.hashed_address; - let prefix_set_len = input.prefix_set.len(); - let target_slots_len = input.target_slots.len(); - let proof_start = Instant::now(); - - // Create factories - let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); - let proof_result = - self.compute_storage_proof(input, &trie_cursor_factory, &hashed_cursor_factory); - let proof_time = proof_start.elapsed(); - let success = proof_result.is_ok(); - trace!( - target: "trie::proof_task", - hashed_address = ?hashed_address, - prefix_set = ?prefix_set_len, - target_slots = ?target_slots_len, - proof_time = ?proof_time, - worker_id = self.id, - success, - "Completed storage proof task calculation" - ); - - // send the result back - if let Err(error) = result_sender.send(proof_result) { - debug!( - target: "trie::proof_task", - hashed_address = ?hashed_address, - ?error, - task_time = ?proof_time, - "Storage proof receiver is dropped, discarding the result" - ); - } - - // send the tx back - let _ = tx_sender.send(ProofTaskMessage::Transaction(self)); - } - /// Retrieves blinded account node by path. fn blinded_account_node( self, @@ -954,8 +907,7 @@ mod tests { ) } - /// Ensures the storage worker pool plus on-demand pool never exceed the requested concurrency - /// when the storage worker count saturates the budget. + /// Ensures the storage workers are capped by max_concurrency. #[test] fn proof_task_manager_within_concurrency_limit() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -965,18 +917,18 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 2, 2); - assert_eq!(manager.storage_worker_count, 1); - assert_eq!(manager.max_on_demand_txs, 1); - assert!(manager.storage_worker_count + manager.max_on_demand_txs <= 2); + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 2, 2).unwrap(); + // With max_concurrency=2 and storage_worker_count=2, we get 2 workers + assert_eq!(manager.storage_worker_count, 2); + // No remaining concurrency for on-demand + assert_eq!(manager.max_concurrency, 0); drop(manager); task::yield_now().await; }); } - /// Ensures the manager falls back to on-demand transactions when the budget only allows a - /// single concurrent transaction. + /// Ensures the manager caps storage workers to max_concurrency when requested count is higher. #[test] fn proof_task_manager_handles_single_concurrency() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -986,10 +938,11 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5); - assert_eq!(manager.storage_worker_count, 0); - assert_eq!(manager.max_on_demand_txs, 1); - assert!(manager.storage_worker_count + manager.max_on_demand_txs <= 1); + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5).unwrap(); + // With max_concurrency=1 and storage_worker_count=5, we get 1 worker + assert_eq!(manager.storage_worker_count, 1); + // No remaining concurrency for on-demand + assert_eq!(manager.max_concurrency, 0); drop(manager); task::yield_now().await; From 58979459331dfcad657ce609a5e9e49a693bf235 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:46:01 +0000 Subject: [PATCH 019/144] fmt, clippy --- crates/engine/tree/src/tree/payload_processor/mod.rs | 2 +- crates/engine/tree/src/tree/payload_validator.rs | 4 ++++ crates/trie/parallel/src/proof_task.rs | 4 ++-- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 18a1978a131..a2d445abb4b 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -488,7 +488,7 @@ impl PayloadHandle { } /// Returns `true` if the handle is connected to a background state root task. - pub fn supports_state_root(&self) -> bool { + pub const fn supports_state_root(&self) -> bool { self.state_root.is_some() } diff --git a/crates/engine/tree/src/tree/payload_validator.rs b/crates/engine/tree/src/tree/payload_validator.rs index 4842f67cd71..a8886b72a24 100644 --- a/crates/engine/tree/src/tree/payload_validator.rs +++ b/crates/engine/tree/src/tree/payload_validator.rs @@ -885,6 +885,10 @@ where trie_input, &self.config, ); + // The payload processor will silently downgrade to cache-only mode if the proof + // task manager fails to initialize (e.g. provider error). Detect that here and + // fall back to the legacy parallel state root computation so we still attempt + // to reuse the caching pipeline. let strategy = if handle.supports_state_root() { StateRootStrategy::StateRootTask } else { diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 1073fb16ea1..7ea84e6f759 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -907,7 +907,7 @@ mod tests { ) } - /// Ensures the storage workers are capped by max_concurrency. + /// Ensures the storage workers are capped by `max_concurrency`. #[test] fn proof_task_manager_within_concurrency_limit() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -928,7 +928,7 @@ mod tests { }); } - /// Ensures the manager caps storage workers to max_concurrency when requested count is higher. + /// Ensures the manager caps storage workers to `max_concurrency` when requested count is higher. #[test] fn proof_task_manager_handles_single_concurrency() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); From 6b5de7c8deeb3c1367bcfe810e802d484dc9aec6 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:48:16 +0000 Subject: [PATCH 020/144] fmt --- crates/trie/parallel/src/proof_task.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 7ea84e6f759..d5687f01a7e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -928,7 +928,8 @@ mod tests { }); } - /// Ensures the manager caps storage workers to `max_concurrency` when requested count is higher. + /// Ensures the manager caps storage workers to `max_concurrency` when requested count is + /// higher. #[test] fn proof_task_manager_handles_single_concurrency() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); From 05e0eb8751e2c139a791eff68afb63cfb298ff9f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:53:56 +0000 Subject: [PATCH 021/144] refactor(proof_task): consolidate blinded storage node with storage proof --- crates/trie/parallel/src/proof_task.rs | 303 +++++++++++++++---------- 1 file changed, 181 insertions(+), 122 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index d5687f01a7e..e772aad4aad 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -49,19 +49,28 @@ use crate::proof_task_metrics::ProofTaskMetrics; type StorageProofResult = Result; type TrieNodeProviderResult = Result, SparseTrieError>; -/// Internal message for storage proof workers. +/// Internal message for storage workers. /// -/// This is NOT exposed publicly. External callers still use `ProofTaskKind::StorageProof` -/// which is routed through the manager's `std::mpsc` channel. +/// This is NOT exposed publicly. External callers use `ProofTaskKind::StorageProof` or +/// `ProofTaskKind::BlindedStorageNode` which are routed through the manager's `std::mpsc` channel. #[derive(Debug)] -struct StorageProofJob { - /// Storage proof input parameters - input: StorageProofInput, - /// Channel to send result back to original caller - /// - /// This is the same `std::mpsc::Sender` that the external caller provided in - /// `ProofTaskKind::StorageProof(input`, sender). - result_sender: Sender, +enum StorageWorkerJob { + /// Storage proof computation request + StorageProof { + /// Storage proof input parameters + input: StorageProofInput, + /// Channel to send result back to original caller + result_sender: Sender, + }, + /// Blinded storage node retrieval request + BlindedStorageNode { + /// Target account + account: B256, + /// Path to the storage node + path: Nibbles, + /// Channel to send result back to original caller + result_sender: Sender, + }, } /// Manager for coordinating proof request execution across different task types. @@ -89,8 +98,8 @@ struct StorageProofJob { /// - Receive consistent return types and error handling #[derive(Debug)] pub struct ProofTaskManager { - /// Sender for storage proof tasks to worker pool. - storage_work_tx: CrossbeamSender, + /// Sender for storage worker jobs to worker pool. + storage_work_tx: CrossbeamSender, /// Number of storage workers successfully spawned. /// @@ -136,20 +145,20 @@ pub struct ProofTaskManager { metrics: ProofTaskMetrics, } -/// Worker loop for storage proof computation. +/// Worker loop for storage trie operations. /// /// # Lifecycle /// /// Each worker: -/// 1. Receives `StorageProofJob` from crossbeam bounded channel -/// 2. Computes proof using its dedicated long-lived transaction +/// 1. Receives `StorageWorkerJob` from crossbeam unbounded channel +/// 2. Computes result using its dedicated long-lived transaction /// 3. Sends result directly to original caller via `std::mpsc` /// 4. Repeats until channel closes (graceful shutdown) /// /// # Transaction Reuse /// -/// Reuses the same transaction across multiple proofs to avoid transaction -/// creation and cursor factory setup overhead. +/// Reuses the same transaction and cursor factories across multiple operations +/// to avoid transaction creation and cursor factory setup overhead. /// /// # Panic Safety /// @@ -161,7 +170,7 @@ pub struct ProofTaskManager { /// Worker shuts down when the crossbeam channel closes (all senders dropped). fn storage_worker_loop( proof_tx: ProofTaskTx, - work_rx: CrossbeamReceiver, + work_rx: CrossbeamReceiver, worker_id: usize, ) where Tx: DbTx, @@ -169,59 +178,113 @@ fn storage_worker_loop( tracing::debug!( target: "trie::proof_task", worker_id, - "Storage proof worker started" + "Storage worker started" ); // Create factories once at worker startup to avoid recreation overhead. let (trie_cursor_factory, hashed_cursor_factory) = proof_tx.create_factories(); - let mut proofs_processed = 0u64; + // Create blinded provider factory once for all blinded node requests + let blinded_provider_factory = ProofTrieNodeProviderFactory::new( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + proof_tx.task_ctx.prefix_sets.clone(), + ); - while let Ok(StorageProofJob { input, result_sender }) = work_rx.recv() { - let hashed_address = input.hashed_address; + let mut storage_proofs_processed = 0u64; + let mut storage_nodes_processed = 0u64; + + while let Ok(job) = work_rx.recv() { + match job { + StorageWorkerJob::StorageProof { input, result_sender } => { + let hashed_address = input.hashed_address; + + trace!( + target: "trie::proof_task", + worker_id, + hashed_address = ?hashed_address, + prefix_set_len = input.prefix_set.len(), + target_slots = input.target_slots.len(), + "Processing storage proof" + ); + + let proof_start = Instant::now(); + let result = proof_tx.compute_storage_proof( + input, + &trie_cursor_factory, + &hashed_cursor_factory, + ); + + let proof_elapsed = proof_start.elapsed(); + storage_proofs_processed += 1; + + if result_sender.send(result).is_err() { + tracing::debug!( + target: "trie::proof_task", + worker_id, + hashed_address = ?hashed_address, + storage_proofs_processed, + "Storage proof receiver dropped, discarding result" + ); + } - trace!( - target: "trie::proof_task", - worker_id, - hashed_address = ?hashed_address, - prefix_set_len = input.prefix_set.len(), - target_slots = input.target_slots.len(), - "Processing storage proof" - ); + trace!( + target: "trie::proof_task", + worker_id, + hashed_address = ?hashed_address, + proof_time_us = proof_elapsed.as_micros(), + total_processed = storage_proofs_processed, + "Storage proof completed" + ); + } - let proof_start = Instant::now(); - let result = - proof_tx.compute_storage_proof(input, &trie_cursor_factory, &hashed_cursor_factory); + StorageWorkerJob::BlindedStorageNode { account, path, result_sender } => { + trace!( + target: "trie::proof_task", + worker_id, + ?account, + ?path, + "Processing blinded storage node" + ); - let proof_elapsed = proof_start.elapsed(); - proofs_processed += 1; + let start = Instant::now(); + let result = + blinded_provider_factory.storage_node_provider(account).trie_node(&path); + let elapsed = start.elapsed(); - if result_sender.send(result).is_err() { - tracing::debug!( - target: "trie::proof_task", - worker_id, - hashed_address = ?hashed_address, - proofs_processed, - "Storage proof receiver dropped, discarding result" - ); - } + storage_nodes_processed += 1; - trace!( - target: "trie::proof_task", - worker_id, - hashed_address = ?hashed_address, - proof_time_us = proof_elapsed.as_micros(), - total_processed = proofs_processed, - "Storage proof completed" - ); + if result_sender.send(result).is_err() { + tracing::debug!( + target: "trie::proof_task", + worker_id, + ?account, + ?path, + storage_nodes_processed, + "Blinded storage node receiver dropped, discarding result" + ); + } + + trace!( + target: "trie::proof_task", + worker_id, + ?account, + ?path, + elapsed_us = elapsed.as_micros(), + total_processed = storage_nodes_processed, + "Blinded storage node completed" + ); + } + } } // Channel closed - graceful shutdown tracing::debug!( target: "trie::proof_task", worker_id, - proofs_processed, - "Storage proof worker shutting down" + storage_proofs_processed, + storage_nodes_processed, + "Storage worker shutting down" ); } @@ -260,15 +323,15 @@ where ); } - // Use unbounded channel to ensure all storage proofs are queued to workers. + // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. - let (storage_work_tx, storage_work_rx) = unbounded::(); + let (storage_work_tx, storage_work_rx) = unbounded::(); tracing::info!( target: "trie::proof_task", storage_worker_count = planned_workers, max_concurrency, - "Initializing storage proof worker pool with unbounded queue" + "Initializing storage worker pool with unbounded queue" ); let mut spawned_workers = 0; @@ -378,10 +441,10 @@ where ProofTaskKind::BlindedAccountNode(path, sender) => { proof_task_tx.blinded_account_node(path, sender, tx_sender); } - ProofTaskKind::BlindedStorageNode(account, path, sender) => { - proof_task_tx.blinded_storage_node(account, path, sender, tx_sender); + // Storage trie operations should never reach here as they're routed to worker pool + ProofTaskKind::BlindedStorageNode(_, _, _) => { + unreachable!("BlindedStorageNode should be routed to worker pool") } - // StorageProof should never reach here as it's routed to worker pool ProofTaskKind::StorageProof(_, _) => { unreachable!("StorageProof should be routed to worker pool") } @@ -411,10 +474,10 @@ where match message { ProofTaskMessage::QueueTask(task) => match task { ProofTaskKind::StorageProof(input, sender) => { - match self - .storage_work_tx - .send(StorageProofJob { input, result_sender: sender }) - { + match self.storage_work_tx.send(StorageWorkerJob::StorageProof { + input, + result_sender: sender, + }) { Ok(_) => { tracing::trace!( target: "trie::proof_task", @@ -429,25 +492,68 @@ where ); // Send error back to caller - let _ = job.result_sender.send(Err( - ParallelStateRootError::Other( - "Storage proof worker pool unavailable".to_string(), - ), - )); + if let StorageWorkerJob::StorageProof { + result_sender, + .. + } = job + { + let _ = result_sender.send(Err( + ParallelStateRootError::Other( + "Storage proof worker pool unavailable" + .to_string(), + ), + )); + } } } } ProofTaskKind::BlindedStorageNode(account, path, sender) => { - // Route storage trie operations to worker pool - // For now, queue to pending_tasks until we add worker pool support #[cfg(feature = "metrics")] { self.metrics.storage_nodes += 1; } - self.queue_proof_task(ProofTaskKind::BlindedStorageNode( - account, path, sender, - )); + + match self.storage_work_tx.send( + StorageWorkerJob::BlindedStorageNode { + account, + path, + result_sender: sender, + }, + ) { + Ok(_) => { + tracing::trace!( + target: "trie::proof_task", + ?account, + ?path, + "Blinded storage node dispatched to worker pool" + ); + } + Err(crossbeam_channel::SendError(job)) => { + tracing::warn!( + target: "trie::proof_task", + storage_worker_count = self.storage_worker_count, + ?account, + ?path, + "Worker pool disconnected, cannot process blinded storage node" + ); + + // Send error back to caller + if let StorageWorkerJob::BlindedStorageNode { + result_sender, + .. + } = job + { + let _ = result_sender.send(Err(SparseTrieError::from( + Box::new(std::io::Error::new( + std::io::ErrorKind::BrokenPipe, + "Storage worker pool unavailable", + )) + as Box, + ))); + } + } + } } ProofTaskKind::BlindedAccountNode(_, _) => { @@ -656,53 +762,6 @@ where // send the tx back let _ = tx_sender.send(ProofTaskMessage::Transaction(self)); } - - /// Retrieves blinded storage node of the given account by path. - fn blinded_storage_node( - self, - account: B256, - path: Nibbles, - result_sender: Sender, - tx_sender: Sender>, - ) { - trace!( - target: "trie::proof_task", - ?account, - ?path, - "Starting blinded storage node retrieval" - ); - - let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); - - let blinded_provider_factory = ProofTrieNodeProviderFactory::new( - trie_cursor_factory, - hashed_cursor_factory, - self.task_ctx.prefix_sets.clone(), - ); - - let start = Instant::now(); - let result = blinded_provider_factory.storage_node_provider(account).trie_node(&path); - trace!( - target: "trie::proof_task", - ?account, - ?path, - elapsed = ?start.elapsed(), - "Completed blinded storage node retrieval" - ); - - if let Err(error) = result_sender.send(result) { - tracing::error!( - target: "trie::proof_task", - ?account, - ?path, - ?error, - "Failed to send blinded storage node result" - ); - } - - // send the tx back - let _ = tx_sender.send(ProofTaskMessage::Transaction(self)); - } } /// This represents an input for a storage proof. From 4829de9d28811e5d797f189be81a2bc42a788350 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 12:58:17 +0000 Subject: [PATCH 022/144] rm comment --- crates/trie/parallel/src/proof_task.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index e772aad4aad..b3ab50c305a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -278,7 +278,6 @@ fn storage_worker_loop( } } - // Channel closed - graceful shutdown tracing::debug!( target: "trie::proof_task", worker_id, From 6472cfe80baf0f4302125e249883d7128513c7aa Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 13:02:20 +0000 Subject: [PATCH 023/144] simplify worker concurrency --- crates/trie/parallel/src/proof_task.rs | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index b3ab50c305a..06b345af156 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -307,21 +307,8 @@ where ) -> ProviderResult { let (tx_sender, proof_task_rx) = channel(); - // Calculate how many workers to spawn. - // We use the full max_concurrency for workers if needed, since on-demand operations - // (BlindedAccountNode) are less frequent and can share the budget. let planned_workers = storage_worker_count.min(max_concurrency); - if planned_workers < storage_worker_count { - tracing::debug!( - target: "trie::proof_task", - requested = storage_worker_count, - capped = planned_workers, - max_concurrency, - "Adjusted storage worker count to fit concurrency budget" - ); - } - // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. let (storage_work_tx, storage_work_rx) = unbounded::(); From 61ecc9a36640ba5ac3f7c198c6c5840f20b2c882 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 13:04:20 +0000 Subject: [PATCH 024/144] bump to error! --- crates/engine/tree/src/tree/payload_processor/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index a2d445abb4b..7803a01b334 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -213,7 +213,7 @@ where // If we cannot bootstrap the proof task manager, continue with cache prewarming // only; the caller will detect the missing state root channel and fall back to the // parallel state root algorithm. - tracing::warn!( + tracing::error!( target: "engine::tree", ?error, max_concurrency = max_proof_task_concurrency, From 30f6fda3c3da79b355f3c22d8b4a1e73da6963aa Mon Sep 17 00:00:00 2001 From: YK Date: Tue, 7 Oct 2025 21:29:30 +0800 Subject: [PATCH 025/144] Update crates/engine/tree/src/tree/payload_processor/mod.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/engine/tree/src/tree/payload_processor/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 7803a01b334..a2e75bac107 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -199,8 +199,7 @@ where // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and // blinded nodes) let storage_worker_count = (max_proof_task_concurrency / 2) - .max(1) - .min(max_proof_task_concurrency.saturating_sub(1)); + .max(1); let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), From 46803362c1802849150f581fa600b530793a4f99 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 14:39:39 +0000 Subject: [PATCH 026/144] handle sending error back --- crates/trie/parallel/src/proof_task.rs | 53 ++++++++++++++------------ 1 file changed, 28 insertions(+), 25 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 06b345af156..c1638ac1a91 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -73,6 +73,32 @@ enum StorageWorkerJob { }, } +impl StorageWorkerJob { + /// Sends an error back to the caller when worker pool is unavailable. + /// + /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was + /// dropped. + fn send_worker_unavailable_error(&self) -> Result<(), ()> { + match self { + Self::StorageProof { result_sender, .. } => { + let error = ParallelStateRootError::Other( + "Storage proof worker pool unavailable".to_string(), + ); + result_sender.send(Err(error)).map_err(|_| ()) + } + Self::BlindedStorageNode { result_sender, .. } => { + let error = SparseTrieError::from(SparseTrieErrorKind::Other(Box::new( + std::io::Error::new( + std::io::ErrorKind::BrokenPipe, + "Storage worker pool unavailable", + ), + ))); + result_sender.send(Err(error)).map_err(|_| ()) + } + } + } +} + /// Manager for coordinating proof request execution across different task types. /// /// # Architecture @@ -478,18 +504,7 @@ where ); // Send error back to caller - if let StorageWorkerJob::StorageProof { - result_sender, - .. - } = job - { - let _ = result_sender.send(Err( - ParallelStateRootError::Other( - "Storage proof worker pool unavailable" - .to_string(), - ), - )); - } + let _ = job.send_worker_unavailable_error(); } } } @@ -525,19 +540,7 @@ where ); // Send error back to caller - if let StorageWorkerJob::BlindedStorageNode { - result_sender, - .. - } = job - { - let _ = result_sender.send(Err(SparseTrieError::from( - Box::new(std::io::Error::new( - std::io::ErrorKind::BrokenPipe, - "Storage worker pool unavailable", - )) - as Box, - ))); - } + let _ = job.send_worker_unavailable_error(); } } } From 58d6f8b276e6fc83f04ee1cd70e4777d7f454fea Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 14:42:48 +0000 Subject: [PATCH 027/144] fmt --- crates/engine/tree/src/tree/payload_processor/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index a2e75bac107..049419a00e5 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -198,8 +198,7 @@ where let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and // blinded nodes) - let storage_worker_count = (max_proof_task_concurrency / 2) - .max(1); + let storage_worker_count = (max_proof_task_concurrency / 2).max(1); let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), From 59b0353f77a6bb5c558dfcb49ccdc0f4b1977dce Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 14:49:35 +0000 Subject: [PATCH 028/144] fix fmt --- crates/trie/parallel/src/proof_task.rs | 55 ++++++++------------------ 1 file changed, 16 insertions(+), 39 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index c1638ac1a91..aaeb37f2d96 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -12,7 +12,7 @@ use crate::root::ParallelStateRootError; use alloy_primitives::{map::B256Set, B256}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use reth_db_api::transaction::DbTx; -use reth_execution_errors::SparseTrieError; +use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, FactoryTx, ProviderResult, @@ -319,9 +319,10 @@ where { /// Creates a new [`ProofTaskManager`] with pre-spawned storage proof workers. /// - /// The `max_concurrency` budget is split between pre-spawned storage workers and an - /// on-demand pool. At least one slot is always reserved for on-demand, so the actual - /// number of workers spawned is `min(storage_worker_count, max_concurrency - 1)`. + /// The `storage_worker_count` determines how many storage workers to spawn, and + /// `max_concurrency` determines the limit for on-demand operations (blinded nodes). + /// These are now independent - storage workers are spawned as requested, and on-demand + /// operations use a separate concurrency pool. /// Returns an error if the underlying provider fails to create the transactions required for /// spawning workers. pub fn new( @@ -333,21 +334,19 @@ where ) -> ProviderResult { let (tx_sender, proof_task_rx) = channel(); - let planned_workers = storage_worker_count.min(max_concurrency); - // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. let (storage_work_tx, storage_work_rx) = unbounded::(); tracing::info!( target: "trie::proof_task", - storage_worker_count = planned_workers, + storage_worker_count, max_concurrency, "Initializing storage worker pool with unbounded queue" ); let mut spawned_workers = 0; - for worker_id in 0..planned_workers { + for worker_id in 0..storage_worker_count { let provider_ro = match view.provider_ro() { Ok(provider_ro) => provider_ro, Err(err) => { @@ -355,7 +354,7 @@ where target: "trie::proof_task", worker_id, ?err, - requested = planned_workers, + requested = storage_worker_count, spawned_workers, "Failed to create transaction for storage worker, falling back to on-demand execution" ); @@ -379,8 +378,8 @@ where ); } - // Allocate remaining capacity to on-demand pool for account trie operations. - let remaining_concurrency = max_concurrency.saturating_sub(spawned_workers); + // max_concurrency is now used solely for on-demand pool (account trie operations). + let remaining_concurrency = max_concurrency; Ok(Self { storage_work_tx, @@ -955,31 +954,9 @@ mod tests { ) } - /// Ensures the storage workers are capped by `max_concurrency`. - #[test] - fn proof_task_manager_within_concurrency_limit() { - let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); - runtime.block_on(async { - let handle = tokio::runtime::Handle::current(); - let factory = create_test_provider_factory(); - let view = ConsistentDbView::new(factory, None); - let ctx = test_ctx(); - - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 2, 2).unwrap(); - // With max_concurrency=2 and storage_worker_count=2, we get 2 workers - assert_eq!(manager.storage_worker_count, 2); - // No remaining concurrency for on-demand - assert_eq!(manager.max_concurrency, 0); - - drop(manager); - task::yield_now().await; - }); - } - - /// Ensures the manager caps storage workers to `max_concurrency` when requested count is - /// higher. + /// Ensures max_concurrency is independent of storage workers. #[test] - fn proof_task_manager_handles_single_concurrency() { + fn proof_task_manager_independent_pools() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); runtime.block_on(async { let handle = tokio::runtime::Handle::current(); @@ -988,10 +965,10 @@ mod tests { let ctx = test_ctx(); let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5).unwrap(); - // With max_concurrency=1 and storage_worker_count=5, we get 1 worker - assert_eq!(manager.storage_worker_count, 1); - // No remaining concurrency for on-demand - assert_eq!(manager.max_concurrency, 0); + // With storage_worker_count=5, we get exactly 5 workers + assert_eq!(manager.storage_worker_count, 5); + // max_concurrency=1 is for on-demand operations only + assert_eq!(manager.max_concurrency, 1); drop(manager); task::yield_now().await; From 93c67e8b837ab653495d248e391c8476c564e882 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 14:52:21 +0000 Subject: [PATCH 029/144] Enhance TreeConfig with storage worker count configuration - Added a function to determine the default number of storage worker threads based on available parallelism. - Updated TreeConfig to include a storage_worker_count field, initialized with the default value. - Modified payload processor to utilize the new storage_worker_count instead of a hardcoded value. --- crates/engine/primitives/src/config.rs | 27 ++++++++++++++++++- .../tree/src/tree/payload_processor/mod.rs | 4 +-- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index f622f32c893..05d11ec830b 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -6,9 +6,18 @@ pub const DEFAULT_PERSISTENCE_THRESHOLD: u64 = 2; /// How close to the canonical head we persist blocks. pub const DEFAULT_MEMORY_BLOCK_BUFFER_TARGET: u64 = 0; -/// Default maximum concurrency for proof tasks +/// Default maximum concurrency for on-demand proof tasks (blinded nodes) pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; +/// Returns the default number of storage worker threads based on available parallelism. +/// Defaults to half of available parallelism, clamped between 2 and 8. +fn default_storage_worker_count() -> usize { + #[cfg(feature = "std")] + { + std::thread::available_parallelism().map(|n| (n.get() / 2).clamp(2, 8)).unwrap_or(4) + } +} + /// The size of proof targets chunk to spawn in one multiproof calculation. pub const DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE: usize = 10; @@ -109,6 +118,8 @@ pub struct TreeConfig { prewarm_max_concurrency: usize, /// Whether to unwind canonical header to ancestor during forkchoice updates. allow_unwind_canonical_header: bool, + /// Number of storage proof worker threads. + storage_worker_count: usize, } impl Default for TreeConfig { @@ -135,6 +146,7 @@ impl Default for TreeConfig { always_process_payload_attributes_on_canonical_head: false, prewarm_max_concurrency: DEFAULT_PREWARM_MAX_CONCURRENCY, allow_unwind_canonical_header: false, + storage_worker_count: default_storage_worker_count(), } } } @@ -164,6 +176,7 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head: bool, prewarm_max_concurrency: usize, allow_unwind_canonical_header: bool, + storage_worker_count: usize, ) -> Self { assert!(max_proof_task_concurrency > 0, "max_proof_task_concurrency must be at least 1"); Self { @@ -188,6 +201,7 @@ impl TreeConfig { always_process_payload_attributes_on_canonical_head, prewarm_max_concurrency, allow_unwind_canonical_header, + storage_worker_count, } } @@ -454,4 +468,15 @@ impl TreeConfig { pub const fn prewarm_max_concurrency(&self) -> usize { self.prewarm_max_concurrency } + + /// Return the number of storage proof worker threads. + pub const fn storage_worker_count(&self) -> usize { + self.storage_worker_count + } + + /// Setter for the number of storage proof worker threads. + pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { + self.storage_worker_count = storage_worker_count; + self + } } diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 049419a00e5..1ea648a20c9 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -196,9 +196,7 @@ where state_root_config.prefix_sets.clone(), ); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - // Default to half of max concurrency, leaving room for on-demand tasks (Accountproof and - // blinded nodes) - let storage_worker_count = (max_proof_task_concurrency / 2).max(1); + let storage_worker_count = config.storage_worker_count(); let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), From 1954502fddf06d4efc7adb149f25b592bdbfca06 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Tue, 7 Oct 2025 14:53:21 +0000 Subject: [PATCH 030/144] update message --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index aaeb37f2d96..da1fb335871 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -356,7 +356,7 @@ where ?err, requested = storage_worker_count, spawned_workers, - "Failed to create transaction for storage worker, falling back to on-demand execution" + "Failed to create transaction for storage worker" ); return Err(err); } From 2429320e294064e226d57d90606e1d7bcf23c5a5 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 01:08:33 +0000 Subject: [PATCH 031/144] refactor(proof_task): use impl bound - Updated function signatures to accept cursor factories as traits instead of specific types, enhancing flexibility. - Simplified the instantiation of storage proofs by removing unnecessary cloning of cursor factories. --- crates/trie/parallel/src/proof_task.rs | 35 ++++++++++---------------- 1 file changed, 13 insertions(+), 22 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index da1fb335871..724bf2733c2 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -18,10 +18,10 @@ use reth_provider::{ ProviderResult, }; use reth_trie::{ - hashed_cursor::HashedPostStateCursorFactory, + hashed_cursor::{HashedCursorFactory, HashedPostStateCursorFactory}, prefix_set::TriePrefixSetsMut, proof::{ProofTrieNodeProviderFactory, StorageProof}, - trie_cursor::InMemoryTrieCursorFactory, + trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory}, updates::TrieUpdatesSorted, DecodedStorageMultiProof, HashedPostStateSorted, Nibbles, }; @@ -237,8 +237,8 @@ fn storage_worker_loop( let proof_start = Instant::now(); let result = proof_tx.compute_storage_proof( input, - &trie_cursor_factory, - &hashed_cursor_factory, + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), ); let proof_elapsed = proof_start.elapsed(); @@ -642,14 +642,8 @@ where fn compute_storage_proof( &self, input: StorageProofInput, - trie_cursor_factory: &InMemoryTrieCursorFactory< - DatabaseTrieCursorFactory<&Tx>, - &TrieUpdatesSorted, - >, - hashed_cursor_factory: &HashedPostStateCursorFactory< - DatabaseHashedCursorFactory<&Tx>, - &HashedPostStateSorted, - >, + trie_cursor_factory: impl TrieCursorFactory, + hashed_cursor_factory: impl HashedCursorFactory, ) -> StorageProofResult { // Consume the input so we can move large collections (e.g. target slots) without cloning. let StorageProofInput { @@ -676,16 +670,13 @@ where let proof_start = Instant::now(); // Compute raw storage multiproof - let raw_proof_result = StorageProof::new_hashed( - trie_cursor_factory.clone(), - hashed_cursor_factory.clone(), - hashed_address, - ) - .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) - .with_branch_node_masks(with_branch_node_masks) - .with_added_removed_keys(added_removed_keys) - .storage_multiproof(target_slots) - .map_err(|e| ParallelStateRootError::Other(e.to_string())); + let raw_proof_result = + StorageProof::new_hashed(trie_cursor_factory, hashed_cursor_factory, hashed_address) + .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().copied())) + .with_branch_node_masks(with_branch_node_masks) + .with_added_removed_keys(added_removed_keys) + .storage_multiproof(target_slots) + .map_err(|e| ParallelStateRootError::Other(e.to_string())); // Decode proof into DecodedStorageMultiProof let decoded_result = raw_proof_result.and_then(|raw_proof| { From 1902b433304d98a6d9cd1aeae2e2d8f5add84546 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 01:30:12 +0000 Subject: [PATCH 032/144] make spawning falliable - addressed alexey comment --- crates/engine/tree/benches/state_root_task.rs | 26 ++++++---- .../tree/src/tree/payload_processor/mod.rs | 47 +++++++++++-------- .../engine/tree/src/tree/payload_validator.rs | 42 ++++++++++------- crates/trie/parallel/src/proof_task.rs | 2 +- 4 files changed, 70 insertions(+), 47 deletions(-) diff --git a/crates/engine/tree/benches/state_root_task.rs b/crates/engine/tree/benches/state_root_task.rs index 9f61e62d2f9..70d9e037e9d 100644 --- a/crates/engine/tree/benches/state_root_task.rs +++ b/crates/engine/tree/benches/state_root_task.rs @@ -228,16 +228,22 @@ fn bench_state_root(c: &mut Criterion) { }, |(genesis_hash, mut payload_processor, provider, state_updates)| { black_box({ - let mut handle = payload_processor.spawn( - Default::default(), - core::iter::empty::< - Result, core::convert::Infallible>, - >(), - StateProviderBuilder::new(provider.clone(), genesis_hash, None), - ConsistentDbView::new_with_latest_tip(provider).unwrap(), - TrieInput::default(), - &TreeConfig::default(), - ); + let mut handle = payload_processor + .spawn( + Default::default(), + core::iter::empty::< + Result< + Recovered, + core::convert::Infallible, + >, + >(), + StateProviderBuilder::new(provider.clone(), genesis_hash, None), + ConsistentDbView::new_with_latest_tip(provider).unwrap(), + TrieInput::default(), + &TreeConfig::default(), + ) + .map_err(|(err, ..)| err) + .expect("failed to spawn payload processor"); let mut state_hook = handle.state_hook(); diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 1ea648a20c9..3febb5498fb 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -166,6 +166,10 @@ where /// /// This returns a handle to await the final state root and to interact with the tasks (e.g. /// canceling) + /// + /// Returns an error with the original transactions iterator if the proof task manager fails to + /// initialize. + #[allow(clippy::type_complexity)] pub fn spawn>( &mut self, env: ExecutionEnv, @@ -174,7 +178,10 @@ where consistent_view: ConsistentDbView

, trie_input: TrieInput, config: &TreeConfig, - ) -> PayloadHandle, I::Tx>, I::Error> + ) -> Result< + PayloadHandle, I::Tx>, I::Error>, + (reth_provider::ProviderError, I, ExecutionEnv, StateProviderBuilder), + > where P: DatabaseProviderFactory + BlockReader @@ -204,19 +211,15 @@ where max_proof_task_concurrency, storage_worker_count, ) { - Ok(proof_task) => proof_task, + Ok(task) => task, Err(error) => { - // If we cannot bootstrap the proof task manager, continue with cache prewarming - // only; the caller will detect the missing state root channel and fall back to the - // parallel state root algorithm. + // Fall back to parallel state root if proof task manager fails to initialize tracing::error!( target: "engine::tree", ?error, - max_concurrency = max_proof_task_concurrency, - requested_workers = storage_worker_count, - "Failed to initialize proof task manager, falling back to cache-only mode" + "Failed to initialize proof task manager, falling back to parallel state root" ); - return self.spawn_cache_exclusive(env, transactions, provider_builder); + return Err((error, transactions, env, provider_builder)); } }; @@ -269,12 +272,12 @@ where } }); - PayloadHandle { + Ok(PayloadHandle { to_multi_proof, prewarm_handle, state_root: Some(state_root_rx), transactions: execution_rx, - } + }) } /// Spawns a task that exclusively handles cache prewarming for transaction execution. @@ -879,14 +882,20 @@ mod tests { PrecompileCacheMap::default(), ); let provider = BlockchainProvider::new(factory).unwrap(); - let mut handle = payload_processor.spawn( - Default::default(), - core::iter::empty::, core::convert::Infallible>>(), - StateProviderBuilder::new(provider.clone(), genesis_hash, None), - ConsistentDbView::new_with_latest_tip(provider).unwrap(), - TrieInput::from_state(hashed_state), - &TreeConfig::default(), - ); + let mut handle = + payload_processor + .spawn( + Default::default(), + core::iter::empty::< + Result, core::convert::Infallible>, + >(), + StateProviderBuilder::new(provider.clone(), genesis_hash, None), + ConsistentDbView::new_with_latest_tip(provider).unwrap(), + TrieInput::from_state(hashed_state), + &TreeConfig::default(), + ) + .map_err(|(err, ..)| err) + .expect("failed to spawn payload processor"); let mut state_hook = handle.state_hook(); diff --git a/crates/engine/tree/src/tree/payload_validator.rs b/crates/engine/tree/src/tree/payload_validator.rs index a8886b72a24..1e63d29bf79 100644 --- a/crates/engine/tree/src/tree/payload_validator.rs +++ b/crates/engine/tree/src/tree/payload_validator.rs @@ -877,29 +877,37 @@ where // too expensive because it requires walking all paths in every proof. let spawn_start = Instant::now(); let (handle, strategy) = if trie_input.prefix_sets.is_empty() { - let handle = self.payload_processor.spawn( + match self.payload_processor.spawn( env, txs, provider_builder, consistent_view, trie_input, &self.config, - ); - // The payload processor will silently downgrade to cache-only mode if the proof - // task manager fails to initialize (e.g. provider error). Detect that here and - // fall back to the legacy parallel state root computation so we still attempt - // to reuse the caching pipeline. - let strategy = if handle.supports_state_root() { - StateRootStrategy::StateRootTask - } else { - debug!( - target: "engine::tree", - block=?block_num_hash, - "Proof task initialization failed, falling back to parallel state root" - ); - StateRootStrategy::Parallel - }; - (handle, strategy) + ) { + Ok(handle) => { + // Successfully spawned with state root task support + (handle, StateRootStrategy::StateRootTask) + } + Err((error, txs, env, provider_builder)) => { + // Failed to initialize proof task manager, fallback to parallel state + // root + error!( + target: "engine::tree", + block=?block_num_hash, + ?error, + "Failed to initialize proof task manager, falling back to parallel state root" + ); + ( + self.payload_processor.spawn_cache_exclusive( + env, + txs, + provider_builder, + ), + StateRootStrategy::Parallel, + ) + } + } // if prefix sets are not empty, we spawn a task that exclusively handles cache // prewarming for transaction execution } else { diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 724bf2733c2..07bc7543e4b 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -945,7 +945,7 @@ mod tests { ) } - /// Ensures max_concurrency is independent of storage workers. + /// Ensures `max_concurrency` is independent of storage workers. #[test] fn proof_task_manager_independent_pools() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); From 8fb0dd12a42cd5932789f414f29435cdec7d7e66 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 01:31:33 +0000 Subject: [PATCH 033/144] remove error log, as we propogate up --- crates/trie/parallel/src/proof_task.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 07bc7543e4b..3a21b03a434 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -350,14 +350,6 @@ where let provider_ro = match view.provider_ro() { Ok(provider_ro) => provider_ro, Err(err) => { - tracing::error!( - target: "trie::proof_task", - worker_id, - ?err, - requested = storage_worker_count, - spawned_workers, - "Failed to create transaction for storage worker" - ); return Err(err); } }; From e0010d7d5e305a2f627338627f76ead23ca8773c Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 09:40:55 +0800 Subject: [PATCH 034/144] Apply suggestion from @Copilot Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/engine/primitives/src/config.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 05d11ec830b..b719c5e10cd 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -16,6 +16,10 @@ fn default_storage_worker_count() -> usize { { std::thread::available_parallelism().map(|n| (n.get() / 2).clamp(2, 8)).unwrap_or(4) } + #[cfg(not(feature = "std"))] + { + 4 + } } /// The size of proof targets chunk to spawn in one multiproof calculation. From 53cd4bae9120745f4cf8fb87d249ba9f864f4baa Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 01:44:37 +0000 Subject: [PATCH 035/144] use expect instead of unwrap --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index b36fcae4f46..18d394477fb 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1238,7 +1238,7 @@ mod tests { 1, 1, ) - .unwrap(); + .expect("Failed to create ProofTaskManager"); let channel = channel(); MultiProofTask::new(config, executor, proof_task.handle(), channel.0, 1, None) From e854628b1c4af4621dccd3a2ea96cca147db166a Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 11:12:45 +0800 Subject: [PATCH 036/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/trie/parallel/src/proof_task.rs | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 3a21b03a434..286b48d7373 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -347,12 +347,7 @@ where let mut spawned_workers = 0; for worker_id in 0..storage_worker_count { - let provider_ro = match view.provider_ro() { - Ok(provider_ro) => provider_ro, - Err(err) => { - return Err(err); - } - }; + let provider_ro = view.provider_ro()?; let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); From 3b17cc75af3e241e1628484a54161cc9f689dff3 Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 11:12:59 +0800 Subject: [PATCH 037/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/trie/parallel/src/proof_task.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 286b48d7373..5db778f1679 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -320,9 +320,9 @@ where /// Creates a new [`ProofTaskManager`] with pre-spawned storage proof workers. /// /// The `storage_worker_count` determines how many storage workers to spawn, and - /// `max_concurrency` determines the limit for on-demand operations (blinded nodes). + /// `max_concurrency` determines the limit for on-demand operations (blinded account nodes). /// These are now independent - storage workers are spawned as requested, and on-demand - /// operations use a separate concurrency pool. + /// operations use a separate concurrency pool for blinded account nodes. /// Returns an error if the underlying provider fails to create the transactions required for /// spawning workers. pub fn new( From 6c89cf4dc954fce44091ba7ecc22bead45bbb392 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 03:11:51 +0000 Subject: [PATCH 038/144] consolidate --- crates/trie/parallel/src/proof_task.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 5db778f1679..f06c20f3205 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -440,11 +440,8 @@ where proof_task_tx.blinded_account_node(path, sender, tx_sender); } // Storage trie operations should never reach here as they're routed to worker pool - ProofTaskKind::BlindedStorageNode(_, _, _) => { - unreachable!("BlindedStorageNode should be routed to worker pool") - } - ProofTaskKind::StorageProof(_, _) => { - unreachable!("StorageProof should be routed to worker pool") + ProofTaskKind::BlindedStorageNode(_, _, _) | ProofTaskKind::StorageProof(_, _) => { + unreachable!("Storage trie operations should be routed to worker pool") } }); From c5f6eb9a58675f24bc937aac8e2c648fa209c0e2 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 03:15:51 +0000 Subject: [PATCH 039/144] removed the unnecessary remaining_concurrency variable allocation --- crates/trie/parallel/src/proof_task.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index f06c20f3205..24b5dd4b13e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -365,16 +365,14 @@ where ); } - // max_concurrency is now used solely for on-demand pool (account trie operations). - let remaining_concurrency = max_concurrency; - Ok(Self { storage_work_tx, storage_worker_count: spawned_workers, - max_concurrency: remaining_concurrency, + max_concurrency, total_transactions: 0, pending_tasks: VecDeque::new(), - proof_task_txs: Vec::with_capacity(remaining_concurrency), + proof_task_txs: Vec::with_capacity(max_concurrency), /* used for on-demand account + * trie operations */ view, task_ctx, executor, From af73c7a35e86f343df8b1ffb242b5f5eecc3f772 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 03:26:19 +0000 Subject: [PATCH 040/144] clippy --- crates/storage/provider/src/providers/blockchain_provider.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/storage/provider/src/providers/blockchain_provider.rs b/crates/storage/provider/src/providers/blockchain_provider.rs index 890b98124a5..d729d1bfa0a 100644 --- a/crates/storage/provider/src/providers/blockchain_provider.rs +++ b/crates/storage/provider/src/providers/blockchain_provider.rs @@ -2272,7 +2272,7 @@ mod tests { // Invalid/Non-existent argument should return `None` { - call_method!($arg_count, provider, $method, |_,_,_,_| ( ($invalid_args, None)), tx_num, tx_hash, &in_memory_blocks[0], &receipts); + call_method!($arg_count, provider, $method, |_,_,_,_| ($invalid_args, None), tx_num, tx_hash, &in_memory_blocks[0], &receipts); } // Check that the item is only in memory and not in database @@ -2283,7 +2283,7 @@ mod tests { call_method!($arg_count, provider, $method, |_,_,_,_| (args.clone(), expected_item), tx_num, tx_hash, last_mem_block, &receipts); // Ensure the item is not in storage - call_method!($arg_count, provider.database, $method, |_,_,_,_| ( (args, None)), tx_num, tx_hash, last_mem_block, &receipts); + call_method!($arg_count, provider.database, $method, |_,_,_,_| (args, None), tx_num, tx_hash, last_mem_block, &receipts); } )* }}; From a8e52bcbc0bc9d0c9c287accbc5ec11327cd29d3 Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 16:03:30 +0800 Subject: [PATCH 041/144] Apply suggestion from @yongkangc --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 24b5dd4b13e..00b252f2fe1 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -478,7 +478,7 @@ where ); } Err(crossbeam_channel::SendError(job)) => { - tracing::warn!( + tracing::error!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, "Worker pool disconnected, cannot process storage proof" From c48b3285db5c55dcee09becb97e6905689a7a8dd Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 10:30:26 +0000 Subject: [PATCH 042/144] address brian's pr --- crates/engine/tree/src/tree/payload_processor/mod.rs | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 3febb5498fb..d449031606e 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -213,12 +213,6 @@ where ) { Ok(task) => task, Err(error) => { - // Fall back to parallel state root if proof task manager fails to initialize - tracing::error!( - target: "engine::tree", - ?error, - "Failed to initialize proof task manager, falling back to parallel state root" - ); return Err((error, transactions, env, provider_builder)); } }; @@ -486,11 +480,6 @@ impl PayloadHandle { .map_err(|_| ParallelStateRootError::Other("sparse trie task dropped".to_string()))? } - /// Returns `true` if the handle is connected to a background state root task. - pub const fn supports_state_root(&self) -> bool { - self.state_root.is_some() - } - /// Returns a state hook to be used to send state updates to this task. /// /// If a multiproof task is spawned the hook will notify it about new states. From 4934099759d274076ef6a82466548c8fdfb1b682 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 05:30:21 +0000 Subject: [PATCH 043/144] feat: added initial structs for acc proofs support for task manager - Added `AccountMultiproofInput` struct for input parameters related to account multiproof computation. - Introduced `AccountMultiproofJob` struct for managing account multiproof tasks. - Updated `ProofTaskKind` enum to include an account multiproof variant. - Modified task routing logic to ensure account multiproof operations are directed to the appropriate worker pools. --- crates/trie/parallel/src/proof_task.rs | 47 +++++++++++++++++++++++--- 1 file changed, 42 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 00b252f2fe1..3a029445055 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -19,11 +19,12 @@ use reth_provider::{ }; use reth_trie::{ hashed_cursor::{HashedCursorFactory, HashedPostStateCursorFactory}, - prefix_set::TriePrefixSetsMut, + prefix_set::{TriePrefixSets, TriePrefixSetsMut}, proof::{ProofTrieNodeProviderFactory, StorageProof}, trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory}, updates::TrieUpdatesSorted, - DecodedStorageMultiProof, HashedPostStateSorted, Nibbles, + DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, + Nibbles, }; use reth_trie_common::{ added_removed_keys::MultiAddedRemovedKeys, @@ -48,6 +49,7 @@ use crate::proof_task_metrics::ProofTaskMetrics; type StorageProofResult = Result; type TrieNodeProviderResult = Result, SparseTrieError>; +type AccountMultiproofResult = Result; /// Internal message for storage workers. /// @@ -437,9 +439,12 @@ where ProofTaskKind::BlindedAccountNode(path, sender) => { proof_task_tx.blinded_account_node(path, sender, tx_sender); } - // Storage trie operations should never reach here as they're routed to worker pool - ProofTaskKind::BlindedStorageNode(_, _, _) | ProofTaskKind::StorageProof(_, _) => { - unreachable!("Storage trie operations should be routed to worker pool") + // Storage trie operations and account multiproofs should never reach here as they're + // routed to worker pools + ProofTaskKind::BlindedStorageNode(_, _, _) + | ProofTaskKind::StorageProof(_, _) + | ProofTaskKind::AccountMultiproof(_, _) => { + unreachable!("Worker pool operations should be routed to their respective pools") } }); @@ -534,6 +539,14 @@ where } self.queue_proof_task(task); } + + ProofTaskKind::AccountMultiproof(input, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.account_nodes += 1; + } + self.queue_proof_task(ProofTaskKind::AccountMultiproof(input, sender)); + } }, ProofTaskMessage::Transaction(tx) => { // Return transaction to pending_tasks pool @@ -760,6 +773,28 @@ impl StorageProofInput { } } +/// Input parameters for account multiproof computation. +#[derive(Debug, Clone)] +pub struct AccountMultiproofInput { + /// The targets for which to compute the multiproof. + pub targets: MultiProofTargets, + /// The prefix sets for the proof calculation. + pub prefix_sets: TriePrefixSets, + /// Whether or not to collect branch node masks. + pub collect_branch_node_masks: bool, + /// Provided by the user to give the necessary context to retain extra proofs. + pub multi_added_removed_keys: Option>, +} + +/// Internal job for account multiproof workers. +#[derive(Debug)] +struct AccountMultiproofJob { + /// Account multiproof input parameters + input: AccountMultiproofInput, + /// Channel to send result back to original caller + result_sender: Sender, +} + /// Data used for initializing cursor factories that is shared across all storage proof instances. #[derive(Debug, Clone)] pub struct ProofTaskCtx { @@ -808,6 +843,8 @@ pub enum ProofTaskKind { BlindedAccountNode(Nibbles, Sender), /// A blinded storage node request. BlindedStorageNode(B256, Nibbles, Sender), + /// An account multiproof request. + AccountMultiproof(AccountMultiproofInput, Sender), } /// A handle that wraps a single proof task sender that sends a terminate message on `Drop` if the From d8477997871c6036b7696aa97aff205a2a1842cd Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 05:56:58 +0000 Subject: [PATCH 044/144] feat: add account worker count to TreeConfig and update task manager - Introduced `account_worker_count` field in `TreeConfig` struct. - Updated default implementation and constructor to include `account_worker_count`. - Modified `ProofTaskManager` instantiation to utilize `account_worker_count`. - Added getter and setter methods for `account_worker_count` in `TreeConfig`. --- crates/engine/primitives/src/config.rs | 16 ++++++++++++++++ .../tree/src/tree/payload_processor/mod.rs | 2 ++ .../src/tree/payload_processor/multiproof.rs | 1 + crates/trie/parallel/src/proof.rs | 2 +- 4 files changed, 20 insertions(+), 1 deletion(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index b719c5e10cd..079f48f7a84 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -124,6 +124,8 @@ pub struct TreeConfig { allow_unwind_canonical_header: bool, /// Number of storage proof worker threads. storage_worker_count: usize, + /// Number of account proof worker threads. + account_worker_count: usize, } impl Default for TreeConfig { @@ -151,6 +153,7 @@ impl Default for TreeConfig { prewarm_max_concurrency: DEFAULT_PREWARM_MAX_CONCURRENCY, allow_unwind_canonical_header: false, storage_worker_count: default_storage_worker_count(), + account_worker_count: default_storage_worker_count(), } } } @@ -181,6 +184,7 @@ impl TreeConfig { prewarm_max_concurrency: usize, allow_unwind_canonical_header: bool, storage_worker_count: usize, + account_worker_count: usize, ) -> Self { assert!(max_proof_task_concurrency > 0, "max_proof_task_concurrency must be at least 1"); Self { @@ -206,6 +210,7 @@ impl TreeConfig { prewarm_max_concurrency, allow_unwind_canonical_header, storage_worker_count, + account_worker_count, } } @@ -483,4 +488,15 @@ impl TreeConfig { self.storage_worker_count = storage_worker_count; self } + + /// Return the number of account proof worker threads. + pub const fn account_worker_count(&self) -> usize { + self.account_worker_count + } + + /// Setter for the number of account proof worker threads. + pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { + self.account_worker_count = account_worker_count; + self + } } diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index d449031606e..378c0803376 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -204,12 +204,14 @@ where ); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let storage_worker_count = config.storage_worker_count(); + let account_worker_count = config.account_worker_count(); let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), task_ctx, max_proof_task_concurrency, storage_worker_count, + account_worker_count, ) { Ok(task) => task, Err(error) => { diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 18d394477fb..868aaa9578f 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1237,6 +1237,7 @@ mod tests { task_ctx, 1, 1, + 1, ) .expect("Failed to create ProofTaskManager"); let channel = channel(); diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 4a2738fd38e..8f92a8b400e 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -448,7 +448,7 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1) + ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1, 1) .unwrap(); let proof_task_handle = proof_task.handle(); From 4c9c2ee679a88cf38316c1b7b7382e77410b7204 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 07:49:59 +0000 Subject: [PATCH 045/144] feat: implement account worker functionality in ProofTaskManager - Added support for account multiproof operations with dedicated account workers. - Introduced `account_work_tx` and `account_worker_count` fields in `ProofTaskManager`. - Updated worker loop to handle account multiproof jobs and collect storage proofs. - Enhanced task routing to ensure proper handling of account multiproof requests. - Improved documentation for clarity on worker lifecycle and transaction reuse. --- crates/trie/parallel/src/proof_task.rs | 40 ++++++++++++++++++++++---- 1 file changed, 35 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 3a029445055..53d5eff6851 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -339,15 +339,18 @@ where // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. let (storage_work_tx, storage_work_rx) = unbounded::(); + let (account_work_tx, account_work_rx) = unbounded::(); tracing::info!( target: "trie::proof_task", storage_worker_count, + account_worker_count, max_concurrency, - "Initializing storage worker pool with unbounded queue" + "Initializing storage and account worker pools with unbounded queues" ); - let mut spawned_workers = 0; + // Spawn storage workers + let mut spawned_storage_workers = 0; for worker_id in 0..storage_worker_count { let provider_ro = view.provider_ro()?; @@ -357,19 +360,46 @@ where executor.spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx, worker_id)); - spawned_workers += 1; + spawned_storage_workers += 1; tracing::debug!( target: "trie::proof_task", worker_id, - spawned_workers, + spawned_storage_workers, "Storage worker spawned successfully" ); } + // Spawn account workers + let mut spawned_account_workers = 0; + for worker_id in 0..account_worker_count { + let provider_ro = view.provider_ro()?; + + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx = account_work_rx.clone(); + let storage_handle = + ProofTaskManagerHandle::new(tx_sender.clone(), Arc::new(AtomicUsize::new(1))); + + executor.spawn_blocking(move || { + account_worker_loop(proof_task_tx, work_rx, storage_handle, worker_id) + }); + + spawned_account_workers += 1; + + tracing::debug!( + target: "trie::proof_task", + worker_id, + spawned_account_workers, + "Account worker spawned successfully" + ); + } + Ok(Self { storage_work_tx, - storage_worker_count: spawned_workers, + storage_worker_count: spawned_storage_workers, + account_work_tx, + account_worker_count: spawned_account_workers, max_concurrency, total_transactions: 0, pending_tasks: VecDeque::new(), From 2f10f99a83c24b88c20f1531b51a1b3caaa0d1c4 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 08:06:44 +0000 Subject: [PATCH 046/144] refactor: enhance ProofTaskManager with account worker integration - Implemented account worker functionality to handle account multiproof operations. - Introduced `account_work_tx` and `account_worker_count` fields for managing account jobs. - Updated worker loop to process account multiproof tasks and collect associated storage proofs. - Improved task routing to ensure efficient handling of account multiproof requests. - Enhanced documentation for better understanding of worker lifecycle and transaction reuse. --- crates/trie/parallel/src/proof_task.rs | 70 +++++++++++++++++++------- 1 file changed, 53 insertions(+), 17 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 53d5eff6851..b451048a86d 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -465,15 +465,16 @@ where }; let tx_sender = self.tx_sender.clone(); + self.executor.spawn_blocking(move || match task { ProofTaskKind::BlindedAccountNode(path, sender) => { proof_task_tx.blinded_account_node(path, sender, tx_sender); } - // Storage trie operations and account multiproofs should never reach here as they're - // routed to worker pools - ProofTaskKind::BlindedStorageNode(_, _, _) - | ProofTaskKind::StorageProof(_, _) - | ProofTaskKind::AccountMultiproof(_, _) => { + // Worker pool operations should never reach here as they're routed to their respective + // worker pools + ProofTaskKind::AccountMultiproof(_, _) | + ProofTaskKind::BlindedStorageNode(_, _, _) | + ProofTaskKind::StorageProof(_, _) => { unreachable!("Worker pool operations should be routed to their respective pools") } }); @@ -486,15 +487,17 @@ where /// # Task Routing /// /// - **Storage Trie Operations** (`StorageProof` and `BlindedStorageNode`): Routed to - /// pre-spawned worker pool via unbounded channel. Only falls back to `pending_tasks` if - /// workers are disconnected (e.g., all workers panicked). + /// pre-spawned storage worker pool via unbounded channel. Returns error if workers are + /// disconnected (e.g., all workers panicked). + /// - **Account Multiproof Operations** (`AccountMultiproof`): Routed to pre-spawned account + /// worker pool via unbounded channel. Returns error if workers are disconnected. /// - **Account Trie Operations** (`BlindedAccountNode`): Queued for on-demand execution via /// `pending_tasks`. /// /// # Shutdown /// - /// On termination, `storage_work_tx` is dropped, closing the channel and - /// signaling all workers to shut down gracefully. + /// On termination, `storage_work_tx` and `account_work_tx` are dropped, closing the channels + /// and signaling all workers to shut down gracefully. pub fn run(mut self) -> ProviderResult<()> { loop { match self.proof_task_rx.recv() { @@ -571,11 +574,27 @@ where } ProofTaskKind::AccountMultiproof(input, sender) => { - #[cfg(feature = "metrics")] + match self + .account_work_tx + .send(AccountMultiproofJob { input, result_sender: sender }) { - self.metrics.account_nodes += 1; + Ok(_) => { + tracing::trace!( + target: "trie::proof_task", + "Account multiproof dispatched to worker pool" + ); + } + Err(crossbeam_channel::SendError(job)) => { + tracing::error!( + target: "trie::proof_task", + account_worker_count = self.account_worker_count, + "Account worker pool disconnected" + ); + + // Send error back to caller + let _ = job.send_worker_unavailable_error(); + } } - self.queue_proof_task(ProofTaskKind::AccountMultiproof(input, sender)); } }, ProofTaskMessage::Transaction(tx) => { @@ -583,12 +602,14 @@ where self.proof_task_txs.push(tx); } ProofTaskMessage::Terminate => { - // Drop storage_work_tx to signal workers to shut down + // Drop worker channels to signal workers to shut down drop(self.storage_work_tx); + drop(self.account_work_tx); tracing::debug!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, + account_worker_count = self.account_worker_count, "Shutting down proof task manager, signaling workers to terminate" ); @@ -769,7 +790,7 @@ where } /// This represents an input for a storage proof. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct StorageProofInput { /// The hashed address for which the proof is calculated. hashed_address: B256, @@ -825,6 +846,19 @@ struct AccountMultiproofJob { result_sender: Sender, } +impl AccountMultiproofJob { + /// Sends an error back to the caller when worker pool is unavailable. + /// + /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was + /// dropped. + fn send_worker_unavailable_error(&self) -> Result<(), ()> { + let error = ParallelStateRootError::Other( + "Account worker pool unavailable".to_string(), + ); + self.result_sender.send(Err(error)).map_err(|_| ()) + } +} + /// Data used for initializing cursor factories that is shared across all storage proof instances. #[derive(Debug, Clone)] pub struct ProofTaskCtx { @@ -994,7 +1028,7 @@ mod tests { ) } - /// Ensures `max_concurrency` is independent of storage workers. + /// Ensures `max_concurrency` is independent of storage and account workers. #[test] fn proof_task_manager_independent_pools() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -1004,9 +1038,11 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5).unwrap(); - // With storage_worker_count=5, we get exactly 5 workers + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5, 3).unwrap(); + // With storage_worker_count=5, we get exactly 5 storage workers assert_eq!(manager.storage_worker_count, 5); + // With account_worker_count=3, we get exactly 3 account workers + assert_eq!(manager.account_worker_count, 3); // max_concurrency=1 is for on-demand operations only assert_eq!(manager.max_concurrency, 1); From f5538b25a8956c115fa2e7d1e5f68fc7b008d855 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 08:17:38 +0000 Subject: [PATCH 047/144] refactor: slight cleanup - Introduced `WorkerType` enum for better logging of worker types. - Updated `ProofTaskManager` to spawn account workers alongside storage workers. - Implemented `account_worker_loop` to handle account multiproof operations and collect storage proofs. - Improved documentation on worker lifecycle and transaction reuse. - Refactored task management to streamline the spawning of worker pools. --- crates/trie/parallel/src/proof_task.rs | 29 +++++++++++++++----------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index b451048a86d..1bc18e7eb5e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -105,16 +105,17 @@ impl StorageWorkerJob { /// /// # Architecture /// -/// This manager handles two distinct execution paths: +/// This manager handles three distinct execution paths: /// -/// 1. **Storage Worker Pool** (for storage trie operations): +/// 1. **Worker Pools** (for storage and account operations): /// - Pre-spawned workers with dedicated long-lived transactions -/// - Handles `StorageProof` and `BlindedStorageNode` requests -/// - Tasks queued via crossbeam unbounded channel +/// - **Storage pool**: Handles `StorageProof` and `BlindedStorageNode` requests +/// - **Account pool**: Handles `AccountMultiproof` requests, delegates to storage pool +/// - Tasks queued via crossbeam unbounded channels /// - Workers continuously process without transaction overhead -/// - Unbounded queue ensures all storage proofs benefit from transaction reuse +/// - Returns error if worker pool is unavailable (all workers panicked) /// -/// 2. **On-Demand Execution** (for account trie operations): +/// 2. **On-Demand Execution** (for blinded account node operations): /// - Lazy transaction creation for `BlindedAccountNode` requests /// - Transactions returned to pool after use for reuse /// @@ -134,6 +135,12 @@ pub struct ProofTaskManager { /// May be less than requested if concurrency limits reduce the worker budget. storage_worker_count: usize, + /// Sender for account worker jobs to worker pool. + account_work_tx: CrossbeamSender, + + /// Number of account workers successfully spawned. + account_worker_count: usize, + /// Max number of database transactions to create for on-demand account trie operations. max_concurrency: usize, @@ -321,10 +328,9 @@ where { /// Creates a new [`ProofTaskManager`] with pre-spawned storage proof workers. /// - /// The `storage_worker_count` determines how many storage workers to spawn, and + /// The `storage_worker_count` determines how many storage workers to spawn, + /// `account_worker_count` determines how many account workers to spawn, and /// `max_concurrency` determines the limit for on-demand operations (blinded account nodes). - /// These are now independent - storage workers are spawned as requested, and on-demand - /// operations use a separate concurrency pool for blinded account nodes. /// Returns an error if the underlying provider fails to create the transactions required for /// spawning workers. pub fn new( @@ -333,6 +339,7 @@ where task_ctx: ProofTaskCtx, max_concurrency: usize, storage_worker_count: usize, + account_worker_count: usize, ) -> ProviderResult { let (tx_sender, proof_task_rx) = channel(); @@ -852,9 +859,7 @@ impl AccountMultiproofJob { /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was /// dropped. fn send_worker_unavailable_error(&self) -> Result<(), ()> { - let error = ParallelStateRootError::Other( - "Account worker pool unavailable".to_string(), - ); + let error = ParallelStateRootError::Other("Account worker pool unavailable".to_string()); self.result_sender.send(Err(error)).map_err(|_| ()) } } From aeff6d2821eef969ec96debe343ef8f0a4f45b0c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 09:06:46 +0000 Subject: [PATCH 048/144] refactor spawning workers into a generic function --- crates/trie/parallel/src/proof_task.rs | 160 +++++++++++++++++-------- 1 file changed, 112 insertions(+), 48 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 1bc18e7eb5e..8af75e9a456 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -9,8 +9,12 @@ //! [`HashedPostStateCursorFactory`], which are each backed by a database transaction. use crate::root::ParallelStateRootError; -use alloy_primitives::{map::B256Set, B256}; +use alloy_primitives::{ + map::{B256Map, B256Set}, + B256, +}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; +use dashmap::DashMap; use reth_db_api::transaction::DbTx; use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; use reth_provider::{ @@ -23,8 +27,7 @@ use reth_trie::{ proof::{ProofTrieNodeProviderFactory, StorageProof}, trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory}, updates::TrieUpdatesSorted, - DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, - Nibbles, + DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, Nibbles, }; use reth_trie_common::{ added_removed_keys::MultiAddedRemovedKeys, @@ -51,6 +54,24 @@ type StorageProofResult = Result, SparseTrieError>; type AccountMultiproofResult = Result; +/// Worker type identifier +#[derive(Debug)] +enum WorkerType { + /// Storage proof worker + Storage, + /// Account multiproof worker + Account, +} + +impl std::fmt::Display for WorkerType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::Storage => write!(f, "Storage"), + Self::Account => write!(f, "Account"), + } + } +} + /// Internal message for storage workers. /// /// This is NOT exposed publicly. External callers use `ProofTaskKind::StorageProof` or @@ -326,7 +347,7 @@ impl ProofTaskManager where Factory: DatabaseProviderFactory, { - /// Creates a new [`ProofTaskManager`] with pre-spawned storage proof workers. + /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. /// /// The `storage_worker_count` determines how many storage workers to spawn, /// `account_worker_count` determines how many account workers to spawn, and @@ -357,50 +378,38 @@ where ); // Spawn storage workers - let mut spawned_storage_workers = 0; - for worker_id in 0..storage_worker_count { - let provider_ro = view.provider_ro()?; - - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx = storage_work_rx.clone(); - - executor.spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx, worker_id)); - - spawned_storage_workers += 1; - - tracing::debug!( - target: "trie::proof_task", - worker_id, - spawned_storage_workers, - "Storage worker spawned successfully" - ); - } - - // Spawn account workers - let mut spawned_account_workers = 0; - for worker_id in 0..account_worker_count { - let provider_ro = view.provider_ro()?; - - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx = account_work_rx.clone(); - let storage_handle = - ProofTaskManagerHandle::new(tx_sender.clone(), Arc::new(AtomicUsize::new(1))); - - executor.spawn_blocking(move || { - account_worker_loop(proof_task_tx, work_rx, storage_handle, worker_id) - }); - - spawned_account_workers += 1; - - tracing::debug!( - target: "trie::proof_task", - worker_id, - spawned_account_workers, - "Account worker spawned successfully" - ); - } + let spawned_storage_workers = Self::spawn_worker_pool( + &executor, + &view, + &task_ctx, + storage_worker_count, + storage_work_rx, + WorkerType::Storage, + storage_worker_loop, + )?; + + // Spawn account workers with storage handle + // Create a separate counter for internal worker handles to avoid triggering shutdown + // when workers drop their handles (workers run forever until channel closes) + let worker_handles_counter = Arc::new(AtomicUsize::new(0)); + let spawned_account_workers = Self::spawn_worker_pool( + &executor, + &view, + &task_ctx, + account_worker_count, + account_work_rx, + WorkerType::Account, + { + let tx_sender = tx_sender.clone(); + move |proof_tx, work_rx, worker_id| { + let storage_handle = ProofTaskManagerHandle::new( + tx_sender.clone(), + worker_handles_counter.clone(), + ); + account_worker_loop(proof_tx, work_rx, storage_handle, worker_id) + } + }, + )?; Ok(Self { storage_work_tx, @@ -428,6 +437,61 @@ where pub fn handle(&self) -> ProofTaskManagerHandle> { ProofTaskManagerHandle::new(self.tx_sender.clone(), self.active_handles.clone()) } + + /// Spawns a pool of workers with dedicated database transactions. + /// + /// # Type Parameters + /// - `Job`: The job type the workers will process + /// - `F`: The worker loop function type + /// + /// # Parameters + /// - `worker_count`: Number of workers to spawn + /// - `work_rx`: Receiver for the worker job channel + /// - `worker_type`: Type of worker for logging + /// - `worker_fn`: The worker loop function to execute + /// + /// Returns + /// The number of workers successfully spawned + fn spawn_worker_pool( + executor: &Handle, + view: &ConsistentDbView, + task_ctx: &ProofTaskCtx, + worker_count: usize, + work_rx: CrossbeamReceiver, + worker_type: WorkerType, + worker_fn: F, + ) -> ProviderResult + where + Job: Send + 'static, + F: Fn(ProofTaskTx>, CrossbeamReceiver, usize) + + Send + + Clone + + 'static, + { + let mut spawned_workers = 0; + for worker_id in 0..worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = work_rx.clone(); + let worker_fn_clone = worker_fn.clone(); + + executor + .spawn_blocking(move || worker_fn_clone(proof_task_tx, work_rx_clone, worker_id)); + + spawned_workers += 1; + + tracing::debug!( + target: "trie::proof_task", + worker_id, + spawned_workers, + worker_type = %worker_type, + "{} worker spawned successfully", worker_type + ); + } + + Ok(spawned_workers) + } } impl ProofTaskManager From c4c2b4b4c01a8c7aad6fa24ad6d5652b1c1a892d Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 10:20:45 +0000 Subject: [PATCH 049/144] refactor: integrate account proof task handling in MultiproofManager - Added `account_proof_task_handle` to manage account multiproof tasks. - Updated `MultiproofManager` constructor to accept the new account proof task handle. - Enhanced task queuing for account multiproof operations, including error handling for task submission and channel communication. - Improved prefix set handling for account multiproof inputs. --- .../src/tree/payload_processor/multiproof.rs | 46 ++++++++++++++----- 1 file changed, 35 insertions(+), 11 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 868aaa9578f..ea1a08cb57c 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -351,6 +351,8 @@ pub struct MultiproofManager { executor: WorkloadExecutor, /// Sender to the storage proof task. storage_proof_task_handle: ProofTaskManagerHandle>, + /// Sender to the account proof task. + account_proof_task_handle: ProofTaskManagerHandle>, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. /// @@ -376,6 +378,7 @@ where executor: WorkloadExecutor, metrics: MultiProofTaskMetrics, storage_proof_task_handle: ProofTaskManagerHandle>, + account_proof_task_handle: ProofTaskManagerHandle>, max_concurrent: usize, ) -> Self { Self { @@ -385,6 +388,7 @@ where inflight: 0, metrics, storage_proof_task_handle, + account_proof_task_handle, missed_leaves_storage_roots: Default::default(), } } @@ -526,7 +530,7 @@ where state_root_message_sender, multi_added_removed_keys, } = multiproof_input; - let storage_proof_task_handle = self.storage_proof_task_handle.clone(); + let account_proof_task_handle = self.account_proof_task_handle.clone(); let missed_leaves_storage_roots = self.missed_leaves_storage_roots.clone(); self.executor.spawn_blocking(move || { @@ -544,17 +548,36 @@ where ); let start = Instant::now(); - let proof_result = ParallelProof::new( - config.consistent_view, - config.nodes_sorted, - config.state_sorted, - config.prefix_sets, + + // Extend prefix sets with targets + let frozen_prefix_sets = ParallelProof::::extend_prefix_sets_with_targets( + &config.prefix_sets, + &proof_targets, + ); + + // Queue account multiproof to worker pool + let input = AccountMultiproofInput { + targets: proof_targets, + prefix_sets: frozen_prefix_sets, + collect_branch_node_masks: true, + multi_added_removed_keys, missed_leaves_storage_roots, - storage_proof_task_handle.clone(), - ) - .with_branch_node_masks(true) - .with_multi_added_removed_keys(multi_added_removed_keys) - .decoded_multiproof(proof_targets); + }; + + let (sender, receiver) = channel(); + let proof_result: Result = (|| { + account_proof_task_handle + .queue_task(ProofTaskKind::AccountMultiproof(input, sender)) + .map_err(|_| { + ParallelStateRootError::Other( + "Failed to queue account multiproof to worker pool".into(), + ) + })?; + + receiver.recv().map_err(|_| { + ParallelStateRootError::Other("Account multiproof channel closed".into()) + })? + })(); let elapsed = start.elapsed(); trace!( target: "engine::root", @@ -698,6 +721,7 @@ where multiproof_manager: MultiproofManager::new( executor, metrics.clone(), + proof_task_handle.clone(), proof_task_handle, max_concurrency, ), From 4e30bba567cfe77b99e80d779dee4a2b560a6667 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 10:21:13 +0000 Subject: [PATCH 050/144] fmt --- .../tree/src/tree/payload_processor/multiproof.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index ea1a08cb57c..a37cba7b11e 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -19,7 +19,11 @@ use reth_trie::{ updates::TrieUpdatesSorted, DecodedMultiProof, HashedPostState, HashedPostStateSorted, HashedStorage, MultiProofTargets, TrieInput, }; -use reth_trie_parallel::{proof::ParallelProof, proof_task::ProofTaskManagerHandle}; +use reth_trie_parallel::{ + proof::ParallelProof, + proof_task::{AccountMultiproofInput, ProofTaskKind, ProofTaskManagerHandle}, + root::ParallelStateRootError, +}; use std::{ collections::{BTreeMap, VecDeque}, ops::DerefMut, @@ -549,13 +553,13 @@ where let start = Instant::now(); - // Extend prefix sets with targets + // Extend prefix sets with targets let frozen_prefix_sets = ParallelProof::::extend_prefix_sets_with_targets( &config.prefix_sets, &proof_targets, ); - // Queue account multiproof to worker pool + // Queue account multiproof to worker pool let input = AccountMultiproofInput { targets: proof_targets, prefix_sets: frozen_prefix_sets, From 509898b444b37e80913072add8e2185327de537b Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 10:23:01 +0000 Subject: [PATCH 051/144] refactor: implement account worker loop - Added `account_worker_loop` function to handle account multiproof tasks. - Introduced `collect_storage_proofs` function for queuing and collecting storage proofs. - Enhanced error handling for task submission and result collection. - Updated documentation to clarify the workflow of account multiproof operations and storage proof handling. --- crates/trie/parallel/src/proof_task.rs | 91 ++++++++++++++++++++++++++ 1 file changed, 91 insertions(+) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 8af75e9a456..c49ffa8ee39 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -343,6 +343,97 @@ fn storage_worker_loop( ); } +// TODO: Refactor this with storage_worker_loop. ProofTaskManager should be removed in the following +// pr and `MultiproofManager` should be used instead to dispatch jobs directly. +fn account_worker_loop( + proof_tx: ProofTaskTx, + work_rx: CrossbeamReceiver, + storage_proof_handle: ProofTaskManagerHandle, + worker_id: usize, +) where + Tx: DbTx, +{ + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Account multiproof worker started" + ); + + let (trie_cursor_factory, hashed_cursor_factory) = proof_tx.create_factories(); + let mut account_proofs_processed = 0u64; + + while let Ok(AccountMultiproofJob { input, result_sender }) = work_rx.recv() { + trace!( + target: "trie::proof_task", + worker_id, + targets = input.targets.len(), + "Processing account multiproof" + ); + + let proof_start = Instant::now(); + + let storage_proofs = match collect_storage_proofs( + &proof_tx, + &storage_proof_handle, + &input.targets, + &input.prefix_sets, + input.collect_branch_node_masks, + input.multi_added_removed_keys.clone(), + ) { + Ok(proofs) => proofs, + Err(error) => { + let _ = result_sender.send(Err(error)); + continue; + } + }; + + // Use the missed leaves cache passed from the multiproof manager + let missed_leaves_storage_roots = &input.missed_leaves_storage_roots; + + // Create tracker for metrics (workers don't use stats, but function requires it) + let mut tracker = crate::stats::ParallelTrieTracker::default(); + + let result = crate::proof::build_account_multiproof_with_storage_roots( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + &input.targets, + &input.prefix_sets.account_prefix_set, + input.collect_branch_node_masks, + input.multi_added_removed_keys.as_ref(), + storage_proofs, + missed_leaves_storage_roots, + &mut tracker, + ); + + let proof_elapsed = proof_start.elapsed(); + account_proofs_processed += 1; + + if result_sender.send(result).is_err() { + tracing::debug!( + target: "trie::proof_task", + worker_id, + account_proofs_processed, + "Account multiproof receiver dropped, discarding result" + ); + } + + trace!( + target: "trie::proof_task", + worker_id, + proof_time_us = proof_elapsed.as_micros(), + total_processed = account_proofs_processed, + "Account multiproof completed" + ); + } + + tracing::debug!( + target: "trie::proof_task", + worker_id, + account_proofs_processed, + "Account multiproof worker shutting down" + ); +} + impl ProofTaskManager where Factory: DatabaseProviderFactory, From 7892d5452b149550110c3fe583addac5d57d1449 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 10:51:49 +0000 Subject: [PATCH 052/144] pass in `missed_leaves_storage_roots` --- crates/trie/parallel/src/proof_task.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index c49ffa8ee39..703ac0f2c91 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -560,6 +560,7 @@ where + 'static, { let mut spawned_workers = 0; + // spawns workers that will execute worker_fn for worker_id in 0..worker_count { let provider_ro = view.provider_ro()?; let tx = provider_ro.into_tx(); @@ -997,6 +998,8 @@ pub struct AccountMultiproofInput { pub collect_branch_node_masks: bool, /// Provided by the user to give the necessary context to retain extra proofs. pub multi_added_removed_keys: Option>, + /// Cached storage proof roots for missed leaves encountered during account trie walk. + pub missed_leaves_storage_roots: Arc>, } /// Internal job for account multiproof workers. From 26b8ca817d2e382cf441d1cd35ae6e78ad0cb560 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:06:43 +0000 Subject: [PATCH 053/144] refactor: remove db - Introduced `extend_prefix_sets_with_targets` method to streamline prefix set preparation for multiproof generation. - Updated `decoded_multiproof` to utilize the new method, improving clarity and reducing code duplication. - Refactored account multiproof task queuing to enhance error handling and communication with worker pools. - Improved documentation for methods related to account multiproof operations and storage proof handling. --- crates/trie/parallel/src/proof.rs | 69 +++++++++++++++++-------------- 1 file changed, 39 insertions(+), 30 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 8f92a8b400e..6d2ca2006d0 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -1,29 +1,27 @@ use crate::{ metrics::ParallelTrieMetrics, - proof_task::{ProofTaskKind, ProofTaskManagerHandle, StorageProofInput}, + proof_task::{ + AccountMultiproofInput, ProofTaskKind, ProofTaskManagerHandle, StorageProofInput, + }, root::ParallelStateRootError, stats::ParallelTrieTracker, StorageRootTargets, }; use alloy_primitives::{ - map::{B256Map, B256Set, HashMap}, + map::{B256Map, B256Set}, B256, }; use alloy_rlp::{BufMut, Encodable}; use dashmap::DashMap; -use itertools::Itertools; use reth_execution_errors::StorageRootError; -use reth_provider::{ - providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, FactoryTx, - ProviderError, -}; +use reth_provider::{BlockReader, DatabaseProviderFactory, FactoryTx, ProviderError}; use reth_storage_errors::db::DatabaseError; use reth_trie::{ - hashed_cursor::{HashedCursorFactory, HashedPostStateCursorFactory}, + hashed_cursor::HashedCursorFactory, node_iter::{TrieElement, TrieNodeIter}, - prefix_set::{PrefixSet, PrefixSetMut, TriePrefixSetsMut}, + prefix_set::{PrefixSet, PrefixSetMut, TriePrefixSets, TriePrefixSetsMut}, proof::StorageProof, - trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory}, + trie_cursor::TrieCursorFactory, updates::TrieUpdatesSorted, walker::TrieWalker, DecodedMultiProof, DecodedStorageMultiProof, HashBuilder, HashedPostStateSorted, @@ -33,8 +31,10 @@ use reth_trie_common::{ added_removed_keys::MultiAddedRemovedKeys, proof::{DecodedProofNodes, ProofRetainer}, }; -use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; -use std::sync::{mpsc::Receiver, Arc}; +use std::sync::{ + mpsc::{channel, Receiver}, + Arc, +}; use tracing::trace; /// Parallel proof calculator. @@ -43,8 +43,6 @@ use tracing::trace; /// that has proof targets. #[derive(Debug)] pub struct ParallelProof { - /// Consistent view of the database. - view: ConsistentDbView, /// The sorted collection of cached in-memory intermediate trie nodes that /// can be reused for computation. pub nodes_sorted: Arc, @@ -70,7 +68,6 @@ pub struct ParallelProof { impl ParallelProof { /// Create new state proof generator. pub fn new( - view: ConsistentDbView, nodes_sorted: Arc, state_sorted: Arc, prefix_sets: Arc, @@ -78,7 +75,6 @@ impl ParallelProof { storage_proof_task_handle: ProofTaskManagerHandle>, ) -> Self { Self { - view, nodes_sorted, state_sorted, prefix_sets, @@ -167,16 +163,16 @@ where proof_result } - /// Generate a state multiproof according to specified targets. - pub fn decoded_multiproof( - self, - targets: MultiProofTargets, - ) -> Result { - let mut tracker = ParallelTrieTracker::default(); - - // Extend prefix sets with targets - let mut prefix_sets = (*self.prefix_sets).clone(); - prefix_sets.extend(TriePrefixSetsMut { + /// Extends prefix sets with the given multiproof targets and returns the frozen result. + /// + /// This is a helper function used to prepare prefix sets before computing multiproofs. + /// Returns frozen (immutable) prefix sets ready for use in proof computation. + pub fn extend_prefix_sets_with_targets( + base_prefix_sets: &TriePrefixSetsMut, + targets: &MultiProofTargets, + ) -> TriePrefixSets { + let mut extended = base_prefix_sets.clone(); + extended.extend(TriePrefixSetsMut { account_prefix_set: PrefixSetMut::from(targets.keys().copied().map(Nibbles::unpack)), storage_prefix_sets: targets .iter() @@ -187,7 +183,18 @@ where .collect(), destroyed_accounts: Default::default(), }); - let prefix_sets = prefix_sets.freeze(); + extended.freeze() + } + + /// Generate a state multiproof according to specified targets. + pub fn decoded_multiproof( + self, + targets: MultiProofTargets, + ) -> Result { + let mut tracker = ParallelTrieTracker::default(); + + // Extend prefix sets with targets + let prefix_sets = Self::extend_prefix_sets_with_targets(&self.prefix_sets, &targets); let storage_root_targets = StorageRootTargets::new( prefix_sets.account_prefix_set.iter().map(|nibbles| B256::from_slice(&nibbles.pack())), @@ -371,13 +378,16 @@ mod tests { use crate::proof_task::{ProofTaskCtx, ProofTaskManager}; use alloy_primitives::{ keccak256, - map::{B256Set, DefaultHashBuilder}, + map::{B256Set, DefaultHashBuilder, HashMap}, Address, U256, }; use rand::Rng; use reth_primitives_traits::{Account, StorageEntry}; - use reth_provider::{test_utils::create_test_provider_factory, HashingWriter}; + use reth_provider::{ + providers::ConsistentDbView, test_utils::create_test_provider_factory, HashingWriter, + }; use reth_trie::proof::Proof; + use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; use tokio::runtime::Runtime; #[test] @@ -457,7 +467,6 @@ mod tests { let join_handle = rt.spawn_blocking(move || proof_task.run()); let parallel_result = ParallelProof::new( - consistent_view, Default::default(), Default::default(), Default::default(), From c48085dfbc54c90f98fca49bea80812066155493 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:18:37 +0000 Subject: [PATCH 054/144] remove db --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index a37cba7b11e..9453fd7acb8 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -475,8 +475,7 @@ where "Starting dedicated storage proof calculation", ); let start = Instant::now(); - let proof_result = ParallelProof::new( - config.consistent_view, + let proof_result = ParallelProof::::new( config.nodes_sorted, config.state_sorted, config.prefix_sets, From 66bf6945fdae65a60710056a1d654324ac437201 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:26:10 +0000 Subject: [PATCH 055/144] refactor: streamline account multiproof generation - Removed unnecessary intermediate variables and streamlined the logic for calculating storage root targets. - Introduced a new helper function `build_account_multiproof_with_storage_roots` to encapsulate the account multiproof construction process. - Enhanced error handling for account multiproof task queuing and result retrieval. - Improved documentation for clarity on the multiproof generation workflow and related methods. --- crates/trie/parallel/src/proof.rs | 124 ++++++++++++++----------- crates/trie/parallel/src/proof_task.rs | 9 +- 2 files changed, 79 insertions(+), 54 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 6d2ca2006d0..82ada097467 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -191,16 +191,14 @@ where self, targets: MultiProofTargets, ) -> Result { - let mut tracker = ParallelTrieTracker::default(); - // Extend prefix sets with targets let prefix_sets = Self::extend_prefix_sets_with_targets(&self.prefix_sets, &targets); - let storage_root_targets = StorageRootTargets::new( + let storage_root_targets_len = StorageRootTargets::new( prefix_sets.account_prefix_set.iter().map(|nibbles| B256::from_slice(&nibbles.pack())), prefix_sets.storage_prefix_sets.clone(), - ); - let storage_root_targets_len = storage_root_targets.len(); + ) + .len(); trace!( target: "trie::parallel_proof", @@ -208,42 +206,80 @@ where "Starting parallel proof generation" ); - // Pre-calculate storage roots for accounts which were changed. - tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); + // Queue account multiproof request to account worker pool - // stores the receiver for the storage proof outcome for the hashed addresses - // this way we can lazily await the outcome when we iterate over the map - let mut storage_proof_receivers = - B256Map::with_capacity_and_hasher(storage_root_targets.len(), Default::default()); + let input = AccountMultiproofInput { + targets, + prefix_sets, + collect_branch_node_masks: self.collect_branch_node_masks, + multi_added_removed_keys: self.multi_added_removed_keys.clone(), + missed_leaves_storage_roots: self.missed_leaves_storage_roots.clone(), + }; - for (hashed_address, prefix_set) in - storage_root_targets.into_iter().sorted_unstable_by_key(|(address, _)| *address) - { - let target_slots = targets.get(&hashed_address).cloned().unwrap_or_default(); - let receiver = self.queue_storage_proof(hashed_address, prefix_set, target_slots); + let (sender, receiver) = channel(); + self.storage_proof_task_handle + .queue_task(ProofTaskKind::AccountMultiproof(input, sender)) + .map_err(|_| { + ParallelStateRootError::Other( + "Failed to queue account multiproof: account worker pool unavailable" + .to_string(), + ) + })?; - // store the receiver for that result with the hashed address so we can await this in - // place when we iterate over the trie - storage_proof_receivers.insert(hashed_address, receiver); - } + // Wait for account multiproof result from worker + let (multiproof, stats) = receiver.recv().map_err(|_| { + ParallelStateRootError::Other( + "Account multiproof channel dropped: worker died or pool shutdown".to_string(), + ) + })??; - let provider_ro = self.view.provider_ro()?; - let trie_cursor_factory = InMemoryTrieCursorFactory::new( - DatabaseTrieCursorFactory::new(provider_ro.tx_ref()), - &self.nodes_sorted, - ); - let hashed_cursor_factory = HashedPostStateCursorFactory::new( - DatabaseHashedCursorFactory::new(provider_ro.tx_ref()), - &self.state_sorted, + #[cfg(feature = "metrics")] + self.metrics.record(stats); + + trace!( + target: "trie::parallel_proof", + total_targets = storage_root_targets_len, + duration = ?stats.duration(), + branches_added = stats.branches_added(), + leaves_added = stats.leaves_added(), + missed_leaves = stats.missed_leaves(), + precomputed_storage_roots = stats.precomputed_storage_roots(), + "Calculated decoded proof" ); + Ok(multiproof) + } +} + +/// Builds an account multiproof given pre-collected storage proofs. +/// +/// This is a helper function used by both `decoded_multiproof` and account workers to build +/// the account subtree proof after storage proofs have been collected. +/// +/// Returns a `DecodedMultiProof` containing the account subtree and storage proofs. +#[allow(clippy::too_many_arguments)] +pub(crate) fn build_account_multiproof_with_storage_roots( + trie_cursor_factory: C, + hashed_cursor_factory: H, + targets: &MultiProofTargets, + prefix_set: PrefixSet, + collect_branch_node_masks: bool, + multi_added_removed_keys: Option<&Arc>, + mut storage_proofs: B256Map, + missed_leaves_storage_roots: &DashMap, + tracker: &mut ParallelTrieTracker, +) -> Result +where + C: TrieCursorFactory + Clone, + H: HashedCursorFactory + Clone, +{ let accounts_added_removed_keys = - self.multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); + multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); // Create the walker. let walker = TrieWalker::<_>::state_trie( trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, - prefix_sets.account_prefix_set, + prefix_set, ) .with_added_removed_keys(accounts_added_removed_keys) .with_deletions_retained(true); @@ -256,7 +292,7 @@ where .with_added_removed_keys(accounts_added_removed_keys); let mut hash_builder = HashBuilder::default() .with_proof_retainer(retainer) - .with_updates(self.collect_branch_node_masks); + .with_updates(collect_branch_node_masks); // Initialize all storage multiproofs as empty. // Storage multiproofs for non empty tries will be overwritten if necessary. @@ -267,9 +303,8 @@ where walker, hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, ); - while let Some(account_node) = - account_node_iter.try_next().map_err(ProviderError::Database)? - { + + while let Some(account_node) = account_node_iter.try_next().map_err(ProviderError::Database)? { match account_node { TrieElement::Branch(node) => { hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); @@ -339,37 +374,22 @@ where let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; - let (branch_node_hash_masks, branch_node_tree_masks) = if self.collect_branch_node_masks { + let (branch_node_hash_masks, branch_node_tree_masks) = if collect_branch_node_masks { let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); ( updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), - updated_branch_nodes - .into_iter() - .map(|(path, node)| (path, node.tree_mask)) - .collect(), + updated_branch_nodes.into_iter().map(|(path, node)| (path, node.tree_mask)).collect(), ) } else { - (HashMap::default(), HashMap::default()) + (Default::default(), Default::default()) }; - trace!( - target: "trie::parallel_proof", - total_targets = storage_root_targets_len, - duration = ?stats.duration(), - branches_added = stats.branches_added(), - leaves_added = stats.leaves_added(), - missed_leaves = stats.missed_leaves(), - precomputed_storage_roots = stats.precomputed_storage_roots(), - "Calculated decoded proof" - ); - Ok(DecodedMultiProof { account_subtree: decoded_account_subtree, branch_node_hash_masks, branch_node_tree_masks, storages: collected_decoded_storages, }) - } } #[cfg(test)] diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 703ac0f2c91..9fbd02916ec 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -8,7 +8,11 @@ //! Individual [`ProofTaskTx`] instances manage a dedicated [`InMemoryTrieCursorFactory`] and //! [`HashedPostStateCursorFactory`], which are each backed by a database transaction. -use crate::root::ParallelStateRootError; +use crate::{ + root::ParallelStateRootError, + stats::{ParallelTrieStats, ParallelTrieTracker}, + StorageRootTargets, +}; use alloy_primitives::{ map::{B256Map, B256Set}, B256, @@ -52,7 +56,8 @@ use crate::proof_task_metrics::ProofTaskMetrics; type StorageProofResult = Result; type TrieNodeProviderResult = Result, SparseTrieError>; -type AccountMultiproofResult = Result; +type AccountMultiproofResult = + Result<(DecodedMultiProof, ParallelTrieStats), ParallelStateRootError>; /// Worker type identifier #[derive(Debug)] From ee166273f8f80b8f82bc0c09c2d55d3ed30568ea Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:27:01 +0000 Subject: [PATCH 056/144] refactor: add mapping MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The change adds .map(|(proof, _stats)| proof) to extract just the proof from the channel's tuple return value (DecodedMultiProof, ParallelTrieStats). Why it's necessary: The worker pool now returns both proof and stats as a tuple, but the function expects just DecodedMultiProof. Without this extraction, you'd get a compile error due to type mismatch. What it does: Takes (proof, stats) → returns just proof, discarding the stats (indicated by _). --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 9453fd7acb8..28f014fa9a7 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -577,9 +577,12 @@ where ) })?; - receiver.recv().map_err(|_| { + receiver + .recv() + .map_err(|_| { ParallelStateRootError::Other("Account multiproof channel closed".into()) })? + .map(|(proof, _stats)| proof) })(); let elapsed = start.elapsed(); trace!( From 7fbd172f81b7156ab3f0e66ee26479acb2774267 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:28:30 +0000 Subject: [PATCH 057/144] refactor: enhance account worker loop and storage proof collection - Updated the `account_worker_loop` to utilize a mutable `input` for better clarity and efficiency. - Introduced a new `collect_storage_proofs` function to streamline the queuing and collection of storage proofs, improving error handling and documentation. - Enhanced the tracking of storage roots and metrics during account multiproof processing. - Improved overall code organization and readability. --- .../src/tree/payload_processor/multiproof.rs | 4 +- crates/trie/parallel/src/proof_task.rs | 96 ++++++++++++++++++- 2 files changed, 93 insertions(+), 7 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 28f014fa9a7..a062793a058 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -580,8 +580,8 @@ where receiver .recv() .map_err(|_| { - ParallelStateRootError::Other("Account multiproof channel closed".into()) - })? + ParallelStateRootError::Other("Account multiproof channel closed".into()) + })? .map(|(proof, _stats)| proof) })(); let elapsed = start.elapsed(); diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 9fbd02916ec..7076825fb3b 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -367,7 +367,7 @@ fn account_worker_loop( let (trie_cursor_factory, hashed_cursor_factory) = proof_tx.create_factories(); let mut account_proofs_processed = 0u64; - while let Ok(AccountMultiproofJob { input, result_sender }) = work_rx.recv() { + while let Ok(AccountMultiproofJob { mut input, result_sender }) = work_rx.recv() { trace!( target: "trie::proof_task", worker_id, @@ -376,6 +376,18 @@ fn account_worker_loop( ); let proof_start = Instant::now(); + let mut tracker = ParallelTrieTracker::default(); + + let storage_root_targets_len = StorageRootTargets::new( + input + .prefix_sets + .account_prefix_set + .iter() + .map(|nibbles| B256::from_slice(&nibbles.pack())), + input.prefix_sets.storage_prefix_sets.clone(), + ) + .len(); + tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); let storage_proofs = match collect_storage_proofs( &proof_tx, @@ -395,14 +407,13 @@ fn account_worker_loop( // Use the missed leaves cache passed from the multiproof manager let missed_leaves_storage_roots = &input.missed_leaves_storage_roots; - // Create tracker for metrics (workers don't use stats, but function requires it) - let mut tracker = crate::stats::ParallelTrieTracker::default(); + let account_prefix_set = std::mem::take(&mut input.prefix_sets.account_prefix_set); let result = crate::proof::build_account_multiproof_with_storage_roots( trie_cursor_factory.clone(), hashed_cursor_factory.clone(), &input.targets, - &input.prefix_sets.account_prefix_set, + account_prefix_set, input.collect_branch_node_masks, input.multi_added_removed_keys.as_ref(), storage_proofs, @@ -411,6 +422,8 @@ fn account_worker_loop( ); let proof_elapsed = proof_start.elapsed(); + let stats = tracker.finish(); + let result = result.map(|proof| (proof, stats)); account_proofs_processed += 1; if result_sender.send(result).is_err() { @@ -439,6 +452,79 @@ fn account_worker_loop( ); } +/// Collects storage proofs for all accounts in the targets by queueing to storage workers. +/// +/// Queues storage proof tasks to the storage worker pool and collects results. +/// Propagates errors up if queuing fails, workers return errors, or channels are dropped. +/// No inline fallback - fails fast if storage workers are unavailable. +fn collect_storage_proofs( + _proof_tx: &ProofTaskTx, + storage_proof_handle: &ProofTaskManagerHandle, + targets: &MultiProofTargets, + prefix_sets: &TriePrefixSets, + with_branch_node_masks: bool, + multi_added_removed_keys: Option>, +) -> Result, ParallelStateRootError> +where + Tx: DbTx, +{ + let mut storage_proofs = B256Map::with_capacity_and_hasher(targets.len(), Default::default()); + let mut pending = Vec::with_capacity(targets.len()); // (address, receiver) + + // Queue all storage proofs to worker pool + for (hashed_address, target_slots) in targets.iter() { + let prefix_set = + prefix_sets.storage_prefix_sets.get(hashed_address).cloned().unwrap_or_default(); + + // Always queue a storage proof so we obtain the storage root even when no slots are + // requested. + let input = StorageProofInput::new( + *hashed_address, + prefix_set, + target_slots.clone(), + with_branch_node_masks, + multi_added_removed_keys.clone(), + ); + + let (sender, receiver) = channel(); + + // If queuing fails, propagate error up (no fallback) + storage_proof_handle.queue_task(ProofTaskKind::StorageProof(input, sender)).map_err( + |_| { + ParallelStateRootError::Other(format!( + "Failed to queue storage proof for {}: storage worker pool unavailable", + hashed_address + )) + }, + )?; + + pending.push((*hashed_address, receiver)); + } + + // Collect all results + for (hashed_address, receiver) in pending { + // If receiving fails or worker returns error, propagate up (no fallback) + let proof = receiver + .recv() + .map_err(|_| { + ParallelStateRootError::Other(format!( + "Storage proof channel dropped for {}: worker died or pool shutdown", + hashed_address + )) + })? + .map_err(|e| { + ParallelStateRootError::Other(format!( + "Storage proof computation failed for {}: {}", + hashed_address, e + )) + })?; + + storage_proofs.insert(hashed_address, proof); + } + + Ok(storage_proofs) +} + impl ProofTaskManager where Factory: DatabaseProviderFactory, @@ -751,7 +837,7 @@ where target: "trie::proof_task", "Account multiproof dispatched to worker pool" ); - } + } Err(crossbeam_channel::SendError(job)) => { tracing::error!( target: "trie::proof_task", From 0884e15357875316f037fd71e539734766222a7f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:30:52 +0000 Subject: [PATCH 058/144] refactor: improve account multiproof processing and storage proof handling - Enhanced the account multiproof generation logic by refining the handling of storage proofs and missed leaves. - Streamlined the walker and hash builder setup for better clarity and efficiency. - Improved error handling for storage proof retrieval and added detailed tracing for missing leaves. - Updated documentation to reflect changes in the multiproof generation workflow. --- crates/trie/parallel/src/proof.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 82ada097467..887a0fe8d25 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -404,7 +404,9 @@ mod tests { use rand::Rng; use reth_primitives_traits::{Account, StorageEntry}; use reth_provider::{ - providers::ConsistentDbView, test_utils::create_test_provider_factory, HashingWriter, + providers::ConsistentDbView, + test_utils::{create_test_provider_factory, MockNodeTypesWithDB}, + HashingWriter, ProviderFactory, }; use reth_trie::proof::Proof; use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; @@ -478,7 +480,7 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view.clone(), task_ctx, 1, 1, 1) + ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1, 1) .unwrap(); let proof_task_handle = proof_task.handle(); @@ -486,7 +488,8 @@ mod tests { // after we compute the state root let join_handle = rt.spawn_blocking(move || proof_task.run()); - let parallel_result = ParallelProof::new( + type Factory = ProviderFactory; + let parallel_result = ParallelProof::::new( Default::default(), Default::default(), Default::default(), From d76cc8618272db317ec26206ce310689c8184297 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:34:37 +0000 Subject: [PATCH 059/144] refactor: optimize account multiproof processing and error handling - Refined the logic for handling account nodes during multiproof generation, improving clarity and efficiency. - Enhanced the management of storage proofs and missed leaves, ensuring better tracking and error handling. - Streamlined the setup of the walker and hash builder, reducing unnecessary complexity. - Updated comments for improved documentation and understanding of the multiproof generation workflow. --- crates/trie/parallel/src/proof.rs | 188 ++++++++++++++---------------- 1 file changed, 90 insertions(+), 98 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 887a0fe8d25..2437d009fb6 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -273,123 +273,116 @@ where C: TrieCursorFactory + Clone, H: HashedCursorFactory + Clone, { - let accounts_added_removed_keys = + let accounts_added_removed_keys = multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); - // Create the walker. - let walker = TrieWalker::<_>::state_trie( - trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, + // Create the walker. + let walker = TrieWalker::<_>::state_trie( + trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, prefix_set, - ) - .with_added_removed_keys(accounts_added_removed_keys) - .with_deletions_retained(true); - - // Create a hash builder to rebuild the root node since it is not available in the database. - let retainer = targets - .keys() - .map(Nibbles::unpack) - .collect::() - .with_added_removed_keys(accounts_added_removed_keys); - let mut hash_builder = HashBuilder::default() - .with_proof_retainer(retainer) + ) + .with_added_removed_keys(accounts_added_removed_keys) + .with_deletions_retained(true); + + // Create a hash builder to rebuild the root node since it is not available in the database. + let retainer = targets + .keys() + .map(Nibbles::unpack) + .collect::() + .with_added_removed_keys(accounts_added_removed_keys); + let mut hash_builder = HashBuilder::default() + .with_proof_retainer(retainer) .with_updates(collect_branch_node_masks); - // Initialize all storage multiproofs as empty. - // Storage multiproofs for non empty tries will be overwritten if necessary. - let mut collected_decoded_storages: B256Map = - targets.keys().map(|key| (*key, DecodedStorageMultiProof::empty())).collect(); - let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); - let mut account_node_iter = TrieNodeIter::state_trie( - walker, - hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, - ); + // Initialize all storage multiproofs as empty. + // Storage multiproofs for non empty tries will be overwritten if necessary. + let mut collected_decoded_storages: B256Map = + targets.keys().map(|key| (*key, DecodedStorageMultiProof::empty())).collect(); + let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); + let mut account_node_iter = TrieNodeIter::state_trie( + walker, + hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, + ); while let Some(account_node) = account_node_iter.try_next().map_err(ProviderError::Database)? { - match account_node { - TrieElement::Branch(node) => { - hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); - } - TrieElement::Leaf(hashed_address, account) => { - let root = match storage_proof_receivers.remove(&hashed_address) { - Some(rx) => { - let decoded_storage_multiproof = rx.recv().map_err(|e| { - ParallelStateRootError::StorageRoot(StorageRootError::Database( - DatabaseError::Other(format!( - "channel closed for {hashed_address}: {e}" - )), - )) - })??; - let root = decoded_storage_multiproof.root; - collected_decoded_storages - .insert(hashed_address, decoded_storage_multiproof); - root + match account_node { + TrieElement::Branch(node) => { + hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); + } + TrieElement::Leaf(hashed_address, account) => { + let root = match storage_proofs.remove(&hashed_address) { + Some(proof) => { + let root = proof.root; + if let Some(entry) = collected_decoded_storages.get_mut(&hashed_address) { + *entry = proof; } - // Since we do not store all intermediate nodes in the database, there might - // be a possibility of re-adding a non-modified leaf to the hash builder. - None => { - tracker.inc_missed_leaves(); - - match self.missed_leaves_storage_roots.entry(hashed_address) { - dashmap::Entry::Occupied(occ) => *occ.get(), - dashmap::Entry::Vacant(vac) => { - let root = StorageProof::new_hashed( - trie_cursor_factory.clone(), - hashed_cursor_factory.clone(), - hashed_address, - ) - .with_prefix_set_mut(Default::default()) - .storage_multiproof( - targets.get(&hashed_address).cloned().unwrap_or_default(), - ) - .map_err(|e| { - ParallelStateRootError::StorageRoot( - StorageRootError::Database(DatabaseError::Other( - e.to_string(), - )), - ) - })? - .root; - vac.insert(root); - root - } + root + } + // Since we do not store all intermediate nodes in the database, there might + // be a possibility of re-adding a non-modified leaf to the hash builder. + None => { + tracker.inc_missed_leaves(); + + match missed_leaves_storage_roots.entry(hashed_address) { + dashmap::Entry::Occupied(occ) => *occ.get(), + dashmap::Entry::Vacant(vac) => { + let root = StorageProof::new_hashed( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + hashed_address, + ) + .with_prefix_set_mut(Default::default()) + .storage_multiproof( + targets.get(&hashed_address).cloned().unwrap_or_default(), + ) + .map_err(|e| { + ParallelStateRootError::StorageRoot(StorageRootError::Database( + DatabaseError::Other(e.to_string()), + )) + })? + .root; + + vac.insert(root); + root } } - }; + } + }; - // Encode account - account_rlp.clear(); - let account = account.into_trie_account(root); - account.encode(&mut account_rlp as &mut dyn BufMut); + // Encode account + account_rlp.clear(); + let account = account.into_trie_account(root); + account.encode(&mut account_rlp as &mut dyn BufMut); - hash_builder.add_leaf(Nibbles::unpack(hashed_address), &account_rlp); - } + hash_builder.add_leaf(Nibbles::unpack(hashed_address), &account_rlp); } } - let _ = hash_builder.root(); + } - let stats = tracker.finish(); - #[cfg(feature = "metrics")] - self.metrics.record(stats); + // Insert storage proofs for accounts not encountered during trie walk. + collected_decoded_storages.extend(storage_proofs); - let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); - let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; + let _ = hash_builder.root(); + + let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); + let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; let (branch_node_hash_masks, branch_node_tree_masks) = if collect_branch_node_masks { - let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); - ( - updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), + let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); + ( + updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), updated_branch_nodes.into_iter().map(|(path, node)| (path, node.tree_mask)).collect(), - ) - } else { + ) + } else { (Default::default(), Default::default()) - }; + }; - Ok(DecodedMultiProof { - account_subtree: decoded_account_subtree, - branch_node_hash_masks, - branch_node_tree_masks, - storages: collected_decoded_storages, - }) + Ok(DecodedMultiProof { + account_subtree: decoded_account_subtree, + branch_node_hash_masks, + branch_node_tree_masks, + storages: collected_decoded_storages, + }) } #[cfg(test)] @@ -480,8 +473,7 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1, 1) - .unwrap(); + ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1, 1).unwrap(); let proof_task_handle = proof_task.handle(); // keep the join handle around to make sure it does not return any errors From 8e45538471d2d7e2cc273cd2db47ab9d0cd0dc0c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 11:56:05 +0000 Subject: [PATCH 060/144] refactor: enhance account worker management and task routing - Updated the `ProofTaskManager` to streamline the handling of account multiproof and blinded account node tasks, improving clarity and efficiency. - Refined the `account_worker_loop` to better manage job processing and error handling, ensuring graceful degradation in case of worker panics. - Consolidated the job types into a single `AccountWorkerJob` enum for improved organization and readability. - Enhanced documentation to reflect changes in task management and worker operations. --- crates/trie/parallel/src/proof_task.rs | 47 ++++++++++++++++++++------ 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 7076825fb3b..8ade23e23d0 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -448,7 +448,8 @@ fn account_worker_loop( target: "trie::proof_task", worker_id, account_proofs_processed, - "Account multiproof worker shutting down" + account_nodes_processed, + "Account worker shutting down" ); } @@ -743,10 +744,9 @@ where /// - **Storage Trie Operations** (`StorageProof` and `BlindedStorageNode`): Routed to /// pre-spawned storage worker pool via unbounded channel. Returns error if workers are /// disconnected (e.g., all workers panicked). - /// - **Account Multiproof Operations** (`AccountMultiproof`): Routed to pre-spawned account - /// worker pool via unbounded channel. Returns error if workers are disconnected. - /// - **Account Trie Operations** (`BlindedAccountNode`): Queued for on-demand execution via - /// `pending_tasks`. + /// - **Account Trie Operations** (`AccountMultiproof` and `BlindedAccountNode`): Routed to + /// pre-spawned account worker pool via unbounded channel. Returns error if workers are + /// disconnected. /// /// # Shutdown /// @@ -1093,23 +1093,50 @@ pub struct AccountMultiproofInput { pub missed_leaves_storage_roots: Arc>, } -/// Internal job for account multiproof workers. +/// Internal message for account workers. +/// +/// This is NOT exposed publicly. External callers use `ProofTaskKind::AccountMultiproof` or +/// `ProofTaskKind::BlindedAccountNode` which are routed through the manager's `std::mpsc` channel. #[derive(Debug)] -struct AccountMultiproofJob { +enum AccountWorkerJob { + /// Account multiproof computation request + AccountMultiproof { /// Account multiproof input parameters input: AccountMultiproofInput, /// Channel to send result back to original caller result_sender: Sender, + }, + /// Blinded account node retrieval request + BlindedAccountNode { + /// Path to the account node + path: Nibbles, + /// Channel to send result back to original caller + result_sender: Sender, + }, } -impl AccountMultiproofJob { +impl AccountWorkerJob { /// Sends an error back to the caller when worker pool is unavailable. /// /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was /// dropped. fn send_worker_unavailable_error(&self) -> Result<(), ()> { - let error = ParallelStateRootError::Other("Account worker pool unavailable".to_string()); - self.result_sender.send(Err(error)).map_err(|_| ()) + match self { + Self::AccountMultiproof { result_sender, .. } => { + let error = + ParallelStateRootError::Other("Account worker pool unavailable".to_string()); + result_sender.send(Err(error)).map_err(|_| ()) + } + Self::BlindedAccountNode { result_sender, .. } => { + let error = SparseTrieError::from(SparseTrieErrorKind::Other(Box::new( + std::io::Error::new( + std::io::ErrorKind::BrokenPipe, + "Account worker pool unavailable", + ), + ))); + result_sender.send(Err(error)).map_err(|_| ()) + } + } } } From 8be550a54201c512698f22345a3b78fb8faaf100 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:00:26 +0000 Subject: [PATCH 061/144] refactor: improve task management in ProofTaskManager - Refactored the `ProofTaskManager` to enhance the handling of account multiproof and blinded account node tasks, consolidating job types into a single `AccountWorkerJob` enum for better organization. - Updated the `account_worker_loop` to streamline job processing and error handling, ensuring graceful degradation during worker panics. - Removed unnecessary parameters and improved documentation to reflect changes in task management and worker operations. --- .../tree/src/tree/payload_processor/mod.rs | 3 +- .../src/tree/payload_processor/multiproof.rs | 1 - crates/trie/parallel/src/proof.rs | 2 +- crates/trie/parallel/src/proof_task.rs | 408 +++++++++--------- 4 files changed, 205 insertions(+), 209 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 378c0803376..c60b73ce514 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -202,14 +202,13 @@ where state_root_config.state_sorted.clone(), state_root_config.prefix_sets.clone(), ); - let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let storage_worker_count = config.storage_worker_count(); let account_worker_count = config.account_worker_count(); + let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), state_root_config.consistent_view.clone(), task_ctx, - max_proof_task_concurrency, storage_worker_count, account_worker_count, ) { diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index a062793a058..dfa33838e4b 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1267,7 +1267,6 @@ mod tests { task_ctx, 1, 1, - 1, ) .expect("Failed to create ProofTaskManager"); let channel = channel(); diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 2437d009fb6..e17c38da1a7 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -473,7 +473,7 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1, 1).unwrap(); + ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1).unwrap(); let proof_task_handle = proof_task.handle(); // keep the join handle around to make sure it does not return any errors diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 8ade23e23d0..d2d0edd55e0 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -40,7 +40,6 @@ use reth_trie_common::{ use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; use reth_trie_sparse::provider::{RevealedNode, TrieNodeProvider, TrieNodeProviderFactory}; use std::{ - collections::VecDeque, sync::{ atomic::{AtomicUsize, Ordering}, mpsc::{channel, Receiver, SendError, Sender}, @@ -133,18 +132,15 @@ impl StorageWorkerJob { /// /// This manager handles three distinct execution paths: /// -/// 1. **Worker Pools** (for storage and account operations): +/// **Worker Pools** (for all trie operations): /// - Pre-spawned workers with dedicated long-lived transactions /// - **Storage pool**: Handles `StorageProof` and `BlindedStorageNode` requests -/// - **Account pool**: Handles `AccountMultiproof` requests, delegates to storage pool +/// - **Account pool**: Handles `AccountMultiproof` and `BlindedAccountNode` requests, delegates +/// storage proof computation to storage pool /// - Tasks queued via crossbeam unbounded channels /// - Workers continuously process without transaction overhead /// - Returns error if worker pool is unavailable (all workers panicked) /// -/// 2. **On-Demand Execution** (for blinded account node operations): -/// - Lazy transaction creation for `BlindedAccountNode` requests -/// - Transactions returned to pool after use for reuse -/// /// # Public Interface /// /// The public interface through `ProofTaskManagerHandle` allows external callers to: @@ -162,38 +158,16 @@ pub struct ProofTaskManager { storage_worker_count: usize, /// Sender for account worker jobs to worker pool. - account_work_tx: CrossbeamSender, + account_work_tx: CrossbeamSender, /// Number of account workers successfully spawned. account_worker_count: usize, - /// Max number of database transactions to create for on-demand account trie operations. - max_concurrency: usize, - - /// Number of database transactions created for on-demand operations. - total_transactions: usize, - - /// Proof tasks pending execution (account trie operations only). - pending_tasks: VecDeque, - - /// The proof task transactions, containing owned cursor factories that are reused for proof - /// calculation (account trie operations only). - proof_task_txs: Vec>>, - - /// Consistent view provider used for creating transactions on-demand. - view: ConsistentDbView, - - /// Proof task context shared across all proof tasks. - task_ctx: ProofTaskCtx, - - /// The underlying handle from which to spawn proof tasks. - executor: Handle, - /// Receives proof task requests from [`ProofTaskManagerHandle`]. proof_task_rx: Receiver>>, - /// Internal channel for on-demand tasks to return transactions after use. - tx_sender: Sender>>, + /// Sender for creating handles that can queue tasks. + proof_task_tx: Sender>>, /// The number of active handles. /// @@ -350,9 +324,32 @@ fn storage_worker_loop( // TODO: Refactor this with storage_worker_loop. ProofTaskManager should be removed in the following // pr and `MultiproofManager` should be used instead to dispatch jobs directly. +/// Worker loop for account trie operations. +/// +/// # Lifecycle +/// +/// Each worker: +/// 1. Receives `AccountWorkerJob` from crossbeam unbounded channel +/// 2. Computes result using its dedicated long-lived transaction +/// 3. Sends result directly to original caller via `std::mpsc` +/// 4. Repeats until channel closes (graceful shutdown) +/// +/// # Transaction Reuse +/// +/// Reuses the same transaction and cursor factories across multiple operations +/// to avoid transaction creation and cursor factory setup overhead. +/// +/// # Panic Safety +/// +/// If this function panics, the worker thread terminates but other workers +/// continue operating and the system degrades gracefully. +/// +/// # Shutdown +/// +/// Worker shuts down when the crossbeam channel closes (all senders dropped). fn account_worker_loop( proof_tx: ProofTaskTx, - work_rx: CrossbeamReceiver, + work_rx: CrossbeamReceiver, storage_proof_handle: ProofTaskManagerHandle, worker_id: usize, ) where @@ -361,87 +358,135 @@ fn account_worker_loop( tracing::debug!( target: "trie::proof_task", worker_id, - "Account multiproof worker started" + "Account worker started" ); + // Create factories once at worker startup to avoid recreation overhead. let (trie_cursor_factory, hashed_cursor_factory) = proof_tx.create_factories(); + + // Create blinded provider factory once for all blinded node requests + let blinded_provider_factory = ProofTrieNodeProviderFactory::new( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + proof_tx.task_ctx.prefix_sets.clone(), + ); + let mut account_proofs_processed = 0u64; + let mut account_nodes_processed = 0u64; - while let Ok(AccountMultiproofJob { mut input, result_sender }) = work_rx.recv() { - trace!( - target: "trie::proof_task", - worker_id, - targets = input.targets.len(), - "Processing account multiproof" - ); + while let Ok(job) = work_rx.recv() { + match job { + AccountWorkerJob::AccountMultiproof { mut input, result_sender } => { + trace!( + target: "trie::proof_task", + worker_id, + targets = input.targets.len(), + "Processing account multiproof" + ); - let proof_start = Instant::now(); - let mut tracker = ParallelTrieTracker::default(); - - let storage_root_targets_len = StorageRootTargets::new( - input - .prefix_sets - .account_prefix_set - .iter() - .map(|nibbles| B256::from_slice(&nibbles.pack())), - input.prefix_sets.storage_prefix_sets.clone(), - ) - .len(); - tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); - - let storage_proofs = match collect_storage_proofs( - &proof_tx, - &storage_proof_handle, - &input.targets, - &input.prefix_sets, - input.collect_branch_node_masks, - input.multi_added_removed_keys.clone(), - ) { - Ok(proofs) => proofs, - Err(error) => { - let _ = result_sender.send(Err(error)); - continue; + let proof_start = Instant::now(); + let mut tracker = ParallelTrieTracker::default(); + + let storage_root_targets_len = StorageRootTargets::new( + input + .prefix_sets + .account_prefix_set + .iter() + .map(|nibbles| B256::from_slice(&nibbles.pack())), + input.prefix_sets.storage_prefix_sets.clone(), + ) + .len(); + tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); + + let storage_proofs = match collect_storage_proofs( + &proof_tx, + &storage_proof_handle, + &input.targets, + &input.prefix_sets, + input.collect_branch_node_masks, + input.multi_added_removed_keys.clone(), + ) { + Ok(proofs) => proofs, + Err(error) => { + let _ = result_sender.send(Err(error)); + continue; + } + }; + + // Use the missed leaves cache passed from the multiproof manager + let missed_leaves_storage_roots = &input.missed_leaves_storage_roots; + + let account_prefix_set = std::mem::take(&mut input.prefix_sets.account_prefix_set); + + let result = crate::proof::build_account_multiproof_with_storage_roots( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + &input.targets, + account_prefix_set, + input.collect_branch_node_masks, + input.multi_added_removed_keys.as_ref(), + storage_proofs, + missed_leaves_storage_roots, + &mut tracker, + ); + + let proof_elapsed = proof_start.elapsed(); + let stats = tracker.finish(); + let result = result.map(|proof| (proof, stats)); + account_proofs_processed += 1; + + if result_sender.send(result).is_err() { + tracing::debug!( + target: "trie::proof_task", + worker_id, + account_proofs_processed, + "Account multiproof receiver dropped, discarding result" + ); + } + + trace!( + target: "trie::proof_task", + worker_id, + proof_time_us = proof_elapsed.as_micros(), + total_processed = account_proofs_processed, + "Account multiproof completed" + ); } - }; - - // Use the missed leaves cache passed from the multiproof manager - let missed_leaves_storage_roots = &input.missed_leaves_storage_roots; - - let account_prefix_set = std::mem::take(&mut input.prefix_sets.account_prefix_set); - - let result = crate::proof::build_account_multiproof_with_storage_roots( - trie_cursor_factory.clone(), - hashed_cursor_factory.clone(), - &input.targets, - account_prefix_set, - input.collect_branch_node_masks, - input.multi_added_removed_keys.as_ref(), - storage_proofs, - missed_leaves_storage_roots, - &mut tracker, - ); - let proof_elapsed = proof_start.elapsed(); - let stats = tracker.finish(); - let result = result.map(|proof| (proof, stats)); - account_proofs_processed += 1; + AccountWorkerJob::BlindedAccountNode { path, result_sender } => { + trace!( + target: "trie::proof_task", + worker_id, + ?path, + "Processing blinded account node" + ); - if result_sender.send(result).is_err() { - tracing::debug!( - target: "trie::proof_task", - worker_id, - account_proofs_processed, - "Account multiproof receiver dropped, discarding result" - ); - } + let start = Instant::now(); + let result = blinded_provider_factory.account_node_provider().trie_node(&path); + let elapsed = start.elapsed(); - trace!( - target: "trie::proof_task", - worker_id, - proof_time_us = proof_elapsed.as_micros(), - total_processed = account_proofs_processed, - "Account multiproof completed" - ); + account_nodes_processed += 1; + + if result_sender.send(result).is_err() { + tracing::debug!( + target: "trie::proof_task", + worker_id, + ?path, + account_nodes_processed, + "Blinded account node receiver dropped, discarding result" + ); + } + + trace!( + target: "trie::proof_task", + worker_id, + ?path, + node_time_us = elapsed.as_micros(), + total_processed = account_nodes_processed, + "Blinded account node completed" + ); + } + } } tracing::debug!( @@ -532,31 +577,28 @@ where { /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. /// - /// The `storage_worker_count` determines how many storage workers to spawn, - /// `account_worker_count` determines how many account workers to spawn, and - /// `max_concurrency` determines the limit for on-demand operations (blinded account nodes). + /// The `storage_worker_count` determines how many storage workers to spawn, and + /// `account_worker_count` determines how many account workers to spawn. /// Returns an error if the underlying provider fails to create the transactions required for /// spawning workers. pub fn new( executor: Handle, view: ConsistentDbView, task_ctx: ProofTaskCtx, - max_concurrency: usize, storage_worker_count: usize, account_worker_count: usize, ) -> ProviderResult { - let (tx_sender, proof_task_rx) = channel(); + let (proof_task_tx, proof_task_rx) = channel(); // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. let (storage_work_tx, storage_work_rx) = unbounded::(); - let (account_work_tx, account_work_rx) = unbounded::(); + let (account_work_tx, account_work_rx) = unbounded::(); tracing::info!( target: "trie::proof_task", storage_worker_count, account_worker_count, - max_concurrency, "Initializing storage and account worker pools with unbounded queues" ); @@ -583,10 +625,10 @@ where account_work_rx, WorkerType::Account, { - let tx_sender = tx_sender.clone(); + let task_tx = proof_task_tx.clone(); move |proof_tx, work_rx, worker_id| { let storage_handle = ProofTaskManagerHandle::new( - tx_sender.clone(), + task_tx.clone(), worker_handles_counter.clone(), ); account_worker_loop(proof_tx, work_rx, storage_handle, worker_id) @@ -599,16 +641,8 @@ where storage_worker_count: spawned_storage_workers, account_work_tx, account_worker_count: spawned_account_workers, - max_concurrency, - total_transactions: 0, - pending_tasks: VecDeque::new(), - proof_task_txs: Vec::with_capacity(max_concurrency), /* used for on-demand account - * trie operations */ - view, - task_ctx, - executor, proof_task_rx, - tx_sender, + proof_task_tx, active_handles: Arc::new(AtomicUsize::new(0)), #[cfg(feature = "metrics")] @@ -618,7 +652,7 @@ where /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. pub fn handle(&self) -> ProofTaskManagerHandle> { - ProofTaskManagerHandle::new(self.tx_sender.clone(), self.active_handles.clone()) + ProofTaskManagerHandle::new(self.proof_task_tx.clone(), self.active_handles.clone()) } /// Spawns a pool of workers with dedicated database transactions. @@ -682,62 +716,7 @@ impl ProofTaskManager where Factory: DatabaseProviderFactory + 'static, { - /// Inserts the task into the pending tasks queue. - pub fn queue_proof_task(&mut self, task: ProofTaskKind) { - self.pending_tasks.push_back(task); - } - - /// Gets either the next available transaction, or creates a new one if all are in use and the - /// total number of transactions created is less than the max concurrency. - pub fn get_or_create_tx(&mut self) -> ProviderResult>>> { - if let Some(proof_task_tx) = self.proof_task_txs.pop() { - return Ok(Some(proof_task_tx)); - } - - // if we can create a new tx within our concurrency limits, create one on-demand - if self.total_transactions < self.max_concurrency { - let provider_ro = self.view.provider_ro()?; - let tx = provider_ro.into_tx(); - self.total_transactions += 1; - return Ok(Some(ProofTaskTx::new(tx, self.task_ctx.clone(), self.total_transactions))); - } - - Ok(None) - } - - /// Spawns the next queued proof task on the executor with the given input, if there are any - /// transactions available. - /// - /// This will return an error if a transaction must be created on-demand and the consistent view - /// provider fails. - pub fn try_spawn_next(&mut self) -> ProviderResult<()> { - let Some(task) = self.pending_tasks.pop_front() else { return Ok(()) }; - - let Some(proof_task_tx) = self.get_or_create_tx()? else { - // if there are no txs available, requeue the proof task - self.pending_tasks.push_front(task); - return Ok(()) - }; - - let tx_sender = self.tx_sender.clone(); - - self.executor.spawn_blocking(move || match task { - ProofTaskKind::BlindedAccountNode(path, sender) => { - proof_task_tx.blinded_account_node(path, sender, tx_sender); - } - // Worker pool operations should never reach here as they're routed to their respective - // worker pools - ProofTaskKind::AccountMultiproof(_, _) | - ProofTaskKind::BlindedStorageNode(_, _, _) | - ProofTaskKind::StorageProof(_, _) => { - unreachable!("Worker pool operations should be routed to their respective pools") - } - }); - - Ok(()) - } - - /// Loops, managing the proof tasks, and sending new tasks to the executor. + /// Loops, managing the proof tasks, routing them to the appropriate worker pools. /// /// # Task Routing /// @@ -818,20 +797,46 @@ where } } - ProofTaskKind::BlindedAccountNode(_, _) => { - // Route account trie operations to pending_tasks + ProofTaskKind::BlindedAccountNode(path, sender) => { #[cfg(feature = "metrics")] { self.metrics.account_nodes += 1; } - self.queue_proof_task(task); + + match self.account_work_tx.send( + AccountWorkerJob::BlindedAccountNode { + path, + result_sender: sender, + }, + ) { + Ok(_) => { + tracing::trace!( + target: "trie::proof_task", + ?path, + "Blinded account node dispatched to worker pool" + ); + } + Err(crossbeam_channel::SendError(job)) => { + tracing::warn!( + target: "trie::proof_task", + account_worker_count = self.account_worker_count, + ?path, + "Worker pool disconnected, cannot process blinded account node" + ); + + // Send error back to caller + let _ = job.send_worker_unavailable_error(); + } + } } ProofTaskKind::AccountMultiproof(input, sender) => { - match self - .account_work_tx - .send(AccountMultiproofJob { input, result_sender: sender }) - { + match self.account_work_tx.send( + AccountWorkerJob::AccountMultiproof { + input, + result_sender: sender, + }, + ) { Ok(_) => { tracing::trace!( target: "trie::proof_task", @@ -851,10 +856,6 @@ where } } }, - ProofTaskMessage::Transaction(tx) => { - // Return transaction to pending_tasks pool - self.proof_task_txs.push(tx); - } ProofTaskMessage::Terminate => { // Drop worker channels to signal workers to shut down drop(self.storage_work_tx); @@ -873,15 +874,15 @@ where return Ok(()) } + ProofTaskMessage::_Phantom(_) => { + unreachable!("_Phantom variant should never be constructed") + } } } // All senders are disconnected, so we can terminate // However this should never happen, as this struct stores a sender Err(_) => return Ok(()), }; - - // Try spawning pending account trie tasks - self.try_spawn_next()?; } } } @@ -1004,7 +1005,7 @@ where self, path: Nibbles, result_sender: Sender, - tx_sender: Sender>, + _tx_sender: Sender>, ) { trace!( target: "trie::proof_task", @@ -1038,8 +1039,7 @@ where ); } - // send the tx back - let _ = tx_sender.send(ProofTaskMessage::Transaction(self)); + // Transaction is no longer returned - BlindedAccountNode now uses worker pool } } @@ -1101,10 +1101,10 @@ pub struct AccountMultiproofInput { enum AccountWorkerJob { /// Account multiproof computation request AccountMultiproof { - /// Account multiproof input parameters - input: AccountMultiproofInput, - /// Channel to send result back to original caller - result_sender: Sender, + /// Account multiproof input parameters + input: AccountMultiproofInput, + /// Channel to send result back to original caller + result_sender: Sender, }, /// Blinded account node retrieval request BlindedAccountNode { @@ -1170,10 +1170,10 @@ impl ProofTaskCtx { pub enum ProofTaskMessage { /// A request to queue a proof task. QueueTask(ProofTaskKind), - /// A returned database transaction. - Transaction(ProofTaskTx), /// A request to terminate the proof task manager. Terminate, + #[doc(hidden)] + _Phantom(std::marker::PhantomData), } /// Proof task kind. @@ -1319,13 +1319,11 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 1, 5, 3).unwrap(); + let manager = ProofTaskManager::new(handle.clone(), view, ctx, 5, 3).unwrap(); // With storage_worker_count=5, we get exactly 5 storage workers assert_eq!(manager.storage_worker_count, 5); // With account_worker_count=3, we get exactly 3 account workers assert_eq!(manager.account_worker_count, 3); - // max_concurrency=1 is for on-demand operations only - assert_eq!(manager.max_concurrency, 1); drop(manager); task::yield_now().await; From 087e875b4b706143dee2d567fc6b0df42750016c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:05:25 +0000 Subject: [PATCH 062/144] refactor: remove unused blinded account node retrieval function - Eliminated the `blinded_account_node` function from `proof_task.rs` to streamline the c - New approach (worker pool): - BlindedAccountNode requests are sent to the account worker pool via AccountWorkerJob::BlindedAccountNode (crates/trie/parallel/src/proof_task.rs:456-458) - Pre-spawned workers with long-lived transactions handle these requests - Routed through the manager at crates/trie/parallel/src/proof_task.rs:800-809 --- crates/trie/parallel/src/proof_task.rs | 41 -------------------------- 1 file changed, 41 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index d2d0edd55e0..ec8a38d0844 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1000,47 +1000,6 @@ where decoded_result } - /// Retrieves blinded account node by path. - fn blinded_account_node( - self, - path: Nibbles, - result_sender: Sender, - _tx_sender: Sender>, - ) { - trace!( - target: "trie::proof_task", - ?path, - "Starting blinded account node retrieval" - ); - - let (trie_cursor_factory, hashed_cursor_factory) = self.create_factories(); - - let blinded_provider_factory = ProofTrieNodeProviderFactory::new( - trie_cursor_factory, - hashed_cursor_factory, - self.task_ctx.prefix_sets.clone(), - ); - - let start = Instant::now(); - let result = blinded_provider_factory.account_node_provider().trie_node(&path); - trace!( - target: "trie::proof_task", - ?path, - elapsed = ?start.elapsed(), - "Completed blinded account node retrieval" - ); - - if let Err(error) = result_sender.send(result) { - tracing::error!( - target: "trie::proof_task", - ?path, - ?error, - "Failed to send blinded account node result" - ); - } - - // Transaction is no longer returned - BlindedAccountNode now uses worker pool - } } /// This represents an input for a storage proof. From 0646f14f0955a18b419b279eb64c4950cfd7dab3 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:06:08 +0000 Subject: [PATCH 063/144] fmt --- crates/trie/parallel/src/proof_task.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index ec8a38d0844..a35fa7e0524 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -999,7 +999,6 @@ where decoded_result } - } /// This represents an input for a storage proof. From 886cb6fc019dfea6754820006f3ac0a72f3b3744 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:06:48 +0000 Subject: [PATCH 064/144] changed to error --- crates/trie/parallel/src/proof_task.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index a35fa7e0524..95833c95d0d 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -783,7 +783,7 @@ where ); } Err(crossbeam_channel::SendError(job)) => { - tracing::warn!( + tracing::error!( target: "trie::proof_task", storage_worker_count = self.storage_worker_count, ?account, @@ -817,7 +817,7 @@ where ); } Err(crossbeam_channel::SendError(job)) => { - tracing::warn!( + tracing::error!( target: "trie::proof_task", account_worker_count = self.account_worker_count, ?path, From e51707a62c604546c14adb405f162a27f08cb096 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:56:25 +0000 Subject: [PATCH 065/144] refactor: simplify error handling in proof task management - Removed the `send_worker_unavailable_error` method from both `StorageWorkerJob` and `AccountWorkerJob` enums to streamline error handling. - Updated task dispatching logic in `ProofTaskMessage` to ensure that errors are handled more directly without the need for separate error sending methods. - Enhanced clarity and maintainability of the code by reducing redundancy in error management. --- crates/trie/parallel/src/proof_task.rs | 237 ++++++++----------------- 1 file changed, 76 insertions(+), 161 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 95833c95d0d..fec68df3917 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -20,7 +20,7 @@ use alloy_primitives::{ use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use dashmap::DashMap; use reth_db_api::transaction::DbTx; -use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; +use reth_execution_errors::SparseTrieError; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, FactoryTx, ProviderResult, @@ -100,32 +100,6 @@ enum StorageWorkerJob { }, } -impl StorageWorkerJob { - /// Sends an error back to the caller when worker pool is unavailable. - /// - /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was - /// dropped. - fn send_worker_unavailable_error(&self) -> Result<(), ()> { - match self { - Self::StorageProof { result_sender, .. } => { - let error = ParallelStateRootError::Other( - "Storage proof worker pool unavailable".to_string(), - ); - result_sender.send(Err(error)).map_err(|_| ()) - } - Self::BlindedStorageNode { result_sender, .. } => { - let error = SparseTrieError::from(SparseTrieErrorKind::Other(Box::new( - std::io::Error::new( - std::io::ErrorKind::BrokenPipe, - "Storage worker pool unavailable", - ), - ))); - result_sender.send(Err(error)).map_err(|_| ()) - } - } - } -} - /// Manager for coordinating proof request execution across different task types. /// /// # Architecture @@ -736,126 +710,92 @@ where match self.proof_task_rx.recv() { Ok(message) => { match message { - ProofTaskMessage::QueueTask(task) => match task { - ProofTaskKind::StorageProof(input, sender) => { - match self.storage_work_tx.send(StorageWorkerJob::StorageProof { - input, - result_sender: sender, - }) { - Ok(_) => { - tracing::trace!( - target: "trie::proof_task", - "Storage proof dispatched to worker pool" - ); - } - Err(crossbeam_channel::SendError(job)) => { - tracing::error!( - target: "trie::proof_task", - storage_worker_count = self.storage_worker_count, - "Worker pool disconnected, cannot process storage proof" - ); - - // Send error back to caller - let _ = job.send_worker_unavailable_error(); - } - } - } - - ProofTaskKind::BlindedStorageNode(account, path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_nodes += 1; + ProofTaskMessage::QueueTask(task) => { + // SAFETY: Storage worker sends cannot fail because: + // 1. Unbounded channel created during initialization (line 595) + // 2. Workers only exit when channel closes (storage_worker_loop recv() + // returns Err) + // 3. Channel only closes when all senders are dropped + // 4. self.storage_work_tx is only dropped on Terminate (line 861) + // 5. If we're processing QueueTask messages, Terminate hasn't occurred + // yet + // Therefore, the channel is guaranteed open with workers receiving. + // + // Same logic applies to account workers (unbounded channel at line 596, + // dropped at line 862). + + match task { + ProofTaskKind::StorageProof(input, sender) => { + self.storage_work_tx + .send(StorageWorkerJob::StorageProof { + input, + result_sender: sender, + }) + .expect("storage workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + "Storage proof dispatched to worker pool" + ); } - match self.storage_work_tx.send( - StorageWorkerJob::BlindedStorageNode { - account, - path, - result_sender: sender, - }, - ) { - Ok(_) => { - tracing::trace!( - target: "trie::proof_task", - ?account, - ?path, - "Blinded storage node dispatched to worker pool" - ); + ProofTaskKind::BlindedStorageNode(account, path, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_nodes += 1; } - Err(crossbeam_channel::SendError(job)) => { - tracing::error!( - target: "trie::proof_task", - storage_worker_count = self.storage_worker_count, - ?account, - ?path, - "Worker pool disconnected, cannot process blinded storage node" - ); - - // Send error back to caller - let _ = job.send_worker_unavailable_error(); - } - } - } - ProofTaskKind::BlindedAccountNode(path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.account_nodes += 1; + self.storage_work_tx + .send(StorageWorkerJob::BlindedStorageNode { + account, + path, + result_sender: sender, + }) + .expect("storage workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + ?account, + ?path, + "Blinded storage node dispatched to worker pool" + ); } - match self.account_work_tx.send( - AccountWorkerJob::BlindedAccountNode { - path, - result_sender: sender, - }, - ) { - Ok(_) => { - tracing::trace!( - target: "trie::proof_task", - ?path, - "Blinded account node dispatched to worker pool" - ); - } - Err(crossbeam_channel::SendError(job)) => { - tracing::error!( - target: "trie::proof_task", - account_worker_count = self.account_worker_count, - ?path, - "Worker pool disconnected, cannot process blinded account node" - ); - - // Send error back to caller - let _ = job.send_worker_unavailable_error(); + ProofTaskKind::BlindedAccountNode(path, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.account_nodes += 1; } + + self.account_work_tx + .send(AccountWorkerJob::BlindedAccountNode { + path, + result_sender: sender, + }) + .expect("account workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + ?path, + "Blinded account node dispatched to worker pool" + ); } - } - ProofTaskKind::AccountMultiproof(input, sender) => { - match self.account_work_tx.send( - AccountWorkerJob::AccountMultiproof { - input, - result_sender: sender, - }, - ) { - Ok(_) => { - tracing::trace!( - target: "trie::proof_task", - "Account multiproof dispatched to worker pool" - ); - } - Err(crossbeam_channel::SendError(job)) => { - tracing::error!( - target: "trie::proof_task", - account_worker_count = self.account_worker_count, - "Account worker pool disconnected" - ); - - // Send error back to caller - let _ = job.send_worker_unavailable_error(); - } + ProofTaskKind::AccountMultiproof(input, sender) => { + self.account_work_tx + .send(AccountWorkerJob::AccountMultiproof { + input, + result_sender: sender, + }) + .expect("account workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + "Account multiproof dispatched to worker pool" + ); } } - }, + } ProofTaskMessage::Terminate => { // Drop worker channels to signal workers to shut down drop(self.storage_work_tx); @@ -1073,31 +1013,6 @@ enum AccountWorkerJob { }, } -impl AccountWorkerJob { - /// Sends an error back to the caller when worker pool is unavailable. - /// - /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was - /// dropped. - fn send_worker_unavailable_error(&self) -> Result<(), ()> { - match self { - Self::AccountMultiproof { result_sender, .. } => { - let error = - ParallelStateRootError::Other("Account worker pool unavailable".to_string()); - result_sender.send(Err(error)).map_err(|_| ()) - } - Self::BlindedAccountNode { result_sender, .. } => { - let error = SparseTrieError::from(SparseTrieErrorKind::Other(Box::new( - std::io::Error::new( - std::io::ErrorKind::BrokenPipe, - "Account worker pool unavailable", - ), - ))); - result_sender.send(Err(error)).map_err(|_| ()) - } - } - } -} - /// Data used for initializing cursor factories that is shared across all storage proof instances. #[derive(Debug, Clone)] pub struct ProofTaskCtx { From 49410719eb91fd9c1510bc87330197112aa81aa1 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 12:59:29 +0000 Subject: [PATCH 066/144] rm comment --- crates/trie/parallel/src/proof_task.rs | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index fec68df3917..6186fc833e9 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -711,19 +711,6 @@ where Ok(message) => { match message { ProofTaskMessage::QueueTask(task) => { - // SAFETY: Storage worker sends cannot fail because: - // 1. Unbounded channel created during initialization (line 595) - // 2. Workers only exit when channel closes (storage_worker_loop recv() - // returns Err) - // 3. Channel only closes when all senders are dropped - // 4. self.storage_work_tx is only dropped on Terminate (line 861) - // 5. If we're processing QueueTask messages, Terminate hasn't occurred - // yet - // Therefore, the channel is guaranteed open with workers receiving. - // - // Same logic applies to account workers (unbounded channel at line 596, - // dropped at line 862). - match task { ProofTaskKind::StorageProof(input, sender) => { self.storage_work_tx From d6401abba18d6dd1a829c9c38db3e45ecf862748 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:06:17 +0000 Subject: [PATCH 067/144] fix rebase conflict --- crates/trie/parallel/src/proof_task.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 6186fc833e9..92af5165689 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -710,8 +710,7 @@ where match self.proof_task_rx.recv() { Ok(message) => { match message { - ProofTaskMessage::QueueTask(task) => { - match task { + ProofTaskMessage::QueueTask(task) => match task { ProofTaskKind::StorageProof(input, sender) => { self.storage_work_tx .send(StorageWorkerJob::StorageProof { From 9fa34b02fe89f0dacdf23423375be11ccb8904f4 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:06:51 +0000 Subject: [PATCH 068/144] fmt --- crates/trie/parallel/src/proof_task.rs | 129 ++++++++++++------------- 1 file changed, 64 insertions(+), 65 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 92af5165689..6c7f4efe5bc 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -711,77 +711,76 @@ where Ok(message) => { match message { ProofTaskMessage::QueueTask(task) => match task { - ProofTaskKind::StorageProof(input, sender) => { - self.storage_work_tx - .send(StorageWorkerJob::StorageProof { - input, - result_sender: sender, - }) - .expect("storage workers are running until Terminate"); - - tracing::trace!( - target: "trie::proof_task", - "Storage proof dispatched to worker pool" - ); - } + ProofTaskKind::StorageProof(input, sender) => { + self.storage_work_tx + .send(StorageWorkerJob::StorageProof { + input, + result_sender: sender, + }) + .expect("storage workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + "Storage proof dispatched to worker pool" + ); + } - ProofTaskKind::BlindedStorageNode(account, path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_nodes += 1; - } - - self.storage_work_tx - .send(StorageWorkerJob::BlindedStorageNode { - account, - path, - result_sender: sender, - }) - .expect("storage workers are running until Terminate"); - - tracing::trace!( - target: "trie::proof_task", - ?account, - ?path, - "Blinded storage node dispatched to worker pool" - ); + ProofTaskKind::BlindedStorageNode(account, path, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.storage_nodes += 1; } - ProofTaskKind::BlindedAccountNode(path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.account_nodes += 1; - } - - self.account_work_tx - .send(AccountWorkerJob::BlindedAccountNode { - path, - result_sender: sender, - }) - .expect("account workers are running until Terminate"); - - tracing::trace!( - target: "trie::proof_task", - ?path, - "Blinded account node dispatched to worker pool" - ); - } + self.storage_work_tx + .send(StorageWorkerJob::BlindedStorageNode { + account, + path, + result_sender: sender, + }) + .expect("storage workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + ?account, + ?path, + "Blinded storage node dispatched to worker pool" + ); + } - ProofTaskKind::AccountMultiproof(input, sender) => { - self.account_work_tx - .send(AccountWorkerJob::AccountMultiproof { - input, - result_sender: sender, - }) - .expect("account workers are running until Terminate"); - - tracing::trace!( - target: "trie::proof_task", - "Account multiproof dispatched to worker pool" - ); + ProofTaskKind::BlindedAccountNode(path, sender) => { + #[cfg(feature = "metrics")] + { + self.metrics.account_nodes += 1; } + + self.account_work_tx + .send(AccountWorkerJob::BlindedAccountNode { + path, + result_sender: sender, + }) + .expect("account workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + ?path, + "Blinded account node dispatched to worker pool" + ); } - } + + ProofTaskKind::AccountMultiproof(input, sender) => { + self.account_work_tx + .send(AccountWorkerJob::AccountMultiproof { + input, + result_sender: sender, + }) + .expect("account workers are running until Terminate"); + + tracing::trace!( + target: "trie::proof_task", + "Account multiproof dispatched to worker pool" + ); + } + }, ProofTaskMessage::Terminate => { // Drop worker channels to signal workers to shut down drop(self.storage_work_tx); From d0e2ba1efe11edc8c09b5cc5008283ab9a658d49 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:08:42 +0000 Subject: [PATCH 069/144] removed generic based on brian's suggestion --- crates/trie/parallel/src/proof_task.rs | 32 ++++++++++++-------------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 6c7f4efe5bc..c99a331a60d 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1025,13 +1025,11 @@ impl ProofTaskCtx { /// Message used to communicate with [`ProofTaskManager`]. #[derive(Debug)] -pub enum ProofTaskMessage { +pub enum ProofTaskMessage { /// A request to queue a proof task. QueueTask(ProofTaskKind), /// A request to terminate the proof task manager. Terminate, - #[doc(hidden)] - _Phantom(std::marker::PhantomData), } /// Proof task kind. @@ -1053,22 +1051,22 @@ pub enum ProofTaskKind { /// A handle that wraps a single proof task sender that sends a terminate message on `Drop` if the /// number of active handles went to zero. #[derive(Debug)] -pub struct ProofTaskManagerHandle { +pub struct ProofTaskManagerHandle { /// The sender for the proof task manager. - sender: Sender>, + sender: Sender, /// The number of active handles. active_handles: Arc, } -impl ProofTaskManagerHandle { +impl ProofTaskManagerHandle { /// Creates a new [`ProofTaskManagerHandle`] with the given sender. - pub fn new(sender: Sender>, active_handles: Arc) -> Self { + pub fn new(sender: Sender, active_handles: Arc) -> Self { active_handles.fetch_add(1, Ordering::SeqCst); Self { sender, active_handles } } /// Queues a task to the proof task manager. - pub fn queue_task(&self, task: ProofTaskKind) -> Result<(), SendError>> { + pub fn queue_task(&self, task: ProofTaskKind) -> Result<(), SendError> { self.sender.send(ProofTaskMessage::QueueTask(task)) } @@ -1078,13 +1076,13 @@ impl ProofTaskManagerHandle { } } -impl Clone for ProofTaskManagerHandle { +impl Clone for ProofTaskManagerHandle { fn clone(&self) -> Self { Self::new(self.sender.clone(), self.active_handles.clone()) } } -impl Drop for ProofTaskManagerHandle { +impl Drop for ProofTaskManagerHandle { fn drop(&mut self) { // Decrement the number of active handles and terminate the manager if it was the last // handle. @@ -1094,9 +1092,9 @@ impl Drop for ProofTaskManagerHandle { } } -impl TrieNodeProviderFactory for ProofTaskManagerHandle { - type AccountNodeProvider = ProofTaskTrieNodeProvider; - type StorageNodeProvider = ProofTaskTrieNodeProvider; +impl TrieNodeProviderFactory for ProofTaskManagerHandle { + type AccountNodeProvider = ProofTaskTrieNodeProvider; + type StorageNodeProvider = ProofTaskTrieNodeProvider; fn account_node_provider(&self) -> Self::AccountNodeProvider { ProofTaskTrieNodeProvider::AccountNode { sender: self.sender.clone() } @@ -1109,22 +1107,22 @@ impl TrieNodeProviderFactory for ProofTaskManagerHandle { /// Trie node provider for retrieving trie nodes by path. #[derive(Debug)] -pub enum ProofTaskTrieNodeProvider { +pub enum ProofTaskTrieNodeProvider { /// Blinded account trie node provider. AccountNode { /// Sender to the proof task. - sender: Sender>, + sender: Sender, }, /// Blinded storage trie node provider. StorageNode { /// Target account. account: B256, /// Sender to the proof task. - sender: Sender>, + sender: Sender, }, } -impl TrieNodeProvider for ProofTaskTrieNodeProvider { +impl TrieNodeProvider for ProofTaskTrieNodeProvider { fn trie_node(&self, path: &Nibbles) -> Result, SparseTrieError> { let (tx, rx) = channel(); match self { From 7efff3d4e5215c0553ab09636b044bbec8982fea Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 21:14:37 +0800 Subject: [PATCH 070/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> --- crates/trie/parallel/src/proof_task.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 00b252f2fe1..2212d7652c7 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -371,8 +371,7 @@ where max_concurrency, total_transactions: 0, pending_tasks: VecDeque::new(), - proof_task_txs: Vec::with_capacity(max_concurrency), /* used for on-demand account - * trie operations */ + proof_task_txs: Vec::with_capacity(max_concurrency), view, task_ctx, executor, From f7cd93fb85586038260099ea500be32a66b94629 Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 8 Oct 2025 21:14:58 +0800 Subject: [PATCH 071/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> --- crates/trie/parallel/src/proof_task.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 2212d7652c7..685ec1c085c 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -450,8 +450,7 @@ where /// # Task Routing /// /// - **Storage Trie Operations** (`StorageProof` and `BlindedStorageNode`): Routed to - /// pre-spawned worker pool via unbounded channel. Only falls back to `pending_tasks` if - /// workers are disconnected (e.g., all workers panicked). + /// pre-spawned worker pool via unbounded channel. /// - **Account Trie Operations** (`BlindedAccountNode`): Queued for on-demand execution via /// `pending_tasks`. /// From f823c6bcd94211ab85ab709786ec0ceaf00f8150 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:17:40 +0000 Subject: [PATCH 072/144] Refactor error handling in StorageWorkerJob to use a consistent error message format --- crates/trie/parallel/src/proof_task.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 685ec1c085c..866f0f6cb98 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -79,23 +79,19 @@ impl StorageWorkerJob { /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was /// dropped. fn send_worker_unavailable_error(&self) -> Result<(), ()> { + let error_msg = "Storage proof worker pool unavailable"; + match self { Self::StorageProof { result_sender, .. } => { - let error = ParallelStateRootError::Other( - "Storage proof worker pool unavailable".to_string(), - ); - result_sender.send(Err(error)).map_err(|_| ()) + result_sender.send(Err(ParallelStateRootError::Other(error_msg.to_string()))) } Self::BlindedStorageNode { result_sender, .. } => { - let error = SparseTrieError::from(SparseTrieErrorKind::Other(Box::new( - std::io::Error::new( - std::io::ErrorKind::BrokenPipe, - "Storage worker pool unavailable", - ), - ))); - result_sender.send(Err(error)).map_err(|_| ()) + result_sender.send(Err(SparseTrieError::from(SparseTrieErrorKind::Other( + Box::new(std::io::Error::new(std::io::ErrorKind::BrokenPipe, error_msg)) + )))) } } + .map_err(|_| ()) } } From 9c08aedb4c20bcde723ec47afb7f30738f3b4c7f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:28:21 +0000 Subject: [PATCH 073/144] Refactor error handling in StorageWorkerJob to use structured error types --- crates/trie/parallel/src/proof_task.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 866f0f6cb98..ce845e2407e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -79,15 +79,14 @@ impl StorageWorkerJob { /// Returns `Ok(())` if the error was sent successfully, or `Err(())` if the receiver was /// dropped. fn send_worker_unavailable_error(&self) -> Result<(), ()> { - let error_msg = "Storage proof worker pool unavailable"; + let error = + ParallelStateRootError::Other("Storage proof worker pool unavailable".to_string()); match self { - Self::StorageProof { result_sender, .. } => { - result_sender.send(Err(ParallelStateRootError::Other(error_msg.to_string()))) - } + Self::StorageProof { result_sender, .. } => result_sender.send(Err(error)), Self::BlindedStorageNode { result_sender, .. } => { result_sender.send(Err(SparseTrieError::from(SparseTrieErrorKind::Other( - Box::new(std::io::Error::new(std::io::ErrorKind::BrokenPipe, error_msg)) + Box::new(error), )))) } } From 7f9ec065b2c2c3490734ecfcc5a7c3795ac08dc1 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:28:30 +0000 Subject: [PATCH 074/144] fmt, clipy --- crates/trie/parallel/src/proof_task.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index ce845e2407e..fa90e9dad39 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -84,11 +84,8 @@ impl StorageWorkerJob { match self { Self::StorageProof { result_sender, .. } => result_sender.send(Err(error)), - Self::BlindedStorageNode { result_sender, .. } => { - result_sender.send(Err(SparseTrieError::from(SparseTrieErrorKind::Other( - Box::new(error), - )))) - } + Self::BlindedStorageNode { result_sender, .. } => result_sender + .send(Err(SparseTrieError::from(SparseTrieErrorKind::Other(Box::new(error))))), } .map_err(|_| ()) } From 6d413528a0384b2b9217bfacad9dbb99f92544dd Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 8 Oct 2025 13:30:03 +0000 Subject: [PATCH 075/144] fix --- crates/trie/parallel/src/proof_task.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index fa90e9dad39..0c513c55763 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -83,11 +83,13 @@ impl StorageWorkerJob { ParallelStateRootError::Other("Storage proof worker pool unavailable".to_string()); match self { - Self::StorageProof { result_sender, .. } => result_sender.send(Err(error)), + Self::StorageProof { result_sender, .. } => { + result_sender.send(Err(error)).map_err(|_| ()) + } Self::BlindedStorageNode { result_sender, .. } => result_sender - .send(Err(SparseTrieError::from(SparseTrieErrorKind::Other(Box::new(error))))), + .send(Err(SparseTrieError::from(SparseTrieErrorKind::Other(Box::new(error))))) + .map_err(|_| ()), } - .map_err(|_| ()) } } From dc5ba157d6299f7f57c93debbe577da933af595e Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 01:29:38 +0000 Subject: [PATCH 076/144] moved `build_account_multiproof_with_storage_roots` to prooftask.rs - brian --- crates/trie/parallel/src/proof.rs | 134 ------------------------ crates/trie/parallel/src/proof_task.rs | 138 ++++++++++++++++++++++++- 2 files changed, 137 insertions(+), 135 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index e17c38da1a7..ff4544fc8e5 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -251,140 +251,6 @@ where } } -/// Builds an account multiproof given pre-collected storage proofs. -/// -/// This is a helper function used by both `decoded_multiproof` and account workers to build -/// the account subtree proof after storage proofs have been collected. -/// -/// Returns a `DecodedMultiProof` containing the account subtree and storage proofs. -#[allow(clippy::too_many_arguments)] -pub(crate) fn build_account_multiproof_with_storage_roots( - trie_cursor_factory: C, - hashed_cursor_factory: H, - targets: &MultiProofTargets, - prefix_set: PrefixSet, - collect_branch_node_masks: bool, - multi_added_removed_keys: Option<&Arc>, - mut storage_proofs: B256Map, - missed_leaves_storage_roots: &DashMap, - tracker: &mut ParallelTrieTracker, -) -> Result -where - C: TrieCursorFactory + Clone, - H: HashedCursorFactory + Clone, -{ - let accounts_added_removed_keys = - multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); - - // Create the walker. - let walker = TrieWalker::<_>::state_trie( - trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, - prefix_set, - ) - .with_added_removed_keys(accounts_added_removed_keys) - .with_deletions_retained(true); - - // Create a hash builder to rebuild the root node since it is not available in the database. - let retainer = targets - .keys() - .map(Nibbles::unpack) - .collect::() - .with_added_removed_keys(accounts_added_removed_keys); - let mut hash_builder = HashBuilder::default() - .with_proof_retainer(retainer) - .with_updates(collect_branch_node_masks); - - // Initialize all storage multiproofs as empty. - // Storage multiproofs for non empty tries will be overwritten if necessary. - let mut collected_decoded_storages: B256Map = - targets.keys().map(|key| (*key, DecodedStorageMultiProof::empty())).collect(); - let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); - let mut account_node_iter = TrieNodeIter::state_trie( - walker, - hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, - ); - - while let Some(account_node) = account_node_iter.try_next().map_err(ProviderError::Database)? { - match account_node { - TrieElement::Branch(node) => { - hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); - } - TrieElement::Leaf(hashed_address, account) => { - let root = match storage_proofs.remove(&hashed_address) { - Some(proof) => { - let root = proof.root; - if let Some(entry) = collected_decoded_storages.get_mut(&hashed_address) { - *entry = proof; - } - root - } - // Since we do not store all intermediate nodes in the database, there might - // be a possibility of re-adding a non-modified leaf to the hash builder. - None => { - tracker.inc_missed_leaves(); - - match missed_leaves_storage_roots.entry(hashed_address) { - dashmap::Entry::Occupied(occ) => *occ.get(), - dashmap::Entry::Vacant(vac) => { - let root = StorageProof::new_hashed( - trie_cursor_factory.clone(), - hashed_cursor_factory.clone(), - hashed_address, - ) - .with_prefix_set_mut(Default::default()) - .storage_multiproof( - targets.get(&hashed_address).cloned().unwrap_or_default(), - ) - .map_err(|e| { - ParallelStateRootError::StorageRoot(StorageRootError::Database( - DatabaseError::Other(e.to_string()), - )) - })? - .root; - - vac.insert(root); - root - } - } - } - }; - - // Encode account - account_rlp.clear(); - let account = account.into_trie_account(root); - account.encode(&mut account_rlp as &mut dyn BufMut); - - hash_builder.add_leaf(Nibbles::unpack(hashed_address), &account_rlp); - } - } - } - - // Insert storage proofs for accounts not encountered during trie walk. - collected_decoded_storages.extend(storage_proofs); - - let _ = hash_builder.root(); - - let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); - let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; - - let (branch_node_hash_masks, branch_node_tree_masks) = if collect_branch_node_masks { - let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); - ( - updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), - updated_branch_nodes.into_iter().map(|(path, node)| (path, node.tree_mask)).collect(), - ) - } else { - (Default::default(), Default::default()) - }; - - Ok(DecodedMultiProof { - account_subtree: decoded_account_subtree, - branch_node_hash_masks, - branch_node_tree_masks, - storages: collected_decoded_storages, - }) -} - #[cfg(test)] mod tests { use super::*; diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index c99a331a60d..29064378e75 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -392,7 +392,7 @@ fn account_worker_loop( let account_prefix_set = std::mem::take(&mut input.prefix_sets.account_prefix_set); - let result = crate::proof::build_account_multiproof_with_storage_roots( + let result = build_account_multiproof_with_storage_roots( trie_cursor_factory.clone(), hashed_cursor_factory.clone(), &input.targets, @@ -472,6 +472,142 @@ fn account_worker_loop( ); } +/// Builds an account multiproof given pre-collected storage proofs. +/// +/// This is a helper function used by account workers to build the account subtree proof +/// after storage proofs have been collected. +/// +/// Returns a `DecodedMultiProof` containing the account subtree and storage proofs. +#[allow(clippy::too_many_arguments)] +fn build_account_multiproof_with_storage_roots( + trie_cursor_factory: C, + hashed_cursor_factory: H, + targets: &MultiProofTargets, + prefix_set: PrefixSet, + collect_branch_node_masks: bool, + multi_added_removed_keys: Option<&Arc>, + mut storage_proofs: B256Map, + missed_leaves_storage_roots: &DashMap, + tracker: &mut ParallelTrieTracker, +) -> Result +where + C: TrieCursorFactory + Clone, + H: HashedCursorFactory + Clone, +{ + let accounts_added_removed_keys = + multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); + + // Create the walker. + let walker = TrieWalker::<_>::state_trie( + trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, + prefix_set, + ) + .with_added_removed_keys(accounts_added_removed_keys) + .with_deletions_retained(true); + + // Create a hash builder to rebuild the root node since it is not available in the database. + let retainer = targets + .keys() + .map(Nibbles::unpack) + .collect::() + .with_added_removed_keys(accounts_added_removed_keys); + let mut hash_builder = HashBuilder::default() + .with_proof_retainer(retainer) + .with_updates(collect_branch_node_masks); + + // Initialize all storage multiproofs as empty. + // Storage multiproofs for non empty tries will be overwritten if necessary. + let mut collected_decoded_storages: B256Map = + targets.keys().map(|key| (*key, DecodedStorageMultiProof::empty())).collect(); + let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); + let mut account_node_iter = TrieNodeIter::state_trie( + walker, + hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, + ); + + while let Some(account_node) = account_node_iter.try_next().map_err(ProviderError::Database)? { + match account_node { + TrieElement::Branch(node) => { + hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); + } + TrieElement::Leaf(hashed_address, account) => { + let root = match storage_proofs.remove(&hashed_address) { + Some(proof) => { + let root = proof.root; + if let Some(entry) = collected_decoded_storages.get_mut(&hashed_address) { + *entry = proof; + } + root + } + // Since we do not store all intermediate nodes in the database, there might + // be a possibility of re-adding a non-modified leaf to the hash builder. + None => { + tracker.inc_missed_leaves(); + + match missed_leaves_storage_roots.entry(hashed_address) { + dashmap::Entry::Occupied(occ) => *occ.get(), + dashmap::Entry::Vacant(vac) => { + let root = StorageProof::new_hashed( + trie_cursor_factory.clone(), + hashed_cursor_factory.clone(), + hashed_address, + ) + .with_prefix_set_mut(Default::default()) + .storage_multiproof( + targets.get(&hashed_address).cloned().unwrap_or_default(), + ) + .map_err(|e| { + ParallelStateRootError::StorageRoot( + reth_execution_errors::StorageRootError::Database( + DatabaseError::Other(e.to_string()), + ), + ) + })? + .root; + + vac.insert(root); + root + } + } + } + }; + + // Encode account + account_rlp.clear(); + let account = account.into_trie_account(root); + account.encode(&mut account_rlp as &mut dyn BufMut); + + hash_builder.add_leaf(Nibbles::unpack(hashed_address), &account_rlp); + } + } + } + + // Insert storage proofs for accounts not encountered during trie walk. + collected_decoded_storages.extend(storage_proofs); + + let _ = hash_builder.root(); + + let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); + let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; + + let (branch_node_hash_masks, branch_node_tree_masks) = if collect_branch_node_masks { + let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); + ( + updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), + updated_branch_nodes.into_iter().map(|(path, node)| (path, node.tree_mask)).collect(), + ) + } else { + (Default::default(), Default::default()) + }; + + Ok(DecodedMultiProof { + account_subtree: decoded_account_subtree, + branch_node_hash_masks, + branch_node_tree_masks, + storages: collected_decoded_storages, + }) +} + /// Collects storage proofs for all accounts in the targets by queueing to storage workers. /// /// Queues storage proof tasks to the storage worker pool and collects results. From 14b635686817ee421fba9d165e97191b1cc5b1eb Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 01:51:37 +0000 Subject: [PATCH 077/144] refactor: remove generic type parameter from `ProofTaskManagerHandle` - Removed the generic type parameter from `ProofTaskManagerHandle` and related structures to streamline the code. - Updated the `MultiproofManager` and `ProofTaskManager` to use the simplified handle, enhancing clarity and reducing complexity. - Adjusted method signatures and internal logic to accommodate the changes while maintaining functionality. --- .../src/tree/payload_processor/multiproof.rs | 14 ++++----- crates/trie/parallel/src/proof.rs | 29 ++++++------------ crates/trie/parallel/src/proof_task.rs | 30 ++++++++++++------- 3 files changed, 35 insertions(+), 38 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index dfa33838e4b..a3c4dc676d3 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -12,7 +12,7 @@ use derive_more::derive::Deref; use metrics::Histogram; use reth_errors::ProviderError; use reth_metrics::Metrics; -use reth_provider::{providers::ConsistentDbView, BlockReader, DatabaseProviderFactory, FactoryTx}; +use reth_provider::{providers::ConsistentDbView, BlockReader, DatabaseProviderFactory}; use reth_revm::state::EvmState; use reth_trie::{ added_removed_keys::MultiAddedRemovedKeys, prefix_set::TriePrefixSetsMut, @@ -354,9 +354,9 @@ pub struct MultiproofManager { /// Executor for tasks executor: WorkloadExecutor, /// Sender to the storage proof task. - storage_proof_task_handle: ProofTaskManagerHandle>, + storage_proof_task_handle: ProofTaskManagerHandle, /// Sender to the account proof task. - account_proof_task_handle: ProofTaskManagerHandle>, + account_proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. /// @@ -381,8 +381,8 @@ where fn new( executor: WorkloadExecutor, metrics: MultiProofTaskMetrics, - storage_proof_task_handle: ProofTaskManagerHandle>, - account_proof_task_handle: ProofTaskManagerHandle>, + storage_proof_task_handle: ProofTaskManagerHandle, + account_proof_task_handle: ProofTaskManagerHandle, max_concurrent: usize, ) -> Self { Self { @@ -707,7 +707,7 @@ where pub(super) fn new( config: MultiProofConfig, executor: WorkloadExecutor, - proof_task_handle: ProofTaskManagerHandle>, + proof_task_handle: ProofTaskManagerHandle, to_sparse_trie: Sender, max_concurrency: usize, chunk_size: Option, @@ -727,7 +727,7 @@ where multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle.clone(), + proof_task_handle.clone(), proof_task_handle, max_concurrency, ), diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index ff4544fc8e5..198431d2824 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -4,33 +4,19 @@ use crate::{ AccountMultiproofInput, ProofTaskKind, ProofTaskManagerHandle, StorageProofInput, }, root::ParallelStateRootError, - stats::ParallelTrieTracker, StorageRootTargets, }; -use alloy_primitives::{ - map::{B256Map, B256Set}, - B256, -}; -use alloy_rlp::{BufMut, Encodable}; +use alloy_primitives::{map::B256Set, B256}; use dashmap::DashMap; use reth_execution_errors::StorageRootError; -use reth_provider::{BlockReader, DatabaseProviderFactory, FactoryTx, ProviderError}; +use reth_provider::{BlockReader, DatabaseProviderFactory}; use reth_storage_errors::db::DatabaseError; use reth_trie::{ - hashed_cursor::HashedCursorFactory, - node_iter::{TrieElement, TrieNodeIter}, prefix_set::{PrefixSet, PrefixSetMut, TriePrefixSets, TriePrefixSetsMut}, - proof::StorageProof, - trie_cursor::TrieCursorFactory, updates::TrieUpdatesSorted, - walker::TrieWalker, - DecodedMultiProof, DecodedStorageMultiProof, HashBuilder, HashedPostStateSorted, - MultiProofTargets, Nibbles, TRIE_ACCOUNT_RLP_MAX_SIZE, -}; -use reth_trie_common::{ - added_removed_keys::MultiAddedRemovedKeys, - proof::{DecodedProofNodes, ProofRetainer}, + DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, Nibbles, }; +use reth_trie_common::added_removed_keys::MultiAddedRemovedKeys; use std::sync::{ mpsc::{channel, Receiver}, Arc, @@ -57,10 +43,12 @@ pub struct ParallelProof { /// Provided by the user to give the necessary context to retain extra proofs. multi_added_removed_keys: Option>, /// Handle to the storage proof task. - storage_proof_task_handle: ProofTaskManagerHandle>, + storage_proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, + /// Marker to keep the Factory type parameter. + _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics, } @@ -72,7 +60,7 @@ impl ParallelProof { state_sorted: Arc, prefix_sets: Arc, missed_leaves_storage_roots: Arc>, - storage_proof_task_handle: ProofTaskManagerHandle>, + storage_proof_task_handle: ProofTaskManagerHandle, ) -> Self { Self { nodes_sorted, @@ -82,6 +70,7 @@ impl ParallelProof { collect_branch_node_masks: false, multi_added_removed_keys: None, storage_proof_task_handle, + _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics::new_with_labels(&[("type", "proof")]), } diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 29064378e75..61be9e1a4c2 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -17,25 +17,31 @@ use alloy_primitives::{ map::{B256Map, B256Set}, B256, }; +use alloy_rlp::{BufMut, Encodable}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use dashmap::DashMap; use reth_db_api::transaction::DbTx; use reth_execution_errors::SparseTrieError; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, FactoryTx, - ProviderResult, + ProviderError, ProviderResult, }; +use reth_storage_errors::db::DatabaseError; use reth_trie::{ hashed_cursor::{HashedCursorFactory, HashedPostStateCursorFactory}, + node_iter::{TrieElement, TrieNodeIter}, prefix_set::{TriePrefixSets, TriePrefixSetsMut}, proof::{ProofTrieNodeProviderFactory, StorageProof}, trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory}, updates::TrieUpdatesSorted, - DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, Nibbles, + walker::TrieWalker, + DecodedMultiProof, DecodedStorageMultiProof, HashBuilder, HashedPostStateSorted, + MultiProofTargets, Nibbles, TRIE_ACCOUNT_RLP_MAX_SIZE, }; use reth_trie_common::{ added_removed_keys::MultiAddedRemovedKeys, prefix_set::{PrefixSet, PrefixSetMut}, + proof::{DecodedProofNodes, ProofRetainer}, }; use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; use reth_trie_sparse::provider::{RevealedNode, TrieNodeProvider, TrieNodeProviderFactory}; @@ -138,10 +144,10 @@ pub struct ProofTaskManager { account_worker_count: usize, /// Receives proof task requests from [`ProofTaskManagerHandle`]. - proof_task_rx: Receiver>>, + proof_task_rx: Receiver, /// Sender for creating handles that can queue tasks. - proof_task_tx: Sender>>, + proof_task_tx: Sender, /// The number of active handles. /// @@ -152,6 +158,9 @@ pub struct ProofTaskManager { /// Metrics tracking proof task operations. #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, + + /// Marker to keep the Factory type parameter. + _phantom: std::marker::PhantomData, } /// Worker loop for storage trie operations. @@ -324,7 +333,7 @@ fn storage_worker_loop( fn account_worker_loop( proof_tx: ProofTaskTx, work_rx: CrossbeamReceiver, - storage_proof_handle: ProofTaskManagerHandle, + storage_proof_handle: ProofTaskManagerHandle, worker_id: usize, ) where Tx: DbTx, @@ -615,7 +624,7 @@ where /// No inline fallback - fails fast if storage workers are unavailable. fn collect_storage_proofs( _proof_tx: &ProofTaskTx, - storage_proof_handle: &ProofTaskManagerHandle, + account_proof_handle: &ProofTaskManagerHandle, targets: &MultiProofTargets, prefix_sets: &TriePrefixSets, with_branch_node_masks: bool, @@ -645,7 +654,7 @@ where let (sender, receiver) = channel(); // If queuing fails, propagate error up (no fallback) - storage_proof_handle.queue_task(ProofTaskKind::StorageProof(input, sender)).map_err( + account_proof_handle.queue_task(ProofTaskKind::StorageProof(input, sender)).map_err( |_| { ParallelStateRootError::Other(format!( "Failed to queue storage proof for {}: storage worker pool unavailable", @@ -757,11 +766,13 @@ where #[cfg(feature = "metrics")] metrics: ProofTaskMetrics::default(), + + _phantom: std::marker::PhantomData, // TODO: we can remove this once we remove ProofTaskManager / ConsistentDbView }) } /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. - pub fn handle(&self) -> ProofTaskManagerHandle> { + pub fn handle(&self) -> ProofTaskManagerHandle { ProofTaskManagerHandle::new(self.proof_task_tx.clone(), self.active_handles.clone()) } @@ -935,9 +946,6 @@ where return Ok(()) } - ProofTaskMessage::_Phantom(_) => { - unreachable!("_Phantom variant should never be constructed") - } } } // All senders are disconnected, so we can terminate From 5e1ca830d2970ba2661f40de9144ccb961d91bb1 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 01:59:19 +0000 Subject: [PATCH 078/144] fmt --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index a3c4dc676d3..29f78f54006 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -727,7 +727,7 @@ where multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle.clone(), + proof_task_handle.clone(), proof_task_handle, max_concurrency, ), From 3b815f691e0a9ba8b10db512c025a0773c590c52 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 01:59:49 +0000 Subject: [PATCH 079/144] refactor: rename storage proof task handle for clarity - Updated the field name from `storage_proof_task_handle` to `proof_task_handle` in the `ParallelProof` struct to better reflect its purpose. - Adjusted related method signatures and internal references to maintain consistency throughout the codebase. --- crates/trie/parallel/src/proof.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 198431d2824..d547f9b69d7 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -42,8 +42,8 @@ pub struct ParallelProof { collect_branch_node_masks: bool, /// Provided by the user to give the necessary context to retain extra proofs. multi_added_removed_keys: Option>, - /// Handle to the storage proof task. - storage_proof_task_handle: ProofTaskManagerHandle, + /// Handle to the proof task manager. + proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, @@ -60,7 +60,7 @@ impl ParallelProof { state_sorted: Arc, prefix_sets: Arc, missed_leaves_storage_roots: Arc>, - storage_proof_task_handle: ProofTaskManagerHandle, + proof_task_handle: ProofTaskManagerHandle, ) -> Self { Self { nodes_sorted, @@ -69,7 +69,7 @@ impl ParallelProof { missed_leaves_storage_roots, collect_branch_node_masks: false, multi_added_removed_keys: None, - storage_proof_task_handle, + proof_task_handle, _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics::new_with_labels(&[("type", "proof")]), @@ -113,8 +113,7 @@ where ); let (sender, receiver) = std::sync::mpsc::channel(); - let _ = - self.storage_proof_task_handle.queue_task(ProofTaskKind::StorageProof(input, sender)); + let _ = self.proof_task_handle.queue_task(ProofTaskKind::StorageProof(input, sender)); receiver } @@ -206,7 +205,7 @@ where }; let (sender, receiver) = channel(); - self.storage_proof_task_handle + self.proof_task_handle .queue_task(ProofTaskKind::AccountMultiproof(input, sender)) .map_err(|_| { ParallelStateRootError::Other( From 7df34889c2c0a509fc48f4a3f04cb524b74ba5bf Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 02:01:19 +0000 Subject: [PATCH 080/144] refactor: simplify `ProofTaskManager` by removing generic type parameter - Removed the generic type parameter `Factory` from the `ProofTaskManager` struct and its implementation, streamlining the code. - Updated the `new` and `spawn_worker_pool` methods to maintain functionality while simplifying type handling. - Enhanced clarity and reduced complexity in the proof task management logic. --- crates/trie/parallel/src/proof_task.rs | 27 +++++++++----------------- 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 61be9e1a4c2..3cf35b7857a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -128,7 +128,7 @@ enum StorageWorkerJob { /// - Use standard `std::mpsc` message passing /// - Receive consistent return types and error handling #[derive(Debug)] -pub struct ProofTaskManager { +pub struct ProofTaskManager { /// Sender for storage worker jobs to worker pool. storage_work_tx: CrossbeamSender, @@ -158,9 +158,6 @@ pub struct ProofTaskManager { /// Metrics tracking proof task operations. #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, - - /// Marker to keep the Factory type parameter. - _phantom: std::marker::PhantomData, } /// Worker loop for storage trie operations. @@ -690,23 +687,23 @@ where Ok(storage_proofs) } -impl ProofTaskManager -where - Factory: DatabaseProviderFactory, -{ +impl ProofTaskManager { /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. /// /// The `storage_worker_count` determines how many storage workers to spawn, and /// `account_worker_count` determines how many account workers to spawn. /// Returns an error if the underlying provider fails to create the transactions required for /// spawning workers. - pub fn new( + pub fn new( executor: Handle, view: ConsistentDbView, task_ctx: ProofTaskCtx, storage_worker_count: usize, account_worker_count: usize, - ) -> ProviderResult { + ) -> ProviderResult + where + Factory: DatabaseProviderFactory, + { let (proof_task_tx, proof_task_rx) = channel(); // Use unbounded channel to ensure all storage operations are queued to workers. @@ -766,8 +763,6 @@ where #[cfg(feature = "metrics")] metrics: ProofTaskMetrics::default(), - - _phantom: std::marker::PhantomData, // TODO: we can remove this once we remove ProofTaskManager / ConsistentDbView }) } @@ -790,7 +785,7 @@ where /// /// Returns /// The number of workers successfully spawned - fn spawn_worker_pool( + fn spawn_worker_pool( executor: &Handle, view: &ConsistentDbView, task_ctx: &ProofTaskCtx, @@ -800,6 +795,7 @@ where worker_fn: F, ) -> ProviderResult where + Factory: DatabaseProviderFactory, Job: Send + 'static, F: Fn(ProofTaskTx>, CrossbeamReceiver, usize) + Send @@ -831,12 +827,7 @@ where Ok(spawned_workers) } -} -impl ProofTaskManager -where - Factory: DatabaseProviderFactory + 'static, -{ /// Loops, managing the proof tasks, routing them to the appropriate worker pools. /// /// # Task Routing From e418bb14963555edd1296b9f1aa830180aea1142 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 02:01:29 +0000 Subject: [PATCH 081/144] refactor: improve documentation for proof task handles in `MultiproofManager` - Updated comments for `storage_proof_task_handle` and `account_proof_task_handle` to clarify their roles in managing proof tasks. - Adjusted method calls to use the updated handle references, enhancing code readability and maintainability. --- .../engine/tree/src/tree/payload_processor/multiproof.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 29f78f54006..137c7d76705 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -353,9 +353,10 @@ pub struct MultiproofManager { pending: VecDeque>, /// Executor for tasks executor: WorkloadExecutor, - /// Sender to the storage proof task. + /// Handle to the proof task manager used for creating ParallelProof instances for storage + /// proofs. storage_proof_task_handle: ProofTaskManagerHandle, - /// Sender to the account proof task. + /// Handle to the proof task manager used for account multiproofs. account_proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. @@ -480,7 +481,7 @@ where config.state_sorted, config.prefix_sets, missed_leaves_storage_roots, - storage_proof_task_handle.clone(), + storage_proof_task_handle, ) .with_branch_node_masks(true) .with_multi_added_removed_keys(Some(multi_added_removed_keys)) From 1bbfc7c33ddb3277f77e4d795075e8c408b79c44 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 02:40:50 +0000 Subject: [PATCH 082/144] clippy --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 137c7d76705..aac9f3cddeb 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -353,7 +353,7 @@ pub struct MultiproofManager { pending: VecDeque>, /// Executor for tasks executor: WorkloadExecutor, - /// Handle to the proof task manager used for creating ParallelProof instances for storage + /// Handle to the proof task manager used for creating `ParallelProof` instances for storage /// proofs. storage_proof_task_handle: ProofTaskManagerHandle, /// Handle to the proof task manager used for account multiproofs. From a654cf8d730cf8272665f3abc9f46c9fd02c5c59 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 03:31:49 +0000 Subject: [PATCH 083/144] rm clone from `StorageProofInput` --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 3cf35b7857a..2894c4217d6 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1062,7 +1062,7 @@ where } /// This represents an input for a storage proof. -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct StorageProofInput { /// The hashed address for which the proof is calculated. hashed_address: B256, From 3c00e98302a685e5aa7ab2b443a21a24fba9f718 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 04:08:17 +0000 Subject: [PATCH 084/144] refactor: use receiver to allow Account worker and storage workers run in parallel - replace standard mpsc channels with crossbeam channels in `ProofTaskManager` - Updated `proof_task_rx` and `proof_task_tx` to use `CrossbeamReceiver` and `CrossbeamSender` for improved performance and non-blocking behavior. - Renamed `collect_storage_proofs` to `queue_storage_proofs` to better reflect its functionality of returning receivers for storage proofs. - Enhanced documentation to clarify the interleaved parallelism between account trie traversal and storage proof computation. --- crates/trie/parallel/src/proof_task.rs | 118 +++++++++++++------------ 1 file changed, 62 insertions(+), 56 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 2894c4217d6..5dc8421258b 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -48,7 +48,7 @@ use reth_trie_sparse::provider::{RevealedNode, TrieNodeProvider, TrieNodeProvide use std::{ sync::{ atomic::{AtomicUsize, Ordering}, - mpsc::{channel, Receiver, SendError, Sender}, + mpsc::{channel, Receiver, Sender}, Arc, }, time::Instant, @@ -144,10 +144,10 @@ pub struct ProofTaskManager { account_worker_count: usize, /// Receives proof task requests from [`ProofTaskManagerHandle`]. - proof_task_rx: Receiver, + proof_task_rx: CrossbeamReceiver, /// Sender for creating handles that can queue tasks. - proof_task_tx: Sender, + proof_task_tx: CrossbeamSender, /// The number of active handles. /// @@ -378,15 +378,15 @@ fn account_worker_loop( .len(); tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); - let storage_proofs = match collect_storage_proofs( + let storage_proof_receivers = match queue_storage_proofs( &proof_tx, &storage_proof_handle, - &input.targets, + input.targets.clone(), &input.prefix_sets, input.collect_branch_node_masks, input.multi_added_removed_keys.clone(), ) { - Ok(proofs) => proofs, + Ok(receivers) => receivers, Err(error) => { let _ = result_sender.send(Err(error)); continue; @@ -405,7 +405,7 @@ fn account_worker_loop( account_prefix_set, input.collect_branch_node_masks, input.multi_added_removed_keys.as_ref(), - storage_proofs, + storage_proof_receivers, missed_leaves_storage_roots, &mut tracker, ); @@ -478,10 +478,11 @@ fn account_worker_loop( ); } -/// Builds an account multiproof given pre-collected storage proofs. +/// Builds an account multiproof by consuming storage proof receivers lazily during trie walk. /// /// This is a helper function used by account workers to build the account subtree proof -/// after storage proofs have been collected. +/// while storage proofs are still being computed. Receivers are consumed only when needed, +/// enabling interleaved parallelism between account trie traversal and storage proof computation. /// /// Returns a `DecodedMultiProof` containing the account subtree and storage proofs. #[allow(clippy::too_many_arguments)] @@ -492,7 +493,7 @@ fn build_account_multiproof_with_storage_roots( prefix_set: PrefixSet, collect_branch_node_masks: bool, multi_added_removed_keys: Option<&Arc>, - mut storage_proofs: B256Map, + mut storage_proof_receivers: B256Map>, missed_leaves_storage_roots: &DashMap, tracker: &mut ParallelTrieTracker, ) -> Result @@ -537,8 +538,19 @@ where hash_builder.add_branch(node.key, node.value, node.children_are_in_trie); } TrieElement::Leaf(hashed_address, account) => { - let root = match storage_proofs.remove(&hashed_address) { - Some(proof) => { + let root = match storage_proof_receivers.remove(&hashed_address) { + Some(receiver) => { + // Block on this specific storage proof receiver - enables interleaved + // parallelism + let proof = receiver.recv().map_err(|_| { + ParallelStateRootError::StorageRoot( + reth_execution_errors::StorageRootError::Database( + DatabaseError::Other(format!( + "Storage proof channel closed for {hashed_address}" + )), + ), + ) + })??; let root = proof.root; if let Some(entry) = collected_decoded_storages.get_mut(&hashed_address) { *entry = proof; @@ -588,8 +600,12 @@ where } } - // Insert storage proofs for accounts not encountered during trie walk. - collected_decoded_storages.extend(storage_proofs); + // Consume remaining storage proof receivers for accounts not encountered during trie walk. + for (hashed_address, receiver) in storage_proof_receivers { + if let Ok(Ok(proof)) = receiver.recv() { + collected_decoded_storages.insert(hashed_address, proof); + } + } let _ = hash_builder.root(); @@ -614,36 +630,38 @@ where }) } -/// Collects storage proofs for all accounts in the targets by queueing to storage workers. +/// Queues storage proofs for all accounts in the targets and returns receivers. /// -/// Queues storage proof tasks to the storage worker pool and collects results. -/// Propagates errors up if queuing fails, workers return errors, or channels are dropped. -/// No inline fallback - fails fast if storage workers are unavailable. -fn collect_storage_proofs( +/// This function queues all storage proof tasks to the worker pool but returns immediately +/// with receivers, allowing the account trie walk to proceed in parallel with storage proof +/// computation. This enables interleaved parallelism for better performance. +/// +/// Propagates errors up if queuing fails. Receivers must be consumed by the caller. +fn queue_storage_proofs( _proof_tx: &ProofTaskTx, account_proof_handle: &ProofTaskManagerHandle, - targets: &MultiProofTargets, + targets: MultiProofTargets, prefix_sets: &TriePrefixSets, with_branch_node_masks: bool, multi_added_removed_keys: Option>, -) -> Result, ParallelStateRootError> +) -> Result>, ParallelStateRootError> where Tx: DbTx, { - let mut storage_proofs = B256Map::with_capacity_and_hasher(targets.len(), Default::default()); - let mut pending = Vec::with_capacity(targets.len()); // (address, receiver) + let mut storage_proof_receivers = + B256Map::with_capacity_and_hasher(targets.len(), Default::default()); // Queue all storage proofs to worker pool - for (hashed_address, target_slots) in targets.iter() { + for (hashed_address, target_slots) in targets { let prefix_set = - prefix_sets.storage_prefix_sets.get(hashed_address).cloned().unwrap_or_default(); + prefix_sets.storage_prefix_sets.get(&hashed_address).cloned().unwrap_or_default(); // Always queue a storage proof so we obtain the storage root even when no slots are // requested. let input = StorageProofInput::new( - *hashed_address, + hashed_address, prefix_set, - target_slots.clone(), + target_slots, with_branch_node_masks, multi_added_removed_keys.clone(), ); @@ -660,31 +678,10 @@ where }, )?; - pending.push((*hashed_address, receiver)); - } - - // Collect all results - for (hashed_address, receiver) in pending { - // If receiving fails or worker returns error, propagate up (no fallback) - let proof = receiver - .recv() - .map_err(|_| { - ParallelStateRootError::Other(format!( - "Storage proof channel dropped for {}: worker died or pool shutdown", - hashed_address - )) - })? - .map_err(|e| { - ParallelStateRootError::Other(format!( - "Storage proof computation failed for {}: {}", - hashed_address, e - )) - })?; - - storage_proofs.insert(hashed_address, proof); + storage_proof_receivers.insert(hashed_address, receiver); } - Ok(storage_proofs) + Ok(storage_proof_receivers) } impl ProofTaskManager { @@ -704,7 +701,10 @@ impl ProofTaskManager { where Factory: DatabaseProviderFactory, { - let (proof_task_tx, proof_task_rx) = channel(); + // Use unbounded channel for the router to prevent account workers from blocking + // when queuing storage proofs. Account workers queue many storage proofs through + // this channel, and blocking on a bounded channel wastes parallel worker capacity. + let (proof_task_tx, proof_task_rx) = unbounded(); // Use unbounded channel to ensure all storage operations are queued to workers. // This maintains transaction reuse benefits and avoids fallback to on-demand execution. @@ -1188,20 +1188,26 @@ pub enum ProofTaskKind { #[derive(Debug)] pub struct ProofTaskManagerHandle { /// The sender for the proof task manager. - sender: Sender, + sender: CrossbeamSender, /// The number of active handles. active_handles: Arc, } impl ProofTaskManagerHandle { /// Creates a new [`ProofTaskManagerHandle`] with the given sender. - pub fn new(sender: Sender, active_handles: Arc) -> Self { + pub fn new( + sender: CrossbeamSender, + active_handles: Arc, + ) -> Self { active_handles.fetch_add(1, Ordering::SeqCst); Self { sender, active_handles } } /// Queues a task to the proof task manager. - pub fn queue_task(&self, task: ProofTaskKind) -> Result<(), SendError> { + pub fn queue_task( + &self, + task: ProofTaskKind, + ) -> Result<(), crossbeam_channel::SendError> { self.sender.send(ProofTaskMessage::QueueTask(task)) } @@ -1246,14 +1252,14 @@ pub enum ProofTaskTrieNodeProvider { /// Blinded account trie node provider. AccountNode { /// Sender to the proof task. - sender: Sender, + sender: CrossbeamSender, }, /// Blinded storage trie node provider. StorageNode { /// Target account. account: B256, /// Sender to the proof task. - sender: Sender, + sender: CrossbeamSender, }, } From e427485e1607fafa968c90e47133a3e4bbadc432 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 04:39:23 +0000 Subject: [PATCH 085/144] Removed the unused _proof_tx parameter from queue_storage_proofs --- crates/trie/parallel/src/proof_task.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 5dc8421258b..6b2f6384c47 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -379,7 +379,6 @@ fn account_worker_loop( tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); let storage_proof_receivers = match queue_storage_proofs( - &proof_tx, &storage_proof_handle, input.targets.clone(), &input.prefix_sets, @@ -637,17 +636,13 @@ where /// computation. This enables interleaved parallelism for better performance. /// /// Propagates errors up if queuing fails. Receivers must be consumed by the caller. -fn queue_storage_proofs( - _proof_tx: &ProofTaskTx, +fn queue_storage_proofs( account_proof_handle: &ProofTaskManagerHandle, targets: MultiProofTargets, prefix_sets: &TriePrefixSets, with_branch_node_masks: bool, multi_added_removed_keys: Option>, -) -> Result>, ParallelStateRootError> -where - Tx: DbTx, -{ +) -> Result>, ParallelStateRootError> { let mut storage_proof_receivers = B256Map::with_capacity_and_hasher(targets.len(), Default::default()); From 5647e22989e2e01e4f58da0cef0d669dbca739d6 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 05:16:04 +0000 Subject: [PATCH 086/144] avoid extra alloc on map - Before this change the code did: StorageRootTargets::new( prefix_sets.account_prefix_set.iter().map(|n| B256::from_slice(&nibbles.pack())), prefix_sets.storage_prefix_sets.clone(), ).len() StorageRootTargets::new(..) builds a brand-new B256Map by: - cloning every entry in storage_prefix_sets (each PrefixSet can hold dozens of nibble prefixes), and - allocating a fresh B256Map to hold them. - We only needed the resulting length, so those clones/allocations were immediately thrown away. --- crates/trie/parallel/src/proof.rs | 9 ++++----- .../trie/parallel/src/storage_root_targets.rs | 17 +++++++++++++++++ 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index d547f9b69d7..cf7f1e969e6 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -182,11 +182,10 @@ where // Extend prefix sets with targets let prefix_sets = Self::extend_prefix_sets_with_targets(&self.prefix_sets, &targets); - let storage_root_targets_len = StorageRootTargets::new( - prefix_sets.account_prefix_set.iter().map(|nibbles| B256::from_slice(&nibbles.pack())), - prefix_sets.storage_prefix_sets.clone(), - ) - .len(); + let storage_root_targets_len = StorageRootTargets::count( + &prefix_sets.account_prefix_set, + &prefix_sets.storage_prefix_sets, + ); trace!( target: "trie::parallel_proof", diff --git a/crates/trie/parallel/src/storage_root_targets.rs b/crates/trie/parallel/src/storage_root_targets.rs index f844b70fca5..786f93030ca 100644 --- a/crates/trie/parallel/src/storage_root_targets.rs +++ b/crates/trie/parallel/src/storage_root_targets.rs @@ -24,6 +24,23 @@ impl StorageRootTargets { .collect(), ) } + + /// Returns the total number of unique storage root targets without allocating new maps. + pub fn count( + account_prefix_set: &PrefixSet, + storage_prefix_sets: &B256Map, + ) -> usize { + let mut count = storage_prefix_sets.len(); + + for nibbles in account_prefix_set.iter() { + let hashed_address = B256::from_slice(&nibbles.pack()); + if !storage_prefix_sets.contains_key(&hashed_address) { + count += 1; + } + } + + count + } } impl IntoIterator for StorageRootTargets { From b4d109bfa061815b5d3639b9327d2c2e351b192a Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 05:20:45 +0000 Subject: [PATCH 087/144] clippy --- crates/trie/parallel/src/storage_root_targets.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/storage_root_targets.rs b/crates/trie/parallel/src/storage_root_targets.rs index 786f93030ca..0c6d9f43498 100644 --- a/crates/trie/parallel/src/storage_root_targets.rs +++ b/crates/trie/parallel/src/storage_root_targets.rs @@ -32,7 +32,7 @@ impl StorageRootTargets { ) -> usize { let mut count = storage_prefix_sets.len(); - for nibbles in account_prefix_set.iter() { + for nibbles in account_prefix_set { let hashed_address = B256::from_slice(&nibbles.pack()); if !storage_prefix_sets.contains_key(&hashed_address) { count += 1; From 84ad2180e3f762225dc04f0cf854972545cc8614 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 05:22:25 +0000 Subject: [PATCH 088/144] refactor: update account worker loop to use crossbeam sender for storage jobs - Replaced the `storage_proof_handle` parameter with `storage_work_tx` in the `account_worker_loop` function to facilitate direct communication with the storage worker queue. - Modified the `queue_storage_proofs` function to accept a mutable reference to `storage_prefix_sets`, improving efficiency by avoiding unnecessary clones. - Enhanced the initialization of `collected_decoded_storages` to use pre-allocated capacity, optimizing memory usage during proof collection. --- crates/trie/parallel/src/proof_task.rs | 78 +++++++++++--------------- 1 file changed, 33 insertions(+), 45 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 6b2f6384c47..c6a83ac481f 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -330,7 +330,7 @@ fn storage_worker_loop( fn account_worker_loop( proof_tx: ProofTaskTx, work_rx: CrossbeamReceiver, - storage_proof_handle: ProofTaskManagerHandle, + storage_work_tx: CrossbeamSender, worker_id: usize, ) where Tx: DbTx, @@ -367,23 +367,21 @@ fn account_worker_loop( let proof_start = Instant::now(); let mut tracker = ParallelTrieTracker::default(); - let storage_root_targets_len = StorageRootTargets::new( - input - .prefix_sets - .account_prefix_set - .iter() - .map(|nibbles| B256::from_slice(&nibbles.pack())), - input.prefix_sets.storage_prefix_sets.clone(), - ) - .len(); + let mut storage_prefix_sets = + std::mem::take(&mut input.prefix_sets.storage_prefix_sets); + + let storage_root_targets_len = StorageRootTargets::count( + &input.prefix_sets.account_prefix_set, + &storage_prefix_sets, + ); tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); let storage_proof_receivers = match queue_storage_proofs( - &storage_proof_handle, - input.targets.clone(), - &input.prefix_sets, + &storage_work_tx, + &input.targets, + &mut storage_prefix_sets, input.collect_branch_node_masks, - input.multi_added_removed_keys.clone(), + input.multi_added_removed_keys.as_ref(), ) { Ok(receivers) => receivers, Err(error) => { @@ -521,10 +519,10 @@ where .with_proof_retainer(retainer) .with_updates(collect_branch_node_masks); - // Initialize all storage multiproofs as empty. - // Storage multiproofs for non empty tries will be overwritten if necessary. + // Initialize storage multiproofs map with pre-allocated capacity. + // Proofs will be inserted as they're consumed from receivers during trie walk. let mut collected_decoded_storages: B256Map = - targets.keys().map(|key| (*key, DecodedStorageMultiProof::empty())).collect(); + B256Map::with_capacity_and_hasher(targets.len(), Default::default()); let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); let mut account_node_iter = TrieNodeIter::state_trie( walker, @@ -551,9 +549,7 @@ where ) })??; let root = proof.root; - if let Some(entry) = collected_decoded_storages.get_mut(&hashed_address) { - *entry = proof; - } + collected_decoded_storages.insert(hashed_address, proof); root } // Since we do not store all intermediate nodes in the database, there might @@ -637,43 +633,42 @@ where /// /// Propagates errors up if queuing fails. Receivers must be consumed by the caller. fn queue_storage_proofs( - account_proof_handle: &ProofTaskManagerHandle, - targets: MultiProofTargets, - prefix_sets: &TriePrefixSets, + storage_work_tx: &CrossbeamSender, + targets: &MultiProofTargets, + storage_prefix_sets: &mut B256Map, with_branch_node_masks: bool, - multi_added_removed_keys: Option>, + multi_added_removed_keys: Option<&Arc>, ) -> Result>, ParallelStateRootError> { let mut storage_proof_receivers = B256Map::with_capacity_and_hasher(targets.len(), Default::default()); // Queue all storage proofs to worker pool - for (hashed_address, target_slots) in targets { - let prefix_set = - prefix_sets.storage_prefix_sets.get(&hashed_address).cloned().unwrap_or_default(); + for (hashed_address, target_slots) in targets.iter() { + let prefix_set = storage_prefix_sets.remove(hashed_address).unwrap_or_default(); // Always queue a storage proof so we obtain the storage root even when no slots are // requested. let input = StorageProofInput::new( - hashed_address, + *hashed_address, prefix_set, - target_slots, + target_slots.clone(), with_branch_node_masks, - multi_added_removed_keys.clone(), + multi_added_removed_keys.cloned(), ); let (sender, receiver) = channel(); // If queuing fails, propagate error up (no fallback) - account_proof_handle.queue_task(ProofTaskKind::StorageProof(input, sender)).map_err( - |_| { + storage_work_tx + .send(StorageWorkerJob::StorageProof { input, result_sender: sender }) + .map_err(|_| { ParallelStateRootError::Other(format!( "Failed to queue storage proof for {}: storage worker pool unavailable", hashed_address )) - }, - )?; + })?; - storage_proof_receivers.insert(hashed_address, receiver); + storage_proof_receivers.insert(*hashed_address, receiver); } Ok(storage_proof_receivers) @@ -724,10 +719,7 @@ impl ProofTaskManager { storage_worker_loop, )?; - // Spawn account workers with storage handle - // Create a separate counter for internal worker handles to avoid triggering shutdown - // when workers drop their handles (workers run forever until channel closes) - let worker_handles_counter = Arc::new(AtomicUsize::new(0)); + // Spawn account workers with direct access to the storage worker queue. let spawned_account_workers = Self::spawn_worker_pool( &executor, &view, @@ -736,13 +728,9 @@ impl ProofTaskManager { account_work_rx, WorkerType::Account, { - let task_tx = proof_task_tx.clone(); + let storage_work_tx = storage_work_tx.clone(); move |proof_tx, work_rx, worker_id| { - let storage_handle = ProofTaskManagerHandle::new( - task_tx.clone(), - worker_handles_counter.clone(), - ); - account_worker_loop(proof_tx, work_rx, storage_handle, worker_id) + account_worker_loop(proof_tx, work_rx, storage_work_tx.clone(), worker_id) } }, )?; From 9e177f44a5892282f87ac16b8de6721a64d04c5a Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 05:33:05 +0000 Subject: [PATCH 089/144] feat: add default account worker count for optimized I/O-bound coordination - Introduced a new function `default_account_worker_count` to calculate the default number of account worker threads, set to 1.5 times the storage worker count. - Updated the `TreeConfig` implementation to use the new account worker count function, enhancing performance for I/O-bound tasks related to storage proof collection and account trie traversal. --- crates/engine/primitives/src/config.rs | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 079f48f7a84..93a1c59955c 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -22,6 +22,16 @@ fn default_storage_worker_count() -> usize { } } +/// Returns the default number of account worker threads optimized for I/O-bound coordination. +/// +/// Account workers primarily coordinate storage proof collection and account trie traversal. +/// They spend significant time blocked on `receiver.recv()` calls waiting for storage proofs, +/// so we use higher concurrency (1.5x storage workers) to maximize throughput and overlap. +/// While storage workers are CPU-bound, account workers are I/O-bound coordinators. +fn default_account_worker_count() -> usize { + (default_storage_worker_count() * 3) / 2 +} + /// The size of proof targets chunk to spawn in one multiproof calculation. pub const DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE: usize = 10; @@ -153,7 +163,7 @@ impl Default for TreeConfig { prewarm_max_concurrency: DEFAULT_PREWARM_MAX_CONCURRENCY, allow_unwind_canonical_header: false, storage_worker_count: default_storage_worker_count(), - account_worker_count: default_storage_worker_count(), + account_worker_count: default_account_worker_count(), } } } From 459636625707c74de8e7130fa3f89a9c1bd91b69 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 07:40:56 +0000 Subject: [PATCH 090/144] refactor: clarify comments for proof task handles in MultiproofManager - Updated comments for `proof_task_handle` parameters in the `MultiproofManager` initialization to specify their roles for storage and account proof workers, improving code clarity and maintainability. --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index aac9f3cddeb..9045f057d24 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -728,8 +728,8 @@ where multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle.clone(), - proof_task_handle, + proof_task_handle.clone(), // handle for storage proof workers + proof_task_handle, // handle for account proof workers max_concurrency, ), metrics, From 19a9e8b1d087bd6af34302866fc4eee7fd0148b3 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 07:45:26 +0000 Subject: [PATCH 091/144] comment --- crates/trie/parallel/src/proof_task.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index c6a83ac481f..5393cd6e841 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -110,9 +110,9 @@ enum StorageWorkerJob { /// /// # Architecture /// -/// This manager handles three distinct execution paths: +/// This manager operates two distinct worker pools for parallel trie operations: /// -/// **Worker Pools** (for all trie operations): +/// **Worker Pools**: /// - Pre-spawned workers with dedicated long-lived transactions /// - **Storage pool**: Handles `StorageProof` and `BlindedStorageNode` requests /// - **Account pool**: Handles `AccountMultiproof` and `BlindedAccountNode` requests, delegates From 2091f0d18f766cc124b364902cf996eb2cbc3897 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 08:05:14 +0000 Subject: [PATCH 092/144] expect workers to be avaliable --- crates/trie/parallel/src/proof_task.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 5393cd6e841..811a202b25e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -677,6 +677,10 @@ fn queue_storage_proofs( impl ProofTaskManager { /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. /// + /// This manager coordinates both storage and account worker pools: + /// - Storage workers handle `StorageProof` and `BlindedStorageNode` requests + /// - Account workers handle `AccountMultiproof` and `BlindedAccountNode` requests + /// /// The `storage_worker_count` determines how many storage workers to spawn, and /// `account_worker_count` determines how many account workers to spawn. /// Returns an error if the underlying provider fails to create the transactions required for @@ -838,7 +842,7 @@ impl ProofTaskManager { input, result_sender: sender, }) - .expect("storage workers are running until Terminate"); + .expect("storage worker pool should be available"); tracing::trace!( target: "trie::proof_task", @@ -858,7 +862,7 @@ impl ProofTaskManager { path, result_sender: sender, }) - .expect("storage workers are running until Terminate"); + .expect("storage worker pool should be available"); tracing::trace!( target: "trie::proof_task", @@ -879,7 +883,7 @@ impl ProofTaskManager { path, result_sender: sender, }) - .expect("account workers are running until Terminate"); + .expect("account worker pool should be available"); tracing::trace!( target: "trie::proof_task", @@ -894,7 +898,7 @@ impl ProofTaskManager { input, result_sender: sender, }) - .expect("account workers are running until Terminate"); + .expect("account worker pool should be available"); tracing::trace!( target: "trie::proof_task", From 4ca404e51733af40f978a165a682744c2fe7db6a Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 14:35:13 +0000 Subject: [PATCH 093/144] bump up workers --- crates/engine/primitives/src/config.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index b719c5e10cd..b2f8da4d424 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -10,15 +10,14 @@ pub const DEFAULT_MEMORY_BLOCK_BUFFER_TARGET: u64 = 0; pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; /// Returns the default number of storage worker threads based on available parallelism. -/// Defaults to half of available parallelism, clamped between 2 and 8. fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] { - std::thread::available_parallelism().map(|n| (n.get() / 2).clamp(2, 8)).unwrap_or(4) + std::thread::available_parallelism().map(|n| (n.get() * 2).clamp(2, 64)).unwrap_or(8) } #[cfg(not(feature = "std"))] { - 4 + 8 } } From d66ed613973f2a46f82ddf88fa61bcb7cec61bf3 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Thu, 9 Oct 2025 14:57:53 +0000 Subject: [PATCH 094/144] cli flag for storage --- crates/node/core/src/args/engine.rs | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index 88179a6b40e..5acf47ae740 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -108,6 +108,11 @@ pub struct EngineArgs { /// See `TreeConfig::unwind_canonical_header` for more details. #[arg(long = "engine.allow-unwind-canonical-header", default_value = "false")] pub allow_unwind_canonical_header: bool, + + /// Configure the number of storage proof worker threads. + /// If not specified, defaults to 2x available parallelism, clamped between 2 and 64. + #[arg(long = "engine.storage-worker-count")] + pub storage_worker_count: Option, } #[allow(deprecated)] @@ -134,6 +139,7 @@ impl Default for EngineArgs { state_root_fallback: false, always_process_payload_attributes_on_canonical_head: false, allow_unwind_canonical_header: false, + storage_worker_count: None, } } } @@ -141,7 +147,7 @@ impl Default for EngineArgs { impl EngineArgs { /// Creates a [`TreeConfig`] from the engine arguments. pub fn tree_config(&self) -> TreeConfig { - TreeConfig::default() + let mut config = TreeConfig::default() .with_persistence_threshold(self.persistence_threshold) .with_memory_block_buffer_target(self.memory_block_buffer_target) .with_legacy_state_root(self.legacy_state_root_task_enabled) @@ -159,7 +165,13 @@ impl EngineArgs { .with_always_process_payload_attributes_on_canonical_head( self.always_process_payload_attributes_on_canonical_head, ) - .with_unwind_canonical_header(self.allow_unwind_canonical_header) + .with_unwind_canonical_header(self.allow_unwind_canonical_header); + + if let Some(count) = self.storage_worker_count { + config = config.with_storage_worker_count(count); + } + + config } } From 4aba3de0764bf71b44eb3b3022d743c723a10e52 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 00:19:27 +0000 Subject: [PATCH 095/144] docs: update CLI reference for storage-worker-count flag --- docs/vocs/docs/pages/cli/reth/node.mdx | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/vocs/docs/pages/cli/reth/node.mdx b/docs/vocs/docs/pages/cli/reth/node.mdx index 2021b342d62..07ba5e5d84e 100644 --- a/docs/vocs/docs/pages/cli/reth/node.mdx +++ b/docs/vocs/docs/pages/cli/reth/node.mdx @@ -864,6 +864,9 @@ Engine: --engine.allow-unwind-canonical-header Allow unwinding canonical header to ancestor during forkchoice updates. See `TreeConfig::unwind_canonical_header` for more details + --engine.storage-worker-count + Configure the number of storage proof worker threads. If not specified, defaults to 2x available parallelism, clamped between 2 and 64 + ERA: --era.enable Enable import from ERA1 files From 8e64738b03bea504fcca0741536f2275f1e41b93 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 00:21:52 +0000 Subject: [PATCH 096/144] docs: clarify storage-worker-count uses Tokio blocking pool --- crates/node/core/src/args/engine.rs | 2 +- docs/vocs/docs/pages/cli/reth/node.mdx | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index 5acf47ae740..2298b28f9ce 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -109,7 +109,7 @@ pub struct EngineArgs { #[arg(long = "engine.allow-unwind-canonical-header", default_value = "false")] pub allow_unwind_canonical_header: bool, - /// Configure the number of storage proof worker threads. + /// Configure the number of storage proof workers in the Tokio blocking pool. /// If not specified, defaults to 2x available parallelism, clamped between 2 and 64. #[arg(long = "engine.storage-worker-count")] pub storage_worker_count: Option, diff --git a/docs/vocs/docs/pages/cli/reth/node.mdx b/docs/vocs/docs/pages/cli/reth/node.mdx index 07ba5e5d84e..394854f7246 100644 --- a/docs/vocs/docs/pages/cli/reth/node.mdx +++ b/docs/vocs/docs/pages/cli/reth/node.mdx @@ -865,7 +865,7 @@ Engine: Allow unwinding canonical header to ancestor during forkchoice updates. See `TreeConfig::unwind_canonical_header` for more details --engine.storage-worker-count - Configure the number of storage proof worker threads. If not specified, defaults to 2x available parallelism, clamped between 2 and 64 + Configure the number of storage proof workers in the Tokio blocking pool. If not specified, defaults to 2x available parallelism, clamped between 2 and 64 ERA: --era.enable From 5d109344c0559c6bf3a246656e25034c7e63bedc Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 04:24:25 +0000 Subject: [PATCH 097/144] feat: add storage and account worker count configuration options - Introduced `storage_worker_count` and `account_worker_count` fields in `EngineArgs` to allow configuration of the number of workers in the Tokio blocking pool. - Updated the `tree_config` method to incorporate these new fields, enhancing flexibility in managing proof worker resources. - Updated CLI documentation to reflect the new configuration options for improved user guidance. --- crates/node/core/src/args/engine.rs | 26 ++++++++++++++++++++++++-- docs/vocs/docs/pages/cli/reth/node.mdx | 6 ++++++ 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index 88179a6b40e..fb98847d8c7 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -108,6 +108,16 @@ pub struct EngineArgs { /// See `TreeConfig::unwind_canonical_header` for more details. #[arg(long = "engine.allow-unwind-canonical-header", default_value = "false")] pub allow_unwind_canonical_header: bool, + + /// Configure the number of storage proof workers in the Tokio blocking pool. + /// If not specified, defaults to 2x available parallelism, clamped between 2 and 64. + #[arg(long = "engine.storage-worker-count")] + pub storage_worker_count: Option, + + /// Configure the number of account proof workers in the Tokio blocking pool. + /// If not specified, defaults to 1.5x storage workers. + #[arg(long = "engine.account-worker-count")] + pub account_worker_count: Option, } #[allow(deprecated)] @@ -134,6 +144,8 @@ impl Default for EngineArgs { state_root_fallback: false, always_process_payload_attributes_on_canonical_head: false, allow_unwind_canonical_header: false, + storage_worker_count: None, + account_worker_count: None, } } } @@ -141,7 +153,7 @@ impl Default for EngineArgs { impl EngineArgs { /// Creates a [`TreeConfig`] from the engine arguments. pub fn tree_config(&self) -> TreeConfig { - TreeConfig::default() + let mut config = TreeConfig::default() .with_persistence_threshold(self.persistence_threshold) .with_memory_block_buffer_target(self.memory_block_buffer_target) .with_legacy_state_root(self.legacy_state_root_task_enabled) @@ -159,7 +171,17 @@ impl EngineArgs { .with_always_process_payload_attributes_on_canonical_head( self.always_process_payload_attributes_on_canonical_head, ) - .with_unwind_canonical_header(self.allow_unwind_canonical_header) + .with_unwind_canonical_header(self.allow_unwind_canonical_header); + + if let Some(count) = self.storage_worker_count { + config = config.with_storage_worker_count(count); + } + + if let Some(count) = self.account_worker_count { + config = config.with_account_worker_count(count); + } + + config } } diff --git a/docs/vocs/docs/pages/cli/reth/node.mdx b/docs/vocs/docs/pages/cli/reth/node.mdx index 2021b342d62..6fc9c054f1f 100644 --- a/docs/vocs/docs/pages/cli/reth/node.mdx +++ b/docs/vocs/docs/pages/cli/reth/node.mdx @@ -864,6 +864,12 @@ Engine: --engine.allow-unwind-canonical-header Allow unwinding canonical header to ancestor during forkchoice updates. See `TreeConfig::unwind_canonical_header` for more details + --engine.storage-worker-count + Configure the number of storage proof workers in the Tokio blocking pool. If not specified, defaults to 2x available parallelism, clamped between 2 and 64 + + --engine.account-worker-count + Configure the number of account proof workers in the Tokio blocking pool. If not specified, defaults to 1.5x storage workers + ERA: --era.enable Enable import from ERA1 files From 05f177adf9049e27d9baac9ea06f56f7532236c4 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 04:24:41 +0000 Subject: [PATCH 098/144] added a todo --- crates/trie/parallel/src/proof.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index cf7f1e969e6..36c49c038c9 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -48,6 +48,7 @@ pub struct ParallelProof { /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, /// Marker to keep the Factory type parameter. + /// TODO: Remove this field if the Factory generic is not needed in the future. _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics, From aae5c7a5ac293fe8ecd27f12fee35c0312db3ddb Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 04:32:58 +0000 Subject: [PATCH 099/144] refactor: simplify account multiproof function parameters - Introduced `AccountMultiproofParams` struct to encapsulate parameters for building account multiproofs, reducing the number of arguments in the `build_account_multiproof_with_storage_roots` function. - Updated the `account_worker_loop` to use the new struct, improving code readability and maintainability. --- crates/trie/parallel/src/proof_task.rs | 59 +++++++++++++++++--------- 1 file changed, 38 insertions(+), 21 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 811a202b25e..82a9f52643e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -395,15 +395,19 @@ fn account_worker_loop( let account_prefix_set = std::mem::take(&mut input.prefix_sets.account_prefix_set); + let ctx = AccountMultiproofParams { + targets: &input.targets, + prefix_set: account_prefix_set, + collect_branch_node_masks: input.collect_branch_node_masks, + multi_added_removed_keys: input.multi_added_removed_keys.as_ref(), + storage_proof_receivers, + missed_leaves_storage_roots, + }; + let result = build_account_multiproof_with_storage_roots( trie_cursor_factory.clone(), hashed_cursor_factory.clone(), - &input.targets, - account_prefix_set, - input.collect_branch_node_masks, - input.multi_added_removed_keys.as_ref(), - storage_proof_receivers, - missed_leaves_storage_roots, + ctx, &mut tracker, ); @@ -475,6 +479,22 @@ fn account_worker_loop( ); } +/// Parameters for building an account multiproof with pre-computed storage roots. +struct AccountMultiproofParams<'a> { + /// The targets for which to compute the multiproof. + targets: &'a MultiProofTargets, + /// The prefix set for the account trie walk. + prefix_set: PrefixSet, + /// Whether or not to collect branch node masks. + collect_branch_node_masks: bool, + /// Provided by the user to give the necessary context to retain extra proofs. + multi_added_removed_keys: Option<&'a Arc>, + /// Receivers for storage proofs being computed in parallel. + storage_proof_receivers: B256Map>, + /// Cached storage proof roots for missed leaves encountered during account trie walk. + missed_leaves_storage_roots: &'a DashMap, +} + /// Builds an account multiproof by consuming storage proof receivers lazily during trie walk. /// /// This is a helper function used by account workers to build the account subtree proof @@ -482,16 +502,10 @@ fn account_worker_loop( /// enabling interleaved parallelism between account trie traversal and storage proof computation. /// /// Returns a `DecodedMultiProof` containing the account subtree and storage proofs. -#[allow(clippy::too_many_arguments)] fn build_account_multiproof_with_storage_roots( trie_cursor_factory: C, hashed_cursor_factory: H, - targets: &MultiProofTargets, - prefix_set: PrefixSet, - collect_branch_node_masks: bool, - multi_added_removed_keys: Option<&Arc>, - mut storage_proof_receivers: B256Map>, - missed_leaves_storage_roots: &DashMap, + ctx: AccountMultiproofParams<'_>, tracker: &mut ParallelTrieTracker, ) -> Result where @@ -499,36 +513,39 @@ where H: HashedCursorFactory + Clone, { let accounts_added_removed_keys = - multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); + ctx.multi_added_removed_keys.as_ref().map(|keys| keys.get_accounts()); // Create the walker. let walker = TrieWalker::<_>::state_trie( trie_cursor_factory.account_trie_cursor().map_err(ProviderError::Database)?, - prefix_set, + ctx.prefix_set, ) .with_added_removed_keys(accounts_added_removed_keys) .with_deletions_retained(true); // Create a hash builder to rebuild the root node since it is not available in the database. - let retainer = targets + let retainer = ctx + .targets .keys() .map(Nibbles::unpack) .collect::() .with_added_removed_keys(accounts_added_removed_keys); let mut hash_builder = HashBuilder::default() .with_proof_retainer(retainer) - .with_updates(collect_branch_node_masks); + .with_updates(ctx.collect_branch_node_masks); // Initialize storage multiproofs map with pre-allocated capacity. // Proofs will be inserted as they're consumed from receivers during trie walk. let mut collected_decoded_storages: B256Map = - B256Map::with_capacity_and_hasher(targets.len(), Default::default()); + B256Map::with_capacity_and_hasher(ctx.targets.len(), Default::default()); let mut account_rlp = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE); let mut account_node_iter = TrieNodeIter::state_trie( walker, hashed_cursor_factory.hashed_account_cursor().map_err(ProviderError::Database)?, ); + let mut storage_proof_receivers = ctx.storage_proof_receivers; + while let Some(account_node) = account_node_iter.try_next().map_err(ProviderError::Database)? { match account_node { TrieElement::Branch(node) => { @@ -557,7 +574,7 @@ where None => { tracker.inc_missed_leaves(); - match missed_leaves_storage_roots.entry(hashed_address) { + match ctx.missed_leaves_storage_roots.entry(hashed_address) { dashmap::Entry::Occupied(occ) => *occ.get(), dashmap::Entry::Vacant(vac) => { let root = StorageProof::new_hashed( @@ -567,7 +584,7 @@ where ) .with_prefix_set_mut(Default::default()) .storage_multiproof( - targets.get(&hashed_address).cloned().unwrap_or_default(), + ctx.targets.get(&hashed_address).cloned().unwrap_or_default(), ) .map_err(|e| { ParallelStateRootError::StorageRoot( @@ -607,7 +624,7 @@ where let account_subtree_raw_nodes = hash_builder.take_proof_nodes(); let decoded_account_subtree = DecodedProofNodes::try_from(account_subtree_raw_nodes)?; - let (branch_node_hash_masks, branch_node_tree_masks) = if collect_branch_node_masks { + let (branch_node_hash_masks, branch_node_tree_masks) = if ctx.collect_branch_node_masks { let updated_branch_nodes = hash_builder.updated_branch_nodes.unwrap_or_default(); ( updated_branch_nodes.iter().map(|(path, node)| (*path, node.hash_mask)).collect(), From 1bc2862d17f87b8360e82cca94238019f2fef734 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 04:35:16 +0000 Subject: [PATCH 100/144] move the AccountMultiproofParams struct to just below AccountMultiproofInput so all account multiproof types live together. --- crates/trie/parallel/src/proof_task.rs | 32 +++++++++++++------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 82a9f52643e..cc8ee47636d 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -479,22 +479,6 @@ fn account_worker_loop( ); } -/// Parameters for building an account multiproof with pre-computed storage roots. -struct AccountMultiproofParams<'a> { - /// The targets for which to compute the multiproof. - targets: &'a MultiProofTargets, - /// The prefix set for the account trie walk. - prefix_set: PrefixSet, - /// Whether or not to collect branch node masks. - collect_branch_node_masks: bool, - /// Provided by the user to give the necessary context to retain extra proofs. - multi_added_removed_keys: Option<&'a Arc>, - /// Receivers for storage proofs being computed in parallel. - storage_proof_receivers: B256Map>, - /// Cached storage proof roots for missed leaves encountered during account trie walk. - missed_leaves_storage_roots: &'a DashMap, -} - /// Builds an account multiproof by consuming storage proof receivers lazily during trie walk. /// /// This is a helper function used by account workers to build the account subtree proof @@ -1115,6 +1099,22 @@ pub struct AccountMultiproofInput { pub missed_leaves_storage_roots: Arc>, } +/// Parameters for building an account multiproof with pre-computed storage roots. +struct AccountMultiproofParams<'a> { + /// The targets for which to compute the multiproof. + targets: &'a MultiProofTargets, + /// The prefix set for the account trie walk. + prefix_set: PrefixSet, + /// Whether or not to collect branch node masks. + collect_branch_node_masks: bool, + /// Provided by the user to give the necessary context to retain extra proofs. + multi_added_removed_keys: Option<&'a Arc>, + /// Receivers for storage proofs being computed in parallel. + storage_proof_receivers: B256Map>, + /// Cached storage proof roots for missed leaves encountered during account trie walk. + missed_leaves_storage_roots: &'a DashMap, +} + /// Internal message for account workers. /// /// This is NOT exposed publicly. External callers use `ProofTaskKind::AccountMultiproof` or From f23cfab54faf7e4cb58ca6a9807cf9d120fb3e0d Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 04:56:05 +0000 Subject: [PATCH 101/144] refactor: addressed dan's comment on making structs that encapsulate loops: - an account worker / loop - a storage worker / loop - streamline worker spawning functions in ProofTaskManager - Consolidated worker spawning logic by renaming and simplifying the `spawn_worker_pool` function to `spawn_storage_workers` and `spawn_account_workers`, enhancing clarity and maintainability. - Removed the `WorkerType` enum and associated logic, as the worker type is now inferred from the context, reducing complexity. - Updated comments to better describe the purpose and parameters of the new functions, improving code documentation. --- crates/trie/parallel/src/proof_task.rs | 133 ++++++++++++++----------- 1 file changed, 76 insertions(+), 57 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index cc8ee47636d..18062747901 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -23,8 +23,8 @@ use dashmap::DashMap; use reth_db_api::transaction::DbTx; use reth_execution_errors::SparseTrieError; use reth_provider::{ - providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, FactoryTx, - ProviderError, ProviderResult, + providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, ProviderError, + ProviderResult, }; use reth_storage_errors::db::DatabaseError; use reth_trie::{ @@ -64,24 +64,6 @@ type TrieNodeProviderResult = Result, SparseTrieError>; type AccountMultiproofResult = Result<(DecodedMultiProof, ParallelTrieStats), ParallelStateRootError>; -/// Worker type identifier -#[derive(Debug)] -enum WorkerType { - /// Storage proof worker - Storage, - /// Account multiproof worker - Account, -} - -impl std::fmt::Display for WorkerType { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Self::Storage => write!(f, "Storage"), - Self::Account => write!(f, "Account"), - } - } -} - /// Internal message for storage workers. /// /// This is NOT exposed publicly. External callers use `ProofTaskKind::StorageProof` or @@ -714,30 +696,22 @@ impl ProofTaskManager { ); // Spawn storage workers - let spawned_storage_workers = Self::spawn_worker_pool( + let spawned_storage_workers = Self::spawn_storage_workers( &executor, &view, &task_ctx, storage_worker_count, storage_work_rx, - WorkerType::Storage, - storage_worker_loop, )?; - // Spawn account workers with direct access to the storage worker queue. - let spawned_account_workers = Self::spawn_worker_pool( + // Spawn account workers with direct access to the storage worker queue + let spawned_account_workers = Self::spawn_account_workers( &executor, &view, &task_ctx, account_worker_count, account_work_rx, - WorkerType::Account, - { - let storage_work_tx = storage_work_tx.clone(); - move |proof_tx, work_rx, worker_id| { - account_worker_loop(proof_tx, work_rx, storage_work_tx.clone(), worker_id) - } - }, + storage_work_tx.clone(), )?; Ok(Self { @@ -759,48 +733,94 @@ impl ProofTaskManager { ProofTaskManagerHandle::new(self.proof_task_tx.clone(), self.active_handles.clone()) } - /// Spawns a pool of workers with dedicated database transactions. + /// Spawns a pool of storage workers with dedicated database transactions. /// - /// # Type Parameters - /// - `Job`: The job type the workers will process - /// - `F`: The worker loop function type + /// Each worker receives `StorageWorkerJob` from the channel and processes storage proofs + /// and blinded storage node requests using a dedicated long-lived transaction. /// /// # Parameters - /// - `worker_count`: Number of workers to spawn - /// - `work_rx`: Receiver for the worker job channel - /// - `worker_type`: Type of worker for logging - /// - `worker_fn`: The worker loop function to execute + /// - `executor`: Tokio runtime handle for spawning blocking tasks + /// - `view`: Consistent database view for creating transactions + /// - `task_ctx`: Shared context with trie updates and prefix sets + /// - `worker_count`: Number of storage workers to spawn + /// - `work_rx`: Receiver for storage worker jobs /// - /// Returns - /// The number of workers successfully spawned - fn spawn_worker_pool( + /// # Returns + /// The number of storage workers successfully spawned + fn spawn_storage_workers( executor: &Handle, view: &ConsistentDbView, task_ctx: &ProofTaskCtx, worker_count: usize, - work_rx: CrossbeamReceiver, - worker_type: WorkerType, - worker_fn: F, + work_rx: CrossbeamReceiver, ) -> ProviderResult where Factory: DatabaseProviderFactory, - Job: Send + 'static, - F: Fn(ProofTaskTx>, CrossbeamReceiver, usize) - + Send - + Clone - + 'static, { let mut spawned_workers = 0; - // spawns workers that will execute worker_fn + + for worker_id in 0..worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = work_rx.clone(); + + executor.spawn_blocking(move || { + storage_worker_loop(proof_task_tx, work_rx_clone, worker_id) + }); + + spawned_workers += 1; + + tracing::debug!( + target: "trie::proof_task", + worker_id, + spawned_workers, + "Storage worker spawned successfully" + ); + } + + Ok(spawned_workers) + } + + /// Spawns a pool of account workers with dedicated database transactions. + /// + /// Each worker receives `AccountWorkerJob` from the channel and processes account multiproofs + /// and blinded account node requests using a dedicated long-lived transaction. Account workers + /// can delegate storage proof computation to the storage worker pool. + /// + /// # Parameters + /// - `executor`: Tokio runtime handle for spawning blocking tasks + /// - `view`: Consistent database view for creating transactions + /// - `task_ctx`: Shared context with trie updates and prefix sets + /// - `worker_count`: Number of account workers to spawn + /// - `work_rx`: Receiver for account worker jobs + /// - `storage_work_tx`: Sender to delegate storage proofs to storage worker pool + /// + /// # Returns + /// The number of account workers successfully spawned + fn spawn_account_workers( + executor: &Handle, + view: &ConsistentDbView, + task_ctx: &ProofTaskCtx, + worker_count: usize, + work_rx: CrossbeamReceiver, + storage_work_tx: CrossbeamSender, + ) -> ProviderResult + where + Factory: DatabaseProviderFactory, + { + let mut spawned_workers = 0; + for worker_id in 0..worker_count { let provider_ro = view.provider_ro()?; let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); let work_rx_clone = work_rx.clone(); - let worker_fn_clone = worker_fn.clone(); + let storage_work_tx_clone = storage_work_tx.clone(); - executor - .spawn_blocking(move || worker_fn_clone(proof_task_tx, work_rx_clone, worker_id)); + executor.spawn_blocking(move || { + account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) + }); spawned_workers += 1; @@ -808,8 +828,7 @@ impl ProofTaskManager { target: "trie::proof_task", worker_id, spawned_workers, - worker_type = %worker_type, - "{} worker spawned successfully", worker_type + "Account worker spawned successfully" ); } From 47398ce49a9949f1cdd53de3182b4ca791dcce62 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 05:57:19 +0000 Subject: [PATCH 102/144] refactor: simplify ParallelProof struct by removing generic Factory type - Removed the Factory type parameter from the ParallelProof struct, streamlining its definition and implementation. - Updated the constructor and related methods to reflect this change, enhancing code clarity and maintainability. - Eliminated unused PhantomData field, reducing complexity in the struct's design. --- crates/trie/parallel/src/proof.rs | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 36c49c038c9..82554be54d3 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -9,7 +9,6 @@ use crate::{ use alloy_primitives::{map::B256Set, B256}; use dashmap::DashMap; use reth_execution_errors::StorageRootError; -use reth_provider::{BlockReader, DatabaseProviderFactory}; use reth_storage_errors::db::DatabaseError; use reth_trie::{ prefix_set::{PrefixSet, PrefixSetMut, TriePrefixSets, TriePrefixSetsMut}, @@ -28,7 +27,7 @@ use tracing::trace; /// This can collect proof for many targets in parallel, spawning a task for each hashed address /// that has proof targets. #[derive(Debug)] -pub struct ParallelProof { +pub struct ParallelProof { /// The sorted collection of cached in-memory intermediate trie nodes that /// can be reused for computation. pub nodes_sorted: Arc, @@ -47,14 +46,11 @@ pub struct ParallelProof { /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, - /// Marker to keep the Factory type parameter. - /// TODO: Remove this field if the Factory generic is not needed in the future. - _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics, } -impl ParallelProof { +impl ParallelProof { /// Create new state proof generator. pub fn new( nodes_sorted: Arc, @@ -71,7 +67,6 @@ impl ParallelProof { collect_branch_node_masks: false, multi_added_removed_keys: None, proof_task_handle, - _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics::new_with_labels(&[("type", "proof")]), } @@ -92,12 +87,6 @@ impl ParallelProof { self.multi_added_removed_keys = multi_added_removed_keys; self } -} - -impl ParallelProof -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ /// Queues a storage proof task and returns a receiver for the result. fn queue_storage_proof( &self, From d5fedd80cea67eb1e677995698fd4cfa11db939c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 05:59:03 +0000 Subject: [PATCH 103/144] removing from test --- crates/trie/parallel/src/proof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 82554be54d3..186873addd8 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -324,7 +324,7 @@ mod tests { let join_handle = rt.spawn_blocking(move || proof_task.run()); type Factory = ProviderFactory; - let parallel_result = ParallelProof::::new( + let parallel_result = ParallelProof::new( Default::default(), Default::default(), Default::default(), From 444b24c7e9a8c2d68ce8b4e99d80f5ab5db312c1 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 05:59:30 +0000 Subject: [PATCH 104/144] refactor: remove unused Factory type from proof tests - Eliminated the Factory type definition from the proof tests, simplifying the code structure. - This change contributes to improved clarity and maintainability of the test implementation. --- crates/trie/parallel/src/proof.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 186873addd8..d0fa9e7e40b 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -323,7 +323,6 @@ mod tests { // after we compute the state root let join_handle = rt.spawn_blocking(move || proof_task.run()); - type Factory = ProviderFactory; let parallel_result = ParallelProof::new( Default::default(), Default::default(), From 11130c7fcecb9c89c7bc41354fc5256c52ed545e Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 08:01:38 +0000 Subject: [PATCH 105/144] refactor: simplify MultiProofConfig and related structures - Removed the generic Factory type from MultiProofConfig and related structs, streamlining their definitions and improving code clarity. - Updated methods to reflect the removal of the Factory type, enhancing maintainability. - Adjusted the implementation of PendingMultiproofTask and its associated methods to eliminate unnecessary type parameters, simplifying the codebase. --- .../tree/src/tree/payload_processor/mod.rs | 5 +- .../src/tree/payload_processor/multiproof.rs | 115 +++++++----------- 2 files changed, 45 insertions(+), 75 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index c60b73ce514..a8d9a6957bf 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -192,8 +192,7 @@ where { let (to_sparse_trie, sparse_trie_rx) = channel(); // spawn multiproof task, save the trie input - let (trie_input, state_root_config) = - MultiProofConfig::new_from_input(consistent_view, trie_input); + let (trie_input, state_root_config) = MultiProofConfig::from_input(trie_input); self.trie_input = Some(trie_input); // Create and spawn the storage proof task @@ -207,7 +206,7 @@ where let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), - state_root_config.consistent_view.clone(), + consistent_view.clone(), task_ctx, storage_worker_count, account_worker_count, diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 9045f057d24..f865312b83d 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -12,7 +12,6 @@ use derive_more::derive::Deref; use metrics::Histogram; use reth_errors::ProviderError; use reth_metrics::Metrics; -use reth_provider::{providers::ConsistentDbView, BlockReader, DatabaseProviderFactory}; use reth_revm::state::EvmState; use reth_trie::{ added_removed_keys::MultiAddedRemovedKeys, prefix_set::TriePrefixSetsMut, @@ -66,9 +65,7 @@ impl SparseTrieUpdate { /// Common configuration for multi proof tasks #[derive(Debug, Clone)] -pub(super) struct MultiProofConfig { - /// View over the state in the database. - pub consistent_view: ConsistentDbView, +pub(super) struct MultiProofConfig { /// The sorted collection of cached in-memory intermediate trie nodes that /// can be reused for computation. pub nodes_sorted: Arc, @@ -80,17 +77,13 @@ pub(super) struct MultiProofConfig { pub prefix_sets: Arc, } -impl MultiProofConfig { - /// Creates a new state root config from the consistent view and the trie input. +impl MultiProofConfig { + /// Creates a new state root config from the trie input. /// /// This returns a cleared [`TrieInput`] so that we can reuse any allocated space in the /// [`TrieInput`]. - pub(super) fn new_from_input( - consistent_view: ConsistentDbView, - mut input: TrieInput, - ) -> (TrieInput, Self) { + pub(super) fn from_input(mut input: TrieInput) -> (TrieInput, Self) { let config = Self { - consistent_view, nodes_sorted: Arc::new(input.nodes.drain_into_sorted()), state_sorted: Arc::new(input.state.drain_into_sorted()), prefix_sets: Arc::new(input.prefix_sets.clone()), @@ -249,14 +242,14 @@ pub(crate) fn evm_state_to_hashed_post_state(update: EvmState) -> HashedPostStat /// A pending multiproof task, either [`StorageMultiproofInput`] or [`MultiproofInput`]. #[derive(Debug)] -enum PendingMultiproofTask { +enum PendingMultiproofTask { /// A storage multiproof task input. - Storage(StorageMultiproofInput), + Storage(StorageMultiproofInput), /// A regular multiproof task input. - Regular(MultiproofInput), + Regular(MultiproofInput), } -impl PendingMultiproofTask { +impl PendingMultiproofTask { /// Returns the proof sequence number of the task. const fn proof_sequence_number(&self) -> u64 { match self { @@ -282,22 +275,22 @@ impl PendingMultiproofTask { } } -impl From> for PendingMultiproofTask { - fn from(input: StorageMultiproofInput) -> Self { +impl From for PendingMultiproofTask { + fn from(input: StorageMultiproofInput) -> Self { Self::Storage(input) } } -impl From> for PendingMultiproofTask { - fn from(input: MultiproofInput) -> Self { +impl From for PendingMultiproofTask { + fn from(input: MultiproofInput) -> Self { Self::Regular(input) } } /// Input parameters for spawning a dedicated storage multiproof calculation. #[derive(Debug)] -struct StorageMultiproofInput { - config: MultiProofConfig, +struct StorageMultiproofInput { + config: MultiProofConfig, source: Option, hashed_state_update: HashedPostState, hashed_address: B256, @@ -307,7 +300,7 @@ struct StorageMultiproofInput { multi_added_removed_keys: Arc, } -impl StorageMultiproofInput { +impl StorageMultiproofInput { /// Destroys the input and sends a [`MultiProofMessage::EmptyProof`] message to the sender. fn send_empty_proof(self) { let _ = self.state_root_message_sender.send(MultiProofMessage::EmptyProof { @@ -319,8 +312,8 @@ impl StorageMultiproofInput { /// Input parameters for spawning a multiproof calculation. #[derive(Debug)] -struct MultiproofInput { - config: MultiProofConfig, +struct MultiproofInput { + config: MultiProofConfig, source: Option, hashed_state_update: HashedPostState, proof_targets: MultiProofTargets, @@ -329,7 +322,7 @@ struct MultiproofInput { multi_added_removed_keys: Option>, } -impl MultiproofInput { +impl MultiproofInput { /// Destroys the input and sends a [`MultiProofMessage::EmptyProof`] message to the sender. fn send_empty_proof(self) { let _ = self.state_root_message_sender.send(MultiProofMessage::EmptyProof { @@ -344,13 +337,13 @@ impl MultiproofInput { /// concurrency, further calculation requests are queued and spawn later, after /// availability has been signaled. #[derive(Debug)] -pub struct MultiproofManager { +pub struct MultiproofManager { /// Maximum number of concurrent calculations. max_concurrent: usize, /// Currently running calculations. inflight: usize, /// Queued calculations. - pending: VecDeque>, + pending: VecDeque, /// Executor for tasks executor: WorkloadExecutor, /// Handle to the proof task manager used for creating `ParallelProof` instances for storage @@ -374,10 +367,7 @@ pub struct MultiproofManager { metrics: MultiProofTaskMetrics, } -impl MultiproofManager -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ +impl MultiproofManager { /// Creates a new [`MultiproofManager`]. fn new( executor: WorkloadExecutor, @@ -404,7 +394,7 @@ where /// Spawns a new multiproof calculation or enqueues it for later if /// `max_concurrent` are already inflight. - fn spawn_or_queue(&mut self, input: PendingMultiproofTask) { + fn spawn_or_queue(&mut self, input: PendingMultiproofTask) { // If there are no proof targets, we can just send an empty multiproof back immediately if input.proof_targets_is_empty() { debug!( @@ -438,7 +428,7 @@ where /// Spawns a multiproof task, dispatching to `spawn_storage_proof` if the input is a storage /// multiproof, and dispatching to `spawn_multiproof` otherwise. - fn spawn_multiproof_task(&mut self, input: PendingMultiproofTask) { + fn spawn_multiproof_task(&mut self, input: PendingMultiproofTask) { match input { PendingMultiproofTask::Storage(storage_input) => { self.spawn_storage_proof(storage_input); @@ -450,7 +440,7 @@ where } /// Spawns a single storage proof calculation task. - fn spawn_storage_proof(&mut self, storage_multiproof_input: StorageMultiproofInput) { + fn spawn_storage_proof(&mut self, storage_multiproof_input: StorageMultiproofInput) { let StorageMultiproofInput { config, source, @@ -476,7 +466,7 @@ where "Starting dedicated storage proof calculation", ); let start = Instant::now(); - let proof_result = ParallelProof::::new( + let proof_result = ParallelProof::new( config.nodes_sorted, config.state_sorted, config.prefix_sets, @@ -524,7 +514,7 @@ where } /// Spawns a single multiproof calculation task. - fn spawn_multiproof(&mut self, multiproof_input: MultiproofInput) { + fn spawn_multiproof(&mut self, multiproof_input: MultiproofInput) { let MultiproofInput { config, source, @@ -554,10 +544,8 @@ where let start = Instant::now(); // Extend prefix sets with targets - let frozen_prefix_sets = ParallelProof::::extend_prefix_sets_with_targets( - &config.prefix_sets, - &proof_targets, - ); + let frozen_prefix_sets = + ParallelProof::extend_prefix_sets_with_targets(&config.prefix_sets, &proof_targets); // Queue account multiproof to worker pool let input = AccountMultiproofInput { @@ -675,13 +663,13 @@ pub(crate) struct MultiProofTaskMetrics { /// Then it updates relevant leaves according to the result of the transaction. /// This feeds updates to the sparse trie task. #[derive(Debug)] -pub(super) struct MultiProofTask { +pub(super) struct MultiProofTask { /// The size of proof targets chunk to spawn in one calculation. /// /// If [`None`], then chunking is disabled. chunk_size: Option, /// Task configuration. - config: MultiProofConfig, + config: MultiProofConfig, /// Receiver for state root related messages. rx: Receiver, /// Sender for state root related messages. @@ -695,18 +683,15 @@ pub(super) struct MultiProofTask { /// Proof sequencing handler. proof_sequencer: ProofSequencer, /// Manages calculation of multiproofs. - multiproof_manager: MultiproofManager, + multiproof_manager: MultiproofManager, /// multi proof task metrics metrics: MultiProofTaskMetrics, } -impl MultiProofTask -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ +impl MultiProofTask { /// Creates a new multi proof task with the unified message channel pub(super) fn new( - config: MultiProofConfig, + config: MultiProofConfig, executor: WorkloadExecutor, proof_task_handle: ProofTaskManagerHandle, to_sparse_trie: Sender, @@ -1233,43 +1218,29 @@ fn get_proof_targets( mod tests { use super::*; use alloy_primitives::map::B256Set; - use reth_provider::{providers::ConsistentDbView, test_utils::create_test_provider_factory}; + use reth_provider::{ + providers::ConsistentDbView, test_utils::create_test_provider_factory, BlockReader, + DatabaseProviderFactory, + }; use reth_trie::{MultiProof, TrieInput}; use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofTaskManager}; use revm_primitives::{B256, U256}; - use std::sync::Arc; - fn create_state_root_config(factory: F, input: TrieInput) -> MultiProofConfig - where - F: DatabaseProviderFactory + Clone + 'static, - { - let consistent_view = ConsistentDbView::new(factory, None); - let nodes_sorted = Arc::new(input.nodes.clone().into_sorted()); - let state_sorted = Arc::new(input.state.clone().into_sorted()); - let prefix_sets = Arc::new(input.prefix_sets); - - MultiProofConfig { consistent_view, nodes_sorted, state_sorted, prefix_sets } - } - - fn create_test_state_root_task(factory: F) -> MultiProofTask + fn create_test_state_root_task(factory: F) -> MultiProofTask where F: DatabaseProviderFactory + Clone + 'static, { let executor = WorkloadExecutor::default(); - let config = create_state_root_config(factory, TrieInput::default()); + let (_trie_input, config) = MultiProofConfig::from_input(TrieInput::default()); let task_ctx = ProofTaskCtx::new( config.nodes_sorted.clone(), config.state_sorted.clone(), config.prefix_sets.clone(), ); - let proof_task = ProofTaskManager::new( - executor.handle().clone(), - config.consistent_view.clone(), - task_ctx, - 1, - 1, - ) - .expect("Failed to create ProofTaskManager"); + let consistent_view = ConsistentDbView::new(factory, None); + let proof_task = + ProofTaskManager::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) + .expect("Failed to create ProofTaskManager"); let channel = channel(); MultiProofTask::new(config, executor, proof_task.handle(), channel.0, 1, None) From 47da382d011ccc3fb0282b99b678ea2c6e8a6090 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 08:12:09 +0000 Subject: [PATCH 106/144] clippy --- crates/engine/tree/src/tree/payload_processor/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index a8d9a6957bf..c24b0d1fe16 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -206,7 +206,7 @@ where let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), - consistent_view.clone(), + consistent_view, task_ctx, storage_worker_count, account_worker_count, From 0a88a18203eeb5733feae0f7eb075b6e55f120bd Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 08:16:45 +0000 Subject: [PATCH 107/144] fmt --- crates/trie/parallel/src/proof.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index d0fa9e7e40b..7fc1f022a7e 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -240,9 +240,7 @@ mod tests { use rand::Rng; use reth_primitives_traits::{Account, StorageEntry}; use reth_provider::{ - providers::ConsistentDbView, - test_utils::{create_test_provider_factory, MockNodeTypesWithDB}, - HashingWriter, ProviderFactory, + providers::ConsistentDbView, test_utils::create_test_provider_factory, HashingWriter, }; use reth_trie::proof::Proof; use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; From 14def0761e4ce088f4adae56d8fb64ba4c0ad962 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 09:11:30 +0000 Subject: [PATCH 108/144] fmt --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index e1cada03d55..18062747901 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -21,7 +21,7 @@ use alloy_rlp::{BufMut, Encodable}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use dashmap::DashMap; use reth_db_api::transaction::DbTx; -use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; +use reth_execution_errors::SparseTrieError; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, ProviderError, ProviderResult, From 44bca4351577d550aab2443477081e7353ac99e8 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 09:17:07 +0000 Subject: [PATCH 109/144] made count same as storage worker --- crates/engine/primitives/src/config.rs | 10 ++++------ crates/node/core/src/args/engine.rs | 2 +- docs/vocs/docs/pages/cli/reth/node.mdx | 2 +- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 7d3d9c1c37f..70763b6701f 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -21,14 +21,12 @@ fn default_storage_worker_count() -> usize { } } -/// Returns the default number of account worker threads optimized for I/O-bound coordination. +/// Returns the default number of account worker threads. /// -/// Account workers primarily coordinate storage proof collection and account trie traversal. -/// They spend significant time blocked on `receiver.recv()` calls waiting for storage proofs, -/// so we use higher concurrency (1.5x storage workers) to maximize throughput and overlap. -/// While storage workers are CPU-bound, account workers are I/O-bound coordinators. +/// Account workers coordinate storage proof collection and account trie traversal. +/// They are set to the same count as storage workers for simplicity. fn default_account_worker_count() -> usize { - (default_storage_worker_count() * 3) / 2 + default_storage_worker_count() } /// The size of proof targets chunk to spawn in one multiproof calculation. diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index fb98847d8c7..6b678b5789b 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -115,7 +115,7 @@ pub struct EngineArgs { pub storage_worker_count: Option, /// Configure the number of account proof workers in the Tokio blocking pool. - /// If not specified, defaults to 1.5x storage workers. + /// If not specified, defaults to the same count as storage workers. #[arg(long = "engine.account-worker-count")] pub account_worker_count: Option, } diff --git a/docs/vocs/docs/pages/cli/reth/node.mdx b/docs/vocs/docs/pages/cli/reth/node.mdx index 6fc9c054f1f..1acbcf248fc 100644 --- a/docs/vocs/docs/pages/cli/reth/node.mdx +++ b/docs/vocs/docs/pages/cli/reth/node.mdx @@ -868,7 +868,7 @@ Engine: Configure the number of storage proof workers in the Tokio blocking pool. If not specified, defaults to 2x available parallelism, clamped between 2 and 64 --engine.account-worker-count - Configure the number of account proof workers in the Tokio blocking pool. If not specified, defaults to 1.5x storage workers + Configure the number of account proof workers in the Tokio blocking pool. If not specified, defaults to the same count as storage workers ERA: --era.enable From 351645174014d7b52a5f74378770c2b1377ddfde Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 09:49:05 +0000 Subject: [PATCH 110/144] refactor: update proof task management to use spawn_proof_workers - Replaced the ProofTaskManager with a new spawn_proof_workers function for better clarity and maintainability. - Updated related code to utilize the new function, simplifying the worker spawning process. - Enhanced metrics tracking for storage and account proof requests, ensuring thread-safe operations. - Improved error handling and code structure across proof task implementations. --- .../tree/src/tree/payload_processor/mod.rs | 22 +- .../src/tree/payload_processor/multiproof.rs | 23 +- crates/trie/parallel/src/proof.rs | 50 +- crates/trie/parallel/src/proof_task.rs | 518 ++++++++++++------ .../trie/parallel/src/proof_task_metrics.rs | 34 +- 5 files changed, 424 insertions(+), 223 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index c24b0d1fe16..e0189f40c13 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -32,7 +32,7 @@ use reth_provider::{ use reth_revm::{db::BundleState, state::EvmState}; use reth_trie::TrieInput; use reth_trie_parallel::{ - proof_task::{ProofTaskCtx, ProofTaskManager}, + proof_task::{spawn_proof_workers, ProofTaskCtx}, root::ParallelStateRootError, }; use reth_trie_sparse::{ @@ -204,14 +204,14 @@ where let storage_worker_count = config.storage_worker_count(); let account_worker_count = config.account_worker_count(); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - let proof_task = match ProofTaskManager::new( + let proof_handle = match spawn_proof_workers( self.executor.handle().clone(), consistent_view, task_ctx, storage_worker_count, account_worker_count, ) { - Ok(task) => task, + Ok(handle) => handle, Err(error) => { return Err((error, transactions, env, provider_builder)); } @@ -223,7 +223,7 @@ where let multi_proof_task = MultiProofTask::new( state_root_config, self.executor.clone(), - proof_task.handle(), + proof_handle.clone(), to_sparse_trie, max_multi_proof_task_concurrency, config.multiproof_chunking_enabled().then_some(config.multiproof_chunk_size()), @@ -252,19 +252,7 @@ where let (state_root_tx, state_root_rx) = channel(); // Spawn the sparse trie task using any stored trie and parallel trie configuration. - self.spawn_sparse_trie_task(sparse_trie_rx, proof_task.handle(), state_root_tx); - - // spawn the proof task - self.executor.spawn_blocking(move || { - if let Err(err) = proof_task.run() { - // At least log if there is an error at any point - tracing::error!( - target: "engine::root", - ?err, - "Storage proof task returned an error" - ); - } - }); + self.spawn_sparse_trie_task(sparse_trie_rx, proof_handle, state_root_tx); Ok(PayloadHandle { to_multi_proof, diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index f865312b83d..59572a3e9db 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -20,7 +20,7 @@ use reth_trie::{ }; use reth_trie_parallel::{ proof::ParallelProof, - proof_task::{AccountMultiproofInput, ProofTaskKind, ProofTaskManagerHandle}, + proof_task::{AccountMultiproofInput, ProofTaskManagerHandle}, root::ParallelStateRootError, }; use std::{ @@ -556,15 +556,10 @@ impl MultiproofManager { missed_leaves_storage_roots, }; - let (sender, receiver) = channel(); let proof_result: Result = (|| { - account_proof_task_handle - .queue_task(ProofTaskKind::AccountMultiproof(input, sender)) - .map_err(|_| { - ParallelStateRootError::Other( - "Failed to queue account multiproof to worker pool".into(), - ) - })?; + let receiver = account_proof_task_handle + .queue_account_multiproof(input) + .map_err(|e| ParallelStateRootError::Other(e.to_string()))?; receiver .recv() @@ -1223,7 +1218,7 @@ mod tests { DatabaseProviderFactory, }; use reth_trie::{MultiProof, TrieInput}; - use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofTaskManager}; + use reth_trie_parallel::proof_task::{spawn_proof_workers, ProofTaskCtx}; use revm_primitives::{B256, U256}; fn create_test_state_root_task(factory: F) -> MultiProofTask @@ -1238,12 +1233,12 @@ mod tests { config.prefix_sets.clone(), ); let consistent_view = ConsistentDbView::new(factory, None); - let proof_task = - ProofTaskManager::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) - .expect("Failed to create ProofTaskManager"); + let proof_handle = + spawn_proof_workers(executor.handle().clone(), consistent_view, task_ctx, 1, 1) + .expect("Failed to spawn proof workers"); let channel = channel(); - MultiProofTask::new(config, executor, proof_task.handle(), channel.0, 1, None) + MultiProofTask::new(config, executor, proof_handle, channel.0, 1, None) } #[test] diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 7fc1f022a7e..b33ba89bd54 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -1,8 +1,6 @@ use crate::{ metrics::ParallelTrieMetrics, - proof_task::{ - AccountMultiproofInput, ProofTaskKind, ProofTaskManagerHandle, StorageProofInput, - }, + proof_task::{AccountMultiproofInput, ProofTaskManagerHandle, StorageProofInput}, root::ParallelStateRootError, StorageRootTargets, }; @@ -16,10 +14,7 @@ use reth_trie::{ DecodedMultiProof, DecodedStorageMultiProof, HashedPostStateSorted, MultiProofTargets, Nibbles, }; use reth_trie_common::added_removed_keys::MultiAddedRemovedKeys; -use std::sync::{ - mpsc::{channel, Receiver}, - Arc, -}; +use std::sync::{mpsc::Receiver, Arc}; use tracing::trace; /// Parallel proof calculator. @@ -93,7 +88,10 @@ impl ParallelProof { hashed_address: B256, prefix_set: PrefixSet, target_slots: B256Set, - ) -> Receiver> { + ) -> Result< + Receiver>, + ParallelStateRootError, + > { let input = StorageProofInput::new( hashed_address, prefix_set, @@ -102,9 +100,9 @@ impl ParallelProof { self.multi_added_removed_keys.clone(), ); - let (sender, receiver) = std::sync::mpsc::channel(); - let _ = self.proof_task_handle.queue_task(ProofTaskKind::StorageProof(input, sender)); - receiver + self.proof_task_handle + .queue_storage_proof(input) + .map_err(|e| ParallelStateRootError::Other(e.to_string())) } /// Generate a storage multiproof according to the specified targets and hashed address. @@ -124,7 +122,7 @@ impl ParallelProof { "Starting storage proof generation" ); - let receiver = self.queue_storage_proof(hashed_address, prefix_set, target_slots); + let receiver = self.queue_storage_proof(hashed_address, prefix_set, target_slots)?; let proof_result = receiver.recv().map_err(|_| { ParallelStateRootError::StorageRoot(StorageRootError::Database(DatabaseError::Other( format!("channel closed for {hashed_address}"), @@ -193,15 +191,10 @@ impl ParallelProof { missed_leaves_storage_roots: self.missed_leaves_storage_roots.clone(), }; - let (sender, receiver) = channel(); - self.proof_task_handle - .queue_task(ProofTaskKind::AccountMultiproof(input, sender)) - .map_err(|_| { - ParallelStateRootError::Other( - "Failed to queue account multiproof: account worker pool unavailable" - .to_string(), - ) - })?; + let receiver = self + .proof_task_handle + .queue_account_multiproof(input) + .map_err(|e| ParallelStateRootError::Other(e.to_string()))?; // Wait for account multiproof result from worker let (multiproof, stats) = receiver.recv().map_err(|_| { @@ -231,7 +224,7 @@ impl ParallelProof { #[cfg(test)] mod tests { use super::*; - use crate::proof_task::{ProofTaskCtx, ProofTaskManager}; + use crate::proof_task::{spawn_proof_workers, ProofTaskCtx}; use alloy_primitives::{ keccak256, map::{B256Set, DefaultHashBuilder, HashMap}, @@ -313,13 +306,8 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); - let proof_task = - ProofTaskManager::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1).unwrap(); - let proof_task_handle = proof_task.handle(); - - // keep the join handle around to make sure it does not return any errors - // after we compute the state root - let join_handle = rt.spawn_blocking(move || proof_task.run()); + let proof_task_handle = + spawn_proof_workers(rt.handle().clone(), consistent_view, task_ctx, 1, 1).unwrap(); let parallel_result = ParallelProof::new( Default::default(), @@ -354,9 +342,7 @@ mod tests { // then compare the entire thing for any mask differences assert_eq!(parallel_result, sequential_result_decoded); - // drop the handle to terminate the task and then block on the proof task handle to make - // sure it does not return any errors + // Workers shut down automatically when handle is dropped drop(proof_task_handle); - rt.block_on(join_handle).unwrap().expect("The proof task should not return an error"); } } diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 18062747901..cdf7f4e5124 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -21,7 +21,7 @@ use alloy_rlp::{BufMut, Encodable}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use dashmap::DashMap; use reth_db_api::transaction::DbTx; -use reth_execution_errors::SparseTrieError; +use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, ProviderError, ProviderResult, @@ -88,6 +88,150 @@ enum StorageWorkerJob { }, } +/// Spawns storage and account worker pools with dedicated database transactions. +/// +/// Returns a handle for submitting proof tasks to the worker pools. +/// Workers run until the last handle is dropped. +/// +/// # Parameters +/// - `executor`: Tokio runtime handle for spawning blocking tasks +/// - `view`: Consistent database view for creating transactions +/// - `task_ctx`: Shared context with trie updates and prefix sets +/// - `storage_worker_count`: Number of storage workers to spawn +/// - `account_worker_count`: Number of account workers to spawn +pub fn spawn_proof_workers( + executor: Handle, + view: ConsistentDbView, + task_ctx: ProofTaskCtx, + storage_worker_count: usize, + account_worker_count: usize, +) -> ProviderResult +where + Factory: DatabaseProviderFactory, +{ + let (storage_work_tx, storage_work_rx) = unbounded::(); + let (account_work_tx, account_work_rx) = unbounded::(); + + tracing::info!( + target: "trie::proof_task", + storage_worker_count, + account_worker_count, + "Spawning proof worker pools" + ); + + // Spawn storage workers (reuse existing spawn_storage_workers logic) + spawn_storage_workers_internal( + &executor, + &view, + &task_ctx, + storage_worker_count, + storage_work_rx, + )?; + + // Spawn account workers (reuse existing spawn_account_workers logic) + spawn_account_workers_internal( + &executor, + &view, + &task_ctx, + account_worker_count, + account_work_rx, + storage_work_tx.clone(), + )?; + + Ok(ProofTaskManagerHandle::new( + storage_work_tx, + account_work_tx, + Arc::new(AtomicUsize::new(0)), + #[cfg(feature = "metrics")] + Arc::new(ProofTaskMetrics::default()), + )) +} + +/// Spawns a pool of storage workers with dedicated database transactions. +/// +/// Each worker receives `StorageWorkerJob` from the channel and processes storage proofs +/// and blinded storage node requests using a dedicated long-lived transaction. +/// +/// # Parameters +/// - `executor`: Tokio runtime handle for spawning blocking tasks +/// - `view`: Consistent database view for creating transactions +/// - `task_ctx`: Shared context with trie updates and prefix sets +/// - `worker_count`: Number of storage workers to spawn +/// - `work_rx`: Receiver for storage worker jobs +fn spawn_storage_workers_internal( + executor: &Handle, + view: &ConsistentDbView, + task_ctx: &ProofTaskCtx, + worker_count: usize, + work_rx: CrossbeamReceiver, +) -> ProviderResult<()> +where + Factory: DatabaseProviderFactory, +{ + for worker_id in 0..worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = work_rx.clone(); + + executor + .spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx_clone, worker_id)); + + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Storage worker spawned successfully" + ); + } + + Ok(()) +} + +/// Spawns a pool of account workers with dedicated database transactions. +/// +/// Each worker receives `AccountWorkerJob` from the channel and processes account multiproofs +/// and blinded account node requests using a dedicated long-lived transaction. Account workers +/// can delegate storage proof computation to the storage worker pool. +/// +/// # Parameters +/// - `executor`: Tokio runtime handle for spawning blocking tasks +/// - `view`: Consistent database view for creating transactions +/// - `task_ctx`: Shared context with trie updates and prefix sets +/// - `worker_count`: Number of account workers to spawn +/// - `work_rx`: Receiver for account worker jobs +/// - `storage_work_tx`: Sender to delegate storage proofs to storage worker pool +fn spawn_account_workers_internal( + executor: &Handle, + view: &ConsistentDbView, + task_ctx: &ProofTaskCtx, + worker_count: usize, + work_rx: CrossbeamReceiver, + storage_work_tx: CrossbeamSender, +) -> ProviderResult<()> +where + Factory: DatabaseProviderFactory, +{ + for worker_id in 0..worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = work_rx.clone(); + let storage_work_tx_clone = storage_work_tx.clone(); + + executor.spawn_blocking(move || { + account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) + }); + + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Account worker spawned successfully" + ); + } + + Ok(()) +} + /// Manager for coordinating proof request execution across different task types. /// /// # Architecture @@ -109,6 +253,13 @@ enum StorageWorkerJob { /// - Submit tasks via `queue_task(ProofTaskKind)` /// - Use standard `std::mpsc` message passing /// - Receive consistent return types and error handling +/// +/// # Deprecation Notice +/// +/// This struct is deprecated. Use `spawn_proof_workers()` instead, which returns +/// a `ProofTaskManagerHandle` directly without requiring a separate manager instance. +#[deprecated(note = "Use spawn_proof_workers() instead")] +#[allow(deprecated)] #[derive(Debug)] pub struct ProofTaskManager { /// Sender for storage worker jobs to worker pool. @@ -129,6 +280,7 @@ pub struct ProofTaskManager { proof_task_rx: CrossbeamReceiver, /// Sender for creating handles that can queue tasks. + #[allow(dead_code)] proof_task_tx: CrossbeamSender, /// The number of active handles. @@ -657,6 +809,7 @@ fn queue_storage_proofs( Ok(storage_proof_receivers) } +#[allow(deprecated)] impl ProofTaskManager { /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. /// @@ -696,7 +849,7 @@ impl ProofTaskManager { ); // Spawn storage workers - let spawned_storage_workers = Self::spawn_storage_workers( + spawn_storage_workers_internal( &executor, &view, &task_ctx, @@ -705,7 +858,7 @@ impl ProofTaskManager { )?; // Spawn account workers with direct access to the storage worker queue - let spawned_account_workers = Self::spawn_account_workers( + spawn_account_workers_internal( &executor, &view, &task_ctx, @@ -716,9 +869,9 @@ impl ProofTaskManager { Ok(Self { storage_work_tx, - storage_worker_count: spawned_storage_workers, + storage_worker_count, account_work_tx, - account_worker_count: spawned_account_workers, + account_worker_count, proof_task_rx, proof_task_tx, active_handles: Arc::new(AtomicUsize::new(0)), @@ -729,110 +882,18 @@ impl ProofTaskManager { } /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. - pub fn handle(&self) -> ProofTaskManagerHandle { - ProofTaskManagerHandle::new(self.proof_task_tx.clone(), self.active_handles.clone()) - } - - /// Spawns a pool of storage workers with dedicated database transactions. /// - /// Each worker receives `StorageWorkerJob` from the channel and processes storage proofs - /// and blinded storage node requests using a dedicated long-lived transaction. - /// - /// # Parameters - /// - `executor`: Tokio runtime handle for spawning blocking tasks - /// - `view`: Consistent database view for creating transactions - /// - `task_ctx`: Shared context with trie updates and prefix sets - /// - `worker_count`: Number of storage workers to spawn - /// - `work_rx`: Receiver for storage worker jobs - /// - /// # Returns - /// The number of storage workers successfully spawned - fn spawn_storage_workers( - executor: &Handle, - view: &ConsistentDbView, - task_ctx: &ProofTaskCtx, - worker_count: usize, - work_rx: CrossbeamReceiver, - ) -> ProviderResult - where - Factory: DatabaseProviderFactory, - { - let mut spawned_workers = 0; - - for worker_id in 0..worker_count { - let provider_ro = view.provider_ro()?; - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = work_rx.clone(); - - executor.spawn_blocking(move || { - storage_worker_loop(proof_task_tx, work_rx_clone, worker_id) - }); - - spawned_workers += 1; - - tracing::debug!( - target: "trie::proof_task", - worker_id, - spawned_workers, - "Storage worker spawned successfully" - ); - } - - Ok(spawned_workers) - } - - /// Spawns a pool of account workers with dedicated database transactions. - /// - /// Each worker receives `AccountWorkerJob` from the channel and processes account multiproofs - /// and blinded account node requests using a dedicated long-lived transaction. Account workers - /// can delegate storage proof computation to the storage worker pool. - /// - /// # Parameters - /// - `executor`: Tokio runtime handle for spawning blocking tasks - /// - `view`: Consistent database view for creating transactions - /// - `task_ctx`: Shared context with trie updates and prefix sets - /// - `worker_count`: Number of account workers to spawn - /// - `work_rx`: Receiver for account worker jobs - /// - `storage_work_tx`: Sender to delegate storage proofs to storage worker pool - /// - /// # Returns - /// The number of account workers successfully spawned - fn spawn_account_workers( - executor: &Handle, - view: &ConsistentDbView, - task_ctx: &ProofTaskCtx, - worker_count: usize, - work_rx: CrossbeamReceiver, - storage_work_tx: CrossbeamSender, - ) -> ProviderResult - where - Factory: DatabaseProviderFactory, - { - let mut spawned_workers = 0; - - for worker_id in 0..worker_count { - let provider_ro = view.provider_ro()?; - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = work_rx.clone(); - let storage_work_tx_clone = storage_work_tx.clone(); - - executor.spawn_blocking(move || { - account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) - }); - - spawned_workers += 1; - - tracing::debug!( - target: "trie::proof_task", - worker_id, - spawned_workers, - "Account worker spawned successfully" - ); - } - - Ok(spawned_workers) + /// DEPRECATED: This method returns a handle that uses the deprecated routing mechanism. + /// Use `spawn_proof_workers()` instead for direct worker pool access. + #[deprecated(note = "Use spawn_proof_workers() instead")] + pub fn handle(&self) -> ProofTaskManagerHandle { + ProofTaskManagerHandle::new( + self.storage_work_tx.clone(), + self.account_work_tx.clone(), + self.active_handles.clone(), + #[cfg(feature = "metrics")] + Arc::new(ProofTaskMetrics::default()), + ) } /// Loops, managing the proof tasks, routing them to the appropriate worker pools. @@ -850,7 +911,14 @@ impl ProofTaskManager { /// /// On termination, `storage_work_tx` and `account_work_tx` are dropped, closing the channels /// and signaling all workers to shut down gracefully. - pub fn run(mut self) -> ProviderResult<()> { + /// + /// # Deprecation Notice + /// + /// This method is deprecated. With `spawn_proof_workers()`, workers are spawned directly + /// and no routing thread is needed. Workers shut down automatically when all handles are + /// dropped. + #[deprecated(note = "Use spawn_proof_workers() instead - no routing thread needed")] + pub fn run(self) -> ProviderResult<()> { loop { match self.proof_task_rx.recv() { Ok(message) => { @@ -873,7 +941,7 @@ impl ProofTaskManager { ProofTaskKind::BlindedStorageNode(account, path, sender) => { #[cfg(feature = "metrics")] { - self.metrics.storage_nodes += 1; + self.metrics.storage_nodes.fetch_add(1, Ordering::Relaxed); } self.storage_work_tx @@ -895,7 +963,7 @@ impl ProofTaskManager { ProofTaskKind::BlindedAccountNode(path, sender) => { #[cfg(feature = "metrics")] { - self.metrics.account_nodes += 1; + self.metrics.account_nodes.fetch_add(1, Ordering::Relaxed); } self.account_work_tx @@ -1182,6 +1250,11 @@ impl ProofTaskCtx { } /// Message used to communicate with [`ProofTaskManager`]. +/// +/// DEPRECATED: No longer needed with `spawn_proof_workers()` which provides direct +/// worker pool access. Use explicit queue methods on `ProofTaskManagerHandle` instead. +#[deprecated(note = "Use explicit queue methods on ProofTaskManagerHandle instead")] +#[allow(deprecated)] #[derive(Debug)] pub enum ProofTaskMessage { /// A request to queue a proof task. @@ -1194,6 +1267,12 @@ pub enum ProofTaskMessage { /// /// When queueing a task using [`ProofTaskMessage::QueueTask`], this enum /// specifies the type of proof task to be executed. +/// +/// DEPRECATED: Use explicit queue methods on `ProofTaskManagerHandle` instead: +/// - `queue_storage_proof()` for storage proofs +/// - `queue_account_multiproof()` for account multiproofs +#[deprecated(note = "Use explicit queue methods on ProofTaskManagerHandle instead")] +#[allow(deprecated)] #[derive(Debug)] pub enum ProofTaskKind { /// A storage proof request. @@ -1206,53 +1285,183 @@ pub enum ProofTaskKind { AccountMultiproof(AccountMultiproofInput, Sender), } -/// A handle that wraps a single proof task sender that sends a terminate message on `Drop` if the -/// number of active handles went to zero. +/// A handle that provides type-safe access to proof worker pools. +/// +/// The handle stores direct senders to both storage and account worker pools, +/// eliminating the need for a routing thread. All handles share reference-counted +/// channels, and workers shut down gracefully when all handles are dropped. #[derive(Debug)] pub struct ProofTaskManagerHandle { - /// The sender for the proof task manager. - sender: CrossbeamSender, - /// The number of active handles. + /// Direct sender to storage worker pool + storage_work_tx: CrossbeamSender, + /// Direct sender to account worker pool + account_work_tx: CrossbeamSender, + /// Active handle reference count for auto-termination active_handles: Arc, + /// Metrics tracking (lock-free) + #[cfg(feature = "metrics")] + metrics: Arc, } impl ProofTaskManagerHandle { - /// Creates a new [`ProofTaskManagerHandle`] with the given sender. + /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. + #[allow(private_interfaces)] pub fn new( - sender: CrossbeamSender, + storage_work_tx: CrossbeamSender, + account_work_tx: CrossbeamSender, active_handles: Arc, + #[cfg(feature = "metrics")] metrics: Arc, ) -> Self { active_handles.fetch_add(1, Ordering::SeqCst); - Self { sender, active_handles } + Self { + storage_work_tx, + account_work_tx, + active_handles, + #[cfg(feature = "metrics")] + metrics, + } + } + + /// Queue a storage proof computation + pub fn queue_storage_proof( + &self, + input: StorageProofInput, + ) -> Result, ProviderError> { + let (tx, rx) = channel(); + self.storage_work_tx + .send(StorageWorkerJob::StorageProof { input, result_sender: tx }) + .map_err(|_| { + ProviderError::other(std::io::Error::other("storage workers unavailable")) + })?; + + #[cfg(feature = "metrics")] + self.metrics.storage_proofs.fetch_add(1, Ordering::Relaxed); + + Ok(rx) + } + + /// Queue an account multiproof computation + pub fn queue_account_multiproof( + &self, + input: AccountMultiproofInput, + ) -> Result, ProviderError> { + let (tx, rx) = channel(); + self.account_work_tx + .send(AccountWorkerJob::AccountMultiproof { input, result_sender: tx }) + .map_err(|_| { + ProviderError::other(std::io::Error::other("account workers unavailable")) + })?; + + #[cfg(feature = "metrics")] + self.metrics.account_proofs.fetch_add(1, Ordering::Relaxed); + + Ok(rx) + } + + /// Internal: Queue blinded storage node request + fn queue_blinded_storage_node( + &self, + account: B256, + path: Nibbles, + ) -> Result, ProviderError> { + let (tx, rx) = channel(); + self.storage_work_tx + .send(StorageWorkerJob::BlindedStorageNode { account, path, result_sender: tx }) + .map_err(|_| { + ProviderError::other(std::io::Error::other("storage workers unavailable")) + })?; + + #[cfg(feature = "metrics")] + self.metrics.storage_nodes.fetch_add(1, Ordering::Relaxed); + + Ok(rx) + } + + /// Internal: Queue blinded account node request + fn queue_blinded_account_node( + &self, + path: Nibbles, + ) -> Result, ProviderError> { + let (tx, rx) = channel(); + self.account_work_tx + .send(AccountWorkerJob::BlindedAccountNode { path, result_sender: tx }) + .map_err(|_| { + ProviderError::other(std::io::Error::other("account workers unavailable")) + })?; + + #[cfg(feature = "metrics")] + self.metrics.account_nodes.fetch_add(1, Ordering::Relaxed); + + Ok(rx) } /// Queues a task to the proof task manager. + /// + /// DEPRECATED: Use explicit methods like `queue_storage_proof` or `queue_account_multiproof` + /// instead. This method is kept temporarily for backwards compatibility during the + /// migration. + #[allow(deprecated)] + #[deprecated(note = "Use explicit queue methods instead")] pub fn queue_task( &self, task: ProofTaskKind, ) -> Result<(), crossbeam_channel::SendError> { - self.sender.send(ProofTaskMessage::QueueTask(task)) + match task { + ProofTaskKind::StorageProof(input, sender) => { + self.storage_work_tx + .send(StorageWorkerJob::StorageProof { input, result_sender: sender }) + .expect("storage workers should be available"); + } + ProofTaskKind::BlindedStorageNode(account, path, sender) => { + self.storage_work_tx + .send(StorageWorkerJob::BlindedStorageNode { + account, + path, + result_sender: sender, + }) + .expect("storage workers should be available"); + } + ProofTaskKind::BlindedAccountNode(path, sender) => { + self.account_work_tx + .send(AccountWorkerJob::BlindedAccountNode { path, result_sender: sender }) + .expect("account workers should be available"); + } + ProofTaskKind::AccountMultiproof(input, sender) => { + self.account_work_tx + .send(AccountWorkerJob::AccountMultiproof { input, result_sender: sender }) + .expect("account workers should be available"); + } + } + Ok(()) } /// Terminates the proof task manager. - pub fn terminate(&self) { - let _ = self.sender.send(ProofTaskMessage::Terminate); + /// + /// DEPRECATED: Workers now shut down automatically when all handles are dropped. + /// This method is kept for backwards compatibility but does nothing. + #[deprecated(note = "Workers shut down automatically when all handles are dropped")] + pub const fn terminate(&self) { + // No-op: workers shut down when all handles are dropped } } impl Clone for ProofTaskManagerHandle { fn clone(&self) -> Self { - Self::new(self.sender.clone(), self.active_handles.clone()) + Self::new( + self.storage_work_tx.clone(), + self.account_work_tx.clone(), + self.active_handles.clone(), + #[cfg(feature = "metrics")] + self.metrics.clone(), + ) } } impl Drop for ProofTaskManagerHandle { fn drop(&mut self) { - // Decrement the number of active handles and terminate the manager if it was the last - // handle. - if self.active_handles.fetch_sub(1, Ordering::SeqCst) == 1 { - self.terminate(); - } + // Decrement the number of active handles. + // When the last handle is dropped, the channels are dropped and workers shut down. + self.active_handles.fetch_sub(1, Ordering::SeqCst); } } @@ -1261,11 +1470,11 @@ impl TrieNodeProviderFactory for ProofTaskManagerHandle { type StorageNodeProvider = ProofTaskTrieNodeProvider; fn account_node_provider(&self) -> Self::AccountNodeProvider { - ProofTaskTrieNodeProvider::AccountNode { sender: self.sender.clone() } + ProofTaskTrieNodeProvider::AccountNode { handle: self.clone() } } fn storage_node_provider(&self, account: B256) -> Self::StorageNodeProvider { - ProofTaskTrieNodeProvider::StorageNode { account, sender: self.sender.clone() } + ProofTaskTrieNodeProvider::StorageNode { account, handle: self.clone() } } } @@ -1274,35 +1483,38 @@ impl TrieNodeProviderFactory for ProofTaskManagerHandle { pub enum ProofTaskTrieNodeProvider { /// Blinded account trie node provider. AccountNode { - /// Sender to the proof task. - sender: CrossbeamSender, + /// Handle to the proof task manager. + handle: ProofTaskManagerHandle, }, /// Blinded storage trie node provider. StorageNode { /// Target account. account: B256, - /// Sender to the proof task. - sender: CrossbeamSender, + /// Handle to the proof task manager. + handle: ProofTaskManagerHandle, }, } impl TrieNodeProvider for ProofTaskTrieNodeProvider { fn trie_node(&self, path: &Nibbles) -> Result, SparseTrieError> { - let (tx, rx) = channel(); match self { - Self::AccountNode { sender } => { - let _ = sender.send(ProofTaskMessage::QueueTask( - ProofTaskKind::BlindedAccountNode(*path, tx), - )); + Self::AccountNode { handle } => { + let rx = handle.queue_blinded_account_node(*path).map_err(|e| { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))) + })?; + rx.recv().map_err(|_| { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))) + })? } - Self::StorageNode { sender, account } => { - let _ = sender.send(ProofTaskMessage::QueueTask( - ProofTaskKind::BlindedStorageNode(*account, *path, tx), - )); + Self::StorageNode { handle, account } => { + let rx = handle.queue_blinded_storage_node(*account, *path).map_err(|e| { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))) + })?; + rx.recv().map_err(|_| { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))) + })? } } - - rx.recv().unwrap() } } @@ -1329,9 +1541,9 @@ mod tests { ) } - /// Ensures `max_concurrency` is independent of storage and account workers. + /// Ensures `spawn_proof_workers` spawns workers correctly. #[test] - fn proof_task_manager_independent_pools() { + fn spawn_proof_workers_creates_handle() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); runtime.block_on(async { let handle = tokio::runtime::Handle::current(); @@ -1339,13 +1551,13 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let manager = ProofTaskManager::new(handle.clone(), view, ctx, 5, 3).unwrap(); - // With storage_worker_count=5, we get exactly 5 storage workers - assert_eq!(manager.storage_worker_count, 5); - // With account_worker_count=3, we get exactly 3 account workers - assert_eq!(manager.account_worker_count, 3); + let proof_handle = spawn_proof_workers(handle.clone(), view, ctx, 5, 3).unwrap(); + + // Verify handle can be cloned + let _cloned_handle = proof_handle.clone(); - drop(manager); + // Workers shut down automatically when handle is dropped + drop(proof_handle); task::yield_now().await; }); } diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index cdb59d078d8..0274cfec8de 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -1,21 +1,41 @@ use reth_metrics::{metrics::Histogram, Metrics}; +use std::sync::{ + atomic::{AtomicU64, Ordering}, + Arc, +}; /// Metrics for blinded node fetching for the duration of the proof task manager. -#[derive(Clone, Debug, Default)] +#[derive(Clone, Debug)] pub struct ProofTaskMetrics { /// The actual metrics for blinded nodes. pub task_metrics: ProofTaskTrieMetrics, - /// Count of blinded account node requests. - pub account_nodes: usize, - /// Count of blinded storage node requests. - pub storage_nodes: usize, + /// Count of storage proof requests (lock-free). + pub storage_proofs: Arc, + /// Count of account proof requests (lock-free). + pub account_proofs: Arc, + /// Count of blinded account node requests (lock-free). + pub account_nodes: Arc, + /// Count of blinded storage node requests (lock-free). + pub storage_nodes: Arc, +} + +impl Default for ProofTaskMetrics { + fn default() -> Self { + Self { + task_metrics: ProofTaskTrieMetrics::default(), + storage_proofs: Arc::new(AtomicU64::new(0)), + account_proofs: Arc::new(AtomicU64::new(0)), + account_nodes: Arc::new(AtomicU64::new(0)), + storage_nodes: Arc::new(AtomicU64::new(0)), + } + } } impl ProofTaskMetrics { /// Record the blinded node counts into the histograms. pub fn record(&self) { - self.task_metrics.record_account_nodes(self.account_nodes); - self.task_metrics.record_storage_nodes(self.storage_nodes); + self.task_metrics.record_account_nodes(self.account_nodes.load(Ordering::Relaxed) as usize); + self.task_metrics.record_storage_nodes(self.storage_nodes.load(Ordering::Relaxed) as usize); } } From ed45ebda8aec29bff54b2ea644088de1b6917ea3 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:18:58 +0000 Subject: [PATCH 111/144] refactor: yeet proof task manager --- crates/trie/parallel/src/proof_task.rs | 484 ++----------------------- 1 file changed, 26 insertions(+), 458 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index cdf7f4e5124..7d923acc6ac 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1,9 +1,14 @@ -//! A Task that manages sending proof requests to a number of tasks that have longer-running -//! database transactions. +//! Parallel proof computation using worker pools with dedicated database transactions. //! -//! The [`ProofTaskManager`] ensures that there are a max number of currently executing proof tasks, -//! and is responsible for managing the fixed number of database transactions created at the start -//! of the task. +//! +//! # Architecture +//! +//! - **Worker Pools**: Pre-spawned workers with dedicated database transactions +//! - Storage pool: Handles storage proofs and blinded storage node requests +//! - Account pool: Handles account multiproofs and blinded account node requests +//! - **Direct Channel Access**: [`ProofTaskManagerHandle`] provides type-safe queue methods with +//! direct access to worker channels, eliminating routing overhead +//! - **Automatic Shutdown**: Workers terminate gracefully when all handles are dropped //! //! Individual [`ProofTaskTx`] instances manage a dedicated [`InMemoryTrieCursorFactory`] and //! [`HashedPostStateCursorFactory`], which are each backed by a database transaction. @@ -65,9 +70,6 @@ type AccountMultiproofResult = Result<(DecodedMultiProof, ParallelTrieStats), ParallelStateRootError>; /// Internal message for storage workers. -/// -/// This is NOT exposed publicly. External callers use `ProofTaskKind::StorageProof` or -/// `ProofTaskKind::BlindedStorageNode` which are routed through the manager's `std::mpsc` channel. #[derive(Debug)] enum StorageWorkerJob { /// Storage proof computation request @@ -119,60 +121,12 @@ where "Spawning proof worker pools" ); - // Spawn storage workers (reuse existing spawn_storage_workers logic) - spawn_storage_workers_internal( - &executor, - &view, - &task_ctx, - storage_worker_count, - storage_work_rx, - )?; - - // Spawn account workers (reuse existing spawn_account_workers logic) - spawn_account_workers_internal( - &executor, - &view, - &task_ctx, - account_worker_count, - account_work_rx, - storage_work_tx.clone(), - )?; - - Ok(ProofTaskManagerHandle::new( - storage_work_tx, - account_work_tx, - Arc::new(AtomicUsize::new(0)), - #[cfg(feature = "metrics")] - Arc::new(ProofTaskMetrics::default()), - )) -} - -/// Spawns a pool of storage workers with dedicated database transactions. -/// -/// Each worker receives `StorageWorkerJob` from the channel and processes storage proofs -/// and blinded storage node requests using a dedicated long-lived transaction. -/// -/// # Parameters -/// - `executor`: Tokio runtime handle for spawning blocking tasks -/// - `view`: Consistent database view for creating transactions -/// - `task_ctx`: Shared context with trie updates and prefix sets -/// - `worker_count`: Number of storage workers to spawn -/// - `work_rx`: Receiver for storage worker jobs -fn spawn_storage_workers_internal( - executor: &Handle, - view: &ConsistentDbView, - task_ctx: &ProofTaskCtx, - worker_count: usize, - work_rx: CrossbeamReceiver, -) -> ProviderResult<()> -where - Factory: DatabaseProviderFactory, -{ - for worker_id in 0..worker_count { + // Spawn storage workers + for worker_id in 0..storage_worker_count { let provider_ro = view.provider_ro()?; let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = work_rx.clone(); + let work_rx_clone = storage_work_rx.clone(); executor .spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx_clone, worker_id)); @@ -184,38 +138,12 @@ where ); } - Ok(()) -} - -/// Spawns a pool of account workers with dedicated database transactions. -/// -/// Each worker receives `AccountWorkerJob` from the channel and processes account multiproofs -/// and blinded account node requests using a dedicated long-lived transaction. Account workers -/// can delegate storage proof computation to the storage worker pool. -/// -/// # Parameters -/// - `executor`: Tokio runtime handle for spawning blocking tasks -/// - `view`: Consistent database view for creating transactions -/// - `task_ctx`: Shared context with trie updates and prefix sets -/// - `worker_count`: Number of account workers to spawn -/// - `work_rx`: Receiver for account worker jobs -/// - `storage_work_tx`: Sender to delegate storage proofs to storage worker pool -fn spawn_account_workers_internal( - executor: &Handle, - view: &ConsistentDbView, - task_ctx: &ProofTaskCtx, - worker_count: usize, - work_rx: CrossbeamReceiver, - storage_work_tx: CrossbeamSender, -) -> ProviderResult<()> -where - Factory: DatabaseProviderFactory, -{ - for worker_id in 0..worker_count { + // Spawn account workers + for worker_id in 0..account_worker_count { let provider_ro = view.provider_ro()?; let tx = provider_ro.into_tx(); let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = work_rx.clone(); + let work_rx_clone = account_work_rx.clone(); let storage_work_tx_clone = storage_work_tx.clone(); executor.spawn_blocking(move || { @@ -229,69 +157,13 @@ where ); } - Ok(()) -} - -/// Manager for coordinating proof request execution across different task types. -/// -/// # Architecture -/// -/// This manager operates two distinct worker pools for parallel trie operations: -/// -/// **Worker Pools**: -/// - Pre-spawned workers with dedicated long-lived transactions -/// - **Storage pool**: Handles `StorageProof` and `BlindedStorageNode` requests -/// - **Account pool**: Handles `AccountMultiproof` and `BlindedAccountNode` requests, delegates -/// storage proof computation to storage pool -/// - Tasks queued via crossbeam unbounded channels -/// - Workers continuously process without transaction overhead -/// - Returns error if worker pool is unavailable (all workers panicked) -/// -/// # Public Interface -/// -/// The public interface through `ProofTaskManagerHandle` allows external callers to: -/// - Submit tasks via `queue_task(ProofTaskKind)` -/// - Use standard `std::mpsc` message passing -/// - Receive consistent return types and error handling -/// -/// # Deprecation Notice -/// -/// This struct is deprecated. Use `spawn_proof_workers()` instead, which returns -/// a `ProofTaskManagerHandle` directly without requiring a separate manager instance. -#[deprecated(note = "Use spawn_proof_workers() instead")] -#[allow(deprecated)] -#[derive(Debug)] -pub struct ProofTaskManager { - /// Sender for storage worker jobs to worker pool. - storage_work_tx: CrossbeamSender, - - /// Number of storage workers successfully spawned. - /// - /// May be less than requested if concurrency limits reduce the worker budget. - storage_worker_count: usize, - - /// Sender for account worker jobs to worker pool. - account_work_tx: CrossbeamSender, - - /// Number of account workers successfully spawned. - account_worker_count: usize, - - /// Receives proof task requests from [`ProofTaskManagerHandle`]. - proof_task_rx: CrossbeamReceiver, - - /// Sender for creating handles that can queue tasks. - #[allow(dead_code)] - proof_task_tx: CrossbeamSender, - - /// The number of active handles. - /// - /// Incremented in [`ProofTaskManagerHandle::new`] and decremented in - /// [`ProofTaskManagerHandle::drop`]. - active_handles: Arc, - - /// Metrics tracking proof task operations. - #[cfg(feature = "metrics")] - metrics: ProofTaskMetrics, + Ok(ProofTaskManagerHandle::new( + storage_work_tx, + account_work_tx, + Arc::new(AtomicUsize::new(0)), + #[cfg(feature = "metrics")] + Arc::new(ProofTaskMetrics::default()), + )) } /// Worker loop for storage trie operations. @@ -436,8 +308,6 @@ fn storage_worker_loop( ); } -// TODO: Refactor this with storage_worker_loop. ProofTaskManager should be removed in the following -// pr and `MultiproofManager` should be used instead to dispatch jobs directly. /// Worker loop for account trie operations. /// /// # Lifecycle @@ -809,219 +679,6 @@ fn queue_storage_proofs( Ok(storage_proof_receivers) } -#[allow(deprecated)] -impl ProofTaskManager { - /// Creates a new [`ProofTaskManager`] with pre-spawned storage and account proof workers. - /// - /// This manager coordinates both storage and account worker pools: - /// - Storage workers handle `StorageProof` and `BlindedStorageNode` requests - /// - Account workers handle `AccountMultiproof` and `BlindedAccountNode` requests - /// - /// The `storage_worker_count` determines how many storage workers to spawn, and - /// `account_worker_count` determines how many account workers to spawn. - /// Returns an error if the underlying provider fails to create the transactions required for - /// spawning workers. - pub fn new( - executor: Handle, - view: ConsistentDbView, - task_ctx: ProofTaskCtx, - storage_worker_count: usize, - account_worker_count: usize, - ) -> ProviderResult - where - Factory: DatabaseProviderFactory, - { - // Use unbounded channel for the router to prevent account workers from blocking - // when queuing storage proofs. Account workers queue many storage proofs through - // this channel, and blocking on a bounded channel wastes parallel worker capacity. - let (proof_task_tx, proof_task_rx) = unbounded(); - - // Use unbounded channel to ensure all storage operations are queued to workers. - // This maintains transaction reuse benefits and avoids fallback to on-demand execution. - let (storage_work_tx, storage_work_rx) = unbounded::(); - let (account_work_tx, account_work_rx) = unbounded::(); - - tracing::info!( - target: "trie::proof_task", - storage_worker_count, - account_worker_count, - "Initializing storage and account worker pools with unbounded queues" - ); - - // Spawn storage workers - spawn_storage_workers_internal( - &executor, - &view, - &task_ctx, - storage_worker_count, - storage_work_rx, - )?; - - // Spawn account workers with direct access to the storage worker queue - spawn_account_workers_internal( - &executor, - &view, - &task_ctx, - account_worker_count, - account_work_rx, - storage_work_tx.clone(), - )?; - - Ok(Self { - storage_work_tx, - storage_worker_count, - account_work_tx, - account_worker_count, - proof_task_rx, - proof_task_tx, - active_handles: Arc::new(AtomicUsize::new(0)), - - #[cfg(feature = "metrics")] - metrics: ProofTaskMetrics::default(), - }) - } - - /// Returns a handle for sending new proof tasks to the [`ProofTaskManager`]. - /// - /// DEPRECATED: This method returns a handle that uses the deprecated routing mechanism. - /// Use `spawn_proof_workers()` instead for direct worker pool access. - #[deprecated(note = "Use spawn_proof_workers() instead")] - pub fn handle(&self) -> ProofTaskManagerHandle { - ProofTaskManagerHandle::new( - self.storage_work_tx.clone(), - self.account_work_tx.clone(), - self.active_handles.clone(), - #[cfg(feature = "metrics")] - Arc::new(ProofTaskMetrics::default()), - ) - } - - /// Loops, managing the proof tasks, routing them to the appropriate worker pools. - /// - /// # Task Routing - /// - /// - **Storage Trie Operations** (`StorageProof` and `BlindedStorageNode`): Routed to - /// pre-spawned storage worker pool via unbounded channel. Returns error if workers are - /// disconnected (e.g., all workers panicked). - /// - **Account Trie Operations** (`AccountMultiproof` and `BlindedAccountNode`): Routed to - /// pre-spawned account worker pool via unbounded channel. Returns error if workers are - /// disconnected. - /// - /// # Shutdown - /// - /// On termination, `storage_work_tx` and `account_work_tx` are dropped, closing the channels - /// and signaling all workers to shut down gracefully. - /// - /// # Deprecation Notice - /// - /// This method is deprecated. With `spawn_proof_workers()`, workers are spawned directly - /// and no routing thread is needed. Workers shut down automatically when all handles are - /// dropped. - #[deprecated(note = "Use spawn_proof_workers() instead - no routing thread needed")] - pub fn run(self) -> ProviderResult<()> { - loop { - match self.proof_task_rx.recv() { - Ok(message) => { - match message { - ProofTaskMessage::QueueTask(task) => match task { - ProofTaskKind::StorageProof(input, sender) => { - self.storage_work_tx - .send(StorageWorkerJob::StorageProof { - input, - result_sender: sender, - }) - .expect("storage worker pool should be available"); - - tracing::trace!( - target: "trie::proof_task", - "Storage proof dispatched to worker pool" - ); - } - - ProofTaskKind::BlindedStorageNode(account, path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.storage_nodes.fetch_add(1, Ordering::Relaxed); - } - - self.storage_work_tx - .send(StorageWorkerJob::BlindedStorageNode { - account, - path, - result_sender: sender, - }) - .expect("storage worker pool should be available"); - - tracing::trace!( - target: "trie::proof_task", - ?account, - ?path, - "Blinded storage node dispatched to worker pool" - ); - } - - ProofTaskKind::BlindedAccountNode(path, sender) => { - #[cfg(feature = "metrics")] - { - self.metrics.account_nodes.fetch_add(1, Ordering::Relaxed); - } - - self.account_work_tx - .send(AccountWorkerJob::BlindedAccountNode { - path, - result_sender: sender, - }) - .expect("account worker pool should be available"); - - tracing::trace!( - target: "trie::proof_task", - ?path, - "Blinded account node dispatched to worker pool" - ); - } - - ProofTaskKind::AccountMultiproof(input, sender) => { - self.account_work_tx - .send(AccountWorkerJob::AccountMultiproof { - input, - result_sender: sender, - }) - .expect("account worker pool should be available"); - - tracing::trace!( - target: "trie::proof_task", - "Account multiproof dispatched to worker pool" - ); - } - }, - ProofTaskMessage::Terminate => { - // Drop worker channels to signal workers to shut down - drop(self.storage_work_tx); - drop(self.account_work_tx); - - tracing::debug!( - target: "trie::proof_task", - storage_worker_count = self.storage_worker_count, - account_worker_count = self.account_worker_count, - "Shutting down proof task manager, signaling workers to terminate" - ); - - // Record metrics before terminating - #[cfg(feature = "metrics")] - self.metrics.record(); - - return Ok(()) - } - } - } - // All senders are disconnected, so we can terminate - // However this should never happen, as this struct stores a sender - Err(_) => return Ok(()), - }; - } - } -} - /// Type alias for the factory tuple returned by `create_factories` type ProofFactories<'a, Tx> = ( InMemoryTrieCursorFactory, &'a TrieUpdatesSorted>, @@ -1037,8 +694,7 @@ pub struct ProofTaskTx { /// Trie updates, prefix sets, and state updates task_ctx: ProofTaskCtx, - /// Identifier for the tx within the context of a single [`ProofTaskManager`], used only for - /// tracing. + /// Identifier for the worker within the worker pool, used only for tracing. id: usize, } @@ -1203,9 +859,6 @@ struct AccountMultiproofParams<'a> { } /// Internal message for account workers. -/// -/// This is NOT exposed publicly. External callers use `ProofTaskKind::AccountMultiproof` or -/// `ProofTaskKind::BlindedAccountNode` which are routed through the manager's `std::mpsc` channel. #[derive(Debug)] enum AccountWorkerJob { /// Account multiproof computation request @@ -1249,42 +902,6 @@ impl ProofTaskCtx { } } -/// Message used to communicate with [`ProofTaskManager`]. -/// -/// DEPRECATED: No longer needed with `spawn_proof_workers()` which provides direct -/// worker pool access. Use explicit queue methods on `ProofTaskManagerHandle` instead. -#[deprecated(note = "Use explicit queue methods on ProofTaskManagerHandle instead")] -#[allow(deprecated)] -#[derive(Debug)] -pub enum ProofTaskMessage { - /// A request to queue a proof task. - QueueTask(ProofTaskKind), - /// A request to terminate the proof task manager. - Terminate, -} - -/// Proof task kind. -/// -/// When queueing a task using [`ProofTaskMessage::QueueTask`], this enum -/// specifies the type of proof task to be executed. -/// -/// DEPRECATED: Use explicit queue methods on `ProofTaskManagerHandle` instead: -/// - `queue_storage_proof()` for storage proofs -/// - `queue_account_multiproof()` for account multiproofs -#[deprecated(note = "Use explicit queue methods on ProofTaskManagerHandle instead")] -#[allow(deprecated)] -#[derive(Debug)] -pub enum ProofTaskKind { - /// A storage proof request. - StorageProof(StorageProofInput, Sender), - /// A blinded account node request. - BlindedAccountNode(Nibbles, Sender), - /// A blinded storage node request. - BlindedStorageNode(B256, Nibbles, Sender), - /// An account multiproof request. - AccountMultiproof(AccountMultiproofInput, Sender), -} - /// A handle that provides type-safe access to proof worker pools. /// /// The handle stores direct senders to both storage and account worker pools, @@ -1394,55 +1011,6 @@ impl ProofTaskManagerHandle { Ok(rx) } - - /// Queues a task to the proof task manager. - /// - /// DEPRECATED: Use explicit methods like `queue_storage_proof` or `queue_account_multiproof` - /// instead. This method is kept temporarily for backwards compatibility during the - /// migration. - #[allow(deprecated)] - #[deprecated(note = "Use explicit queue methods instead")] - pub fn queue_task( - &self, - task: ProofTaskKind, - ) -> Result<(), crossbeam_channel::SendError> { - match task { - ProofTaskKind::StorageProof(input, sender) => { - self.storage_work_tx - .send(StorageWorkerJob::StorageProof { input, result_sender: sender }) - .expect("storage workers should be available"); - } - ProofTaskKind::BlindedStorageNode(account, path, sender) => { - self.storage_work_tx - .send(StorageWorkerJob::BlindedStorageNode { - account, - path, - result_sender: sender, - }) - .expect("storage workers should be available"); - } - ProofTaskKind::BlindedAccountNode(path, sender) => { - self.account_work_tx - .send(AccountWorkerJob::BlindedAccountNode { path, result_sender: sender }) - .expect("account workers should be available"); - } - ProofTaskKind::AccountMultiproof(input, sender) => { - self.account_work_tx - .send(AccountWorkerJob::AccountMultiproof { input, result_sender: sender }) - .expect("account workers should be available"); - } - } - Ok(()) - } - - /// Terminates the proof task manager. - /// - /// DEPRECATED: Workers now shut down automatically when all handles are dropped. - /// This method is kept for backwards compatibility but does nothing. - #[deprecated(note = "Workers shut down automatically when all handles are dropped")] - pub const fn terminate(&self) { - // No-op: workers shut down when all handles are dropped - } } impl Clone for ProofTaskManagerHandle { @@ -1483,14 +1051,14 @@ impl TrieNodeProviderFactory for ProofTaskManagerHandle { pub enum ProofTaskTrieNodeProvider { /// Blinded account trie node provider. AccountNode { - /// Handle to the proof task manager. + /// Handle to the proof worker pools. handle: ProofTaskManagerHandle, }, /// Blinded storage trie node provider. StorageNode { /// Target account. account: B256, - /// Handle to the proof task manager. + /// Handle to the proof worker pools. handle: ProofTaskManagerHandle, }, } From d44180dc1b94909d1175cadf3c5363f7543ae53f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:25:49 +0000 Subject: [PATCH 112/144] fix comment --- crates/engine/tree/src/tree/payload_processor/mod.rs | 3 +-- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 5 ++--- crates/engine/tree/src/tree/payload_validator.rs | 5 ++--- crates/trie/parallel/src/proof.rs | 2 +- crates/trie/parallel/src/proof_task_metrics.rs | 2 +- 5 files changed, 7 insertions(+), 10 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index e0189f40c13..87ca695689d 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -167,8 +167,7 @@ where /// This returns a handle to await the final state root and to interact with the tasks (e.g. /// canceling) /// - /// Returns an error with the original transactions iterator if the proof task manager fails to - /// initialize. + /// Returns an error with the original transactions iterator if proof worker spawning fails. #[allow(clippy::type_complexity)] pub fn spawn>( &mut self, diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 59572a3e9db..b852027e466 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -346,10 +346,9 @@ pub struct MultiproofManager { pending: VecDeque, /// Executor for tasks executor: WorkloadExecutor, - /// Handle to the proof task manager used for creating `ParallelProof` instances for storage - /// proofs. + /// Handle to the proof worker pool for storage proofs. storage_proof_task_handle: ProofTaskManagerHandle, - /// Handle to the proof task manager used for account multiproofs. + /// Handle to the proof worker pool for account multiproofs. account_proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. diff --git a/crates/engine/tree/src/tree/payload_validator.rs b/crates/engine/tree/src/tree/payload_validator.rs index 1e63d29bf79..6c08748cdb5 100644 --- a/crates/engine/tree/src/tree/payload_validator.rs +++ b/crates/engine/tree/src/tree/payload_validator.rs @@ -890,13 +890,12 @@ where (handle, StateRootStrategy::StateRootTask) } Err((error, txs, env, provider_builder)) => { - // Failed to initialize proof task manager, fallback to parallel state - // root + // Failed to spawn proof workers, fallback to parallel state root error!( target: "engine::tree", block=?block_num_hash, ?error, - "Failed to initialize proof task manager, falling back to parallel state root" + "Failed to spawn proof workers, falling back to parallel state root" ); ( self.payload_processor.spawn_cache_exclusive( diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index b33ba89bd54..841779aa982 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -36,7 +36,7 @@ pub struct ParallelProof { collect_branch_node_masks: bool, /// Provided by the user to give the necessary context to retain extra proofs. multi_added_removed_keys: Option>, - /// Handle to the proof task manager. + /// Handle to the proof worker pools. proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index 0274cfec8de..86523bedf69 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -4,7 +4,7 @@ use std::sync::{ Arc, }; -/// Metrics for blinded node fetching for the duration of the proof task manager. +/// Metrics for blinded node fetching by proof workers. #[derive(Clone, Debug)] pub struct ProofTaskMetrics { /// The actual metrics for blinded nodes. From 0b18f6488af12707c60db2882e49baf73b3b00d7 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:26:56 +0000 Subject: [PATCH 113/144] feat: introduce minimum worker count configuration - Added a constant `MIN_WORKER_COUNT` to enforce a minimum number of workers for storage and account proof tasks. - Updated `default_storage_worker_count` and `default_account_worker_count` functions to utilize the new minimum constraint. - Enhanced setter methods in `TreeConfig` to ensure worker counts do not fall below the minimum. - Modified command-line argument parsing to validate worker counts against the minimum requirement. --- crates/engine/primitives/src/config.rs | 13 +++++++++---- crates/node/core/src/args/engine.rs | 12 +++++++++--- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 7d3d9c1c37f..1d0c3d7a991 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -9,11 +9,16 @@ pub const DEFAULT_MEMORY_BLOCK_BUFFER_TARGET: u64 = 0; /// Default maximum concurrency for on-demand proof tasks (blinded nodes) pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; +/// Minimum number of workers we allow configuring explicitly. +pub const MIN_WORKER_COUNT: usize = 2; + /// Returns the default number of storage worker threads based on available parallelism. fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] { - std::thread::available_parallelism().map(|n| (n.get() * 2).clamp(2, 64)).unwrap_or(8) + std::thread::available_parallelism() + .map(|n| (n.get() * 2).clamp(MIN_WORKER_COUNT, 64)) + .unwrap_or(8) } #[cfg(not(feature = "std"))] { @@ -28,7 +33,7 @@ fn default_storage_worker_count() -> usize { /// so we use higher concurrency (1.5x storage workers) to maximize throughput and overlap. /// While storage workers are CPU-bound, account workers are I/O-bound coordinators. fn default_account_worker_count() -> usize { - (default_storage_worker_count() * 3) / 2 + ((default_storage_worker_count() * 3) / 2).max(MIN_WORKER_COUNT) } /// The size of proof targets chunk to spawn in one multiproof calculation. @@ -494,7 +499,7 @@ impl TreeConfig { /// Setter for the number of storage proof worker threads. pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { - self.storage_worker_count = storage_worker_count; + self.storage_worker_count = storage_worker_count.max(MIN_WORKER_COUNT); self } @@ -505,7 +510,7 @@ impl TreeConfig { /// Setter for the number of account proof worker threads. pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = account_worker_count; + self.account_worker_count = account_worker_count.max(MIN_WORKER_COUNT); self } } diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index fb98847d8c7..c4681fbe77d 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -1,7 +1,7 @@ //! clap [Args](clap::Args) for engine purposes use clap::Args; -use reth_engine_primitives::{TreeConfig, DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE}; +use reth_engine_primitives::{TreeConfig, DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE, MIN_WORKER_COUNT}; use crate::node_config::{ DEFAULT_CROSS_BLOCK_CACHE_SIZE_MB, DEFAULT_MAX_PROOF_TASK_CONCURRENCY, @@ -111,12 +111,18 @@ pub struct EngineArgs { /// Configure the number of storage proof workers in the Tokio blocking pool. /// If not specified, defaults to 2x available parallelism, clamped between 2 and 64. - #[arg(long = "engine.storage-worker-count")] + #[arg( + long = "engine.storage-worker-count", + value_parser = clap::value_parser!(usize).range(MIN_WORKER_COUNT..) + )] pub storage_worker_count: Option, /// Configure the number of account proof workers in the Tokio blocking pool. /// If not specified, defaults to 1.5x storage workers. - #[arg(long = "engine.account-worker-count")] + #[arg( + long = "engine.account-worker-count", + value_parser = clap::value_parser!(usize).range(MIN_WORKER_COUNT..) + )] pub account_worker_count: Option, } From 42ceeddef79e557cbda775de553fc926f42bb44c Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:34:14 +0000 Subject: [PATCH 114/144] fix: prevent active_handles underflow in ProofTaskManagerHandle - Added a debug assertion to ensure active_handles does not underflow when dropping a ProofTaskManagerHandle. - Implemented metrics recording to flush before exit when the last handle is dropped, enhancing monitoring capabilities. --- crates/trie/parallel/src/proof_task.rs | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 7d923acc6ac..a01b1540e2e 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1029,7 +1029,19 @@ impl Drop for ProofTaskManagerHandle { fn drop(&mut self) { // Decrement the number of active handles. // When the last handle is dropped, the channels are dropped and workers shut down. - self.active_handles.fetch_sub(1, Ordering::SeqCst); + // atomically grab the current handle count and decrement it for Drop. + let previous_handles = self.active_handles.fetch_sub(1, Ordering::SeqCst); + + debug_assert_ne!( + previous_handles, 0, + "active_handles underflow in ProofTaskManagerHandle::drop" + ); + + #[cfg(feature = "metrics")] + if previous_handles == 1 { + // Flush metrics before exit. + self.metrics.record(); + } } } From 8e00a4a5b98a11c156c6ac5186bf1bf08b4da33a Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:39:12 +0000 Subject: [PATCH 115/144] clippy --- crates/engine/primitives/src/config.rs | 12 ++++++++++-- crates/node/core/src/args/engine.rs | 12 +++--------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 1d0c3d7a991..ded1175524e 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -499,7 +499,11 @@ impl TreeConfig { /// Setter for the number of storage proof worker threads. pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { - self.storage_worker_count = storage_worker_count.max(MIN_WORKER_COUNT); + self.storage_worker_count = if storage_worker_count > MIN_WORKER_COUNT { + storage_worker_count + } else { + MIN_WORKER_COUNT + }; self } @@ -510,7 +514,11 @@ impl TreeConfig { /// Setter for the number of account proof worker threads. pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = account_worker_count.max(MIN_WORKER_COUNT); + self.account_worker_count = if account_worker_count > MIN_WORKER_COUNT { + account_worker_count + } else { + MIN_WORKER_COUNT + }; self } } diff --git a/crates/node/core/src/args/engine.rs b/crates/node/core/src/args/engine.rs index c4681fbe77d..fb98847d8c7 100644 --- a/crates/node/core/src/args/engine.rs +++ b/crates/node/core/src/args/engine.rs @@ -1,7 +1,7 @@ //! clap [Args](clap::Args) for engine purposes use clap::Args; -use reth_engine_primitives::{TreeConfig, DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE, MIN_WORKER_COUNT}; +use reth_engine_primitives::{TreeConfig, DEFAULT_MULTIPROOF_TASK_CHUNK_SIZE}; use crate::node_config::{ DEFAULT_CROSS_BLOCK_CACHE_SIZE_MB, DEFAULT_MAX_PROOF_TASK_CONCURRENCY, @@ -111,18 +111,12 @@ pub struct EngineArgs { /// Configure the number of storage proof workers in the Tokio blocking pool. /// If not specified, defaults to 2x available parallelism, clamped between 2 and 64. - #[arg( - long = "engine.storage-worker-count", - value_parser = clap::value_parser!(usize).range(MIN_WORKER_COUNT..) - )] + #[arg(long = "engine.storage-worker-count")] pub storage_worker_count: Option, /// Configure the number of account proof workers in the Tokio blocking pool. /// If not specified, defaults to 1.5x storage workers. - #[arg( - long = "engine.account-worker-count", - value_parser = clap::value_parser!(usize).range(MIN_WORKER_COUNT..) - )] + #[arg(long = "engine.account-worker-count")] pub account_worker_count: Option, } From 2b90133ca02bb03e5dfc6f7f528179921be50730 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:39:25 +0000 Subject: [PATCH 116/144] fmt --- crates/trie/parallel/src/proof_task.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index a01b1540e2e..01397c98445 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -922,8 +922,10 @@ pub struct ProofTaskManagerHandle { impl ProofTaskManagerHandle { /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. - #[allow(private_interfaces)] - pub fn new( + /// + /// This is an internal constructor used by `spawn_proof_workers`. + /// External users should call `spawn_proof_workers` to create handles. + fn new( storage_work_tx: CrossbeamSender, account_work_tx: CrossbeamSender, active_handles: Arc, @@ -1034,7 +1036,8 @@ impl Drop for ProofTaskManagerHandle { debug_assert_ne!( previous_handles, 0, - "active_handles underflow in ProofTaskManagerHandle::drop" + "active_handles underflow in ProofTaskManagerHandle::drop (previous={})", + previous_handles ); #[cfg(feature = "metrics")] From 3f6400eb8ce3b3780173c3cc330787836a82f94e Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:50:54 +0000 Subject: [PATCH 117/144] refactor: improve error handling in ProofTaskTrieNodeProvider - Introduced helper functions to streamline error conversion from ProviderError and channel receive errors to SparseTrieError. - Enhanced readability and maintainability of the trie_node method by reducing repetitive error handling code. --- crates/trie/parallel/src/proof_task.rs | 29 +++++++++++++++----------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 01397c98445..5342fff1eb7 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1080,22 +1080,27 @@ pub enum ProofTaskTrieNodeProvider { impl TrieNodeProvider for ProofTaskTrieNodeProvider { fn trie_node(&self, path: &Nibbles) -> Result, SparseTrieError> { + /// Helper to convert `ProviderError` to `SparseTrieError` + fn provider_err_to_trie_err(e: ProviderError) -> SparseTrieError { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))).into() + } + + /// Helper to convert channel recv error to `SparseTrieError` + fn recv_err_to_trie_err(_: std::sync::mpsc::RecvError) -> SparseTrieError { + SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))).into() + } + match self { Self::AccountNode { handle } => { - let rx = handle.queue_blinded_account_node(*path).map_err(|e| { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))) - })?; - rx.recv().map_err(|_| { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))) - })? + let rx = + handle.queue_blinded_account_node(*path).map_err(provider_err_to_trie_err)?; + rx.recv().map_err(recv_err_to_trie_err)? } Self::StorageNode { handle, account } => { - let rx = handle.queue_blinded_storage_node(*account, *path).map_err(|e| { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))) - })?; - rx.recv().map_err(|_| { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))) - })? + let rx = handle + .queue_blinded_storage_node(*account, *path) + .map_err(provider_err_to_trie_err)?; + rx.recv().map_err(recv_err_to_trie_err)? } } } From f302447480ed7e792c6b0a3e483d3623b67e71d8 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 10:55:40 +0000 Subject: [PATCH 118/144] fix count --- crates/engine/primitives/src/config.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index ded1175524e..91f7613072a 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -499,7 +499,7 @@ impl TreeConfig { /// Setter for the number of storage proof worker threads. pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { - self.storage_worker_count = if storage_worker_count > MIN_WORKER_COUNT { + self.storage_worker_count = if storage_worker_count >= MIN_WORKER_COUNT { storage_worker_count } else { MIN_WORKER_COUNT @@ -514,7 +514,7 @@ impl TreeConfig { /// Setter for the number of account proof worker threads. pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = if account_worker_count > MIN_WORKER_COUNT { + self.account_worker_count = if account_worker_count >= MIN_WORKER_COUNT { account_worker_count } else { MIN_WORKER_COUNT From d1eb0ecd5c8f3007e75465b484fa16b14c1959ab Mon Sep 17 00:00:00 2001 From: YK Date: Fri, 10 Oct 2025 18:58:31 +0800 Subject: [PATCH 119/144] Update crates/trie/parallel/src/proof_task.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 18062747901..1d64ef25d3a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -903,7 +903,7 @@ impl ProofTaskManager { path, result_sender: sender, }) - .expect("account worker pool should be available"); + .expect("failed to send job to account worker pool: all workers panicked or pool was shut down"); tracing::trace!( target: "trie::proof_task", From 4e009444d151ecbad86c1e551d57ff71e038ac06 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 11:01:58 +0000 Subject: [PATCH 120/144] merge --- crates/trie/parallel/src/proof_task.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 1d64ef25d3a..780839c238a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -862,7 +862,7 @@ impl ProofTaskManager { input, result_sender: sender, }) - .expect("storage worker pool should be available"); + .expect("failed to dispatch storage proof: storage worker pool unavailable (all workers panicked or pool shut down)"); tracing::trace!( target: "trie::proof_task", @@ -882,7 +882,7 @@ impl ProofTaskManager { path, result_sender: sender, }) - .expect("storage worker pool should be available"); + .expect("failed to dispatch blinded storage node: storage worker pool unavailable (all workers panicked or pool shut down)"); tracing::trace!( target: "trie::proof_task", @@ -903,7 +903,7 @@ impl ProofTaskManager { path, result_sender: sender, }) - .expect("failed to send job to account worker pool: all workers panicked or pool was shut down"); + .expect("failed to dispatch blinded account node: account worker pool unavailable (all workers panicked or pool shut down)"); tracing::trace!( target: "trie::proof_task", @@ -918,7 +918,7 @@ impl ProofTaskManager { input, result_sender: sender, }) - .expect("account worker pool should be available"); + .expect("failed to dispatch account multiproof: account worker pool unavailable (all workers panicked or pool shut down)"); tracing::trace!( target: "trie::proof_task", From 833b0310256c5483380dff30d1d345d517aa27ee Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 11:04:50 +0000 Subject: [PATCH 121/144] refactor: streamline worker count validation - Introduced a `clamp_worker_count` function to centralize the logic for enforcing the minimum worker count. - Updated setter methods in `TreeConfig` to utilize the new clamping function, improving code readability and maintainability. --- crates/engine/primitives/src/config.rs | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 91f7613072a..aaa043cb5b7 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -12,6 +12,17 @@ pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; /// Minimum number of workers we allow configuring explicitly. pub const MIN_WORKER_COUNT: usize = 2; +/// Clamps the worker count to the minimum allowed value. +/// +/// Ensures that the worker count is at least [`MIN_WORKER_COUNT`]. +const fn clamp_worker_count(count: usize) -> usize { + if count >= MIN_WORKER_COUNT { + count + } else { + MIN_WORKER_COUNT + } +} + /// Returns the default number of storage worker threads based on available parallelism. fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] @@ -499,11 +510,7 @@ impl TreeConfig { /// Setter for the number of storage proof worker threads. pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { - self.storage_worker_count = if storage_worker_count >= MIN_WORKER_COUNT { - storage_worker_count - } else { - MIN_WORKER_COUNT - }; + self.storage_worker_count = clamp_worker_count(storage_worker_count); self } @@ -514,11 +521,7 @@ impl TreeConfig { /// Setter for the number of account proof worker threads. pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = if account_worker_count >= MIN_WORKER_COUNT { - account_worker_count - } else { - MIN_WORKER_COUNT - }; + self.account_worker_count = clamp_worker_count(account_worker_count); self } } From 40bb506162f93c9e452fd288d65ffa51be98fef7 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 12:42:32 +0000 Subject: [PATCH 122/144] refactor: consolidate proof task handles in MultiproofManager - Merged separate storage and account proof task handles into a single proof task handle for improved code clarity and maintainability. - Updated related methods to utilize the consolidated handle, streamlining the management of proof tasks. --- .../src/tree/payload_processor/multiproof.rs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index b852027e466..3d2a7a87da7 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -346,10 +346,8 @@ pub struct MultiproofManager { pending: VecDeque, /// Executor for tasks executor: WorkloadExecutor, - /// Handle to the proof worker pool for storage proofs. - storage_proof_task_handle: ProofTaskManagerHandle, - /// Handle to the proof worker pool for account multiproofs. - account_proof_task_handle: ProofTaskManagerHandle, + /// Handle to the proof worker pools (storage and account). + proof_task_handle: ProofTaskManagerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. /// @@ -371,8 +369,7 @@ impl MultiproofManager { fn new( executor: WorkloadExecutor, metrics: MultiProofTaskMetrics, - storage_proof_task_handle: ProofTaskManagerHandle, - account_proof_task_handle: ProofTaskManagerHandle, + proof_task_handle: ProofTaskManagerHandle, max_concurrent: usize, ) -> Self { Self { @@ -381,8 +378,7 @@ impl MultiproofManager { executor, inflight: 0, metrics, - storage_proof_task_handle, - account_proof_task_handle, + proof_task_handle, missed_leaves_storage_roots: Default::default(), } } @@ -451,7 +447,7 @@ impl MultiproofManager { multi_added_removed_keys, } = storage_multiproof_input; - let storage_proof_task_handle = self.storage_proof_task_handle.clone(); + let storage_proof_task_handle = self.proof_task_handle.clone(); let missed_leaves_storage_roots = self.missed_leaves_storage_roots.clone(); self.executor.spawn_blocking(move || { @@ -523,7 +519,7 @@ impl MultiproofManager { state_root_message_sender, multi_added_removed_keys, } = multiproof_input; - let account_proof_task_handle = self.account_proof_task_handle.clone(); + let account_proof_task_handle = self.proof_task_handle.clone(); let missed_leaves_storage_roots = self.missed_leaves_storage_roots.clone(); self.executor.spawn_blocking(move || { @@ -707,8 +703,7 @@ impl MultiProofTask { multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle.clone(), // handle for storage proof workers - proof_task_handle, // handle for account proof workers + proof_task_handle, max_concurrency, ), metrics, From e49791e9ae54abf1bf95a3291a3e8902d0e1c251 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 12:42:50 +0000 Subject: [PATCH 123/144] refactor: simplify ProofTaskMetrics default implementation - Updated the ProofTaskMetrics struct to derive Default, removing the manual implementation of the default method. - This change enhances code clarity and reduces boilerplate, while maintaining the same functionality. --- crates/trie/parallel/src/proof_task_metrics.rs | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index 86523bedf69..5f8959cea9d 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -5,7 +5,7 @@ use std::sync::{ }; /// Metrics for blinded node fetching by proof workers. -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Default)] pub struct ProofTaskMetrics { /// The actual metrics for blinded nodes. pub task_metrics: ProofTaskTrieMetrics, @@ -19,18 +19,6 @@ pub struct ProofTaskMetrics { pub storage_nodes: Arc, } -impl Default for ProofTaskMetrics { - fn default() -> Self { - Self { - task_metrics: ProofTaskTrieMetrics::default(), - storage_proofs: Arc::new(AtomicU64::new(0)), - account_proofs: Arc::new(AtomicU64::new(0)), - account_nodes: Arc::new(AtomicU64::new(0)), - storage_nodes: Arc::new(AtomicU64::new(0)), - } - } -} - impl ProofTaskMetrics { /// Record the blinded node counts into the histograms. pub fn record(&self) { From c02a68dc78eb4e080288ed6779439fd1d3169667 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Fri, 10 Oct 2025 12:43:21 +0000 Subject: [PATCH 124/144] refactor: improve error handling in trie_node method - Updated the error conversion helper function in ProofTaskTrieNodeProvider to directly wrap the ProviderError, enhancing clarity and maintainability. - This change simplifies the error handling logic within the trie_node method. --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 5342fff1eb7..231b0cb32b0 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1082,7 +1082,7 @@ impl TrieNodeProvider for ProofTaskTrieNodeProvider { fn trie_node(&self, path: &Nibbles) -> Result, SparseTrieError> { /// Helper to convert `ProviderError` to `SparseTrieError` fn provider_err_to_trie_err(e: ProviderError) -> SparseTrieError { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other(e.to_string()))).into() + SparseTrieErrorKind::Other(Box::new(e)).into() } /// Helper to convert channel recv error to `SparseTrieError` From 2698cc80072179a081d5fea56b17821d94199e83 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 02:11:12 +0000 Subject: [PATCH 125/144] refactor: enhance error handling in trie_node method - Removed redundant helper functions for error conversion in ProofTaskTrieNodeProvider. - Simplified error handling by directly mapping errors to SparseTrieError, improving code clarity and maintainability. --- crates/trie/parallel/src/proof_task.rs | 21 ++++++--------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 231b0cb32b0..8545b7d2f28 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -1080,27 +1080,18 @@ pub enum ProofTaskTrieNodeProvider { impl TrieNodeProvider for ProofTaskTrieNodeProvider { fn trie_node(&self, path: &Nibbles) -> Result, SparseTrieError> { - /// Helper to convert `ProviderError` to `SparseTrieError` - fn provider_err_to_trie_err(e: ProviderError) -> SparseTrieError { - SparseTrieErrorKind::Other(Box::new(e)).into() - } - - /// Helper to convert channel recv error to `SparseTrieError` - fn recv_err_to_trie_err(_: std::sync::mpsc::RecvError) -> SparseTrieError { - SparseTrieErrorKind::Other(Box::new(std::io::Error::other("channel closed"))).into() - } - match self { Self::AccountNode { handle } => { - let rx = - handle.queue_blinded_account_node(*path).map_err(provider_err_to_trie_err)?; - rx.recv().map_err(recv_err_to_trie_err)? + let rx = handle + .queue_blinded_account_node(*path) + .map_err(|error| SparseTrieErrorKind::Other(Box::new(error)))?; + rx.recv().map_err(|error| SparseTrieErrorKind::Other(Box::new(error)))? } Self::StorageNode { handle, account } => { let rx = handle .queue_blinded_storage_node(*account, *path) - .map_err(provider_err_to_trie_err)?; - rx.recv().map_err(recv_err_to_trie_err)? + .map_err(|error| SparseTrieErrorKind::Other(Box::new(error)))?; + rx.recv().map_err(|error| SparseTrieErrorKind::Other(Box::new(error)))? } } } From c68eb7a9fd93022bf3c1b59811525b8177863336 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 02:11:29 +0000 Subject: [PATCH 126/144] refactor: simplify worker count handling in TreeConfig - Removed the `clamp_worker_count` function and replaced its logic with direct usage of `max` in the setter methods for storage and account worker counts. - This change enhances code clarity and reduces unnecessary function overhead while ensuring the minimum worker count is enforced. --- crates/engine/primitives/src/config.rs | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index aaa043cb5b7..0b77119c457 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -12,17 +12,6 @@ pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; /// Minimum number of workers we allow configuring explicitly. pub const MIN_WORKER_COUNT: usize = 2; -/// Clamps the worker count to the minimum allowed value. -/// -/// Ensures that the worker count is at least [`MIN_WORKER_COUNT`]. -const fn clamp_worker_count(count: usize) -> usize { - if count >= MIN_WORKER_COUNT { - count - } else { - MIN_WORKER_COUNT - } -} - /// Returns the default number of storage worker threads based on available parallelism. fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] @@ -509,8 +498,8 @@ impl TreeConfig { } /// Setter for the number of storage proof worker threads. - pub const fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { - self.storage_worker_count = clamp_worker_count(storage_worker_count); + pub fn with_storage_worker_count(mut self, storage_worker_count: usize) -> Self { + self.storage_worker_count = storage_worker_count.max(MIN_WORKER_COUNT); self } @@ -520,8 +509,8 @@ impl TreeConfig { } /// Setter for the number of account proof worker threads. - pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = clamp_worker_count(account_worker_count); + pub fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { + self.account_worker_count = account_worker_count.max(MIN_WORKER_COUNT); self } } From 6a4766ce473d3c6e98c320fa2d00f1be12c1c5dc Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 05:07:38 +0000 Subject: [PATCH 127/144] fix clippy --- .../e2e-test-utils/src/testsuite/actions/produce_blocks.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs index 9d2088c11a4..74a5e2ba1d5 100644 --- a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs +++ b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs @@ -510,7 +510,7 @@ where Box::pin(async move { let mut accepted_check: bool = false; - let mut latest_block = env + let latest_block = env .current_block_info() .ok_or_else(|| eyre::eyre!("No latest block information available"))?; @@ -603,10 +603,6 @@ where rpc_latest_header.inner.timestamp; env.active_node_state_mut()?.latest_fork_choice_state.head_block_hash = rpc_latest_header.hash; - - // update local copy for any further usage in this scope - latest_block.hash = rpc_latest_header.hash; - latest_block.number = rpc_latest_header.inner.number; } } From 3e957a5f9e327448df10822e7564527a164a63cc Mon Sep 17 00:00:00 2001 From: YK Date: Mon, 13 Oct 2025 13:13:34 +0800 Subject: [PATCH 128/144] refactor(tree): remove unused Factory generic from multiproof system (#18933) --- .../tree/src/tree/payload_processor/mod.rs | 5 +- .../src/tree/payload_processor/multiproof.rs | 115 +++++++----------- crates/trie/parallel/src/proof.rs | 22 +--- 3 files changed, 49 insertions(+), 93 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index c60b73ce514..c24b0d1fe16 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -192,8 +192,7 @@ where { let (to_sparse_trie, sparse_trie_rx) = channel(); // spawn multiproof task, save the trie input - let (trie_input, state_root_config) = - MultiProofConfig::new_from_input(consistent_view, trie_input); + let (trie_input, state_root_config) = MultiProofConfig::from_input(trie_input); self.trie_input = Some(trie_input); // Create and spawn the storage proof task @@ -207,7 +206,7 @@ where let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; let proof_task = match ProofTaskManager::new( self.executor.handle().clone(), - state_root_config.consistent_view.clone(), + consistent_view, task_ctx, storage_worker_count, account_worker_count, diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 9045f057d24..f865312b83d 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -12,7 +12,6 @@ use derive_more::derive::Deref; use metrics::Histogram; use reth_errors::ProviderError; use reth_metrics::Metrics; -use reth_provider::{providers::ConsistentDbView, BlockReader, DatabaseProviderFactory}; use reth_revm::state::EvmState; use reth_trie::{ added_removed_keys::MultiAddedRemovedKeys, prefix_set::TriePrefixSetsMut, @@ -66,9 +65,7 @@ impl SparseTrieUpdate { /// Common configuration for multi proof tasks #[derive(Debug, Clone)] -pub(super) struct MultiProofConfig { - /// View over the state in the database. - pub consistent_view: ConsistentDbView, +pub(super) struct MultiProofConfig { /// The sorted collection of cached in-memory intermediate trie nodes that /// can be reused for computation. pub nodes_sorted: Arc, @@ -80,17 +77,13 @@ pub(super) struct MultiProofConfig { pub prefix_sets: Arc, } -impl MultiProofConfig { - /// Creates a new state root config from the consistent view and the trie input. +impl MultiProofConfig { + /// Creates a new state root config from the trie input. /// /// This returns a cleared [`TrieInput`] so that we can reuse any allocated space in the /// [`TrieInput`]. - pub(super) fn new_from_input( - consistent_view: ConsistentDbView, - mut input: TrieInput, - ) -> (TrieInput, Self) { + pub(super) fn from_input(mut input: TrieInput) -> (TrieInput, Self) { let config = Self { - consistent_view, nodes_sorted: Arc::new(input.nodes.drain_into_sorted()), state_sorted: Arc::new(input.state.drain_into_sorted()), prefix_sets: Arc::new(input.prefix_sets.clone()), @@ -249,14 +242,14 @@ pub(crate) fn evm_state_to_hashed_post_state(update: EvmState) -> HashedPostStat /// A pending multiproof task, either [`StorageMultiproofInput`] or [`MultiproofInput`]. #[derive(Debug)] -enum PendingMultiproofTask { +enum PendingMultiproofTask { /// A storage multiproof task input. - Storage(StorageMultiproofInput), + Storage(StorageMultiproofInput), /// A regular multiproof task input. - Regular(MultiproofInput), + Regular(MultiproofInput), } -impl PendingMultiproofTask { +impl PendingMultiproofTask { /// Returns the proof sequence number of the task. const fn proof_sequence_number(&self) -> u64 { match self { @@ -282,22 +275,22 @@ impl PendingMultiproofTask { } } -impl From> for PendingMultiproofTask { - fn from(input: StorageMultiproofInput) -> Self { +impl From for PendingMultiproofTask { + fn from(input: StorageMultiproofInput) -> Self { Self::Storage(input) } } -impl From> for PendingMultiproofTask { - fn from(input: MultiproofInput) -> Self { +impl From for PendingMultiproofTask { + fn from(input: MultiproofInput) -> Self { Self::Regular(input) } } /// Input parameters for spawning a dedicated storage multiproof calculation. #[derive(Debug)] -struct StorageMultiproofInput { - config: MultiProofConfig, +struct StorageMultiproofInput { + config: MultiProofConfig, source: Option, hashed_state_update: HashedPostState, hashed_address: B256, @@ -307,7 +300,7 @@ struct StorageMultiproofInput { multi_added_removed_keys: Arc, } -impl StorageMultiproofInput { +impl StorageMultiproofInput { /// Destroys the input and sends a [`MultiProofMessage::EmptyProof`] message to the sender. fn send_empty_proof(self) { let _ = self.state_root_message_sender.send(MultiProofMessage::EmptyProof { @@ -319,8 +312,8 @@ impl StorageMultiproofInput { /// Input parameters for spawning a multiproof calculation. #[derive(Debug)] -struct MultiproofInput { - config: MultiProofConfig, +struct MultiproofInput { + config: MultiProofConfig, source: Option, hashed_state_update: HashedPostState, proof_targets: MultiProofTargets, @@ -329,7 +322,7 @@ struct MultiproofInput { multi_added_removed_keys: Option>, } -impl MultiproofInput { +impl MultiproofInput { /// Destroys the input and sends a [`MultiProofMessage::EmptyProof`] message to the sender. fn send_empty_proof(self) { let _ = self.state_root_message_sender.send(MultiProofMessage::EmptyProof { @@ -344,13 +337,13 @@ impl MultiproofInput { /// concurrency, further calculation requests are queued and spawn later, after /// availability has been signaled. #[derive(Debug)] -pub struct MultiproofManager { +pub struct MultiproofManager { /// Maximum number of concurrent calculations. max_concurrent: usize, /// Currently running calculations. inflight: usize, /// Queued calculations. - pending: VecDeque>, + pending: VecDeque, /// Executor for tasks executor: WorkloadExecutor, /// Handle to the proof task manager used for creating `ParallelProof` instances for storage @@ -374,10 +367,7 @@ pub struct MultiproofManager { metrics: MultiProofTaskMetrics, } -impl MultiproofManager -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ +impl MultiproofManager { /// Creates a new [`MultiproofManager`]. fn new( executor: WorkloadExecutor, @@ -404,7 +394,7 @@ where /// Spawns a new multiproof calculation or enqueues it for later if /// `max_concurrent` are already inflight. - fn spawn_or_queue(&mut self, input: PendingMultiproofTask) { + fn spawn_or_queue(&mut self, input: PendingMultiproofTask) { // If there are no proof targets, we can just send an empty multiproof back immediately if input.proof_targets_is_empty() { debug!( @@ -438,7 +428,7 @@ where /// Spawns a multiproof task, dispatching to `spawn_storage_proof` if the input is a storage /// multiproof, and dispatching to `spawn_multiproof` otherwise. - fn spawn_multiproof_task(&mut self, input: PendingMultiproofTask) { + fn spawn_multiproof_task(&mut self, input: PendingMultiproofTask) { match input { PendingMultiproofTask::Storage(storage_input) => { self.spawn_storage_proof(storage_input); @@ -450,7 +440,7 @@ where } /// Spawns a single storage proof calculation task. - fn spawn_storage_proof(&mut self, storage_multiproof_input: StorageMultiproofInput) { + fn spawn_storage_proof(&mut self, storage_multiproof_input: StorageMultiproofInput) { let StorageMultiproofInput { config, source, @@ -476,7 +466,7 @@ where "Starting dedicated storage proof calculation", ); let start = Instant::now(); - let proof_result = ParallelProof::::new( + let proof_result = ParallelProof::new( config.nodes_sorted, config.state_sorted, config.prefix_sets, @@ -524,7 +514,7 @@ where } /// Spawns a single multiproof calculation task. - fn spawn_multiproof(&mut self, multiproof_input: MultiproofInput) { + fn spawn_multiproof(&mut self, multiproof_input: MultiproofInput) { let MultiproofInput { config, source, @@ -554,10 +544,8 @@ where let start = Instant::now(); // Extend prefix sets with targets - let frozen_prefix_sets = ParallelProof::::extend_prefix_sets_with_targets( - &config.prefix_sets, - &proof_targets, - ); + let frozen_prefix_sets = + ParallelProof::extend_prefix_sets_with_targets(&config.prefix_sets, &proof_targets); // Queue account multiproof to worker pool let input = AccountMultiproofInput { @@ -675,13 +663,13 @@ pub(crate) struct MultiProofTaskMetrics { /// Then it updates relevant leaves according to the result of the transaction. /// This feeds updates to the sparse trie task. #[derive(Debug)] -pub(super) struct MultiProofTask { +pub(super) struct MultiProofTask { /// The size of proof targets chunk to spawn in one calculation. /// /// If [`None`], then chunking is disabled. chunk_size: Option, /// Task configuration. - config: MultiProofConfig, + config: MultiProofConfig, /// Receiver for state root related messages. rx: Receiver, /// Sender for state root related messages. @@ -695,18 +683,15 @@ pub(super) struct MultiProofTask { /// Proof sequencing handler. proof_sequencer: ProofSequencer, /// Manages calculation of multiproofs. - multiproof_manager: MultiproofManager, + multiproof_manager: MultiproofManager, /// multi proof task metrics metrics: MultiProofTaskMetrics, } -impl MultiProofTask -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ +impl MultiProofTask { /// Creates a new multi proof task with the unified message channel pub(super) fn new( - config: MultiProofConfig, + config: MultiProofConfig, executor: WorkloadExecutor, proof_task_handle: ProofTaskManagerHandle, to_sparse_trie: Sender, @@ -1233,43 +1218,29 @@ fn get_proof_targets( mod tests { use super::*; use alloy_primitives::map::B256Set; - use reth_provider::{providers::ConsistentDbView, test_utils::create_test_provider_factory}; + use reth_provider::{ + providers::ConsistentDbView, test_utils::create_test_provider_factory, BlockReader, + DatabaseProviderFactory, + }; use reth_trie::{MultiProof, TrieInput}; use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofTaskManager}; use revm_primitives::{B256, U256}; - use std::sync::Arc; - fn create_state_root_config(factory: F, input: TrieInput) -> MultiProofConfig - where - F: DatabaseProviderFactory + Clone + 'static, - { - let consistent_view = ConsistentDbView::new(factory, None); - let nodes_sorted = Arc::new(input.nodes.clone().into_sorted()); - let state_sorted = Arc::new(input.state.clone().into_sorted()); - let prefix_sets = Arc::new(input.prefix_sets); - - MultiProofConfig { consistent_view, nodes_sorted, state_sorted, prefix_sets } - } - - fn create_test_state_root_task(factory: F) -> MultiProofTask + fn create_test_state_root_task(factory: F) -> MultiProofTask where F: DatabaseProviderFactory + Clone + 'static, { let executor = WorkloadExecutor::default(); - let config = create_state_root_config(factory, TrieInput::default()); + let (_trie_input, config) = MultiProofConfig::from_input(TrieInput::default()); let task_ctx = ProofTaskCtx::new( config.nodes_sorted.clone(), config.state_sorted.clone(), config.prefix_sets.clone(), ); - let proof_task = ProofTaskManager::new( - executor.handle().clone(), - config.consistent_view.clone(), - task_ctx, - 1, - 1, - ) - .expect("Failed to create ProofTaskManager"); + let consistent_view = ConsistentDbView::new(factory, None); + let proof_task = + ProofTaskManager::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) + .expect("Failed to create ProofTaskManager"); let channel = channel(); MultiProofTask::new(config, executor, proof_task.handle(), channel.0, 1, None) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 36c49c038c9..7fc1f022a7e 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -9,7 +9,6 @@ use crate::{ use alloy_primitives::{map::B256Set, B256}; use dashmap::DashMap; use reth_execution_errors::StorageRootError; -use reth_provider::{BlockReader, DatabaseProviderFactory}; use reth_storage_errors::db::DatabaseError; use reth_trie::{ prefix_set::{PrefixSet, PrefixSetMut, TriePrefixSets, TriePrefixSetsMut}, @@ -28,7 +27,7 @@ use tracing::trace; /// This can collect proof for many targets in parallel, spawning a task for each hashed address /// that has proof targets. #[derive(Debug)] -pub struct ParallelProof { +pub struct ParallelProof { /// The sorted collection of cached in-memory intermediate trie nodes that /// can be reused for computation. pub nodes_sorted: Arc, @@ -47,14 +46,11 @@ pub struct ParallelProof { /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, - /// Marker to keep the Factory type parameter. - /// TODO: Remove this field if the Factory generic is not needed in the future. - _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics, } -impl ParallelProof { +impl ParallelProof { /// Create new state proof generator. pub fn new( nodes_sorted: Arc, @@ -71,7 +67,6 @@ impl ParallelProof { collect_branch_node_masks: false, multi_added_removed_keys: None, proof_task_handle, - _phantom: std::marker::PhantomData, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics::new_with_labels(&[("type", "proof")]), } @@ -92,12 +87,6 @@ impl ParallelProof { self.multi_added_removed_keys = multi_added_removed_keys; self } -} - -impl ParallelProof -where - Factory: DatabaseProviderFactory + Clone + 'static, -{ /// Queues a storage proof task and returns a receiver for the result. fn queue_storage_proof( &self, @@ -251,9 +240,7 @@ mod tests { use rand::Rng; use reth_primitives_traits::{Account, StorageEntry}; use reth_provider::{ - providers::ConsistentDbView, - test_utils::{create_test_provider_factory, MockNodeTypesWithDB}, - HashingWriter, ProviderFactory, + providers::ConsistentDbView, test_utils::create_test_provider_factory, HashingWriter, }; use reth_trie::proof::Proof; use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; @@ -334,8 +321,7 @@ mod tests { // after we compute the state root let join_handle = rt.spawn_blocking(move || proof_task.run()); - type Factory = ProviderFactory; - let parallel_result = ParallelProof::::new( + let parallel_result = ParallelProof::new( Default::default(), Default::default(), Default::default(), From 18ff58d146fc1ad2b669c2a10155016352a85e1f Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 07:43:25 +0000 Subject: [PATCH 129/144] fix clippy --- crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs index 9d2088c11a4..4856983ccc0 100644 --- a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs +++ b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs @@ -604,9 +604,6 @@ where env.active_node_state_mut()?.latest_fork_choice_state.head_block_hash = rpc_latest_header.hash; - // update local copy for any further usage in this scope - latest_block.hash = rpc_latest_header.hash; - latest_block.number = rpc_latest_header.inner.number; } } From f692d75be03038642bad0c98ec865d2363c53eb8 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 08:40:20 +0000 Subject: [PATCH 130/144] fmt --- crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs index 4856983ccc0..74a5e2ba1d5 100644 --- a/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs +++ b/crates/e2e-test-utils/src/testsuite/actions/produce_blocks.rs @@ -510,7 +510,7 @@ where Box::pin(async move { let mut accepted_check: bool = false; - let mut latest_block = env + let latest_block = env .current_block_info() .ok_or_else(|| eyre::eyre!("No latest block information available"))?; @@ -603,7 +603,6 @@ where rpc_latest_header.inner.timestamp; env.active_node_state_mut()?.latest_fork_choice_state.head_block_hash = rpc_latest_header.hash; - } } From f9e167ec393e1edd1b83985587955cb1f6097fa8 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 09:19:28 +0000 Subject: [PATCH 131/144] refactor: replace spawn_proof_workers with ProofTaskManagerHandle - Updated the code to utilize ProofTaskManagerHandle for spawning proof workers instead of the deprecated spawn_proof_workers function. - This change enhances code clarity and maintainability by consolidating the worker management logic within the ProofTaskManagerHandle struct. --- .../tree/src/tree/payload_processor/mod.rs | 4 +- .../src/tree/payload_processor/multiproof.rs | 4 +- crates/trie/parallel/src/proof.rs | 5 +- crates/trie/parallel/src/proof_task.rs | 165 +++++++++--------- 4 files changed, 90 insertions(+), 88 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index 87ca695689d..e241740dbd5 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -32,7 +32,7 @@ use reth_provider::{ use reth_revm::{db::BundleState, state::EvmState}; use reth_trie::TrieInput; use reth_trie_parallel::{ - proof_task::{spawn_proof_workers, ProofTaskCtx}, + proof_task::{ProofTaskCtx, ProofTaskManagerHandle}, root::ParallelStateRootError, }; use reth_trie_sparse::{ @@ -203,7 +203,7 @@ where let storage_worker_count = config.storage_worker_count(); let account_worker_count = config.account_worker_count(); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - let proof_handle = match spawn_proof_workers( + let proof_handle = match ProofTaskManagerHandle::new( self.executor.handle().clone(), consistent_view, task_ctx, diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 3d2a7a87da7..6eba0a1bbf7 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -1212,7 +1212,7 @@ mod tests { DatabaseProviderFactory, }; use reth_trie::{MultiProof, TrieInput}; - use reth_trie_parallel::proof_task::{spawn_proof_workers, ProofTaskCtx}; + use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofTaskManagerHandle}; use revm_primitives::{B256, U256}; fn create_test_state_root_task(factory: F) -> MultiProofTask @@ -1228,7 +1228,7 @@ mod tests { ); let consistent_view = ConsistentDbView::new(factory, None); let proof_handle = - spawn_proof_workers(executor.handle().clone(), consistent_view, task_ctx, 1, 1) + ProofTaskManagerHandle::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) .expect("Failed to spawn proof workers"); let channel = channel(); diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 841779aa982..3b0e3aa0279 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -224,7 +224,7 @@ impl ParallelProof { #[cfg(test)] mod tests { use super::*; - use crate::proof_task::{spawn_proof_workers, ProofTaskCtx}; + use crate::proof_task::{ProofTaskCtx, ProofTaskManagerHandle}; use alloy_primitives::{ keccak256, map::{B256Set, DefaultHashBuilder, HashMap}, @@ -307,7 +307,8 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); let proof_task_handle = - spawn_proof_workers(rt.handle().clone(), consistent_view, task_ctx, 1, 1).unwrap(); + ProofTaskManagerHandle::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1) + .unwrap(); let parallel_result = ParallelProof::new( Default::default(), diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 8545b7d2f28..121aab594de 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -90,82 +90,6 @@ enum StorageWorkerJob { }, } -/// Spawns storage and account worker pools with dedicated database transactions. -/// -/// Returns a handle for submitting proof tasks to the worker pools. -/// Workers run until the last handle is dropped. -/// -/// # Parameters -/// - `executor`: Tokio runtime handle for spawning blocking tasks -/// - `view`: Consistent database view for creating transactions -/// - `task_ctx`: Shared context with trie updates and prefix sets -/// - `storage_worker_count`: Number of storage workers to spawn -/// - `account_worker_count`: Number of account workers to spawn -pub fn spawn_proof_workers( - executor: Handle, - view: ConsistentDbView, - task_ctx: ProofTaskCtx, - storage_worker_count: usize, - account_worker_count: usize, -) -> ProviderResult -where - Factory: DatabaseProviderFactory, -{ - let (storage_work_tx, storage_work_rx) = unbounded::(); - let (account_work_tx, account_work_rx) = unbounded::(); - - tracing::info!( - target: "trie::proof_task", - storage_worker_count, - account_worker_count, - "Spawning proof worker pools" - ); - - // Spawn storage workers - for worker_id in 0..storage_worker_count { - let provider_ro = view.provider_ro()?; - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = storage_work_rx.clone(); - - executor - .spawn_blocking(move || storage_worker_loop(proof_task_tx, work_rx_clone, worker_id)); - - tracing::debug!( - target: "trie::proof_task", - worker_id, - "Storage worker spawned successfully" - ); - } - - // Spawn account workers - for worker_id in 0..account_worker_count { - let provider_ro = view.provider_ro()?; - let tx = provider_ro.into_tx(); - let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); - let work_rx_clone = account_work_rx.clone(); - let storage_work_tx_clone = storage_work_tx.clone(); - - executor.spawn_blocking(move || { - account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) - }); - - tracing::debug!( - target: "trie::proof_task", - worker_id, - "Account worker spawned successfully" - ); - } - - Ok(ProofTaskManagerHandle::new( - storage_work_tx, - account_work_tx, - Arc::new(AtomicUsize::new(0)), - #[cfg(feature = "metrics")] - Arc::new(ProofTaskMetrics::default()), - )) -} - /// Worker loop for storage trie operations. /// /// # Lifecycle @@ -921,11 +845,87 @@ pub struct ProofTaskManagerHandle { } impl ProofTaskManagerHandle { + /// Spawns storage and account worker pools with dedicated database transactions. + /// + /// Returns a handle for submitting proof tasks to the worker pools. + /// Workers run until the last handle is dropped. + /// + /// # Parameters + /// - `executor`: Tokio runtime handle for spawning blocking tasks + /// - `view`: Consistent database view for creating transactions + /// - `task_ctx`: Shared context with trie updates and prefix sets + /// - `storage_worker_count`: Number of storage workers to spawn + /// - `account_worker_count`: Number of account workers to spawn + pub fn new( + executor: Handle, + view: ConsistentDbView, + task_ctx: ProofTaskCtx, + storage_worker_count: usize, + account_worker_count: usize, + ) -> ProviderResult + where + Factory: DatabaseProviderFactory, + { + let (storage_work_tx, storage_work_rx) = unbounded::(); + let (account_work_tx, account_work_rx) = unbounded::(); + + tracing::info!( + target: "trie::proof_task", + storage_worker_count, + account_worker_count, + "Spawning proof worker pools" + ); + + // Spawn storage workers + for worker_id in 0..storage_worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = storage_work_rx.clone(); + + executor.spawn_blocking(move || { + storage_worker_loop(proof_task_tx, work_rx_clone, worker_id) + }); + + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Storage worker spawned successfully" + ); + } + + // Spawn account workers + for worker_id in 0..account_worker_count { + let provider_ro = view.provider_ro()?; + let tx = provider_ro.into_tx(); + let proof_task_tx = ProofTaskTx::new(tx, task_ctx.clone(), worker_id); + let work_rx_clone = account_work_rx.clone(); + let storage_work_tx_clone = storage_work_tx.clone(); + + executor.spawn_blocking(move || { + account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) + }); + + tracing::debug!( + target: "trie::proof_task", + worker_id, + "Account worker spawned successfully" + ); + } + + Ok(Self::new_handle( + storage_work_tx, + account_work_tx, + Arc::new(AtomicUsize::new(0)), + #[cfg(feature = "metrics")] + Arc::new(ProofTaskMetrics::default()), + )) + } + /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. /// - /// This is an internal constructor used by `spawn_proof_workers`. - /// External users should call `spawn_proof_workers` to create handles. - fn new( + /// This is an internal constructor used for creating handles. + fn new_handle( storage_work_tx: CrossbeamSender, account_work_tx: CrossbeamSender, active_handles: Arc, @@ -1017,7 +1017,7 @@ impl ProofTaskManagerHandle { impl Clone for ProofTaskManagerHandle { fn clone(&self) -> Self { - Self::new( + Self::new_handle( self.storage_work_tx.clone(), self.account_work_tx.clone(), self.active_handles.clone(), @@ -1120,7 +1120,7 @@ mod tests { ) } - /// Ensures `spawn_proof_workers` spawns workers correctly. + /// Ensures `ProofTaskManagerHandle::new` spawns workers correctly. #[test] fn spawn_proof_workers_creates_handle() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -1130,7 +1130,8 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let proof_handle = spawn_proof_workers(handle.clone(), view, ctx, 5, 3).unwrap(); + let proof_handle = + ProofTaskManagerHandle::new(handle.clone(), view, ctx, 5, 3).unwrap(); // Verify handle can be cloned let _cloned_handle = proof_handle.clone(); From bc4ecf5077dc8ae43f2fd47a16271f582c28e30b Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 09:30:35 +0000 Subject: [PATCH 132/144] addressed sync overhead - Removed lock-free atomic counters from ProofTaskMetrics and replaced them with direct method calls for recording blinded node counts. - Updated storage and account worker loops to utilize the new metrics recording methods, enhancing clarity and maintainability. - Simplified the ProofTaskManagerHandle by removing unnecessary metrics fields, streamlining the overall structure. --- crates/trie/parallel/src/proof_task.rs | 71 ++++++++----------- .../trie/parallel/src/proof_task_metrics.rs | 24 +++---- 2 files changed, 39 insertions(+), 56 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 121aab594de..83fcd59a8c3 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -117,6 +117,7 @@ fn storage_worker_loop( proof_tx: ProofTaskTx, work_rx: CrossbeamReceiver, worker_id: usize, + #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, ) where Tx: DbTx, { @@ -230,6 +231,9 @@ fn storage_worker_loop( storage_nodes_processed, "Storage worker shutting down" ); + + #[cfg(feature = "metrics")] + metrics.record_storage_nodes(storage_nodes_processed as usize); } /// Worker loop for account trie operations. @@ -260,6 +264,7 @@ fn account_worker_loop( work_rx: CrossbeamReceiver, storage_work_tx: CrossbeamSender, worker_id: usize, + #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, ) where Tx: DbTx, { @@ -405,6 +410,9 @@ fn account_worker_loop( account_nodes_processed, "Account worker shutting down" ); + + #[cfg(feature = "metrics")] + metrics.record_account_nodes(account_nodes_processed as usize); } /// Builds an account multiproof by consuming storage proof receivers lazily during trie walk. @@ -839,9 +847,6 @@ pub struct ProofTaskManagerHandle { account_work_tx: CrossbeamSender, /// Active handle reference count for auto-termination active_handles: Arc, - /// Metrics tracking (lock-free) - #[cfg(feature = "metrics")] - metrics: Arc, } impl ProofTaskManagerHandle { @@ -884,7 +889,16 @@ impl ProofTaskManagerHandle { let work_rx_clone = storage_work_rx.clone(); executor.spawn_blocking(move || { - storage_worker_loop(proof_task_tx, work_rx_clone, worker_id) + #[cfg(feature = "metrics")] + let metrics = ProofTaskMetrics::default(); + + storage_worker_loop( + proof_task_tx, + work_rx_clone, + worker_id, + #[cfg(feature = "metrics")] + metrics, + ) }); tracing::debug!( @@ -903,7 +917,17 @@ impl ProofTaskManagerHandle { let storage_work_tx_clone = storage_work_tx.clone(); executor.spawn_blocking(move || { - account_worker_loop(proof_task_tx, work_rx_clone, storage_work_tx_clone, worker_id) + #[cfg(feature = "metrics")] + let metrics = ProofTaskMetrics::default(); + + account_worker_loop( + proof_task_tx, + work_rx_clone, + storage_work_tx_clone, + worker_id, + #[cfg(feature = "metrics")] + metrics, + ) }); tracing::debug!( @@ -913,13 +937,7 @@ impl ProofTaskManagerHandle { ); } - Ok(Self::new_handle( - storage_work_tx, - account_work_tx, - Arc::new(AtomicUsize::new(0)), - #[cfg(feature = "metrics")] - Arc::new(ProofTaskMetrics::default()), - )) + Ok(Self::new_handle(storage_work_tx, account_work_tx, Arc::new(AtomicUsize::new(0)))) } /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. @@ -929,16 +947,9 @@ impl ProofTaskManagerHandle { storage_work_tx: CrossbeamSender, account_work_tx: CrossbeamSender, active_handles: Arc, - #[cfg(feature = "metrics")] metrics: Arc, ) -> Self { active_handles.fetch_add(1, Ordering::SeqCst); - Self { - storage_work_tx, - account_work_tx, - active_handles, - #[cfg(feature = "metrics")] - metrics, - } + Self { storage_work_tx, account_work_tx, active_handles } } /// Queue a storage proof computation @@ -953,9 +964,6 @@ impl ProofTaskManagerHandle { ProviderError::other(std::io::Error::other("storage workers unavailable")) })?; - #[cfg(feature = "metrics")] - self.metrics.storage_proofs.fetch_add(1, Ordering::Relaxed); - Ok(rx) } @@ -971,9 +979,6 @@ impl ProofTaskManagerHandle { ProviderError::other(std::io::Error::other("account workers unavailable")) })?; - #[cfg(feature = "metrics")] - self.metrics.account_proofs.fetch_add(1, Ordering::Relaxed); - Ok(rx) } @@ -990,9 +995,6 @@ impl ProofTaskManagerHandle { ProviderError::other(std::io::Error::other("storage workers unavailable")) })?; - #[cfg(feature = "metrics")] - self.metrics.storage_nodes.fetch_add(1, Ordering::Relaxed); - Ok(rx) } @@ -1008,9 +1010,6 @@ impl ProofTaskManagerHandle { ProviderError::other(std::io::Error::other("account workers unavailable")) })?; - #[cfg(feature = "metrics")] - self.metrics.account_nodes.fetch_add(1, Ordering::Relaxed); - Ok(rx) } } @@ -1021,8 +1020,6 @@ impl Clone for ProofTaskManagerHandle { self.storage_work_tx.clone(), self.account_work_tx.clone(), self.active_handles.clone(), - #[cfg(feature = "metrics")] - self.metrics.clone(), ) } } @@ -1039,12 +1036,6 @@ impl Drop for ProofTaskManagerHandle { "active_handles underflow in ProofTaskManagerHandle::drop (previous={})", previous_handles ); - - #[cfg(feature = "metrics")] - if previous_handles == 1 { - // Flush metrics before exit. - self.metrics.record(); - } } } diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index 5f8959cea9d..89f2fd6c077 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -1,29 +1,21 @@ use reth_metrics::{metrics::Histogram, Metrics}; -use std::sync::{ - atomic::{AtomicU64, Ordering}, - Arc, -}; /// Metrics for blinded node fetching by proof workers. #[derive(Clone, Debug, Default)] pub struct ProofTaskMetrics { /// The actual metrics for blinded nodes. pub task_metrics: ProofTaskTrieMetrics, - /// Count of storage proof requests (lock-free). - pub storage_proofs: Arc, - /// Count of account proof requests (lock-free). - pub account_proofs: Arc, - /// Count of blinded account node requests (lock-free). - pub account_nodes: Arc, - /// Count of blinded storage node requests (lock-free). - pub storage_nodes: Arc, } impl ProofTaskMetrics { - /// Record the blinded node counts into the histograms. - pub fn record(&self) { - self.task_metrics.record_account_nodes(self.account_nodes.load(Ordering::Relaxed) as usize); - self.task_metrics.record_storage_nodes(self.storage_nodes.load(Ordering::Relaxed) as usize); + /// Record the blinded account node count into the histogram. + pub fn record_account_nodes(&self, count: usize) { + self.task_metrics.record_account_nodes(count); + } + + /// Record the blinded storage node count into the histogram. + pub fn record_storage_nodes(&self, count: usize) { + self.task_metrics.record_storage_nodes(count); } } From d8c855932ab74418c2c5dc6cdc014de3222c777e Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 09:57:46 +0000 Subject: [PATCH 133/144] remove wrapper --- crates/trie/parallel/src/proof_task.rs | 10 +++++----- .../trie/parallel/src/proof_task_metrics.rs | 19 ------------------- 2 files changed, 5 insertions(+), 24 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 83fcd59a8c3..f13f9ee5f5a 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -62,7 +62,7 @@ use tokio::runtime::Handle; use tracing::trace; #[cfg(feature = "metrics")] -use crate::proof_task_metrics::ProofTaskMetrics; +use crate::proof_task_metrics::ProofTaskTrieMetrics; type StorageProofResult = Result; type TrieNodeProviderResult = Result, SparseTrieError>; @@ -117,7 +117,7 @@ fn storage_worker_loop( proof_tx: ProofTaskTx, work_rx: CrossbeamReceiver, worker_id: usize, - #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, + #[cfg(feature = "metrics")] metrics: ProofTaskTrieMetrics, ) where Tx: DbTx, { @@ -264,7 +264,7 @@ fn account_worker_loop( work_rx: CrossbeamReceiver, storage_work_tx: CrossbeamSender, worker_id: usize, - #[cfg(feature = "metrics")] metrics: ProofTaskMetrics, + #[cfg(feature = "metrics")] metrics: ProofTaskTrieMetrics, ) where Tx: DbTx, { @@ -890,7 +890,7 @@ impl ProofTaskManagerHandle { executor.spawn_blocking(move || { #[cfg(feature = "metrics")] - let metrics = ProofTaskMetrics::default(); + let metrics = ProofTaskTrieMetrics::default(); storage_worker_loop( proof_task_tx, @@ -918,7 +918,7 @@ impl ProofTaskManagerHandle { executor.spawn_blocking(move || { #[cfg(feature = "metrics")] - let metrics = ProofTaskMetrics::default(); + let metrics = ProofTaskTrieMetrics::default(); account_worker_loop( proof_task_tx, diff --git a/crates/trie/parallel/src/proof_task_metrics.rs b/crates/trie/parallel/src/proof_task_metrics.rs index 89f2fd6c077..6492e28d12d 100644 --- a/crates/trie/parallel/src/proof_task_metrics.rs +++ b/crates/trie/parallel/src/proof_task_metrics.rs @@ -1,24 +1,5 @@ use reth_metrics::{metrics::Histogram, Metrics}; -/// Metrics for blinded node fetching by proof workers. -#[derive(Clone, Debug, Default)] -pub struct ProofTaskMetrics { - /// The actual metrics for blinded nodes. - pub task_metrics: ProofTaskTrieMetrics, -} - -impl ProofTaskMetrics { - /// Record the blinded account node count into the histogram. - pub fn record_account_nodes(&self, count: usize) { - self.task_metrics.record_account_nodes(count); - } - - /// Record the blinded storage node count into the histogram. - pub fn record_storage_nodes(&self, count: usize) { - self.task_metrics.record_storage_nodes(count); - } -} - /// Metrics for the proof task. #[derive(Clone, Metrics)] #[metrics(scope = "trie.proof_task")] From 1e9874b0ad9c96aa4edd7f3115963b14623e02e1 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 10:01:47 +0000 Subject: [PATCH 134/144] remove active handle --- crates/trie/parallel/src/proof_task.rs | 36 +++----------------------- 1 file changed, 3 insertions(+), 33 deletions(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index f13f9ee5f5a..f160d6368a3 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -52,7 +52,6 @@ use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; use reth_trie_sparse::provider::{RevealedNode, TrieNodeProvider, TrieNodeProviderFactory}; use std::{ sync::{ - atomic::{AtomicUsize, Ordering}, mpsc::{channel, Receiver, Sender}, Arc, }, @@ -839,14 +838,12 @@ impl ProofTaskCtx { /// The handle stores direct senders to both storage and account worker pools, /// eliminating the need for a routing thread. All handles share reference-counted /// channels, and workers shut down gracefully when all handles are dropped. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ProofTaskManagerHandle { /// Direct sender to storage worker pool storage_work_tx: CrossbeamSender, /// Direct sender to account worker pool account_work_tx: CrossbeamSender, - /// Active handle reference count for auto-termination - active_handles: Arc, } impl ProofTaskManagerHandle { @@ -937,7 +934,7 @@ impl ProofTaskManagerHandle { ); } - Ok(Self::new_handle(storage_work_tx, account_work_tx, Arc::new(AtomicUsize::new(0)))) + Ok(Self::new_handle(storage_work_tx, account_work_tx)) } /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. @@ -946,10 +943,8 @@ impl ProofTaskManagerHandle { fn new_handle( storage_work_tx: CrossbeamSender, account_work_tx: CrossbeamSender, - active_handles: Arc, ) -> Self { - active_handles.fetch_add(1, Ordering::SeqCst); - Self { storage_work_tx, account_work_tx, active_handles } + Self { storage_work_tx, account_work_tx } } /// Queue a storage proof computation @@ -1014,31 +1009,6 @@ impl ProofTaskManagerHandle { } } -impl Clone for ProofTaskManagerHandle { - fn clone(&self) -> Self { - Self::new_handle( - self.storage_work_tx.clone(), - self.account_work_tx.clone(), - self.active_handles.clone(), - ) - } -} - -impl Drop for ProofTaskManagerHandle { - fn drop(&mut self) { - // Decrement the number of active handles. - // When the last handle is dropped, the channels are dropped and workers shut down. - // atomically grab the current handle count and decrement it for Drop. - let previous_handles = self.active_handles.fetch_sub(1, Ordering::SeqCst); - - debug_assert_ne!( - previous_handles, 0, - "active_handles underflow in ProofTaskManagerHandle::drop (previous={})", - previous_handles - ); - } -} - impl TrieNodeProviderFactory for ProofTaskManagerHandle { type AccountNodeProvider = ProofTaskTrieNodeProvider; type StorageNodeProvider = ProofTaskTrieNodeProvider; From 1619408dfccb8ae9b2b0dd7f5eea666bfd2e242d Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Mon, 13 Oct 2025 10:05:19 +0000 Subject: [PATCH 135/144] fmt --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index f160d6368a3..d3c05bda2a5 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -940,7 +940,7 @@ impl ProofTaskManagerHandle { /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. /// /// This is an internal constructor used for creating handles. - fn new_handle( + const fn new_handle( storage_work_tx: CrossbeamSender, account_work_tx: CrossbeamSender, ) -> Self { From 7041e88c4c54eaaf48229ca70206d1ccfdd10ed6 Mon Sep 17 00:00:00 2001 From: YK Date: Tue, 14 Oct 2025 18:15:14 +0800 Subject: [PATCH 136/144] Apply suggestion from @yongkangc --- crates/engine/primitives/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index c0af60e2f0f..16aa648296b 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -17,7 +17,7 @@ fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] { std::thread::available_parallelism() - .map(|n| (n.get() * 2).clamp(MIN_WORKER_COUNT, 64)) + .map(|n| (n.get() * 2).clamp(MIN_WORKER_COUNT, 128)) .unwrap_or(8) } #[cfg(not(feature = "std"))] From 8f0aa6406fa27c29186bd28505fee7c18fef9c81 Mon Sep 17 00:00:00 2001 From: YK Date: Tue, 14 Oct 2025 18:15:21 +0800 Subject: [PATCH 137/144] Apply suggestion from @yongkangc --- crates/engine/primitives/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 16aa648296b..24f65de8628 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -10,7 +10,7 @@ pub const DEFAULT_MEMORY_BLOCK_BUFFER_TARGET: u64 = 0; pub const DEFAULT_MAX_PROOF_TASK_CONCURRENCY: u64 = 256; /// Minimum number of workers we allow configuring explicitly. -pub const MIN_WORKER_COUNT: usize = 2; +pub const MIN_WORKER_COUNT: usize = 32; /// Returns the default number of storage worker threads based on available parallelism. fn default_storage_worker_count() -> usize { From 0ba03c7adaf7ef5d0014ec4b32ec05616fe4eb7b Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 15 Oct 2025 01:03:21 +0000 Subject: [PATCH 138/144] fix merge conflicts --- crates/engine/primitives/src/config.rs | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 0ec8a2e2be1..24f65de8628 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -511,15 +511,4 @@ impl TreeConfig { self.account_worker_count = account_worker_count.max(MIN_WORKER_COUNT); self } - - /// Return the number of account proof worker threads. - pub const fn account_worker_count(&self) -> usize { - self.account_worker_count - } - - /// Setter for the number of account proof worker threads. - pub const fn with_account_worker_count(mut self, account_worker_count: usize) -> Self { - self.account_worker_count = account_worker_count; - self - } } From 89710947755134d1e4a9eacd8057f6e9af594b09 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 15 Oct 2025 01:05:55 +0000 Subject: [PATCH 139/144] rm --- crates/engine/tree/src/tree/payload_processor/multiproof.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 8621ba1cc60..6eba0a1bbf7 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -703,8 +703,7 @@ impl MultiProofTask { multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle.clone(), // handle for storage proof workers - proof_task_handle, // handle for account proof workers + proof_task_handle, max_concurrency, ), metrics, From f4eda798d4cd6847f1d776a8d4a29aaddbbccdda Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 15 Oct 2025 01:06:04 +0000 Subject: [PATCH 140/144] import --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index 3263c3c6bea..d3c05bda2a5 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -26,7 +26,7 @@ use alloy_rlp::{BufMut, Encodable}; use crossbeam_channel::{unbounded, Receiver as CrossbeamReceiver, Sender as CrossbeamSender}; use dashmap::DashMap; use reth_db_api::transaction::DbTx; -use reth_execution_errors::SparseTrieError; +use reth_execution_errors::{SparseTrieError, SparseTrieErrorKind}; use reth_provider::{ providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, ProviderError, ProviderResult, From d6534d9dd32f8ee383d8c95b023bc16daeabe517 Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 15 Oct 2025 09:06:27 +0800 Subject: [PATCH 141/144] Apply suggestion from @shekhirin Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> --- crates/engine/primitives/src/config.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 24f65de8628..8ecc78ccacc 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -17,8 +17,7 @@ fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] { std::thread::available_parallelism() - .map(|n| (n.get() * 2).clamp(MIN_WORKER_COUNT, 128)) - .unwrap_or(8) + .map_or(8, |n| n.get() * 2).min(MIN_WORKER_COUNT) } #[cfg(not(feature = "std"))] { From 7117729e7a42a682bf4c31a40871227379c573ae Mon Sep 17 00:00:00 2001 From: YK Date: Wed, 15 Oct 2025 09:06:44 +0800 Subject: [PATCH 142/144] Apply suggestion from @yongkangc --- crates/trie/parallel/src/proof_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index d3c05bda2a5..e31326c1101 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -871,7 +871,7 @@ impl ProofTaskManagerHandle { let (storage_work_tx, storage_work_rx) = unbounded::(); let (account_work_tx, account_work_rx) = unbounded::(); - tracing::info!( + tracing::debug!( target: "trie::proof_task", storage_worker_count, account_worker_count, From ea93b9688ddb3d24fb8a30bae8d534defac17153 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 15 Oct 2025 01:17:52 +0000 Subject: [PATCH 143/144] fmt --- crates/engine/primitives/src/config.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/engine/primitives/src/config.rs b/crates/engine/primitives/src/config.rs index 8ecc78ccacc..9e2c8210f08 100644 --- a/crates/engine/primitives/src/config.rs +++ b/crates/engine/primitives/src/config.rs @@ -16,8 +16,7 @@ pub const MIN_WORKER_COUNT: usize = 32; fn default_storage_worker_count() -> usize { #[cfg(feature = "std")] { - std::thread::available_parallelism() - .map_or(8, |n| n.get() * 2).min(MIN_WORKER_COUNT) + std::thread::available_parallelism().map_or(8, |n| n.get() * 2).min(MIN_WORKER_COUNT) } #[cfg(not(feature = "std"))] { From 7d17d8867e77abc1afab42cf736632f80d6002b2 Mon Sep 17 00:00:00 2001 From: Yong Kang Date: Wed, 15 Oct 2025 01:32:56 +0000 Subject: [PATCH 144/144] refactor: rename ProofTaskManagerHandle to ProofWorkerHandle for clarity --- .../tree/src/tree/payload_processor/mod.rs | 8 +++---- .../src/tree/payload_processor/multiproof.rs | 24 +++++++++---------- crates/trie/parallel/src/proof.rs | 23 +++++++++--------- crates/trie/parallel/src/proof_task.rs | 21 ++++++++-------- 4 files changed, 37 insertions(+), 39 deletions(-) diff --git a/crates/engine/tree/src/tree/payload_processor/mod.rs b/crates/engine/tree/src/tree/payload_processor/mod.rs index e241740dbd5..f3ecdfa86d5 100644 --- a/crates/engine/tree/src/tree/payload_processor/mod.rs +++ b/crates/engine/tree/src/tree/payload_processor/mod.rs @@ -32,7 +32,7 @@ use reth_provider::{ use reth_revm::{db::BundleState, state::EvmState}; use reth_trie::TrieInput; use reth_trie_parallel::{ - proof_task::{ProofTaskCtx, ProofTaskManagerHandle}, + proof_task::{ProofTaskCtx, ProofWorkerHandle}, root::ParallelStateRootError, }; use reth_trie_sparse::{ @@ -203,7 +203,7 @@ where let storage_worker_count = config.storage_worker_count(); let account_worker_count = config.account_worker_count(); let max_proof_task_concurrency = config.max_proof_task_concurrency() as usize; - let proof_handle = match ProofTaskManagerHandle::new( + let proof_handle = match ProofWorkerHandle::new( self.executor.handle().clone(), consistent_view, task_ctx, @@ -393,7 +393,7 @@ where fn spawn_sparse_trie_task( &self, sparse_trie_rx: mpsc::Receiver, - proof_task_handle: BPF, + proof_worker_handle: BPF, state_root_tx: mpsc::Sender>, ) where BPF: TrieNodeProviderFactory + Clone + Send + Sync + 'static, @@ -423,7 +423,7 @@ where let task = SparseTrieTask::<_, ConfiguredSparseTrie, ConfiguredSparseTrie>::new_with_cleared_trie( sparse_trie_rx, - proof_task_handle, + proof_worker_handle, self.trie_metrics.clone(), sparse_state_trie, ); diff --git a/crates/engine/tree/src/tree/payload_processor/multiproof.rs b/crates/engine/tree/src/tree/payload_processor/multiproof.rs index 6eba0a1bbf7..4a71bf620f7 100644 --- a/crates/engine/tree/src/tree/payload_processor/multiproof.rs +++ b/crates/engine/tree/src/tree/payload_processor/multiproof.rs @@ -20,7 +20,7 @@ use reth_trie::{ }; use reth_trie_parallel::{ proof::ParallelProof, - proof_task::{AccountMultiproofInput, ProofTaskManagerHandle}, + proof_task::{AccountMultiproofInput, ProofWorkerHandle}, root::ParallelStateRootError, }; use std::{ @@ -347,7 +347,7 @@ pub struct MultiproofManager { /// Executor for tasks executor: WorkloadExecutor, /// Handle to the proof worker pools (storage and account). - proof_task_handle: ProofTaskManagerHandle, + proof_worker_handle: ProofWorkerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. /// @@ -369,7 +369,7 @@ impl MultiproofManager { fn new( executor: WorkloadExecutor, metrics: MultiProofTaskMetrics, - proof_task_handle: ProofTaskManagerHandle, + proof_worker_handle: ProofWorkerHandle, max_concurrent: usize, ) -> Self { Self { @@ -378,7 +378,7 @@ impl MultiproofManager { executor, inflight: 0, metrics, - proof_task_handle, + proof_worker_handle, missed_leaves_storage_roots: Default::default(), } } @@ -447,7 +447,7 @@ impl MultiproofManager { multi_added_removed_keys, } = storage_multiproof_input; - let storage_proof_task_handle = self.proof_task_handle.clone(); + let storage_proof_worker_handle = self.proof_worker_handle.clone(); let missed_leaves_storage_roots = self.missed_leaves_storage_roots.clone(); self.executor.spawn_blocking(move || { @@ -466,7 +466,7 @@ impl MultiproofManager { config.state_sorted, config.prefix_sets, missed_leaves_storage_roots, - storage_proof_task_handle, + storage_proof_worker_handle, ) .with_branch_node_masks(true) .with_multi_added_removed_keys(Some(multi_added_removed_keys)) @@ -519,7 +519,7 @@ impl MultiproofManager { state_root_message_sender, multi_added_removed_keys, } = multiproof_input; - let account_proof_task_handle = self.proof_task_handle.clone(); + let account_proof_worker_handle = self.proof_worker_handle.clone(); let missed_leaves_storage_roots = self.missed_leaves_storage_roots.clone(); self.executor.spawn_blocking(move || { @@ -552,7 +552,7 @@ impl MultiproofManager { }; let proof_result: Result = (|| { - let receiver = account_proof_task_handle + let receiver = account_proof_worker_handle .queue_account_multiproof(input) .map_err(|e| ParallelStateRootError::Other(e.to_string()))?; @@ -683,7 +683,7 @@ impl MultiProofTask { pub(super) fn new( config: MultiProofConfig, executor: WorkloadExecutor, - proof_task_handle: ProofTaskManagerHandle, + proof_worker_handle: ProofWorkerHandle, to_sparse_trie: Sender, max_concurrency: usize, chunk_size: Option, @@ -703,7 +703,7 @@ impl MultiProofTask { multiproof_manager: MultiproofManager::new( executor, metrics.clone(), - proof_task_handle, + proof_worker_handle, max_concurrency, ), metrics, @@ -1212,7 +1212,7 @@ mod tests { DatabaseProviderFactory, }; use reth_trie::{MultiProof, TrieInput}; - use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofTaskManagerHandle}; + use reth_trie_parallel::proof_task::{ProofTaskCtx, ProofWorkerHandle}; use revm_primitives::{B256, U256}; fn create_test_state_root_task(factory: F) -> MultiProofTask @@ -1228,7 +1228,7 @@ mod tests { ); let consistent_view = ConsistentDbView::new(factory, None); let proof_handle = - ProofTaskManagerHandle::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) + ProofWorkerHandle::new(executor.handle().clone(), consistent_view, task_ctx, 1, 1) .expect("Failed to spawn proof workers"); let channel = channel(); diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 3b0e3aa0279..0f29502f8c7 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -1,6 +1,6 @@ use crate::{ metrics::ParallelTrieMetrics, - proof_task::{AccountMultiproofInput, ProofTaskManagerHandle, StorageProofInput}, + proof_task::{AccountMultiproofInput, ProofWorkerHandle, StorageProofInput}, root::ParallelStateRootError, StorageRootTargets, }; @@ -37,7 +37,7 @@ pub struct ParallelProof { /// Provided by the user to give the necessary context to retain extra proofs. multi_added_removed_keys: Option>, /// Handle to the proof worker pools. - proof_task_handle: ProofTaskManagerHandle, + proof_worker_handle: ProofWorkerHandle, /// Cached storage proof roots for missed leaves; this maps /// hashed (missed) addresses to their storage proof roots. missed_leaves_storage_roots: Arc>, @@ -52,7 +52,7 @@ impl ParallelProof { state_sorted: Arc, prefix_sets: Arc, missed_leaves_storage_roots: Arc>, - proof_task_handle: ProofTaskManagerHandle, + proof_worker_handle: ProofWorkerHandle, ) -> Self { Self { nodes_sorted, @@ -61,7 +61,7 @@ impl ParallelProof { missed_leaves_storage_roots, collect_branch_node_masks: false, multi_added_removed_keys: None, - proof_task_handle, + proof_worker_handle, #[cfg(feature = "metrics")] metrics: ParallelTrieMetrics::new_with_labels(&[("type", "proof")]), } @@ -100,7 +100,7 @@ impl ParallelProof { self.multi_added_removed_keys.clone(), ); - self.proof_task_handle + self.proof_worker_handle .queue_storage_proof(input) .map_err(|e| ParallelStateRootError::Other(e.to_string())) } @@ -192,7 +192,7 @@ impl ParallelProof { }; let receiver = self - .proof_task_handle + .proof_worker_handle .queue_account_multiproof(input) .map_err(|e| ParallelStateRootError::Other(e.to_string()))?; @@ -224,7 +224,7 @@ impl ParallelProof { #[cfg(test)] mod tests { use super::*; - use crate::proof_task::{ProofTaskCtx, ProofTaskManagerHandle}; + use crate::proof_task::{ProofTaskCtx, ProofWorkerHandle}; use alloy_primitives::{ keccak256, map::{B256Set, DefaultHashBuilder, HashMap}, @@ -306,16 +306,15 @@ mod tests { let task_ctx = ProofTaskCtx::new(Default::default(), Default::default(), Default::default()); - let proof_task_handle = - ProofTaskManagerHandle::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1) - .unwrap(); + let proof_worker_handle = + ProofWorkerHandle::new(rt.handle().clone(), consistent_view, task_ctx, 1, 1).unwrap(); let parallel_result = ParallelProof::new( Default::default(), Default::default(), Default::default(), Default::default(), - proof_task_handle.clone(), + proof_worker_handle.clone(), ) .decoded_multiproof(targets.clone()) .unwrap(); @@ -344,6 +343,6 @@ mod tests { assert_eq!(parallel_result, sequential_result_decoded); // Workers shut down automatically when handle is dropped - drop(proof_task_handle); + drop(proof_worker_handle); } } diff --git a/crates/trie/parallel/src/proof_task.rs b/crates/trie/parallel/src/proof_task.rs index e31326c1101..2d0f7e933c8 100644 --- a/crates/trie/parallel/src/proof_task.rs +++ b/crates/trie/parallel/src/proof_task.rs @@ -6,8 +6,8 @@ //! - **Worker Pools**: Pre-spawned workers with dedicated database transactions //! - Storage pool: Handles storage proofs and blinded storage node requests //! - Account pool: Handles account multiproofs and blinded account node requests -//! - **Direct Channel Access**: [`ProofTaskManagerHandle`] provides type-safe queue methods with -//! direct access to worker channels, eliminating routing overhead +//! - **Direct Channel Access**: [`ProofWorkerHandle`] provides type-safe queue methods with direct +//! access to worker channels, eliminating routing overhead //! - **Automatic Shutdown**: Workers terminate gracefully when all handles are dropped //! //! Individual [`ProofTaskTx`] instances manage a dedicated [`InMemoryTrieCursorFactory`] and @@ -839,14 +839,14 @@ impl ProofTaskCtx { /// eliminating the need for a routing thread. All handles share reference-counted /// channels, and workers shut down gracefully when all handles are dropped. #[derive(Debug, Clone)] -pub struct ProofTaskManagerHandle { +pub struct ProofWorkerHandle { /// Direct sender to storage worker pool storage_work_tx: CrossbeamSender, /// Direct sender to account worker pool account_work_tx: CrossbeamSender, } -impl ProofTaskManagerHandle { +impl ProofWorkerHandle { /// Spawns storage and account worker pools with dedicated database transactions. /// /// Returns a handle for submitting proof tasks to the worker pools. @@ -937,7 +937,7 @@ impl ProofTaskManagerHandle { Ok(Self::new_handle(storage_work_tx, account_work_tx)) } - /// Creates a new [`ProofTaskManagerHandle`] with direct access to worker pools. + /// Creates a new [`ProofWorkerHandle`] with direct access to worker pools. /// /// This is an internal constructor used for creating handles. const fn new_handle( @@ -1009,7 +1009,7 @@ impl ProofTaskManagerHandle { } } -impl TrieNodeProviderFactory for ProofTaskManagerHandle { +impl TrieNodeProviderFactory for ProofWorkerHandle { type AccountNodeProvider = ProofTaskTrieNodeProvider; type StorageNodeProvider = ProofTaskTrieNodeProvider; @@ -1028,14 +1028,14 @@ pub enum ProofTaskTrieNodeProvider { /// Blinded account trie node provider. AccountNode { /// Handle to the proof worker pools. - handle: ProofTaskManagerHandle, + handle: ProofWorkerHandle, }, /// Blinded storage trie node provider. StorageNode { /// Target account. account: B256, /// Handle to the proof worker pools. - handle: ProofTaskManagerHandle, + handle: ProofWorkerHandle, }, } @@ -1081,7 +1081,7 @@ mod tests { ) } - /// Ensures `ProofTaskManagerHandle::new` spawns workers correctly. + /// Ensures `ProofWorkerHandle::new` spawns workers correctly. #[test] fn spawn_proof_workers_creates_handle() { let runtime = Builder::new_multi_thread().worker_threads(1).enable_all().build().unwrap(); @@ -1091,8 +1091,7 @@ mod tests { let view = ConsistentDbView::new(factory, None); let ctx = test_ctx(); - let proof_handle = - ProofTaskManagerHandle::new(handle.clone(), view, ctx, 5, 3).unwrap(); + let proof_handle = ProofWorkerHandle::new(handle.clone(), view, ctx, 5, 3).unwrap(); // Verify handle can be cloned let _cloned_handle = proof_handle.clone();