reth_rpc/
validation.rs

1use alloy_consensus::{
2    BlobTransactionValidationError, BlockHeader, EnvKzgSettings, Transaction, TxReceipt,
3};
4use alloy_eips::{eip4844::kzg_to_versioned_hash, eip7685::RequestsOrHash};
5use alloy_rpc_types_beacon::relay::{
6    BidTrace, BuilderBlockValidationRequest, BuilderBlockValidationRequestV2,
7    BuilderBlockValidationRequestV3, BuilderBlockValidationRequestV4,
8};
9use alloy_rpc_types_engine::{
10    BlobsBundleV1, CancunPayloadFields, ExecutionData, ExecutionPayload, ExecutionPayloadSidecar,
11    PraguePayloadFields,
12};
13use async_trait::async_trait;
14use core::fmt;
15use jsonrpsee::core::RpcResult;
16use jsonrpsee_types::error::ErrorObject;
17use reth_chainspec::{ChainSpecProvider, EthereumHardforks};
18use reth_consensus::{Consensus, FullConsensus};
19use reth_engine_primitives::PayloadValidator;
20use reth_errors::{BlockExecutionError, ConsensusError, ProviderError};
21use reth_evm::{execute::Executor, ConfigureEvm};
22use reth_execution_types::BlockExecutionOutput;
23use reth_metrics::{metrics, metrics::Gauge, Metrics};
24use reth_node_api::NewPayloadError;
25use reth_primitives_traits::{
26    constants::GAS_LIMIT_BOUND_DIVISOR, BlockBody, GotExpected, NodePrimitives, RecoveredBlock,
27    SealedBlock, SealedHeaderFor,
28};
29use reth_revm::{cached::CachedReads, database::StateProviderDatabase};
30use reth_rpc_api::BlockSubmissionValidationApiServer;
31use reth_rpc_server_types::result::{internal_rpc_err, invalid_params_rpc_err};
32use reth_storage_api::{BlockReaderIdExt, StateProviderFactory};
33use reth_tasks::TaskSpawner;
34use revm_primitives::{Address, B256, U256};
35use serde::{Deserialize, Serialize};
36use std::{collections::HashSet, sync::Arc};
37use tokio::sync::{oneshot, RwLock};
38use tracing::warn;
39
40/// The type that implements the `validation` rpc namespace trait
41#[derive(Clone, Debug, derive_more::Deref)]
42pub struct ValidationApi<Provider, E: ConfigureEvm> {
43    #[deref]
44    inner: Arc<ValidationApiInner<Provider, E>>,
45}
46
47impl<Provider, E> ValidationApi<Provider, E>
48where
49    E: ConfigureEvm,
50{
51    /// Create a new instance of the [`ValidationApi`]
52    pub fn new(
53        provider: Provider,
54        consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
55        evm_config: E,
56        config: ValidationApiConfig,
57        task_spawner: Box<dyn TaskSpawner>,
58        payload_validator: Arc<
59            dyn PayloadValidator<
60                Block = <E::Primitives as NodePrimitives>::Block,
61                ExecutionData = ExecutionData,
62            >,
63        >,
64    ) -> Self {
65        let ValidationApiConfig { disallow, validation_window } = config;
66
67        let inner = Arc::new(ValidationApiInner {
68            provider,
69            consensus,
70            payload_validator,
71            evm_config,
72            disallow,
73            validation_window,
74            cached_state: Default::default(),
75            task_spawner,
76            metrics: Default::default(),
77        });
78
79        inner.metrics.disallow_size.set(inner.disallow.len() as f64);
80        Self { inner }
81    }
82
83    /// Returns the cached reads for the given head hash.
84    async fn cached_reads(&self, head: B256) -> CachedReads {
85        let cache = self.inner.cached_state.read().await;
86        if cache.0 == head {
87            cache.1.clone()
88        } else {
89            Default::default()
90        }
91    }
92
93    /// Updates the cached state for the given head hash.
94    async fn update_cached_reads(&self, head: B256, cached_state: CachedReads) {
95        let mut cache = self.inner.cached_state.write().await;
96        if cache.0 == head {
97            cache.1.extend(cached_state);
98        } else {
99            *cache = (head, cached_state)
100        }
101    }
102}
103
104impl<Provider, E> ValidationApi<Provider, E>
105where
106    Provider: BlockReaderIdExt<Header = <E::Primitives as NodePrimitives>::BlockHeader>
107        + ChainSpecProvider<ChainSpec: EthereumHardforks>
108        + StateProviderFactory
109        + 'static,
110    E: ConfigureEvm + 'static,
111{
112    /// Validates the given block and a [`BidTrace`] against it.
113    pub async fn validate_message_against_block(
114        &self,
115        block: RecoveredBlock<<E::Primitives as NodePrimitives>::Block>,
116        message: BidTrace,
117        registered_gas_limit: u64,
118    ) -> Result<(), ValidationApiError> {
119        self.validate_message_against_header(block.sealed_header(), &message)?;
120
121        self.consensus.validate_header(block.sealed_header())?;
122        self.consensus.validate_block_pre_execution(block.sealed_block())?;
123
124        if !self.disallow.is_empty() {
125            if self.disallow.contains(&block.beneficiary()) {
126                return Err(ValidationApiError::Blacklist(block.beneficiary()))
127            }
128            if self.disallow.contains(&message.proposer_fee_recipient) {
129                return Err(ValidationApiError::Blacklist(message.proposer_fee_recipient))
130            }
131            for (sender, tx) in block.senders_iter().zip(block.body().transactions()) {
132                if self.disallow.contains(sender) {
133                    return Err(ValidationApiError::Blacklist(*sender))
134                }
135                if let Some(to) = tx.to() {
136                    if self.disallow.contains(&to) {
137                        return Err(ValidationApiError::Blacklist(to))
138                    }
139                }
140            }
141        }
142
143        let latest_header =
144            self.provider.latest_header()?.ok_or_else(|| ValidationApiError::MissingLatestBlock)?;
145
146        let parent_header = if block.parent_hash() == latest_header.hash() {
147            latest_header
148        } else {
149            // parent is not the latest header so we need to fetch it and ensure it's not too old
150            let parent_header = self
151                .provider
152                .sealed_header_by_hash(block.parent_hash())?
153                .ok_or_else(|| ValidationApiError::MissingParentBlock)?;
154
155            if latest_header.number().saturating_sub(parent_header.number()) >
156                self.validation_window
157            {
158                return Err(ValidationApiError::BlockTooOld)
159            }
160            parent_header
161        };
162
163        self.consensus.validate_header_against_parent(block.sealed_header(), &parent_header)?;
164        self.validate_gas_limit(registered_gas_limit, &parent_header, block.sealed_header())?;
165        let parent_header_hash = parent_header.hash();
166        let state_provider = self.provider.state_by_block_hash(parent_header_hash)?;
167
168        let mut request_cache = self.cached_reads(parent_header_hash).await;
169
170        let cached_db = request_cache.as_db_mut(StateProviderDatabase::new(&state_provider));
171        let executor = self.evm_config.batch_executor(cached_db);
172
173        let mut accessed_blacklisted = None;
174        let output = executor.execute_with_state_closure(&block, |state| {
175            if !self.disallow.is_empty() {
176                // Check whether the submission interacted with any blacklisted account by scanning
177                // the `State`'s cache that records everything read form database during execution.
178                for account in state.cache.accounts.keys() {
179                    if self.disallow.contains(account) {
180                        accessed_blacklisted = Some(*account);
181                    }
182                }
183            }
184        })?;
185
186        if let Some(account) = accessed_blacklisted {
187            return Err(ValidationApiError::Blacklist(account))
188        }
189
190        // update the cached reads
191        self.update_cached_reads(parent_header_hash, request_cache).await;
192
193        self.consensus.validate_block_post_execution(&block, &output)?;
194
195        self.ensure_payment(&block, &output, &message)?;
196
197        let state_root =
198            state_provider.state_root(state_provider.hashed_post_state(&output.state))?;
199
200        if state_root != block.header().state_root() {
201            return Err(ConsensusError::BodyStateRootDiff(
202                GotExpected { got: state_root, expected: block.header().state_root() }.into(),
203            )
204            .into())
205        }
206
207        Ok(())
208    }
209
210    /// Ensures that fields of [`BidTrace`] match the fields of the [`SealedHeaderFor`].
211    fn validate_message_against_header(
212        &self,
213        header: &SealedHeaderFor<E::Primitives>,
214        message: &BidTrace,
215    ) -> Result<(), ValidationApiError> {
216        if header.hash() != message.block_hash {
217            Err(ValidationApiError::BlockHashMismatch(GotExpected {
218                got: message.block_hash,
219                expected: header.hash(),
220            }))
221        } else if header.parent_hash() != message.parent_hash {
222            Err(ValidationApiError::ParentHashMismatch(GotExpected {
223                got: message.parent_hash,
224                expected: header.parent_hash(),
225            }))
226        } else if header.gas_limit() != message.gas_limit {
227            Err(ValidationApiError::GasLimitMismatch(GotExpected {
228                got: message.gas_limit,
229                expected: header.gas_limit(),
230            }))
231        } else if header.gas_used() != message.gas_used {
232            return Err(ValidationApiError::GasUsedMismatch(GotExpected {
233                got: message.gas_used,
234                expected: header.gas_used(),
235            }))
236        } else {
237            Ok(())
238        }
239    }
240
241    /// Ensures that the chosen gas limit is the closest possible value for the validator's
242    /// registered gas limit.
243    ///
244    /// Ref: <https://github.com/flashbots/builder/blob/a742641e24df68bc2fc476199b012b0abce40ffe/core/blockchain.go#L2474-L2477>
245    fn validate_gas_limit(
246        &self,
247        registered_gas_limit: u64,
248        parent_header: &SealedHeaderFor<E::Primitives>,
249        header: &SealedHeaderFor<E::Primitives>,
250    ) -> Result<(), ValidationApiError> {
251        let max_gas_limit =
252            parent_header.gas_limit() + parent_header.gas_limit() / GAS_LIMIT_BOUND_DIVISOR - 1;
253        let min_gas_limit =
254            parent_header.gas_limit() - parent_header.gas_limit() / GAS_LIMIT_BOUND_DIVISOR + 1;
255
256        let best_gas_limit =
257            std::cmp::max(min_gas_limit, std::cmp::min(max_gas_limit, registered_gas_limit));
258
259        if best_gas_limit != header.gas_limit() {
260            return Err(ValidationApiError::GasLimitMismatch(GotExpected {
261                got: header.gas_limit(),
262                expected: best_gas_limit,
263            }))
264        }
265
266        Ok(())
267    }
268
269    /// Ensures that the proposer has received [`BidTrace::value`] for this block.
270    ///
271    /// Firstly attempts to verify the payment by checking the state changes, otherwise falls back
272    /// to checking the latest block transaction.
273    fn ensure_payment(
274        &self,
275        block: &SealedBlock<<E::Primitives as NodePrimitives>::Block>,
276        output: &BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
277        message: &BidTrace,
278    ) -> Result<(), ValidationApiError> {
279        let (mut balance_before, balance_after) = if let Some(acc) =
280            output.state.state.get(&message.proposer_fee_recipient)
281        {
282            let balance_before = acc.original_info.as_ref().map(|i| i.balance).unwrap_or_default();
283            let balance_after = acc.info.as_ref().map(|i| i.balance).unwrap_or_default();
284
285            (balance_before, balance_after)
286        } else {
287            // account might have balance but considering it zero is fine as long as we know
288            // that balance have not changed
289            (U256::ZERO, U256::ZERO)
290        };
291
292        if let Some(withdrawals) = block.body().withdrawals() {
293            for withdrawal in withdrawals {
294                if withdrawal.address == message.proposer_fee_recipient {
295                    balance_before += withdrawal.amount_wei();
296                }
297            }
298        }
299
300        if balance_after >= balance_before + message.value {
301            return Ok(())
302        }
303
304        let (receipt, tx) = output
305            .receipts
306            .last()
307            .zip(block.body().transactions().last())
308            .ok_or(ValidationApiError::ProposerPayment)?;
309
310        if !receipt.status() {
311            return Err(ValidationApiError::ProposerPayment)
312        }
313
314        if tx.to() != Some(message.proposer_fee_recipient) {
315            return Err(ValidationApiError::ProposerPayment)
316        }
317
318        if tx.value() != message.value {
319            return Err(ValidationApiError::ProposerPayment)
320        }
321
322        if !tx.input().is_empty() {
323            return Err(ValidationApiError::ProposerPayment)
324        }
325
326        if let Some(block_base_fee) = block.header().base_fee_per_gas() {
327            if tx.effective_tip_per_gas(block_base_fee).unwrap_or_default() != 0 {
328                return Err(ValidationApiError::ProposerPayment)
329            }
330        }
331
332        Ok(())
333    }
334
335    /// Validates the given [`BlobsBundleV1`] and returns versioned hashes for blobs.
336    pub fn validate_blobs_bundle(
337        &self,
338        mut blobs_bundle: BlobsBundleV1,
339    ) -> Result<Vec<B256>, ValidationApiError> {
340        if blobs_bundle.commitments.len() != blobs_bundle.proofs.len() ||
341            blobs_bundle.commitments.len() != blobs_bundle.blobs.len()
342        {
343            return Err(ValidationApiError::InvalidBlobsBundle)
344        }
345
346        let versioned_hashes = blobs_bundle
347            .commitments
348            .iter()
349            .map(|c| kzg_to_versioned_hash(c.as_slice()))
350            .collect::<Vec<_>>();
351
352        let sidecar = blobs_bundle.pop_sidecar(blobs_bundle.blobs.len());
353
354        sidecar.validate(&versioned_hashes, EnvKzgSettings::default().get())?;
355
356        Ok(versioned_hashes)
357    }
358
359    /// Core logic for validating the builder submission v3
360    async fn validate_builder_submission_v3(
361        &self,
362        request: BuilderBlockValidationRequestV3,
363    ) -> Result<(), ValidationApiError> {
364        let block = self.payload_validator.ensure_well_formed_payload(ExecutionData {
365            payload: ExecutionPayload::V3(request.request.execution_payload),
366            sidecar: ExecutionPayloadSidecar::v3(CancunPayloadFields {
367                parent_beacon_block_root: request.parent_beacon_block_root,
368                versioned_hashes: self.validate_blobs_bundle(request.request.blobs_bundle)?,
369            }),
370        })?;
371
372        self.validate_message_against_block(
373            block,
374            request.request.message,
375            request.registered_gas_limit,
376        )
377        .await
378    }
379
380    /// Core logic for validating the builder submission v4
381    async fn validate_builder_submission_v4(
382        &self,
383        request: BuilderBlockValidationRequestV4,
384    ) -> Result<(), ValidationApiError> {
385        let block = self.payload_validator.ensure_well_formed_payload(ExecutionData {
386            payload: ExecutionPayload::V3(request.request.execution_payload),
387            sidecar: ExecutionPayloadSidecar::v4(
388                CancunPayloadFields {
389                    parent_beacon_block_root: request.parent_beacon_block_root,
390                    versioned_hashes: self.validate_blobs_bundle(request.request.blobs_bundle)?,
391                },
392                PraguePayloadFields {
393                    requests: RequestsOrHash::Requests(
394                        request.request.execution_requests.to_requests(),
395                    ),
396                },
397            ),
398        })?;
399
400        self.validate_message_against_block(
401            block,
402            request.request.message,
403            request.registered_gas_limit,
404        )
405        .await
406    }
407}
408
409#[async_trait]
410impl<Provider, E> BlockSubmissionValidationApiServer for ValidationApi<Provider, E>
411where
412    Provider: BlockReaderIdExt<Header = <E::Primitives as NodePrimitives>::BlockHeader>
413        + ChainSpecProvider<ChainSpec: EthereumHardforks>
414        + StateProviderFactory
415        + Clone
416        + 'static,
417    E: ConfigureEvm + 'static,
418{
419    async fn validate_builder_submission_v1(
420        &self,
421        _request: BuilderBlockValidationRequest,
422    ) -> RpcResult<()> {
423        warn!(target: "rpc::flashbots", "Method `flashbots_validateBuilderSubmissionV1` is not supported");
424        Err(internal_rpc_err("unimplemented"))
425    }
426
427    async fn validate_builder_submission_v2(
428        &self,
429        _request: BuilderBlockValidationRequestV2,
430    ) -> RpcResult<()> {
431        warn!(target: "rpc::flashbots", "Method `flashbots_validateBuilderSubmissionV2` is not supported");
432        Err(internal_rpc_err("unimplemented"))
433    }
434
435    /// Validates a block submitted to the relay
436    async fn validate_builder_submission_v3(
437        &self,
438        request: BuilderBlockValidationRequestV3,
439    ) -> RpcResult<()> {
440        let this = self.clone();
441        let (tx, rx) = oneshot::channel();
442
443        self.task_spawner.spawn_blocking(Box::pin(async move {
444            let result = Self::validate_builder_submission_v3(&this, request)
445                .await
446                .map_err(ErrorObject::from);
447            let _ = tx.send(result);
448        }));
449
450        rx.await.map_err(|_| internal_rpc_err("Internal blocking task error"))?
451    }
452
453    /// Validates a block submitted to the relay
454    async fn validate_builder_submission_v4(
455        &self,
456        request: BuilderBlockValidationRequestV4,
457    ) -> RpcResult<()> {
458        let this = self.clone();
459        let (tx, rx) = oneshot::channel();
460
461        self.task_spawner.spawn_blocking(Box::pin(async move {
462            let result = Self::validate_builder_submission_v4(&this, request)
463                .await
464                .map_err(ErrorObject::from);
465            let _ = tx.send(result);
466        }));
467
468        rx.await.map_err(|_| internal_rpc_err("Internal blocking task error"))?
469    }
470}
471
472pub struct ValidationApiInner<Provider, E: ConfigureEvm> {
473    /// The provider that can interact with the chain.
474    provider: Provider,
475    /// Consensus implementation.
476    consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
477    /// Execution payload validator.
478    payload_validator: Arc<
479        dyn PayloadValidator<
480            Block = <E::Primitives as NodePrimitives>::Block,
481            ExecutionData = ExecutionData,
482        >,
483    >,
484    /// Block executor factory.
485    evm_config: E,
486    /// Set of disallowed addresses
487    disallow: HashSet<Address>,
488    /// The maximum block distance - parent to latest - allowed for validation
489    validation_window: u64,
490    /// Cached state reads to avoid redundant disk I/O across multiple validation attempts
491    /// targeting the same state. Stores a tuple of (`block_hash`, `cached_reads`) for the
492    /// latest head block state. Uses async `RwLock` to safely handle concurrent validation
493    /// requests.
494    cached_state: RwLock<(B256, CachedReads)>,
495    /// Task spawner for blocking operations
496    task_spawner: Box<dyn TaskSpawner>,
497    /// Validation metrics
498    metrics: ValidationMetrics,
499}
500
501impl<Provider, E: ConfigureEvm> fmt::Debug for ValidationApiInner<Provider, E> {
502    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
503        f.debug_struct("ValidationApiInner").finish_non_exhaustive()
504    }
505}
506
507/// Configuration for validation API.
508#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
509pub struct ValidationApiConfig {
510    /// Disallowed addresses.
511    pub disallow: HashSet<Address>,
512    /// The maximum block distance - parent to latest - allowed for validation
513    pub validation_window: u64,
514}
515
516impl ValidationApiConfig {
517    /// Default validation blocks window of 3 blocks
518    pub const DEFAULT_VALIDATION_WINDOW: u64 = 3;
519}
520
521impl Default for ValidationApiConfig {
522    fn default() -> Self {
523        Self { disallow: Default::default(), validation_window: Self::DEFAULT_VALIDATION_WINDOW }
524    }
525}
526
527/// Errors thrown by the validation API.
528#[derive(Debug, thiserror::Error)]
529pub enum ValidationApiError {
530    #[error("block gas limit mismatch: {_0}")]
531    GasLimitMismatch(GotExpected<u64>),
532    #[error("block gas used mismatch: {_0}")]
533    GasUsedMismatch(GotExpected<u64>),
534    #[error("block parent hash mismatch: {_0}")]
535    ParentHashMismatch(GotExpected<B256>),
536    #[error("block hash mismatch: {_0}")]
537    BlockHashMismatch(GotExpected<B256>),
538    #[error("missing latest block in database")]
539    MissingLatestBlock,
540    #[error("parent block not found")]
541    MissingParentBlock,
542    #[error("block is too old, outside validation window")]
543    BlockTooOld,
544    #[error("could not verify proposer payment")]
545    ProposerPayment,
546    #[error("invalid blobs bundle")]
547    InvalidBlobsBundle,
548    #[error("block accesses blacklisted address: {_0}")]
549    Blacklist(Address),
550    #[error(transparent)]
551    Blob(#[from] BlobTransactionValidationError),
552    #[error(transparent)]
553    Consensus(#[from] ConsensusError),
554    #[error(transparent)]
555    Provider(#[from] ProviderError),
556    #[error(transparent)]
557    Execution(#[from] BlockExecutionError),
558    #[error(transparent)]
559    Payload(#[from] NewPayloadError),
560}
561
562impl From<ValidationApiError> for ErrorObject<'static> {
563    fn from(error: ValidationApiError) -> Self {
564        match error {
565            ValidationApiError::GasLimitMismatch(_) |
566            ValidationApiError::GasUsedMismatch(_) |
567            ValidationApiError::ParentHashMismatch(_) |
568            ValidationApiError::BlockHashMismatch(_) |
569            ValidationApiError::Blacklist(_) |
570            ValidationApiError::ProposerPayment |
571            ValidationApiError::InvalidBlobsBundle |
572            ValidationApiError::Blob(_) => invalid_params_rpc_err(error.to_string()),
573
574            ValidationApiError::MissingLatestBlock |
575            ValidationApiError::MissingParentBlock |
576            ValidationApiError::BlockTooOld |
577            ValidationApiError::Consensus(_) |
578            ValidationApiError::Provider(_) => internal_rpc_err(error.to_string()),
579            ValidationApiError::Execution(err) => match err {
580                error @ BlockExecutionError::Validation(_) => {
581                    invalid_params_rpc_err(error.to_string())
582                }
583                error @ BlockExecutionError::Internal(_) => internal_rpc_err(error.to_string()),
584            },
585            ValidationApiError::Payload(err) => match err {
586                error @ NewPayloadError::Eth(_) => invalid_params_rpc_err(error.to_string()),
587                error @ NewPayloadError::Other(_) => internal_rpc_err(error.to_string()),
588            },
589        }
590    }
591}
592
593/// Metrics for the validation endpoint.
594#[derive(Metrics)]
595#[metrics(scope = "builder.validation")]
596pub(crate) struct ValidationMetrics {
597    /// The number of entries configured in the builder validation disallow list.
598    pub(crate) disallow_size: Gauge,
599}