From fdfcc95a0c841af047cff877ea1a125383f8f06c Mon Sep 17 00:00:00 2001 From: elizabeth Date: Sun, 5 Oct 2025 19:18:32 -0400 Subject: [PATCH 01/13] basic sync and execution pipeline implementation --- crates/op-rbuilder/src/builders/context.rs | 8 + .../src/builders/flashblocks/ctx.rs | 75 ++++ .../src/builders/flashblocks/mod.rs | 1 + .../src/builders/flashblocks/payload.rs | 222 +++++----- .../builders/flashblocks/payload_handler.rs | 386 +++++++++++++++++- .../src/builders/flashblocks/service.rs | 22 +- crates/p2p/src/lib.rs | 8 + 7 files changed, 619 insertions(+), 103 deletions(-) create mode 100644 crates/op-rbuilder/src/builders/flashblocks/ctx.rs diff --git a/crates/op-rbuilder/src/builders/context.rs b/crates/op-rbuilder/src/builders/context.rs index 9604ced71..dfa692b25 100644 --- a/crates/op-rbuilder/src/builders/context.rs +++ b/crates/op-rbuilder/src/builders/context.rs @@ -74,6 +74,14 @@ pub struct OpPayloadBuilderCtx { } impl OpPayloadBuilderCtx { + pub(super) fn with_cancel(self, cancel: CancellationToken) -> Self { + Self { cancel, ..self } + } + + pub(super) fn with_extra_ctx(self, extra_ctx: ExtraCtx) -> Self { + Self { extra_ctx, ..self } + } + /// Returns the parent block the payload will be build on. pub(super) fn parent(&self) -> &SealedHeader { &self.config.parent_header diff --git a/crates/op-rbuilder/src/builders/flashblocks/ctx.rs b/crates/op-rbuilder/src/builders/flashblocks/ctx.rs new file mode 100644 index 000000000..21aa4e1a1 --- /dev/null +++ b/crates/op-rbuilder/src/builders/flashblocks/ctx.rs @@ -0,0 +1,75 @@ +use crate::{ + builders::{BuilderConfig, OpPayloadBuilderCtx, flashblocks::FlashblocksConfig}, + gas_limiter::AddressGasLimiter, + metrics::OpRBuilderMetrics, + traits::ClientBounds, +}; +use op_revm::OpSpecId; +use reth_basic_payload_builder::PayloadConfig; +use reth_evm::EvmEnv; +use reth_optimism_chainspec::OpChainSpec; +use reth_optimism_evm::{OpEvmConfig, OpNextBlockEnvAttributes}; +use reth_optimism_payload_builder::{OpPayloadBuilderAttributes, config::OpDAConfig}; +use reth_optimism_primitives::OpTransactionSigned; +use std::sync::Arc; +use tokio_util::sync::CancellationToken; + +#[derive(Debug, Clone)] +pub(super) struct OpPayloadSyncerCtx { + /// The type that knows how to perform system calls and configure the evm. + evm_config: OpEvmConfig, + /// The DA config for the payload builder + da_config: OpDAConfig, + /// The chainspec + chain_spec: Arc, + /// Max gas that can be used by a transaction. + max_gas_per_txn: Option, +} + +impl OpPayloadSyncerCtx { + pub(super) fn new( + client: &Client, + builder_config: BuilderConfig, + evm_config: OpEvmConfig, + ) -> eyre::Result + where + Client: ClientBounds, + { + let chain_spec = client.chain_spec(); + Ok(Self { + evm_config, + da_config: builder_config.da_config.clone(), + chain_spec, + max_gas_per_txn: builder_config.max_gas_per_txn, + }) + } + + pub(super) fn evm_config(&self) -> &OpEvmConfig { + &self.evm_config + } + + pub(super) fn into_op_payload_builder_ctx( + self, + payload_config: PayloadConfig>, + evm_env: EvmEnv, + block_env_attributes: OpNextBlockEnvAttributes, + cancel: CancellationToken, + metrics: Arc, + address_gas_limiter: AddressGasLimiter, + ) -> OpPayloadBuilderCtx { + OpPayloadBuilderCtx { + evm_config: self.evm_config, + da_config: self.da_config, + chain_spec: self.chain_spec, + config: payload_config, + evm_env, + block_env_attributes, + cancel, + builder_signer: None, + metrics, + extra_ctx: (), + max_gas_per_txn: self.max_gas_per_txn, + address_gas_limiter, + } + } +} diff --git a/crates/op-rbuilder/src/builders/flashblocks/mod.rs b/crates/op-rbuilder/src/builders/flashblocks/mod.rs index 643bf39a3..87d4494cb 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/mod.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/mod.rs @@ -6,6 +6,7 @@ use service::FlashblocksServiceBuilder; mod best_txs; mod builder_tx; mod config; +mod ctx; mod p2p; mod payload; mod payload_handler; diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index c05a141d3..532dc18d7 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -68,12 +68,12 @@ type NextBestFlashblocksTxs = BestFlashblocksTxs< >; #[derive(Debug, Default)] -struct ExtraExecutionInfo { +pub(super) struct ExtraExecutionInfo { /// Index of the last consumed flashblock - pub last_flashblock_index: usize, + last_flashblock_index: usize, } -#[derive(Debug, Default)] +#[derive(Debug, Default, Clone)] pub struct FlashblocksExtraCtx { /// Current flashblock index flashblock_index: u64, @@ -102,16 +102,9 @@ impl OpPayloadBuilderCtx { self.extra_ctx.target_flashblock_count } - /// Increments the flashblock index - pub(crate) fn increment_flashblock_index(&mut self) -> u64 { - self.extra_ctx.flashblock_index += 1; - self.extra_ctx.flashblock_index - } - - /// Sets the target flashblock count - pub(crate) fn set_target_flashblock_count(&mut self, target_flashblock_count: u64) -> u64 { - self.extra_ctx.target_flashblock_count = target_flashblock_count; - self.extra_ctx.target_flashblock_count + /// Returns the next flashblock index + pub(crate) fn next_flashblock_index(&self) -> u64 { + self.extra_ctx.flashblock_index + 1 } /// Returns if the flashblock is the first fallback block @@ -159,8 +152,8 @@ impl OpPayloadBuilder { config: BuilderConfig, builder_tx: BuilderTx, payload_tx: mpsc::Sender, + metrics: Arc, ) -> eyre::Result { - let metrics = Arc::new(OpRBuilderMetrics::default()); let ws_pub = WebSocketPublisher::new(config.specific.ws_addr, Arc::clone(&metrics))?.into(); let address_gas_limiter = AddressGasLimiter::new(config.gas_limiter_config.clone()); Ok(Self { @@ -211,43 +204,16 @@ where Client: ClientBounds, BuilderTx: BuilderTransactions + Send + Sync, { - /// Constructs an Optimism payload from the transactions sent via the - /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in - /// the payload attributes, the transaction pool will be ignored and the only transactions - /// included in the payload will be those sent through the attributes. - /// - /// Given build arguments including an Optimism client, transaction pool, - /// and configuration, this function creates a transaction payload. Returns - /// a result indicating success with the payload or an error in case of failure. - async fn build_payload( + fn get_op_payload_builder_ctx( &self, - args: BuildArguments, OpBuiltPayload>, - best_payload: BlockCell, - ) -> Result<(), PayloadBuilderError> { - let block_build_start_time = Instant::now(); - let BuildArguments { - mut cached_reads, - config, - cancel: block_cancel, - } = args; - - // We log only every 100th block to reduce usage - let span = if cfg!(feature = "telemetry") - && config.parent_header.number % self.config.sampling_ratio == 0 - { - span!(Level::INFO, "build_payload") - } else { - tracing::Span::none() - }; - let _entered = span.enter(); - span.record( - "payload_id", - config.attributes.payload_attributes.id.to_string(), - ); - + config: reth_basic_payload_builder::PayloadConfig< + OpPayloadBuilderAttributes, + >, + cancel: CancellationToken, + extra_ctx: FlashblocksExtraCtx, + ) -> eyre::Result> { let chain_spec = self.client.chain_spec(); let timestamp = config.attributes.timestamp(); - let calculate_state_root = self.config.specific.calculate_state_root; let block_env_attributes = OpNextBlockEnvAttributes { timestamp, suggested_fee_recipient: config.attributes.suggested_fee_recipient(), @@ -264,7 +230,7 @@ where config .attributes .get_holocene_extra_data(chain_spec.base_fee_params_at_timestamp(timestamp)) - .map_err(PayloadBuilderError::other)? + .wrap_err("failed to get holocene extra data for flashblocks payload builder")? } else { Default::default() }, @@ -273,35 +239,78 @@ where let evm_env = self .evm_config .next_evm_env(&config.parent_header, &block_env_attributes) - .map_err(PayloadBuilderError::other)?; + .wrap_err("failed to create next evm env")?; - let mut ctx = OpPayloadBuilderCtx:: { + Ok(OpPayloadBuilderCtx:: { evm_config: self.evm_config.clone(), - chain_spec: self.client.chain_spec(), + chain_spec, config, evm_env, block_env_attributes, - // Here we use parent token because child token handing is only for proper flashblocks - cancel: block_cancel.clone(), + cancel, da_config: self.config.da_config.clone(), builder_signer: self.config.builder_signer, metrics: Default::default(), - extra_ctx: FlashblocksExtraCtx { - flashblock_index: 0, - target_flashblock_count: self.config.flashblocks_per_block(), - target_gas_for_batch: 0, - total_da_per_batch: None, - gas_per_batch: 0, - da_per_batch: None, - calculate_state_root, - }, + extra_ctx, max_gas_per_txn: self.config.max_gas_per_txn, address_gas_limiter: self.address_gas_limiter.clone(), + }) + } + + /// Constructs an Optimism payload from the transactions sent via the + /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in + /// the payload attributes, the transaction pool will be ignored and the only transactions + /// included in the payload will be those sent through the attributes. + /// + /// Given build arguments including an Optimism client, transaction pool, + /// and configuration, this function creates a transaction payload. Returns + /// a result indicating success with the payload or an error in case of failure. + async fn build_payload( + &self, + args: BuildArguments, OpBuiltPayload>, + best_payload: BlockCell, + ) -> Result<(), PayloadBuilderError> { + let block_build_start_time = Instant::now(); + let BuildArguments { + mut cached_reads, + config, + cancel: block_cancel, + } = args; + + // We log only every 100th block to reduce usage + let span = if cfg!(feature = "telemetry") + && config.parent_header.number % self.config.sampling_ratio == 0 + { + span!(Level::INFO, "build_payload") + } else { + tracing::Span::none() }; + let _entered = span.enter(); + span.record( + "payload_id", + config.attributes.payload_attributes.id.to_string(), + ); + + let timestamp = config.attributes.timestamp(); + let calculate_state_root = self.config.specific.calculate_state_root; + let ctx = self + .get_op_payload_builder_ctx( + config.clone(), + block_cancel.clone(), + FlashblocksExtraCtx { + flashblock_index: 0, + target_flashblock_count: self.config.flashblocks_per_block(), + target_gas_for_batch: 0, + total_da_per_batch: None, + gas_per_batch: 0, + da_per_batch: None, + calculate_state_root, + }, + ) + .map_err(|e| PayloadBuilderError::Other(e.into()))?; let state_provider = self.client.state_by_block_hash(ctx.parent().hash())?; let db = StateProviderDatabase::new(&state_provider); - self.address_gas_limiter.refresh(ctx.block_number()); // 1. execute the pre steps and seal an early block with that @@ -395,7 +404,6 @@ where // We adjust our flashblocks timings based on time_drift if dynamic adjustment enable let (flashblocks_per_block, first_flashblock_offset) = self.calculate_flashblocks(timestamp); - ctx.set_target_flashblock_count(flashblocks_per_block); info!( target: "payload_builder", message = "Performed flashblocks timing derivation", @@ -429,13 +437,23 @@ where let mut total_da_per_batch = da_per_batch; // Account for already included builder tx - ctx.extra_ctx.target_gas_for_batch = target_gas_for_batch.saturating_sub(builder_tx_gas); if let Some(da_limit) = total_da_per_batch.as_mut() { *da_limit = da_limit.saturating_sub(builder_tx_da_size); } - ctx.extra_ctx.total_da_per_batch = total_da_per_batch; - ctx.extra_ctx.gas_per_batch = gas_per_batch; - ctx.extra_ctx.da_per_batch = da_per_batch; + let extra_ctx = FlashblocksExtraCtx { + flashblock_index: 0, + target_flashblock_count: flashblocks_per_block, + target_gas_for_batch: target_gas_for_batch.saturating_sub(builder_tx_gas), + total_da_per_batch, + gas_per_batch, + da_per_batch, + calculate_state_root, + }; + + let mut fb_cancel = block_cancel.child_token(); + let mut ctx = self + .get_op_payload_builder_ctx(config, fb_cancel.clone(), extra_ctx) + .map_err(|e| PayloadBuilderError::Other(e.into()))?; // Create best_transaction iterator let mut best_txs = BestFlashblocksTxs::new(BestPayloadTransactions::new( @@ -444,8 +462,6 @@ where )); let interval = self.config.specific.interval; let (tx, mut rx) = mpsc::channel((self.config.flashblocks_per_block() + 1) as usize); - let mut fb_cancel = block_cancel.child_token(); - ctx.cancel = fb_cancel.clone(); tokio::spawn({ let block_cancel = block_cancel.clone(); @@ -496,9 +512,9 @@ where let _entered = fb_span.enter(); // build first flashblock immediately - match self + let next_flashblocks_ctx = match self .build_next_flashblock( - &mut ctx, + &ctx, &mut info, &mut state, &state_provider, @@ -509,7 +525,17 @@ where ) .await { - Ok(()) => {} + Ok(Some(next_flashblocks_ctx)) => next_flashblocks_ctx, + Ok(None) => { + self.record_flashblocks_metrics( + &ctx, + &info, + flashblocks_per_block, + &span, + "Payload building complete, job cancelled or target flashblock count reached", + ); + return Ok(()); + } Err(err) => { error!( target: "payload_builder", @@ -520,11 +546,11 @@ where ); return Err(PayloadBuilderError::Other(err.into())); } - } + }; tokio::select! { Some(fb_cancel) = rx.recv() => { - ctx.cancel = fb_cancel; + ctx = ctx.with_cancel(fb_cancel).with_extra_ctx(next_flashblocks_ctx); }, _ = block_cancel.cancelled() => { self.record_flashblocks_metrics( @@ -546,7 +572,7 @@ where P: StateRootProvider + HashedPostStateProvider + StorageRootProvider, >( &self, - ctx: &mut OpPayloadBuilderCtx, + ctx: &OpPayloadBuilderCtx, info: &mut ExecutionInfo, state: &mut State, state_provider: impl reth::providers::StateProvider + Clone, @@ -554,9 +580,9 @@ where block_cancel: &CancellationToken, best_payload: &BlockCell, span: &tracing::Span, - ) -> eyre::Result<()> { + ) -> eyre::Result> { // fallback block is index 0, so we need to increment here - ctx.increment_flashblock_index(); + let flashblock_index = ctx.next_flashblock_index(); // TODO: remove this if ctx.flashblock_index() > ctx.target_flashblock_count() { @@ -567,7 +593,7 @@ where block_number = ctx.block_number(), "Skipping flashblock reached target", ); - return Ok(()); + return Ok(None); }; // Continue with flashblock building @@ -650,7 +676,7 @@ where span, "Payload building complete, channel closed or job cancelled", ); - return Ok(()); + return Ok(None); } let payload_tx_simulation_time = tx_execution_start_time.elapsed(); @@ -661,15 +687,11 @@ where .payload_tx_simulation_gauge .set(payload_tx_simulation_time); - match self + if let Err(e) = self .builder_tx .add_builder_txs(&state_provider, info, ctx, state, false) { - Ok(builder_txs) => builder_txs, - Err(e) => { - error!(target: "payload_builder", "Error simulating builder txs: {}", e); - vec![] - } + error!(target: "payload_builder", "Error simulating builder txs: {}", e); }; let total_block_built_duration = Instant::now(); @@ -706,7 +728,7 @@ where span, "Payload building complete, channel closed or job cancelled", ); - return Ok(()); + return Ok(None); } let flashblock_byte_size = self .ws_pub @@ -740,20 +762,30 @@ where } } - ctx.extra_ctx.target_gas_for_batch += ctx.extra_ctx.gas_per_batch; - ctx.extra_ctx.total_da_per_batch = total_da_per_batch; + let target_gas_for_batch = + ctx.extra_ctx.target_gas_for_batch + ctx.extra_ctx.gas_per_batch; + let next_extra_ctx = FlashblocksExtraCtx { + flashblock_index, + target_flashblock_count: ctx.target_flashblock_count(), + target_gas_for_batch, + total_da_per_batch, + gas_per_batch: ctx.extra_ctx.gas_per_batch, + da_per_batch: ctx.extra_ctx.da_per_batch, + calculate_state_root: ctx.extra_ctx.calculate_state_root, + }; info!( target: "payload_builder", message = "Flashblock built", - flashblock_index = ctx.flashblock_index(), + flashblock_index = flashblock_index, current_gas = info.cumulative_gas_used, current_da = info.cumulative_da_bytes_used, target_flashblocks = ctx.target_flashblock_count(), ); + + Ok(Some(next_extra_ctx)) } } - Ok(()) } /// Do some logging and metric recording when we stop build flashblocks @@ -892,13 +924,13 @@ where .map_err(PayloadBuilderError::other)? .apply_pre_execution_changes()?; - // 3. execute sequencer transactions + // 2. execute sequencer transactions let info = ctx.execute_sequencer_transactions(state)?; Ok(info) } -fn build_block( +pub(super) fn build_block( state: &mut State, ctx: &OpPayloadBuilderCtx, info: &mut ExecutionInfo, @@ -945,7 +977,7 @@ where .expect("Number is in range"); // TODO: maybe recreate state with bundle in here - // // calculate the state root + // calculate the state root let state_root_start_time = Instant::now(); let mut state_root = B256::ZERO; let mut trie_output = TrieUpdates::default(); diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 38ba6e66c..6812a4cfa 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -1,10 +1,32 @@ -use crate::builders::flashblocks::p2p::Message; +use crate::{ + builders::flashblocks::{ctx::OpPayloadSyncerCtx, p2p::Message, payload::ExtraExecutionInfo}, + gas_limiter::{AddressGasLimiter, args::GasLimiterArgs}, + metrics::OpRBuilderMetrics, + primitives::reth::ExecutionInfo, + traits::ClientBounds, +}; +use alloy_evm::eth::receipt_builder::ReceiptBuilderCtx; +use eyre::WrapErr as _; +use futures::stream::FuturesUnordered; +use futures_util::StreamExt as _; +use op_alloy_consensus::OpTxEnvelope; +use reth::revm::{State, database::StateProviderDatabase}; +use reth_basic_payload_builder::PayloadConfig; +use reth_evm::FromRecoveredTx; use reth_node_builder::Events; -use reth_optimism_node::OpEngineTypes; +use reth_optimism_chainspec::OpChainSpec; +use reth_optimism_evm::{OpEvmConfig, OpNextBlockEnvAttributes}; +use reth_optimism_node::{OpEngineTypes, OpPayloadBuilderAttributes}; use reth_optimism_payload_builder::OpBuiltPayload; +use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; +use rollup_boost::FlashblocksPayloadV1; +use std::{ + collections::{HashSet, VecDeque}, + sync::Arc, +}; use tokio::sync::mpsc; -pub(crate) struct PayloadHandler { +pub(crate) struct PayloadHandler { // receives new payloads built by us. built_rx: mpsc::Receiver, // receives incoming p2p messages from peers. @@ -13,20 +35,39 @@ pub(crate) struct PayloadHandler { p2p_tx: mpsc::Sender, // sends a `Events::BuiltPayload` to the reth payload builder when a new payload is received. payload_events_handle: tokio::sync::broadcast::Sender>, + // context required for execution of blocks during syncing + ctx: OpPayloadSyncerCtx, + metrics: Arc, + gas_limiter_config: GasLimiterArgs, + client: Client, + cancel: tokio_util::sync::CancellationToken, } -impl PayloadHandler { +impl PayloadHandler +where + Client: ClientBounds + 'static, +{ pub(crate) fn new( built_rx: mpsc::Receiver, p2p_rx: mpsc::Receiver, p2p_tx: mpsc::Sender, payload_events_handle: tokio::sync::broadcast::Sender>, + ctx: OpPayloadSyncerCtx, + metrics: Arc, + gas_limiter_config: GasLimiterArgs, + client: Client, + cancel: tokio_util::sync::CancellationToken, ) -> Self { Self { built_rx, p2p_rx, p2p_tx, payload_events_handle, + ctx, + metrics, + gas_limiter_config, + client, + cancel, } } @@ -36,10 +77,17 @@ impl PayloadHandler { mut p2p_rx, p2p_tx, payload_events_handle, + ctx, + metrics, + gas_limiter_config, + client, + cancel, } = self; tracing::info!("flashblocks payload handler started"); + let mut execute_flashblock_futures = FuturesUnordered::new(); + loop { tokio::select! { Some(payload) = built_rx.recv() => { @@ -51,13 +99,337 @@ impl PayloadHandler { match message { Message::OpBuiltPayload(payload) => { let payload: OpBuiltPayload = payload.into(); - let _ = payload_events_handle.send(Events::BuiltPayload(payload)); - // TODO: what other handling is needed here? - // clearing mempool of included txs? + // TODO: is this necessary? maybe only for built? + // let _ = payload_events_handle.send(Events::BuiltPayload(payload.clone())); + let handle = tokio::spawn( + execute_flashblock( + payload, + ctx.clone(), + client.clone(), + metrics.clone(), + cancel.clone(), + gas_limiter_config.clone(), + ) + ); + execute_flashblock_futures.push(handle); } } } + Some(res) = execute_flashblock_futures.next() => { + match res { + Ok(Ok((payload, _))) => { + tracing::info!("successfully executed flashblock"); + let _ = payload_events_handle.send(Events::BuiltPayload(payload)); // TODO is this only for built or also synced? + } + Ok(Err(e)) => { + tracing::error!(error = %e, "failed to execute flashblock"); + } + Err(e) => { + tracing::error!(error = %e, "task panicked while executing flashblock"); + } + } + } + else => break, } } } } + +async fn execute_flashblock( + payload: OpBuiltPayload, + ctx: OpPayloadSyncerCtx, + client: Client, + metrics: Arc, + cancel: tokio_util::sync::CancellationToken, + gas_limiter_config: GasLimiterArgs, +) -> eyre::Result<(OpBuiltPayload, FlashblocksPayloadV1)> +where + Client: ClientBounds, +{ + use reth_evm::{ConfigureEvm as _, execute::BlockBuilder as _}; + use reth_optimism_chainspec::OpHardforks as _; + use reth_payload_primitives::PayloadBuilderAttributes as _; + + let mut cached_reads = reth::revm::cached::CachedReads::default(); // TODO: pass this in from somewhere + let payload_config = PayloadConfig::new( + Arc::new(payload.block().sealed_header().clone()), + OpPayloadBuilderAttributes::default(), + ); + + let state_provider = client.state_by_block_hash(payload_config.parent_header.hash())?; + let db = StateProviderDatabase::new(&state_provider); + let mut state = State::builder() + .with_database(cached_reads.as_db_mut(db)) + .with_bundle_update() + .build(); + + let chain_spec = client.chain_spec(); + let timestamp = payload_config.attributes.timestamp(); + let block_env_attributes = OpNextBlockEnvAttributes { + timestamp, + suggested_fee_recipient: payload_config.attributes.suggested_fee_recipient(), + prev_randao: payload_config.attributes.prev_randao(), + gas_limit: payload_config + .attributes + .gas_limit + .unwrap_or(payload_config.parent_header.gas_limit), + parent_beacon_block_root: payload_config + .attributes + .payload_attributes + .parent_beacon_block_root, + extra_data: if chain_spec.is_holocene_active_at_timestamp(timestamp) { + payload_config + .attributes + .get_holocene_extra_data(chain_spec.base_fee_params_at_timestamp(timestamp)) + .wrap_err("failed to get holocene extra data for flashblocks payload builder")? + } else { + Default::default() + }, + }; + + let evm_env = ctx + .evm_config() + .next_evm_env(&payload_config.parent_header, &block_env_attributes) + .wrap_err("failed to create next evm env")?; + + let address_gas_limiter = AddressGasLimiter::new(gas_limiter_config); + // TODO: can probably refactor this + let builder_ctx = ctx.into_op_payload_builder_ctx( + payload_config, + evm_env.clone(), + block_env_attributes, + cancel, + metrics, + address_gas_limiter, + ); + + // copy of `execute_pre_steps()` + builder_ctx + .evm_config + .builder_for_next_block( + &mut state, + &builder_ctx.config.parent_header, + builder_ctx.block_env_attributes.clone(), + ) + .wrap_err("failed to create evm builder for next block")? + .apply_pre_execution_changes() + .wrap_err("failed to apply pre execution changes")?; + let mut info: ExecutionInfo = builder_ctx + .execute_sequencer_transactions(&mut state) + .wrap_err("failed to execute sequencer transactions")?; + + execute_transactions( + &mut info, + &mut state, + &mut FlashblockTransactions::new(payload.block().body().transactions.clone()), // TODO: unnecessary + payload.block().header().gas_used, + &builder_ctx.evm_config, + evm_env, + builder_ctx.max_gas_per_txn, + is_canyon_active(&chain_spec, timestamp), + ) + .wrap_err("failed to execute best transactions")?; + + let (payload, fb_payload) = crate::builders::flashblocks::payload::build_block( + &mut state, + &builder_ctx, + &mut info, + true, // TODO: do we need this always? + )?; + + Ok((payload, fb_payload)) +} + +struct FlashblockTransactions { + txs: VecDeque, + invalid_txs: HashSet, +} + +impl FlashblockTransactions { + fn new(txs: Vec) -> Self { + Self { + txs: txs.into(), + invalid_txs: HashSet::new(), + } + } +} + +impl reth_payload_util::PayloadTransactions for FlashblockTransactions { + type Transaction = OpTransactionSigned; + + /// Exclude descendants of the transaction with given sender and nonce from the iterator, + /// because this transaction won't be included in the block. + fn mark_invalid(&mut self, sender: alloy_primitives::Address, nonce: u64) { + use alloy_consensus::Transaction as _; + use reth_primitives_traits::SignerRecoverable as _; + + for tx in &self.txs { + let Ok(signer) = tx.recover_signer() else { + self.invalid_txs.insert(*tx.hash()); + continue; + }; + + if signer == sender && tx.nonce() >= nonce { + self.invalid_txs.insert(*tx.hash()); + } + } + } + + fn next(&mut self, _ctx: ()) -> Option { + while let Some(tx) = self.txs.pop_front() { + if !self.invalid_txs.contains(tx.hash()) { + return Some(tx); + } + } + None + } +} + +fn execute_transactions( + info: &mut ExecutionInfo, + state: &mut State, + txs: &mut impl reth_payload_util::PayloadTransactions, + gas_limit: u64, + evm_config: &reth_optimism_evm::OpEvmConfig, + evm_env: alloy_evm::EvmEnv, + max_gas_per_txn: Option, + is_canyon_active: bool, +) -> eyre::Result<()> { + use alloy_evm::{Evm as _, EvmError as _}; + use op_revm::{OpTransaction, transaction::deposit::DepositTransactionParts}; + use reth_evm::ConfigureEvm as _; + use reth_primitives_traits::SignerRecoverable as _; + use revm::{ + DatabaseCommit as _, + context::{TxEnv, result::ResultAndState}, + }; + + let mut gas_used: u64 = 0; + let mut evm = evm_config.evm_with_env(&mut *state, evm_env); + + while let Some(ref tx) = txs.next(()) { + let sender = tx + .recover_signer() + .wrap_err("failed to recover tx signer")?; + let tx_env = TxEnv::from_recovered_tx(&tx, sender); + let executable_tx = match tx { + OpTxEnvelope::Deposit(tx) => { + let deposit = DepositTransactionParts { + mint: Some(tx.mint), + source_hash: tx.source_hash, + is_system_transaction: tx.is_system_transaction, + }; + OpTransaction { + base: tx_env, + enveloped_tx: None, + deposit, + } + } + OpTxEnvelope::Legacy(_) => OpTransaction::new(tx_env), + OpTxEnvelope::Eip2930(_) => OpTransaction::new(tx_env), + OpTxEnvelope::Eip1559(_) => OpTransaction::new(tx_env), + OpTxEnvelope::Eip7702(_) => OpTransaction::new(tx_env), + }; + + let ResultAndState { result, state } = match evm.transact_raw(executable_tx) { + Ok(res) => res, + Err(err) => { + if let Some(err) = err.as_invalid_tx_err() { + // TODO: what invalid txs are allowed in the block? + // reverting txs should be allowed (?) but not straight up invalid ones + tracing::error!(error = %err, "skipping invalid transaction in flashblock"); + continue; + } + return Err(err).wrap_err("failed to execute flashblock transaction"); + } + }; + + if let Some(max_gas_per_txn) = max_gas_per_txn { + if result.gas_used() > max_gas_per_txn { + return Err(eyre::eyre!( + "transaction exceeded max gas per txn limit in flashblock" + )); + } + } + + let tx_gas_used = result.gas_used(); + gas_used = gas_used.checked_add(tx_gas_used).ok_or_else(|| { + eyre::eyre!("total gas used overflowed when executing flashblock transactions") + })?; + if gas_used > gas_limit { + return Err(eyre::eyre!( + "flashblock exceeded gas limit when executing transactions" + )); + } + + info.cumulative_gas_used += gas_used; + // info.cumulative_da_bytes_used += tx_da_size; + + let ctx = ReceiptBuilderCtx { + tx, + evm: &evm, + result, + state: &state, + cumulative_gas_used: info.cumulative_gas_used, + }; + // TODO: deposit_nonce may be Some in the case of a sequencer tx + info.receipts + .push(build_receipt(evm_config, ctx, None, is_canyon_active)); + + evm.db_mut().commit(state); + + // // update add to total fees + // let miner_fee = tx + // .effective_tip_per_gas(base_fee) + // .expect("fee is always valid; execution succeeded"); + // info.total_fees += U256::from(miner_fee) * U256::from(gas_used); + + // append sender and transaction to the respective lists + info.executed_senders.push(sender); + info.executed_transactions.push(tx.clone()); + } + + Ok(()) +} + +fn build_receipt( + evm_config: &OpEvmConfig, + ctx: ReceiptBuilderCtx<'_, OpTransactionSigned, E>, + deposit_nonce: Option, + is_canyon_active: bool, +) -> OpReceipt { + use alloy_consensus::Eip658Value; + use alloy_op_evm::block::receipt_builder::OpReceiptBuilder as _; + use op_alloy_consensus::OpDepositReceipt; + use reth_evm::ConfigureEvm as _; + + let receipt_builder = evm_config.block_executor_factory().receipt_builder(); + match receipt_builder.build_receipt(ctx) { + Ok(receipt) => receipt, + Err(ctx) => { + let receipt = alloy_consensus::Receipt { + // Success flag was added in `EIP-658: Embedding transaction status code + // in receipts`. + status: Eip658Value::Eip658(ctx.result.is_success()), + cumulative_gas_used: ctx.cumulative_gas_used, + logs: ctx.result.into_logs(), + }; + + receipt_builder.build_deposit_receipt(OpDepositReceipt { + inner: receipt, + deposit_nonce, + // The deposit receipt version was introduced in Canyon to indicate an + // update to how receipt hashes should be computed + // when set. The state transition process ensures + // this is only set for post-Canyon deposit + // transactions. + deposit_receipt_version: is_canyon_active.then_some(1), + }) + } + } +} + +fn is_canyon_active(chain_spec: &OpChainSpec, timestamp: u64) -> bool { + use reth_optimism_chainspec::OpHardforks as _; + chain_spec.is_canyon_active_at_timestamp(timestamp) +} diff --git a/crates/op-rbuilder/src/builders/flashblocks/service.rs b/crates/op-rbuilder/src/builders/flashblocks/service.rs index b9c98e1bc..aae03148b 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/service.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/service.rs @@ -12,6 +12,7 @@ use crate::{ generator::BlockPayloadJobGenerator, }, flashtestations::service::bootstrap_flashtestations, + metrics::OpRBuilderMetrics, traits::{NodeBounds, PoolBounds}, }; use eyre::WrapErr as _; @@ -21,6 +22,7 @@ use reth_node_builder::{BuilderContext, components::PayloadServiceBuilder}; use reth_optimism_evm::OpEvmConfig; use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService}; use reth_provider::CanonStateSubscriptions; +use std::sync::Arc; pub struct FlashblocksServiceBuilder(pub BuilderConfig); @@ -36,7 +38,10 @@ impl FlashblocksServiceBuilder { Pool: PoolBounds, BuilderTx: BuilderTransactions + Unpin + Clone + Send + Sync + 'static, { - let mut builder = p2p::NodeBuilder::new(); + // TODO: is there a different global token? + let cancel = tokio_util::sync::CancellationToken::new(); + + let mut builder = p2p::NodeBuilder::new().with_cancellation_token(cancel.clone()); if let Some(ref private_key_hex) = self.0.p2p_private_key_hex && !private_key_hex.is_empty() @@ -79,6 +84,8 @@ impl FlashblocksServiceBuilder { .remove(&FLASHBLOCKS_STREAM_PROTOCOL) .expect("flashblocks p2p protocol must be found in receiver map"); + let metrics = Arc::new(OpRBuilderMetrics::default()); + let gas_limiter_config = self.0.gas_limiter_config.clone(); let (built_payload_tx, built_payload_rx) = tokio::sync::mpsc::channel(16); let payload_builder = OpPayloadBuilder::new( OpEvmConfig::optimism(ctx.chain_spec()), @@ -87,6 +94,7 @@ impl FlashblocksServiceBuilder { self.0.clone(), builder_tx, built_payload_tx, + metrics.clone(), ) .wrap_err("failed to create flashblocks payload builder")?; @@ -104,11 +112,23 @@ impl FlashblocksServiceBuilder { let (payload_service, payload_builder_handle) = PayloadBuilderService::new(payload_generator, ctx.provider().canonical_state_stream()); + let syncer_ctx = crate::builders::flashblocks::ctx::OpPayloadSyncerCtx::new( + &ctx.provider().clone(), + self.0, + OpEvmConfig::optimism(ctx.chain_spec()), + ) + .wrap_err("failed to create flashblocks payload builder context")?; + let payload_handler = PayloadHandler::new( built_payload_rx, incoming_message_rx, outgoing_message_tx, payload_service.payload_events_handle(), + syncer_ctx, + metrics, + gas_limiter_config, + ctx.provider().clone(), + cancel, ); ctx.task_executor() diff --git a/crates/p2p/src/lib.rs b/crates/p2p/src/lib.rs index 1b36f8345..a598db7c5 100644 --- a/crates/p2p/src/lib.rs +++ b/crates/p2p/src/lib.rs @@ -219,6 +219,14 @@ impl NodeBuilder { self } + pub fn with_cancellation_token( + mut self, + cancellation_token: tokio_util::sync::CancellationToken, + ) -> Self { + self.cancellation_token = Some(cancellation_token); + self + } + pub fn with_known_peers(mut self, addresses: I) -> Self where I: IntoIterator, From 15b6c0348e312adfccf6b6a70b13d7f42f001199 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Thu, 9 Oct 2025 19:19:10 -0400 Subject: [PATCH 02/13] bump op-revm, debugging sync --- Cargo.lock | 2857 +++++++++-------- Cargo.toml | 159 +- crates/op-rbuilder/Cargo.toml | 2 +- crates/op-rbuilder/src/args/op.rs | 2 +- crates/op-rbuilder/src/builders/context.rs | 2 +- .../src/builders/flashblocks/payload.rs | 2 +- .../builders/flashblocks/payload_handler.rs | 48 +- crates/p2p/src/behaviour.rs | 18 +- crates/p2p/src/lib.rs | 15 +- crates/p2p/src/outgoing.rs | 10 +- 10 files changed, 1584 insertions(+), 1531 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c40ffbe3e..8d07b5362 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4,9 +4,9 @@ version = 4 [[package]] name = "addr2line" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" +checksum = "1b5d307320b3181d6d7954e663bd7c774a838b8220fe0593c86d9fb09f498b4b" dependencies = [ "gimli", ] @@ -97,9 +97,9 @@ checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" [[package]] name = "alloy" -version = "1.0.25" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "992a9d0732a0e0e1a34d61a6553ad28f761c9049bb46572d3916f172348d2cb7" +checksum = "b17c19591d57add4f0c47922877a48aae1f47074e3433436545f8948353b3bbb" dependencies = [ "alloy-consensus", "alloy-contract", @@ -120,11 +120,11 @@ dependencies = [ [[package]] name = "alloy-chains" -version = "0.2.7" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a379c0d821498c996ceb9e7519fc2dab8286c35a203c1fb95f80ecd66e07cf2f" +checksum = "bf01dd83a1ca5e4807d0ca0223c9615e211ce5db0a9fd1443c2778cacf89b546" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "num_enum", "serde", @@ -133,12 +133,12 @@ dependencies = [ [[package]] name = "alloy-consensus" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bf3c28aa7a5765042739f964e335408e434819b96fdda97f12eb1beb46dead0" +checksum = "6a0dd3ed764953a6b20458b2b7abbfdc93d20d14b38babe1a70fe631a443a9f1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-serde", "alloy-trie", @@ -155,18 +155,18 @@ dependencies = [ "serde", "serde_json", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-consensus-any" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbfda7b14f1664b6c23d7f38bca2b73c460f2497cf93dd1589753890cb0da158" +checksum = "9556182afa73cddffa91e64a5aa9508d5e8c912b3a15f26998d2388a824d2c7b" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-serde", "arbitrary", @@ -175,49 +175,49 @@ dependencies = [ [[package]] name = "alloy-contract" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6cb079f711129dd32d6c3a0581013c927eb30d32e929d606cd8c0fe1022ec041" +checksum = "b19d7092c96defc3d132ee0d8969ca1b79ef512b5eda5c66e3065266b253adf2" dependencies = [ "alloy-consensus", "alloy-dyn-abi", - "alloy-json-abi 1.3.1", + "alloy-json-abi 1.4.0", "alloy-network", "alloy-network-primitives", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-provider", "alloy-rpc-types-eth", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "alloy-transport", "futures", "futures-util", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-core" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfe6c56d58fbfa9f0f6299376e8ce33091fc6494239466814c3f54b55743cb09" +checksum = "575053cea24ea8cb7e775e39d5c53c33b19cfd0ca1cf6c0fd653f3d8c682095f" dependencies = [ "alloy-dyn-abi", - "alloy-json-abi 1.3.1", - "alloy-primitives 1.3.1", + "alloy-json-abi 1.4.0", + "alloy-primitives 1.4.0", "alloy-rlp", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", ] [[package]] name = "alloy-dyn-abi" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f56873f3cac7a2c63d8e98a4314b8311aa96adb1a0f82ae923eb2119809d2c" +checksum = "a6c2905bafc2df7ccd32ca3af13f0b0d82f2e2ff9dfbeb12196c0d978d5c0deb" dependencies = [ - "alloy-json-abi 1.3.1", - "alloy-primitives 1.3.1", - "alloy-sol-type-parser 1.3.1", - "alloy-sol-types 1.3.1", + "alloy-json-abi 1.4.0", + "alloy-primitives 1.4.0", + "alloy-sol-type-parser 1.4.0", + "alloy-sol-types 1.4.0", "derive_more", "itoa", "serde", @@ -231,13 +231,13 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "741bdd7499908b3aa0b159bba11e71c8cddd009a2c2eb7a06e825f1ec87900a5" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "arbitrary", "crc", "rand 0.8.5", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -246,7 +246,7 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b82752a889170df67bbb36d42ca63c531eb16274f0d7299ae2a680facba17bd" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "arbitrary", "rand 0.8.5", @@ -259,26 +259,26 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d4769c6ffddca380b0070d71c8b7f30bed375543fe76bb2f74ec0acf4b7cd16" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "arbitrary", "k256", "rand 0.8.5", "serde", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-eips" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72e57928382e5c7890ef90ded9f814d85a1c3db79ceb4a3c5079f1be4cadeeb4" +checksum = "305fa99b538ca7006b0c03cfed24ec6d82beda67aac857ef4714be24231d15e6" dependencies = [ "alloy-eip2124", "alloy-eip2930", "alloy-eip7702", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-serde", "arbitrary", @@ -290,38 +290,40 @@ dependencies = [ "ethereum_ssz_derive", "serde", "serde_with", - "sha2 0.10.9", - "thiserror 2.0.16", + "sha2", + "thiserror 2.0.17", ] [[package]] name = "alloy-evm" -version = "0.21.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48e536feefca2ba96c75798ac75a31046e8adfcefecdb6653803361045cc65b9" +version = "0.21.2" +source = "git+https://github.com/noot/evm?rev=c7ea94eaa1bfc0971624ccd3e6973059335cb862#c7ea94eaa1bfc0971624ccd3e6973059335cb862" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-hardforks", - "alloy-primitives 1.3.1", + "alloy-op-hardforks", + "alloy-primitives 1.4.0", + "alloy-rpc-types-engine", "alloy-rpc-types-eth", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "auto_impl", "derive_more", "op-alloy-consensus", + "op-alloy-rpc-types-engine", "op-revm", "revm", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-genesis" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3419410cdd67fb7d5d016d9d16cf3ea8cc365fcbcf15d086afdd02eaef17e4" +checksum = "a272533715aefc900f89d51db00c96e6fd4f517ea081a12fea482a352c8c815c" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-serde", "alloy-trie", "serde", @@ -336,7 +338,7 @@ checksum = "889eb3949b58368a09d4f16931c660275ef5fb08e5fbd4a96573b19c7085c41f" dependencies = [ "alloy-chains", "alloy-eip2124", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "dyn-clone", "serde", @@ -356,81 +358,80 @@ dependencies = [ [[package]] name = "alloy-json-abi" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "125a1c373261b252e53e04d6e92c37d881833afc1315fceab53fd46045695640" +checksum = "a2acb6637a9c0e1cdf8971e0ced8f3fa34c04c5e9dccf6bb184f6a64fe0e37d8" dependencies = [ - "alloy-primitives 1.3.1", - "alloy-sol-type-parser 1.3.1", + "alloy-primitives 1.4.0", + "alloy-sol-type-parser 1.4.0", "serde", "serde_json", ] [[package]] name = "alloy-json-rpc" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17248e392e79658b1faca7946bfe59825b891c3f6e382044499d99c57ba36a89" +checksum = "d91676d242c0ced99c0dd6d0096d7337babe9457cc43407d26aa6367fcf90553" dependencies = [ - "alloy-primitives 1.3.1", - "alloy-sol-types 1.3.1", + "alloy-primitives 1.4.0", + "alloy-sol-types 1.4.0", "http", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] [[package]] name = "alloy-network" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe43d21867dc0dcf71aacffc891ae75fd587154f0d907ceb7340fc5f0271276d" +checksum = "77f82150116b30ba92f588b87f08fa97a46a1bd5ffc0d0597efdf0843d36bfda" dependencies = [ "alloy-consensus", "alloy-consensus-any", "alloy-eips", "alloy-json-rpc", "alloy-network-primitives", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-any", "alloy-rpc-types-eth", "alloy-serde", "alloy-signer", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "async-trait", "auto_impl", "derive_more", "futures-utils-wasm", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-network-primitives" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67f3b37447082a47289f26e26c0686ac6407710fdd4e818043d9b6d37f2ab55c" +checksum = "223612259a080160ce839a4e5df0125ca403a1d5e7206cc911cea54af5d769aa" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-serde", "serde", ] [[package]] name = "alloy-op-evm" -version = "0.21.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19f09c7785a3f2df462e4bb898e8b682b43de488d9d44bf2e5264e0bba44af21" +version = "0.21.2" +source = "git+https://github.com/noot/evm?rev=c7ea94eaa1bfc0971624ccd3e6973059335cb862#c7ea94eaa1bfc0971624ccd3e6973059335cb862" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-op-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "op-alloy-consensus", "op-revm", @@ -445,7 +446,7 @@ checksum = "599c1d7dfbccb66603cb93fde00980d12848d32fe5e814f50562104a92df6487" dependencies = [ "alloy-chains", "alloy-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", ] @@ -460,9 +461,9 @@ dependencies = [ "cfg-if", "const-hex", "derive_more", - "foldhash", + "foldhash 0.1.5", "hashbrown 0.15.5", - "indexmap 2.10.0", + "indexmap 2.11.4", "itoa", "k256", "keccak-asm", @@ -478,9 +479,9 @@ dependencies = [ [[package]] name = "alloy-primitives" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc9485c56de23438127a731a6b4c87803d49faf1a7068dcd1d8768aca3a9edb9" +checksum = "5b77f7d5e60ad8ae6bd2200b8097919712a07a6db622a4b201e7ead6166f02e5" dependencies = [ "alloy-rlp", "arbitrary", @@ -488,16 +489,16 @@ dependencies = [ "cfg-if", "const-hex", "derive_more", - "foldhash", + "foldhash 0.2.0", "getrandom 0.3.3", - "hashbrown 0.15.5", - "indexmap 2.10.0", + "hashbrown 0.16.0", + "indexmap 2.11.4", "itoa", "k256", "keccak-asm", "paste", "proptest", - "proptest-derive", + "proptest-derive 0.6.0", "rand 0.9.2", "ruint", "rustc-hash 2.1.1", @@ -508,9 +509,9 @@ dependencies = [ [[package]] name = "alloy-provider" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b6377212f3e659173b939e8d3ec3292e246cb532eafd5a4f91e57fdb104b43c" +checksum = "f7283b81b6f136100b152e699171bc7ed8184a58802accbc91a7df4ebb944445" dependencies = [ "alloy-chains", "alloy-consensus", @@ -518,14 +519,14 @@ dependencies = [ "alloy-json-rpc", "alloy-network", "alloy-network-primitives", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-pubsub", "alloy-rpc-client", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-rpc-types-txpool", "alloy-signer", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "alloy-transport", "alloy-transport-http", "alloy-transport-ipc", @@ -543,7 +544,7 @@ dependencies = [ "reqwest", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", "url", @@ -552,12 +553,12 @@ dependencies = [ [[package]] name = "alloy-pubsub" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d27b4f1ac3a0388065f933f957f80e03d06c47ce6a4389ac8cb9f72c30d8d823" +checksum = "eee7e3d343814ec0dfea69bd1820042a133a9d0b9ac5faf1e6eb133b43366315" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-transport", "auto_impl", "bimap", @@ -596,12 +597,12 @@ dependencies = [ [[package]] name = "alloy-rpc-client" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b80c8cafc1735ce6776bccc25f0c3b7583074897b8ec4f3a129e4d25e09d65c" +checksum = "1154b12d470bef59951c62676e106f4ce5de73b987d86b9faa935acebb138ded" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-pubsub", "alloy-transport", "alloy-transport-http", @@ -622,11 +623,11 @@ dependencies = [ [[package]] name = "alloy-rpc-types" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bc0818982bb868acc877f2623ad1fc8f2a4b244074919212bfe476fcadca6d3" +checksum = "47ab76bf97648a1c6ad8fb00f0d594618942b5a9e008afbfb5c8a8fca800d574" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-serde", @@ -635,23 +636,23 @@ dependencies = [ [[package]] name = "alloy-rpc-types-admin" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9359aabfc2ae906ea9f904c6cf6a63d12fc6510e655a64c38aa601a739602e84" +checksum = "af8ae38824376e855d73d4060462d86c32afe548af632597ccfd161bdd0fc628" dependencies = [ "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "serde", "serde_json", ] [[package]] name = "alloy-rpc-types-anvil" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "410403528db87ab4618e7f517b0f54e493c8a17bb61102cbccbb7a35e8719b5b" +checksum = "456cfc2c1677260edbd7ce3eddb7de419cb46de0e9826c43401f42b0286a779a" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -659,9 +660,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-any" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af8448a1eb2c81115fc8d9d50da24156c9ce8fca78a19a997184dcd81f99c229" +checksum = "23cc57ee0c1ac9fb14854195fc249494da7416591dc4a4d981ddfd5dd93b9bce" dependencies = [ "alloy-consensus-any", "alloy-rpc-types-eth", @@ -670,30 +671,31 @@ dependencies = [ [[package]] name = "alloy-rpc-types-beacon" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c20f653a4c1ab8289470e8eed55fe4f11354865b730685bb70b69a375524b27" +checksum = "cfa4edd92c3124ec19b9d572dc7923d070fe5c2efb677519214affd6156a4463" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", + "derive_more", "ethereum_ssz", "ethereum_ssz_derive", "serde", "serde_json", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tree_hash", "tree_hash_derive", ] [[package]] name = "alloy-rpc-types-debug" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fb22d465e02c015648138bc0d46951d267827551fc85922b60f58caa6a0e9c9" +checksum = "4a0ac29dd005c33e3f7e09087accc80843315303685c3f7a1b888002cd27785b" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "derive_more", "serde", "serde_with", @@ -701,13 +703,13 @@ dependencies = [ [[package]] name = "alloy-rpc-types-engine" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b968beee2ada53ef150fd90fbd2b7a3e5bcb66650e4d01757ff769c8af3d5ee" +checksum = "1d9d173854879bcf26c7d71c1c3911972a3314df526f4349ffe488e676af577d" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-serde", "derive_more", @@ -721,35 +723,35 @@ dependencies = [ [[package]] name = "alloy-rpc-types-eth" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd7c1bc07b6c9222c4ad822da3cea0fbbfcbe2876cf5d4780e147a0da6fe2862" +checksum = "6d7d47bca1a2a1541e4404aa38b7e262bb4dffd9ac23b4f178729a4ddc5a5caa" dependencies = [ "alloy-consensus", "alloy-consensus-any", "alloy-eips", "alloy-network-primitives", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-serde", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "arbitrary", "itertools 0.14.0", "serde", "serde_json", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-rpc-types-mev" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad56da776d84940f075f6cdb27c95c17f5d8947ed89947d61b686247ec4e2200" +checksum = "d3820683ece7cdc31e44d87c88c0ff9b972a1a2fd1f2124cc72ce5c928e64f0d" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -758,25 +760,25 @@ dependencies = [ [[package]] name = "alloy-rpc-types-trace" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e54b3f616d9f30e11bc73e685f71da6f1682da5a3c2ca5206ec47f1d3bc96c7" +checksum = "c331c8e48665607682e8a9549a2347c13674d4fbcbdc342e7032834eba2424f4" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-serde", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-rpc-types-txpool" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15fc6b7b9465393a5b3fd38aba979f44438f172d9d0e6de732243c17d4246060" +checksum = "5e2f66afe1e76ca4485e593980056f061b2bdae2055486a062fca050ff111a52" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -784,11 +786,11 @@ dependencies = [ [[package]] name = "alloy-serde" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8603b89af4ba0acb94465319e506b8c0b40a5daf563046bedd58d26c98dbd62c" +checksum = "6a8468f1a7f9ee3bae73c24eead0239abea720dbf7779384b9c7e20d51bfb6b0" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "arbitrary", "serde", "serde_json", @@ -796,33 +798,33 @@ dependencies = [ [[package]] name = "alloy-signer" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78ddbea0531837cc7784ae6669b4a66918e6fb34c2daa2a7a888549dd565151c" +checksum = "33387c90b0a5021f45a5a77c2ce6c49b8f6980e66a318181468fb24cea771670" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "async-trait", "auto_impl", "either", "elliptic-curve", "k256", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "alloy-signer-local" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3497f79c8a818f736d8de1c157a1ec66c0ce1da3fbb2f54c005097798282e59b" +checksum = "b55d9e795c85e36dcea08786d2e7ae9b73cb554b6bea6ac4c212def24e1b4d03" dependencies = [ "alloy-consensus", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-signer", "async-trait", "k256", "rand 0.8.5", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -841,12 +843,12 @@ dependencies = [ [[package]] name = "alloy-sol-macro" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d20d867dcf42019d4779519a1ceb55eba8d7f3d0e4f0a89bcba82b8f9eb01e48" +checksum = "78c84c3637bee9b5c4a4d2b93360ee16553d299c3b932712353caf1cea76d0e6" dependencies = [ - "alloy-sol-macro-expander 1.3.1", - "alloy-sol-macro-input 1.3.1", + "alloy-sol-macro-expander 1.4.0", + "alloy-sol-macro-input 1.4.0", "proc-macro-error2", "proc-macro2", "quote", @@ -862,7 +864,7 @@ dependencies = [ "alloy-sol-macro-input 0.8.25", "const-hex", "heck", - "indexmap 2.10.0", + "indexmap 2.11.4", "proc-macro-error2", "proc-macro2", "quote", @@ -873,20 +875,20 @@ dependencies = [ [[package]] name = "alloy-sol-macro-expander" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b74e91b0b553c115d14bd0ed41898309356dc85d0e3d4b9014c4e7715e48c8ad" +checksum = "a882aa4e1790063362434b9b40d358942b188477ac1c44cfb8a52816ffc0cc17" dependencies = [ - "alloy-json-abi 1.3.1", - "alloy-sol-macro-input 1.3.1", + "alloy-json-abi 1.4.0", + "alloy-sol-macro-input 1.4.0", "const-hex", "heck", - "indexmap 2.10.0", + "indexmap 2.11.4", "proc-macro-error2", "proc-macro2", "quote", "syn 2.0.106", - "syn-solidity 1.3.1", + "syn-solidity 1.4.0", "tiny-keccak", ] @@ -908,11 +910,11 @@ dependencies = [ [[package]] name = "alloy-sol-macro-input" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84194d31220803f5f62d0a00f583fd3a062b36382e2bea446f1af96727754565" +checksum = "18e5772107f9bb265d8d8c86e0733937bb20d0857ea5425b1b6ddf51a9804042" dependencies = [ - "alloy-json-abi 1.3.1", + "alloy-json-abi 1.4.0", "const-hex", "dunce", "heck", @@ -921,7 +923,7 @@ dependencies = [ "quote", "serde_json", "syn 2.0.106", - "syn-solidity 1.3.1", + "syn-solidity 1.4.0", ] [[package]] @@ -936,9 +938,9 @@ dependencies = [ [[package]] name = "alloy-sol-type-parser" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe8c27b3cf6b2bb8361904732f955bc7c05e00be5f469cec7e2280b6167f3ff0" +checksum = "e188b939aa4793edfaaa099cb1be4e620036a775b4bdf24fdc56f1cd6fd45890" dependencies = [ "serde", "winnow", @@ -959,24 +961,24 @@ dependencies = [ [[package]] name = "alloy-sol-types" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5383d34ea00079e6dd89c652bcbdb764db160cef84e6250926961a0b2295d04" +checksum = "c3c8a9a909872097caffc05df134e5ef2253a1cdb56d3a9cf0052a042ac763f9" dependencies = [ - "alloy-json-abi 1.3.1", - "alloy-primitives 1.3.1", - "alloy-sol-macro 1.3.1", + "alloy-json-abi 1.4.0", + "alloy-primitives 1.4.0", + "alloy-sol-macro 1.4.0", "serde", ] [[package]] name = "alloy-transport" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d259738315db0a2460581e22a1ca73ff02ef44687b43c0dad0834999090b3e7e" +checksum = "702002659778d89a94cd4ff2044f6b505460df6c162e2f47d1857573845b0ace" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "base64 0.22.1", "derive_more", @@ -985,7 +987,7 @@ dependencies = [ "parking_lot", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tower 0.5.2", "tracing", @@ -995,12 +997,16 @@ dependencies = [ [[package]] name = "alloy-transport-http" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6332f6d470e465bf00f9306743ff172f54b83e7e31edfe28f1444c085ccb0e4" +checksum = "0d6bdc0830e5e8f08a4c70a4c791d400a86679c694a3b4b986caf26fad680438" dependencies = [ "alloy-json-rpc", "alloy-transport", + "http-body-util", + "hyper", + "hyper-tls", + "hyper-util", "reqwest", "serde_json", "tower 0.5.2", @@ -1010,9 +1016,9 @@ dependencies = [ [[package]] name = "alloy-transport-ipc" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "865c13b9ce32b1a5227ac0f796faa9c08416aa4ea4e22b3a61a21ef110bda5ad" +checksum = "87ce41d99a32346f354725fe62eadd271cdbae45fe6b3cc40cb054e0bf763112" dependencies = [ "alloy-json-rpc", "alloy-pubsub", @@ -1030,9 +1036,9 @@ dependencies = [ [[package]] name = "alloy-transport-ws" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da655a5099cc037cad636425cec389320a694b6ec0302472a74f71b3637d842d" +checksum = "686219dcef201655763bd3d4eabe42388d9368bfbf6f1c8016d14e739ec53aac" dependencies = [ "alloy-pubsub", "alloy-transport", @@ -1052,7 +1058,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3412d52bb97c6c6cc27ccc28d4e6e8cf605469101193b50b0bd5813b1f990b5" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "arbitrary", "arrayvec", @@ -1060,7 +1066,7 @@ dependencies = [ "derive_more", "nybbles", "proptest", - "proptest-derive", + "proptest-derive 0.5.1", "serde", "smallvec", "tracing", @@ -1068,23 +1074,17 @@ dependencies = [ [[package]] name = "alloy-tx-macros" -version = "1.0.35" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2765badc6f621e1fc26aa70c520315866f0db6b8bd6bf3c560920d4fb33b08de" +checksum = "7bf39928a5e70c9755d6811a2928131b53ba785ad37c8bf85c90175b5d43b818" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "darling 0.21.3", "proc-macro2", "quote", "syn 2.0.106", ] -[[package]] -name = "android-tzdata" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" - [[package]] name = "android_system_properties" version = "0.1.5" @@ -1096,9 +1096,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.20" +version = "0.6.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae563653d1938f79b1ab1b5e668c87c76a9930414574a6583a7b7e11a8e6192" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" dependencies = [ "anstyle", "anstyle-parse", @@ -1111,9 +1111,9 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.11" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" +checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" [[package]] name = "anstyle-parse" @@ -1146,9 +1146,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.99" +version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0674a1ddeecb70197781e945de4b3b8ffb61fa939a5597bcf48503737663100" +checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" [[package]] name = "aquamarine" @@ -1506,7 +1506,7 @@ dependencies = [ "nom", "num-traits", "rusticata-macros", - "thiserror 2.0.16", + "thiserror 2.0.17", "time", ] @@ -1564,18 +1564,15 @@ checksum = "155a5a185e42c6b77ac7b88a15143d930a9e9727a5b7b77eed417404ab15c247" [[package]] name = "async-compression" -version = "0.4.27" +version = "0.4.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddb939d66e4ae03cee6091612804ba446b12878410cfa17f785f4dd67d4014e8" +checksum = "5a89bce6054c720275ac2432fbba080a66a2106a44a1b804553930ca6909f4e0" dependencies = [ - "brotli", - "flate2", + "compression-codecs", + "compression-core", "futures-core", - "memchr", "pin-project-lite", "tokio", - "zstd", - "zstd-safe", ] [[package]] @@ -1591,9 +1588,9 @@ dependencies = [ "futures-lite", "parking", "polling", - "rustix 1.0.8", + "rustix 1.1.2", "slab", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -1761,11 +1758,11 @@ dependencies = [ [[package]] name = "axum" -version = "0.8.4" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "021e862c184ae977658b36c4500f7feac3221ca5da43e3f25bd04ab6c79a29b5" +checksum = "8a18ed336352031311f4e0b4dd2ff392d4fbb370777c9d18d7fc9d7359f73871" dependencies = [ - "axum-core 0.5.2", + "axum-core 0.5.5", "bytes", "form_urlencoded", "futures-util", @@ -1780,8 +1777,7 @@ dependencies = [ "mime", "percent-encoding", "pin-project-lite", - "rustversion", - "serde", + "serde_core", "serde_json", "serde_path_to_error", "serde_urlencoded", @@ -1815,9 +1811,9 @@ dependencies = [ [[package]] name = "axum-core" -version = "0.5.2" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68464cd0412f486726fb3373129ef5d2993f90c34bc2bc1c1e9943b2f4fc7ca6" +checksum = "59446ce19cd142f8833f856eb31f3eb097812d1479ab224f54d72428ca21ea22" dependencies = [ "bytes", "futures-core", @@ -1826,7 +1822,6 @@ dependencies = [ "http-body-util", "mime", "pin-project-lite", - "rustversion", "sync_wrapper", "tower-layer", "tower-service", @@ -1851,9 +1846,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.75" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" +checksum = "bb531853791a215d7c62a30daf0dde835f381ab5de4589cfe7c649d2cbe92bd6" dependencies = [ "addr2line", "cfg-if", @@ -1861,7 +1856,7 @@ dependencies = [ "miniz_oxide", "object", "rustc-demangle", - "windows-targets 0.52.6", + "windows-link 0.2.1", ] [[package]] @@ -1876,6 +1871,16 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c7f02d4ea65f2c1853089ffd8d2787bdbc63de2f0d29dedbcf8ccdfa0ccd4cf" +[[package]] +name = "base256emoji" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e9430d9a245a77c92176e649af6e275f20839a48389859d1661e9a128d077c" +dependencies = [ + "const-str", + "match-lookup", +] + [[package]] name = "base64" version = "0.21.7" @@ -1924,7 +1929,7 @@ version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1943,18 +1948,36 @@ dependencies = [ [[package]] name = "bindgen" -version = "0.70.1" +version = "0.71.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f49d8fed880d473ea71efb9bf597651e77201bdd4893efe54c9e5d65ae04ce6f" +checksum = "5f58bf3d7db68cfbac37cfc485a8d711e87e064c3d0fe0435b92f7a407f9d6b3" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "cexpr", "clang-sys", "itertools 0.13.0", "proc-macro2", "quote", "regex", - "rustc-hash 1.1.0", + "rustc-hash 2.1.1", + "shlex", + "syn 2.0.106", +] + +[[package]] +name = "bindgen" +version = "0.72.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "993776b509cfb49c750f11b8f07a46fa23e0a1386ffc01fb1e7d343efc387895" +dependencies = [ + "bitflags 2.9.4", + "cexpr", + "clang-sys", + "itertools 0.13.0", + "proc-macro2", + "quote", + "regex", + "rustc-hash 2.1.1", "shlex", "syn 2.0.106", ] @@ -1998,18 +2021,18 @@ checksum = "2d7e60934ceec538daadb9d8432424ed043a904d8e0243f3c6446bce549a46ac" [[package]] name = "bitfield" -version = "0.19.1" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db1bcd90f88eabbf0cadbfb87a45bceeaebcd3b4bc9e43da379cd2ef0162590d" +checksum = "62a3a774b2fcac1b726922b921ebba5e9fe36ad37659c822cf8ff2c1e0819892" dependencies = [ "bitfield-macros", ] [[package]] name = "bitfield-macros" -version = "0.19.1" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3787a07661997bfc05dd3431e379c0188573f78857080cf682e1393ab8e4d64c" +checksum = "52511b09931f7d5fe3a14f23adefbc23e5725b184013e96c8419febb61f14734" dependencies = [ "proc-macro2", "quote", @@ -2024,9 +2047,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.9.2" +version = "2.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a65b545ab31d687cff52899d4890855fec459eb6afe0da6417b8a18da87aa29" +checksum = "2261d10cca569e4643e526d8dc2e62e433cc8aba21ab764233731f8d369bf394" dependencies = [ "serde", ] @@ -2053,15 +2076,6 @@ dependencies = [ "digest 0.10.7", ] -[[package]] -name = "block-buffer" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" -dependencies = [ - "generic-array", -] - [[package]] name = "block-buffer" version = "0.10.4" @@ -2082,9 +2096,9 @@ dependencies = [ [[package]] name = "blst" -version = "0.3.15" +version = "0.3.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fd49896f12ac9b6dcd7a5998466b9b58263a695a3dd1ecc1aaca2e12a90b080" +checksum = "dcdb4c7013139a150f9fc55d123186dbfaba0d912817466282c73ac49e71fb45" dependencies = [ "cc", "glob", @@ -2098,11 +2112,11 @@ version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c340fe0f0b267787095cbe35240c6786ff19da63ec7b69367ba338eace8169b" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "boa_interner", "boa_macros", "boa_string", - "indexmap 2.10.0", + "indexmap 2.11.4", "num-bigint", "rustc-hash 2.1.1", ] @@ -2114,7 +2128,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f620c3f06f51e65c0504ddf04978be1b814ac6586f0b45f6019801ab5efd37f9" dependencies = [ "arrayvec", - "bitflags 2.9.2", + "bitflags 2.9.4", "boa_ast", "boa_gc", "boa_interner", @@ -2128,7 +2142,7 @@ dependencies = [ "fast-float2", "hashbrown 0.15.5", "icu_normalizer 1.5.0", - "indexmap 2.10.0", + "indexmap 2.11.4", "intrusive-collections", "itertools 0.13.0", "num-bigint", @@ -2148,7 +2162,7 @@ dependencies = [ "static_assertions", "tap", "thin-vec", - "thiserror 2.0.16", + "thiserror 2.0.17", "time", ] @@ -2174,9 +2188,9 @@ dependencies = [ "boa_gc", "boa_macros", "hashbrown 0.15.5", - "indexmap 2.10.0", + "indexmap 2.11.4", "once_cell", - "phf", + "phf 0.11.3", "rustc-hash 2.1.1", "static_assertions", ] @@ -2199,7 +2213,7 @@ version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9cc142dac798cdc6e2dbccfddeb50f36d2523bb977a976e19bdb3ae19b740804" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "boa_ast", "boa_interner", "boa_macros", @@ -2262,7 +2276,7 @@ dependencies = [ "serde_json", "serde_repr", "serde_urlencoded", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-util", "tower-service", @@ -2340,18 +2354,18 @@ checksum = "175812e0be2bccb6abe50bb8d566126198344f707e304f45c648fd8f2cc0365e" [[package]] name = "bytemuck" -version = "1.23.2" +version = "1.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3995eaeebcdf32f91f980d360f78732ddc061097ab4e39991ae7a6ace9194677" +checksum = "1fbdf580320f38b612e485521afda1ee26d10cc9884efaaa750d383e13e3c5f4" dependencies = [ "bytemuck_derive", ] [[package]] name = "bytemuck_derive" -version = "1.10.1" +version = "1.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f154e572231cb6ba2bd1176980827e3d5dc04cc183a75dea38109fbdd672d29" +checksum = "f9abbd1bc6865053c427f7198e6af43bfdedc55ab791faed4fbd361d789575ff" dependencies = [ "proc-macro2", "quote", @@ -2375,9 +2389,9 @@ dependencies = [ [[package]] name = "c-kzg" -version = "2.1.4" +version = "2.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "137a2a2878ed823ef1bd73e5441e245602aae5360022113b8ad259ca4b5b8727" +checksum = "e00bf4b112b07b505472dbefd19e37e53307e2bfed5a79e0cc161d58ccd0e687" dependencies = [ "arbitrary", "blst", @@ -2391,11 +2405,11 @@ dependencies = [ [[package]] name = "camino" -version = "1.1.11" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d07aa9a93b00c76f71bc35d598bed923f6d4f3a9ca5c24b7737ae1a292841c0" +checksum = "276a59bf2b2c967788139340c9f0c5b12d7fd6630315c15c217e559de85d2609" dependencies = [ - "serde", + "serde_core", ] [[package]] @@ -2415,7 +2429,7 @@ checksum = "4acbb09d9ee8e23699b9634375c72795d095bf268439da88562cf9b501f181fa" dependencies = [ "camino", "cargo-platform", - "semver 1.0.26", + "semver 1.0.27", "serde", "serde_json", ] @@ -2428,10 +2442,10 @@ checksum = "dd5eb614ed4c27c5d706420e4320fbe3216ab31fa1c33cd8246ac36dae4479ba" dependencies = [ "camino", "cargo-platform", - "semver 1.0.26", + "semver 1.0.27", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -2457,7 +2471,7 @@ checksum = "975982cdb7ad6a142be15bdf84aea7ec6a9e5d4d797c004d43185b24cfe4e684" dependencies = [ "clap", "heck", - "indexmap 2.10.0", + "indexmap 2.11.4", "log", "proc-macro2", "quote", @@ -2541,17 +2555,16 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.41" +version = "0.4.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c469d952047f47f91b68d1cba3f10d63c11d73e4636f24f08daf0278abf01c4d" +checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" dependencies = [ - "android-tzdata", "iana-time-zone", "js-sys", "num-traits", "serde", "wasm-bindgen", - "windows-link 0.1.3", + "windows-link 0.2.1", ] [[package]] @@ -2578,9 +2591,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.45" +version = "4.5.48" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fc0e74a703892159f5ae7d3aac52c8e6c392f5ae5f359c70b5881d60aaac318" +checksum = "e2134bb3ea021b78629caa971416385309e0131b351b25e01dc16fb54e1b5fae" dependencies = [ "clap_builder", "clap_derive", @@ -2588,9 +2601,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.44" +version = "4.5.48" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3e7f4214277f3c7aa526a59dd3fbe306a370daee1f8b7b8c987069cd8e888a8" +checksum = "c2ba64afa3c0a6df7fa517765e31314e983f51dda798ffba27b988194fb65dc9" dependencies = [ "anstream", "anstyle", @@ -2600,9 +2613,9 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.45" +version = "4.5.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14cb31bb0a7d536caef2639baa7fad459e15c3144efefa6dbd1c84562c4739f6" +checksum = "bbfd7eae0b0f1a6e63d4b13c9c478de77c2eb546fba158ad50b4203dc24b9f9c" dependencies = [ "heck", "proc-macro2", @@ -2631,7 +2644,7 @@ version = "0.3.0" source = "git+https://github.com/automata-network/coco-provider-sdk#3a832b8cf5e88ef71649ab56e4efd67067b26b7c" dependencies = [ "bincode", - "bitfield 0.19.1", + "bitfield 0.19.2", "cbindgen", "iocuddle", "libc", @@ -2662,11 +2675,11 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.1.4" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a65ebfec4fb190b6f90e944a817d60499ee0744e582530e2c9900a22e591d9a" +checksum = "b03b7db8e0b4b2fdad6c551e634134e99ec000e5c8c3b6856c65e8bbaded7a3b" dependencies = [ - "crossterm", + "crossterm 0.29.0", "unicode-segmentation", "unicode-width 0.2.0", ] @@ -2685,6 +2698,26 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "compression-codecs" +version = "0.4.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef8a506ec4b81c460798f572caead636d57d3d7e940f998160f52bd254bf2d23" +dependencies = [ + "brotli", + "compression-core", + "flate2", + "memchr", + "zstd", + "zstd-safe", +] + +[[package]] +name = "compression-core" +version = "0.4.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e47641d3deaf41fb1538ac1f54735925e275eaf3bf4d55c81b137fba797e5cbb" + [[package]] name = "concat-kdf" version = "0.1.0" @@ -2705,15 +2738,14 @@ dependencies = [ [[package]] name = "const-hex" -version = "1.15.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dccd746bf9b1038c0507b7cec21eb2b11222db96a2902c96e8c185d6d20fb9c4" +checksum = "b6407bff74dea37e0fa3dc1c1c974e5d46405f0c987bf9997a0762adce71eda6" dependencies = [ "cfg-if", "cpufeatures", - "hex", "proptest", - "serde", + "serde_core", ] [[package]] @@ -2722,11 +2754,17 @@ version = "0.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" +[[package]] +name = "const-str" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2f421161cb492475f1661ddc9815a745a1c894592070661180fdec3d4872e9c3" + [[package]] name = "const_format" -version = "0.2.34" +version = "0.2.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "126f97965c8ad46d6d9163268ff28432e8f6a1196a55578867832e3049df63dd" +checksum = "7faa7469a93a566e9ccc1c73fe783b4a65c274c5ace346038dca9c39fe0030ad" dependencies = [ "const_format_proc_macros", ] @@ -2865,7 +2903,7 @@ version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "829d955a0bb380ef178a640b91779e3987da38c9aea133b20614cfed8cdea9c6" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "crossterm_winapi", "mio", "parking_lot", @@ -2875,6 +2913,20 @@ dependencies = [ "winapi", ] +[[package]] +name = "crossterm" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8b9f2e4c67f833b660cdb0a3523065869fb35570177239812ed4c905aeff87b" +dependencies = [ + "bitflags 2.9.4", + "crossterm_winapi", + "document-features", + "parking_lot", + "rustix 1.1.2", + "winapi", +] + [[package]] name = "crossterm_winapi" version = "0.9.1" @@ -3086,7 +3138,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8d162beedaa69905488a8da94f5ac3edb4dd4788b732fadb7bd120b2625c1976" dependencies = [ "data-encoding", - "syn 1.0.109", + "syn 2.0.106", ] [[package]] @@ -3100,7 +3152,7 @@ dependencies = [ "p256", "serde", "serde_json", - "sha2 0.10.9", + "sha2", "sha3", "time", "x509-parser 0.15.1", @@ -3164,12 +3216,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.4.0" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c9e6a11ca8224451684bc0d7d5a7adbf8f2fd6887261a1cfc3c0432f9d4068e" +checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" dependencies = [ "powerfmt", - "serde", + "serde_core", ] [[package]] @@ -3279,7 +3331,7 @@ version = "0.10.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ - "block-buffer 0.10.4", + "block-buffer", "const-oid", "crypto-common", "subtle", @@ -3313,7 +3365,7 @@ dependencies = [ "libc", "option-ext", "redox_users 0.5.2", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -3388,6 +3440,15 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aac81fa3e28d21450aa4d2ac065992ba96a1d7303efbce51a95f4fd175b67562" +[[package]] +name = "document-features" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95249b50c6c185bee49034bcb378a49dc2b5dff0be90ff6616d31d64febab05d" +dependencies = [ + "litrs", +] + [[package]] name = "dotenvy" version = "0.15.7" @@ -3462,7 +3523,7 @@ dependencies = [ "ed25519", "rand_core 0.6.4", "serde", - "sha2 0.10.9", + "sha2", "subtle", "zeroize", ] @@ -3604,12 +3665,12 @@ checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" +checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -3651,7 +3712,7 @@ checksum = "c853bd72c9e5787f8aafc3df2907c2ed03cff3150c3acd94e2e53a98ab70a8ab" dependencies = [ "cpufeatures", "ring", - "sha2 0.10.9", + "sha2", ] [[package]] @@ -3660,7 +3721,7 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3dc1355dbb41fbbd34ec28d4fb2a57d9a70c67ac3c19f6a5ca4d4a176b9e997a" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "hex", "serde", "serde_derive", @@ -3669,11 +3730,11 @@ dependencies = [ [[package]] name = "ethereum_ssz" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ca8ba45b63c389c6e115b095ca16381534fdcc03cf58176a3f8554db2dbe19b" +checksum = "0dcddb2554d19cde19b099fadddde576929d7a4d0c1cd3512d1fd95cf174375c" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "ethereum_serde_utils", "itertools 0.13.0", "serde", @@ -3684,9 +3745,9 @@ dependencies = [ [[package]] name = "ethereum_ssz_derive" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0dd55d08012b4e0dfcc92b8d6081234df65f2986ad34cc76eeed69c5e2ce7506" +checksum = "a657b6b3b7e153637dc6bdc6566ad9279d9ee11a15b12cfb24a2e04360637e9f" dependencies = [ "darling 0.20.11", "proc-macro2", @@ -3787,14 +3848,14 @@ checksum = "28dea519a9695b9977216879a3ebfddf92f1c08c05d984f8996aecd6ecdc811d" [[package]] name = "filetime" -version = "0.2.25" +version = "0.2.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35c0522e981e68cbfa8c3f978441a5f34b30b96e146b33cd3359176b50fe8586" +checksum = "bc0505cd1b6fa6580283f6bdf70a73fcf4aba1184038c90902b92b3dd0df63ed" dependencies = [ "cfg-if", "libc", "libredox", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -3811,9 +3872,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.1.2" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" +checksum = "dc5a4e564e38c699f2880d3fda590bedc2e69f3f84cd48b457bd892ce61d0aa9" dependencies = [ "crc32fast", "miniz_oxide", @@ -3831,6 +3892,12 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" +[[package]] +name = "foldhash" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77ce24cb58228fbb8aa041425bb1050850ac19177686ea6e0f41a70416f56fdb" + [[package]] name = "foreign-types" version = "0.3.2" @@ -3848,9 +3915,9 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "form_urlencoded" -version = "1.2.1" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +checksum = "cb4cb245038516f5f85277875cdaa4f7d2c9a0fa0468de06ed190163b1581fcf" dependencies = [ "percent-encoding", ] @@ -4013,20 +4080,6 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42012b0f064e01aa58b545fe3727f90f7dd4020f4a3ea735b50344965f5a57e9" -[[package]] -name = "generator" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d18470a76cb7f8ff746cf1f7470914f900252ec36bbc40b569d74b1258446827" -dependencies = [ - "cc", - "cfg-if", - "libc", - "log", - "rustversion", - "windows 0.61.3", -] - [[package]] name = "generic-array" version = "0.14.7" @@ -4062,7 +4115,7 @@ dependencies = [ "js-sys", "libc", "r-efi", - "wasi 0.14.2+wasi-0.2.4", + "wasi 0.14.7+wasi-0.2.4", "wasm-bindgen", ] @@ -4078,9 +4131,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.31.1" +version = "0.32.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" +checksum = "e629b9b98ef3dd8afe6ca2bd0f89306cec16d43d907889945bc5d6687f2f13c7" [[package]] name = "git2" @@ -4088,7 +4141,7 @@ version = "0.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2deb07a133b1520dc1a5690e9bd08950108873d7ed5de38dcc74d3b5ebffa110" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "libc", "libgit2-sys", "log", @@ -4149,12 +4202,12 @@ dependencies = [ [[package]] name = "gmp-mpfr-sys" -version = "1.6.5" +version = "1.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c66d61197a68f6323b9afa616cf83d55d69191e1bf364d4eb7d35ae18defe776" +checksum = "60f8970a75c006bb2f8ae79c6768a116dd215fa8346a87aed99bf9d82ca43394" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -4180,7 +4233,7 @@ dependencies = [ "futures-core", "futures-sink", "http", - "indexmap 2.10.0", + "indexmap 2.11.4", "slab", "tokio", "tokio-util", @@ -4222,7 +4275,17 @@ checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ "allocator-api2", "equivalent", - "foldhash", + "foldhash 0.1.5", + "serde", +] + +[[package]] +name = "hashbrown" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419bdc4f6a9207fbeba6d11b604d481addf78ecd10c11ad51e76c2f6482748d" +dependencies = [ + "foldhash 0.2.0", "serde", ] @@ -4262,9 +4325,6 @@ name = "hex" version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" -dependencies = [ - "serde", -] [[package]] name = "hex-conservative" @@ -4295,7 +4355,7 @@ dependencies = [ "ring", "serde", "socket2 0.5.10", - "thiserror 2.0.16", + "thiserror 2.0.17", "tinyvec", "tokio", "tracing", @@ -4319,7 +4379,7 @@ dependencies = [ "resolv-conf", "serde", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] @@ -4417,9 +4477,9 @@ checksum = "91f255a4535024abf7640cb288260811fc14794f62b063652ed349f9a6c2348e" [[package]] name = "humantime" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" +checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" [[package]] name = "humantime-serde" @@ -4485,7 +4545,7 @@ dependencies = [ "tokio", "tokio-rustls", "tower-service", - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] @@ -4519,9 +4579,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.16" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" +checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" dependencies = [ "base64 0.22.1", "bytes", @@ -4560,9 +4620,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.63" +version = "0.1.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" +checksum = "33e57f83510bb73707521ebaffa789ec8caf86f9657cad665b092b581d40e9fb" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -4570,7 +4630,7 @@ dependencies = [ "js-sys", "log", "wasm-bindgen", - "windows-core 0.61.2", + "windows-core 0.62.2", ] [[package]] @@ -4794,9 +4854,9 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "idna" -version = "1.0.3" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +checksum = "3b0875f23caa03898994f6ddc501886a45c7d3d62d04d2d90788d47be1b1e4de" dependencies = [ "idna_adapter", "smallvec", @@ -4935,14 +4995,15 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.10.0" +version = "2.11.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe4cd85333e22411419a0bcae1297d25e58c9443848b11dc6a86fefe8c78a661" +checksum = "4b0f83760fb341a774ed326568e19f5a863af4a952def8c39f9ab92fd95b88e5" dependencies = [ "arbitrary", "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", + "serde_core", ] [[package]] @@ -4957,7 +5018,7 @@ version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f37dccff2791ab604f9babef0ba14fbe0be30bd368dc541e2b08d07c8aa908f3" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "inotify-sys", "libc", ] @@ -5020,11 +5081,11 @@ dependencies = [ [[package]] name = "io-uring" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d93587f37623a1a17d94ef2bc9ada592f5465fe7732084ab7beefabe5c77c0c4" +checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "cfg-if", "libc", ] @@ -5135,9 +5196,9 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.33" +version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38f262f097c174adebe41eb73d66ae9c06b2844fb0da69969647bbddd9b0538a" +checksum = "9afb3de4395d6b3e67a780b6de64b51c978ecf11cb9a462c66be7d4ca9039d33" dependencies = [ "getrandom 0.3.3", "libc", @@ -5145,28 +5206,14 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.77" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cfaf33c695fc6e08064efbc1f72ec937429614f25eef83af942d0e227c3a28f" +checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" dependencies = [ "once_cell", "wasm-bindgen", ] -[[package]] -name = "jsonrpsee" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" -dependencies = [ - "jsonrpsee-core 0.25.1", - "jsonrpsee-http-client 0.25.1", - "jsonrpsee-proc-macros 0.25.1", - "jsonrpsee-server 0.25.1", - "jsonrpsee-types 0.25.1", - "tokio", - "tracing", -] - [[package]] name = "jsonrpsee" version = "0.26.0" @@ -5174,11 +5221,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f3f48dc3e6b8bd21e15436c1ddd0bc22a6a54e8ec46fedd6adf3425f396ec6a" dependencies = [ "jsonrpsee-client-transport", - "jsonrpsee-core 0.26.0", - "jsonrpsee-http-client 0.26.0", - "jsonrpsee-proc-macros 0.26.0", - "jsonrpsee-server 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-http-client", + "jsonrpsee-proc-macros", + "jsonrpsee-server", + "jsonrpsee-types", "jsonrpsee-wasm-client", "jsonrpsee-ws-client", "tokio", @@ -5196,13 +5243,13 @@ dependencies = [ "futures-util", "gloo-net", "http", - "jsonrpsee-core 0.26.0", + "jsonrpsee-core", "pin-project", "rustls", "rustls-pki-types", "rustls-platform-verifier", "soketto", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-rustls", "tokio-util", @@ -5210,30 +5257,6 @@ dependencies = [ "url", ] -[[package]] -name = "jsonrpsee-core" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" -dependencies = [ - "async-trait", - "bytes", - "futures-util", - "http", - "http-body", - "http-body-util", - "jsonrpsee-types 0.25.1", - "parking_lot", - "pin-project", - "rand 0.9.2", - "rustc-hash 2.1.1", - "serde", - "serde_json", - "thiserror 2.0.16", - "tokio", - "tower 0.5.2", - "tracing", -] - [[package]] name = "jsonrpsee-core" version = "0.26.0" @@ -5247,14 +5270,14 @@ dependencies = [ "http", "http-body", "http-body-util", - "jsonrpsee-types 0.26.0", + "jsonrpsee-types", "parking_lot", "pin-project", "rand 0.9.2", "rustc-hash 2.1.1", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tower 0.5.2", @@ -5262,28 +5285,6 @@ dependencies = [ "wasm-bindgen-futures", ] -[[package]] -name = "jsonrpsee-http-client" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" -dependencies = [ - "base64 0.22.1", - "http-body", - "hyper", - "hyper-rustls", - "hyper-util", - "jsonrpsee-core 0.25.1", - "jsonrpsee-types 0.25.1", - "rustls", - "rustls-platform-verifier", - "serde", - "serde_json", - "thiserror 2.0.16", - "tokio", - "tower 0.5.2", - "url", -] - [[package]] name = "jsonrpsee-http-client" version = "0.26.0" @@ -5295,30 +5296,18 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "rustls", "rustls-platform-verifier", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tower 0.5.2", "url", ] -[[package]] -name = "jsonrpsee-proc-macros" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" -dependencies = [ - "heck", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn 2.0.106", -] - [[package]] name = "jsonrpsee-proc-macros" version = "0.26.0" @@ -5334,8 +5323,9 @@ dependencies = [ [[package]] name = "jsonrpsee-server" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +version = "0.26.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c51b7c290bb68ce3af2d029648148403863b982f138484a73f02a9dd52dbd7f" dependencies = [ "futures-util", "http", @@ -5343,14 +5333,14 @@ dependencies = [ "http-body-util", "hyper", "hyper-util", - "jsonrpsee-core 0.25.1", - "jsonrpsee-types 0.25.1", + "jsonrpsee-core", + "jsonrpsee-types", "pin-project", "route-recognizer", "serde", "serde_json", "soketto", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -5359,53 +5349,15 @@ dependencies = [ ] [[package]] -name = "jsonrpsee-server" +name = "jsonrpsee-types" version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c51b7c290bb68ce3af2d029648148403863b982f138484a73f02a9dd52dbd7f" -dependencies = [ - "futures-util", - "http", - "http-body", - "http-body-util", - "hyper", - "hyper-util", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", - "pin-project", - "route-recognizer", - "serde", - "serde_json", - "soketto", - "thiserror 2.0.16", - "tokio", - "tokio-stream", - "tokio-util", - "tower 0.5.2", - "tracing", -] - -[[package]] -name = "jsonrpsee-types" -version = "0.25.1" -source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" -dependencies = [ - "http", - "serde", - "serde_json", - "thiserror 2.0.16", -] - -[[package]] -name = "jsonrpsee-types" -version = "0.26.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc88ff4688e43cc3fa9883a8a95c6fa27aa2e76c96e610b737b6554d650d7fd5" +checksum = "bc88ff4688e43cc3fa9883a8a95c6fa27aa2e76c96e610b737b6554d650d7fd5" dependencies = [ "http", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -5415,8 +5367,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7902885de4779f711a95d82c8da2d7e5f9f3a7c7cfa44d51c067fd1c29d72a3c" dependencies = [ "jsonrpsee-client-transport", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "tower 0.5.2", ] @@ -5428,8 +5380,8 @@ checksum = "9b6fceceeb05301cc4c065ab3bd2fa990d41ff4eb44e4ca1b30fa99c057c3e79" dependencies = [ "http", "jsonrpsee-client-transport", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "tower 0.5.2", "url", ] @@ -5460,7 +5412,7 @@ dependencies = [ "elliptic-curve", "once_cell", "serdect", - "sha2 0.10.9", + "sha2", "signature", ] @@ -5517,9 +5469,9 @@ checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" [[package]] name = "libc" -version = "0.2.175" +version = "0.2.177" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a82ae493e598baaea5209805c49bbf2ea7de956d50d7da0da1164f9c6d28543" +checksum = "2874a2af47a2325c2001a6e6fad9b16a53b802102b528163885171cf92b15976" [[package]] name = "libgit2-sys" @@ -5535,12 +5487,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" dependencies = [ "cfg-if", - "windows-targets 0.53.3", + "windows-link 0.2.1", ] [[package]] @@ -5580,7 +5532,7 @@ dependencies = [ "multiaddr", "pin-project", "rw-stream-sink", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -5614,7 +5566,7 @@ dependencies = [ "quick-protobuf-codec", "rand 0.8.5", "rand_core 0.6.4", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "web-time", ] @@ -5649,7 +5601,7 @@ dependencies = [ "quick-protobuf", "rand 0.8.5", "rw-stream-sink", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "unsigned-varint 0.8.0", "web-time", @@ -5688,7 +5640,7 @@ dependencies = [ "quick-protobuf", "quick-protobuf-codec", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -5706,8 +5658,8 @@ dependencies = [ "multihash", "quick-protobuf", "rand 0.8.5", - "sha2 0.10.9", - "thiserror 2.0.16", + "sha2", + "thiserror 2.0.17", "tracing", "zeroize", ] @@ -5765,7 +5717,7 @@ dependencies = [ "rand 0.8.5", "snow", "static_assertions", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "x25519-dalek", "zeroize", @@ -5804,7 +5756,7 @@ dependencies = [ "ring", "rustls", "socket2 0.5.10", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] @@ -5905,7 +5857,7 @@ dependencies = [ "ring", "rustls", "rustls-webpki", - "thiserror 2.0.16", + "thiserror 2.0.17", "x509-parser 0.17.0", "yasna", ] @@ -5934,78 +5886,32 @@ dependencies = [ "either", "futures", "libp2p-core", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "yamux 0.12.1", - "yamux 0.13.6", + "yamux 0.13.7", ] [[package]] name = "libproc" -version = "0.14.10" +version = "0.14.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78a09b56be5adbcad5aa1197371688dc6bb249a26da3bca2011ee2fb987ebfb" +checksum = "a54ad7278b8bc5301d5ffd2a94251c004feb971feba96c971ea4063645990757" dependencies = [ - "bindgen 0.70.1", + "bindgen 0.72.1", "errno", "libc", ] [[package]] name = "libredox" -version = "0.1.9" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" +checksum = "416f7e718bdb06000964960ffa43b4335ad4012ae8b99060261aa4a8088d5ccb" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "libc", - "redox_syscall 0.5.17", -] - -[[package]] -name = "libsecp256k1" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e79019718125edc905a079a70cfa5f3820bc76139fc91d6f9abc27ea2a887139" -dependencies = [ - "arrayref", - "base64 0.22.1", - "digest 0.9.0", - "libsecp256k1-core", - "libsecp256k1-gen-ecmult", - "libsecp256k1-gen-genmult", - "rand 0.8.5", - "serde", - "sha2 0.9.9", -] - -[[package]] -name = "libsecp256k1-core" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5be9b9bb642d8522a44d533eab56c16c738301965504753b03ad1de3425d5451" -dependencies = [ - "crunchy", - "digest 0.9.0", - "subtle", -] - -[[package]] -name = "libsecp256k1-gen-ecmult" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3038c808c55c87e8a172643a7d87187fc6c4174468159cb3090659d55bcb4809" -dependencies = [ - "libsecp256k1-core", -] - -[[package]] -name = "libsecp256k1-gen-genmult" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3db8d6ba2cec9eacc40e6e8ccc98931840301f1006e95647ceb2dd5c3aa06f7c" -dependencies = [ - "libsecp256k1-core", + "redox_syscall 0.5.18", ] [[package]] @@ -6044,9 +5950,9 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.9.4" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" +checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" @@ -6060,35 +5966,27 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +[[package]] +name = "litrs" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f5e54036fe321fd421e10d732f155734c4e4afd610dd556d9a82833ab3ee0bed" + [[package]] name = "lock_api" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" dependencies = [ - "autocfg", "scopeguard", "serde", ] [[package]] name = "log" -version = "0.4.27" +version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94" - -[[package]] -name = "loom" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "419e0dc8046cb947daa77eb95ae174acfbddb7673b4151f56d1eed8e93fbfaca" -dependencies = [ - "cfg-if", - "generator", - "scoped-tls", - "tracing", - "tracing-subscriber 0.3.20", -] +checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" [[package]] name = "lru" @@ -6141,9 +6039,9 @@ checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" [[package]] name = "mach2" -version = "0.4.3" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d640282b302c0bb0a2a8e0233ead9035e3bed871f0b7e81fe4a1ec829765db44" +checksum = "6a1b95cd5421ec55b445b5ae102f5ea0e768de1f82bd3001e11f426c269c3aea" dependencies = [ "libc", ] @@ -6169,6 +6067,17 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "match-lookup" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1265724d8cb29dbbc2b0f06fffb8bf1a8c0cf73a78eede9ba73a4a66c52a981e" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "matchers" version = "0.2.0" @@ -6202,15 +6111,15 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "memmap2" -version = "0.9.7" +version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "483758ad303d734cec05e5c12b41d7e93e6a6390c5e9dae6bdeb7c1259012d28" +checksum = "843a98750cd611cc2965a8213b53b43e715f13c37a9e096c6408e69990961db7" dependencies = [ "libc", ] @@ -6257,7 +6166,7 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "indexmap 2.10.0", + "indexmap 2.11.4", "ipnet", "metrics", "metrics-util 0.19.1", @@ -6278,30 +6187,30 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "indexmap 2.10.0", + "indexmap 2.11.4", "ipnet", "metrics", "metrics-util 0.20.0", "quanta", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "metrics-process" -version = "2.4.0" +version = "2.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a82c8add4382f29a122fa64fff1891453ed0f6b2867d971e7d60cb8dfa322ff" +checksum = "f615e08e049bd14a44c4425415782efb9bcd479fc1e19ddeb971509074c060d0" dependencies = [ "libc", "libproc", "mach2", "metrics", "once_cell", - "procfs", + "procfs 0.18.0", "rlimit", - "windows 0.58.0", + "windows 0.62.2", ] [[package]] @@ -6314,7 +6223,7 @@ dependencies = [ "crossbeam-epoch", "crossbeam-utils", "hashbrown 0.15.5", - "indexmap 2.10.0", + "indexmap 2.11.4", "metrics", "ordered-float", "quanta", @@ -6384,6 +6293,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" dependencies = [ "adler2", + "simd-adler32", ] [[package]] @@ -6421,23 +6331,22 @@ dependencies = [ [[package]] name = "moka" -version = "0.12.10" +version = "0.12.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9321642ca94a4282428e6ea4af8cc2ca4eac48ac7a6a4ea8f33f76d0ce70926" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" dependencies = [ "async-lock", "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", + "equivalent", "event-listener", "futures-util", - "loom", "parking_lot", "portable-atomic", "rustc_version 0.4.1", "smallvec", "tagptr", - "thiserror 1.0.69", "uuid", ] @@ -6468,11 +6377,12 @@ dependencies = [ [[package]] name = "multibase" -version = "0.9.1" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b3539ec3c1f04ac9748a260728e855f261b4977f5c3406612c884564f329404" +checksum = "8694bb4835f452b0e3bb06dbebb1d6fc5385b6ca1caf2e55fd165c042390ec77" dependencies = [ "base-x", + "base256emoji", "data-encoding", "data-encoding-macro", ] @@ -6575,7 +6485,7 @@ dependencies = [ "log", "netlink-packet-core", "netlink-sys", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -6633,7 +6543,7 @@ version = "8.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4d3d07927151ff8575b7087f245456e549fea62edf0ec4e565a5ee50c8402bc3" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "fsevent-sys", "inotify", "kqueue", @@ -6804,9 +6714,9 @@ dependencies = [ [[package]] name = "nybbles" -version = "0.4.3" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63cb50036b1ad148038105af40aaa70ff24d8a14fbc44ae5c914e1348533d12e" +checksum = "2c4b5ecbd0beec843101bffe848217f770e8b8da81d8355b7d6e226f2199b3dc" dependencies = [ "alloy-rlp", "arbitrary", @@ -6819,18 +6729,18 @@ dependencies = [ [[package]] name = "objc2-core-foundation" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c10c2894a6fed806ade6027bcd50662746363a9589d3ec9d9bef30a4e4bc166" +checksum = "2a180dd8642fa45cdb7dd721cd4c11b1cadd4929ce112ebd8b9f5803cc79d536" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", ] [[package]] name = "objc2-io-kit" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71c1c64d6120e51cd86033f67176b1cb66780c2efe34dec55176f77befd93c0a" +checksum = "33fafba39597d6dc1fb709123dfa8289d39406734be322956a69f0931c73bb15" dependencies = [ "libc", "objc2-core-foundation", @@ -6838,9 +6748,9 @@ dependencies = [ [[package]] name = "object" -version = "0.36.7" +version = "0.37.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" +checksum = "ff76201f031d8863c38aa7f905eca4f53abbfa15f609db4277d44cd8938f33fe" dependencies = [ "memchr", ] @@ -6897,7 +6807,7 @@ dependencies = [ "alloy-consensus", "alloy-eips", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", @@ -6905,7 +6815,7 @@ dependencies = [ "derive_more", "serde", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -6922,7 +6832,7 @@ checksum = "f80108e3b36901200a4c5df1db1ee9ef6ce685b59ea79d7be1713c845e3765da" dependencies = [ "alloy-consensus", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-provider", "alloy-rpc-types-eth", "alloy-signer", @@ -6936,8 +6846,8 @@ version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8eb878fc5ea95adb5abe55fb97475b3eb0dcc77dfcd6f61bd626a68ae0bdba1" dependencies = [ - "alloy-primitives 1.3.1", - "jsonrpsee 0.26.0", + "alloy-primitives 1.4.0", + "jsonrpsee", ] [[package]] @@ -6949,14 +6859,14 @@ dependencies = [ "alloy-consensus", "alloy-eips", "alloy-network-primitives", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-serde", "derive_more", "op-alloy-consensus", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -6967,7 +6877,7 @@ checksum = "14e50c94013a1d036a529df259151991dbbd6cf8dc215e3b68b784f95eec60e6" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-engine", "alloy-serde", @@ -6977,7 +6887,7 @@ dependencies = [ "op-alloy-consensus", "serde", "snap", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -6991,7 +6901,7 @@ dependencies = [ "alloy-json-rpc", "alloy-network", "alloy-op-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-provider", "alloy-rpc-client", "alloy-rpc-types-beacon", @@ -6999,7 +6909,7 @@ dependencies = [ "alloy-rpc-types-eth", "alloy-serde", "alloy-signer-local", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "alloy-transport", "alloy-transport-http", "anyhow", @@ -7019,9 +6929,9 @@ dependencies = [ "http-body-util", "hyper", "hyper-util", - "jsonrpsee 0.26.0", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee", + "jsonrpsee-core", + "jsonrpsee-types", "k256", "macros", "metrics", @@ -7115,9 +7025,9 @@ dependencies = [ [[package]] name = "op-revm" -version = "10.1.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9ba4f4693811e73449193c8bd656d3978f265871916882e6a51a487e4f96217" +checksum = "bb8cb3ecd7aaff3a18e15a155665b1e9b1979dff93d3ccf12161a85682237dbc" dependencies = [ "auto_impl", "revm", @@ -7136,7 +7046,7 @@ version = "0.10.73" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8505734d46c8ab1e19a1dce3aef597ad87dcb4c37e7188231769bd6bd51cebf8" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "cfg-if", "foreign-types", "libc", @@ -7184,7 +7094,7 @@ dependencies = [ "futures-sink", "js-sys", "pin-project-lite", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -7198,7 +7108,7 @@ dependencies = [ "futures-sink", "js-sys", "pin-project-lite", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -7246,7 +7156,7 @@ dependencies = [ "prost", "reqwest", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tonic", "tracing", @@ -7266,7 +7176,7 @@ dependencies = [ "opentelemetry_sdk 0.29.0", "prost", "reqwest", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -7318,7 +7228,7 @@ dependencies = [ "percent-encoding", "rand 0.8.5", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -7338,7 +7248,7 @@ dependencies = [ "percent-encoding", "rand 0.9.2", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -7366,7 +7276,7 @@ dependencies = [ "ecdsa", "elliptic-curve", "primeorder", - "sha2 0.10.9", + "sha2", ] [[package]] @@ -7436,9 +7346,9 @@ checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +checksum = "93857453250e3077bd71ff98b6a65ea6621a19bb0f559a85248955ac12c45a1a" dependencies = [ "lock_api", "parking_lot_core", @@ -7446,15 +7356,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.11" +version = "0.9.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.5.17", + "redox_syscall 0.5.18", "smallvec", - "windows-targets 0.52.6", + "windows-link 0.2.1", ] [[package]] @@ -7509,18 +7419,17 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.3.1" +version = "2.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" +checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" [[package]] name = "pest" -version = "2.8.1" +version = "2.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" +checksum = "989e7521a040efde50c3ab6bbadafbe15ab6dc042686926be59ac35d74607df4" dependencies = [ "memchr", - "thiserror 2.0.16", "ucd-trie", ] @@ -7540,8 +7449,18 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" dependencies = [ - "phf_macros", - "phf_shared", + "phf_macros 0.11.3", + "phf_shared 0.11.3", +] + +[[package]] +name = "phf" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1562dc717473dbaa4c1f85a36410e03c047b2e7df7f45ee938fbef64ae7fadf" +dependencies = [ + "phf_macros 0.13.1", + "phf_shared 0.13.1", "serde", ] @@ -7551,18 +7470,41 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" dependencies = [ - "phf_shared", + "phf_shared 0.11.3", "rand 0.8.5", ] +[[package]] +name = "phf_generator" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "135ace3a761e564ec88c03a77317a7c6b80bb7f7135ef2544dbe054243b89737" +dependencies = [ + "fastrand", + "phf_shared 0.13.1", +] + [[package]] name = "phf_macros" version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f84ac04429c13a7ff43785d75ad27569f2951ce0ffd30a3321230db2fc727216" dependencies = [ - "phf_generator", - "phf_shared", + "phf_generator 0.11.3", + "phf_shared 0.11.3", + "proc-macro2", + "quote", + "syn 2.0.106", +] + +[[package]] +name = "phf_macros" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "812f032b54b1e759ccd5f8b6677695d5268c588701effba24601f6932f8269ef" +dependencies = [ + "phf_generator 0.13.1", + "phf_shared 0.13.1", "proc-macro2", "quote", "syn 2.0.106", @@ -7577,6 +7519,15 @@ dependencies = [ "siphasher", ] +[[package]] +name = "phf_shared" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e57fef6bc5981e38c2ce2d63bfa546861309f875b8a75f092d1d54ae2d64f266" +dependencies = [ + "siphasher", +] + [[package]] name = "picky-asn1" version = "0.8.0" @@ -7679,8 +7630,8 @@ dependencies = [ "concurrent-queue", "hermit-abi", "pin-project-lite", - "rustix 1.0.8", - "windows-sys 0.61.0", + "rustix 1.1.2", + "windows-sys 0.61.2", ] [[package]] @@ -7720,9 +7671,9 @@ checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" [[package]] name = "potential_utf" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5a7c30837279ca13e7c867e9e40053bc68740f988cb07f7ca6df43cc734b585" +checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" dependencies = [ "zerovec 0.11.4", ] @@ -7784,11 +7735,11 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "3.3.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edce586971a4dfaa28950c6f18ed55e0406c1ab88bbce2c6f6293a7aaba73d35" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" dependencies = [ - "toml_edit", + "toml_edit 0.23.6", ] [[package]] @@ -7828,25 +7779,46 @@ version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cc5b72d8145275d844d4b5f6d4e1eef00c8cd889edb6035c21675d1bb1f45c9f" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "chrono", "flate2", "hex", - "procfs-core", + "procfs-core 0.17.0", "rustix 0.38.44", ] +[[package]] +name = "procfs" +version = "0.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25485360a54d6861439d60facef26de713b1e126bf015ec8f98239467a2b82f7" +dependencies = [ + "bitflags 2.9.4", + "procfs-core 0.18.0", + "rustix 1.1.2", +] + [[package]] name = "procfs-core" version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "239df02d8349b06fc07398a3a1697b06418223b1c7725085e801e7c0fc6a12ec" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "chrono", "hex", ] +[[package]] +name = "procfs-core" +version = "0.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6401bf7b6af22f78b563665d15a22e9aef27775b79b149a66ca022468a4e405" +dependencies = [ + "bitflags 2.9.4", + "hex", +] + [[package]] name = "prometheus-client" version = "0.23.1" @@ -7872,13 +7844,13 @@ dependencies = [ [[package]] name = "proptest" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fcdab19deb5195a31cf7726a210015ff1496ba1464fd42cb4f537b8b01b471f" +checksum = "2bb0be07becd10686a0bb407298fb425360a5c44a663774406340c59a22de4ce" dependencies = [ "bit-set", "bit-vec", - "bitflags 2.9.2", + "bitflags 2.9.4", "lazy_static", "num-traits", "rand 0.9.2", @@ -7911,6 +7883,17 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "proptest-derive" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "095a99f75c69734802359b682be8daaf8980296731f6470434ea2c652af1dd30" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "prost" version = "0.13.5" @@ -7940,7 +7923,7 @@ version = "0.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "57206b407293d2bcd3af849ce869d52068623f19e1b5ff8e8778e3309439682b" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "memchr", "unicase", ] @@ -7990,9 +7973,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.8" +version = "0.11.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626214629cda6781b6dc1d316ba307189c85ba657213ce642d9c77670f8202c8" +checksum = "b9e20a958963c291dc322d98411f541009df2ced7b5a4f2bd52337638cfccf20" dependencies = [ "bytes", "cfg_aliases", @@ -8002,8 +7985,8 @@ dependencies = [ "quinn-udp", "rustc-hash 2.1.1", "rustls", - "socket2 0.5.10", - "thiserror 2.0.16", + "socket2 0.6.0", + "thiserror 2.0.17", "tokio", "tracing", "web-time", @@ -8011,9 +7994,9 @@ dependencies = [ [[package]] name = "quinn-proto" -version = "0.11.12" +version = "0.11.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49df843a9161c85bb8aae55f101bc0bac8bcafd637a620d9122fd7e0b2f7422e" +checksum = "f1906b49b0c3bc04b5fe5d86a77925ae6524a19b816ae38ce1e426255f1d8a31" dependencies = [ "bytes", "getrandom 0.3.3", @@ -8024,7 +8007,7 @@ dependencies = [ "rustls", "rustls-pki-types", "slab", - "thiserror 2.0.16", + "thiserror 2.0.17", "tinyvec", "tracing", "web-time", @@ -8032,23 +8015,23 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.13" +version = "0.5.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcebb1209ee276352ef14ff8732e24cc2b02bbac986cd74a4c81bcb2f9881970" +checksum = "addec6a0dcad8a8d96a771f815f0eaf55f9d1805756410b39f5fa81332574cbd" dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.6.0", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] name = "quote" -version = "1.0.40" +version = "1.0.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" dependencies = [ "proc-macro2", ] @@ -8161,10 +8144,10 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eabd94c2f37801c20583fc49dd5cd6b0ba68c716787c2dd6ed18571e1e63117b" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "cassowary", "compact_str", - "crossterm", + "crossterm 0.28.1", "indoc", "instability", "itertools 0.13.0", @@ -8178,11 +8161,11 @@ dependencies = [ [[package]] name = "raw-cpuid" -version = "11.5.0" +version = "11.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6df7ab838ed27997ba19a4664507e6f82b41fe6e20be42929332156e5e85146" +checksum = "498cd0dc59d73224351ee52a95fee0f1a617a2eae0e7d9d720cc622c73a54186" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", ] [[package]] @@ -8235,11 +8218,11 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.17" +version = "0.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", ] [[package]] @@ -8261,23 +8244,23 @@ checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" dependencies = [ "getrandom 0.2.16", "libredox", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "ref-cast" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", @@ -8286,9 +8269,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.11.1" +version = "1.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" +checksum = "8b5288124840bee7b386bc413c487869b360b2b4ec421ea56425128692f2a82c" dependencies = [ "aho-corasick", "memchr", @@ -8298,9 +8281,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.9" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" +checksum = "833eb9ce86d40ef33cb1306d8accf7bc8ec2bfea4355cbdebb3df68b40925cad" dependencies = [ "aho-corasick", "memchr", @@ -8309,9 +8292,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" +checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" [[package]] name = "regress" @@ -8369,19 +8352,19 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] name = "resolv-conf" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95325155c684b1c89f7765e30bc1c42e4a6da51ca513615660cb8a62ef9a88e3" +checksum = "6b3789b30bd25ba102de4beabd95d21ac45b69b1be7d14522bab988c526d6799" [[package]] name = "reth" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-rpc-types", "aquamarine", @@ -8426,12 +8409,12 @@ dependencies = [ [[package]] name = "reth-basic-payload-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "futures-core", "futures-util", "metrics", @@ -8450,12 +8433,12 @@ dependencies = [ [[package]] name = "reth-chain-state" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-signer", "alloy-signer-local", "derive_more", @@ -8481,15 +8464,15 @@ dependencies = [ [[package]] name = "reth-chainspec" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-trie", "auto_impl", "derive_more", @@ -8501,8 +8484,8 @@ dependencies = [ [[package]] name = "reth-cli" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-genesis", "clap", @@ -8515,18 +8498,18 @@ dependencies = [ [[package]] name = "reth-cli-commands" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "backon", "clap", "comfy-table", - "crossterm", + "crossterm 0.28.1", "eyre", "fdlimit", "futures", @@ -8590,8 +8573,8 @@ dependencies = [ [[package]] name = "reth-cli-runner" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "reth-tasks", "tokio", @@ -8600,11 +8583,11 @@ dependencies = [ [[package]] name = "reth-cli-util" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "cfg-if", "eyre", "libc", @@ -8612,19 +8595,19 @@ dependencies = [ "reth-fs-util", "secp256k1 0.30.0", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tikv-jemallocator", ] [[package]] name = "reth-codecs" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-trie", "arbitrary", "bytes", @@ -8638,8 +8621,8 @@ dependencies = [ [[package]] name = "reth-codecs-derive" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "convert_case", "proc-macro2", @@ -8649,8 +8632,8 @@ dependencies = [ [[package]] name = "reth-config" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "eyre", "humantime-serde", @@ -8664,21 +8647,21 @@ dependencies = [ [[package]] name = "reth-consensus" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "reth-execution-types", "reth-primitives-traits", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-consensus-common" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8689,13 +8672,13 @@ dependencies = [ [[package]] name = "reth-consensus-debug-client" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-provider", "alloy-rpc-types-engine", "alloy-transport", @@ -8715,10 +8698,10 @@ dependencies = [ [[package]] name = "reth-db" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "derive_more", "eyre", "metrics", @@ -8736,17 +8719,17 @@ dependencies = [ "strum 0.27.2", "sysinfo 0.33.1", "tempfile", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-db-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "arbitrary", "bytes", "derive_more", @@ -8769,12 +8752,12 @@ dependencies = [ [[package]] name = "reth-db-common" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "boyer-moore-magiclen", "eyre", "reth-chainspec", @@ -8793,17 +8776,17 @@ dependencies = [ "reth-trie-db", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] [[package]] name = "reth-db-models" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "arbitrary", "bytes", "modular-bitfield", @@ -8814,10 +8797,10 @@ dependencies = [ [[package]] name = "reth-discv4" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "discv5", "enr", @@ -8832,7 +8815,7 @@ dependencies = [ "schnellru", "secp256k1 0.30.0", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -8840,10 +8823,10 @@ dependencies = [ [[package]] name = "reth-discv5" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "derive_more", "discv5", @@ -8857,17 +8840,17 @@ dependencies = [ "reth-metrics", "reth-network-peers", "secp256k1 0.30.0", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-dns-discovery" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "data-encoding", "enr", "hickory-resolver", @@ -8880,7 +8863,7 @@ dependencies = [ "secp256k1 0.30.0", "serde", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -8888,12 +8871,12 @@ dependencies = [ [[package]] name = "reth-downloaders" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "futures", "futures-util", @@ -8914,7 +8897,7 @@ dependencies = [ "reth-tasks", "reth-testing-utils", "tempfile", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -8923,11 +8906,11 @@ dependencies = [ [[package]] name = "reth-ecies" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "aes", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "block-padding", "byteorder", @@ -8942,9 +8925,9 @@ dependencies = [ "rand 0.8.5", "reth-network-peers", "secp256k1 0.30.0", - "sha2 0.10.9", + "sha2", "sha3", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -8954,11 +8937,11 @@ dependencies = [ [[package]] name = "reth-engine-local" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "eyre", "futures-util", @@ -8978,12 +8961,12 @@ dependencies = [ [[package]] name = "reth-engine-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "auto_impl", "futures", @@ -8997,14 +8980,14 @@ dependencies = [ "reth-primitives-traits", "reth-trie-common", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", ] [[package]] name = "reth-engine-service" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "futures", "pin-project", @@ -9021,18 +9004,18 @@ dependencies = [ "reth-prune", "reth-stages-api", "reth-tasks", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-engine-tree" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-engine", "derive_more", @@ -9073,15 +9056,15 @@ dependencies = [ "revm-primitives", "schnellru", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-engine-util" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", @@ -9108,42 +9091,42 @@ dependencies = [ [[package]] name = "reth-era" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "ethereum_ssz", "ethereum_ssz_derive", "reth-ethereum-primitives", "snap", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-era-downloader" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "bytes", "eyre", "futures-util", "reqwest", "reth-fs-util", - "sha2 0.10.9", + "sha2", "tokio", ] [[package]] name = "reth-era-utils" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "eyre", "futures-util", "reth-db-api", @@ -9161,22 +9144,22 @@ dependencies = [ [[package]] name = "reth-errors" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "reth-consensus", "reth-execution-errors", "reth-storage-errors", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-eth-wire" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-chains", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "bytes", "derive_more", @@ -9191,7 +9174,7 @@ dependencies = [ "reth-primitives-traits", "serde", "snap", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -9200,14 +9183,14 @@ dependencies = [ [[package]] name = "reth-eth-wire-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", "alloy-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "bytes", "derive_more", @@ -9216,13 +9199,13 @@ dependencies = [ "reth-ethereum-primitives", "reth-primitives-traits", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-ethereum-cli" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "clap", "eyre", @@ -9243,12 +9226,12 @@ dependencies = [ [[package]] name = "reth-ethereum-consensus" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "reth-chainspec", "reth-consensus", "reth-consensus-common", @@ -9259,11 +9242,11 @@ dependencies = [ [[package]] name = "reth-ethereum-engine-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-engine", "reth-engine-primitives", @@ -9271,18 +9254,18 @@ dependencies = [ "reth-payload-primitives", "reth-primitives-traits", "serde", - "sha2 0.10.9", - "thiserror 2.0.16", + "sha2", + "thiserror 2.0.17", ] [[package]] name = "reth-ethereum-forks" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eip2124", "alloy-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "once_cell", "rustc-hash 2.1.1", @@ -9290,12 +9273,12 @@ dependencies = [ [[package]] name = "reth-ethereum-payload-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-engine", "reth-basic-payload-builder", @@ -9319,12 +9302,12 @@ dependencies = [ [[package]] name = "reth-ethereum-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", @@ -9339,8 +9322,8 @@ dependencies = [ [[package]] name = "reth-etl" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "rayon", "reth-db-api", @@ -9349,13 +9332,13 @@ dependencies = [ [[package]] name = "reth-evm" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "derive_more", "futures-util", @@ -9372,13 +9355,13 @@ dependencies = [ [[package]] name = "reth-evm-ethereum" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "reth-chainspec", "reth-ethereum-forks", @@ -9392,26 +9375,26 @@ dependencies = [ [[package]] name = "reth-execution-errors" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "nybbles", "reth-storage-errors", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-execution-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "derive_more", "reth-ethereum-primitives", "reth-primitives-traits", @@ -9423,12 +9406,12 @@ dependencies = [ [[package]] name = "reth-exex" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "eyre", "futures", "itertools 0.14.0", @@ -9453,7 +9436,7 @@ dependencies = [ "reth-tasks", "reth-tracing", "rmp-serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-util", "tracing", @@ -9461,11 +9444,11 @@ dependencies = [ [[package]] name = "reth-exex-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "reth-chain-state", "reth-execution-types", "reth-primitives-traits", @@ -9475,26 +9458,26 @@ dependencies = [ [[package]] name = "reth-fs-util" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-invalid-block-hooks" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-debug", "eyre", "futures", - "jsonrpsee 0.26.0", + "jsonrpsee", "pretty_assertions", "reth-engine-primitives", "reth-evm", @@ -9512,17 +9495,17 @@ dependencies = [ [[package]] name = "reth-ipc" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "bytes", "futures", "futures-util", "interprocess", - "jsonrpsee 0.26.0", + "jsonrpsee", "pin-project", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -9532,33 +9515,33 @@ dependencies = [ [[package]] name = "reth-libmdbx" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "byteorder", "dashmap 6.1.0", "derive_more", "parking_lot", "reth-mdbx-sys", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] [[package]] name = "reth-mdbx-sys" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "bindgen 0.70.1", + "bindgen 0.71.1", "cc", ] [[package]] name = "reth-metrics" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "futures", "metrics", @@ -9569,34 +9552,34 @@ dependencies = [ [[package]] name = "reth-net-banlist" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", ] [[package]] name = "reth-net-nat" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "futures-util", "if-addrs 0.13.4", "reqwest", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-network" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "aquamarine", "auto_impl", @@ -9637,7 +9620,7 @@ dependencies = [ "secp256k1 0.30.0", "serde", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-util", @@ -9646,11 +9629,11 @@ dependencies = [ [[package]] name = "reth-network-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-admin", "alloy-rpc-types-eth", "auto_impl", @@ -9664,19 +9647,19 @@ dependencies = [ "reth-network-types", "reth-tokio-util", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", ] [[package]] name = "reth-network-p2p" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "auto_impl", "derive_more", "futures", @@ -9694,23 +9677,23 @@ dependencies = [ [[package]] name = "reth-network-peers" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "enr", "secp256k1 0.30.0", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "url", ] [[package]] name = "reth-network-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eip2124", "humantime-serde", @@ -9723,8 +9706,8 @@ dependencies = [ [[package]] name = "reth-nippy-jar" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "anyhow", "bincode", @@ -9733,15 +9716,15 @@ dependencies = [ "memmap2", "reth-fs-util", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "zstd", ] [[package]] name = "reth-node-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-rpc-types-engine", "eyre", @@ -9764,12 +9747,12 @@ dependencies = [ [[package]] name = "reth-node-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-provider", "alloy-rpc-types", "alloy-rpc-types-engine", @@ -9777,7 +9760,7 @@ dependencies = [ "eyre", "fdlimit", "futures", - "jsonrpsee 0.26.0", + "jsonrpsee", "rayon", "reth-basic-payload-builder", "reth-chain-state", @@ -9832,12 +9815,12 @@ dependencies = [ [[package]] name = "reth-node-core" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "clap", "derive_more", @@ -9874,7 +9857,7 @@ dependencies = [ "serde", "shellexpand", "strum 0.27.2", - "thiserror 2.0.16", + "thiserror 2.0.17", "toml", "tracing", "url", @@ -9884,8 +9867,8 @@ dependencies = [ [[package]] name = "reth-node-ethereum" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", "alloy-network", @@ -9922,11 +9905,11 @@ dependencies = [ [[package]] name = "reth-node-ethstats" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "chrono", "futures-util", "reth-chain-state", @@ -9936,7 +9919,7 @@ dependencies = [ "reth-transaction-pool", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-tungstenite", @@ -9946,12 +9929,12 @@ dependencies = [ [[package]] name = "reth-node-events" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "derive_more", "futures", @@ -9970,17 +9953,17 @@ dependencies = [ [[package]] name = "reth-node-metrics" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "eyre", "http", - "jsonrpsee-server 0.26.0", + "jsonrpsee-server", "metrics", "metrics-exporter-prometheus 0.16.2", "metrics-process", "metrics-util 0.19.1", - "procfs", + "procfs 0.17.0", "reth-metrics", "reth-tasks", "tikv-jemalloc-ctl", @@ -9991,8 +9974,8 @@ dependencies = [ [[package]] name = "reth-node-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "reth-chainspec", "reth-db-api", @@ -10003,15 +9986,15 @@ dependencies = [ [[package]] name = "reth-optimism-chainspec" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", "alloy-genesis", "alloy-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "derive_more", "miniz_oxide", "op-alloy-consensus", @@ -10026,17 +10009,17 @@ dependencies = [ "serde", "serde_json", "tar-no-std", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-optimism-cli" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "clap", "derive_more", @@ -10079,12 +10062,12 @@ dependencies = [ [[package]] name = "reth-optimism-consensus" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-trie", "reth-chainspec", "reth-consensus", @@ -10098,20 +10081,20 @@ dependencies = [ "reth-storage-errors", "reth-trie-common", "revm", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] [[package]] name = "reth-optimism-evm" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-op-evm", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "op-alloy-consensus", "op-alloy-rpc-types-engine", "op-revm", @@ -10127,17 +10110,17 @@ dependencies = [ "reth-rpc-eth-api", "reth-storage-errors", "revm", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-optimism-flashblocks" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "alloy-serde", "brotli", @@ -10167,22 +10150,22 @@ dependencies = [ [[package]] name = "reth-optimism-forks" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-op-hardforks", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "once_cell", "reth-ethereum-forks", ] [[package]] name = "reth-optimism-node" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "clap", @@ -10224,12 +10207,12 @@ dependencies = [ [[package]] name = "reth-optimism-payload-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-debug", "alloy-rpc-types-engine", @@ -10256,19 +10239,19 @@ dependencies = [ "reth-transaction-pool", "revm", "serde", - "sha2 0.10.9", - "thiserror 2.0.16", + "sha2", + "thiserror 2.0.17", "tracing", ] [[package]] name = "reth-optimism-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "arbitrary", "bytes", @@ -10283,13 +10266,13 @@ dependencies = [ [[package]] name = "reth-optimism-rpc" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-client", "alloy-rpc-types-debug", "alloy-rpc-types-engine", @@ -10300,9 +10283,9 @@ dependencies = [ "derive_more", "eyre", "futures", - "jsonrpsee 0.26.0", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee", + "jsonrpsee-core", + "jsonrpsee-types", "metrics", "op-alloy-consensus", "op-alloy-network", @@ -10336,7 +10319,7 @@ dependencies = [ "reth-transaction-pool", "revm", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tower 0.5.2", @@ -10345,8 +10328,8 @@ dependencies = [ [[package]] name = "reth-optimism-storage" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "reth-optimism-primitives", @@ -10355,13 +10338,13 @@ dependencies = [ [[package]] name = "reth-optimism-txpool" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-client", "alloy-rpc-types-eth", "alloy-serde", @@ -10384,18 +10367,18 @@ dependencies = [ "reth-storage-api", "reth-transaction-pool", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-payload-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types", "futures-util", "metrics", @@ -10412,8 +10395,8 @@ dependencies = [ [[package]] name = "reth-payload-builder-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "pin-project", "reth-payload-primitives", @@ -10424,11 +10407,11 @@ dependencies = [ [[package]] name = "reth-payload-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "auto_impl", "either", @@ -10438,24 +10421,24 @@ dependencies = [ "reth-errors", "reth-primitives-traits", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", ] [[package]] name = "reth-payload-util" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "reth-transaction-pool", ] [[package]] name = "reth-payload-validator" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", @@ -10464,8 +10447,8 @@ dependencies = [ [[package]] name = "reth-primitives" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "c-kzg", @@ -10478,13 +10461,13 @@ dependencies = [ [[package]] name = "reth-primitives-traits" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-eth", "alloy-trie", @@ -10506,17 +10489,17 @@ dependencies = [ "secp256k1 0.30.0", "serde", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-provider" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "dashmap 6.1.0", "eyre", @@ -10556,12 +10539,12 @@ dependencies = [ [[package]] name = "reth-prune" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "itertools 0.14.0", "metrics", "rayon", @@ -10577,32 +10560,32 @@ dependencies = [ "reth-static-file-types", "reth-tokio-util", "rustc-hash 2.1.1", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-prune-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "arbitrary", "derive_more", "modular-bitfield", "reth-codecs", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-ress-protocol" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "futures", "reth-eth-wire", @@ -10617,11 +10600,11 @@ dependencies = [ [[package]] name = "reth-ress-provider" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "eyre", "futures", "parking_lot", @@ -10644,10 +10627,10 @@ dependencies = [ [[package]] name = "reth-revm" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "reth-primitives-traits", "reth-storage-api", "reth-storage-errors", @@ -10657,8 +10640,8 @@ dependencies = [ [[package]] name = "reth-rpc" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10666,7 +10649,7 @@ dependencies = [ "alloy-evm", "alloy-genesis", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-client", "alloy-rpc-types", @@ -10689,8 +10672,8 @@ dependencies = [ "http-body", "hyper", "itertools 0.14.0", - "jsonrpsee 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee", + "jsonrpsee-types", "jsonwebtoken", "parking_lot", "pin-project", @@ -10725,8 +10708,8 @@ dependencies = [ "revm-primitives", "serde", "serde_json", - "sha2 0.10.9", - "thiserror 2.0.16", + "sha2", + "thiserror 2.0.17", "tokio", "tokio-stream", "tower 0.5.2", @@ -10736,13 +10719,13 @@ dependencies = [ [[package]] name = "reth-rpc-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", "alloy-genesis", "alloy-json-rpc", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types", "alloy-rpc-types-admin", "alloy-rpc-types-anvil", @@ -10754,7 +10737,7 @@ dependencies = [ "alloy-rpc-types-trace", "alloy-rpc-types-txpool", "alloy-serde", - "jsonrpsee 0.26.0", + "jsonrpsee", "reth-chain-state", "reth-engine-primitives", "reth-network-peers", @@ -10764,14 +10747,14 @@ dependencies = [ [[package]] name = "reth-rpc-builder" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-network", "alloy-provider", "dyn-clone", "http", - "jsonrpsee 0.26.0", + "jsonrpsee", "metrics", "pin-project", "reth-chain-state", @@ -10793,7 +10776,7 @@ dependencies = [ "reth-tasks", "reth-transaction-pool", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-util", "tower 0.5.2", @@ -10803,18 +10786,18 @@ dependencies = [ [[package]] name = "reth-rpc-convert" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-json-rpc", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-signer", "auto_impl", "dyn-clone", - "jsonrpsee-types 0.26.0", + "jsonrpsee-types", "op-alloy-consensus", "op-alloy-network", "op-alloy-rpc-types", @@ -10825,20 +10808,20 @@ dependencies = [ "reth-primitives-traits", "reth-storage-api", "revm-context", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-rpc-engine-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "async-trait", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "metrics", "parking_lot", "reth-chainspec", @@ -10853,15 +10836,15 @@ dependencies = [ "reth-tasks", "reth-transaction-pool", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-rpc-eth-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10869,7 +10852,7 @@ dependencies = [ "alloy-evm", "alloy-json-rpc", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-eth", "alloy-rpc-types-mev", @@ -10878,8 +10861,8 @@ dependencies = [ "auto_impl", "dyn-clone", "futures", - "jsonrpsee 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee", + "jsonrpsee-types", "parking_lot", "reth-chain-state", "reth-chainspec", @@ -10904,23 +10887,23 @@ dependencies = [ [[package]] name = "reth-rpc-eth-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-network", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-client", "alloy-rpc-types-eth", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "alloy-transport", "derive_more", "futures", "itertools 0.14.0", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "metrics", "rand 0.9.2", "reqwest", @@ -10943,7 +10926,7 @@ dependencies = [ "revm-inspectors", "schnellru", "serde", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -10951,12 +10934,12 @@ dependencies = [ [[package]] name = "reth-rpc-layer" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-rpc-types-engine", "http", - "jsonrpsee-http-client 0.26.0", + "jsonrpsee-http-client", "pin-project", "tower 0.5.2", "tower-http", @@ -10965,14 +10948,14 @@ dependencies = [ [[package]] name = "reth-rpc-server-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", - "jsonrpsee-core 0.26.0", - "jsonrpsee-types 0.26.0", + "jsonrpsee-core", + "jsonrpsee-types", "reth-errors", "reth-network-api", "serde", @@ -10981,12 +10964,12 @@ dependencies = [ [[package]] name = "reth-stages" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "bincode", "eyre", "futures-util", @@ -11022,18 +11005,18 @@ dependencies = [ "reth-trie", "reth-trie-db", "tempfile", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-stages-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "aquamarine", "auto_impl", "futures-util", @@ -11049,17 +11032,17 @@ dependencies = [ "reth-static-file", "reth-static-file-types", "reth-tokio-util", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-stages-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "arbitrary", "bytes", "modular-bitfield", @@ -11070,10 +11053,10 @@ dependencies = [ [[package]] name = "reth-static-file" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "parking_lot", "rayon", "reth-codecs", @@ -11090,10 +11073,10 @@ dependencies = [ [[package]] name = "reth-static-file-types" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "clap", "derive_more", "serde", @@ -11102,12 +11085,12 @@ dependencies = [ [[package]] name = "reth-storage-api" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-engine", "auto_impl", "reth-chainspec", @@ -11125,24 +11108,24 @@ dependencies = [ [[package]] name = "reth-storage-errors" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "derive_more", "reth-primitives-traits", "reth-prune-types", "reth-static-file-types", "revm-database-interface", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "reth-tasks" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "auto_impl", "dyn-clone", @@ -11151,7 +11134,7 @@ dependencies = [ "pin-project", "rayon", "reth-metrics", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", "tracing-futures", @@ -11159,13 +11142,13 @@ dependencies = [ [[package]] name = "reth-testing-utils" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "rand 0.8.5", "rand 0.9.2", "reth-ethereum-primitives", @@ -11175,8 +11158,8 @@ dependencies = [ [[package]] name = "reth-tokio-util" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "tokio", "tokio-stream", @@ -11185,8 +11168,8 @@ dependencies = [ [[package]] name = "reth-tracing" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "clap", "eyre", @@ -11200,8 +11183,8 @@ dependencies = [ [[package]] name = "reth-tracing-otlp" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "opentelemetry 0.29.1", "opentelemetry-otlp 0.29.0", @@ -11214,16 +11197,16 @@ dependencies = [ [[package]] name = "reth-transaction-pool" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "aquamarine", "auto_impl", - "bitflags 2.9.2", + "bitflags 2.9.4", "futures-util", "metrics", "parking_lot", @@ -11247,7 +11230,7 @@ dependencies = [ "serde", "serde_json", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -11255,12 +11238,12 @@ dependencies = [ [[package]] name = "reth-trie" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-trie", "auto_impl", @@ -11280,11 +11263,11 @@ dependencies = [ [[package]] name = "reth-trie-common" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "alloy-consensus", - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", @@ -11306,10 +11289,10 @@ dependencies = [ [[package]] name = "reth-trie-db" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "reth-db-api", "reth-execution-errors", "reth-primitives-traits", @@ -11319,10 +11302,10 @@ dependencies = [ [[package]] name = "reth-trie-parallel" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "derive_more", "itertools 0.14.0", @@ -11337,17 +11320,17 @@ dependencies = [ "reth-trie-common", "reth-trie-db", "reth-trie-sparse", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", ] [[package]] name = "reth-trie-sparse" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-trie", "auto_impl", @@ -11363,10 +11346,10 @@ dependencies = [ [[package]] name = "reth-trie-sparse-parallel" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rlp", "alloy-trie", "metrics", @@ -11381,17 +11364,17 @@ dependencies = [ [[package]] name = "reth-zstd-compressors" -version = "1.8.1" -source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +version = "1.8.2" +source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" dependencies = [ "zstd", ] [[package]] name = "revm" -version = "29.0.1" +version = "30.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "718d90dce5f07e115d0e66450b1b8aa29694c1cf3f89ebddaddccc2ccbd2f13e" +checksum = "cafc7f271a0ff9b5bc52f4ee5c2ba0ea9e5a491919e9ed0ea4b80b8d87487e79" dependencies = [ "revm-bytecode", "revm-context", @@ -11408,21 +11391,21 @@ dependencies = [ [[package]] name = "revm-bytecode" -version = "6.2.2" +version = "7.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66c52031b73cae95d84cd1b07725808b5fd1500da3e5e24574a3b2dc13d9f16d" +checksum = "bb7c72e495ec8732ac35b9d0b99b721f9099d261961dc9d230043af22c08f918" dependencies = [ "bitvec", - "phf", + "phf 0.13.1", "revm-primitives", "serde", ] [[package]] name = "revm-context" -version = "9.1.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a20c98e7008591a6f012550c2a00aa36cba8c14cc88eb88dec32eb9102554b4" +checksum = "a9453598455c34d8a3665d62ed4e4f578f03823194075d6fb713e4621a6494ad" dependencies = [ "bitvec", "cfg-if", @@ -11437,9 +11420,9 @@ dependencies = [ [[package]] name = "revm-context-interface" -version = "10.2.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b50d241ed1ce647b94caf174fcd0239b7651318b2c4c06b825b59b973dfb8495" +checksum = "4e340a0ae6cd19b90231a88c45608b2ed9851a7dd4fdb8ecf4e5e3e5ca7b6ed4" dependencies = [ "alloy-eip2930", "alloy-eip7702", @@ -11453,9 +11436,9 @@ dependencies = [ [[package]] name = "revm-database" -version = "7.0.5" +version = "8.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39a276ed142b4718dcf64bc9624f474373ed82ef20611025045c3fb23edbef9c" +checksum = "ef14e9bfb8ce40515fb0c7e1bea5f941136ab850ae2b761835fc6790d2e99348" dependencies = [ "alloy-eips", "revm-bytecode", @@ -11467,9 +11450,9 @@ dependencies = [ [[package]] name = "revm-database-interface" -version = "7.0.5" +version = "8.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c523c77e74eeedbac5d6f7c092e3851dbe9c7fec6f418b85992bd79229db361" +checksum = "330203b2f1d4de6ccd8bd6493158ff171d8ac88cce82cf2e4d60554b63e6923a" dependencies = [ "auto_impl", "either", @@ -11480,9 +11463,9 @@ dependencies = [ [[package]] name = "revm-handler" -version = "10.0.1" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "550331ea85c1d257686e672081576172fe3d5a10526248b663bbf54f1bef226a" +checksum = "b02883e8ea5d1c35e25ea04ddbcdf048ebdab4d329223d7798c5676c575b3b68" dependencies = [ "auto_impl", "derive-where", @@ -11499,9 +11482,9 @@ dependencies = [ [[package]] name = "revm-inspector" -version = "10.0.1" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c0a6e9ccc2ae006f5bed8bd80cd6f8d3832cd55c5e861b9402fdd556098512f" +checksum = "4dd956c37c3fb40843d6ae78092fd5cf211a17812e9dd0176663f4dc3c37cf68" dependencies = [ "auto_impl", "either", @@ -11518,13 +11501,12 @@ dependencies = [ [[package]] name = "revm-inspectors" version = "0.30.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9b329afcc0f9fd5adfa2c6349a7435a8558e82bcae203142103a9a95e2a63b6" +source = "git+https://github.com/noot/revm-inspectors?rev=1af65e18317c33a37492689c5b11c6ddbfe3bc51#1af65e18317c33a37492689c5b11c6ddbfe3bc51" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "alloy-rpc-types-eth", "alloy-rpc-types-trace", - "alloy-sol-types 1.3.1", + "alloy-sol-types 1.4.0", "anstyle", "boa_engine", "boa_gc", @@ -11532,26 +11514,27 @@ dependencies = [ "revm", "serde", "serde_json", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "revm-interpreter" -version = "25.0.3" +version = "26.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06575dc51b1d8f5091daa12a435733a90b4a132dca7ccee0666c7db3851bc30c" +checksum = "f3600bda3a01929495d6c3a36a713de2e3c4aac9e001dcd3f6211213b5c30205" dependencies = [ "revm-bytecode", "revm-context-interface", "revm-primitives", + "revm-state", "serde", ] [[package]] name = "revm-precompile" -version = "27.0.0" +version = "28.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25b57d4bd9e6b5fe469da5452a8a137bc2d030a3cd47c46908efc615bbc699da" +checksum = "73ba7917ca5dae8de3265adcb2a305dce4ca33b68e601aa06b281000d5ae9947" dependencies = [ "ark-bls12-381", "ark-bn254", @@ -11564,22 +11547,21 @@ dependencies = [ "c-kzg", "cfg-if", "k256", - "libsecp256k1", "p256", "revm-primitives", "ripemd", "rug", "secp256k1 0.31.1", - "sha2 0.10.9", + "sha2", ] [[package]] name = "revm-primitives" -version = "20.2.1" +version = "21.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aa29d9da06fe03b249b6419b33968ecdf92ad6428e2f012dc57bcd619b5d94e" +checksum = "38271b8b85f00154bdcf9f2ab0a3ec7a8100377d2c7a0d8eb23e19389b42c795" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "num_enum", "once_cell", "serde", @@ -11587,11 +11569,11 @@ dependencies = [ [[package]] name = "revm-state" -version = "7.0.5" +version = "8.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f64fbacb86008394aaebd3454f9643b7d5a782bd251135e17c5b33da592d84d" +checksum = "264baea816b732f7b49e7370eae52ac5f45f827e3b7c8494a2b2aa38e3a8cecc" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "revm-bytecode", "revm-primitives", "serde", @@ -11699,12 +11681,16 @@ dependencies = [ [[package]] name = "rollup-boost" version = "0.1.0" -source = "git+http://github.com/flashbots/rollup-boost?rev=b86af43969557bee18f17ec1d6bcd3e984f910b2#b86af43969557bee18f17ec1d6bcd3e984f910b2" +source = "git+http://github.com/flashbots/rollup-boost?rev=8cabb2c2056fddb55b39369b404d21989eab8bb7#8cabb2c2056fddb55b39369b404d21989eab8bb7" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-json-rpc", + "alloy-primitives 1.4.0", + "alloy-rpc-client", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-serde", + "alloy-transport", + "alloy-transport-http", "clap", "dashmap 6.1.0", "dotenvy", @@ -11715,7 +11701,11 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "jsonrpsee 0.25.1", + "jsonrpsee", + "jsonrpsee-core", + "jsonrpsee-http-client", + "jsonrpsee-server", + "jsonrpsee-types", "metrics", "metrics-derive", "metrics-exporter-prometheus 0.16.2", @@ -11728,12 +11718,13 @@ dependencies = [ "parking_lot", "paste", "reth-optimism-payload-builder", + "reth-rpc-eth-types", "rustls", "serde", "serde_json", - "sha2 0.10.9", + "sha2", "testcontainers 0.23.3", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-tungstenite", "tokio-util", @@ -11773,9 +11764,9 @@ dependencies = [ [[package]] name = "rug" -version = "1.27.0" +version = "1.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4207e8d668e5b8eb574bda8322088ccd0d7782d3d03c7e8d562e82ed82bdcbc3" +checksum = "58ad2e973fe3c3214251a840a621812a4f40468da814b1a3d6947d433c2af11f" dependencies = [ "az", "gmp-mpfr-sys", @@ -11785,14 +11776,15 @@ dependencies = [ [[package]] name = "ruint" -version = "1.16.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ecb38f82477f20c5c3d62ef52d7c4e536e38ea9b73fb570a20c5cae0e14bcf6" +checksum = "a68df0380e5c9d20ce49534f292a36a7514ae21350726efe1865bdb1fa91d278" dependencies = [ "alloy-rlp", "arbitrary", "ark-ff 0.3.0", "ark-ff 0.4.2", + "ark-ff 0.5.0", "bytes", "fastrlp 0.3.1", "fastrlp 0.4.0", @@ -11806,7 +11798,7 @@ dependencies = [ "rand 0.9.2", "rlp", "ruint-macro", - "serde", + "serde_core", "valuable", "zeroize", ] @@ -11859,7 +11851,7 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" dependencies = [ - "semver 1.0.26", + "semver 1.0.27", ] [[package]] @@ -11877,7 +11869,7 @@ version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "errno", "libc", "linux-raw-sys 0.4.15", @@ -11886,15 +11878,15 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.8" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" +checksum = "cd15f8a2c5551a84d56efdc1cd049089e409ac19a3072d5037a17fd70719ff3e" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "errno", "libc", - "linux-raw-sys 0.9.4", - "windows-sys 0.60.2", + "linux-raw-sys 0.11.0", + "windows-sys 0.61.2", ] [[package]] @@ -11922,7 +11914,7 @@ dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.3.0", + "security-framework 3.5.1", ] [[package]] @@ -11959,7 +11951,7 @@ dependencies = [ "rustls-native-certs", "rustls-platform-verifier-android", "rustls-webpki", - "security-framework 3.3.0", + "security-framework 3.5.1", "security-framework-sys", "webpki-root-certs 0.26.11", "windows-sys 0.59.0", @@ -11991,9 +11983,9 @@ checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" [[package]] name = "rusty-fork" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb3dcc6e454c328bb824492db107ab7c0ae8fcffe4ad210136ef014458c1bc4f" +checksum = "cc6bf79ff24e648f6da1f8d1f011e9cac26491b619e6b9280f2b47f1774e6ee2" dependencies = [ "fnv", "quick-error", @@ -12035,11 +12027,11 @@ dependencies = [ [[package]] name = "schannel" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f29ebaa345f945cec9fbbc532eb307f0fdad8161f281b6369539c8d84876b3d" +checksum = "891d81b926048e76efe18581bf793546b4c0eaf8448d72be8de2bbee5fd166e1" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -12077,12 +12069,6 @@ dependencies = [ "hashbrown 0.13.2", ] -[[package]] -name = "scoped-tls" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" - [[package]] name = "scopeguard" version = "1.2.0" @@ -12151,7 +12137,7 @@ version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "core-foundation 0.9.4", "core-foundation-sys", "libc", @@ -12160,11 +12146,11 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.3.0" +version = "3.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80fb1d92c5028aa318b4b8bd7302a5bfcf48be96a37fc6fc790f806b0004ee0c" +checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "core-foundation 0.10.1", "core-foundation-sys", "libc", @@ -12173,9 +12159,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.14.0" +version = "2.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49db231d56a190491cb4aeda9527f1ad45345af50b0851622a7adb8c03b01c32" +checksum = "cc1f0cbffaac4852523ce30d8bd3c5cdc873501d96ff467ca09b6767bb8cd5c0" dependencies = [ "core-foundation-sys", "libc", @@ -12192,11 +12178,12 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.26" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" +checksum = "d767eb0aabc880b29956c35734170f26ed551a859dbd361d140cdbeca61ab1e2" dependencies = [ "serde", + "serde_core", ] [[package]] @@ -12222,9 +12209,9 @@ checksum = "cd0b0ec5f1c1ca621c432a25813d8d60c88abe6d3e08a3eb9cf37d97a0fe3d73" [[package]] name = "serde" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0dca6411025b24b60bfa7ec1fe1f8e710ac09782dca409ee8237ba74b51295fd" +checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" dependencies = [ "serde_core", "serde_derive", @@ -12241,27 +12228,28 @@ dependencies = [ [[package]] name = "serde_bytes" -version = "0.11.17" +version = "0.11.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8437fd221bde2d4ca316d61b90e337e9e702b3820b87d63caa9ba6c02bd06d96" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" dependencies = [ "serde", + "serde_core", ] [[package]] name = "serde_core" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba2ba63999edb9dac981fb34b3e5c0d111a69b0924e253ed29d83f7c99e966a4" +checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8db53ae22f34573731bafa1db20f04027b2d25e02d8205921b569171699cdb33" +checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", @@ -12270,25 +12258,27 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.143" +version = "1.0.145" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d401abef1d108fbd9cbaebc3e46611f4b1021f714a0597a71f41ee463f5f4a5a" +checksum = "402a6f66d8c709116cf22f558eab210f5a50187f702eb4d7e5ef38d9a7f1c79c" dependencies = [ - "indexmap 2.10.0", + "indexmap 2.11.4", "itoa", "memchr", "ryu", "serde", + "serde_core", ] [[package]] name = "serde_path_to_error" -version = "0.1.17" +version = "0.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fab13f937fa393d08645bf3a84bdfe86e296747b506ada67bb15f10f218b2a" +checksum = "10a9ff822e371bb5403e391ecd83e182e0e77ba7f6fe0160b795797109d1b457" dependencies = [ "itoa", "serde", + "serde_core", ] [[package]] @@ -12325,19 +12315,18 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.14.0" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2c45cd61fefa9db6f254525d46e392b852e0e61d9a1fd36e5bd183450a556d5" +checksum = "6093cd8c01b25262b84927e0f7151692158fab02d961e04c979d3903eba7ecc5" dependencies = [ "base64 0.22.1", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.10.0", + "indexmap 2.11.4", "schemars 0.9.0", "schemars 1.0.4", - "serde", - "serde_derive", + "serde_core", "serde_json", "serde_with_macros", "time", @@ -12345,11 +12334,11 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.14.0" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de90945e6565ce0d9a25098082ed4ee4002e047cb59892c318d66821e14bb30f" +checksum = "a7e6c180db0816026a61afa1cff5344fb7ebded7e4d3062772179f2501481c27" dependencies = [ - "darling 0.20.11", + "darling 0.21.3", "proc-macro2", "quote", "syn 2.0.106", @@ -12361,7 +12350,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.10.0", + "indexmap 2.11.4", "itoa", "ryu", "serde", @@ -12395,19 +12384,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bbfa15b3dddfee50a0fff136974b3e1bde555604ba463834a7eb7deb6417705d" -[[package]] -name = "sha2" -version = "0.9.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d58a1e1bf39749807d89cf2d98ac2dfa0ff1cb3faa38fbb64dd88ac8013d800" -dependencies = [ - "block-buffer 0.9.0", - "cfg-if", - "cpufeatures", - "digest 0.9.0", - "opaque-debug", -] - [[package]] name = "sha2" version = "0.10.9" @@ -12503,6 +12479,12 @@ dependencies = [ "rand_core 0.6.4", ] +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + [[package]] name = "simple_asn1" version = "0.6.3" @@ -12511,7 +12493,7 @@ checksum = "297f631f50729c8c99b84667867963997ec0b50f32b2a7dbcab828ef0541e8bb" dependencies = [ "num-bigint", "num-traits", - "thiserror 2.0.16", + "thiserror 2.0.17", "time", ] @@ -12577,7 +12559,7 @@ dependencies = [ "rand_core 0.6.4", "ring", "rustc_version 0.4.1", - "sha2 0.10.9", + "sha2", "subtle", ] @@ -12635,9 +12617,9 @@ checksum = "3b9b39299b249ad65f3b7e96443bad61c02ca5cd3589f46cb6d610a0fd6c0d6a" [[package]] name = "stable_deref_trait" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" [[package]] name = "static_assertions" @@ -12759,9 +12741,9 @@ dependencies = [ [[package]] name = "syn-solidity" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0b198d366dbec045acfcd97295eb653a7a2b40e4dc764ef1e79aafcad439d3c" +checksum = "2375c17f6067adc651d8c2c51658019cef32edfff4a982adaf1d7fd1c039f08b" dependencies = [ "paste", "proc-macro2", @@ -12834,7 +12816,7 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c879d448e9d986b661742763247d3693ed13609438cf3d006f51f5368a5ba6b" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "core-foundation 0.9.4", "system-configuration-sys", ] @@ -12878,7 +12860,7 @@ version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ac9ee8b664c9f1740cd813fea422116f8ba29997bb7c878d1940424889802897" dependencies = [ - "bitflags 2.9.2", + "bitflags 2.9.4", "log", "num-traits", ] @@ -12914,7 +12896,7 @@ dependencies = [ name = "tdx-quote-provider" version = "0.1.0" dependencies = [ - "axum 0.8.4", + "axum 0.8.6", "clap", "dotenvy", "eyre", @@ -12934,15 +12916,15 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.21.0" +version = "3.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15b61f8f20e3a6f7e0649d825294eaf317edce30f82cf6026e7e4cb9222a7d1e" +checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" dependencies = [ "fastrand", "getrandom 0.3.3", "once_cell", - "rustix 1.0.8", - "windows-sys 0.60.2", + "rustix 1.1.2", + "windows-sys 0.61.2", ] [[package]] @@ -12966,7 +12948,7 @@ dependencies = [ "serde", "serde_json", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-tar", @@ -12995,7 +12977,7 @@ dependencies = [ "serde", "serde_json", "serde_with", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tokio-tar", @@ -13020,11 +13002,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.16" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3467d614147380f2e4e374161426ff399c91084acd2363eaf549172b3d5e60c0" +checksum = "f63587ca0f12b72a0600bcba1d40081f830876000bb46dd2337a3051618f4fc8" dependencies = [ - "thiserror-impl 2.0.16", + "thiserror-impl 2.0.17", ] [[package]] @@ -13040,9 +13022,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -version = "2.0.16" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5e1be1c48b9172ee610da68fd9cd2770e7a4056cb3fc98710ee6906f0c7960" +checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", @@ -13100,9 +13082,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.41" +version = "0.3.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a7619e19bc266e0f9c5e6686659d394bc57973859340060a69221e57dbc0c40" +checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", "itoa", @@ -13118,15 +13100,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.4" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e9a38711f559d9e3ce1cdb06dd7c5b8ea546bc90052da6d06bb76da74bb07c" +checksum = "40868e7c1d2f0b8d73e4a8c7f0ff63af4f6d19be117e90bd73eb1d62cf831c6b" [[package]] name = "time-macros" -version = "0.2.22" +version = "0.2.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3526739392ec93fd8b359c8e98514cb3e8e021beb4e5f597b00a0221f8ed8a49" +checksum = "30cfb0125f12d9c277f35663a0a33f8c30190f4e4574868a330595412d34ebf3" dependencies = [ "num-conv", "time-core", @@ -13219,9 +13201,9 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.2" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ "rustls", "tokio", @@ -13296,8 +13278,8 @@ checksum = "dc1beb996b9d83529a9e75c17a1686767d148d70663143c7854d8b4a09ced362" dependencies = [ "serde", "serde_spanned", - "toml_datetime", - "toml_edit", + "toml_datetime 0.6.11", + "toml_edit 0.22.27", ] [[package]] @@ -13309,20 +13291,50 @@ dependencies = [ "serde", ] +[[package]] +name = "toml_datetime" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32f1085dec27c2b6632b04c80b3bb1b4300d6495d1e129693bdda7d91e72eec1" +dependencies = [ + "serde_core", +] + [[package]] name = "toml_edit" version = "0.22.27" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "41fe8c660ae4257887cf66394862d21dbca4a6ddd26f04a3560410406a2f819a" dependencies = [ - "indexmap 2.10.0", + "indexmap 2.11.4", "serde", "serde_spanned", - "toml_datetime", + "toml_datetime 0.6.11", "toml_write", "winnow", ] +[[package]] +name = "toml_edit" +version = "0.23.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3effe7c0e86fdff4f69cdd2ccc1b96f933e24811c5441d44904e8683e27184b" +dependencies = [ + "indexmap 2.11.4", + "toml_datetime 0.7.2", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cf893c33be71572e0e9aa6dd15e6677937abd686b066eac3f8cd3531688a627" +dependencies = [ + "winnow", +] + [[package]] name = "toml_write" version = "0.1.2" @@ -13388,7 +13400,7 @@ dependencies = [ "futures-core", "futures-util", "hdrhistogram", - "indexmap 2.10.0", + "indexmap 2.11.4", "pin-project-lite", "slab", "sync_wrapper", @@ -13407,7 +13419,7 @@ checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" dependencies = [ "async-compression", "base64 0.22.1", - "bitflags 2.9.2", + "bitflags 2.9.4", "bytes", "futures-core", "futures-util", @@ -13613,7 +13625,7 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ee44f4cef85f88b4dea21c0b1f58320bdf35715cf56d840969487cff00613321" dependencies = [ - "alloy-primitives 1.3.1", + "alloy-primitives 1.4.0", "ethereum_hashing", "ethereum_ssz", "smallvec", @@ -13644,9 +13656,9 @@ dependencies = [ [[package]] name = "triomphe" -version = "0.1.14" +version = "0.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef8f7726da4807b58ea5c96fdc122f80702030edc33b35aff9190a51148ccc85" +checksum = "dd69c5aa8f924c7519d6372789a74eac5b94fb0f8fcf0d4a97eb0bfc3e785f39" [[package]] name = "try-lock" @@ -13703,15 +13715,15 @@ dependencies = [ "rustls", "rustls-pki-types", "sha1", - "thiserror 2.0.16", + "thiserror 2.0.17", "utf-8", ] [[package]] name = "typenum" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "ucd-trie" @@ -13757,9 +13769,9 @@ checksum = "75b844d17643ee918803943289730bec8aac480150456169e647ed0b576ba539" [[package]] name = "unicode-ident" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" +checksum = "f63a545481291138910575129486daeaf8ac54aee4387fe7906919f7830c7d9d" [[package]] name = "unicode-segmentation" @@ -13850,9 +13862,9 @@ dependencies = [ [[package]] name = "url" -version = "2.5.4" +version = "2.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" +checksum = "08bc136a29a3d1758e07a9cca267be308aeebf5cfd5a10f3f67ab2097683ef5b" dependencies = [ "form_urlencoded", "idna", @@ -13886,9 +13898,9 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.18.0" +version = "1.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f33196643e165781c20a5ead5582283a7dacbb87855d867fbc2df3f81eddc1be" +checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" dependencies = [ "getrandom 0.3.3", "js-sys", @@ -14003,30 +14015,40 @@ checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" [[package]] name = "wasi" -version = "0.14.2+wasi-0.2.4" +version = "0.14.7+wasi-0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9683f9a5a998d873c0d21fcbe3c083009670149a8fab228644b8bd36b2c48cb3" +checksum = "883478de20367e224c0090af9cf5f9fa85bed63a95c1abf3afc5c083ebc06e8c" dependencies = [ - "wit-bindgen-rt", + "wasip2", +] + +[[package]] +name = "wasip2" +version = "1.0.1+wasi-0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0562428422c63773dad2c345a1882263bbf4d65cf3f42e90921f787ef5ad58e7" +dependencies = [ + "wit-bindgen", ] [[package]] name = "wasm-bindgen" -version = "0.2.100" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1edc8929d7499fc4e8f0be2262a241556cfc54a0bea223790e71446f2aab1ef5" +checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" dependencies = [ "cfg-if", "once_cell", "rustversion", "wasm-bindgen-macro", + "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.100" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f0a0651a5c2bc21487bde11ee802ccaf4c51935d0d3d42a6101f98161700bc6" +checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" dependencies = [ "bumpalo", "log", @@ -14038,9 +14060,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.50" +version = "0.4.54" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "555d470ec0bc3bb57890405e5d4322cc9ea83cebb085523ced7be4144dac1e61" +checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" dependencies = [ "cfg-if", "js-sys", @@ -14051,9 +14073,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.100" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fe63fc6d09ed3792bd0897b314f53de8e16568c2b3f7982f468c0bf9bd0b407" +checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -14061,9 +14083,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.100" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" +checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" dependencies = [ "proc-macro2", "quote", @@ -14074,9 +14096,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.100" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a05d73b933a847d6cccdda8f838a22ff101ad9bf93e33684f39c1f5f0eece3d" +checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" dependencies = [ "unicode-ident", ] @@ -14096,9 +14118,9 @@ dependencies = [ [[package]] name = "wasmtimer" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8d49b5d6c64e8558d9b1b065014426f35c18de636895d24893dbbd329743446" +checksum = "1c598d6b99ea013e35844697fc4670d08339d5cda15588f193c6beedd12f644b" dependencies = [ "futures", "js-sys", @@ -14110,9 +14132,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.77" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33b6dd2ef9186f1f2072e409e99cd22a975331a6b3591b12c764e0e55c60d5d2" +checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" dependencies = [ "js-sys", "wasm-bindgen", @@ -14134,14 +14156,14 @@ version = "0.26.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "75c7f0ef91146ebfb530314f5f1d24528d7f0767efbfd31dce919275413e393e" dependencies = [ - "webpki-root-certs 1.0.2", + "webpki-root-certs 1.0.3", ] [[package]] name = "webpki-root-certs" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4ffd8df1c57e87c325000a3d6ef93db75279dc3a231125aac571650f22b12a" +checksum = "05d651ec480de84b762e7be71e6efa7461699c19d9e2c272c8d93455f567786e" dependencies = [ "rustls-pki-types", ] @@ -14152,14 +14174,14 @@ version = "0.26.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "521bc38abb08001b01866da9f51eb7c5d647a19260e00054a8c7fd5f9e57f7a9" dependencies = [ - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] name = "webpki-roots" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8983c3ab33d6fb807cfcdad2491c4ea8cbc8ed839181c7dfd9c67c83e261b2" +checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" dependencies = [ "rustls-pki-types", ] @@ -14178,9 +14200,9 @@ dependencies = [ [[package]] name = "widestring" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd7cf3379ca1aac9eea11fba24fd7e315d621f8dfe35c8d7d2be8b793726e07d" +checksum = "72069c3113ab32ab29e5584db3c6ec55d416895e60715417b5b883a357c3e471" [[package]] name = "winapi" @@ -14200,11 +14222,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.10" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0978bf7171b3d90bac376700cb56d606feb40f251a475a5d6634613564460b22" +checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -14235,25 +14257,27 @@ dependencies = [ [[package]] name = "windows" -version = "0.58.0" +version = "0.61.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd04d41d93c4992d421894c18c8b43496aa748dd4c081bac0dc93eb0489272b6" +checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" dependencies = [ - "windows-core 0.58.0", - "windows-targets 0.52.6", + "windows-collections 0.2.0", + "windows-core 0.61.2", + "windows-future 0.2.1", + "windows-link 0.1.3", + "windows-numerics 0.2.0", ] [[package]] name = "windows" -version = "0.61.3" +version = "0.62.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" +checksum = "527fadee13e0c05939a6a05d5bd6eec6cd2e3dbd648b9f8e447c6518133d8580" dependencies = [ - "windows-collections", - "windows-core 0.61.2", - "windows-future", - "windows-link 0.1.3", - "windows-numerics", + "windows-collections 0.3.2", + "windows-core 0.62.2", + "windows-future 0.3.2", + "windows-numerics 0.3.1", ] [[package]] @@ -14265,6 +14289,15 @@ dependencies = [ "windows-core 0.61.2", ] +[[package]] +name = "windows-collections" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23b2d95af1a8a14a3c7367e1ed4fc9c20e0a26e79551b1454d72583c97cc6610" +dependencies = [ + "windows-core 0.62.2", +] + [[package]] name = "windows-core" version = "0.53.0" @@ -14289,28 +14322,28 @@ dependencies = [ [[package]] name = "windows-core" -version = "0.58.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ba6d44ec8c2591c134257ce647b7ea6b20335bf6379a27dac5f1641fcf59f99" +checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" dependencies = [ - "windows-implement 0.58.0", - "windows-interface 0.58.0", - "windows-result 0.2.0", - "windows-strings 0.1.0", - "windows-targets 0.52.6", + "windows-implement 0.60.2", + "windows-interface 0.59.3", + "windows-link 0.1.3", + "windows-result 0.3.4", + "windows-strings 0.4.2", ] [[package]] name = "windows-core" -version = "0.61.2" +version = "0.62.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" +checksum = "b8e83a14d34d0623b51dce9581199302a221863196a1dde71a7663a4c2be9deb" dependencies = [ - "windows-implement 0.60.0", - "windows-interface 0.59.1", - "windows-link 0.1.3", - "windows-result 0.3.4", - "windows-strings 0.4.2", + "windows-implement 0.60.2", + "windows-interface 0.59.3", + "windows-link 0.2.1", + "windows-result 0.4.1", + "windows-strings 0.5.1", ] [[package]] @@ -14321,25 +14354,25 @@ checksum = "fc6a41e98427b19fe4b73c550f060b59fa592d7d686537eebf9385621bfbad8e" dependencies = [ "windows-core 0.61.2", "windows-link 0.1.3", - "windows-threading", + "windows-threading 0.1.0", ] [[package]] -name = "windows-implement" -version = "0.57.0" +name = "windows-future" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" +checksum = "e1d6f90251fe18a279739e78025bd6ddc52a7e22f921070ccdc67dde84c605cb" dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.106", + "windows-core 0.62.2", + "windows-link 0.2.1", + "windows-threading 0.2.1", ] [[package]] name = "windows-implement" -version = "0.58.0" +version = "0.57.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bbd5b46c938e506ecbce286b6628a02171d56153ba733b6c741fc627ec9579b" +checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" dependencies = [ "proc-macro2", "quote", @@ -14348,9 +14381,9 @@ dependencies = [ [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", @@ -14370,20 +14403,9 @@ dependencies = [ [[package]] name = "windows-interface" -version = "0.58.0" +version = "0.59.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "053c4c462dc91d3b1504c6fe5a726dd15e216ba718e84a0e46a88fbe5ded3515" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.106", -] - -[[package]] -name = "windows-interface" -version = "0.59.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", @@ -14398,9 +14420,9 @@ checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" [[package]] name = "windows-link" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45e46c0661abb7180e7b9c281db115305d49ca1709ab8242adf09666d2173c65" +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" [[package]] name = "windows-numerics" @@ -14412,6 +14434,16 @@ dependencies = [ "windows-link 0.1.3", ] +[[package]] +name = "windows-numerics" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e2e40844ac143cdb44aead537bbf727de9b044e107a0f1220392177d15b0f26" +dependencies = [ + "windows-core 0.62.2", + "windows-link 0.2.1", +] + [[package]] name = "windows-registry" version = "0.5.3" @@ -14434,39 +14466,38 @@ dependencies = [ [[package]] name = "windows-result" -version = "0.2.0" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" +checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" dependencies = [ - "windows-targets 0.52.6", + "windows-link 0.1.3", ] [[package]] name = "windows-result" -version = "0.3.4" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" +checksum = "7781fa89eaf60850ac3d2da7af8e5242a5ea78d1a11c49bf2910bb5a73853eb5" dependencies = [ - "windows-link 0.1.3", + "windows-link 0.2.1", ] [[package]] name = "windows-strings" -version = "0.1.0" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" +checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" dependencies = [ - "windows-result 0.2.0", - "windows-targets 0.52.6", + "windows-link 0.1.3", ] [[package]] name = "windows-strings" -version = "0.4.2" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" +checksum = "7837d08f69c77cf6b07689544538e017c1bfcf57e34b4c0ff58e6c2cd3b37091" dependencies = [ - "windows-link 0.1.3", + "windows-link 0.2.1", ] [[package]] @@ -14511,16 +14542,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.3", + "windows-targets 0.53.5", ] [[package]] name = "windows-sys" -version = "0.61.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" dependencies = [ - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -14571,19 +14602,19 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.3" +version = "0.53.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" dependencies = [ - "windows-link 0.1.3", - "windows_aarch64_gnullvm 0.53.0", - "windows_aarch64_msvc 0.53.0", - "windows_i686_gnu 0.53.0", - "windows_i686_gnullvm 0.53.0", - "windows_i686_msvc 0.53.0", - "windows_x86_64_gnu 0.53.0", - "windows_x86_64_gnullvm 0.53.0", - "windows_x86_64_msvc 0.53.0", + "windows-link 0.2.1", + "windows_aarch64_gnullvm 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", ] [[package]] @@ -14595,6 +14626,15 @@ dependencies = [ "windows-link 0.1.3", ] +[[package]] +name = "windows-threading" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3949bd5b99cafdf1c7ca86b43ca564028dfe27d66958f2470940f73d86d75b37" +dependencies = [ + "windows-link 0.2.1", +] + [[package]] name = "windows_aarch64_gnullvm" version = "0.42.2" @@ -14615,9 +14655,9 @@ checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" [[package]] name = "windows_aarch64_msvc" @@ -14639,9 +14679,9 @@ checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_aarch64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" [[package]] name = "windows_i686_gnu" @@ -14663,9 +14703,9 @@ checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" [[package]] name = "windows_i686_gnullvm" @@ -14675,9 +14715,9 @@ checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" [[package]] name = "windows_i686_msvc" @@ -14699,9 +14739,9 @@ checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_i686_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" [[package]] name = "windows_x86_64_gnu" @@ -14723,9 +14763,9 @@ checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" [[package]] name = "windows_x86_64_gnullvm" @@ -14747,9 +14787,9 @@ checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" [[package]] name = "windows_x86_64_msvc" @@ -14771,15 +14811,15 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "windows_x86_64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" [[package]] name = "winnow" -version = "0.7.12" +version = "0.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3edebf492c8125044983378ecb5766203ad3b4c2f7a922bd7dd207f6d443e95" +checksum = "21a0236b59786fed61e2a80582dd500fe61f18b5dca67a4a067d0bc9039339cf" dependencies = [ "memchr", ] @@ -14795,13 +14835,10 @@ dependencies = [ ] [[package]] -name = "wit-bindgen-rt" -version = "0.39.0" +name = "wit-bindgen" +version = "0.46.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f42320e61fe2cfd34354ecb597f86f413484a798ba44a8ca1165c58d42da6c1" -dependencies = [ - "bitflags 2.9.2", -] +checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "write16" @@ -14834,7 +14871,7 @@ dependencies = [ "pharos", "rustc_version 0.4.1", "send_wrapper 0.6.0", - "thiserror 2.0.16", + "thiserror 2.0.17", "wasm-bindgen", "wasm-bindgen-futures", "web-sys", @@ -14891,18 +14928,18 @@ dependencies = [ "nom", "oid-registry 0.8.1", "rusticata-macros", - "thiserror 2.0.16", + "thiserror 2.0.17", "time", ] [[package]] name = "xattr" -version = "1.5.1" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af3a19837351dc82ba89f8a125e22a3c475f05aba604acc023d62b2739ae2909" +checksum = "32e45ad4206f6d2479085147f02bc2ef834ac85886624a23575ae137c8aa8156" dependencies = [ "libc", - "rustix 1.0.8", + "rustix 1.1.2", ] [[package]] @@ -14937,9 +14974,9 @@ dependencies = [ [[package]] name = "yamux" -version = "0.13.6" +version = "0.13.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b2dd50a6d6115feb3e5d7d0efd45e8ca364b6c83722c1e9c602f5764e0e9597" +checksum = "6927cfe0edfae4b26a369df6bad49cd0ef088c0ec48f4045b2084bcaedc10246" dependencies = [ "futures", "log", @@ -15016,18 +15053,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.26" +version = "0.8.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1039dd0d3c310cf05de012d8a39ff557cb0d23087fd44cad61df08fc31907a2f" +checksum = "0894878a5fa3edfd6da3f88c4805f4c8558e2b996227a3d864f47fe11e38282c" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.8.26" +version = "0.8.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ecf5b4cc5364572d7f4c329661bcc82724222973f2cab6f050a4e5c22f75181" +checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", @@ -15057,9 +15094,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" dependencies = [ "zeroize_derive", ] @@ -15150,9 +15187,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.15+zstd.1.5.7" +version = "2.0.16+zstd.1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb81183ddd97d0c74cedf1d50d85c8d08c1b8b68ee863bdee9e706eedba1a237" +checksum = "91e19ebc2adc8f83e43039e79776e3fda8ca919132d68a1fed6a5faca2683748" dependencies = [ "cc", "pkg-config", diff --git a/Cargo.toml b/Cargo.toml index 2ec380aaf..7907bd9f5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,73 +43,73 @@ incremental = false unreachable_pub = "warn" [workspace.dependencies] -reth = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-chain-state = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-cli = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-cli-commands = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-cli-util = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-db = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-db-common = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-errors = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-node-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-rpc-engine-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-node-ethereum = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-trie = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-trie-parallel = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-basic-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-node-core = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-primitives-traits = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-provider = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1", features = [ +reth = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-chain-state = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-cli = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-cli-commands = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-cli-util = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-db = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-db-common = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-errors = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-node-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-rpc-engine-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-node-ethereum = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-trie = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-trie-parallel = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-basic-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-node-core = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-primitives-traits = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-provider = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1", features = [ "test-utils", ] } -reth-chainspec = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-storage-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-rpc-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-evm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-evm-ethereum = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-execution-errors = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-exex = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-tasks = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-metrics = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-trie-db = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-payload-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-transaction-pool = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-execution-types = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-revm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-payload-builder-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-payload-util = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-rpc-layer = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-network-peers = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-testing-utils = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-node-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-rpc-eth-types = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-tracing-otlp = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-ipc = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } +reth-chainspec = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-storage-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-rpc-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-evm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-evm-ethereum = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-execution-errors = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-exex = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-tasks = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-metrics = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-trie-db = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-payload-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-transaction-pool = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-execution-types = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-revm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-payload-builder-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-payload-util = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-rpc-layer = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-network-peers = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-testing-utils = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-node-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-rpc-eth-types = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-tracing-otlp = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-ipc = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } # reth optimism -reth-optimism-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-consensus = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-cli = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-forks = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-evm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-node = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-chainspec = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-txpool = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1" } -reth-optimism-rpc = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.1", features = [ +reth-optimism-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-consensus = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-cli = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-forks = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-evm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-node = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-chainspec = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-txpool = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-optimism-rpc = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1", features = [ "client", ] } -revm = { version = "29.0.1", features = [ +revm = { version = "30.0", features = [ "std", "secp256k1", "optional_balance_check", ], default-features = false } revm-inspectors = { version = "0.30.0", default-features = false } -op-revm = { version = "10.1.0", default-features = false } +op-revm = { version = "11.0.0", default-features = false } ethereum_ssz_derive = "0.9.0" ethereum_ssz = "0.9.0" @@ -117,32 +117,32 @@ ethereum_ssz = "0.9.0" alloy-primitives = { version = "1.3.1", default-features = false } alloy-rlp = "0.3.10" alloy-chains = "0.2.5" -alloy-contract = { version = "1.0.35" } -alloy-evm = { version = "0.21.0", default-features = false } -alloy-provider = { version = "1.0.35", features = [ +alloy-contract = { version = "1.0.37" } +alloy-evm = { version = "0.21.2", default-features = false } +alloy-provider = { version = "1.0.37", features = [ "ipc", "pubsub", "txpool-api", "engine-api", ] } -alloy-pubsub = { version = "1.0.35" } -alloy-eips = { version = "1.0.35" } -alloy-rpc-types = { version = "1.0.35" } -alloy-json-rpc = { version = "1.0.35" } -alloy-transport-http = { version = "1.0.35" } -alloy-network = { version = "1.0.35" } -alloy-network-primitives = { version = "1.0.35" } -alloy-transport = { version = "1.0.35" } -alloy-node-bindings = { version = "1.0.35" } -alloy-consensus = { version = "1.0.35", features = ["kzg"] } -alloy-serde = { version = "1.0.35" } +alloy-pubsub = { version = "1.0.37" } +alloy-eips = { version = "1.0.37" } +alloy-rpc-types = { version = "1.0.37" } +alloy-json-rpc = { version = "1.0.37" } +alloy-transport-http = { version = "1.0.37" } +alloy-network = { version = "1.0.37" } +alloy-network-primitives = { version = "1.0.37" } +alloy-transport = { version = "1.0.37" } +alloy-node-bindings = { version = "1.0.37" } +alloy-consensus = { version = "1.0.37", features = ["kzg"] } +alloy-serde = { version = "1.0.37" } alloy-sol-types = { version = "1.2.1", features = ["json"] } -alloy-rpc-types-beacon = { version = "1.0.35", features = ["ssz"] } -alloy-rpc-types-engine = { version = "1.0.35", features = ["ssz"] } -alloy-rpc-types-eth = { version = "1.0.35" } -alloy-signer-local = { version = "1.0.35" } -alloy-rpc-client = { version = "1.0.35" } -alloy-genesis = { version = "1.0.35" } +alloy-rpc-types-beacon = { version = "1.0.37", features = ["ssz"] } +alloy-rpc-types-engine = { version = "1.0.37", features = ["ssz"] } +alloy-rpc-types-eth = { version = "1.0.37" } +alloy-signer-local = { version = "1.0.37" } +alloy-rpc-client = { version = "1.0.37" } +alloy-genesis = { version = "1.0.37" } alloy-trie = { version = "0.9.1" } # optimism @@ -166,8 +166,8 @@ parking_lot = { version = "0.12.3" } tokio = { version = "1.40.0" } auto_impl = { version = "1.2.0" } reqwest = { version = "0.12.8" } -serde = { version = "1.0.210" } -serde_json = { version = "1.0.128" } +serde = { version = "1.0" } +serde_json = { version = "1.0" } serde_with = { version = "3.8.1" } secp256k1 = { version = "0.30" } derive_more = { version = "2" } @@ -175,7 +175,7 @@ tokio-stream = "0.1.16" tokio-util = "0.7.12" url = "2.5.2" warp = "0.3.7" -flate2 = "1.0.35" +flate2 = "1.0.37" prometheus = "0.13.4" ctor = "0.2" dashmap = "6.1" @@ -192,3 +192,8 @@ time = { version = "0.3.36", features = ["macros", "formatting", "parsing"] } vergen = "9.0.4" vergen-git2 = "1.0.5" opentelemetry = { version = "0.29.1", features = ["trace"] } + +[patch.crates-io] +alloy-evm = { git = "https://github.com/noot/evm", rev = "c7ea94eaa1bfc0971624ccd3e6973059335cb862" } +alloy-op-evm = { git = "https://github.com/noot/evm", rev = "c7ea94eaa1bfc0971624ccd3e6973059335cb862" } +revm-inspectors = { git = "https://github.com/noot/revm-inspectors", rev = "1af65e18317c33a37492689c5b11c6ddbfe3bc51" } diff --git a/crates/op-rbuilder/Cargo.toml b/crates/op-rbuilder/Cargo.toml index 6a62052f8..c54f6d0fe 100644 --- a/crates/op-rbuilder/Cargo.toml +++ b/crates/op-rbuilder/Cargo.toml @@ -131,7 +131,7 @@ ureq = "2.10" reqwest = "0.12.23" k256 = "0.13.4" -rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "b86af43969557bee18f17ec1d6bcd3e984f910b2" } +rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "8cabb2c2056fddb55b39369b404d21989eab8bb7" } nanoid = { version = "0.4", optional = true } reth-ipc = { workspace = true, optional = true } diff --git a/crates/op-rbuilder/src/args/op.rs b/crates/op-rbuilder/src/args/op.rs index 1a6b69767..f35f48aa9 100644 --- a/crates/op-rbuilder/src/args/op.rs +++ b/crates/op-rbuilder/src/args/op.rs @@ -167,7 +167,7 @@ pub struct FlashblocksArgs { /// Port for the flashblocks p2p node #[arg( - long = "flasblocks.p2p_port", + long = "flashblocks.p2p_port", env = "FLASHBLOCK_P2P_PORT", default_value = "9009" )] diff --git a/crates/op-rbuilder/src/builders/context.rs b/crates/op-rbuilder/src/builders/context.rs index f8766d074..58f09a301 100644 --- a/crates/op-rbuilder/src/builders/context.rs +++ b/crates/op-rbuilder/src/builders/context.rs @@ -338,7 +338,7 @@ impl OpPayloadBuilderCtx { let tx_da_limit = self.da_config.max_da_tx_size(); let mut evm = self.evm_config.evm_with_env(&mut *db, self.evm_env.clone()); - info!( + debug!( target: "payload_builder", message = "Executing best transactions", block_da_limit = ?block_da_limit, diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index 90fd81251..6d0e8e5f2 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -1085,7 +1085,7 @@ where }, trie: ExecutedTrieUpdates::Present(Arc::new(trie_output)), }; - info!(target: "payload_builder", message = "Executed block created"); + debug!(target: "payload_builder", message = "Executed block created"); let sealed_block = Arc::new(block.seal_slow()); debug!(target: "payload_builder", ?sealed_block, "sealed built block"); diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index c22a04b1c..d843b80f2 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -121,10 +121,10 @@ where let _ = payload_events_handle.send(Events::BuiltPayload(payload)); // TODO is this only for built or also synced? } Ok(Err(e)) => { - tracing::error!(error = %e, "failed to execute flashblock"); + tracing::error!(error = ?e, "failed to execute flashblock"); } Err(e) => { - tracing::error!(error = %e, "task panicked while executing flashblock"); + tracing::error!(error = ?e, "task panicked while executing flashblock"); } } } @@ -145,17 +145,25 @@ async fn execute_flashblock( where Client: ClientBounds, { + use alloy_consensus::BlockHeader as _; + use reth::primitives::SealedHeader; use reth_evm::{ConfigureEvm as _, execute::BlockBuilder as _}; - use reth_optimism_chainspec::OpHardforks as _; - use reth_payload_primitives::PayloadBuilderAttributes as _; let mut cached_reads = reth::revm::cached::CachedReads::default(); // TODO: pass this in from somewhere + let parent_hash = payload.block().sealed_header().parent_hash; + let parent_header = client + .header_by_id(parent_hash.into()) + .wrap_err("failed to get parent header")? + .ok_or_else(|| eyre::eyre!("parent header not found"))?; + // TODO: can refactor this out probably let payload_config = PayloadConfig::new( - Arc::new(payload.block().sealed_header().clone()), + Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), OpPayloadBuilderAttributes::default(), ); - let state_provider = client.state_by_block_hash(payload_config.parent_header.hash())?; + let state_provider = client + .state_by_block_hash(parent_hash) + .wrap_err("failed to get state for parent hash")?; let db = StateProviderDatabase::new(&state_provider); let mut state = State::builder() .with_database(cached_reads.as_db_mut(db)) @@ -163,32 +171,19 @@ where .build(); let chain_spec = client.chain_spec(); - let timestamp = payload_config.attributes.timestamp(); + let timestamp = payload.block().sealed_header().timestamp(); let block_env_attributes = OpNextBlockEnvAttributes { timestamp, - suggested_fee_recipient: payload_config.attributes.suggested_fee_recipient(), - prev_randao: payload_config.attributes.prev_randao(), - gas_limit: payload_config - .attributes - .gas_limit - .unwrap_or(payload_config.parent_header.gas_limit), - parent_beacon_block_root: payload_config - .attributes - .payload_attributes - .parent_beacon_block_root, - extra_data: if chain_spec.is_holocene_active_at_timestamp(timestamp) { - payload_config - .attributes - .get_holocene_extra_data(chain_spec.base_fee_params_at_timestamp(timestamp)) - .wrap_err("failed to get holocene extra data for flashblocks payload builder")? - } else { - Default::default() - }, + suggested_fee_recipient: payload.block().sealed_header().beneficiary, + prev_randao: Default::default(), // TODO: is this needed? + gas_limit: payload.block().sealed_header().gas_limit, + parent_beacon_block_root: payload.block().sealed_header().parent_beacon_block_root, + extra_data: payload.block().sealed_header().extra_data.clone(), }; let evm_env = ctx .evm_config() - .next_evm_env(&payload_config.parent_header, &block_env_attributes) + .next_evm_env(&parent_header, &block_env_attributes) .wrap_err("failed to create next evm env")?; let address_gas_limiter = AddressGasLimiter::new(gas_limiter_config); @@ -213,6 +208,7 @@ where .wrap_err("failed to create evm builder for next block")? .apply_pre_execution_changes() .wrap_err("failed to apply pre execution changes")?; + // this is a no-op rn because attributes aren't set let mut info: ExecutionInfo = builder_ctx .execute_sequencer_transactions(&mut state) .wrap_err("failed to execute sequencer transactions")?; diff --git a/crates/p2p/src/behaviour.rs b/crates/p2p/src/behaviour.rs index a61ea7354..651123e9e 100644 --- a/crates/p2p/src/behaviour.rs +++ b/crates/p2p/src/behaviour.rs @@ -1,7 +1,9 @@ use eyre::WrapErr as _; use libp2p::{ - autonat, connection_limits, connection_limits::ConnectionLimits, identify, identity, mdns, - ping, swarm::NetworkBehaviour, + Swarm, autonat, + connection_limits::{self, ConnectionLimits}, + identify, identity, mdns, ping, + swarm::NetworkBehaviour, }; use std::{convert::Infallible, time::Duration}; @@ -81,14 +83,22 @@ impl Behaviour { } impl BehaviourEvent { - pub(crate) async fn handle(self) { + pub(crate) fn handle(self, swarm: &mut Swarm) { match self { BehaviourEvent::Autonat(_event) => {} BehaviourEvent::Identify(_event) => {} BehaviourEvent::Mdns(event) => match event { mdns::Event::Discovered(list) => { for (peer_id, multiaddr) in list { - tracing::debug!("mDNS discovered peer {peer_id} at {multiaddr}"); + if swarm.is_connected(&peer_id) { + continue; + } + + tracing::info!("mDNS discovered peer {peer_id} at {multiaddr}"); + swarm.add_peer_address(peer_id, multiaddr); + swarm.dial(peer_id).unwrap_or_else(|e| { + tracing::error!("failed to dial mDNS discovered peer {peer_id}: {e}") + }); } } mdns::Event::Expired(list) => { diff --git a/crates/p2p/src/lib.rs b/crates/p2p/src/lib.rs index 619bcaf1c..048f6cac4 100644 --- a/crates/p2p/src/lib.rs +++ b/crates/p2p/src/lib.rs @@ -15,7 +15,7 @@ use libp2p::{ use std::{collections::HashMap, time::Duration}; use tokio::sync::mpsc; use tokio_util::sync::CancellationToken; -use tracing::{debug, warn}; +use tracing::{debug, info, warn}; pub use libp2p::{Multiaddr, StreamProtocol}; @@ -133,6 +133,7 @@ impl Node { } Some(message) = outgoing_message_rx.recv() => { let protocol = message.protocol(); + info!("received message to broadcast on protocol {protocol}"); if let Err(e) = outgoing_streams_handler.broadcast_message(message).await { warn!("failed to broadcast message on protocol {protocol}: {e:?}"); } @@ -158,10 +159,10 @@ impl Node { // // If we already have a connection with this peer, close the new connection, // as we only want one connection per peer. - debug!("connection established with peer {peer_id}"); + info!("connection established with peer {peer_id}"); if outgoing_streams_handler.has_peer(&peer_id) { - swarm.close_connection(connection_id); - debug!("already have connection with peer {peer_id}, closed connection {connection_id}"); + //swarm.close_connection(connection_id); + //info!("already have connection with peer {peer_id}, closed connection {connection_id}"); } else { for protocol in &protocols { match swarm @@ -171,7 +172,7 @@ impl Node { .await { Ok(stream) => { outgoing_streams_handler.insert_peer_and_stream(peer_id, protocol.clone(), stream); - debug!("opened outbound stream with peer {peer_id} with protocol {protocol} on connection {connection_id}"); + info!("opened outbound stream with peer {peer_id} with protocol {protocol} on connection {connection_id}"); } Err(e) => { warn!("failed to open stream with peer {peer_id} on connection {connection_id}: {e:?}"); @@ -185,10 +186,10 @@ impl Node { cause, .. } => { - debug!("connection closed with peer {peer_id}: {cause:?}"); + info!("connection closed with peer {peer_id}: {cause:?}"); outgoing_streams_handler.remove_peer(&peer_id); } - SwarmEvent::Behaviour(event) => event.handle().await, + SwarmEvent::Behaviour(event) => event.handle(&mut swarm), _ => continue, } }, diff --git a/crates/p2p/src/outgoing.rs b/crates/p2p/src/outgoing.rs index 5b86b7b30..3fb4c2b27 100644 --- a/crates/p2p/src/outgoing.rs +++ b/crates/p2p/src/outgoing.rs @@ -3,7 +3,7 @@ use eyre::Context; use futures::stream::FuturesUnordered; use libp2p::{PeerId, StreamProtocol, swarm::Stream}; use std::collections::HashMap; -use tracing::{debug, warn}; +use tracing::{info, warn}; pub(crate) struct StreamsHandler { peers_to_stream: HashMap>, @@ -61,7 +61,10 @@ impl StreamsHandler { let payload = payload.clone(); let fut = async move { let mut writer = FramedWrite::new(stream, LinesCodec::new()); - writer.send(payload).await?; + writer + .send(payload) + .await + .wrap_err("failed to send message to peer")?; Ok::<(PeerId, libp2p::swarm::Stream), eyre::ErrReport>(( peer, writer.into_inner().into_inner(), @@ -69,6 +72,7 @@ impl StreamsHandler { }; futures.push(fut); } + while let Some(result) = futures.next().await { match result { Ok((peer, stream)) => { @@ -83,7 +87,7 @@ impl StreamsHandler { } } } - debug!( + info!( "broadcasted message to {} peers", self.peers_to_stream.len() ); From 181689eb8cd0a8678b92d3c85302a27fb546660c Mon Sep 17 00:00:00 2001 From: elizabeth Date: Sat, 11 Oct 2025 00:17:13 -0400 Subject: [PATCH 03/13] sync working but flashblock hash mismatch --- Cargo.lock | 351 +++++++++--------- Cargo.toml | 122 +++--- crates/op-rbuilder/Cargo.toml | 2 +- .../builders/flashblocks/payload_handler.rs | 46 ++- 4 files changed, 276 insertions(+), 245 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8d07b5362..842924a57 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -297,7 +297,7 @@ dependencies = [ [[package]] name = "alloy-evm" version = "0.21.2" -source = "git+https://github.com/noot/evm?rev=c7ea94eaa1bfc0971624ccd3e6973059335cb862#c7ea94eaa1bfc0971624ccd3e6973059335cb862" +source = "git+https://github.com/noot/evm?rev=f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e#f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" dependencies = [ "alloy-consensus", "alloy-eips", @@ -425,7 +425,7 @@ dependencies = [ [[package]] name = "alloy-op-evm" version = "0.21.2" -source = "git+https://github.com/noot/evm?rev=c7ea94eaa1bfc0971624ccd3e6973059335cb862#c7ea94eaa1bfc0971624ccd3e6973059335cb862" +source = "git+https://github.com/noot/evm?rev=f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e#f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" dependencies = [ "alloy-consensus", "alloy-eips", @@ -2021,18 +2021,18 @@ checksum = "2d7e60934ceec538daadb9d8432424ed043a904d8e0243f3c6446bce549a46ac" [[package]] name = "bitfield" -version = "0.19.2" +version = "0.19.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62a3a774b2fcac1b726922b921ebba5e9fe36ad37659c822cf8ff2c1e0819892" +checksum = "6bf79f42d21f18b5926a959280215903e659760da994835d27c3a0c5ff4f898f" dependencies = [ "bitfield-macros", ] [[package]] name = "bitfield-macros" -version = "0.19.2" +version = "0.19.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52511b09931f7d5fe3a14f23adefbc23e5725b184013e96c8419febb61f14734" +checksum = "6115af052c7914c0cbb97195e5c72cb61c511527250074f5c041d1048b0d8b16" dependencies = [ "proc-macro2", "quote", @@ -2644,7 +2644,7 @@ version = "0.3.0" source = "git+https://github.com/automata-network/coco-provider-sdk#3a832b8cf5e88ef71649ab56e4efd67067b26b7c" dependencies = [ "bincode", - "bitfield 0.19.2", + "bitfield 0.19.3", "cbindgen", "iocuddle", "libc", @@ -6572,11 +6572,11 @@ dependencies = [ [[package]] name = "nu-ansi-term" -version = "0.50.1" +version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a28e057d01f97e61255210fcff094d74ed0466038633e95017f5beb68e4399" +checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.61.2", ] [[package]] @@ -7025,9 +7025,8 @@ dependencies = [ [[package]] name = "op-revm" -version = "11.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb8cb3ecd7aaff3a18e15a155665b1e9b1979dff93d3ccf12161a85682237dbc" +version = "11.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "auto_impl", "revm", @@ -7400,12 +7399,12 @@ checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" [[package]] name = "pem" -version = "3.0.5" +version = "3.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38af38e8470ac9dee3ce1bae1af9c1671fffc44ddfd8bd1d0a3445bf349a8ef3" +checksum = "1d30c53c26bc5b31a98cd02d20f25a7c8567146caf63ed593a9d87b2775291be" dependencies = [ "base64 0.22.1", - "serde", + "serde_core", ] [[package]] @@ -7739,7 +7738,7 @@ version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" dependencies = [ - "toml_edit 0.23.6", + "toml_edit 0.23.7", ] [[package]] @@ -8364,7 +8363,7 @@ checksum = "6b3789b30bd25ba102de4beabd95d21ac45b69b1be7d14522bab988c526d6799" [[package]] name = "reth" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-rpc-types", "aquamarine", @@ -8410,7 +8409,7 @@ dependencies = [ [[package]] name = "reth-basic-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8434,7 +8433,7 @@ dependencies = [ [[package]] name = "reth-chain-state" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8465,7 +8464,7 @@ dependencies = [ [[package]] name = "reth-chainspec" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-chains", "alloy-consensus", @@ -8485,7 +8484,7 @@ dependencies = [ [[package]] name = "reth-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-genesis", "clap", @@ -8499,7 +8498,7 @@ dependencies = [ [[package]] name = "reth-cli-commands" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-chains", "alloy-consensus", @@ -8574,7 +8573,7 @@ dependencies = [ [[package]] name = "reth-cli-runner" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "reth-tasks", "tokio", @@ -8584,7 +8583,7 @@ dependencies = [ [[package]] name = "reth-cli-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -8602,7 +8601,7 @@ dependencies = [ [[package]] name = "reth-codecs" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8622,7 +8621,7 @@ dependencies = [ [[package]] name = "reth-codecs-derive" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "convert_case", "proc-macro2", @@ -8633,7 +8632,7 @@ dependencies = [ [[package]] name = "reth-config" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "eyre", "humantime-serde", @@ -8648,7 +8647,7 @@ dependencies = [ [[package]] name = "reth-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -8661,7 +8660,7 @@ dependencies = [ [[package]] name = "reth-consensus-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8673,7 +8672,7 @@ dependencies = [ [[package]] name = "reth-consensus-debug-client" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8699,7 +8698,7 @@ dependencies = [ [[package]] name = "reth-db" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "derive_more", @@ -8725,7 +8724,7 @@ dependencies = [ [[package]] name = "reth-db-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-genesis", @@ -8753,7 +8752,7 @@ dependencies = [ [[package]] name = "reth-db-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-genesis", @@ -8783,7 +8782,7 @@ dependencies = [ [[package]] name = "reth-db-models" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -8798,7 +8797,7 @@ dependencies = [ [[package]] name = "reth-discv4" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -8824,7 +8823,7 @@ dependencies = [ [[package]] name = "reth-discv5" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -8848,7 +8847,7 @@ dependencies = [ [[package]] name = "reth-dns-discovery" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "data-encoding", @@ -8872,7 +8871,7 @@ dependencies = [ [[package]] name = "reth-downloaders" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8907,7 +8906,7 @@ dependencies = [ [[package]] name = "reth-ecies" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "aes", "alloy-primitives 1.4.0", @@ -8938,7 +8937,7 @@ dependencies = [ [[package]] name = "reth-engine-local" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -8962,7 +8961,7 @@ dependencies = [ [[package]] name = "reth-engine-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -8987,7 +8986,7 @@ dependencies = [ [[package]] name = "reth-engine-service" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "futures", "pin-project", @@ -9010,7 +9009,7 @@ dependencies = [ [[package]] name = "reth-engine-tree" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9064,7 +9063,7 @@ dependencies = [ [[package]] name = "reth-engine-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", @@ -9092,7 +9091,7 @@ dependencies = [ [[package]] name = "reth-era" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9108,7 +9107,7 @@ dependencies = [ [[package]] name = "reth-era-downloader" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "bytes", @@ -9123,7 +9122,7 @@ dependencies = [ [[package]] name = "reth-era-utils" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -9145,7 +9144,7 @@ dependencies = [ [[package]] name = "reth-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "reth-consensus", "reth-execution-errors", @@ -9156,7 +9155,7 @@ dependencies = [ [[package]] name = "reth-eth-wire" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-chains", "alloy-primitives 1.4.0", @@ -9184,7 +9183,7 @@ dependencies = [ [[package]] name = "reth-eth-wire-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-chains", "alloy-consensus", @@ -9205,7 +9204,7 @@ dependencies = [ [[package]] name = "reth-ethereum-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "clap", "eyre", @@ -9227,7 +9226,7 @@ dependencies = [ [[package]] name = "reth-ethereum-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9243,7 +9242,7 @@ dependencies = [ [[package]] name = "reth-ethereum-engine-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -9261,7 +9260,7 @@ dependencies = [ [[package]] name = "reth-ethereum-forks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eip2124", "alloy-hardforks", @@ -9274,7 +9273,7 @@ dependencies = [ [[package]] name = "reth-ethereum-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9303,7 +9302,7 @@ dependencies = [ [[package]] name = "reth-ethereum-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9323,7 +9322,7 @@ dependencies = [ [[package]] name = "reth-etl" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "rayon", "reth-db-api", @@ -9333,7 +9332,7 @@ dependencies = [ [[package]] name = "reth-evm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9356,7 +9355,7 @@ dependencies = [ [[package]] name = "reth-evm-ethereum" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9376,7 +9375,7 @@ dependencies = [ [[package]] name = "reth-execution-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-evm", "alloy-primitives 1.4.0", @@ -9389,7 +9388,7 @@ dependencies = [ [[package]] name = "reth-execution-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9407,7 +9406,7 @@ dependencies = [ [[package]] name = "reth-exex" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9445,7 +9444,7 @@ dependencies = [ [[package]] name = "reth-exex-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -9459,7 +9458,7 @@ dependencies = [ [[package]] name = "reth-fs-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "serde", "serde_json", @@ -9469,7 +9468,7 @@ dependencies = [ [[package]] name = "reth-invalid-block-hooks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -9496,7 +9495,7 @@ dependencies = [ [[package]] name = "reth-ipc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "bytes", "futures", @@ -9516,7 +9515,7 @@ dependencies = [ [[package]] name = "reth-libmdbx" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "bitflags 2.9.4", "byteorder", @@ -9532,7 +9531,7 @@ dependencies = [ [[package]] name = "reth-mdbx-sys" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "bindgen 0.71.1", "cc", @@ -9541,7 +9540,7 @@ dependencies = [ [[package]] name = "reth-metrics" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "futures", "metrics", @@ -9553,7 +9552,7 @@ dependencies = [ [[package]] name = "reth-net-banlist" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", ] @@ -9561,7 +9560,7 @@ dependencies = [ [[package]] name = "reth-net-nat" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "futures-util", "if-addrs 0.13.4", @@ -9575,7 +9574,7 @@ dependencies = [ [[package]] name = "reth-network" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9630,7 +9629,7 @@ dependencies = [ [[package]] name = "reth-network-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -9655,7 +9654,7 @@ dependencies = [ [[package]] name = "reth-network-p2p" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9678,7 +9677,7 @@ dependencies = [ [[package]] name = "reth-network-peers" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -9693,7 +9692,7 @@ dependencies = [ [[package]] name = "reth-network-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eip2124", "humantime-serde", @@ -9707,7 +9706,7 @@ dependencies = [ [[package]] name = "reth-nippy-jar" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "anyhow", "bincode", @@ -9724,7 +9723,7 @@ dependencies = [ [[package]] name = "reth-node-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-rpc-types-engine", "eyre", @@ -9748,7 +9747,7 @@ dependencies = [ [[package]] name = "reth-node-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9816,7 +9815,7 @@ dependencies = [ [[package]] name = "reth-node-core" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9868,7 +9867,7 @@ dependencies = [ [[package]] name = "reth-node-ethereum" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-network", @@ -9906,7 +9905,7 @@ dependencies = [ [[package]] name = "reth-node-ethstats" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -9930,7 +9929,7 @@ dependencies = [ [[package]] name = "reth-node-events" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -9954,7 +9953,7 @@ dependencies = [ [[package]] name = "reth-node-metrics" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "eyre", "http", @@ -9975,7 +9974,7 @@ dependencies = [ [[package]] name = "reth-node-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "reth-chainspec", "reth-db-api", @@ -9987,7 +9986,7 @@ dependencies = [ [[package]] name = "reth-optimism-chainspec" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-chains", "alloy-consensus", @@ -10015,7 +10014,7 @@ dependencies = [ [[package]] name = "reth-optimism-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10063,7 +10062,7 @@ dependencies = [ [[package]] name = "reth-optimism-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10088,7 +10087,7 @@ dependencies = [ [[package]] name = "reth-optimism-evm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10116,7 +10115,7 @@ dependencies = [ [[package]] name = "reth-optimism-flashblocks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10151,7 +10150,7 @@ dependencies = [ [[package]] name = "reth-optimism-forks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-op-hardforks", "alloy-primitives 1.4.0", @@ -10162,7 +10161,7 @@ dependencies = [ [[package]] name = "reth-optimism-node" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -10208,7 +10207,7 @@ dependencies = [ [[package]] name = "reth-optimism-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10247,7 +10246,7 @@ dependencies = [ [[package]] name = "reth-optimism-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10267,7 +10266,7 @@ dependencies = [ [[package]] name = "reth-optimism-rpc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10329,7 +10328,7 @@ dependencies = [ [[package]] name = "reth-optimism-storage" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "reth-optimism-primitives", @@ -10339,7 +10338,7 @@ dependencies = [ [[package]] name = "reth-optimism-txpool" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10375,7 +10374,7 @@ dependencies = [ [[package]] name = "reth-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -10396,7 +10395,7 @@ dependencies = [ [[package]] name = "reth-payload-builder-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "pin-project", "reth-payload-primitives", @@ -10408,7 +10407,7 @@ dependencies = [ [[package]] name = "reth-payload-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -10428,7 +10427,7 @@ dependencies = [ [[package]] name = "reth-payload-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -10438,7 +10437,7 @@ dependencies = [ [[package]] name = "reth-payload-validator" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", @@ -10448,7 +10447,7 @@ dependencies = [ [[package]] name = "reth-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "c-kzg", @@ -10462,7 +10461,7 @@ dependencies = [ [[package]] name = "reth-primitives-traits" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10495,7 +10494,7 @@ dependencies = [ [[package]] name = "reth-provider" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10540,7 +10539,7 @@ dependencies = [ [[package]] name = "reth-prune" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10568,7 +10567,7 @@ dependencies = [ [[package]] name = "reth-prune-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "arbitrary", @@ -10582,7 +10581,7 @@ dependencies = [ [[package]] name = "reth-ress-protocol" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -10601,7 +10600,7 @@ dependencies = [ [[package]] name = "reth-ress-provider" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -10628,7 +10627,7 @@ dependencies = [ [[package]] name = "reth-revm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "reth-primitives-traits", @@ -10641,7 +10640,7 @@ dependencies = [ [[package]] name = "reth-rpc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10720,7 +10719,7 @@ dependencies = [ [[package]] name = "reth-rpc-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-genesis", @@ -10748,7 +10747,7 @@ dependencies = [ [[package]] name = "reth-rpc-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-network", "alloy-provider", @@ -10787,7 +10786,7 @@ dependencies = [ [[package]] name = "reth-rpc-convert" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-json-rpc", @@ -10814,7 +10813,7 @@ dependencies = [ [[package]] name = "reth-rpc-engine-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -10844,7 +10843,7 @@ dependencies = [ [[package]] name = "reth-rpc-eth-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10888,7 +10887,7 @@ dependencies = [ [[package]] name = "reth-rpc-eth-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -10935,7 +10934,7 @@ dependencies = [ [[package]] name = "reth-rpc-layer" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-rpc-types-engine", "http", @@ -10949,7 +10948,7 @@ dependencies = [ [[package]] name = "reth-rpc-server-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -10965,7 +10964,7 @@ dependencies = [ [[package]] name = "reth-stages" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -11013,7 +11012,7 @@ dependencies = [ [[package]] name = "reth-stages-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -11040,7 +11039,7 @@ dependencies = [ [[package]] name = "reth-stages-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "arbitrary", @@ -11054,7 +11053,7 @@ dependencies = [ [[package]] name = "reth-static-file" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "parking_lot", @@ -11074,7 +11073,7 @@ dependencies = [ [[package]] name = "reth-static-file-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "clap", @@ -11086,7 +11085,7 @@ dependencies = [ [[package]] name = "reth-storage-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -11109,7 +11108,7 @@ dependencies = [ [[package]] name = "reth-storage-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-eips", "alloy-primitives 1.4.0", @@ -11125,7 +11124,7 @@ dependencies = [ [[package]] name = "reth-tasks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "auto_impl", "dyn-clone", @@ -11143,7 +11142,7 @@ dependencies = [ [[package]] name = "reth-testing-utils" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -11159,7 +11158,7 @@ dependencies = [ [[package]] name = "reth-tokio-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "tokio", "tokio-stream", @@ -11169,7 +11168,7 @@ dependencies = [ [[package]] name = "reth-tracing" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "clap", "eyre", @@ -11184,7 +11183,7 @@ dependencies = [ [[package]] name = "reth-tracing-otlp" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "opentelemetry 0.29.1", "opentelemetry-otlp 0.29.0", @@ -11198,7 +11197,7 @@ dependencies = [ [[package]] name = "reth-transaction-pool" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -11239,7 +11238,7 @@ dependencies = [ [[package]] name = "reth-trie" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-eips", @@ -11264,7 +11263,7 @@ dependencies = [ [[package]] name = "reth-trie-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-consensus", "alloy-primitives 1.4.0", @@ -11290,7 +11289,7 @@ dependencies = [ [[package]] name = "reth-trie-db" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "reth-db-api", @@ -11303,7 +11302,7 @@ dependencies = [ [[package]] name = "reth-trie-parallel" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -11328,7 +11327,7 @@ dependencies = [ [[package]] name = "reth-trie-sparse" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -11347,7 +11346,7 @@ dependencies = [ [[package]] name = "reth-trie-sparse-parallel" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "alloy-primitives 1.4.0", "alloy-rlp", @@ -11365,16 +11364,15 @@ dependencies = [ [[package]] name = "reth-zstd-compressors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=69069f05addf64af04257a7414d83d74bd554de1#69069f05addf64af04257a7414d83d74bd554de1" +source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" dependencies = [ "zstd", ] [[package]] name = "revm" -version = "30.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cafc7f271a0ff9b5bc52f4ee5c2ba0ea9e5a491919e9ed0ea4b80b8d87487e79" +version = "30.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "revm-bytecode", "revm-context", @@ -11392,8 +11390,7 @@ dependencies = [ [[package]] name = "revm-bytecode" version = "7.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb7c72e495ec8732ac35b9d0b99b721f9099d261961dc9d230043af22c08f918" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "bitvec", "phf 0.13.1", @@ -11403,9 +11400,8 @@ dependencies = [ [[package]] name = "revm-context" -version = "10.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9453598455c34d8a3665d62ed4e4f578f03823194075d6fb713e4621a6494ad" +version = "10.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "bitvec", "cfg-if", @@ -11420,9 +11416,8 @@ dependencies = [ [[package]] name = "revm-context-interface" -version = "11.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e340a0ae6cd19b90231a88c45608b2ed9851a7dd4fdb8ecf4e5e3e5ca7b6ed4" +version = "11.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "alloy-eip2930", "alloy-eip7702", @@ -11436,9 +11431,8 @@ dependencies = [ [[package]] name = "revm-database" -version = "8.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef14e9bfb8ce40515fb0c7e1bea5f941136ab850ae2b761835fc6790d2e99348" +version = "9.0.0" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "alloy-eips", "revm-bytecode", @@ -11450,9 +11444,8 @@ dependencies = [ [[package]] name = "revm-database-interface" -version = "8.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "330203b2f1d4de6ccd8bd6493158ff171d8ac88cce82cf2e4d60554b63e6923a" +version = "8.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "auto_impl", "either", @@ -11463,9 +11456,8 @@ dependencies = [ [[package]] name = "revm-handler" -version = "11.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b02883e8ea5d1c35e25ea04ddbcdf048ebdab4d329223d7798c5676c575b3b68" +version = "11.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "auto_impl", "derive-where", @@ -11482,9 +11474,8 @@ dependencies = [ [[package]] name = "revm-inspector" -version = "11.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dd956c37c3fb40843d6ae78092fd5cf211a17812e9dd0176663f4dc3c37cf68" +version = "11.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "auto_impl", "either", @@ -11501,7 +11492,7 @@ dependencies = [ [[package]] name = "revm-inspectors" version = "0.30.0" -source = "git+https://github.com/noot/revm-inspectors?rev=1af65e18317c33a37492689c5b11c6ddbfe3bc51#1af65e18317c33a37492689c5b11c6ddbfe3bc51" +source = "git+https://github.com/noot/revm-inspectors?rev=84bcd1cb78c3c9dd84ad7a50a6a9e13333704598#84bcd1cb78c3c9dd84ad7a50a6a9e13333704598" dependencies = [ "alloy-primitives 1.4.0", "alloy-rpc-types-eth", @@ -11519,9 +11510,8 @@ dependencies = [ [[package]] name = "revm-interpreter" -version = "26.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3600bda3a01929495d6c3a36a713de2e3c4aac9e001dcd3f6211213b5c30205" +version = "27.0.0" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "revm-bytecode", "revm-context-interface", @@ -11532,9 +11522,8 @@ dependencies = [ [[package]] name = "revm-precompile" -version = "28.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73ba7917ca5dae8de3265adcb2a305dce4ca33b68e601aa06b281000d5ae9947" +version = "28.0.1" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "ark-bls12-381", "ark-bn254", @@ -11558,8 +11547,7 @@ dependencies = [ [[package]] name = "revm-primitives" version = "21.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38271b8b85f00154bdcf9f2ab0a3ec7a8100377d2c7a0d8eb23e19389b42c795" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "alloy-primitives 1.4.0", "num_enum", @@ -11570,8 +11558,7 @@ dependencies = [ [[package]] name = "revm-state" version = "8.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "264baea816b732f7b49e7370eae52ac5f45f827e3b7c8494a2b2aa38e3a8cecc" +source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" dependencies = [ "bitflags 2.9.4", "revm-bytecode", @@ -11681,7 +11668,7 @@ dependencies = [ [[package]] name = "rollup-boost" version = "0.1.0" -source = "git+http://github.com/flashbots/rollup-boost?rev=8cabb2c2056fddb55b39369b404d21989eab8bb7#8cabb2c2056fddb55b39369b404d21989eab8bb7" +source = "git+http://github.com/flashbots/rollup-boost?rev=b5caff73878dbde5917a475201b8766affe22ce1#b5caff73878dbde5917a475201b8766affe22ce1" dependencies = [ "alloy-json-rpc", "alloy-primitives 1.4.0", @@ -13293,9 +13280,9 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32f1085dec27c2b6632b04c80b3bb1b4300d6495d1e129693bdda7d91e72eec1" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" dependencies = [ "serde_core", ] @@ -13316,21 +13303,21 @@ dependencies = [ [[package]] name = "toml_edit" -version = "0.23.6" +version = "0.23.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3effe7c0e86fdff4f69cdd2ccc1b96f933e24811c5441d44904e8683e27184b" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" dependencies = [ "indexmap 2.11.4", - "toml_datetime 0.7.2", + "toml_datetime 0.7.3", "toml_parser", "winnow", ] [[package]] name = "toml_parser" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cf893c33be71572e0e9aa6dd15e6677937abd686b066eac3f8cd3531688a627" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" dependencies = [ "winnow", ] diff --git a/Cargo.toml b/Cargo.toml index 7907bd9f5..b731de058 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,63 +43,63 @@ incremental = false unreachable_pub = "warn" [workspace.dependencies] -reth = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-chain-state = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-cli = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-cli-commands = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-cli-util = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-db = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-db-common = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-errors = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-node-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-rpc-engine-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-node-ethereum = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-trie = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-trie-parallel = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-basic-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-node-core = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-primitives-traits = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-provider = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1", features = [ +reth = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-chain-state = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-cli = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-cli-commands = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-cli-util = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-db = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-db-common = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-errors = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-node-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-rpc-engine-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-node-ethereum = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-trie = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-trie-parallel = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-basic-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-node-core = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-primitives-traits = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-provider = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702", features = [ "test-utils", ] } -reth-chainspec = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-storage-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-rpc-api = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-evm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-evm-ethereum = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-execution-errors = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-exex = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-tasks = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-metrics = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-trie-db = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-payload-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-transaction-pool = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-execution-types = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-revm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-payload-builder-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-payload-util = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-rpc-layer = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-network-peers = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-testing-utils = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-node-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-rpc-eth-types = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-tracing-otlp = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-ipc = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } +reth-chainspec = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-storage-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-rpc-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-evm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-evm-ethereum = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-execution-errors = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-exex = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-tasks = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-metrics = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-trie-db = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-payload-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-transaction-pool = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-execution-types = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-revm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-payload-builder-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-payload-util = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-rpc-layer = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-network-peers = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-testing-utils = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-node-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-rpc-eth-types = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-tracing-otlp = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-ipc = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } # reth optimism -reth-optimism-primitives = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-consensus = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-cli = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-forks = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-evm = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-node = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-payload-builder = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-chainspec = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-txpool = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1" } -reth-optimism-rpc = { git = "https://github.com/noot/reth", rev = "69069f05addf64af04257a7414d83d74bd554de1", features = [ +reth-optimism-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-consensus = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-cli = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-forks = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-evm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-node = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-chainspec = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-txpool = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-optimism-rpc = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702", features = [ "client", ] } @@ -194,6 +194,18 @@ vergen-git2 = "1.0.5" opentelemetry = { version = "0.29.1", features = ["trace"] } [patch.crates-io] -alloy-evm = { git = "https://github.com/noot/evm", rev = "c7ea94eaa1bfc0971624ccd3e6973059335cb862" } -alloy-op-evm = { git = "https://github.com/noot/evm", rev = "c7ea94eaa1bfc0971624ccd3e6973059335cb862" } -revm-inspectors = { git = "https://github.com/noot/revm-inspectors", rev = "1af65e18317c33a37492689c5b11c6ddbfe3bc51" } +alloy-evm = { git = "https://github.com/noot/evm", rev = "f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" } +alloy-op-evm = { git = "https://github.com/noot/evm", rev = "f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" } +revm-inspectors = { git = "https://github.com/noot/revm-inspectors", rev = "84bcd1cb78c3c9dd84ad7a50a6a9e13333704598" } +revm = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +op-revm = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } + +revm-bytecode = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-database = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-state = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-primitives = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-interpreter = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-inspector = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-context = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-context-interface = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } +revm-database-interface = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } diff --git a/crates/op-rbuilder/Cargo.toml b/crates/op-rbuilder/Cargo.toml index c54f6d0fe..7440bdacd 100644 --- a/crates/op-rbuilder/Cargo.toml +++ b/crates/op-rbuilder/Cargo.toml @@ -131,7 +131,7 @@ ureq = "2.10" reqwest = "0.12.23" k256 = "0.13.4" -rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "8cabb2c2056fddb55b39369b404d21989eab8bb7" } +rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "b5caff73878dbde5917a475201b8766affe22ce1" } nanoid = { version = "0.4", optional = true } reth-ipc = { workspace = true, optional = true } diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index d843b80f2..89db49f22 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -6,6 +6,7 @@ use crate::{ traits::ClientBounds, }; use alloy_evm::eth::receipt_builder::ReceiptBuilderCtx; +use alloy_primitives::B64; use eyre::WrapErr as _; use futures::stream::FuturesUnordered; use futures_util::StreamExt as _; @@ -155,7 +156,7 @@ where .header_by_id(parent_hash.into()) .wrap_err("failed to get parent header")? .ok_or_else(|| eyre::eyre!("parent header not found"))?; - // TODO: can refactor this out probably + // TODO: can refactor this out probably let payload_config = PayloadConfig::new( Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), OpPayloadBuilderAttributes::default(), @@ -188,7 +189,7 @@ where let address_gas_limiter = AddressGasLimiter::new(gas_limiter_config); // TODO: can probably refactor this - let builder_ctx = ctx.into_op_payload_builder_ctx( + let mut builder_ctx = ctx.into_op_payload_builder_ctx( payload_config, evm_env.clone(), block_env_attributes, @@ -213,6 +214,20 @@ where .execute_sequencer_transactions(&mut state) .wrap_err("failed to execute sequencer transactions")?; + let extra_data = payload.block().sealed_header().extra_data.clone(); + if extra_data.len() != 9 { + tracing::error!(len = extra_data.len(), data = ?extra_data, "invalid extra data length in flashblock"); + eyre::bail!("extra data length should be 9 bytes"); + } + + let eip_1559_parameters: B64 = extra_data[1..9].try_into().unwrap(); + builder_ctx.config.attributes.eip_1559_params = Some(eip_1559_parameters); + builder_ctx + .config + .attributes + .payload_attributes + .parent_beacon_block_root = payload.block().sealed_header().parent_beacon_block_root; + execute_transactions( &mut info, &mut state, @@ -230,7 +245,8 @@ where &builder_ctx, &mut info, true, // TODO: do we need this always? - )?; + ) + .wrap_err("failed to build flashblock")?; Ok((payload, fb_payload)) } @@ -320,10 +336,26 @@ fn execute_transactions( deposit, } } - OpTxEnvelope::Legacy(_) => OpTransaction::new(tx_env), - OpTxEnvelope::Eip2930(_) => OpTransaction::new(tx_env), - OpTxEnvelope::Eip1559(_) => OpTransaction::new(tx_env), - OpTxEnvelope::Eip7702(_) => OpTransaction::new(tx_env), + OpTxEnvelope::Legacy(_) => { + let mut tx = OpTransaction::new(tx_env); + tx.enveloped_tx = Some(vec![0x00].into()); + tx + } + OpTxEnvelope::Eip2930(_) => { + let mut tx = OpTransaction::new(tx_env); + tx.enveloped_tx = Some(vec![0x00].into()); + tx + } + OpTxEnvelope::Eip1559(_) => { + let mut tx = OpTransaction::new(tx_env); + tx.enveloped_tx = Some(vec![0x00].into()); + tx + } + OpTxEnvelope::Eip7702(_) => { + let mut tx = OpTransaction::new(tx_env); + tx.enveloped_tx = Some(vec![0x00].into()); + tx + } }; let ResultAndState { result, state } = match evm.transact_raw(executable_tx) { From 2c30d3e389de35bd631f7ee4088e637358dc4e6f Mon Sep 17 00:00:00 2001 From: elizabeth Date: Tue, 14 Oct 2025 17:26:58 -0400 Subject: [PATCH 04/13] fix receipt calculation; hashes match when syncing --- .../builders/flashblocks/payload_handler.rs | 75 ++++++++++++------- 1 file changed, 47 insertions(+), 28 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 89db49f22..82b8f49fb 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -7,7 +7,7 @@ use crate::{ }; use alloy_evm::eth::receipt_builder::ReceiptBuilderCtx; use alloy_primitives::B64; -use eyre::WrapErr as _; +use eyre::{WrapErr as _, bail}; use futures::stream::FuturesUnordered; use futures_util::StreamExt as _; use op_alloy_consensus::OpTxEnvelope; @@ -85,7 +85,7 @@ where cancel, } = self; - tracing::info!("flashblocks payload handler started"); + tracing::debug!("flashblocks payload handler started"); let mut execute_flashblock_futures = FuturesUnordered::new(); @@ -118,7 +118,7 @@ where Some(res) = execute_flashblock_futures.next() => { match res { Ok(Ok((payload, _))) => { - tracing::info!("successfully executed flashblock"); + tracing::debug!("successfully executed flashblock"); let _ = payload_events_handle.send(Events::BuiltPayload(payload)); // TODO is this only for built or also synced? } Ok(Err(e)) => { @@ -150,6 +150,8 @@ where use reth::primitives::SealedHeader; use reth_evm::{ConfigureEvm as _, execute::BlockBuilder as _}; + tracing::info!(header = ?payload.block().header(), "executing flashblock"); + let mut cached_reads = reth::revm::cached::CachedReads::default(); // TODO: pass this in from somewhere let parent_hash = payload.block().sealed_header().parent_hash; let parent_header = client @@ -172,11 +174,11 @@ where .build(); let chain_spec = client.chain_spec(); - let timestamp = payload.block().sealed_header().timestamp(); + let timestamp = payload.block().header().timestamp(); let block_env_attributes = OpNextBlockEnvAttributes { timestamp, suggested_fee_recipient: payload.block().sealed_header().beneficiary, - prev_randao: Default::default(), // TODO: is this needed? + prev_randao: payload.block().sealed_header().mix_hash, gas_limit: payload.block().sealed_header().gas_limit, parent_beacon_block_root: payload.block().sealed_header().parent_beacon_block_root, extra_data: payload.block().sealed_header().extra_data.clone(), @@ -209,10 +211,11 @@ where .wrap_err("failed to create evm builder for next block")? .apply_pre_execution_changes() .wrap_err("failed to apply pre execution changes")?; - // this is a no-op rn because attributes aren't set - let mut info: ExecutionInfo = builder_ctx - .execute_sequencer_transactions(&mut state) - .wrap_err("failed to execute sequencer transactions")?; + // this is a no-op rn because `payload_config` attributes aren't set + // let mut info: ExecutionInfo = builder_ctx + // .execute_sequencer_transactions(&mut state) + // .wrap_err("failed to execute sequencer transactions")?; + let mut info = ExecutionInfo::with_capacity(payload.block().body().transactions.len()); let extra_data = payload.block().sealed_header().extra_data.clone(); if extra_data.len() != 9 { @@ -227,6 +230,9 @@ where .attributes .payload_attributes .parent_beacon_block_root = payload.block().sealed_header().parent_beacon_block_root; + builder_ctx.config.attributes.payload_attributes.timestamp = timestamp; + builder_ctx.config.attributes.payload_attributes.prev_randao = + payload.block().sealed_header().mix_hash; execute_transactions( &mut info, @@ -237,6 +243,7 @@ where evm_env, builder_ctx.max_gas_per_txn, is_canyon_active(&chain_spec, timestamp), + is_regolith_active(&chain_spec, timestamp), ) .wrap_err("failed to execute best transactions")?; @@ -248,6 +255,8 @@ where ) .wrap_err("failed to build flashblock")?; + tracing::info!(header = ?payload.block().header(), "successfully executed flashblock"); + Ok((payload, fb_payload)) } @@ -305,6 +314,7 @@ fn execute_transactions( evm_env: alloy_evm::EvmEnv, max_gas_per_txn: Option, is_canyon_active: bool, + is_regolith_active: bool, ) -> eyre::Result<()> { use alloy_evm::{Evm as _, EvmError as _}; use op_revm::{OpTransaction, transaction::deposit::DepositTransactionParts}; @@ -315,7 +325,6 @@ fn execute_transactions( context::{TxEnv, result::ResultAndState}, }; - let mut gas_used: u64 = 0; let mut evm = evm_config.evm_with_env(&mut *state, evm_env); while let Some(ref tx) = txs.next(()) { @@ -380,17 +389,24 @@ fn execute_transactions( } let tx_gas_used = result.gas_used(); - gas_used = gas_used.checked_add(tx_gas_used).ok_or_else(|| { - eyre::eyre!("total gas used overflowed when executing flashblock transactions") - })?; - if gas_used > gas_limit { - return Err(eyre::eyre!( - "flashblock exceeded gas limit when executing transactions" - )); + info.cumulative_gas_used = info + .cumulative_gas_used + .checked_add(tx_gas_used) + .ok_or_else(|| { + eyre::eyre!("total gas used overflowed when executing flashblock transactions") + })?; + if info.cumulative_gas_used > gas_limit { + bail!("flashblock exceeded gas limit when executing transactions"); } - info.cumulative_gas_used += gas_used; - // info.cumulative_da_bytes_used += tx_da_size; + let depositor_nonce = (is_regolith_active && tx.is_deposit()) + .then(|| { + evm.db_mut() + .load_cache_account(sender) + .map(|acc| acc.account_info().unwrap_or_default().nonce) + }) + .transpose() + .wrap_err("failed to get depositor nonce")?; let ctx = ReceiptBuilderCtx { tx, @@ -399,17 +415,15 @@ fn execute_transactions( state: &state, cumulative_gas_used: info.cumulative_gas_used, }; - // TODO: deposit_nonce may be Some in the case of a sequencer tx - info.receipts - .push(build_receipt(evm_config, ctx, None, is_canyon_active)); - evm.db_mut().commit(state); + info.receipts.push(build_receipt( + evm_config, + ctx, + depositor_nonce, + is_canyon_active, + )); - // // update add to total fees - // let miner_fee = tx - // .effective_tip_per_gas(base_fee) - // .expect("fee is always valid; execution succeeded"); - // info.total_fees += U256::from(miner_fee) * U256::from(gas_used); + evm.db_mut().commit(state); // append sender and transaction to the respective lists info.executed_senders.push(sender); @@ -460,3 +474,8 @@ fn is_canyon_active(chain_spec: &OpChainSpec, timestamp: u64) -> bool { use reth_optimism_chainspec::OpHardforks as _; chain_spec.is_canyon_active_at_timestamp(timestamp) } + +fn is_regolith_active(chain_spec: &OpChainSpec, timestamp: u64) -> bool { + use reth_optimism_chainspec::OpHardforks as _; + chain_spec.is_regolith_active_at_timestamp(timestamp) +} From f3f47eb03ae9f8a789927d3e8e5e1c91a2b33894 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 14:53:34 -0400 Subject: [PATCH 05/13] revert dep changes --- Cargo.lock | 3364 +++++++++++------ Cargo.toml | 133 +- crates/op-rbuilder/Cargo.toml | 2 +- .../builders/flashblocks/payload_handler.rs | 8 +- 4 files changed, 2174 insertions(+), 1333 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 842924a57..1da58c68a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2,15 +2,6 @@ # It is not intended for manual editing. version = 4 -[[package]] -name = "addr2line" -version = "0.25.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b5d307320b3181d6d7954e663bd7c774a838b8220fe0593c86d9fb09f498b4b" -dependencies = [ - "gimli", -] - [[package]] name = "adler2" version = "2.0.1" @@ -59,7 +50,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" dependencies = [ "cfg-if", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", "version_check", "zerocopy", @@ -124,7 +115,7 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bf01dd83a1ca5e4807d0ca0223c9615e211ce5db0a9fd1443c2778cacf89b546" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "num_enum", "serde", @@ -138,7 +129,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a0dd3ed764953a6b20458b2b7abbfdc93d20d14b38babe1a70fe631a443a9f1" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-serde", "alloy-trie", @@ -166,7 +157,7 @@ checksum = "9556182afa73cddffa91e64a5aa9508d5e8c912b3a15f26998d2388a824d2c7b" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-serde", "arbitrary", @@ -181,13 +172,13 @@ checksum = "b19d7092c96defc3d132ee0d8969ca1b79ef512b5eda5c66e3065266b253adf2" dependencies = [ "alloy-consensus", "alloy-dyn-abi", - "alloy-json-abi 1.4.0", + "alloy-json-abi 1.4.1", "alloy-network", "alloy-network-primitives", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-provider", "alloy-rpc-types-eth", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "alloy-transport", "futures", "futures-util", @@ -197,27 +188,27 @@ dependencies = [ [[package]] name = "alloy-core" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "575053cea24ea8cb7e775e39d5c53c33b19cfd0ca1cf6c0fd653f3d8c682095f" +checksum = "5ca96214615ec8cf3fa2a54b32f486eb49100ca7fe7eb0b8c1137cd316e7250a" dependencies = [ "alloy-dyn-abi", - "alloy-json-abi 1.4.0", - "alloy-primitives 1.4.0", + "alloy-json-abi 1.4.1", + "alloy-primitives 1.4.1", "alloy-rlp", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", ] [[package]] name = "alloy-dyn-abi" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6c2905bafc2df7ccd32ca3af13f0b0d82f2e2ff9dfbeb12196c0d978d5c0deb" +checksum = "3fdff496dd4e98a81f4861e66f7eaf5f2488971848bb42d9c892f871730245c8" dependencies = [ - "alloy-json-abi 1.4.0", - "alloy-primitives 1.4.0", - "alloy-sol-type-parser 1.4.0", - "alloy-sol-types 1.4.0", + "alloy-json-abi 1.4.1", + "alloy-primitives 1.4.1", + "alloy-sol-type-parser 1.4.1", + "alloy-sol-types 1.4.1", "derive_more", "itoa", "serde", @@ -231,7 +222,7 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "741bdd7499908b3aa0b159bba11e71c8cddd009a2c2eb7a06e825f1ec87900a5" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "arbitrary", "crc", @@ -246,7 +237,7 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b82752a889170df67bbb36d42ca63c531eb16274f0d7299ae2a680facba17bd" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "arbitrary", "rand 0.8.5", @@ -259,7 +250,7 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d4769c6ffddca380b0070d71c8b7f30bed375543fe76bb2f74ec0acf4b7cd16" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "arbitrary", "k256", @@ -278,7 +269,7 @@ dependencies = [ "alloy-eip2124", "alloy-eip2930", "alloy-eip7702", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-serde", "arbitrary", @@ -290,23 +281,24 @@ dependencies = [ "ethereum_ssz_derive", "serde", "serde_with", - "sha2", + "sha2 0.10.9", "thiserror 2.0.17", ] [[package]] name = "alloy-evm" version = "0.21.2" -source = "git+https://github.com/noot/evm?rev=f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e#f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06a5f67ee74999aa4fe576a83be1996bdf74a30fce3d248bf2007d6fc7dae8aa" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-hardforks", "alloy-op-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "alloy-rpc-types-eth", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "auto_impl", "derive_more", "op-alloy-consensus", @@ -323,7 +315,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a272533715aefc900f89d51db00c96e6fd4f517ea081a12fea482a352c8c815c" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-serde", "alloy-trie", "serde", @@ -338,7 +330,7 @@ checksum = "889eb3949b58368a09d4f16931c660275ef5fb08e5fbd4a96573b19c7085c41f" dependencies = [ "alloy-chains", "alloy-eip2124", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "dyn-clone", "serde", @@ -346,24 +338,24 @@ dependencies = [ [[package]] name = "alloy-json-abi" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe6beff64ad0aa6ad1019a3db26fef565aefeb011736150ab73ed3366c3cfd1b" +checksum = "4584e3641181ff073e9d5bec5b3b8f78f9749d9fb108a1cfbc4399a4a139c72a" dependencies = [ - "alloy-primitives 0.8.25", - "alloy-sol-type-parser 0.8.25", + "alloy-primitives 0.8.26", + "alloy-sol-type-parser 0.8.26", "serde", "serde_json", ] [[package]] name = "alloy-json-abi" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2acb6637a9c0e1cdf8971e0ced8f3fa34c04c5e9dccf6bb184f6a64fe0e37d8" +checksum = "5513d5e6bd1cba6bdcf5373470f559f320c05c8c59493b6e98912fbe6733943f" dependencies = [ - "alloy-primitives 1.4.0", - "alloy-sol-type-parser 1.4.0", + "alloy-primitives 1.4.1", + "alloy-sol-type-parser 1.4.1", "serde", "serde_json", ] @@ -374,8 +366,8 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d91676d242c0ced99c0dd6d0096d7337babe9457cc43407d26aa6367fcf90553" dependencies = [ - "alloy-primitives 1.4.0", - "alloy-sol-types 1.4.0", + "alloy-primitives 1.4.1", + "alloy-sol-types 1.4.1", "http", "serde", "serde_json", @@ -394,12 +386,12 @@ dependencies = [ "alloy-eips", "alloy-json-rpc", "alloy-network-primitives", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-any", "alloy-rpc-types-eth", "alloy-serde", "alloy-signer", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "async-trait", "auto_impl", "derive_more", @@ -417,7 +409,7 @@ checksum = "223612259a080160ce839a4e5df0125ca403a1d5e7206cc911cea54af5d769aa" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-serde", "serde", ] @@ -425,13 +417,14 @@ dependencies = [ [[package]] name = "alloy-op-evm" version = "0.21.2" -source = "git+https://github.com/noot/evm?rev=f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e#f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17aaeb600740c181bf29c9f138f9b228d115ea74fa6d0f0343e1952f1a766968" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-op-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "op-alloy-consensus", "op-revm", @@ -446,15 +439,15 @@ checksum = "599c1d7dfbccb66603cb93fde00980d12848d32fe5e814f50562104a92df6487" dependencies = [ "alloy-chains", "alloy-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", ] [[package]] name = "alloy-primitives" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c77490fe91a0ce933a1f219029521f20fc28c2c0ca95d53fa4da9c00b8d9d4e" +checksum = "777d58b30eb9a4db0e5f59bc30e8c2caef877fee7dc8734cf242a51a60f22e05" dependencies = [ "alloy-rlp", "bytes", @@ -479,9 +472,9 @@ dependencies = [ [[package]] name = "alloy-primitives" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b77f7d5e60ad8ae6bd2200b8097919712a07a6db622a4b201e7ead6166f02e5" +checksum = "355bf68a433e0fd7f7d33d5a9fc2583fde70bf5c530f63b80845f8da5505cf28" dependencies = [ "alloy-rlp", "arbitrary", @@ -490,7 +483,7 @@ dependencies = [ "const-hex", "derive_more", "foldhash 0.2.0", - "getrandom 0.3.3", + "getrandom 0.3.4", "hashbrown 0.16.0", "indexmap 2.11.4", "itoa", @@ -519,14 +512,14 @@ dependencies = [ "alloy-json-rpc", "alloy-network", "alloy-network-primitives", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-pubsub", "alloy-rpc-client", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-rpc-types-txpool", "alloy-signer", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "alloy-transport", "alloy-transport-http", "alloy-transport-ipc", @@ -558,7 +551,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eee7e3d343814ec0dfea69bd1820042a133a9d0b9ac5faf1e6eb133b43366315" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-transport", "auto_impl", "bimap", @@ -602,7 +595,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1154b12d470bef59951c62676e106f4ce5de73b987d86b9faa935acebb138ded" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-pubsub", "alloy-transport", "alloy-transport-http", @@ -627,7 +620,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "47ab76bf97648a1c6ad8fb00f0d594618942b5a9e008afbfb5c8a8fca800d574" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-serde", @@ -641,7 +634,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af8ae38824376e855d73d4060462d86c32afe548af632597ccfd161bdd0fc628" dependencies = [ "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "serde", "serde_json", ] @@ -652,7 +645,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "456cfc2c1677260edbd7ce3eddb7de419cb46de0e9826c43401f42b0286a779a" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -676,7 +669,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cfa4edd92c3124ec19b9d572dc7923d070fe5c2efb677519214affd6156a4463" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "derive_more", "ethereum_ssz", @@ -695,7 +688,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a0ac29dd005c33e3f7e09087accc80843315303685c3f7a1b888002cd27785b" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "derive_more", "serde", "serde_with", @@ -709,7 +702,7 @@ checksum = "1d9d173854879bcf26c7d71c1c3911972a3314df526f4349ffe488e676af577d" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-serde", "derive_more", @@ -731,10 +724,10 @@ dependencies = [ "alloy-consensus-any", "alloy-eips", "alloy-network-primitives", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-serde", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "arbitrary", "itertools 0.14.0", "serde", @@ -751,7 +744,7 @@ checksum = "d3820683ece7cdc31e44d87c88c0ff9b972a1a2fd1f2124cc72ce5c928e64f0d" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -764,7 +757,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c331c8e48665607682e8a9549a2347c13674d4fbcbdc342e7032834eba2424f4" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -778,7 +771,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5e2f66afe1e76ca4485e593980056f061b2bdae2055486a062fca050ff111a52" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-serde", "serde", @@ -790,7 +783,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8468f1a7f9ee3bae73c24eead0239abea720dbf7779384b9c7e20d51bfb6b0" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "arbitrary", "serde", "serde_json", @@ -802,7 +795,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33387c90b0a5021f45a5a77c2ce6c49b8f6980e66a318181468fb24cea771670" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "async-trait", "auto_impl", "either", @@ -819,7 +812,7 @@ checksum = "b55d9e795c85e36dcea08786d2e7ae9b73cb554b6bea6ac4c212def24e1b4d03" dependencies = [ "alloy-consensus", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-signer", "async-trait", "k256", @@ -829,12 +822,12 @@ dependencies = [ [[package]] name = "alloy-sol-macro" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e10ae8e9a91d328ae954c22542415303919aabe976fe7a92eb06db1b68fd59f2" +checksum = "e68b32b6fa0d09bb74b4cefe35ccc8269d711c26629bc7cd98a47eeb12fe353f" dependencies = [ - "alloy-sol-macro-expander 0.8.25", - "alloy-sol-macro-input 0.8.25", + "alloy-sol-macro-expander 0.8.26", + "alloy-sol-macro-input 0.8.26", "proc-macro-error2", "proc-macro2", "quote", @@ -843,12 +836,12 @@ dependencies = [ [[package]] name = "alloy-sol-macro" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78c84c3637bee9b5c4a4d2b93360ee16553d299c3b932712353caf1cea76d0e6" +checksum = "f3ce480400051b5217f19d6e9a82d9010cdde20f1ae9c00d53591e4a1afbb312" dependencies = [ - "alloy-sol-macro-expander 1.4.0", - "alloy-sol-macro-input 1.4.0", + "alloy-sol-macro-expander 1.4.1", + "alloy-sol-macro-input 1.4.1", "proc-macro-error2", "proc-macro2", "quote", @@ -857,11 +850,11 @@ dependencies = [ [[package]] name = "alloy-sol-macro-expander" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83ad5da86c127751bc607c174d6c9fe9b85ef0889a9ca0c641735d77d4f98f26" +checksum = "2afe6879ac373e58fd53581636f2cce843998ae0b058ebe1e4f649195e2bd23c" dependencies = [ - "alloy-sol-macro-input 0.8.25", + "alloy-sol-macro-input 0.8.26", "const-hex", "heck", "indexmap 2.11.4", @@ -869,18 +862,18 @@ dependencies = [ "proc-macro2", "quote", "syn 2.0.106", - "syn-solidity 0.8.25", + "syn-solidity 0.8.26", "tiny-keccak", ] [[package]] name = "alloy-sol-macro-expander" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a882aa4e1790063362434b9b40d358942b188477ac1c44cfb8a52816ffc0cc17" +checksum = "6d792e205ed3b72f795a8044c52877d2e6b6e9b1d13f431478121d8d4eaa9028" dependencies = [ - "alloy-json-abi 1.4.0", - "alloy-sol-macro-input 1.4.0", + "alloy-json-abi 1.4.1", + "alloy-sol-macro-input 1.4.1", "const-hex", "heck", "indexmap 2.11.4", @@ -888,15 +881,15 @@ dependencies = [ "proc-macro2", "quote", "syn 2.0.106", - "syn-solidity 1.4.0", + "syn-solidity 1.4.1", "tiny-keccak", ] [[package]] name = "alloy-sol-macro-input" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba3d30f0d3f9ba3b7686f3ff1de9ee312647aac705604417a2f40c604f409a9e" +checksum = "c3ba01aee235a8c699d07e5be97ba215607564e71be72f433665329bec307d28" dependencies = [ "const-hex", "dunce", @@ -905,16 +898,16 @@ dependencies = [ "proc-macro2", "quote", "syn 2.0.106", - "syn-solidity 0.8.25", + "syn-solidity 0.8.26", ] [[package]] name = "alloy-sol-macro-input" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18e5772107f9bb265d8d8c86e0733937bb20d0857ea5425b1b6ddf51a9804042" +checksum = "0bd1247a8f90b465ef3f1207627547ec16940c35597875cdc09c49d58b19693c" dependencies = [ - "alloy-json-abi 1.4.0", + "alloy-json-abi 1.4.1", "const-hex", "dunce", "heck", @@ -923,14 +916,14 @@ dependencies = [ "quote", "serde_json", "syn 2.0.106", - "syn-solidity 1.4.0", + "syn-solidity 1.4.1", ] [[package]] name = "alloy-sol-type-parser" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d162f8524adfdfb0e4bd0505c734c985f3e2474eb022af32eef0d52a4f3935c" +checksum = "4c13fc168b97411e04465f03e632f31ef94cad1c7c8951bf799237fd7870d535" dependencies = [ "serde", "winnow", @@ -938,9 +931,9 @@ dependencies = [ [[package]] name = "alloy-sol-type-parser" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e188b939aa4793edfaaa099cb1be4e620036a775b4bdf24fdc56f1cd6fd45890" +checksum = "954d1b2533b9b2c7959652df3076954ecb1122a28cc740aa84e7b0a49f6ac0a9" dependencies = [ "serde", "winnow", @@ -948,26 +941,26 @@ dependencies = [ [[package]] name = "alloy-sol-types" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d43d5e60466a440230c07761aa67671d4719d46f43be8ea6e7ed334d8db4a9ab" +checksum = "6e960c4b52508ef2ae1e37cae5058e905e9ae099b107900067a503f8c454036f" dependencies = [ - "alloy-json-abi 0.8.25", - "alloy-primitives 0.8.25", - "alloy-sol-macro 0.8.25", + "alloy-json-abi 0.8.26", + "alloy-primitives 0.8.26", + "alloy-sol-macro 0.8.26", "const-hex", "serde", ] [[package]] name = "alloy-sol-types" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3c8a9a909872097caffc05df134e5ef2253a1cdb56d3a9cf0052a042ac763f9" +checksum = "70319350969a3af119da6fb3e9bddb1bce66c9ea933600cb297c8b1850ad2a3c" dependencies = [ - "alloy-json-abi 1.4.0", - "alloy-primitives 1.4.0", - "alloy-sol-macro 1.4.0", + "alloy-json-abi 1.4.1", + "alloy-primitives 1.4.1", + "alloy-sol-macro 1.4.1", "serde", ] @@ -978,7 +971,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "702002659778d89a94cd4ff2044f6b505460df6c162e2f47d1857573845b0ace" dependencies = [ "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "base64 0.22.1", "derive_more", @@ -1003,10 +996,6 @@ checksum = "0d6bdc0830e5e8f08a4c70a4c791d400a86679c694a3b4b986caf26fad680438" dependencies = [ "alloy-json-rpc", "alloy-transport", - "http-body-util", - "hyper", - "hyper-tls", - "hyper-util", "reqwest", "serde_json", "tower 0.5.2", @@ -1058,7 +1047,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3412d52bb97c6c6cc27ccc28d4e6e8cf605469101193b50b0bd5813b1f990b5" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "arbitrary", "arrayvec", @@ -1078,7 +1067,7 @@ version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7bf39928a5e70c9755d6811a2928131b53ba785ad37c8bf85c90175b5d43b818" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "darling 0.21.3", "proc-macro2", "quote", @@ -1844,21 +1833,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "backtrace" -version = "0.3.76" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb531853791a215d7c62a30daf0dde835f381ab5de4589cfe7c649d2cbe92bd6" -dependencies = [ - "addr2line", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", - "windows-link 0.2.1", -] - [[package]] name = "base-x" version = "0.2.11" @@ -2076,6 +2050,15 @@ dependencies = [ "digest 0.10.7", ] +[[package]] +name = "block-buffer" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" +dependencies = [ + "generic-array", +] + [[package]] name = "block-buffer" version = "0.10.4" @@ -2190,7 +2173,7 @@ dependencies = [ "hashbrown 0.15.5", "indexmap 2.11.4", "once_cell", - "phf 0.11.3", + "phf", "rustc-hash 2.1.1", "static_assertions", ] @@ -2591,9 +2574,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.48" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2134bb3ea021b78629caa971416385309e0131b351b25e01dc16fb54e1b5fae" +checksum = "f4512b90fa68d3a9932cea5184017c5d200f5921df706d45e853537dea51508f" dependencies = [ "clap_builder", "clap_derive", @@ -2601,9 +2584,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.48" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2ba64afa3c0a6df7fa517765e31314e983f51dda798ffba27b988194fb65dc9" +checksum = "0025e98baa12e766c67ba13ff4695a887a1eba19569aad00a472546795bd6730" dependencies = [ "anstream", "anstyle", @@ -2613,9 +2596,9 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.47" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbfd7eae0b0f1a6e63d4b13c9c478de77c2eb546fba158ad50b4203dc24b9f9c" +checksum = "2a0b5487afeab2deb2ff4e03a807ad1a03ac532ff5a2cee5d86884440c7f7671" dependencies = [ "heck", "proc-macro2", @@ -2625,9 +2608,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.5" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" +checksum = "a1d728cc89cf3aee9ff92b05e62b19ee65a02b5702cff7d5a377e32c6ae29d8d" [[package]] name = "cmake" @@ -2738,9 +2721,9 @@ dependencies = [ [[package]] name = "const-hex" -version = "1.16.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6407bff74dea37e0fa3dc1c1c974e5d46405f0c987bf9997a0762adce71eda6" +checksum = "3bb320cac8a0750d7f25280aa97b09c26edfe161164238ecbbb31092b079e735" dependencies = [ "cfg-if", "cpufeatures", @@ -3146,13 +3129,13 @@ name = "dcap-rs" version = "0.1.0" source = "git+https://github.com/automata-network/dcap-rs.git?rev=d847b8f75a493640c4881bdf67775250b6baefab#d847b8f75a493640c4881bdf67775250b6baefab" dependencies = [ - "alloy-sol-types 0.8.25", + "alloy-sol-types 0.8.26", "chrono", "hex", "p256", "serde", "serde_json", - "sha2", + "sha2 0.10.9", "sha3", "time", "x509-parser 0.15.1", @@ -3331,7 +3314,7 @@ version = "0.10.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ - "block-buffer", + "block-buffer 0.10.4", "const-oid", "crypto-common", "subtle", @@ -3523,7 +3506,7 @@ dependencies = [ "ed25519", "rand_core 0.6.4", "serde", - "sha2", + "sha2 0.10.9", "subtle", "zeroize", ] @@ -3712,7 +3695,7 @@ checksum = "c853bd72c9e5787f8aafc3df2907c2ed03cff3150c3acd94e2e53a98ab70a8ab" dependencies = [ "cpufeatures", "ring", - "sha2", + "sha2 0.10.9", ] [[package]] @@ -3721,7 +3704,7 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3dc1355dbb41fbbd34ec28d4fb2a57d9a70c67ac3c19f6a5ca4d4a176b9e997a" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "hex", "serde", "serde_derive", @@ -3734,7 +3717,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0dcddb2554d19cde19b099fadddde576929d7a4d0c1cd3512d1fd95cf174375c" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "ethereum_serde_utils", "itertools 0.13.0", "serde", @@ -4082,9 +4065,9 @@ checksum = "42012b0f064e01aa58b545fe3727f90f7dd4020f4a3ea735b50344965f5a57e9" [[package]] name = "generic-array" -version = "0.14.7" +version = "0.14.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +checksum = "4bb6743198531e02858aeaea5398fcc883e71851fcbcb5a2f773e2fb6cb1edf2" dependencies = [ "serde", "typenum", @@ -4101,21 +4084,21 @@ dependencies = [ "cfg-if", "js-sys", "libc", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi", "wasm-bindgen", ] [[package]] name = "getrandom" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" dependencies = [ "cfg-if", "js-sys", "libc", "r-efi", - "wasi 0.14.7+wasi-0.2.4", + "wasip2", "wasm-bindgen", ] @@ -4129,12 +4112,6 @@ dependencies = [ "polyval", ] -[[package]] -name = "gimli" -version = "0.32.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e629b9b98ef3dd8afe6ca2bd0f89306cec16d43d907889945bc5d6687f2f13c7" - [[package]] name = "git2" version = "0.20.2" @@ -4595,7 +4572,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.6.0", + "socket2 0.6.1", "system-configuration", "tokio", "tower-service", @@ -5079,17 +5056,6 @@ dependencies = [ "memoffset", ] -[[package]] -name = "io-uring" -version = "0.7.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" -dependencies = [ - "bitflags 2.9.4", - "cfg-if", - "libc", -] - [[package]] name = "iocuddle" version = "0.1.1" @@ -5200,7 +5166,7 @@ version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9afb3de4395d6b3e67a780b6de64b51c978ecf11cb9a462c66be7d4ca9039d33" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "libc", ] @@ -5214,6 +5180,20 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "jsonrpsee" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "jsonrpsee-core 0.25.1", + "jsonrpsee-http-client 0.25.1", + "jsonrpsee-proc-macros 0.25.1", + "jsonrpsee-server 0.25.1", + "jsonrpsee-types 0.25.1", + "tokio", + "tracing", +] + [[package]] name = "jsonrpsee" version = "0.26.0" @@ -5221,11 +5201,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f3f48dc3e6b8bd21e15436c1ddd0bc22a6a54e8ec46fedd6adf3425f396ec6a" dependencies = [ "jsonrpsee-client-transport", - "jsonrpsee-core", - "jsonrpsee-http-client", - "jsonrpsee-proc-macros", - "jsonrpsee-server", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-http-client 0.26.0", + "jsonrpsee-proc-macros 0.26.0", + "jsonrpsee-server 0.26.0", + "jsonrpsee-types 0.26.0", "jsonrpsee-wasm-client", "jsonrpsee-ws-client", "tokio", @@ -5243,7 +5223,7 @@ dependencies = [ "futures-util", "gloo-net", "http", - "jsonrpsee-core", + "jsonrpsee-core 0.26.0", "pin-project", "rustls", "rustls-pki-types", @@ -5257,6 +5237,30 @@ dependencies = [ "url", ] +[[package]] +name = "jsonrpsee-core" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http", + "http-body", + "http-body-util", + "jsonrpsee-types 0.25.1", + "parking_lot", + "pin-project", + "rand 0.9.2", + "rustc-hash 2.1.1", + "serde", + "serde_json", + "thiserror 2.0.17", + "tokio", + "tower 0.5.2", + "tracing", +] + [[package]] name = "jsonrpsee-core" version = "0.26.0" @@ -5270,7 +5274,7 @@ dependencies = [ "http", "http-body", "http-body-util", - "jsonrpsee-types", + "jsonrpsee-types 0.26.0", "parking_lot", "pin-project", "rand 0.9.2", @@ -5285,6 +5289,28 @@ dependencies = [ "wasm-bindgen-futures", ] +[[package]] +name = "jsonrpsee-http-client" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "base64 0.22.1", + "http-body", + "hyper", + "hyper-rustls", + "hyper-util", + "jsonrpsee-core 0.25.1", + "jsonrpsee-types 0.25.1", + "rustls", + "rustls-platform-verifier", + "serde", + "serde_json", + "thiserror 2.0.17", + "tokio", + "tower 0.5.2", + "url", +] + [[package]] name = "jsonrpsee-http-client" version = "0.26.0" @@ -5296,8 +5322,8 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "rustls", "rustls-platform-verifier", "serde", @@ -5308,6 +5334,18 @@ dependencies = [ "url", ] +[[package]] +name = "jsonrpsee-proc-macros" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "heck", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "jsonrpsee-proc-macros" version = "0.26.0" @@ -5321,6 +5359,32 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "jsonrpsee-server" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "futures-util", + "http", + "http-body", + "http-body-util", + "hyper", + "hyper-util", + "jsonrpsee-core 0.25.1", + "jsonrpsee-types 0.25.1", + "pin-project", + "route-recognizer", + "serde", + "serde_json", + "soketto", + "thiserror 2.0.17", + "tokio", + "tokio-stream", + "tokio-util", + "tower 0.5.2", + "tracing", +] + [[package]] name = "jsonrpsee-server" version = "0.26.0" @@ -5333,8 +5397,8 @@ dependencies = [ "http-body-util", "hyper", "hyper-util", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "pin-project", "route-recognizer", "serde", @@ -5348,6 +5412,17 @@ dependencies = [ "tracing", ] +[[package]] +name = "jsonrpsee-types" +version = "0.25.1" +source = "git+https://github.com/paritytech/jsonrpsee?rev=f04afa740e55db60dce20d9839758792f035ffff#f04afa740e55db60dce20d9839758792f035ffff" +dependencies = [ + "http", + "serde", + "serde_json", + "thiserror 2.0.17", +] + [[package]] name = "jsonrpsee-types" version = "0.26.0" @@ -5367,8 +5442,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7902885de4779f711a95d82c8da2d7e5f9f3a7c7cfa44d51c067fd1c29d72a3c" dependencies = [ "jsonrpsee-client-transport", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "tower 0.5.2", ] @@ -5380,8 +5455,8 @@ checksum = "9b6fceceeb05301cc4c065ab3bd2fa990d41ff4eb44e4ca1b30fa99c057c3e79" dependencies = [ "http", "jsonrpsee-client-transport", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "tower 0.5.2", "url", ] @@ -5412,7 +5487,7 @@ dependencies = [ "elliptic-curve", "once_cell", "serdect", - "sha2", + "sha2 0.10.9", "signature", ] @@ -5658,7 +5733,7 @@ dependencies = [ "multihash", "quick-protobuf", "rand 0.8.5", - "sha2", + "sha2 0.10.9", "thiserror 2.0.17", "tracing", "zeroize", @@ -5914,6 +5989,52 @@ dependencies = [ "redox_syscall 0.5.18", ] +[[package]] +name = "libsecp256k1" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e79019718125edc905a079a70cfa5f3820bc76139fc91d6f9abc27ea2a887139" +dependencies = [ + "arrayref", + "base64 0.22.1", + "digest 0.9.0", + "libsecp256k1-core", + "libsecp256k1-gen-ecmult", + "libsecp256k1-gen-genmult", + "rand 0.8.5", + "serde", + "sha2 0.9.9", +] + +[[package]] +name = "libsecp256k1-core" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5be9b9bb642d8522a44d533eab56c16c738301965504753b03ad1de3425d5451" +dependencies = [ + "crunchy", + "digest 0.9.0", + "subtle", +] + +[[package]] +name = "libsecp256k1-gen-ecmult" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3038c808c55c87e8a172643a7d87187fc6c4174468159cb3090659d55bcb4809" +dependencies = [ + "libsecp256k1-core", +] + +[[package]] +name = "libsecp256k1-gen-genmult" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3db8d6ba2cec9eacc40e6e8ccc98931840301f1006e95647ceb2dd5c3aa06f7c" +dependencies = [ + "libsecp256k1-core", +] + [[package]] name = "libz-sys" version = "1.1.22" @@ -6304,7 +6425,7 @@ checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" dependencies = [ "libc", "log", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi", "windows-sys 0.59.0", ] @@ -6746,15 +6867,6 @@ dependencies = [ "objc2-core-foundation", ] -[[package]] -name = "object" -version = "0.37.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff76201f031d8863c38aa7f905eca4f53abbfa15f609db4277d44cd8938f33fe" -dependencies = [ - "memchr", -] - [[package]] name = "oid" version = "0.2.1" @@ -6807,7 +6919,7 @@ dependencies = [ "alloy-consensus", "alloy-eips", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", @@ -6832,7 +6944,7 @@ checksum = "f80108e3b36901200a4c5df1db1ee9ef6ce685b59ea79d7be1713c845e3765da" dependencies = [ "alloy-consensus", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-provider", "alloy-rpc-types-eth", "alloy-signer", @@ -6846,8 +6958,8 @@ version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8eb878fc5ea95adb5abe55fb97475b3eb0dcc77dfcd6f61bd626a68ae0bdba1" dependencies = [ - "alloy-primitives 1.4.0", - "jsonrpsee", + "alloy-primitives 1.4.1", + "jsonrpsee 0.26.0", ] [[package]] @@ -6859,7 +6971,7 @@ dependencies = [ "alloy-consensus", "alloy-eips", "alloy-network-primitives", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-serde", "derive_more", @@ -6877,7 +6989,7 @@ checksum = "14e50c94013a1d036a529df259151991dbbd6cf8dc215e3b68b784f95eec60e6" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-engine", "alloy-serde", @@ -6892,7 +7004,7 @@ dependencies = [ [[package]] name = "op-rbuilder" -version = "0.2.4" +version = "0.2.6" dependencies = [ "alloy-consensus", "alloy-contract", @@ -6901,7 +7013,7 @@ dependencies = [ "alloy-json-rpc", "alloy-network", "alloy-op-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-provider", "alloy-rpc-client", "alloy-rpc-types-beacon", @@ -6909,7 +7021,7 @@ dependencies = [ "alloy-rpc-types-eth", "alloy-serde", "alloy-signer-local", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "alloy-transport", "alloy-transport-http", "anyhow", @@ -6929,9 +7041,9 @@ dependencies = [ "http-body-util", "hyper", "hyper-util", - "jsonrpsee", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee 0.26.0", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "k256", "macros", "metrics", @@ -6949,51 +7061,51 @@ dependencies = [ "rand 0.9.2", "reqwest", "reth", - "reth-basic-payload-builder", - "reth-chain-state", - "reth-chainspec", + "reth-basic-payload-builder 1.8.2", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", "reth-cli", "reth-cli-commands", "reth-cli-util", "reth-db", - "reth-evm", - "reth-execution-types", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", "reth-exex", "reth-ipc", - "reth-metrics", - "reth-network-peers", + "reth-metrics 1.8.2", + "reth-network-peers 1.8.2", "reth-node-api", "reth-node-builder", "reth-node-core", "reth-node-ethereum", - "reth-optimism-chainspec", + "reth-optimism-chainspec 1.8.2", "reth-optimism-cli", - "reth-optimism-consensus", - "reth-optimism-evm", - "reth-optimism-forks", + "reth-optimism-consensus 1.8.2", + "reth-optimism-evm 1.8.2", + "reth-optimism-forks 1.8.2", "reth-optimism-node", - "reth-optimism-payload-builder", - "reth-optimism-primitives", + "reth-optimism-payload-builder 1.8.2", + "reth-optimism-primitives 1.8.2", "reth-optimism-rpc", - "reth-optimism-txpool", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-payload-util", + "reth-optimism-txpool 1.8.2", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-payload-util 1.8.2", "reth-primitives", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-revm", + "reth-revm 1.8.2", "reth-rpc-api", "reth-rpc-engine-api", "reth-rpc-eth-types", "reth-rpc-layer", - "reth-storage-api", - "reth-tasks", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "reth-testing-utils", "reth-tracing-otlp", - "reth-transaction-pool", - "reth-trie", + "reth-transaction-pool 1.8.2", + "reth-trie 1.8.2", "revm", "rlimit", "rollup-boost", @@ -7025,8 +7137,9 @@ dependencies = [ [[package]] name = "op-revm" -version = "11.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "10.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "826f43a5b1613c224f561847c152bfbaefcb593a9ae2c612ff4dc4661c6e625f" dependencies = [ "auto_impl", "revm", @@ -7275,12 +7388,12 @@ dependencies = [ "ecdsa", "elliptic-curve", "primeorder", - "sha2", + "sha2 0.10.9", ] [[package]] name = "p2p" -version = "0.2.4" +version = "0.2.6" dependencies = [ "derive_more", "eyre", @@ -7448,18 +7561,8 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" dependencies = [ - "phf_macros 0.11.3", - "phf_shared 0.11.3", -] - -[[package]] -name = "phf" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1562dc717473dbaa4c1f85a36410e03c047b2e7df7f45ee938fbef64ae7fadf" -dependencies = [ - "phf_macros 0.13.1", - "phf_shared 0.13.1", + "phf_macros", + "phf_shared", "serde", ] @@ -7469,41 +7572,18 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" dependencies = [ - "phf_shared 0.11.3", + "phf_shared", "rand 0.8.5", ] -[[package]] -name = "phf_generator" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "135ace3a761e564ec88c03a77317a7c6b80bb7f7135ef2544dbe054243b89737" -dependencies = [ - "fastrand", - "phf_shared 0.13.1", -] - [[package]] name = "phf_macros" version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f84ac04429c13a7ff43785d75ad27569f2951ce0ffd30a3321230db2fc727216" dependencies = [ - "phf_generator 0.11.3", - "phf_shared 0.11.3", - "proc-macro2", - "quote", - "syn 2.0.106", -] - -[[package]] -name = "phf_macros" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "812f032b54b1e759ccd5f8b6677695d5268c588701effba24601f6932f8269ef" -dependencies = [ - "phf_generator 0.13.1", - "phf_shared 0.13.1", + "phf_generator", + "phf_shared", "proc-macro2", "quote", "syn 2.0.106", @@ -7518,15 +7598,6 @@ dependencies = [ "siphasher", ] -[[package]] -name = "phf_shared" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e57fef6bc5981e38c2ce2d63bfa546861309f875b8a75f092d1d54ae2d64f266" -dependencies = [ - "siphasher", -] - [[package]] name = "picky-asn1" version = "0.8.0" @@ -7937,7 +8008,7 @@ dependencies = [ "libc", "once_cell", "raw-cpuid", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi", "web-sys", "winapi", ] @@ -7984,7 +8055,7 @@ dependencies = [ "quinn-udp", "rustc-hash 2.1.1", "rustls", - "socket2 0.6.0", + "socket2 0.6.1", "thiserror 2.0.17", "tokio", "tracing", @@ -7998,7 +8069,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1906b49b0c3bc04b5fe5d86a77925ae6524a19b816ae38ce1e426255f1d8a31" dependencies = [ "bytes", - "getrandom 0.3.3", + "getrandom 0.3.4", "lru-slab", "rand 0.9.2", "ring", @@ -8021,7 +8092,7 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.6.0", + "socket2 0.6.1", "tracing", "windows-sys 0.60.2", ] @@ -8115,7 +8186,7 @@ version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "serde", ] @@ -8268,9 +8339,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.11.3" +version = "1.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b5288124840bee7b386bc413c487869b360b2b4ec421ea56425128692f2a82c" +checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" dependencies = [ "aho-corasick", "memchr", @@ -8280,9 +8351,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.11" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "833eb9ce86d40ef33cb1306d8accf7bc8ec2bfea4355cbdebb3df68b40925cad" +checksum = "5276caf25ac86c8d810222b3dbb938e512c55c6831a10f3e6ed1c93b84041f1c" dependencies = [ "aho-corasick", "memchr", @@ -8291,9 +8362,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" +checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" [[package]] name = "regress" @@ -8307,9 +8378,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.23" +version = "0.12.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" +checksum = "9d0946410b9f7b082a427e4ef5c8ff541a88b357bc6c637c40db3a68ac70a36f" dependencies = [ "base64 0.22.1", "bytes", @@ -8363,22 +8434,22 @@ checksum = "6b3789b30bd25ba102de4beabd95d21ac45b69b1be7d14522bab988c526d6799" [[package]] name = "reth" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-rpc-types", "aquamarine", "clap", "eyre", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-cli-runner", "reth-cli-util", - "reth-consensus", - "reth-consensus-common", + "reth-consensus 1.8.2", + "reth-consensus-common 1.8.2", "reth-db", "reth-ethereum-cli", "reth-ethereum-payload-builder", - "reth-ethereum-primitives", - "reth-evm", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-network", "reth-network-api", "reth-node-api", @@ -8386,22 +8457,46 @@ dependencies = [ "reth-node-core", "reth-node-ethereum", "reth-node-metrics", - "reth-payload-builder", - "reth-payload-primitives", + "reth-payload-builder 1.8.2", + "reth-payload-primitives 1.8.2", "reth-primitives", "reth-provider", "reth-ress-protocol", "reth-ress-provider", - "reth-revm", + "reth-revm 1.8.2", "reth-rpc", "reth-rpc-api", "reth-rpc-builder", "reth-rpc-convert", "reth-rpc-eth-types", "reth-rpc-server-types", - "reth-tasks", + "reth-tasks 1.8.2", "reth-tokio-util", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", + "tokio", + "tracing", +] + +[[package]] +name = "reth-basic-payload-builder" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "futures-core", + "futures-util", + "metrics", + "reth-chain-state 1.8.1", + "reth-metrics 1.8.1", + "reth-payload-builder 1.8.1", + "reth-payload-builder-primitives 1.8.1", + "reth-payload-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-revm 1.8.1", + "reth-storage-api 1.8.1", + "reth-tasks 1.8.1", "tokio", "tracing", ] @@ -8409,82 +8504,128 @@ dependencies = [ [[package]] name = "reth-basic-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "futures-core", "futures-util", "metrics", - "reth-chain-state", - "reth-metrics", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-primitives-traits", - "reth-revm", - "reth-storage-api", - "reth-tasks", + "reth-chain-state 1.8.2", + "reth-metrics 1.8.2", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "tokio", "tracing", ] [[package]] name = "reth-chain-state" -version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", - "alloy-signer", - "alloy-signer-local", + "alloy-primitives 1.4.1", "derive_more", "metrics", "parking_lot", "pin-project", - "rand 0.9.2", - "reth-chainspec", - "reth-errors", - "reth-ethereum-primitives", - "reth-execution-types", - "reth-metrics", - "reth-primitives-traits", - "reth-storage-api", - "reth-trie", + "reth-chainspec 1.8.1", + "reth-errors 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-execution-types 1.8.1", + "reth-metrics 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-trie 1.8.1", "revm-database", - "revm-state", - "serde", "tokio", "tokio-stream", "tracing", ] [[package]] -name = "reth-chainspec" +name = "reth-chain-state" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-chains", "alloy-consensus", "alloy-eips", - "alloy-evm", - "alloy-genesis", - "alloy-primitives 1.4.0", - "alloy-trie", - "auto_impl", - "derive_more", - "reth-ethereum-forks", - "reth-network-peers", - "reth-primitives-traits", + "alloy-primitives 1.4.1", + "alloy-signer", + "alloy-signer-local", + "derive_more", + "metrics", + "parking_lot", + "pin-project", + "rand 0.9.2", + "reth-chainspec 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-execution-types 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-trie 1.8.2", + "revm-database", + "revm-state", + "serde", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-chainspec" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-chains", + "alloy-consensus", + "alloy-eips", + "alloy-evm", + "alloy-genesis", + "alloy-primitives 1.4.1", + "alloy-trie", + "auto_impl", + "derive_more", + "reth-ethereum-forks 1.8.1", + "reth-network-peers 1.8.1", + "reth-primitives-traits 1.8.1", + "serde_json", +] + +[[package]] +name = "reth-chainspec" +version = "1.8.2" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" +dependencies = [ + "alloy-chains", + "alloy-consensus", + "alloy-eips", + "alloy-evm", + "alloy-genesis", + "alloy-primitives 1.4.1", + "alloy-trie", + "auto_impl", + "derive_more", + "reth-ethereum-forks 1.8.2", + "reth-network-peers 1.8.2", + "reth-primitives-traits 1.8.2", "serde_json", ] [[package]] name = "reth-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-genesis", "clap", @@ -8498,12 +8639,12 @@ dependencies = [ [[package]] name = "reth-cli-commands" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "backon", "clap", @@ -8518,13 +8659,13 @@ dependencies = [ "lz4", "ratatui", "reqwest", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-cli", "reth-cli-runner", "reth-cli-util", - "reth-codecs", + "reth-codecs 1.8.2", "reth-config", - "reth-consensus", + "reth-consensus 1.8.2", "reth-db", "reth-db-api", "reth-db-common", @@ -8537,27 +8678,27 @@ dependencies = [ "reth-era-utils", "reth-eth-wire", "reth-etl", - "reth-evm", + "reth-evm 1.8.2", "reth-exex", - "reth-fs-util", + "reth-fs-util 1.8.2", "reth-net-nat", "reth-network", "reth-network-p2p", - "reth-network-peers", + "reth-network-peers 1.8.2", "reth-node-api", "reth-node-builder", "reth-node-core", "reth-node-events", "reth-node-metrics", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", - "reth-revm", + "reth-revm 1.8.2", "reth-stages", "reth-static-file", - "reth-static-file-types", - "reth-trie", - "reth-trie-common", + "reth-static-file-types 1.8.2", + "reth-trie 1.8.2", + "reth-trie-common 1.8.2", "reth-trie-db", "secp256k1 0.30.0", "serde", @@ -8573,9 +8714,9 @@ dependencies = [ [[package]] name = "reth-cli-runner" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "reth-tasks", + "reth-tasks 1.8.2", "tokio", "tracing", ] @@ -8583,45 +8724,74 @@ dependencies = [ [[package]] name = "reth-cli-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "cfg-if", "eyre", "libc", "rand 0.8.5", - "reth-fs-util", + "reth-fs-util 1.8.2", "secp256k1 0.30.0", "serde", "thiserror 2.0.17", "tikv-jemallocator", ] +[[package]] +name = "reth-codecs" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-genesis", + "alloy-primitives 1.4.1", + "alloy-trie", + "bytes", + "modular-bitfield", + "op-alloy-consensus", + "reth-codecs-derive 1.8.1", + "reth-zstd-compressors 1.8.1", + "serde", +] + [[package]] name = "reth-codecs" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-trie", "arbitrary", "bytes", "modular-bitfield", "op-alloy-consensus", - "reth-codecs-derive", - "reth-zstd-compressors", + "reth-codecs-derive 1.8.2", + "reth-zstd-compressors 1.8.2", "serde", "visibility", ] +[[package]] +name = "reth-codecs-derive" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "convert_case", + "proc-macro2", + "quote", + "syn 2.0.106", +] + [[package]] name = "reth-codecs-derive" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "convert_case", "proc-macro2", @@ -8632,52 +8802,77 @@ dependencies = [ [[package]] name = "reth-config" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "eyre", "humantime-serde", "reth-network-types", - "reth-prune-types", - "reth-stages-types", + "reth-prune-types 1.8.2", + "reth-stages-types 1.8.2", "serde", "toml", "url", ] +[[package]] +name = "reth-consensus" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-primitives 1.4.1", + "auto_impl", + "reth-execution-types 1.8.1", + "reth-primitives-traits 1.8.1", + "thiserror 2.0.17", +] + [[package]] name = "reth-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", - "reth-execution-types", - "reth-primitives-traits", + "reth-execution-types 1.8.2", + "reth-primitives-traits 1.8.2", "thiserror 2.0.17", ] +[[package]] +name = "reth-consensus-common" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "reth-chainspec 1.8.1", + "reth-consensus 1.8.1", + "reth-primitives-traits 1.8.1", +] + [[package]] name = "reth-consensus-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "reth-chainspec", - "reth-consensus", - "reth-primitives-traits", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-primitives-traits 1.8.2", ] [[package]] name = "reth-consensus-debug-client" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-provider", "alloy-rpc-types-engine", "alloy-transport", @@ -8687,7 +8882,7 @@ dependencies = [ "futures", "reqwest", "reth-node-api", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-tracing", "ringbuffer", "serde", @@ -8698,21 +8893,21 @@ dependencies = [ [[package]] name = "reth-db" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "derive_more", "eyre", "metrics", "page_size", "parking_lot", "reth-db-api", - "reth-fs-util", + "reth-fs-util 1.8.2", "reth-libmdbx", - "reth-metrics", + "reth-metrics 1.8.2", "reth-nippy-jar", - "reth-static-file-types", - "reth-storage-errors", + "reth-static-file-types 1.8.2", + "reth-storage-errors 1.8.2", "reth-tracing", "rustc-hash 2.1.1", "strum 0.27.2", @@ -8724,11 +8919,11 @@ dependencies = [ [[package]] name = "reth-db-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "arbitrary", "bytes", "derive_more", @@ -8736,15 +8931,15 @@ dependencies = [ "modular-bitfield", "parity-scale-codec", "proptest", - "reth-codecs", - "reth-db-models", - "reth-ethereum-primitives", - "reth-optimism-primitives", - "reth-primitives-traits", - "reth-prune-types", - "reth-stages-types", - "reth-storage-errors", - "reth-trie-common", + "reth-codecs 1.8.2", + "reth-db-models 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", + "reth-stages-types 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie-common 1.8.2", "roaring", "serde", ] @@ -8752,26 +8947,26 @@ dependencies = [ [[package]] name = "reth-db-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "boyer-moore-magiclen", "eyre", - "reth-chainspec", - "reth-codecs", + "reth-chainspec 1.8.2", + "reth-codecs 1.8.2", "reth-config", "reth-db-api", "reth-etl", - "reth-execution-errors", - "reth-fs-util", + "reth-execution-errors 1.8.2", + "reth-fs-util 1.8.2", "reth-node-types", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-stages-types", - "reth-static-file-types", - "reth-trie", + "reth-stages-types 1.8.2", + "reth-static-file-types 1.8.2", + "reth-trie 1.8.2", "reth-trie-db", "serde", "serde_json", @@ -8779,27 +8974,37 @@ dependencies = [ "tracing", ] +[[package]] +name = "reth-db-models" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-eips", + "alloy-primitives 1.4.1", + "reth-primitives-traits 1.8.1", +] + [[package]] name = "reth-db-models" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "arbitrary", "bytes", "modular-bitfield", - "reth-codecs", - "reth-primitives-traits", + "reth-codecs 1.8.2", + "reth-primitives-traits 1.8.2", "serde", ] [[package]] name = "reth-discv4" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "discv5", "enr", @@ -8807,10 +9012,10 @@ dependencies = [ "itertools 0.14.0", "parking_lot", "rand 0.8.5", - "reth-ethereum-forks", + "reth-ethereum-forks 1.8.2", "reth-net-banlist", "reth-net-nat", - "reth-network-peers", + "reth-network-peers 1.8.2", "schnellru", "secp256k1 0.30.0", "serde", @@ -8823,9 +9028,9 @@ dependencies = [ [[package]] name = "reth-discv5" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "derive_more", "discv5", @@ -8834,10 +9039,10 @@ dependencies = [ "itertools 0.14.0", "metrics", "rand 0.9.2", - "reth-chainspec", - "reth-ethereum-forks", - "reth-metrics", - "reth-network-peers", + "reth-chainspec 1.8.2", + "reth-ethereum-forks 1.8.2", + "reth-metrics 1.8.2", + "reth-network-peers 1.8.2", "secp256k1 0.30.0", "thiserror 2.0.17", "tokio", @@ -8847,16 +9052,16 @@ dependencies = [ [[package]] name = "reth-dns-discovery" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "data-encoding", "enr", "hickory-resolver", "linked_hash_set", "parking_lot", - "reth-ethereum-forks", - "reth-network-peers", + "reth-ethereum-forks 1.8.2", + "reth-network-peers 1.8.2", "reth-tokio-util", "schnellru", "secp256k1 0.30.0", @@ -8871,11 +9076,11 @@ dependencies = [ [[package]] name = "reth-downloaders" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "futures", "futures-util", @@ -8884,16 +9089,16 @@ dependencies = [ "pin-project", "rayon", "reth-config", - "reth-consensus", + "reth-consensus 1.8.2", "reth-db", "reth-db-api", - "reth-ethereum-primitives", - "reth-metrics", + "reth-ethereum-primitives 1.8.2", + "reth-metrics 1.8.2", "reth-network-p2p", - "reth-network-peers", - "reth-primitives-traits", - "reth-storage-api", - "reth-tasks", + "reth-network-peers 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "reth-testing-utils", "tempfile", "thiserror 2.0.17", @@ -8906,10 +9111,10 @@ dependencies = [ [[package]] name = "reth-ecies" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "aes", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "block-padding", "byteorder", @@ -8922,9 +9127,9 @@ dependencies = [ "hmac", "pin-project", "rand 0.8.5", - "reth-network-peers", + "reth-network-peers 1.8.2", "secp256k1 0.30.0", - "sha2", + "sha2 0.10.9", "sha3", "thiserror 2.0.17", "tokio", @@ -8937,47 +9142,70 @@ dependencies = [ [[package]] name = "reth-engine-local" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "eyre", "futures-util", "op-alloy-rpc-types-engine", - "reth-chainspec", - "reth-engine-primitives", - "reth-ethereum-engine-primitives", - "reth-optimism-chainspec", - "reth-payload-builder", - "reth-payload-primitives", + "reth-chainspec 1.8.2", + "reth-engine-primitives 1.8.2", + "reth-ethereum-engine-primitives 1.8.2", + "reth-optimism-chainspec 1.8.2", + "reth-payload-builder 1.8.2", + "reth-payload-primitives 1.8.2", "reth-provider", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", "tokio", "tokio-stream", "tracing", ] +[[package]] +name = "reth-engine-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rpc-types-engine", + "auto_impl", + "reth-chain-state 1.8.1", + "reth-errors 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-evm 1.8.1", + "reth-execution-types 1.8.1", + "reth-payload-builder-primitives 1.8.1", + "reth-payload-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-trie-common 1.8.1", + "serde", + "thiserror 2.0.17", +] + [[package]] name = "reth-engine-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "auto_impl", "futures", - "reth-chain-state", - "reth-errors", - "reth-ethereum-primitives", - "reth-evm", - "reth-execution-types", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-primitives-traits", - "reth-trie-common", + "reth-chain-state 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-trie-common 1.8.2", "serde", "thiserror 2.0.17", "tokio", @@ -8986,35 +9214,35 @@ dependencies = [ [[package]] name = "reth-engine-service" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "futures", "pin-project", - "reth-chainspec", - "reth-consensus", - "reth-engine-primitives", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-engine-primitives 1.8.2", "reth-engine-tree", - "reth-ethereum-primitives", - "reth-evm", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-network-p2p", "reth-node-types", - "reth-payload-builder", + "reth-payload-builder 1.8.2", "reth-provider", "reth-prune", "reth-stages-api", - "reth-tasks", + "reth-tasks 1.8.2", "thiserror 2.0.17", ] [[package]] name = "reth-engine-tree" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-engine", "derive_more", @@ -9023,33 +9251,33 @@ dependencies = [ "mini-moka", "parking_lot", "rayon", - "reth-chain-state", - "reth-chainspec", - "reth-consensus", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", "reth-db", - "reth-engine-primitives", - "reth-errors", - "reth-ethereum-primitives", - "reth-evm", - "reth-execution-types", - "reth-metrics", + "reth-engine-primitives 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-metrics 1.8.2", "reth-network-p2p", - "reth-payload-builder", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-payload-builder 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", - "reth-prune-types", - "reth-revm", + "reth-prune-types 1.8.2", + "reth-revm 1.8.2", "reth-stages", "reth-stages-api", "reth-static-file", - "reth-tasks", + "reth-tasks 1.8.2", "reth-tracing", - "reth-trie", + "reth-trie 1.8.2", "reth-trie-db", "reth-trie-parallel", - "reth-trie-sparse", + "reth-trie-sparse 1.8.2", "reth-trie-sparse-parallel", "revm", "revm-primitives", @@ -9063,7 +9291,7 @@ dependencies = [ [[package]] name = "reth-engine-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", @@ -9071,16 +9299,16 @@ dependencies = [ "futures", "itertools 0.14.0", "pin-project", - "reth-chainspec", - "reth-engine-primitives", + "reth-chainspec 1.8.2", + "reth-engine-primitives 1.8.2", "reth-engine-tree", - "reth-errors", - "reth-evm", - "reth-fs-util", - "reth-payload-primitives", - "reth-primitives-traits", - "reth-revm", - "reth-storage-api", + "reth-errors 1.8.2", + "reth-evm 1.8.2", + "reth-fs-util 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", + "reth-storage-api 1.8.2", "serde", "serde_json", "tokio", @@ -9091,15 +9319,15 @@ dependencies = [ [[package]] name = "reth-era" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "ethereum_ssz", "ethereum_ssz_derive", - "reth-ethereum-primitives", + "reth-ethereum-primitives 1.8.2", "snap", "thiserror 2.0.17", ] @@ -9107,70 +9335,81 @@ dependencies = [ [[package]] name = "reth-era-downloader" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "bytes", "eyre", "futures-util", "reqwest", - "reth-fs-util", - "sha2", + "reth-fs-util 1.8.2", + "sha2 0.10.9", "tokio", ] [[package]] name = "reth-era-utils" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "eyre", "futures-util", "reth-db-api", "reth-era", "reth-era-downloader", "reth-etl", - "reth-fs-util", - "reth-primitives-traits", + "reth-fs-util 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-stages-types", - "reth-storage-api", + "reth-stages-types 1.8.2", + "reth-storage-api 1.8.2", "tokio", "tracing", ] +[[package]] +name = "reth-errors" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "reth-consensus 1.8.1", + "reth-execution-errors 1.8.1", + "reth-storage-errors 1.8.1", + "thiserror 2.0.17", +] + [[package]] name = "reth-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "reth-consensus", - "reth-execution-errors", - "reth-storage-errors", + "reth-consensus 1.8.2", + "reth-execution-errors 1.8.2", + "reth-storage-errors 1.8.2", "thiserror 2.0.17", ] [[package]] name = "reth-eth-wire" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-chains", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "bytes", "derive_more", "futures", "pin-project", - "reth-codecs", + "reth-codecs 1.8.2", "reth-ecies", - "reth-eth-wire-types", - "reth-ethereum-forks", - "reth-metrics", - "reth-network-peers", - "reth-primitives-traits", + "reth-eth-wire-types 1.8.2", + "reth-ethereum-forks 1.8.2", + "reth-metrics 1.8.2", + "reth-network-peers 1.8.2", + "reth-primitives-traits 1.8.2", "serde", "snap", "thiserror 2.0.17", @@ -9180,23 +9419,44 @@ dependencies = [ "tracing", ] +[[package]] +name = "reth-eth-wire-types" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-chains", + "alloy-consensus", + "alloy-eips", + "alloy-hardforks", + "alloy-primitives 1.4.1", + "alloy-rlp", + "bytes", + "derive_more", + "reth-chainspec 1.8.1", + "reth-codecs-derive 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "serde", + "thiserror 2.0.17", +] + [[package]] name = "reth-eth-wire-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", "alloy-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "bytes", "derive_more", - "reth-chainspec", - "reth-codecs-derive", - "reth-ethereum-primitives", - "reth-primitives-traits", + "reth-chainspec 1.8.2", + "reth-codecs-derive 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "serde", "thiserror 2.0.17", ] @@ -9204,11 +9464,11 @@ dependencies = [ [[package]] name = "reth-ethereum-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "clap", "eyre", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-cli", "reth-cli-commands", "reth-cli-runner", @@ -9226,45 +9486,76 @@ dependencies = [ [[package]] name = "reth-ethereum-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", - "reth-chainspec", - "reth-consensus", - "reth-consensus-common", - "reth-execution-types", - "reth-primitives-traits", + "alloy-primitives 1.4.1", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-consensus-common 1.8.2", + "reth-execution-types 1.8.2", + "reth-primitives-traits 1.8.2", "tracing", ] +[[package]] +name = "reth-ethereum-engine-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-rpc-types-engine", + "reth-engine-primitives 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-payload-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "serde", + "sha2 0.10.9", + "thiserror 2.0.17", +] + [[package]] name = "reth-ethereum-engine-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-engine", - "reth-engine-primitives", - "reth-ethereum-primitives", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-engine-primitives 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "serde", - "sha2", + "sha2 0.10.9", "thiserror 2.0.17", ] +[[package]] +name = "reth-ethereum-forks" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-eip2124", + "alloy-hardforks", + "alloy-primitives 1.4.1", + "auto_impl", + "once_cell", + "rustc-hash 2.1.1", +] + [[package]] name = "reth-ethereum-forks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eip2124", "alloy-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "once_cell", "rustc-hash 2.1.1", @@ -9273,48 +9564,65 @@ dependencies = [ [[package]] name = "reth-ethereum-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-engine", - "reth-basic-payload-builder", - "reth-chainspec", - "reth-consensus-common", - "reth-errors", - "reth-ethereum-primitives", - "reth-evm", + "reth-basic-payload-builder 1.8.2", + "reth-chainspec 1.8.2", + "reth-consensus-common 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-evm-ethereum", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-payload-validator", - "reth-primitives-traits", - "reth-revm", - "reth-storage-api", - "reth-transaction-pool", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-payload-validator 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", + "reth-storage-api 1.8.2", + "reth-transaction-pool 1.8.2", "revm", "tracing", ] +[[package]] +name = "reth-ethereum-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-rpc-types-eth", + "alloy-serde", + "reth-primitives-traits 1.8.1", + "reth-zstd-compressors 1.8.1", + "serde", + "serde_with", +] + [[package]] name = "reth-ethereum-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", "arbitrary", "modular-bitfield", - "reth-codecs", - "reth-primitives-traits", - "reth-zstd-compressors", + "reth-codecs 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-zstd-compressors 1.8.2", "serde", "serde_with", ] @@ -9322,82 +9630,132 @@ dependencies = [ [[package]] name = "reth-etl" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "rayon", "reth-db-api", "tempfile", ] +[[package]] +name = "reth-evm" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-evm", + "alloy-primitives 1.4.1", + "auto_impl", + "derive_more", + "futures-util", + "reth-execution-errors 1.8.1", + "reth-execution-types 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-storage-errors 1.8.1", + "reth-trie-common 1.8.1", + "revm", +] + [[package]] name = "reth-evm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "derive_more", "futures-util", "metrics", - "reth-execution-errors", - "reth-execution-types", - "reth-metrics", - "reth-primitives-traits", - "reth-storage-api", - "reth-storage-errors", - "reth-trie-common", + "reth-execution-errors 1.8.2", + "reth-execution-types 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie-common 1.8.2", "revm", ] [[package]] name = "reth-evm-ethereum" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", - "reth-chainspec", - "reth-ethereum-forks", - "reth-ethereum-primitives", - "reth-evm", - "reth-execution-types", - "reth-primitives-traits", - "reth-storage-errors", + "reth-chainspec 1.8.2", + "reth-ethereum-forks 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-errors 1.8.2", "revm", ] +[[package]] +name = "reth-execution-errors" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-evm", + "alloy-primitives 1.4.1", + "alloy-rlp", + "nybbles", + "reth-storage-errors 1.8.1", + "thiserror 2.0.17", +] + [[package]] name = "reth-execution-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "nybbles", - "reth-storage-errors", + "reth-storage-errors 1.8.2", "thiserror 2.0.17", ] +[[package]] +name = "reth-execution-types" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-evm", + "alloy-primitives 1.4.1", + "derive_more", + "reth-ethereum-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-trie-common 1.8.1", + "revm", +] + [[package]] name = "reth-execution-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "derive_more", - "reth-ethereum-primitives", - "reth-primitives-traits", - "reth-trie-common", + "reth-ethereum-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-trie-common 1.8.2", "revm", "serde", "serde_with", @@ -9406,33 +9764,33 @@ dependencies = [ [[package]] name = "reth-exex" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "eyre", "futures", "itertools 0.14.0", "metrics", "parking_lot", - "reth-chain-state", - "reth-chainspec", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", "reth-config", - "reth-ethereum-primitives", - "reth-evm", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-exex-types", - "reth-fs-util", - "reth-metrics", + "reth-fs-util 1.8.2", + "reth-metrics 1.8.2", "reth-node-api", "reth-node-core", - "reth-payload-builder", - "reth-primitives-traits", + "reth-payload-builder 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-prune-types", - "reth-revm", + "reth-prune-types 1.8.2", + "reth-revm 1.8.2", "reth-stages-api", - "reth-tasks", + "reth-tasks 1.8.2", "reth-tracing", "rmp-serde", "thiserror 2.0.17", @@ -9444,21 +9802,31 @@ dependencies = [ [[package]] name = "reth-exex-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", - "reth-chain-state", - "reth-execution-types", - "reth-primitives-traits", + "alloy-primitives 1.4.1", + "reth-chain-state 1.8.2", + "reth-execution-types 1.8.2", + "reth-primitives-traits 1.8.2", "serde", "serde_with", ] +[[package]] +name = "reth-fs-util" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "serde", + "serde_json", + "thiserror 2.0.17", +] + [[package]] name = "reth-fs-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "serde", "serde_json", @@ -9468,24 +9836,24 @@ dependencies = [ [[package]] name = "reth-invalid-block-hooks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-debug", "eyre", "futures", - "jsonrpsee", + "jsonrpsee 0.26.0", "pretty_assertions", - "reth-engine-primitives", - "reth-evm", - "reth-primitives-traits", + "reth-engine-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-revm", + "reth-revm 1.8.2", "reth-rpc-api", "reth-tracing", - "reth-trie", + "reth-trie 1.8.2", "revm-bytecode", "revm-database", "serde", @@ -9495,13 +9863,13 @@ dependencies = [ [[package]] name = "reth-ipc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "bytes", "futures", "futures-util", "interprocess", - "jsonrpsee", + "jsonrpsee 0.26.0", "pin-project", "serde_json", "thiserror 2.0.17", @@ -9515,7 +9883,7 @@ dependencies = [ [[package]] name = "reth-libmdbx" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "bitflags 2.9.4", "byteorder", @@ -9531,16 +9899,25 @@ dependencies = [ [[package]] name = "reth-mdbx-sys" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "bindgen 0.71.1", "cc", ] +[[package]] +name = "reth-metrics" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "metrics", + "metrics-derive", +] + [[package]] name = "reth-metrics" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "futures", "metrics", @@ -9552,15 +9929,15 @@ dependencies = [ [[package]] name = "reth-net-banlist" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", ] [[package]] name = "reth-net-nat" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "futures-util", "if-addrs 0.13.4", @@ -9574,11 +9951,11 @@ dependencies = [ [[package]] name = "reth-network" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "aquamarine", "auto_impl", @@ -9592,28 +9969,28 @@ dependencies = [ "pin-project", "rand 0.8.5", "rand 0.9.2", - "reth-chainspec", - "reth-consensus", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", "reth-discv4", "reth-discv5", "reth-dns-discovery", "reth-ecies", "reth-eth-wire", - "reth-eth-wire-types", - "reth-ethereum-forks", - "reth-ethereum-primitives", - "reth-fs-util", - "reth-metrics", + "reth-eth-wire-types 1.8.2", + "reth-ethereum-forks 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-fs-util 1.8.2", + "reth-metrics 1.8.2", "reth-net-banlist", "reth-network-api", "reth-network-p2p", - "reth-network-peers", + "reth-network-peers 1.8.2", "reth-network-types", - "reth-primitives-traits", - "reth-storage-api", - "reth-tasks", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "reth-tokio-util", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", "rustc-hash 2.1.1", "schnellru", "secp256k1 0.30.0", @@ -9629,20 +10006,20 @@ dependencies = [ [[package]] name = "reth-network-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-admin", "alloy-rpc-types-eth", "auto_impl", "derive_more", "enr", "futures", - "reth-eth-wire-types", - "reth-ethereum-forks", + "reth-eth-wire-types 1.8.2", + "reth-ethereum-forks 1.8.2", "reth-network-p2p", - "reth-network-peers", + "reth-network-peers 1.8.2", "reth-network-types", "reth-tokio-util", "serde", @@ -9654,32 +10031,45 @@ dependencies = [ [[package]] name = "reth-network-p2p" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "auto_impl", "derive_more", "futures", "parking_lot", - "reth-consensus", - "reth-eth-wire-types", - "reth-ethereum-primitives", - "reth-network-peers", + "reth-consensus 1.8.2", + "reth-eth-wire-types 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-network-peers 1.8.2", "reth-network-types", - "reth-primitives-traits", - "reth-storage-errors", + "reth-primitives-traits 1.8.2", + "reth-storage-errors 1.8.2", "tokio", "tracing", ] +[[package]] +name = "reth-network-peers" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "alloy-rlp", + "secp256k1 0.30.0", + "serde_with", + "thiserror 2.0.17", + "url", +] + [[package]] name = "reth-network-peers" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "enr", "secp256k1 0.30.0", @@ -9692,12 +10082,12 @@ dependencies = [ [[package]] name = "reth-network-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eip2124", "humantime-serde", "reth-net-banlist", - "reth-network-peers", + "reth-network-peers 1.8.2", "serde", "serde_json", "tracing", @@ -9706,14 +10096,14 @@ dependencies = [ [[package]] name = "reth-nippy-jar" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "anyhow", "bincode", "derive_more", "lz4_flex", "memmap2", - "reth-fs-util", + "reth-fs-util 1.8.2", "serde", "thiserror 2.0.17", "tracing", @@ -9723,35 +10113,35 @@ dependencies = [ [[package]] name = "reth-node-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-rpc-types-engine", "eyre", - "reth-basic-payload-builder", - "reth-consensus", + "reth-basic-payload-builder 1.8.2", + "reth-consensus 1.8.2", "reth-db-api", - "reth-engine-primitives", - "reth-evm", + "reth-engine-primitives 1.8.2", + "reth-evm 1.8.2", "reth-network-api", "reth-node-core", "reth-node-types", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", "reth-provider", - "reth-tasks", + "reth-tasks 1.8.2", "reth-tokio-util", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", ] [[package]] name = "reth-node-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-provider", "alloy-rpc-types", "alloy-rpc-types-engine", @@ -9759,27 +10149,27 @@ dependencies = [ "eyre", "fdlimit", "futures", - "jsonrpsee", + "jsonrpsee 0.26.0", "rayon", - "reth-basic-payload-builder", - "reth-chain-state", - "reth-chainspec", + "reth-basic-payload-builder 1.8.2", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", "reth-cli-util", "reth-config", - "reth-consensus", + "reth-consensus 1.8.2", "reth-consensus-debug-client", "reth-db", "reth-db-api", "reth-db-common", "reth-downloaders", "reth-engine-local", - "reth-engine-primitives", + "reth-engine-primitives 1.8.2", "reth-engine-service", "reth-engine-tree", "reth-engine-util", - "reth-evm", + "reth-evm 1.8.2", "reth-exex", - "reth-fs-util", + "reth-fs-util 1.8.2", "reth-invalid-block-hooks", "reth-network", "reth-network-api", @@ -9789,8 +10179,8 @@ dependencies = [ "reth-node-ethstats", "reth-node-events", "reth-node-metrics", - "reth-payload-builder", - "reth-primitives-traits", + "reth-payload-builder 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", "reth-rpc", @@ -9801,10 +10191,10 @@ dependencies = [ "reth-rpc-layer", "reth-stages", "reth-static-file", - "reth-tasks", + "reth-tasks 1.8.2", "reth-tokio-util", "reth-tracing", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", "secp256k1 0.30.0", "serde_json", "tokio", @@ -9815,11 +10205,11 @@ dependencies = [ [[package]] name = "reth-node-core" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "clap", "derive_more", @@ -9828,30 +10218,30 @@ dependencies = [ "futures", "humantime", "rand 0.9.2", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-cli-util", "reth-config", - "reth-consensus", + "reth-consensus 1.8.2", "reth-db", "reth-discv4", "reth-discv5", "reth-engine-local", - "reth-engine-primitives", - "reth-ethereum-forks", + "reth-engine-primitives 1.8.2", + "reth-ethereum-forks 1.8.2", "reth-net-nat", "reth-network", "reth-network-p2p", - "reth-network-peers", - "reth-primitives-traits", - "reth-prune-types", + "reth-network-peers 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", "reth-rpc-convert", "reth-rpc-eth-types", "reth-rpc-server-types", - "reth-stages-types", - "reth-storage-api", - "reth-storage-errors", + "reth-stages-types 1.8.2", + "reth-storage-api 1.8.2", + "reth-storage-errors 1.8.2", "reth-tracing", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", "secp256k1 0.30.0", "serde", "shellexpand", @@ -9867,29 +10257,29 @@ dependencies = [ [[package]] name = "reth-node-ethereum" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", "alloy-network", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "eyre", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-engine-local", - "reth-engine-primitives", + "reth-engine-primitives 1.8.2", "reth-ethereum-consensus", - "reth-ethereum-engine-primitives", + "reth-ethereum-engine-primitives 1.8.2", "reth-ethereum-payload-builder", - "reth-ethereum-primitives", - "reth-evm", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-evm-ethereum", "reth-network", "reth-node-api", "reth-node-builder", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-revm", + "reth-revm 1.8.2", "reth-rpc", "reth-rpc-api", "reth-rpc-builder", @@ -9897,7 +10287,7 @@ dependencies = [ "reth-rpc-eth-types", "reth-rpc-server-types", "reth-tracing", - "reth-transaction-pool", + "reth-transaction-pool 1.8.2", "revm", "tokio", ] @@ -9905,17 +10295,17 @@ dependencies = [ [[package]] name = "reth-node-ethstats" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "chrono", "futures-util", - "reth-chain-state", + "reth-chain-state 1.8.2", "reth-network-api", - "reth-primitives-traits", - "reth-storage-api", - "reth-transaction-pool", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-transaction-pool 1.8.2", "serde", "serde_json", "thiserror 2.0.17", @@ -9929,23 +10319,23 @@ dependencies = [ [[package]] name = "reth-node-events" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "derive_more", "futures", "humantime", "pin-project", - "reth-engine-primitives", + "reth-engine-primitives 1.8.2", "reth-network-api", - "reth-primitives-traits", - "reth-prune-types", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", "reth-stages", - "reth-static-file-types", - "reth-storage-api", + "reth-static-file-types 1.8.2", + "reth-storage-api 1.8.2", "tokio", "tracing", ] @@ -9953,18 +10343,18 @@ dependencies = [ [[package]] name = "reth-node-metrics" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "eyre", "http", - "jsonrpsee-server", + "jsonrpsee-server 0.26.0", "metrics", "metrics-exporter-prometheus 0.16.2", "metrics-process", "metrics-util 0.19.1", "procfs 0.17.0", - "reth-metrics", - "reth-tasks", + "reth-metrics 1.8.2", + "reth-tasks 1.8.2", "tikv-jemalloc-ctl", "tokio", "tower 0.5.2", @@ -9974,37 +10364,60 @@ dependencies = [ [[package]] name = "reth-node-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-db-api", - "reth-engine-primitives", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-engine-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", +] + +[[package]] +name = "reth-optimism-chainspec" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-chains", + "alloy-consensus", + "alloy-eips", + "alloy-genesis", + "alloy-hardforks", + "alloy-primitives 1.4.1", + "derive_more", + "op-alloy-consensus", + "op-alloy-rpc-types", + "reth-chainspec 1.8.1", + "reth-ethereum-forks 1.8.1", + "reth-network-peers 1.8.1", + "reth-optimism-forks 1.8.1", + "reth-optimism-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "serde_json", ] [[package]] name = "reth-optimism-chainspec" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-chains", "alloy-consensus", "alloy-eips", "alloy-genesis", "alloy-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "derive_more", "miniz_oxide", "op-alloy-consensus", "op-alloy-rpc-types", "paste", - "reth-chainspec", - "reth-ethereum-forks", - "reth-network-peers", - "reth-optimism-forks", - "reth-optimism-primitives", - "reth-primitives-traits", + "reth-chainspec 1.8.2", + "reth-ethereum-forks 1.8.2", + "reth-network-peers 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "serde", "serde_json", "tar-no-std", @@ -10014,44 +10427,44 @@ dependencies = [ [[package]] name = "reth-optimism-cli" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "clap", "derive_more", "eyre", "futures-util", "op-alloy-consensus", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-cli", "reth-cli-commands", "reth-cli-runner", - "reth-consensus", + "reth-consensus 1.8.2", "reth-db", "reth-db-api", "reth-db-common", "reth-downloaders", - "reth-execution-types", - "reth-fs-util", + "reth-execution-types 1.8.2", + "reth-fs-util 1.8.2", "reth-node-builder", "reth-node-core", "reth-node-events", "reth-node-metrics", - "reth-optimism-chainspec", - "reth-optimism-consensus", - "reth-optimism-evm", + "reth-optimism-chainspec 1.8.2", + "reth-optimism-consensus 1.8.2", + "reth-optimism-evm 1.8.2", "reth-optimism-node", - "reth-optimism-primitives", - "reth-primitives-traits", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", "reth-rpc-server-types", "reth-stages", "reth-static-file", - "reth-static-file-types", + "reth-static-file-types 1.8.2", "reth-tracing", "serde", "tokio", @@ -10059,55 +10472,107 @@ dependencies = [ "tracing", ] +[[package]] +name = "reth-optimism-consensus" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-trie", + "reth-chainspec 1.8.1", + "reth-consensus 1.8.1", + "reth-consensus-common 1.8.1", + "reth-execution-types 1.8.1", + "reth-optimism-chainspec 1.8.1", + "reth-optimism-forks 1.8.1", + "reth-optimism-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-storage-errors 1.8.1", + "reth-trie-common 1.8.1", + "revm", + "thiserror 2.0.17", + "tracing", +] + [[package]] name = "reth-optimism-consensus" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-trie", - "reth-chainspec", - "reth-consensus", - "reth-consensus-common", - "reth-execution-types", - "reth-optimism-chainspec", - "reth-optimism-forks", - "reth-optimism-primitives", - "reth-primitives-traits", - "reth-storage-api", - "reth-storage-errors", - "reth-trie-common", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-consensus-common 1.8.2", + "reth-execution-types 1.8.2", + "reth-optimism-chainspec 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie-common 1.8.2", "revm", "thiserror 2.0.17", "tracing", ] +[[package]] +name = "reth-optimism-evm" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-evm", + "alloy-op-evm", + "alloy-primitives 1.4.1", + "op-alloy-consensus", + "op-alloy-rpc-types-engine", + "op-revm", + "reth-chainspec 1.8.1", + "reth-evm 1.8.1", + "reth-execution-errors 1.8.1", + "reth-execution-types 1.8.1", + "reth-optimism-chainspec 1.8.1", + "reth-optimism-consensus 1.8.1", + "reth-optimism-forks 1.8.1", + "reth-optimism-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-errors 1.8.1", + "revm", + "thiserror 2.0.17", +] + [[package]] name = "reth-optimism-evm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-op-evm", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "op-alloy-consensus", "op-alloy-rpc-types-engine", "op-revm", - "reth-chainspec", - "reth-evm", - "reth-execution-errors", - "reth-execution-types", - "reth-optimism-chainspec", - "reth-optimism-consensus", - "reth-optimism-forks", - "reth-optimism-primitives", - "reth-primitives-traits", + "reth-chainspec 1.8.2", + "reth-evm 1.8.2", + "reth-execution-errors 1.8.2", + "reth-execution-types 1.8.2", + "reth-optimism-chainspec 1.8.2", + "reth-optimism-consensus 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "reth-rpc-eth-api", - "reth-storage-errors", + "reth-storage-errors 1.8.2", "revm", "thiserror 2.0.17", ] @@ -10115,29 +10580,29 @@ dependencies = [ [[package]] name = "reth-optimism-flashblocks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "alloy-serde", "brotli", "eyre", "futures-util", - "reth-chain-state", - "reth-errors", - "reth-evm", - "reth-execution-types", + "reth-chain-state 1.8.2", + "reth-errors 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", "reth-node-api", - "reth-optimism-evm", - "reth-optimism-payload-builder", - "reth-optimism-primitives", - "reth-primitives-traits", - "reth-revm", + "reth-optimism-evm 1.8.2", + "reth-optimism-payload-builder 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", "reth-rpc-eth-types", - "reth-storage-api", - "reth-tasks", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "ringbuffer", "serde", "serde_json", @@ -10147,24 +10612,35 @@ dependencies = [ "url", ] +[[package]] +name = "reth-optimism-forks" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-op-hardforks", + "alloy-primitives 1.4.1", + "once_cell", + "reth-ethereum-forks 1.8.1", +] + [[package]] name = "reth-optimism-forks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-op-hardforks", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "once_cell", - "reth-ethereum-forks", + "reth-ethereum-forks 1.8.2", ] [[package]] name = "reth-optimism-node" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "clap", @@ -10172,93 +10648,145 @@ dependencies = [ "op-alloy-consensus", "op-alloy-rpc-types-engine", "op-revm", - "reth-chainspec", - "reth-consensus", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", "reth-engine-local", - "reth-evm", + "reth-evm 1.8.2", "reth-network", "reth-node-api", "reth-node-builder", "reth-node-core", - "reth-optimism-chainspec", - "reth-optimism-consensus", - "reth-optimism-evm", - "reth-optimism-forks", - "reth-optimism-payload-builder", - "reth-optimism-primitives", + "reth-optimism-chainspec 1.8.2", + "reth-optimism-consensus 1.8.2", + "reth-optimism-evm 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-payload-builder 1.8.2", + "reth-optimism-primitives 1.8.2", "reth-optimism-rpc", "reth-optimism-storage", - "reth-optimism-txpool", - "reth-payload-builder", - "reth-primitives-traits", + "reth-optimism-txpool 1.8.2", + "reth-payload-builder 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-rpc-api", "reth-rpc-engine-api", "reth-rpc-server-types", "reth-tracing", - "reth-transaction-pool", - "reth-trie-common", + "reth-transaction-pool 1.8.2", + "reth-trie-common 1.8.2", "revm", "serde", "tokio", "url", ] +[[package]] +name = "reth-optimism-payload-builder" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-rpc-types-debug", + "alloy-rpc-types-engine", + "derive_more", + "op-alloy-consensus", + "op-alloy-rpc-types-engine", + "reth-basic-payload-builder 1.8.1", + "reth-chain-state 1.8.1", + "reth-chainspec 1.8.1", + "reth-evm 1.8.1", + "reth-execution-types 1.8.1", + "reth-optimism-evm 1.8.1", + "reth-optimism-forks 1.8.1", + "reth-optimism-primitives 1.8.1", + "reth-optimism-txpool 1.8.1", + "reth-payload-builder 1.8.1", + "reth-payload-builder-primitives 1.8.1", + "reth-payload-primitives 1.8.1", + "reth-payload-util 1.8.1", + "reth-payload-validator 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-revm 1.8.1", + "reth-storage-api 1.8.1", + "reth-transaction-pool 1.8.1", + "revm", + "serde", + "sha2 0.10.9", + "thiserror 2.0.17", + "tracing", +] + [[package]] name = "reth-optimism-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-debug", "alloy-rpc-types-engine", "derive_more", "op-alloy-consensus", "op-alloy-rpc-types-engine", - "reth-basic-payload-builder", - "reth-chain-state", - "reth-chainspec", - "reth-evm", - "reth-execution-types", - "reth-optimism-evm", - "reth-optimism-forks", - "reth-optimism-primitives", - "reth-optimism-txpool", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-payload-util", - "reth-payload-validator", - "reth-primitives-traits", - "reth-revm", - "reth-storage-api", - "reth-transaction-pool", + "reth-basic-payload-builder 1.8.2", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-optimism-evm 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-optimism-txpool 1.8.2", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-payload-util 1.8.2", + "reth-payload-validator 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", + "reth-storage-api 1.8.2", + "reth-transaction-pool 1.8.2", "revm", "serde", - "sha2", + "sha2 0.10.9", "thiserror 2.0.17", "tracing", ] +[[package]] +name = "reth-optimism-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "op-alloy-consensus", + "reth-primitives-traits 1.8.1", +] + [[package]] name = "reth-optimism-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "arbitrary", "bytes", "modular-bitfield", "op-alloy-consensus", - "reth-codecs", - "reth-primitives-traits", - "reth-zstd-compressors", + "reth-codecs 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-zstd-compressors 1.8.2", "serde", "serde_with", ] @@ -10266,12 +10794,12 @@ dependencies = [ [[package]] name = "reth-optimism-rpc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-client", "alloy-rpc-types-debug", "alloy-rpc-types-engine", @@ -10282,9 +10810,9 @@ dependencies = [ "derive_more", "eyre", "futures", - "jsonrpsee", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee 0.26.0", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "metrics", "op-alloy-consensus", "op-alloy-network", @@ -10293,19 +10821,19 @@ dependencies = [ "op-alloy-rpc-types-engine", "op-revm", "reqwest", - "reth-chain-state", - "reth-chainspec", - "reth-evm", - "reth-metrics", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-evm 1.8.2", + "reth-metrics 1.8.2", "reth-node-api", "reth-node-builder", - "reth-optimism-evm", + "reth-optimism-evm 1.8.2", "reth-optimism-flashblocks", - "reth-optimism-forks", - "reth-optimism-payload-builder", - "reth-optimism-primitives", - "reth-optimism-txpool", - "reth-primitives-traits", + "reth-optimism-forks 1.8.2", + "reth-optimism-payload-builder 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-optimism-txpool 1.8.2", + "reth-primitives-traits 1.8.2", "reth-rpc", "reth-rpc-api", "reth-rpc-convert", @@ -10313,9 +10841,9 @@ dependencies = [ "reth-rpc-eth-api", "reth-rpc-eth-types", "reth-rpc-server-types", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", "revm", "serde_json", "thiserror 2.0.17", @@ -10328,22 +10856,58 @@ dependencies = [ [[package]] name = "reth-optimism-storage" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" +dependencies = [ + "alloy-consensus", + "reth-optimism-primitives 1.8.2", + "reth-storage-api 1.8.2", +] + +[[package]] +name = "reth-optimism-txpool" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" dependencies = [ "alloy-consensus", - "reth-optimism-primitives", - "reth-storage-api", + "alloy-eips", + "alloy-json-rpc", + "alloy-primitives 1.4.1", + "alloy-rpc-client", + "alloy-rpc-types-eth", + "alloy-serde", + "c-kzg", + "derive_more", + "futures-util", + "metrics", + "op-alloy-consensus", + "op-alloy-flz", + "op-alloy-rpc-types", + "op-revm", + "parking_lot", + "reth-chain-state 1.8.1", + "reth-chainspec 1.8.1", + "reth-metrics 1.8.1", + "reth-optimism-evm 1.8.1", + "reth-optimism-forks 1.8.1", + "reth-optimism-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-transaction-pool 1.8.1", + "serde", + "thiserror 2.0.17", + "tokio", + "tracing", ] [[package]] name = "reth-optimism-txpool" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-client", "alloy-rpc-types-eth", "alloy-serde", @@ -10356,37 +10920,70 @@ dependencies = [ "op-alloy-rpc-types", "op-revm", "parking_lot", - "reth-chain-state", - "reth-chainspec", - "reth-metrics", - "reth-optimism-evm", - "reth-optimism-forks", - "reth-optimism-primitives", - "reth-primitives-traits", - "reth-storage-api", - "reth-transaction-pool", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-metrics 1.8.2", + "reth-optimism-evm 1.8.2", + "reth-optimism-forks 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-transaction-pool 1.8.2", "serde", "thiserror 2.0.17", "tokio", "tracing", ] +[[package]] +name = "reth-payload-builder" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-primitives 1.4.1", + "alloy-rpc-types", + "futures-util", + "metrics", + "reth-chain-state 1.8.1", + "reth-ethereum-engine-primitives 1.8.1", + "reth-metrics 1.8.1", + "reth-payload-builder-primitives 1.8.1", + "reth-payload-primitives 1.8.1", + "reth-primitives-traits 1.8.1", + "tokio", + "tokio-stream", + "tracing", +] + [[package]] name = "reth-payload-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types", "futures-util", "metrics", - "reth-chain-state", - "reth-ethereum-engine-primitives", - "reth-metrics", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-chain-state 1.8.2", + "reth-ethereum-engine-primitives 1.8.2", + "reth-metrics 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-payload-builder-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "pin-project", + "reth-payload-primitives 1.8.1", "tokio", "tokio-stream", "tracing", @@ -10395,78 +10992,145 @@ dependencies = [ [[package]] name = "reth-payload-builder-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "pin-project", - "reth-payload-primitives", + "reth-payload-primitives 1.8.2", "tokio", "tokio-stream", "tracing", ] +[[package]] +name = "reth-payload-primitives" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rpc-types-engine", + "auto_impl", + "either", + "op-alloy-rpc-types-engine", + "reth-chain-state 1.8.1", + "reth-chainspec 1.8.1", + "reth-errors 1.8.1", + "reth-primitives-traits 1.8.1", + "serde", + "thiserror 2.0.17", + "tokio", +] + [[package]] name = "reth-payload-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "auto_impl", "either", "op-alloy-rpc-types-engine", - "reth-chain-state", - "reth-chainspec", - "reth-errors", - "reth-primitives-traits", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-errors 1.8.2", + "reth-primitives-traits 1.8.2", "serde", "thiserror 2.0.17", "tokio", ] +[[package]] +name = "reth-payload-util" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-primitives 1.4.1", + "reth-transaction-pool 1.8.1", +] + [[package]] name = "reth-payload-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" +dependencies = [ + "alloy-consensus", + "alloy-primitives 1.4.1", + "reth-transaction-pool 1.8.2", +] + +[[package]] +name = "reth-payload-validator" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", - "reth-transaction-pool", + "alloy-rpc-types-engine", + "reth-primitives-traits 1.8.1", ] [[package]] name = "reth-payload-validator" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-rpc-types-engine", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", ] [[package]] name = "reth-primitives" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "c-kzg", "once_cell", - "reth-ethereum-forks", - "reth-ethereum-primitives", - "reth-primitives-traits", - "reth-static-file-types", + "reth-ethereum-forks 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-static-file-types 1.8.2", +] + +[[package]] +name = "reth-primitives-traits" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-genesis", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-rpc-types-eth", + "alloy-trie", + "auto_impl", + "bytes", + "derive_more", + "once_cell", + "op-alloy-consensus", + "reth-codecs 1.8.1", + "revm-bytecode", + "revm-primitives", + "revm-state", + "secp256k1 0.30.0", + "serde", + "serde_with", + "thiserror 2.0.17", ] [[package]] name = "reth-primitives-traits" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-eth", "alloy-trie", @@ -10481,7 +11145,7 @@ dependencies = [ "proptest", "proptest-arbitrary-interop", "rayon", - "reth-codecs", + "reth-codecs 1.8.2", "revm-bytecode", "revm-primitives", "revm-state", @@ -10494,11 +11158,11 @@ dependencies = [ [[package]] name = "reth-provider" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "dashmap 6.1.0", "eyre", @@ -10507,27 +11171,27 @@ dependencies = [ "notify", "parking_lot", "rayon", - "reth-chain-state", - "reth-chainspec", - "reth-codecs", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-codecs 1.8.2", "reth-db", "reth-db-api", - "reth-errors", - "reth-ethereum-engine-primitives", - "reth-ethereum-primitives", - "reth-evm", - "reth-execution-types", - "reth-fs-util", - "reth-metrics", + "reth-errors 1.8.2", + "reth-ethereum-engine-primitives 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-fs-util 1.8.2", + "reth-metrics 1.8.2", "reth-nippy-jar", "reth-node-types", - "reth-primitives-traits", - "reth-prune-types", - "reth-stages-types", - "reth-static-file-types", - "reth-storage-api", - "reth-storage-errors", - "reth-trie", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", + "reth-stages-types 1.8.2", + "reth-static-file-types 1.8.2", + "reth-storage-api 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie 1.8.2", "reth-trie-db", "revm-database", "revm-state", @@ -10539,24 +11203,24 @@ dependencies = [ [[package]] name = "reth-prune" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "itertools 0.14.0", "metrics", "rayon", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-config", "reth-db-api", - "reth-errors", + "reth-errors 1.8.2", "reth-exex-types", - "reth-metrics", - "reth-primitives-traits", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-prune-types", - "reth-static-file-types", + "reth-prune-types 1.8.2", + "reth-static-file-types 1.8.2", "reth-tokio-util", "rustc-hash 2.1.1", "thiserror 2.0.17", @@ -10564,16 +11228,26 @@ dependencies = [ "tracing", ] +[[package]] +name = "reth-prune-types" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "derive_more", + "thiserror 2.0.17", +] + [[package]] name = "reth-prune-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "arbitrary", "derive_more", "modular-bitfield", - "reth-codecs", + "reth-codecs 1.8.2", "serde", "thiserror 2.0.17", ] @@ -10581,17 +11255,17 @@ dependencies = [ [[package]] name = "reth-ress-protocol" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "futures", "reth-eth-wire", - "reth-ethereum-primitives", + "reth-ethereum-primitives 1.8.2", "reth-network", "reth-network-api", - "reth-storage-errors", + "reth-storage-errors 1.8.2", "tokio", "tokio-stream", "tracing", @@ -10600,47 +11274,60 @@ dependencies = [ [[package]] name = "reth-ress-provider" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "eyre", "futures", "parking_lot", - "reth-chain-state", - "reth-errors", - "reth-ethereum-primitives", - "reth-evm", + "reth-chain-state 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", "reth-node-api", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-ress-protocol", - "reth-revm", - "reth-storage-api", - "reth-tasks", + "reth-revm 1.8.2", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "reth-tokio-util", - "reth-trie", + "reth-trie 1.8.2", "schnellru", "tokio", "tracing", ] +[[package]] +name = "reth-revm" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-storage-errors 1.8.1", + "reth-trie 1.8.1", + "revm", +] + [[package]] name = "reth-revm" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", - "reth-primitives-traits", - "reth-storage-api", - "reth-storage-errors", - "reth-trie", + "alloy-primitives 1.4.1", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie 1.8.2", "revm", ] [[package]] name = "reth-rpc" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10648,7 +11335,7 @@ dependencies = [ "alloy-evm", "alloy-genesis", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-client", "alloy-rpc-types", @@ -10671,43 +11358,43 @@ dependencies = [ "http-body", "hyper", "itertools 0.14.0", - "jsonrpsee", - "jsonrpsee-types", + "jsonrpsee 0.26.0", + "jsonrpsee-types 0.26.0", "jsonwebtoken", "parking_lot", "pin-project", - "reth-chain-state", - "reth-chainspec", - "reth-consensus", - "reth-consensus-common", - "reth-engine-primitives", - "reth-errors", - "reth-evm", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-consensus-common 1.8.2", + "reth-engine-primitives 1.8.2", + "reth-errors 1.8.2", + "reth-evm 1.8.2", "reth-evm-ethereum", - "reth-execution-types", - "reth-metrics", + "reth-execution-types 1.8.2", + "reth-metrics 1.8.2", "reth-network-api", - "reth-network-peers", + "reth-network-peers 1.8.2", "reth-network-types", "reth-node-api", - "reth-primitives-traits", - "reth-revm", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", "reth-rpc-api", "reth-rpc-convert", "reth-rpc-engine-api", "reth-rpc-eth-api", "reth-rpc-eth-types", "reth-rpc-server-types", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", - "reth-trie-common", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", + "reth-trie-common 1.8.2", "revm", "revm-inspectors", "revm-primitives", "serde", "serde_json", - "sha2", + "sha2 0.10.9", "thiserror 2.0.17", "tokio", "tokio-stream", @@ -10719,12 +11406,12 @@ dependencies = [ [[package]] name = "reth-rpc-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", "alloy-genesis", "alloy-json-rpc", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types", "alloy-rpc-types-admin", "alloy-rpc-types-anvil", @@ -10736,44 +11423,44 @@ dependencies = [ "alloy-rpc-types-trace", "alloy-rpc-types-txpool", "alloy-serde", - "jsonrpsee", - "reth-chain-state", - "reth-engine-primitives", - "reth-network-peers", + "jsonrpsee 0.26.0", + "reth-chain-state 1.8.2", + "reth-engine-primitives 1.8.2", + "reth-network-peers 1.8.2", "reth-rpc-eth-api", - "reth-trie-common", + "reth-trie-common 1.8.2", ] [[package]] name = "reth-rpc-builder" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-network", "alloy-provider", "dyn-clone", "http", - "jsonrpsee", + "jsonrpsee 0.26.0", "metrics", "pin-project", - "reth-chain-state", - "reth-chainspec", - "reth-consensus", - "reth-evm", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-consensus 1.8.2", + "reth-evm 1.8.2", "reth-ipc", - "reth-metrics", + "reth-metrics 1.8.2", "reth-network-api", "reth-node-core", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-rpc", "reth-rpc-api", "reth-rpc-eth-api", "reth-rpc-eth-types", "reth-rpc-layer", "reth-rpc-server-types", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", "serde", "thiserror 2.0.17", "tokio", @@ -10786,26 +11473,26 @@ dependencies = [ [[package]] name = "reth-rpc-convert" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-json-rpc", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-signer", "auto_impl", "dyn-clone", - "jsonrpsee-types", + "jsonrpsee-types 0.26.0", "op-alloy-consensus", "op-alloy-network", "op-alloy-rpc-types", "op-revm", - "reth-ethereum-primitives", - "reth-evm", - "reth-optimism-primitives", - "reth-primitives-traits", - "reth-storage-api", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-optimism-primitives 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", "revm-context", "thiserror 2.0.17", ] @@ -10813,27 +11500,27 @@ dependencies = [ [[package]] name = "reth-rpc-engine-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "async-trait", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "metrics", "parking_lot", - "reth-chainspec", - "reth-engine-primitives", - "reth-metrics", - "reth-payload-builder", - "reth-payload-builder-primitives", - "reth-payload-primitives", - "reth-primitives-traits", + "reth-chainspec 1.8.2", + "reth-engine-primitives 1.8.2", + "reth-metrics 1.8.2", + "reth-payload-builder 1.8.2", + "reth-payload-builder-primitives 1.8.2", + "reth-payload-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "reth-rpc-api", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", "serde", "thiserror 2.0.17", "tokio", @@ -10843,7 +11530,7 @@ dependencies = [ [[package]] name = "reth-rpc-eth-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-dyn-abi", @@ -10851,7 +11538,7 @@ dependencies = [ "alloy-evm", "alloy-json-rpc", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-eth", "alloy-rpc-types-mev", @@ -10860,24 +11547,24 @@ dependencies = [ "auto_impl", "dyn-clone", "futures", - "jsonrpsee", - "jsonrpsee-types", + "jsonrpsee 0.26.0", + "jsonrpsee-types 0.26.0", "parking_lot", - "reth-chain-state", - "reth-chainspec", - "reth-errors", - "reth-evm", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-errors 1.8.2", + "reth-evm 1.8.2", "reth-network-api", "reth-node-api", - "reth-primitives-traits", - "reth-revm", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", "reth-rpc-convert", "reth-rpc-eth-types", "reth-rpc-server-types", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", - "reth-trie-common", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", + "reth-trie-common 1.8.2", "revm", "revm-inspectors", "tokio", @@ -10887,40 +11574,40 @@ dependencies = [ [[package]] name = "reth-rpc-eth-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-evm", "alloy-network", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-client", "alloy-rpc-types-eth", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "alloy-transport", "derive_more", "futures", "itertools 0.14.0", - "jsonrpsee-core", - "jsonrpsee-types", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", "metrics", "rand 0.9.2", "reqwest", - "reth-chain-state", - "reth-chainspec", - "reth-errors", - "reth-ethereum-primitives", - "reth-evm", - "reth-execution-types", - "reth-metrics", - "reth-primitives-traits", - "reth-revm", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-errors 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-revm 1.8.2", "reth-rpc-convert", "reth-rpc-server-types", - "reth-storage-api", - "reth-tasks", - "reth-transaction-pool", - "reth-trie", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", + "reth-transaction-pool 1.8.2", + "reth-trie 1.8.2", "revm", "revm-inspectors", "schnellru", @@ -10934,11 +11621,11 @@ dependencies = [ [[package]] name = "reth-rpc-layer" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-rpc-types-engine", "http", - "jsonrpsee-http-client", + "jsonrpsee-http-client 0.26.0", "pin-project", "tower 0.5.2", "tower-http", @@ -10948,14 +11635,14 @@ dependencies = [ [[package]] name = "reth-rpc-server-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", - "jsonrpsee-core", - "jsonrpsee-types", - "reth-errors", + "jsonrpsee-core 0.26.0", + "jsonrpsee-types 0.26.0", + "reth-errors 1.8.2", "reth-network-api", "serde", "strum 0.27.2", @@ -10964,11 +11651,11 @@ dependencies = [ [[package]] name = "reth-stages" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "bincode", "eyre", "futures-util", @@ -10976,32 +11663,32 @@ dependencies = [ "num-traits", "rayon", "reqwest", - "reth-chainspec", - "reth-codecs", + "reth-chainspec 1.8.2", + "reth-codecs 1.8.2", "reth-config", - "reth-consensus", + "reth-consensus 1.8.2", "reth-db", "reth-db-api", "reth-era", "reth-era-downloader", "reth-era-utils", - "reth-ethereum-primitives", + "reth-ethereum-primitives 1.8.2", "reth-etl", - "reth-evm", - "reth-execution-types", + "reth-evm 1.8.2", + "reth-execution-types 1.8.2", "reth-exex", - "reth-fs-util", + "reth-fs-util 1.8.2", "reth-network-p2p", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", - "reth-prune-types", - "reth-revm", + "reth-prune-types 1.8.2", + "reth-revm 1.8.2", "reth-stages-api", - "reth-static-file-types", - "reth-storage-errors", + "reth-static-file-types 1.8.2", + "reth-storage-errors 1.8.2", "reth-testing-utils", - "reth-trie", + "reth-trie 1.8.2", "reth-trie-db", "tempfile", "thiserror 2.0.17", @@ -11012,119 +11699,193 @@ dependencies = [ [[package]] name = "reth-stages-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "aquamarine", "auto_impl", "futures-util", "metrics", - "reth-consensus", - "reth-errors", - "reth-metrics", + "reth-consensus 1.8.2", + "reth-errors 1.8.2", + "reth-metrics 1.8.2", "reth-network-p2p", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", "reth-prune", - "reth-stages-types", + "reth-stages-types 1.8.2", "reth-static-file", - "reth-static-file-types", + "reth-static-file-types 1.8.2", "reth-tokio-util", "thiserror 2.0.17", "tokio", "tracing", ] +[[package]] +name = "reth-stages-types" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "reth-trie-common 1.8.1", +] + [[package]] name = "reth-stages-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "arbitrary", "bytes", "modular-bitfield", - "reth-codecs", - "reth-trie-common", + "reth-codecs 1.8.2", + "reth-trie-common 1.8.2", "serde", ] [[package]] name = "reth-static-file" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "parking_lot", "rayon", - "reth-codecs", + "reth-codecs 1.8.2", "reth-db-api", - "reth-primitives-traits", + "reth-primitives-traits 1.8.2", "reth-provider", - "reth-prune-types", - "reth-stages-types", - "reth-static-file-types", - "reth-storage-errors", + "reth-prune-types 1.8.2", + "reth-stages-types 1.8.2", + "reth-static-file-types 1.8.2", + "reth-storage-errors 1.8.2", "reth-tokio-util", "tracing", ] +[[package]] +name = "reth-static-file-types" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "derive_more", + "serde", + "strum 0.27.2", +] + [[package]] name = "reth-static-file-types" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "clap", "derive_more", "serde", "strum 0.27.2", ] +[[package]] +name = "reth-storage-api" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rpc-types-engine", + "auto_impl", + "reth-chainspec 1.8.1", + "reth-db-models 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-execution-types 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-prune-types 1.8.1", + "reth-stages-types 1.8.1", + "reth-storage-errors 1.8.1", + "reth-trie-common 1.8.1", + "revm-database", +] + [[package]] name = "reth-storage-api" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "auto_impl", - "reth-chainspec", + "reth-chainspec 1.8.2", "reth-db-api", - "reth-db-models", - "reth-ethereum-primitives", - "reth-execution-types", - "reth-primitives-traits", - "reth-prune-types", - "reth-stages-types", - "reth-storage-errors", - "reth-trie-common", + "reth-db-models 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-execution-types 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", + "reth-stages-types 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie-common 1.8.2", "revm-database", ] +[[package]] +name = "reth-storage-errors" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "derive_more", + "reth-primitives-traits 1.8.1", + "reth-prune-types 1.8.1", + "reth-static-file-types 1.8.1", + "revm-database-interface", + "thiserror 2.0.17", +] + [[package]] name = "reth-storage-errors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "derive_more", - "reth-primitives-traits", - "reth-prune-types", - "reth-static-file-types", + "reth-primitives-traits 1.8.2", + "reth-prune-types 1.8.2", + "reth-static-file-types 1.8.2", "revm-database-interface", "thiserror 2.0.17", ] +[[package]] +name = "reth-tasks" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "auto_impl", + "dyn-clone", + "futures-util", + "metrics", + "reth-metrics 1.8.1", + "thiserror 2.0.17", + "tokio", + "tracing", + "tracing-futures", +] + [[package]] name = "reth-tasks" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "auto_impl", "dyn-clone", @@ -11132,7 +11893,7 @@ dependencies = [ "metrics", "pin-project", "rayon", - "reth-metrics", + "reth-metrics 1.8.2", "thiserror 2.0.17", "tokio", "tracing", @@ -11142,23 +11903,23 @@ dependencies = [ [[package]] name = "reth-testing-utils" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", "alloy-genesis", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "rand 0.8.5", "rand 0.9.2", - "reth-ethereum-primitives", - "reth-primitives-traits", + "reth-ethereum-primitives 1.8.2", + "reth-primitives-traits 1.8.2", "secp256k1 0.30.0", ] [[package]] name = "reth-tokio-util" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "tokio", "tokio-stream", @@ -11168,7 +11929,7 @@ dependencies = [ [[package]] name = "reth-tracing" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "clap", "eyre", @@ -11183,7 +11944,7 @@ dependencies = [ [[package]] name = "reth-tracing-otlp" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "opentelemetry 0.29.1", "opentelemetry-otlp 0.29.0", @@ -11194,14 +11955,53 @@ dependencies = [ "tracing-subscriber 0.3.20", ] +[[package]] +name = "reth-transaction-pool" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "aquamarine", + "auto_impl", + "bitflags 2.9.4", + "futures-util", + "metrics", + "parking_lot", + "pin-project", + "reth-chain-state 1.8.1", + "reth-chainspec 1.8.1", + "reth-eth-wire-types 1.8.1", + "reth-ethereum-primitives 1.8.1", + "reth-execution-types 1.8.1", + "reth-fs-util 1.8.1", + "reth-metrics 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-storage-api 1.8.1", + "reth-tasks 1.8.1", + "revm-interpreter", + "revm-primitives", + "rustc-hash 2.1.1", + "schnellru", + "serde", + "serde_json", + "smallvec", + "thiserror 2.0.17", + "tokio", + "tokio-stream", + "tracing", +] + [[package]] name = "reth-transaction-pool" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "aquamarine", "auto_impl", @@ -11212,16 +12012,16 @@ dependencies = [ "paste", "pin-project", "rand 0.9.2", - "reth-chain-state", - "reth-chainspec", - "reth-eth-wire-types", - "reth-ethereum-primitives", - "reth-execution-types", - "reth-fs-util", - "reth-metrics", - "reth-primitives-traits", - "reth-storage-api", - "reth-tasks", + "reth-chain-state 1.8.2", + "reth-chainspec 1.8.2", + "reth-eth-wire-types 1.8.2", + "reth-ethereum-primitives 1.8.2", + "reth-execution-types 1.8.2", + "reth-fs-util 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-storage-api 1.8.2", + "reth-tasks 1.8.2", "revm-interpreter", "revm-primitives", "rustc-hash 2.1.1", @@ -11235,38 +12035,77 @@ dependencies = [ "tracing", ] +[[package]] +name = "reth-trie" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-trie", + "auto_impl", + "itertools 0.14.0", + "reth-execution-errors 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-stages-types 1.8.1", + "reth-storage-errors 1.8.1", + "reth-trie-common 1.8.1", + "reth-trie-sparse 1.8.1", + "revm-database", + "tracing", +] + [[package]] name = "reth-trie" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", "alloy-eips", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-trie", "auto_impl", "itertools 0.14.0", "metrics", - "reth-execution-errors", - "reth-metrics", - "reth-primitives-traits", - "reth-stages-types", - "reth-storage-errors", - "reth-trie-common", - "reth-trie-sparse", + "reth-execution-errors 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-stages-types 1.8.2", + "reth-storage-errors 1.8.2", + "reth-trie-common 1.8.2", + "reth-trie-sparse 1.8.2", "revm-database", "tracing", "triehash", ] +[[package]] +name = "reth-trie-common" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-consensus", + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-trie", + "derive_more", + "itertools 0.14.0", + "nybbles", + "rayon", + "reth-primitives-traits 1.8.1", + "revm-database", +] + [[package]] name = "reth-trie-common" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "alloy-consensus", - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-rpc-types-eth", "alloy-serde", @@ -11279,8 +12118,8 @@ dependencies = [ "nybbles", "plain_hasher", "rayon", - "reth-codecs", - "reth-primitives-traits", + "reth-codecs 1.8.2", + "reth-primitives-traits 1.8.2", "revm-database", "serde", "serde_with", @@ -11289,56 +12128,72 @@ dependencies = [ [[package]] name = "reth-trie-db" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "reth-db-api", - "reth-execution-errors", - "reth-primitives-traits", - "reth-trie", + "reth-execution-errors 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-trie 1.8.2", "tracing", ] [[package]] name = "reth-trie-parallel" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "derive_more", "itertools 0.14.0", "metrics", "rayon", "reth-db-api", - "reth-execution-errors", - "reth-metrics", + "reth-execution-errors 1.8.2", + "reth-metrics 1.8.2", "reth-provider", - "reth-storage-errors", - "reth-trie", - "reth-trie-common", + "reth-storage-errors 1.8.2", + "reth-trie 1.8.2", + "reth-trie-common 1.8.2", "reth-trie-db", - "reth-trie-sparse", + "reth-trie-sparse 1.8.2", "thiserror 2.0.17", "tokio", "tracing", ] +[[package]] +name = "reth-trie-sparse" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "alloy-primitives 1.4.1", + "alloy-rlp", + "alloy-trie", + "auto_impl", + "reth-execution-errors 1.8.1", + "reth-primitives-traits 1.8.1", + "reth-trie-common 1.8.1", + "smallvec", + "tracing", +] + [[package]] name = "reth-trie-sparse" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-trie", "auto_impl", "metrics", "rayon", - "reth-execution-errors", - "reth-metrics", - "reth-primitives-traits", - "reth-trie-common", + "reth-execution-errors 1.8.2", + "reth-metrics 1.8.2", + "reth-primitives-traits 1.8.2", + "reth-trie-common 1.8.2", "smallvec", "tracing", ] @@ -11346,33 +12201,42 @@ dependencies = [ [[package]] name = "reth-trie-sparse-parallel" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rlp", "alloy-trie", "metrics", "rayon", - "reth-execution-errors", - "reth-metrics", - "reth-trie-common", - "reth-trie-sparse", + "reth-execution-errors 1.8.2", + "reth-metrics 1.8.2", + "reth-trie-common 1.8.2", + "reth-trie-sparse 1.8.2", "smallvec", "tracing", ] +[[package]] +name = "reth-zstd-compressors" +version = "1.8.1" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.1#e6608be51ea34424b8e3693cf1f946a3eb224736" +dependencies = [ + "zstd", +] + [[package]] name = "reth-zstd-compressors" version = "1.8.2" -source = "git+https://github.com/noot/reth?rev=00d352d37c3f3ba7c23f370964b92cd13ce08702#00d352d37c3f3ba7c23f370964b92cd13ce08702" +source = "git+https://github.com/paradigmxyz/reth?tag=v1.8.2#9c30bf7af5e0d45deaf5917375c9922c16654b28" dependencies = [ "zstd", ] [[package]] name = "revm" -version = "30.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "29.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "718d90dce5f07e115d0e66450b1b8aa29694c1cf3f89ebddaddccc2ccbd2f13e" dependencies = [ "revm-bytecode", "revm-context", @@ -11389,19 +12253,21 @@ dependencies = [ [[package]] name = "revm-bytecode" -version = "7.0.0" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "6.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66c52031b73cae95d84cd1b07725808b5fd1500da3e5e24574a3b2dc13d9f16d" dependencies = [ "bitvec", - "phf 0.13.1", + "phf", "revm-primitives", "serde", ] [[package]] name = "revm-context" -version = "10.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "9.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a20c98e7008591a6f012550c2a00aa36cba8c14cc88eb88dec32eb9102554b4" dependencies = [ "bitvec", "cfg-if", @@ -11416,8 +12282,9 @@ dependencies = [ [[package]] name = "revm-context-interface" -version = "11.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "10.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b50d241ed1ce647b94caf174fcd0239b7651318b2c4c06b825b59b973dfb8495" dependencies = [ "alloy-eip2930", "alloy-eip7702", @@ -11431,8 +12298,9 @@ dependencies = [ [[package]] name = "revm-database" -version = "9.0.0" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "7.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39a276ed142b4718dcf64bc9624f474373ed82ef20611025045c3fb23edbef9c" dependencies = [ "alloy-eips", "revm-bytecode", @@ -11444,8 +12312,9 @@ dependencies = [ [[package]] name = "revm-database-interface" -version = "8.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "7.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c523c77e74eeedbac5d6f7c092e3851dbe9c7fec6f418b85992bd79229db361" dependencies = [ "auto_impl", "either", @@ -11456,8 +12325,9 @@ dependencies = [ [[package]] name = "revm-handler" -version = "11.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "10.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "550331ea85c1d257686e672081576172fe3d5a10526248b663bbf54f1bef226a" dependencies = [ "auto_impl", "derive-where", @@ -11474,8 +12344,9 @@ dependencies = [ [[package]] name = "revm-inspector" -version = "11.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "10.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c0a6e9ccc2ae006f5bed8bd80cd6f8d3832cd55c5e861b9402fdd556098512f" dependencies = [ "auto_impl", "either", @@ -11492,12 +12363,13 @@ dependencies = [ [[package]] name = "revm-inspectors" version = "0.30.0" -source = "git+https://github.com/noot/revm-inspectors?rev=84bcd1cb78c3c9dd84ad7a50a6a9e13333704598#84bcd1cb78c3c9dd84ad7a50a6a9e13333704598" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9b329afcc0f9fd5adfa2c6349a7435a8558e82bcae203142103a9a95e2a63b6" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "alloy-rpc-types-eth", "alloy-rpc-types-trace", - "alloy-sol-types 1.4.0", + "alloy-sol-types 1.4.1", "anstyle", "boa_engine", "boa_gc", @@ -11510,20 +12382,21 @@ dependencies = [ [[package]] name = "revm-interpreter" -version = "27.0.0" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "25.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575dc51b1d8f5091daa12a435733a90b4a132dca7ccee0666c7db3851bc30c" dependencies = [ "revm-bytecode", "revm-context-interface", "revm-primitives", - "revm-state", "serde", ] [[package]] name = "revm-precompile" -version = "28.0.1" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "27.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25b57d4bd9e6b5fe469da5452a8a137bc2d030a3cd47c46908efc615bbc699da" dependencies = [ "ark-bls12-381", "ark-bn254", @@ -11536,20 +12409,22 @@ dependencies = [ "c-kzg", "cfg-if", "k256", + "libsecp256k1", "p256", "revm-primitives", "ripemd", "rug", "secp256k1 0.31.1", - "sha2", + "sha2 0.10.9", ] [[package]] name = "revm-primitives" -version = "21.0.0" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "20.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aa29d9da06fe03b249b6419b33968ecdf92ad6428e2f012dc57bcd619b5d94e" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "num_enum", "once_cell", "serde", @@ -11557,8 +12432,9 @@ dependencies = [ [[package]] name = "revm-state" -version = "8.0.0" -source = "git+https://github.com/bluealloy/revm?rev=a9b4ea798aba62bfc4ff869147d764f4467aecd4#a9b4ea798aba62bfc4ff869147d764f4467aecd4" +version = "7.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f64fbacb86008394aaebd3454f9643b7d5a782bd251135e17c5b33da592d84d" dependencies = [ "bitflags 2.9.4", "revm-bytecode", @@ -11668,16 +12544,12 @@ dependencies = [ [[package]] name = "rollup-boost" version = "0.1.0" -source = "git+http://github.com/flashbots/rollup-boost?rev=b5caff73878dbde5917a475201b8766affe22ce1#b5caff73878dbde5917a475201b8766affe22ce1" +source = "git+http://github.com/flashbots/rollup-boost?rev=b86af43969557bee18f17ec1d6bcd3e984f910b2#b86af43969557bee18f17ec1d6bcd3e984f910b2" dependencies = [ - "alloy-json-rpc", - "alloy-primitives 1.4.0", - "alloy-rpc-client", + "alloy-primitives 1.4.1", "alloy-rpc-types-engine", "alloy-rpc-types-eth", "alloy-serde", - "alloy-transport", - "alloy-transport-http", "clap", "dashmap 6.1.0", "dotenvy", @@ -11688,11 +12560,7 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-util", - "jsonrpsee", - "jsonrpsee-core", - "jsonrpsee-http-client", - "jsonrpsee-server", - "jsonrpsee-types", + "jsonrpsee 0.25.1", "metrics", "metrics-derive", "metrics-exporter-prometheus 0.16.2", @@ -11704,12 +12572,11 @@ dependencies = [ "opentelemetry_sdk 0.28.0", "parking_lot", "paste", - "reth-optimism-payload-builder", - "reth-rpc-eth-types", + "reth-optimism-payload-builder 1.8.1", "rustls", "serde", "serde_json", - "sha2", + "sha2 0.10.9", "testcontainers 0.23.3", "thiserror 2.0.17", "tokio", @@ -11796,12 +12663,6 @@ version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "48fd7bd8a6377e15ad9d42a8ec25371b94ddc67abe7c8b9127bec79bebaaae18" -[[package]] -name = "rustc-demangle" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" - [[package]] name = "rustc-hash" version = "1.1.0" @@ -12371,6 +13232,19 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bbfa15b3dddfee50a0fff136974b3e1bde555604ba463834a7eb7deb6417705d" +[[package]] +name = "sha2" +version = "0.9.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d58a1e1bf39749807d89cf2d98ac2dfa0ff1cb3faa38fbb64dd88ac8013d800" +dependencies = [ + "block-buffer 0.9.0", + "cfg-if", + "cpufeatures", + "digest 0.9.0", + "opaque-debug", +] + [[package]] name = "sha2" version = "0.10.9" @@ -12546,7 +13420,7 @@ dependencies = [ "rand_core 0.6.4", "ring", "rustc_version 0.4.1", - "sha2", + "sha2 0.10.9", "subtle", ] @@ -12562,12 +13436,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233504af464074f9d066d7b5416c5f9b894a5862a6506e306f7b816cdd6f1807" +checksum = "17129e116933cf371d018bb80ae557e889637989d8638274fb25622827b03881" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -12716,9 +13590,9 @@ dependencies = [ [[package]] name = "syn-solidity" -version = "0.8.25" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4560533fbd6914b94a8fb5cc803ed6801c3455668db3b810702c57612bac9412" +checksum = "ab4e6eed052a117409a1a744c8bda9c3ea6934597cf7419f791cb7d590871c4c" dependencies = [ "paste", "proc-macro2", @@ -12728,9 +13602,9 @@ dependencies = [ [[package]] name = "syn-solidity" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2375c17f6067adc651d8c2c51658019cef32edfff4a982adaf1d7fd1c039f08b" +checksum = "ff790eb176cc81bb8936aed0f7b9f14fc4670069a2d371b3e3b0ecce908b2cb3" dependencies = [ "paste", "proc-macro2", @@ -12908,7 +13782,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" dependencies = [ "fastrand", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", "rustix 1.1.2", "windows-sys 0.61.2", @@ -13147,29 +14021,26 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.47.1" +version = "1.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89e49afdadebb872d3145a5638b59eb0691ea23e46ca484037cfab3b76b95038" +checksum = "ff360e02eab121e0bc37a2d3b4d4dc622e6eda3a8e5253d5435ecf5bd4c68408" dependencies = [ - "backtrace", "bytes", - "io-uring", "libc", "mio", "parking_lot", "pin-project-lite", "signal-hook-registry", - "slab", - "socket2 0.6.0", + "socket2 0.6.1", "tokio-macros", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] name = "tokio-macros" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" +checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", @@ -13612,7 +14483,7 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ee44f4cef85f88b4dea21c0b1f58320bdf35715cf56d840969487cff00613321" dependencies = [ - "alloy-primitives 1.4.0", + "alloy-primitives 1.4.1", "ethereum_hashing", "ethereum_ssz", "smallvec", @@ -13889,7 +14760,7 @@ version = "1.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "js-sys", "serde", "sha1_smol", @@ -14000,15 +14871,6 @@ version = "0.11.1+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" -[[package]] -name = "wasi" -version = "0.14.7+wasi-0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "883478de20367e224c0090af9cf5f9fa85bed63a95c1abf3afc5c083ebc06e8c" -dependencies = [ - "wasip2", -] - [[package]] name = "wasip2" version = "1.0.1+wasi-0.2.4" diff --git a/Cargo.toml b/Cargo.toml index b731de058..4ad71ebcd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,5 +1,5 @@ [workspace.package] -version = "0.2.4" +version = "0.2.6" edition = "2024" rust-version = "1.86" license = "MIT OR Apache-2.0" @@ -43,73 +43,73 @@ incremental = false unreachable_pub = "warn" [workspace.dependencies] -reth = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-chain-state = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-cli = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-cli-commands = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-cli-util = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-db = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-db-common = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-errors = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-node-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-rpc-engine-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-node-ethereum = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-trie = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-trie-parallel = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-basic-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-node-core = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-primitives-traits = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-provider = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702", features = [ +reth = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-chain-state = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-cli = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-cli-commands = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-cli-util = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-db = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-db-common = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-errors = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-node-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-rpc-engine-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-node-ethereum = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-trie = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-trie-parallel = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-basic-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-node-core = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-primitives-traits = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-provider = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2", features = [ "test-utils", ] } -reth-chainspec = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-storage-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-rpc-api = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-evm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-evm-ethereum = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-execution-errors = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-exex = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-tasks = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-metrics = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-trie-db = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-payload-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-transaction-pool = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-execution-types = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-revm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-payload-builder-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-payload-util = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-rpc-layer = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-network-peers = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-testing-utils = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-node-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-rpc-eth-types = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-tracing-otlp = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-ipc = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } +reth-chainspec = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-storage-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-rpc-api = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-evm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-evm-ethereum = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-execution-errors = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-exex = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-tasks = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-metrics = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-trie-db = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-payload-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-transaction-pool = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-execution-types = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-revm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-payload-builder-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-payload-util = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-rpc-layer = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-network-peers = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-testing-utils = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-node-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-rpc-eth-types = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-tracing-otlp = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-ipc = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } # reth optimism -reth-optimism-primitives = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-consensus = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-cli = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-forks = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-evm = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-node = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-payload-builder = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-chainspec = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-txpool = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702" } -reth-optimism-rpc = { git = "https://github.com/noot/reth", rev = "00d352d37c3f3ba7c23f370964b92cd13ce08702", features = [ +reth-optimism-primitives = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-consensus = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-cli = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-forks = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-evm = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-node = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-payload-builder = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-chainspec = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-txpool = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2" } +reth-optimism-rpc = { git = "https://github.com/paradigmxyz/reth", tag = "v1.8.2", features = [ "client", ] } -revm = { version = "30.0", features = [ +revm = { version = "29.0.1", features = [ "std", "secp256k1", "optional_balance_check", ], default-features = false } revm-inspectors = { version = "0.30.0", default-features = false } -op-revm = { version = "11.0.0", default-features = false } +op-revm = { version = "10.1.0", default-features = false } ethereum_ssz_derive = "0.9.0" ethereum_ssz = "0.9.0" @@ -118,7 +118,7 @@ alloy-primitives = { version = "1.3.1", default-features = false } alloy-rlp = "0.3.10" alloy-chains = "0.2.5" alloy-contract = { version = "1.0.37" } -alloy-evm = { version = "0.21.2", default-features = false } +alloy-evm = { version = "0.21.0", default-features = false } alloy-provider = { version = "1.0.37", features = [ "ipc", "pubsub", @@ -166,8 +166,8 @@ parking_lot = { version = "0.12.3" } tokio = { version = "1.40.0" } auto_impl = { version = "1.2.0" } reqwest = { version = "0.12.8" } -serde = { version = "1.0" } -serde_json = { version = "1.0" } +serde = { version = "1.0.210" } +serde_json = { version = "1.0.128" } serde_with = { version = "3.8.1" } secp256k1 = { version = "0.30" } derive_more = { version = "2" } @@ -192,20 +192,3 @@ time = { version = "0.3.36", features = ["macros", "formatting", "parsing"] } vergen = "9.0.4" vergen-git2 = "1.0.5" opentelemetry = { version = "0.29.1", features = ["trace"] } - -[patch.crates-io] -alloy-evm = { git = "https://github.com/noot/evm", rev = "f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" } -alloy-op-evm = { git = "https://github.com/noot/evm", rev = "f58c9c2e6397744939f8da9f0d5c4f0e5cd69d2e" } -revm-inspectors = { git = "https://github.com/noot/revm-inspectors", rev = "84bcd1cb78c3c9dd84ad7a50a6a9e13333704598" } -revm = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -op-revm = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } - -revm-bytecode = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-database = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-state = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-primitives = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-interpreter = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-inspector = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-context = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-context-interface = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } -revm-database-interface = { git = "https://github.com/bluealloy/revm", rev = "a9b4ea798aba62bfc4ff869147d764f4467aecd4" } diff --git a/crates/op-rbuilder/Cargo.toml b/crates/op-rbuilder/Cargo.toml index 7440bdacd..6a62052f8 100644 --- a/crates/op-rbuilder/Cargo.toml +++ b/crates/op-rbuilder/Cargo.toml @@ -131,7 +131,7 @@ ureq = "2.10" reqwest = "0.12.23" k256 = "0.13.4" -rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "b5caff73878dbde5917a475201b8766affe22ce1" } +rollup-boost = { git = "http://github.com/flashbots/rollup-boost", rev = "b86af43969557bee18f17ec1d6bcd3e984f910b2" } nanoid = { version = "0.4", optional = true } reth-ipc = { workspace = true, optional = true } diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 82b8f49fb..b17af408d 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -118,7 +118,7 @@ where Some(res) = execute_flashblock_futures.next() => { match res { Ok(Ok((payload, _))) => { - tracing::debug!("successfully executed flashblock"); + tracing::info!(hash = payload.block().hash().to_string(), block_number = payload.block().header().number, "successfully executed flashblock"); let _ = payload_events_handle.send(Events::BuiltPayload(payload)); // TODO is this only for built or also synced? } Ok(Err(e)) => { @@ -200,7 +200,6 @@ where address_gas_limiter, ); - // copy of `execute_pre_steps()` builder_ctx .evm_config .builder_for_next_block( @@ -211,10 +210,7 @@ where .wrap_err("failed to create evm builder for next block")? .apply_pre_execution_changes() .wrap_err("failed to apply pre execution changes")?; - // this is a no-op rn because `payload_config` attributes aren't set - // let mut info: ExecutionInfo = builder_ctx - // .execute_sequencer_transactions(&mut state) - // .wrap_err("failed to execute sequencer transactions")?; + let mut info = ExecutionInfo::with_capacity(payload.block().body().transactions.len()); let extra_data = payload.block().sealed_header().extra_data.clone(); From 0193f43e5b0a372e85850b98a1dd2bf52ceff35e Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 15:09:07 -0400 Subject: [PATCH 06/13] lint --- crates/op-rbuilder/src/builders/flashblocks/payload.rs | 2 +- crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index 6d0e8e5f2..ea55cd7ee 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -712,7 +712,7 @@ where match build_result { Err(err) => { ctx.metrics.invalid_blocks_count.increment(1); - return Err(err).wrap_err("failed to build payload"); + Err(err).wrap_err("failed to build payload") } Ok((new_payload, mut fb_payload)) => { fb_payload.index = ctx.flashblock_index(); diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index b17af408d..ed8ed7f87 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -48,6 +48,7 @@ impl PayloadHandler where Client: ClientBounds + 'static, { + #[allow(clippy::too_many_arguments)] pub(crate) fn new( built_rx: mpsc::Receiver, p2p_rx: mpsc::Receiver, @@ -301,6 +302,7 @@ impl reth_payload_util::PayloadTransactions for FlashblockTransactions { } } +#[allow(clippy::too_many_arguments)] fn execute_transactions( info: &mut ExecutionInfo, state: &mut State, From 378e13905fc7688f1b7f6c346cc099061830e9ab Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 15:30:18 -0400 Subject: [PATCH 07/13] remove unneeded struct --- .../builders/flashblocks/payload_handler.rs | 82 ++++++------------- 1 file changed, 23 insertions(+), 59 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index ed8ed7f87..1f73d58a7 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -21,14 +21,15 @@ use reth_optimism_node::{OpEngineTypes, OpPayloadBuilderAttributes}; use reth_optimism_payload_builder::OpBuiltPayload; use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; use rollup_boost::FlashblocksPayloadV1; -use std::{ - collections::{HashSet, VecDeque}, - sync::Arc, -}; +use std::sync::Arc; use tokio::sync::mpsc; +/// Handles newly built or received flashblock payloads. +/// +/// In the case of a payload built by this node, it is broadcast to peers and an event is sent to the payload builder. +/// In the case of a payload received from a peer, it is executed and if successful, an event is sent to the payload builder. pub(crate) struct PayloadHandler { - // receives new payloads built by us. + // receives new payloads built by this builder. built_rx: mpsc::Receiver, // receives incoming p2p messages from peers. p2p_rx: mpsc::Receiver, @@ -120,7 +121,7 @@ where match res { Ok(Ok((payload, _))) => { tracing::info!(hash = payload.block().hash().to_string(), block_number = payload.block().header().number, "successfully executed flashblock"); - let _ = payload_events_handle.send(Events::BuiltPayload(payload)); // TODO is this only for built or also synced? + let _ = payload_events_handle.send(Events::BuiltPayload(payload)); } Ok(Err(e)) => { tracing::error!(error = ?e, "failed to execute flashblock"); @@ -153,7 +154,7 @@ where tracing::info!(header = ?payload.block().header(), "executing flashblock"); - let mut cached_reads = reth::revm::cached::CachedReads::default(); // TODO: pass this in from somewhere + let mut cached_reads = reth::revm::cached::CachedReads::default(); let parent_hash = payload.block().sealed_header().parent_hash; let parent_header = client .header_by_id(parent_hash.into()) @@ -234,7 +235,7 @@ where execute_transactions( &mut info, &mut state, - &mut FlashblockTransactions::new(payload.block().body().transactions.clone()), // TODO: unnecessary + payload.block().body().transactions.clone(), payload.block().header().gas_used, &builder_ctx.evm_config, evm_env, @@ -244,7 +245,7 @@ where ) .wrap_err("failed to execute best transactions")?; - let (payload, fb_payload) = crate::builders::flashblocks::payload::build_block( + let (built_payload, fb_payload) = crate::builders::flashblocks::payload::build_block( &mut state, &builder_ctx, &mut info, @@ -252,61 +253,24 @@ where ) .wrap_err("failed to build flashblock")?; - tracing::info!(header = ?payload.block().header(), "successfully executed flashblock"); - - Ok((payload, fb_payload)) -} - -struct FlashblockTransactions { - txs: VecDeque, - invalid_txs: HashSet, -} - -impl FlashblockTransactions { - fn new(txs: Vec) -> Self { - Self { - txs: txs.into(), - invalid_txs: HashSet::new(), - } - } -} - -impl reth_payload_util::PayloadTransactions for FlashblockTransactions { - type Transaction = OpTransactionSigned; - - /// Exclude descendants of the transaction with given sender and nonce from the iterator, - /// because this transaction won't be included in the block. - fn mark_invalid(&mut self, sender: alloy_primitives::Address, nonce: u64) { - use alloy_consensus::Transaction as _; - use reth_primitives_traits::SignerRecoverable as _; - - for tx in &self.txs { - let Ok(signer) = tx.recover_signer() else { - self.invalid_txs.insert(*tx.hash()); - continue; - }; - - if signer == sender && tx.nonce() >= nonce { - self.invalid_txs.insert(*tx.hash()); - } - } + if built_payload.block().hash() != payload.block().hash() { + tracing::error!( + expected = %payload.block().hash(), + got = %built_payload.block().hash(), + "flashblock hash mismatch after execution" + ); + bail!("flashblock hash mismatch after execution"); } - fn next(&mut self, _ctx: ()) -> Option { - while let Some(tx) = self.txs.pop_front() { - if !self.invalid_txs.contains(tx.hash()) { - return Some(tx); - } - } - None - } + tracing::info!(header = ?built_payload.block().header(), "successfully executed flashblock"); + Ok((built_payload, fb_payload)) } #[allow(clippy::too_many_arguments)] fn execute_transactions( info: &mut ExecutionInfo, state: &mut State, - txs: &mut impl reth_payload_util::PayloadTransactions, + txs: Vec, gas_limit: u64, evm_config: &reth_optimism_evm::OpEvmConfig, evm_env: alloy_evm::EvmEnv, @@ -325,13 +289,13 @@ fn execute_transactions( let mut evm = evm_config.evm_with_env(&mut *state, evm_env); - while let Some(ref tx) = txs.next(()) { + for tx in txs { let sender = tx .recover_signer() .wrap_err("failed to recover tx signer")?; let tx_env = TxEnv::from_recovered_tx(&tx, sender); let executable_tx = match tx { - OpTxEnvelope::Deposit(tx) => { + OpTxEnvelope::Deposit(ref tx) => { let deposit = DepositTransactionParts { mint: Some(tx.mint), source_hash: tx.source_hash, @@ -407,7 +371,7 @@ fn execute_transactions( .wrap_err("failed to get depositor nonce")?; let ctx = ReceiptBuilderCtx { - tx, + tx: &tx, evm: &evm, result, state: &state, From 86a36331edd6c0c34aee6e13396a84e8c91d8025 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 15:56:28 -0400 Subject: [PATCH 08/13] execute flashblock cleanup --- .../src/builders/flashblocks/ctx.rs | 16 ++-- .../builders/flashblocks/payload_handler.rs | 80 +++++++++---------- .../src/builders/flashblocks/service.rs | 3 +- 3 files changed, 49 insertions(+), 50 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/ctx.rs b/crates/op-rbuilder/src/builders/flashblocks/ctx.rs index 21aa4e1a1..c465445f5 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/ctx.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/ctx.rs @@ -1,6 +1,6 @@ use crate::{ builders::{BuilderConfig, OpPayloadBuilderCtx, flashblocks::FlashblocksConfig}, - gas_limiter::AddressGasLimiter, + gas_limiter::{AddressGasLimiter, args::GasLimiterArgs}, metrics::OpRBuilderMetrics, traits::ClientBounds, }; @@ -24,6 +24,8 @@ pub(super) struct OpPayloadSyncerCtx { chain_spec: Arc, /// Max gas that can be used by a transaction. max_gas_per_txn: Option, + /// The metrics for the builder + metrics: Arc, } impl OpPayloadSyncerCtx { @@ -31,6 +33,7 @@ impl OpPayloadSyncerCtx { client: &Client, builder_config: BuilderConfig, evm_config: OpEvmConfig, + metrics: Arc, ) -> eyre::Result where Client: ClientBounds, @@ -41,6 +44,7 @@ impl OpPayloadSyncerCtx { da_config: builder_config.da_config.clone(), chain_spec, max_gas_per_txn: builder_config.max_gas_per_txn, + metrics, }) } @@ -48,14 +52,16 @@ impl OpPayloadSyncerCtx { &self.evm_config } + pub(super) fn max_gas_per_txn(&self) -> Option { + self.max_gas_per_txn + } + pub(super) fn into_op_payload_builder_ctx( self, payload_config: PayloadConfig>, evm_env: EvmEnv, block_env_attributes: OpNextBlockEnvAttributes, cancel: CancellationToken, - metrics: Arc, - address_gas_limiter: AddressGasLimiter, ) -> OpPayloadBuilderCtx { OpPayloadBuilderCtx { evm_config: self.evm_config, @@ -66,10 +72,10 @@ impl OpPayloadSyncerCtx { block_env_attributes, cancel, builder_signer: None, - metrics, + metrics: self.metrics, extra_ctx: (), max_gas_per_txn: self.max_gas_per_txn, - address_gas_limiter, + address_gas_limiter: AddressGasLimiter::new(GasLimiterArgs::default()), } } } diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 1f73d58a7..1c8db1122 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -1,6 +1,5 @@ use crate::{ builders::flashblocks::{ctx::OpPayloadSyncerCtx, p2p::Message, payload::ExtraExecutionInfo}, - gas_limiter::{AddressGasLimiter, args::GasLimiterArgs}, metrics::OpRBuilderMetrics, primitives::reth::ExecutionInfo, traits::ClientBounds, @@ -20,6 +19,7 @@ use reth_optimism_evm::{OpEvmConfig, OpNextBlockEnvAttributes}; use reth_optimism_node::{OpEngineTypes, OpPayloadBuilderAttributes}; use reth_optimism_payload_builder::OpBuiltPayload; use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; +use reth_payload_builder::EthPayloadBuilderAttributes; use rollup_boost::FlashblocksPayloadV1; use std::sync::Arc; use tokio::sync::mpsc; @@ -40,7 +40,6 @@ pub(crate) struct PayloadHandler { // context required for execution of blocks during syncing ctx: OpPayloadSyncerCtx, metrics: Arc, - gas_limiter_config: GasLimiterArgs, client: Client, cancel: tokio_util::sync::CancellationToken, } @@ -57,7 +56,6 @@ where payload_events_handle: tokio::sync::broadcast::Sender>, ctx: OpPayloadSyncerCtx, metrics: Arc, - gas_limiter_config: GasLimiterArgs, client: Client, cancel: tokio_util::sync::CancellationToken, ) -> Self { @@ -68,7 +66,6 @@ where payload_events_handle, ctx, metrics, - gas_limiter_config, client, cancel, } @@ -82,7 +79,6 @@ where payload_events_handle, ctx, metrics, - gas_limiter_config, client, cancel, } = self; @@ -96,7 +92,7 @@ where Some(payload) = built_rx.recv() => { let _ = payload_events_handle.send(Events::BuiltPayload(payload.clone())); // TODO: only broadcast if `!no_tx_pool`? - // ignore error here; if p2p was disabled, the channel will be closed. + // ignore error here; if p2p is disabled, the channel will be closed. let _ = p2p_tx.send(payload.into()).await; } Some(message) = p2p_rx.recv() => { @@ -108,9 +104,7 @@ where payload, ctx.clone(), client.clone(), - metrics.clone(), cancel.clone(), - gas_limiter_config.clone(), ) ); execute_flashblock_futures.push(handle); @@ -141,9 +135,7 @@ async fn execute_flashblock( payload: OpBuiltPayload, ctx: OpPayloadSyncerCtx, client: Client, - metrics: Arc, cancel: tokio_util::sync::CancellationToken, - gas_limiter_config: GasLimiterArgs, ) -> eyre::Result<(OpBuiltPayload, FlashblocksPayloadV1)> where Client: ClientBounds, @@ -160,11 +152,6 @@ where .header_by_id(parent_hash.into()) .wrap_err("failed to get parent header")? .ok_or_else(|| eyre::eyre!("parent header not found"))?; - // TODO: can refactor this out probably - let payload_config = PayloadConfig::new( - Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), - OpPayloadBuilderAttributes::default(), - ); let state_provider = client .state_by_block_hash(parent_hash) @@ -191,23 +178,11 @@ where .next_evm_env(&parent_header, &block_env_attributes) .wrap_err("failed to create next evm env")?; - let address_gas_limiter = AddressGasLimiter::new(gas_limiter_config); - // TODO: can probably refactor this - let mut builder_ctx = ctx.into_op_payload_builder_ctx( - payload_config, - evm_env.clone(), - block_env_attributes, - cancel, - metrics, - address_gas_limiter, - ); - - builder_ctx - .evm_config + ctx.evm_config() .builder_for_next_block( &mut state, - &builder_ctx.config.parent_header, - builder_ctx.block_env_attributes.clone(), + &Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), + block_env_attributes.clone(), ) .wrap_err("failed to create evm builder for next block")? .apply_pre_execution_changes() @@ -222,34 +197,53 @@ where } let eip_1559_parameters: B64 = extra_data[1..9].try_into().unwrap(); - builder_ctx.config.attributes.eip_1559_params = Some(eip_1559_parameters); - builder_ctx - .config - .attributes - .payload_attributes - .parent_beacon_block_root = payload.block().sealed_header().parent_beacon_block_root; - builder_ctx.config.attributes.payload_attributes.timestamp = timestamp; - builder_ctx.config.attributes.payload_attributes.prev_randao = - payload.block().sealed_header().mix_hash; + let payload_config = PayloadConfig::new( + Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), + OpPayloadBuilderAttributes { + eip_1559_params: Some(eip_1559_parameters), + payload_attributes: EthPayloadBuilderAttributes { + id: payload.id(), // unused + parent: parent_hash, // unused + suggested_fee_recipient: payload.block().sealed_header().beneficiary, + withdrawals: payload + .block() + .body() + .withdrawals + .clone() + .unwrap_or_default(), + parent_beacon_block_root: payload.block().sealed_header().parent_beacon_block_root, + timestamp, + prev_randao: payload.block().sealed_header().mix_hash, + }, + ..Default::default() + }, + ); execute_transactions( &mut info, &mut state, payload.block().body().transactions.clone(), payload.block().header().gas_used, - &builder_ctx.evm_config, - evm_env, - builder_ctx.max_gas_per_txn, + ctx.evm_config(), + evm_env.clone(), + ctx.max_gas_per_txn(), is_canyon_active(&chain_spec, timestamp), is_regolith_active(&chain_spec, timestamp), ) .wrap_err("failed to execute best transactions")?; + let builder_ctx = ctx.into_op_payload_builder_ctx( + payload_config, + evm_env.clone(), + block_env_attributes, + cancel, + ); + let (built_payload, fb_payload) = crate::builders::flashblocks::payload::build_block( &mut state, &builder_ctx, &mut info, - true, // TODO: do we need this always? + true, // ) .wrap_err("failed to build flashblock")?; diff --git a/crates/op-rbuilder/src/builders/flashblocks/service.rs b/crates/op-rbuilder/src/builders/flashblocks/service.rs index 119674b2b..cc0022eb2 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/service.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/service.rs @@ -98,7 +98,6 @@ impl FlashblocksServiceBuilder { }; let metrics = Arc::new(OpRBuilderMetrics::default()); - let gas_limiter_config = self.0.gas_limiter_config.clone(); let (built_payload_tx, built_payload_rx) = tokio::sync::mpsc::channel(16); let payload_builder = OpPayloadBuilder::new( OpEvmConfig::optimism(ctx.chain_spec()), @@ -129,6 +128,7 @@ impl FlashblocksServiceBuilder { &ctx.provider().clone(), self.0, OpEvmConfig::optimism(ctx.chain_spec()), + metrics.clone(), ) .wrap_err("failed to create flashblocks payload builder context")?; @@ -139,7 +139,6 @@ impl FlashblocksServiceBuilder { payload_service.payload_events_handle(), syncer_ctx, metrics, - gas_limiter_config, ctx.provider().clone(), cancel, ); From c8c26e0f0b247977dbfefbeb5ab3f4677fa6d173 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 16:06:18 -0400 Subject: [PATCH 09/13] add metrics --- .../src/builders/flashblocks/payload.rs | 2 +- .../src/builders/flashblocks/payload_handler.rs | 16 +++++++++++----- .../src/builders/flashblocks/service.rs | 1 - crates/op-rbuilder/src/metrics.rs | 8 +++++++- 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index ea55cd7ee..2beecd057 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -711,7 +711,7 @@ where match build_result { Err(err) => { - ctx.metrics.invalid_blocks_count.increment(1); + ctx.metrics.invalid_built_blocks_count.increment(1); Err(err).wrap_err("failed to build payload") } Ok((new_payload, mut fb_payload)) => { diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 1c8db1122..380787bdb 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -1,6 +1,5 @@ use crate::{ builders::flashblocks::{ctx::OpPayloadSyncerCtx, p2p::Message, payload::ExtraExecutionInfo}, - metrics::OpRBuilderMetrics, primitives::reth::ExecutionInfo, traits::ClientBounds, }; @@ -39,7 +38,7 @@ pub(crate) struct PayloadHandler { payload_events_handle: tokio::sync::broadcast::Sender>, // context required for execution of blocks during syncing ctx: OpPayloadSyncerCtx, - metrics: Arc, + // chain client client: Client, cancel: tokio_util::sync::CancellationToken, } @@ -55,7 +54,6 @@ where p2p_tx: mpsc::Sender, payload_events_handle: tokio::sync::broadcast::Sender>, ctx: OpPayloadSyncerCtx, - metrics: Arc, client: Client, cancel: tokio_util::sync::CancellationToken, ) -> Self { @@ -65,7 +63,6 @@ where p2p_tx, payload_events_handle, ctx, - metrics, client, cancel, } @@ -78,7 +75,6 @@ where p2p_tx, payload_events_handle, ctx, - metrics, client, cancel, } = self; @@ -144,6 +140,8 @@ where use reth::primitives::SealedHeader; use reth_evm::{ConfigureEvm as _, execute::BlockBuilder as _}; + let start = tokio::time::Instant::now(); + tracing::info!(header = ?payload.block().header(), "executing flashblock"); let mut cached_reads = reth::revm::cached::CachedReads::default(); @@ -247,15 +245,23 @@ where ) .wrap_err("failed to build flashblock")?; + builder_ctx + .metrics + .flashblock_sync_duration + .record(start.elapsed()); + if built_payload.block().hash() != payload.block().hash() { tracing::error!( expected = %payload.block().hash(), got = %built_payload.block().hash(), "flashblock hash mismatch after execution" ); + builder_ctx.metrics.invalid_synced_blocks_count.increment(1); bail!("flashblock hash mismatch after execution"); } + builder_ctx.metrics.block_synced_success.increment(1); + tracing::info!(header = ?built_payload.block().header(), "successfully executed flashblock"); Ok((built_payload, fb_payload)) } diff --git a/crates/op-rbuilder/src/builders/flashblocks/service.rs b/crates/op-rbuilder/src/builders/flashblocks/service.rs index cc0022eb2..9493ecb5f 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/service.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/service.rs @@ -138,7 +138,6 @@ impl FlashblocksServiceBuilder { outgoing_message_tx, payload_service.payload_events_handle(), syncer_ctx, - metrics, ctx.provider().clone(), cancel, ); diff --git a/crates/op-rbuilder/src/metrics.rs b/crates/op-rbuilder/src/metrics.rs index dd43d9580..d5b54069a 100644 --- a/crates/op-rbuilder/src/metrics.rs +++ b/crates/op-rbuilder/src/metrics.rs @@ -63,6 +63,8 @@ pub const VERSION: VersionInfo = VersionInfo { pub struct OpRBuilderMetrics { /// Block built success pub block_built_success: Counter, + /// Block synced success + pub block_synced_success: Counter, /// Number of flashblocks added to block (Total per block) pub flashblock_count: Histogram, /// Number of messages sent @@ -73,12 +75,16 @@ pub struct OpRBuilderMetrics { pub total_block_built_gauge: Gauge, /// Histogram of the time taken to build a Flashblock pub flashblock_build_duration: Histogram, + /// Histogram of the time taken to sync a Flashblock + pub flashblock_sync_duration: Histogram, /// Flashblock UTF8 payload byte size histogram pub flashblock_byte_size_histogram: Histogram, /// Histogram of transactions in a Flashblock pub flashblock_num_tx_histogram: Histogram, /// Number of invalid blocks - pub invalid_blocks_count: Counter, + pub invalid_built_blocks_count: Counter, + /// Number of invalid synced blocks + pub invalid_synced_blocks_count: Counter, /// Histogram of fetching transactions from the pool duration pub transaction_pool_fetch_duration: Histogram, /// Latest time taken to fetch tx from the pool From 29e1a857a0bc15aa7d59972bb3e14b3a1b8ba724 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 16:13:36 -0400 Subject: [PATCH 10/13] cleanup --- .../builders/flashblocks/payload_handler.rs | 5 +-- .../src/builders/flashblocks/service.rs | 1 + crates/p2p/src/lib.rs | 34 ++++++++----------- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index 380787bdb..bcc2f1433 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -191,9 +191,10 @@ where let extra_data = payload.block().sealed_header().extra_data.clone(); if extra_data.len() != 9 { tracing::error!(len = extra_data.len(), data = ?extra_data, "invalid extra data length in flashblock"); - eyre::bail!("extra data length should be 9 bytes"); + bail!("extra data length should be 9 bytes"); } + // see https://specs.optimism.io/protocol/holocene/exec-engine.html#eip-1559-parameters-in-block-header let eip_1559_parameters: B64 = extra_data[1..9].try_into().unwrap(); let payload_config = PayloadConfig::new( Arc::new(SealedHeader::new(parent_header.clone(), parent_hash)), @@ -241,7 +242,7 @@ where &mut state, &builder_ctx, &mut info, - true, // + true, ) .wrap_err("failed to build flashblock")?; diff --git a/crates/op-rbuilder/src/builders/flashblocks/service.rs b/crates/op-rbuilder/src/builders/flashblocks/service.rs index 9493ecb5f..25d928c40 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/service.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/service.rs @@ -39,6 +39,7 @@ impl FlashblocksServiceBuilder { BuilderTx: BuilderTransactions + Unpin + Clone + Send + Sync + 'static, { // TODO: is there a different global token? + // this is effectively unused right now due to the usage of reth's `task_executor`. let cancel = tokio_util::sync::CancellationToken::new(); let (incoming_message_rx, outgoing_message_tx) = if self.0.p2p_enabled { diff --git a/crates/p2p/src/lib.rs b/crates/p2p/src/lib.rs index 93912cb07..f15f12b5b 100644 --- a/crates/p2p/src/lib.rs +++ b/crates/p2p/src/lib.rs @@ -144,7 +144,7 @@ impl Node { } Some(message) = outgoing_message_rx.recv() => { let protocol = message.protocol(); - info!("received message to broadcast on protocol {protocol}"); + debug!("received message to broadcast on protocol {protocol}"); if let Err(e) = outgoing_streams_handler.broadcast_message(message).await { warn!("failed to broadcast message on protocol {protocol}: {e:?}"); } @@ -167,28 +167,22 @@ impl Node { } => { // when a new connection is established, open outbound streams for each protocol // and add them to the outgoing streams handler. - // - // If we already have a connection with this peer, close the new connection, - // as we only want one connection per peer. info!("connection established with peer {peer_id}"); - if outgoing_streams_handler.has_peer(&peer_id) { - //swarm.close_connection(connection_id); - //info!("already have connection with peer {peer_id}, closed connection {connection_id}"); - } else { + if !outgoing_streams_handler.has_peer(&peer_id) { for protocol in &protocols { - match swarm - .behaviour_mut() - .new_control() - .open_stream(peer_id, protocol.clone()) - .await - { - Ok(stream) => { outgoing_streams_handler.insert_peer_and_stream(peer_id, protocol.clone(), stream); - info!("opened outbound stream with peer {peer_id} with protocol {protocol} on connection {connection_id}"); + match swarm + .behaviour_mut() + .new_control() + .open_stream(peer_id, protocol.clone()) + .await + { + Ok(stream) => { outgoing_streams_handler.insert_peer_and_stream(peer_id, protocol.clone(), stream); + debug!("opened outbound stream with peer {peer_id} with protocol {protocol} on connection {connection_id}"); + } + Err(e) => { + warn!("failed to open stream with peer {peer_id} on connection {connection_id}: {e:?}"); + } } - Err(e) => { - warn!("failed to open stream with peer {peer_id} on connection {connection_id}: {e:?}"); - } - } } } } From 27c4046358ab4d0d327ae38f2c711e1b44237432 Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 20:00:06 -0400 Subject: [PATCH 11/13] wip fix tests --- crates/op-rbuilder/src/builders/flashblocks/payload.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index 2beecd057..6e38beaf9 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -407,7 +407,7 @@ where info!( target: "payload_builder", message = "Performed flashblocks timing derivation", - flashblocks_per_block = ctx.target_flashblock_count(), + flashblocks_per_block, first_flashblock_offset = first_flashblock_offset.as_millis(), flashblocks_interval = self.config.specific.interval.as_millis(), ); @@ -419,12 +419,12 @@ where ctx.metrics .first_flashblock_time_offset .record(first_flashblock_offset.as_millis() as f64); - let gas_per_batch = ctx.block_gas_limit() / ctx.target_flashblock_count(); + let gas_per_batch = ctx.block_gas_limit() / flashblocks_per_block; let target_gas_for_batch = gas_per_batch; let da_per_batch = ctx .da_config .max_da_block_size() - .map(|da_limit| da_limit / ctx.target_flashblock_count()); + .map(|da_limit| da_limit / flashblocks_per_block); // Check that builder tx won't affect fb limit too much if let Some(da_limit) = da_per_batch { // We error if we can't insert any tx aside from builder tx in flashblock From 37d46af33164c60b8dc3bfe554f0880cfa7865ef Mon Sep 17 00:00:00 2001 From: elizabeth Date: Wed, 15 Oct 2025 23:45:04 -0400 Subject: [PATCH 12/13] fix tests --- crates/op-rbuilder/src/builders/flashblocks/payload.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload.rs b/crates/op-rbuilder/src/builders/flashblocks/payload.rs index 6e38beaf9..9623d0f3b 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload.rs @@ -441,7 +441,7 @@ where *da_limit = da_limit.saturating_sub(builder_tx_da_size); } let extra_ctx = FlashblocksExtraCtx { - flashblock_index: 0, + flashblock_index: 1, target_flashblock_count: flashblocks_per_block, target_gas_for_batch: target_gas_for_batch.saturating_sub(builder_tx_gas), target_da_for_batch, @@ -580,9 +580,6 @@ where best_payload: &BlockCell, span: &tracing::Span, ) -> eyre::Result> { - // fallback block is index 0, so we need to increment here - let flashblock_index = ctx.next_flashblock_index(); - // TODO: remove this if ctx.flashblock_index() > ctx.target_flashblock_count() { info!( @@ -765,7 +762,7 @@ where let target_gas_for_batch = ctx.extra_ctx.target_gas_for_batch + ctx.extra_ctx.gas_per_batch; let next_extra_ctx = FlashblocksExtraCtx { - flashblock_index, + flashblock_index: ctx.next_flashblock_index(), target_flashblock_count: ctx.target_flashblock_count(), target_gas_for_batch, target_da_for_batch, @@ -777,7 +774,7 @@ where info!( target: "payload_builder", message = "Flashblock built", - flashblock_index = flashblock_index, + flashblock_index = ctx.flashblock_index(), current_gas = info.cumulative_gas_used, current_da = info.cumulative_da_bytes_used, target_flashblocks = ctx.target_flashblock_count(), From 986c99ec5e68ce5437eb77b14baddb1efbe686fa Mon Sep 17 00:00:00 2001 From: elizabeth Date: Thu, 16 Oct 2025 00:40:43 -0400 Subject: [PATCH 13/13] remove comment --- crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs index bcc2f1433..4c1108d0c 100644 --- a/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs +++ b/crates/op-rbuilder/src/builders/flashblocks/payload_handler.rs @@ -87,7 +87,6 @@ where tokio::select! { Some(payload) = built_rx.recv() => { let _ = payload_events_handle.send(Events::BuiltPayload(payload.clone())); - // TODO: only broadcast if `!no_tx_pool`? // ignore error here; if p2p is disabled, the channel will be closed. let _ = p2p_tx.send(payload.into()).await; }