mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat(op, txpool): impl send_raw_tx_cond endpoint (#14311)
This commit is contained in:
@ -55,6 +55,7 @@ revm.workspace = true
|
||||
parking_lot.workspace = true
|
||||
tokio.workspace = true
|
||||
reqwest = { workspace = true, features = ["rustls-tls-native-roots"] }
|
||||
async-trait.workspace = true
|
||||
|
||||
# rpc
|
||||
jsonrpsee-core.workspace = true
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
//! RPC errors specific to OP.
|
||||
|
||||
use alloy_rpc_types_eth::{error::EthRpcErrorCode, BlockError};
|
||||
use jsonrpsee_types::error::INTERNAL_ERROR_CODE;
|
||||
use jsonrpsee_types::error::{INTERNAL_ERROR_CODE, INVALID_PARAMS_CODE};
|
||||
use reth_optimism_evm::OpBlockExecutionError;
|
||||
use reth_rpc_eth_api::AsEthApiError;
|
||||
use reth_rpc_eth_types::{error::api::FromEvmHalt, EthApiError};
|
||||
@ -62,6 +62,9 @@ pub enum OpInvalidTransactionError {
|
||||
/// A deposit transaction halted post-regolith
|
||||
#[error("deposit transaction halted after regolith")]
|
||||
HaltedDepositPostRegolith,
|
||||
/// Transaction conditional errors.
|
||||
#[error(transparent)]
|
||||
TxConditionalErr(#[from] TxConditionalErr),
|
||||
}
|
||||
|
||||
impl From<OpInvalidTransactionError> for jsonrpsee_types::error::ErrorObject<'static> {
|
||||
@ -71,6 +74,7 @@ impl From<OpInvalidTransactionError> for jsonrpsee_types::error::ErrorObject<'st
|
||||
OpInvalidTransactionError::HaltedDepositPostRegolith => {
|
||||
rpc_err(EthRpcErrorCode::TransactionRejected.code(), err.to_string(), None)
|
||||
}
|
||||
OpInvalidTransactionError::TxConditionalErr(_) => err.into(),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -93,6 +97,27 @@ impl TryFrom<InvalidTransaction> for OpInvalidTransactionError {
|
||||
}
|
||||
}
|
||||
|
||||
/// Transaction conditional related error type
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum TxConditionalErr {
|
||||
/// Transaction conditional cost exceeded maximum allowed
|
||||
#[error("conditional cost exceeded maximum allowed")]
|
||||
ConditionalCostExceeded,
|
||||
/// Invalid conditional parameters
|
||||
#[error("invalid conditional parameters")]
|
||||
InvalidCondition,
|
||||
}
|
||||
|
||||
impl From<TxConditionalErr> for jsonrpsee_types::error::ErrorObject<'static> {
|
||||
fn from(err: TxConditionalErr) -> Self {
|
||||
jsonrpsee_types::error::ErrorObject::owned(
|
||||
INVALID_PARAMS_CODE,
|
||||
err.to_string(),
|
||||
None::<String>,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/// Error type when interacting with the Sequencer
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum SequencerClientError {
|
||||
|
||||
107
crates/optimism/rpc/src/eth/ext.rs
Normal file
107
crates/optimism/rpc/src/eth/ext.rs
Normal file
@ -0,0 +1,107 @@
|
||||
use super::{OpEthApiInner, OpNodeCore};
|
||||
use crate::{error::TxConditionalErr, OpEthApiError, SequencerClient};
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::BlockNumberOrTag;
|
||||
use alloy_primitives::{Bytes, B256};
|
||||
use alloy_rpc_types_eth::erc4337::TransactionConditional;
|
||||
use jsonrpsee_core::RpcResult;
|
||||
use op_alloy_network::TransactionResponse;
|
||||
use op_alloy_rpc_types::Transaction;
|
||||
use reth_provider::{BlockReaderIdExt, StateProviderFactory};
|
||||
use reth_rpc_eth_api::L2EthApiExtServer;
|
||||
use reth_transaction_pool::{TransactionOrigin, TransactionPool};
|
||||
use std::sync::Arc;
|
||||
|
||||
/// Maximum execution const for conditional transactions.
|
||||
const MAX_CONDITIONAL_EXECUTION_COST: u64 = 5000;
|
||||
|
||||
/// OP-Reth `Eth` API extensions implementation.
|
||||
///
|
||||
/// Separate from [`super::OpEthApi`] to allow to enable it conditionally,
|
||||
#[derive(Clone)]
|
||||
#[allow(dead_code)]
|
||||
pub(crate) struct OpEthApiExt<N: OpNodeCore> {
|
||||
/// Gateway to node's core components.
|
||||
inner: Arc<OpEthApiInner<N>>,
|
||||
}
|
||||
|
||||
impl<N> OpEthApiExt<N>
|
||||
where
|
||||
N: OpNodeCore<Provider: BlockReaderIdExt + Clone + 'static>,
|
||||
{
|
||||
/// Returns the configured sequencer client, if any.
|
||||
pub(crate) fn sequencer_client(&self) -> Option<&SequencerClient> {
|
||||
self.inner.sequencer_client()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn pool(&self) -> &N::Pool {
|
||||
self.inner.eth_api.pool()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn provider(&self) -> &N::Provider {
|
||||
self.inner.eth_api.provider()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<N> L2EthApiExtServer for OpEthApiExt<N>
|
||||
where
|
||||
N: OpNodeCore + 'static,
|
||||
N::Provider: BlockReaderIdExt + StateProviderFactory,
|
||||
N::Pool: TransactionPool<Transaction = Transaction>,
|
||||
{
|
||||
async fn send_raw_transaction_conditional(
|
||||
&self,
|
||||
bytes: Bytes,
|
||||
condition: TransactionConditional,
|
||||
) -> RpcResult<B256> {
|
||||
// calculate and validate cost
|
||||
let cost = condition.cost();
|
||||
if cost > MAX_CONDITIONAL_EXECUTION_COST {
|
||||
return Err(TxConditionalErr::ConditionalCostExceeded.into());
|
||||
}
|
||||
|
||||
let tx: Transaction = serde_json::from_slice(&bytes).map_err(|_| {
|
||||
OpEthApiError::Eth(reth_rpc_eth_types::EthApiError::FailedToDecodeSignedTransaction)
|
||||
})?;
|
||||
|
||||
// get current header
|
||||
let header_not_found = || {
|
||||
OpEthApiError::Eth(reth_rpc_eth_types::EthApiError::HeaderNotFound(
|
||||
alloy_eips::BlockId::Number(BlockNumberOrTag::Latest),
|
||||
))
|
||||
};
|
||||
let header = self
|
||||
.provider()
|
||||
.latest_header()
|
||||
.map_err(|_| header_not_found())?
|
||||
.ok_or_else(header_not_found)?;
|
||||
|
||||
// check condition against header
|
||||
if !condition.has_exceeded_block_number(header.header().number()) ||
|
||||
!condition.has_exceeded_timestamp(header.header().timestamp())
|
||||
{
|
||||
return Err(TxConditionalErr::InvalidCondition.into());
|
||||
}
|
||||
|
||||
// TODO: check condition against state
|
||||
|
||||
if let Some(sequencer) = self.sequencer_client() {
|
||||
// If we have a sequencer client, forward the transaction
|
||||
let _ = sequencer
|
||||
.forward_raw_transaction_conditional(bytes.as_ref(), condition)
|
||||
.await
|
||||
.map_err(OpEthApiError::Sequencer)?;
|
||||
Ok(tx.tx_hash())
|
||||
} else {
|
||||
// otherwise, add to pool
|
||||
// TODO: include conditional
|
||||
let hash = self.pool().add_transaction(TransactionOrigin::External, tx).await.map_err(
|
||||
|e| OpEthApiError::Eth(reth_rpc_eth_types::EthApiError::PoolError(e.into())),
|
||||
)?;
|
||||
Ok(hash)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -5,20 +5,19 @@ pub mod transaction;
|
||||
|
||||
mod block;
|
||||
mod call;
|
||||
mod ext;
|
||||
mod pending_block;
|
||||
|
||||
pub use receipt::{OpReceiptBuilder, OpReceiptFieldsBuilder};
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_optimism_primitives::OpPrimitives;
|
||||
|
||||
use std::{fmt, sync::Arc};
|
||||
|
||||
use alloy_primitives::U256;
|
||||
use op_alloy_network::Optimism;
|
||||
use reth_chainspec::{EthChainSpec, EthereumHardforks};
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_node_builder::EthApiBuilderCtx;
|
||||
use reth_optimism_primitives::OpPrimitives;
|
||||
use reth_provider::{
|
||||
BlockNumReader, BlockReader, BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider,
|
||||
NodePrimitivesProvider, ProviderBlock, ProviderHeader, ProviderReceipt, ProviderTx,
|
||||
@ -38,6 +37,7 @@ use reth_tasks::{
|
||||
TaskSpawner,
|
||||
};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
use std::{fmt, sync::Arc};
|
||||
|
||||
use crate::{OpEthApiError, SequencerClient};
|
||||
|
||||
|
||||
@ -6,6 +6,7 @@ use std::sync::{
|
||||
};
|
||||
|
||||
use alloy_primitives::hex;
|
||||
use alloy_rpc_types_eth::erc4337::TransactionConditional;
|
||||
use reqwest::Client;
|
||||
use serde_json::json;
|
||||
use tracing::warn;
|
||||
@ -82,6 +83,43 @@ impl SequencerClient {
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Forwards a transaction conditional to the sequencer endpoint.
|
||||
pub async fn forward_raw_transaction_conditional(
|
||||
&self,
|
||||
tx: &[u8],
|
||||
condition: TransactionConditional,
|
||||
) -> Result<(), SequencerClientError> {
|
||||
let body = serde_json::to_string(&json!({
|
||||
"jsonrpc": "2.0",
|
||||
"method": "eth_sendRawTransactionConditional",
|
||||
"params": [format!("0x{}", hex::encode(tx)), condition],
|
||||
"id": self.next_request_id()
|
||||
}))
|
||||
.map_err(|_| {
|
||||
warn!(
|
||||
target = "rpc::eth",
|
||||
"Failed to serialize transaction conditional for forwarding to sequencer"
|
||||
);
|
||||
SequencerClientError::InvalidSequencerTransaction
|
||||
})?;
|
||||
|
||||
self.http_client()
|
||||
.post(self.endpoint())
|
||||
.header(reqwest::header::CONTENT_TYPE, "application/json")
|
||||
.body(body)
|
||||
.send()
|
||||
.await
|
||||
.inspect_err(|err| {
|
||||
warn!(
|
||||
target = "rpc::eth",
|
||||
%err,
|
||||
"Failed to forward transaction conditional to sequencer",
|
||||
);
|
||||
})?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Default)]
|
||||
|
||||
Reference in New Issue
Block a user