diff --git a/crates/derive/src/stages/batch_queue.rs b/crates/derive/src/stages/batch_queue.rs index 8b1378917..b1ba033c5 100644 --- a/crates/derive/src/stages/batch_queue.rs +++ b/crates/derive/src/stages/batch_queue.rs @@ -1 +1,367 @@ +//! This module contains the `BatchQueue` stage implementation. +use crate::{ + stages::channel_reader::ChannelReader, + traits::{ChainProvider, DataAvailabilityProvider, ResettableStage, SafeBlockFetcher}, + types::{ + Batch, BatchValidity, BatchWithInclusionBlock, BlockInfo, L2BlockInfo, RollupConfig, + SingleBatch, StageError, StageResult, SystemConfig, + }, +}; +use alloc::{boxed::Box, vec::Vec}; +use anyhow::anyhow; +use async_trait::async_trait; +use core::fmt::Debug; + +/// [BatchQueue] is responsible for o rdering unordered batches +/// and gnerating empty batches when the sequence window has passed. +/// +/// It receives batches that are tagged with the L1 Inclusion block of the batch. +/// It only considers batches that are inside the sequencing window of a specific L1 Origin. +/// It tries to eagerly pull batches based on the current L2 safe head. +/// Otherwise it filters/creates an entire epoch's worth of batches at once. +/// +/// This stage tracks a range of L1 blocks with the assumption that all batches with an L1 inclusion +/// block inside that range have been added to the stage by the time that it attempts to advance a +/// full epoch. +/// +/// It is internally responsible for making sure that batches with L1 inclusions block outside it's +/// working range are not considered or pruned. +#[derive(Debug)] +pub struct BatchQueue +where + DAP: DataAvailabilityProvider + Debug, + CP: ChainProvider + Debug, + BF: SafeBlockFetcher + Debug, +{ + /// The rollup config. + cfg: RollupConfig, + /// The previous stage of the derivation pipeline. + prev: ChannelReader, + /// The l1 block ref + origin: Option, + + /// A consecutive, time-centric window of L1 Blocks. + /// Every L1 origin of unsafe L2 Blocks must be included in this list. + /// If every L2 Block corresponding to a single L1 Block becomes safe, + /// the block is popped from this list. + /// If new L2 Block's L1 origin is not included in this list, fetch and + /// push it to the list. + l1_blocks: Vec, + + /// A set of batches in order from when we've seen them. + batches: Vec, + + /// A set of cached [SingleBatche]s derived from [SpanBatch]s. + next_spans: Vec, + + /// Used to validate the batches. + fetcher: BF, +} + +impl BatchQueue +where + DAP: DataAvailabilityProvider + Debug, + CP: ChainProvider + Debug, + BF: SafeBlockFetcher + Debug, +{ + /// Creates a new [BatchQueue] stage. + pub fn new(cfg: RollupConfig, prev: ChannelReader, fetcher: BF) -> Self { + Self { + cfg, + prev, + origin: None, + l1_blocks: Vec::new(), + batches: Vec::new(), + next_spans: Vec::new(), + fetcher, + } + } + + /// Returns the L1 origin [BlockInfo]. + pub fn origin(&self) -> Option<&BlockInfo> { + self.prev.origin() + } + + /// Pops the next batch from the current queued up span-batch cache. + /// The parent is used to set the parent hash of the batch. + /// The parent is verified when the batch is later validated. + pub fn pop_next_batch(&mut self, parent: L2BlockInfo) -> Option { + if self.next_spans.is_empty() { + panic!("Invalid state: must have next spans to pop"); + } + let mut next = self.next_spans.remove(0); + next.parent_hash = parent.block_info.hash; + Some(next) + } + + /// Returns the next valid batch upon the given safe head. + /// Also returns the boolean that indicates if the batch is the last block in the batch. + pub async fn next_batch(&mut self, parent: L2BlockInfo) -> StageResult { + if !self.next_spans.is_empty() { + // There are cached singular batches derived from the span batch. + // Check if the next cached batch matches the given parent block. + if self.next_spans[0].timestamp == parent.block_info.timestamp + self.cfg.block_time { + return self + .pop_next_batch(parent) + .ok_or(anyhow!("failed to pop next batch from span batch").into()); + } + // Parent block does not match the next batch. + // Means the previously returned batch is invalid. + // Drop cached batches and find another batch. + self.next_spans.clear(); + // TODO: log that the provided parent block does not match the next batch. + // TODO: metrice the internal batch drop. + } + + // If the epoch is advanced, update the l1 blocks. + // Advancing epoch must be done after the pipeline successfully applies the entire span + // batch to the chain. + // Because the span batch can be reverted during processing the batch, then we must + // preserve existing l1 blocks to verify the epochs of the next candidate batch. + if !self.l1_blocks.is_empty() && parent.l1_origin.number > self.l1_blocks[0].number { + for (i, block) in self.l1_blocks.iter().enumerate() { + if parent.l1_origin.number == block.number { + self.l1_blocks.drain(0..i); + // TODO: log that the pipelien has advanced the epoch. + // TODO: metrice the internal epoch advancement. + break; + } + } + // If the origin of the parent block is not included, we must advance the origin. + } + + // NOTE: The origin is used to determine if it's behind. + // It is the future origin that gets saved into the l1 blocks array. + // We always update the origin of this stage if it's not the same so + // after the update code runs, this is consistent. + let origin_behind = + self.origin.map_or(true, |origin| origin.number < parent.l1_origin.number); + + // Advance the origin if needed. + // The entire pipeline has the same origin. + // Batches prior to the l1 origin of the l2 safe head are not accepted. + if self.origin != self.prev.origin().copied() { + self.origin = self.prev.origin().cloned(); + if !origin_behind { + let origin = self.origin.as_ref().ok_or_else(|| anyhow!("missing origin"))?; + self.l1_blocks.push(*origin); + } else { + // This is to handle the special case of startup. + // At startup, the batch queue is reset and includes the + // l1 origin. That is the only time where immediately after + // reset is called, the origin behind is false. + self.l1_blocks.clear(); + } + // TODO: log batch queue origin advancement. + } + + // Load more data into the batch queue. + let mut out_of_data = false; + match self.prev.next_batch().await { + Ok(b) => { + if !origin_behind { + self.add_batch(b, parent).ok(); + } else { + // TODO: metrice when the batch is dropped because the origin is behind. + } + } + Err(StageError::Eof) => out_of_data = true, + Err(e) => return Err(e), + } + + // Skip adding the data unless up to date with the origin, + // but still fully empty the previous stages. + if origin_behind { + if out_of_data { + return Err(StageError::Eof); + } + return Err(StageError::NotEnoughData); + } + + // Attempt to derive more batches. + let batch = match self.derive_next_batch(out_of_data, parent) { + Ok(b) => b, + Err(e) => match e { + StageError::Eof => { + if out_of_data { + return Err(StageError::Eof); + } + return Err(StageError::NotEnoughData); + } + _ => return Err(e), + }, + }; + + // If the next batch is derived from the span batch, it's the last batch of the span. + // For singular batches, the span batch cache should be empty. + match batch { + Batch::Single(sb) => Ok(sb), + Batch::Span(sb) => { + let batches = sb.get_singular_batches(&self.l1_blocks, parent); + self.next_spans = batches; + let nb = self + .pop_next_batch(parent) + .ok_or_else(|| anyhow!("failed to pop next batch from span batch"))?; + Ok(nb) + } + } + } + + /// Derives the next batch to apply on top of the current L2 safe head. + /// Follows the validity rules imposed on consecutive batches. + /// Based on currently available buffered batch and L1 origin information. + /// A [StageError::Eof] is returned if no batch can be derived yet. + pub fn derive_next_batch(&mut self, empty: bool, parent: L2BlockInfo) -> StageResult { + // Cannot derive a batch if no origin was prepared. + if self.l1_blocks.is_empty() { + return Err(StageError::Custom(anyhow!( + "failed to derive batch: no origin was prepared" + ))); + } + + // Get the epoch + let epoch = self.l1_blocks[0]; + // TODO: log that the next batch is being derived. + // TODO: metrice the time it takes to derive the next batch. + + // Note: epoch origin can now be one block ahead of the L2 Safe Head + // This is in the case where we auto generate all batches in an epoch & advance the epoch + // but don't advance the L2 Safe Head's epoch + if parent.l1_origin != epoch.id() && parent.l1_origin.number != epoch.number - 1 { + return Err(StageError::Custom(anyhow!( + "buffered L1 chain epoch {} in batch queue does not match safe head origin {:?}", + epoch, + parent.l1_origin + ))); + } + + // Find the first-seen batch that matches all validity conditions. + // We may not have sufficient information to proceed filtering, and then we stop. + // There may be none: in that case we force-create an empty batch + let mut next_batch = None; + let next_timestamp = parent.block_info.timestamp + self.cfg.block_time; + + // Go over all batches, in order of inclusion, and find the first batch we can accept. + // Filter in-place by only remembering the batches that may be processed in the future, or + // any undecided ones. + let mut remaining = Vec::new(); + for i in 0..self.batches.len() { + let batch = &self.batches[i]; + let validity = batch.check_batch(&self.cfg, &self.l1_blocks, parent, &self.fetcher); + match validity { + BatchValidity::Future => { + remaining.push(batch.clone()); + } + BatchValidity::Drop => { + // TODO: Log the drop reason with WARN level. + // batch.log_context(self.log).warn("Dropping batch", "parent", parent.id(), + // "parent_time", parent.info.time); + continue; + } + BatchValidity::Accept => { + next_batch = Some(batch.clone()); + // Don't keep the current batch in the remaining items since we are processing + // it now, but retain every batch we didn't get to yet. + remaining.extend_from_slice(&self.batches[i + 1..]); + break; + } + BatchValidity::Undecided => { + remaining.extend_from_slice(&self.batches[i..]); + self.batches = remaining; + return Err(StageError::Eof); + } + } + } + self.batches = remaining; + + if let Some(nb) = next_batch { + // TODO: log that the next batch is found. + return Ok(nb.batch); + } + + // If the current epoch is too old compared to the L1 block we are at, + // i.e. if the sequence window expired, we create empty batches for the current epoch + let expiry_epoch = epoch.number + self.cfg.seq_window_size; + let force_empty_batches = (expiry_epoch == parent.l1_origin.number && empty) || + expiry_epoch < parent.l1_origin.number; + let first_of_epoch = epoch.number == parent.l1_origin.number + 1; + + // TODO: Log the empty batch generation. + + // If the sequencer window did not expire, + // there is still room to receive batches for the current epoch. + // No need to force-create empty batch(es) towards the next epoch yet. + if !force_empty_batches { + return Err(StageError::Eof); + } + + // The next L1 block is needed to proceed towards the next epoch. + if self.l1_blocks.len() < 2 { + return Err(StageError::Eof); + } + + let next_epoch = self.l1_blocks[1]; + + // Fill with empty L2 blocks of the same epoch until we meet the time of the next L1 origin, + // to preserve that L2 time >= L1 time. If this is the first block of the epoch, always + // generate a batch to ensure that we at least have one batch per epoch. + if next_timestamp < next_epoch.timestamp || first_of_epoch { + // TODO: log next batch generation. + return Ok(Batch::Single(SingleBatch { + parent_hash: parent.block_info.hash, + epoch_num: epoch.number, + epoch_hash: epoch.hash, + timestamp: next_timestamp, + transactions: Vec::new(), + })); + } + + // At this point we have auto generated every batch for the current epoch + // that we can, so we can advance to the next epoch. + // TODO: log that the epoch is advanced. + // bq.log.Trace("Advancing internal L1 blocks", "next_timestamp", nextTimestamp, + // "next_epoch_time", nextEpoch.Time) + self.l1_blocks.remove(0); + Err(StageError::Eof) + } + + /// Adds a batch to the queue. + pub fn add_batch(&mut self, batch: Batch, parent: L2BlockInfo) -> StageResult<()> { + if self.l1_blocks.is_empty() { + // TODO: log that the batch cannot be added without an origin + panic!("Cannot add batch without an origin"); + } + let origin = self.origin.ok_or_else(|| anyhow!("cannot add batch with missing origin"))?; + let data = BatchWithInclusionBlock { inclusion_block: origin, batch }; + // If we drop the batch, validation logs the drop reason with WARN level. + if data.check_batch(&self.cfg, &self.l1_blocks, parent, &self.fetcher).is_drop() { + return Ok(()); + } + self.batches.push(data); + Ok(()) + } +} + +#[async_trait] +impl ResettableStage for BatchQueue +where + DAP: DataAvailabilityProvider + Send + Debug, + CP: ChainProvider + Send + Debug, + BF: SafeBlockFetcher + Send + Debug, +{ + async fn reset(&mut self, base: BlockInfo, _: SystemConfig) -> StageResult<()> { + // Copy over the Origin from the next stage. + // It is set in the engine queue (two stages away) + // such that the L2 Safe Head origin is the progress. + self.origin = Some(base); + self.batches.clear(); + // Include the new origin as an origin to build on. + // This is only for the initialization case. + // During normal resets we will later throw out this block. + self.l1_blocks.clear(); + self.l1_blocks.push(base); + self.next_spans.clear(); + Err(StageError::Eof) + } +} diff --git a/crates/derive/src/stages/channel_reader.rs b/crates/derive/src/stages/channel_reader.rs index 8ebea8ff1..e89d28122 100644 --- a/crates/derive/src/stages/channel_reader.rs +++ b/crates/derive/src/stages/channel_reader.rs @@ -5,6 +5,7 @@ use crate::{ traits::{ChainProvider, DataAvailabilityProvider}, types::{Batch, BlockInfo, StageError, StageResult}, }; + use alloc::vec::Vec; use anyhow::anyhow; use core::fmt::Debug; diff --git a/crates/derive/src/stages/mod.rs b/crates/derive/src/stages/mod.rs index 1557d9cbe..1717c306c 100644 --- a/crates/derive/src/stages/mod.rs +++ b/crates/derive/src/stages/mod.rs @@ -28,5 +28,7 @@ mod channel_reader; pub use channel_reader::ChannelReader; mod batch_queue; +pub use batch_queue::BatchQueue; + mod engine_queue; mod payload_derivation; diff --git a/crates/derive/src/traits/data_sources.rs b/crates/derive/src/traits/data_sources.rs index 5721e00be..89ea08d5e 100644 --- a/crates/derive/src/traits/data_sources.rs +++ b/crates/derive/src/traits/data_sources.rs @@ -1,7 +1,7 @@ //! Contains traits that describe the functionality of various data sources used in the derivation //! pipeline's stages. -use crate::types::{BlockInfo, Receipt, StageResult}; +use crate::types::{BlockInfo, ExecutionPayloadEnvelope, L2BlockInfo, Receipt, StageResult}; use alloc::{boxed::Box, vec::Vec}; use alloy_primitives::{Address, Bytes, B256}; use anyhow::Result; @@ -20,6 +20,18 @@ pub trait ChainProvider { async fn receipts_by_hash(&self, hash: B256) -> Result>; } +/// Describes the functionality of a data source that fetches safe blocks. +#[async_trait] +pub trait SafeBlockFetcher { + /// Returns the L2 block info given a block number. + /// Errors if the block does not exist. + async fn l2_block_info_by_number(&self, number: u64) -> Result; + + /// Returns an execution payload for a given number. + /// Errors if the execution payload does not exist. + async fn payload_by_number(&self, number: u64) -> Result; +} + /// Describes the functionality of a data source that can provide data availability information. #[async_trait] pub trait DataAvailabilityProvider { diff --git a/crates/derive/src/traits/mod.rs b/crates/derive/src/traits/mod.rs index a9993a70b..5e9281617 100644 --- a/crates/derive/src/traits/mod.rs +++ b/crates/derive/src/traits/mod.rs @@ -2,7 +2,7 @@ //! pipeline. mod data_sources; -pub use data_sources::{ChainProvider, DataAvailabilityProvider, DataIter}; +pub use data_sources::{ChainProvider, DataAvailabilityProvider, DataIter, SafeBlockFetcher}; mod stages; pub use stages::ResettableStage; diff --git a/crates/derive/src/types/batch/mod.rs b/crates/derive/src/types/batch/mod.rs index 9102d24ba..3f0c6298a 100644 --- a/crates/derive/src/types/batch/mod.rs +++ b/crates/derive/src/types/batch/mod.rs @@ -2,12 +2,19 @@ //! [SingleBatch]. use super::DecodeError; +use crate::{ + traits::SafeBlockFetcher, + types::{BlockInfo, L2BlockInfo, RollupConfig}, +}; use alloc::vec::Vec; use alloy_rlp::{Buf, Decodable, Encodable}; mod batch_type; pub use batch_type::BatchType; +mod validity; +pub use validity::BatchValidity; + mod span_batch; pub use span_batch::{ RawSpanBatch, SpanBatch, SpanBatchBits, SpanBatchBuilder, SpanBatchEip1559TransactionData, @@ -19,6 +26,39 @@ pub use span_batch::{ mod single_batch; pub use single_batch::SingleBatch; +/// A batch with its inclusion block. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct BatchWithInclusionBlock { + /// The inclusion block + pub inclusion_block: BlockInfo, + /// The batch + pub batch: Batch, +} + +impl BatchWithInclusionBlock { + /// Validates the batch can be applied on top of the specified L2 safe head. + /// The first entry of the l1_blocks should match the origin of the l2_safe_head. + /// One or more consecutive l1_blocks should be provided. + /// In case of only a single L1 block, the decision whether a batch is valid may have to stay + /// undecided. + pub fn check_batch( + &self, + cfg: &RollupConfig, + l1_blocks: &[BlockInfo], + l2_safe_head: L2BlockInfo, + fetcher: &BF, + ) -> BatchValidity { + match &self.batch { + Batch::Single(single_batch) => { + single_batch.check_batch(cfg, l1_blocks, l2_safe_head, &self.inclusion_block) + } + Batch::Span(span_batch) => { + span_batch.check_batch(cfg, l1_blocks, l2_safe_head, &self.inclusion_block, fetcher) + } + } + } +} + /// A Batch. #[derive(Debug, Clone, PartialEq, Eq)] #[allow(clippy::large_enum_variant)] @@ -30,6 +70,14 @@ pub enum Batch { } impl Batch { + /// Returns the timestamp for the batch. + pub fn timestamp(&self) -> u64 { + match self { + Self::Single(sb) => sb.timestamp, + Self::Span(sb) => sb.timestamp(), + } + } + /// Attempts to encode a batch into a writer. pub fn encode(&self, w: &mut Vec) -> Result<(), DecodeError> { match self { diff --git a/crates/derive/src/types/batch/single_batch.rs b/crates/derive/src/types/batch/single_batch.rs index 6b77e913c..273c0a38a 100644 --- a/crates/derive/src/types/batch/single_batch.rs +++ b/crates/derive/src/types/batch/single_batch.rs @@ -1,6 +1,7 @@ //! This module contains the [SingleBatch] type. -use crate::types::RawTransaction; +use super::validity::BatchValidity; +use crate::types::{BlockInfo, L2BlockInfo, RawTransaction, RollupConfig}; use alloc::vec::Vec; use alloy_primitives::BlockHash; use alloy_rlp::{Decodable, Encodable}; @@ -26,6 +27,143 @@ impl SingleBatch { pub fn has_invalid_transactions(&self) -> bool { self.transactions.iter().any(|tx| tx.0.is_empty() || tx.0[0] == 0x7E) } + + /// Checks if the batch is valid. + pub fn check_batch( + &self, + cfg: &RollupConfig, + l1_blocks: &[BlockInfo], + l2_safe_head: L2BlockInfo, + inclusion_block: &BlockInfo, + ) -> BatchValidity { + // Sanity check input consistency + if l1_blocks.is_empty() { + // TODO: log a warning: "missing L1 block input, cannot proceed with batch checking" + return BatchValidity::Undecided; + } + + let epoch = l1_blocks[0]; + let next_timestamp = l2_safe_head.block_info.timestamp + cfg.block_time; + if self.timestamp > next_timestamp { + // TODO: trace log: "received out-of-order batch for future processing after next batch" + return BatchValidity::Future; + } + if self.timestamp < next_timestamp { + // TODO: warn log: "dropping batch with old timestamp", "min_timestamp", next_timestamp + return BatchValidity::Drop; + } + + // Dependent on the above timestamp check. + // If the timestamp is correct, then it must build on top of the safe head. + if self.parent_hash != l2_safe_head.block_info.hash { + // TODO: warn log: "ignoring batch with mismatching parent hash", "current_safe_head", + // l2_safe_head.info.hash + return BatchValidity::Drop; + } + + // Filter out batches that were included too late. + if self.epoch_num + cfg.seq_window_size < inclusion_block.number { + // TODO: warn log: "batch was included too late, sequence window expired" + return BatchValidity::Drop; + } + + // Check the L1 origin of the batch + let mut batch_origin = epoch; + if self.epoch_num < epoch.number { + // TODO: warn log: "dropped batch, epoch is too old", "minimum", epoch.id() + return BatchValidity::Drop; + } else if self.epoch_num == epoch.number { + // Batch is sticking to the current epoch, continue. + } else if self.epoch_num == epoch.number + 1 { + // With only 1 l1Block we cannot look at the next L1 Origin. + // Note: This means that we are unable to determine validity of a batch + // without more information. In this case we should bail out until we have + // more information otherwise the eager algorithm may diverge from a non-eager + // algorithm. + if l1_blocks.len() < 2 { + // TODO: info log: "eager batch wants to advance epoch, but could not without more + // L1 blocks", "current_epoch", epoch.id() + return BatchValidity::Undecided; + } + batch_origin = l1_blocks[1]; + } else { + // TODO: warn log: "batch is for future epoch too far ahead, while it has the next + // timestamp, so it must be invalid", "current_epoch", epoch.id() + return BatchValidity::Drop; + } + + // Validate the batch epoch hash + if self.epoch_hash != batch_origin.hash { + // TODO: warn log: "batch is for different L1 chain, epoch hash does not match", + // "expected", batch_origin.id() + return BatchValidity::Drop; + } + + if self.timestamp < batch_origin.timestamp { + // TODO: warn log: "batch timestamp is less than L1 origin timestamp", "l2_timestamp", + // self.timestamp, "l1_timestamp", batch_origin.timestamp, "origin", batch_origin.id() + return BatchValidity::Drop; + } + + // Check if we ran out of sequencer time drift + let max = if let Some(max) = batch_origin.timestamp.checked_add(cfg.max_sequencer_drift) { + max + } else { + // TODO: log that the batch exceeds time drift. + return BatchValidity::Drop; + }; + + let no_txs = self.transactions.is_empty(); + if self.timestamp > max && !no_txs { + // If the sequencer is ignoring the time drift rule, then drop the batch and force an + // empty batch instead, as the sequencer is not allowed to include anything + // past this point without moving to the next epoch. TODO: warn log: "batch + // exceeded sequencer time drift, sequencer must adopt new L1 origin to include + // transactions again", "max_time", max + return BatchValidity::Drop; + } + if self.timestamp > max && no_txs { + // If the sequencer is co-operating by producing an empty batch, + // allow the batch if it was the right thing to do to maintain the L2 time >= L1 time + // invariant. Only check batches that do not advance the epoch, to ensure + // epoch advancement regardless of time drift is allowed. + if epoch.number == batch_origin.number { + if l1_blocks.len() < 2 { + // TODO: info log: "without the next L1 origin we cannot determine yet if this + // empty batch that exceeds the time drift is still valid" + return BatchValidity::Undecided; + } + let next_origin = l1_blocks[1]; + // Check if the next L1 Origin could have been adopted + if self.timestamp >= next_origin.timestamp { + // TODO: log that the batch exceeded the time drift without adopting the next + // origin. + return BatchValidity::Drop; + } else { + // TODO: log that we are continuing with an empty batch before the late L1 block + // to preserve the L2 time invariant. TODO: metrice empty + // batch continuation + } + } + } + + // We can do this check earlier, but it's a more intensive one, so we do this last. + // TODO: metrice & allow configurability to measure the time it takes to check the batch. + for tx in self.transactions.iter() { + if tx.is_empty() { + // TODO: warn log: "transaction data must not be empty, but found empty tx", + // "tx_index", i + return BatchValidity::Drop; + } + if tx.is_deposit() { + // TODO: warn log: "sequencers may not embed any deposits into batch data, but found + // tx that has one", "tx_index", i + return BatchValidity::Drop; + } + } + + BatchValidity::Accept + } } impl Encodable for SingleBatch { @@ -50,7 +188,7 @@ impl Decodable for SingleBatch { } #[cfg(test)] -mod test { +mod tests { use super::SingleBatch; use crate::types::RawTransaction; use alloc::vec; diff --git a/crates/derive/src/types/batch/span_batch/batch.rs b/crates/derive/src/types/batch/span_batch/batch.rs index 23c9247a6..e580b0eaa 100644 --- a/crates/derive/src/types/batch/span_batch/batch.rs +++ b/crates/derive/src/types/batch/span_batch/batch.rs @@ -3,9 +3,12 @@ #![allow(unused)] use super::{SpanBatchError, SpanBatchTransactions}; -use crate::types::{ - block::L2BlockInfo, BlockInfo, RawSpanBatch, SingleBatch, SpanBatchBits, SpanBatchElement, - SpanBatchPayload, SpanBatchPrefix, +use crate::{ + traits::SafeBlockFetcher, + types::{ + BatchValidity, BlockInfo, L2BlockInfo, RawSpanBatch, RollupConfig, SingleBatch, + SpanBatchBits, SpanBatchElement, SpanBatchPayload, SpanBatchPrefix, + }, }; use alloc::{vec, vec::Vec}; use alloy_primitives::FixedBytes; @@ -37,6 +40,18 @@ impl SpanBatch { self.batches[0].timestamp } + /// Checks if the span batch is valid. + pub fn check_batch( + &self, + _cfg: &RollupConfig, + _l1_blocks: &[BlockInfo], + _l2_safe_head: L2BlockInfo, + _inclusion_block: &BlockInfo, + _fetcher: &BF, + ) -> BatchValidity { + unimplemented!() + } + /// Converts the span batch to a raw span batch. pub fn to_raw_span_batch( &self, diff --git a/crates/derive/src/types/batch/span_batch/raw.rs b/crates/derive/src/types/batch/span_batch/raw.rs index 0dcc3eacc..e1f8ac181 100644 --- a/crates/derive/src/types/batch/span_batch/raw.rs +++ b/crates/derive/src/types/batch/span_batch/raw.rs @@ -2,9 +2,12 @@ use alloc::vec::Vec; -use crate::types::{ - BatchType, RawTransaction, SpanBatchElement, SpanBatchPayload, SpanBatchPrefix, - SpanDecodingError, +use crate::{ + traits::SafeBlockFetcher, + types::{ + BatchType, BatchValidity, BlockInfo, L2BlockInfo, RawTransaction, RollupConfig, + SingleBatch, SpanBatchElement, SpanBatchPayload, SpanBatchPrefix, SpanDecodingError, + }, }; use super::{SpanBatch, SpanBatchError}; @@ -24,6 +27,32 @@ impl RawSpanBatch { BatchType::Span } + /// Returns the timestamp for the span batch. + pub fn timestamp(&self) -> u64 { + self.prefix.rel_timestamp + } + + /// Checks if the span batch is valid. + pub fn check_batch( + &self, + _cfg: &RollupConfig, + _l1_blocks: &[BlockInfo], + _l2_safe_head: L2BlockInfo, + _inclusion_block: &BlockInfo, + _fetcher: &BF, + ) -> BatchValidity { + unimplemented!() + } + + /// Derives [SingleBatch]s from the span batch. + pub fn get_singular_batches( + &self, + _l1_blocks: &[BlockInfo], + _parent: L2BlockInfo, + ) -> Vec { + unimplemented!() + } + /// Encodes the [RawSpanBatch] into a writer. pub fn encode(&self, w: &mut Vec) -> Result<(), SpanBatchError> { self.prefix.encode_prefix(w); diff --git a/crates/derive/src/types/batch/validity.rs b/crates/derive/src/types/batch/validity.rs new file mode 100644 index 000000000..0e074983a --- /dev/null +++ b/crates/derive/src/types/batch/validity.rs @@ -0,0 +1,25 @@ +//! Contains the [BatchValidity] and its encodings. + +#[cfg(feature = "serde")] +use serde::{Deserialize, Serialize}; + +/// Batch Validity +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum BatchValidity { + /// The batch is invalid now and in the future, unless we reorg + Drop, + /// The batch is valid and should be processed + Accept, + /// We are lacking L1 information until we can proceed batch filtering + Undecided, + /// The batch may be valid, but cannot be processed yet and should be checked again later + Future, +} + +impl BatchValidity { + /// Returns if the batch is dropped. + pub fn is_drop(&self) -> bool { + matches!(self, BatchValidity::Drop) + } +} diff --git a/crates/derive/src/types/block.rs b/crates/derive/src/types/block.rs index aa56b83e1..a473c38b8 100644 --- a/crates/derive/src/types/block.rs +++ b/crates/derive/src/types/block.rs @@ -24,6 +24,37 @@ impl BlockInfo { pub fn new(hash: B256, number: u64, parent_hash: B256, timestamp: u64) -> Self { Self { hash, number, parent_hash, timestamp } } + + /// Returns the block ID. + pub fn id(&self) -> BlockID { + BlockID { hash: self.hash, number: self.number } + } +} + +impl core::fmt::Display for BlockInfo { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + write!( + f, + "BlockInfo {{ hash: {}, number: {}, parent_hash: {}, timestamp: {} }}", + self.hash, self.number, self.parent_hash, self.timestamp + ) + } +} + +/// Block ID identifies a block by its hash and number +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone, Copy, Eq, PartialEq, Default)] +pub struct BlockID { + /// The block hash + pub hash: BlockHash, + /// The block number + pub number: BlockNumber, +} + +impl core::fmt::Display for BlockID { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + write!(f, "{{ hash: {}, number: {} }}", self.hash, self.number) + } } /// L2 Block Header Info @@ -32,37 +63,19 @@ impl BlockInfo { pub struct L2BlockInfo { /// The base [BlockInfo] pub block_info: BlockInfo, - /// The L1 origin [BlockId] - pub l1_origin: BlockId, + /// The L1 origin [BlockID] + pub l1_origin: BlockID, /// The sequence number of the L2 block pub seq_num: u64, } impl L2BlockInfo { /// Instantiates a new [L2BlockInfo]. - pub fn new(block_info: BlockInfo, l1_origin: BlockId, seq_num: u64) -> Self { + pub fn new(block_info: BlockInfo, l1_origin: BlockID, seq_num: u64) -> Self { Self { block_info, l1_origin, seq_num } } } -/// A Block Identifier -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Debug, Clone, Copy, Eq, PartialEq)] -pub enum BlockId { - /// The block hash - Hash(BlockHash), - /// The block number - Number(BlockNumber), - /// The block kind - Kind(BlockKind), -} - -impl Default for BlockId { - fn default() -> Self { - BlockId::Kind(BlockKind::Latest) - } -} - /// The Block Kind /// /// The block kinds are: diff --git a/crates/derive/src/types/genesis.rs b/crates/derive/src/types/genesis.rs index feff51780..bac8d02a7 100644 --- a/crates/derive/src/types/genesis.rs +++ b/crates/derive/src/types/genesis.rs @@ -1,15 +1,15 @@ //! This module contains the [Genesis] type. -use super::{BlockId, SystemConfig}; +use super::{BlockID, SystemConfig}; /// Represents the genesis state of the rollup. #[derive(Debug, Clone, Copy, Default)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct Genesis { /// The L1 block that the rollup starts *after* (no derived transactions) - pub l1: BlockId, + pub l1: BlockID, /// The L2 block the rollup starts from (no transactions, pre-configured state) - pub l2: BlockId, + pub l2: BlockID, /// Timestamp of the L2 block. pub timestamp: u64, /// Initial system configuration values. diff --git a/crates/derive/src/types/mod.rs b/crates/derive/src/types/mod.rs index fe5cbb87e..b1ee8ae05 100644 --- a/crates/derive/src/types/mod.rs +++ b/crates/derive/src/types/mod.rs @@ -15,10 +15,11 @@ pub use rollup_config::RollupConfig; pub mod batch; pub use batch::{ - Batch, BatchType, RawSpanBatch, SingleBatch, SpanBatch, SpanBatchBits, SpanBatchBuilder, - SpanBatchEip1559TransactionData, SpanBatchEip2930TransactionData, SpanBatchElement, - SpanBatchError, SpanBatchLegacyTransactionData, SpanBatchPayload, SpanBatchPrefix, - SpanBatchTransactionData, SpanBatchTransactions, SpanDecodingError, MAX_SPAN_BATCH_SIZE, + Batch, BatchType, BatchValidity, BatchWithInclusionBlock, RawSpanBatch, SingleBatch, SpanBatch, + SpanBatchBits, SpanBatchBuilder, SpanBatchEip1559TransactionData, + SpanBatchEip2930TransactionData, SpanBatchElement, SpanBatchError, + SpanBatchLegacyTransactionData, SpanBatchPayload, SpanBatchPrefix, SpanBatchTransactionData, + SpanBatchTransactions, SpanDecodingError, MAX_SPAN_BATCH_SIZE, }; mod alloy; @@ -29,8 +30,11 @@ pub use alloy::{ EMPTY_OMMER_ROOT_HASH, EMPTY_ROOT_HASH, }; +mod payload; +pub use payload::{ExecutionPayload, ExecutionPayloadEnvelope}; + mod block; -pub use block::{BlockId, BlockInfo, BlockKind}; +pub use block::{BlockID, BlockInfo, BlockKind, L2BlockInfo}; mod genesis; pub use genesis::Genesis; @@ -48,6 +52,18 @@ pub use errors::{DecodeError, StageError, StageResult}; #[derive(Debug, Clone, PartialEq, Eq)] pub struct RawTransaction(pub Bytes); +impl RawTransaction { + /// Returns if the transaction is empty + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } + + /// Returns if the transaction is a deposit + pub fn is_deposit(&self) -> bool { + !self.0.is_empty() && self.0[0] == 0x7E + } +} + impl Encodable for RawTransaction { fn encode(&self, out: &mut dyn alloy_rlp::BufMut) { self.0.encode(out) diff --git a/crates/derive/src/types/payload.rs b/crates/derive/src/types/payload.rs new file mode 100644 index 000000000..b06d3b925 --- /dev/null +++ b/crates/derive/src/types/payload.rs @@ -0,0 +1,64 @@ +//! Contains the execution payload type. + +use alloc::vec::Vec; +use alloy_primitives::{Address, Bytes, B256, U256}; + +#[cfg(feature = "serde")] +use serde::{Deserialize, Serialize}; + +/// Envelope wrapping the [ExecutionPayload]. +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExecutionPayloadEnvelope { + /// Parent beacon block root. + #[cfg_attr(feature = "serde", serde(rename = "parentBeaconBlockRoot"))] + parent_beacon_block_root: Option, + /// The inner execution payload. + #[cfg_attr(feature = "serde", serde(rename = "executionPayload"))] + execution_payload: ExecutionPayload, +} + +/// The execution payload. +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExecutionPayload { + #[cfg_attr(feature = "serde", serde(rename = "parentHash"))] + parent_hash: B256, + #[cfg_attr(feature = "serde", serde(rename = "feeRecipient"))] + fee_recipient: Address, + #[cfg_attr(feature = "serde", serde(rename = "stateRoot"))] + state_root: B256, + #[cfg_attr(feature = "serde", serde(rename = "receiptsRoot"))] + receipts_root: B256, + #[cfg_attr(feature = "serde", serde(rename = "logsBloom"))] + logs_bloom: B256, + #[cfg_attr(feature = "serde", serde(rename = "prevRandao"))] + prev_randao: B256, + #[cfg_attr(feature = "serde", serde(rename = "blockNumber"))] + block_number: u64, + #[cfg_attr(feature = "serde", serde(rename = "gasLimit"))] + gas_limit: u64, + #[cfg_attr(feature = "serde", serde(rename = "gasUsed"))] + gas_used: u64, + #[cfg_attr(feature = "serde", serde(rename = "timestamp"))] + timestamp: u64, + #[cfg_attr(feature = "serde", serde(rename = "extraData"))] + extra_data: B256, + #[cfg_attr(feature = "serde", serde(rename = "baseFeePerGas"))] + base_fee_per_gas: U256, + #[cfg_attr(feature = "serde", serde(rename = "blockHash"))] + block_hash: B256, + #[cfg_attr(feature = "serde", serde(rename = "transactions"))] + transactions: Vec, + #[cfg_attr(feature = "serde", serde(rename = "withdrawals"))] + withdrawals: Option, + #[cfg_attr(feature = "serde", serde(rename = "blobGasUsed"))] + blob_gas_used: Option, + #[cfg_attr(feature = "serde", serde(rename = "excessBlobGas"))] + excess_blob_gas: Option, +} + +/// Withdrawal Type +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone, PartialEq, Eq)] +pub(crate) struct Withdrawals {} diff --git a/crates/derive/src/types/system_config.rs b/crates/derive/src/types/system_config.rs index cdab8fe8b..758f3099b 100644 --- a/crates/derive/src/types/system_config.rs +++ b/crates/derive/src/types/system_config.rs @@ -247,8 +247,8 @@ mod test { fn mock_rollup_config(system_config: SystemConfig) -> RollupConfig { RollupConfig { genesis: Genesis { - l1: crate::types::BlockId::Number(0), - l2: crate::types::BlockId::Number(0), + l1: crate::types::BlockID::default(), + l2: crate::types::BlockID::default(), timestamp: 0, system_config, },