From 5a60f8db2dd86488f5488b25d4ee02c1732f4aa7 Mon Sep 17 00:00:00 2001 From: refcell Date: Fri, 19 Apr 2024 13:37:56 -0400 Subject: [PATCH] feat(derive): Span Batch Validation (#121) * feat(derive): span batch validation * feat(derive): span batch validity unit tests * feat(derive): span batch unit tests with acceptance test * fix(derive): unit tests * fix(derive): add more unit tests * feat(derive): span batch validity unit tests for txs --- crates/derive/src/stages/batch_queue.rs | 23 +- .../derive/src/stages/test_utils/tracing.rs | 26 +- crates/derive/src/types/batch/mod.rs | 8 +- .../src/types/batch/span_batch/batch.rs | 1347 ++++++++++++++++- .../src/types/batch/span_batch/element.rs | 2 +- crates/derive/src/types/payload.rs | 2 +- 6 files changed, 1385 insertions(+), 23 deletions(-) diff --git a/crates/derive/src/stages/batch_queue.rs b/crates/derive/src/stages/batch_queue.rs index ca6d80ae9..9a66fb20a 100644 --- a/crates/derive/src/stages/batch_queue.rs +++ b/crates/derive/src/stages/batch_queue.rs @@ -107,7 +107,11 @@ where /// 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 { + pub async 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::MissingOrigin); @@ -140,7 +144,8 @@ where 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); + let validity = + batch.check_batch(&self.cfg, &self.l1_blocks, parent, &mut self.fetcher).await; match validity { BatchValidity::Future => { remaining.push(batch.clone()); @@ -221,7 +226,7 @@ where } /// Adds a batch to the queue. - pub fn add_batch(&mut self, batch: Batch, parent: L2BlockInfo) -> StageResult<()> { + pub async fn add_batch(&mut self, batch: Batch, parent: L2BlockInfo) -> StageResult<()> { if self.l1_blocks.is_empty() { error!("Cannot add batch without an origin"); panic!("Cannot add batch without an origin"); @@ -229,7 +234,7 @@ where 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() { + if data.check_batch(&self.cfg, &self.l1_blocks, parent, &mut self.fetcher).await.is_drop() { return Ok(()); } self.batches.push(data); @@ -310,7 +315,7 @@ where match self.prev.next_batch().await { Ok(b) => { if !origin_behind { - self.add_batch(b, parent).ok(); + self.add_batch(b, parent).await.ok(); } else { warn!("Dropping batch: Origin is behind"); } @@ -329,7 +334,7 @@ where } // Attempt to derive more batches. - let batch = match self.derive_next_batch(out_of_data, parent) { + let batch = match self.derive_next_batch(out_of_data, parent).await { Ok(b) => b, Err(e) => match e { StageError::Eof => { @@ -418,15 +423,15 @@ mod tests { BatchReader::from(compressed) } - #[test] - fn test_derive_next_batch_missing_origin() { + #[tokio::test] + async fn test_derive_next_batch_missing_origin() { let data = vec![Ok(Batch::Single(SingleBatch::default()))]; let cfg = Arc::new(RollupConfig::default()); let mock = MockBatchQueueProvider::new(data); let fetcher = MockBlockFetcher::default(); let mut bq = BatchQueue::new(cfg, mock, fetcher); let parent = L2BlockInfo::default(); - let result = bq.derive_next_batch(false, parent).unwrap_err(); + let result = bq.derive_next_batch(false, parent).await.unwrap_err(); assert_eq!(result, StageError::MissingOrigin); } diff --git a/crates/derive/src/stages/test_utils/tracing.rs b/crates/derive/src/stages/test_utils/tracing.rs index a73910e9b..eb065c203 100644 --- a/crates/derive/src/stages/test_utils/tracing.rs +++ b/crates/derive/src/stages/test_utils/tracing.rs @@ -7,7 +7,29 @@ use tracing::{Event, Level, Subscriber}; use tracing_subscriber::{layer::Context, Layer}; /// The storage for the collected traces. -pub type TraceStorage = Arc>>; +#[derive(Debug, Default, Clone)] +pub struct TraceStorage(pub Arc>>); + +impl TraceStorage { + /// Returns the items in the storage that match the specified level. + pub fn get_by_level(&self, level: Level) -> Vec { + self.0 + .lock() + .iter() + .filter_map(|(l, message)| if *l == level { Some(message.clone()) } else { None }) + .collect() + } + + /// Locks the storage and returns the items. + pub fn lock(&self) -> spin::MutexGuard<'_, Vec<(Level, String)>> { + self.0.lock() + } + + /// Returns if the storage is empty. + pub fn is_empty(&self) -> bool { + self.0.lock().is_empty() + } +} #[derive(Debug, Default)] pub struct CollectingLayer { @@ -26,7 +48,7 @@ impl Layer for CollectingLayer { let level = *metadata.level(); let message = format!("{:?}", event); - let mut storage = self.storage.lock(); + let mut storage = self.storage.0.lock(); storage.push((level, message)); } } diff --git a/crates/derive/src/types/batch/mod.rs b/crates/derive/src/types/batch/mod.rs index e01be15b7..877fae810 100644 --- a/crates/derive/src/types/batch/mod.rs +++ b/crates/derive/src/types/batch/mod.rs @@ -40,19 +40,21 @@ impl BatchWithInclusionBlock { /// 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( + pub async fn check_batch( &self, cfg: &RollupConfig, l1_blocks: &[BlockInfo], l2_safe_head: L2BlockInfo, - fetcher: &BF, + fetcher: &mut 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) + span_batch + .check_batch(cfg, l1_blocks, l2_safe_head, &self.inclusion_block, fetcher) + .await } } } diff --git a/crates/derive/src/types/batch/span_batch/batch.rs b/crates/derive/src/types/batch/span_batch/batch.rs index 050ba2892..f82288465 100644 --- a/crates/derive/src/types/batch/span_batch/batch.rs +++ b/crates/derive/src/types/batch/span_batch/batch.rs @@ -12,6 +12,8 @@ use crate::{ }; use alloc::{vec, vec::Vec}; use alloy_primitives::FixedBytes; +use op_alloy_consensus::OpTxType; +use tracing::{info, warn}; /// The span batch contains the input to build a span of L2 blocks in derived form. #[derive(Debug, Default, Clone, PartialEq, Eq)] @@ -40,16 +42,280 @@ impl SpanBatch { self.batches[0].timestamp } + /// Returns the epoch number for the first batch in the span. + pub fn starting_epoch_num(&self) -> u64 { + self.batches[0].epoch_num + } + + /// Checks if the first 20 bytes of the given hash match the L1 origin check. + pub fn check_origin_hash(&self, hash: FixedBytes<32>) -> bool { + self.l1_origin_check == hash[..20] + } + + /// Checks if the first 20 bytes of the given hash match the parent check. + pub fn check_parent_hash(&self, hash: FixedBytes<32>) -> bool { + self.parent_check == hash[..20] + } + /// Checks if the span batch is valid. - pub fn check_batch( + pub async fn check_batch( &self, - _cfg: &RollupConfig, - _l1_blocks: &[BlockInfo], - _l2_safe_head: L2BlockInfo, - _inclusion_block: &BlockInfo, - _fetcher: &BF, + cfg: &RollupConfig, + l1_blocks: &[BlockInfo], + l2_safe_head: L2BlockInfo, + inclusion_block: &BlockInfo, + fetcher: &mut BF, ) -> BatchValidity { - unimplemented!() + if l1_blocks.is_empty() { + warn!("missing L1 block input, cannot proceed with batch checking"); + return BatchValidity::Undecided; + } + if self.batches.is_empty() { + warn!("empty span batch, cannot proceed with batch checking"); + return BatchValidity::Undecided; + } + let epoch = l1_blocks[0]; + let mut batch_origin = epoch; + let starting_epoch_num = self.starting_epoch_num(); + if starting_epoch_num == batch_origin.number + 1 { + if l1_blocks.len() < 2 { + info!("eager batch wants to advance current epoch {}, but could not without more L1 blocks", epoch.id()); + return BatchValidity::Undecided; + } + batch_origin = l1_blocks[1]; + } + + // Span batches are only valid after the Delta hard fork. + if !cfg.is_delta_active(batch_origin.timestamp) { + warn!( + "received SpanBatch (id {}) with L1 origin (timestamp {}) before Delta hard fork", + batch_origin.id(), + batch_origin.timestamp + ); + return BatchValidity::Drop; + } + + // Skip out of order batches. + let next_timestamp = l2_safe_head.block_info.timestamp + cfg.block_time; + if self.timestamp() > next_timestamp { + warn!( + "received out-of-order batch for future processing after next batch ({} > {})", + self.timestamp(), + next_timestamp + ); + return BatchValidity::Future; + } + // SAFETY: The span batch is not empty so the last element exists. + if self.batches.last().unwrap().timestamp < next_timestamp { + warn!("span batch has no new blocks after safe head"); + return BatchValidity::Drop; + } + + // Find the parent block of the span batch. + // If the span batch does not overlap the current safe chain, parent block should be the L2 + // safe head. + let mut parent_num = l2_safe_head.block_info.number; + let parent_block = l2_safe_head; + if self.timestamp() < next_timestamp { + if self.timestamp() > l2_safe_head.block_info.timestamp { + // Batch timestamp cannot be between safe head and next timestamp. + warn!("batch has misaligned timestamp, block time is too short"); + return BatchValidity::Drop; + } + if (l2_safe_head.block_info.timestamp - self.timestamp()) % cfg.block_time != 0 { + warn!("batch has misaligned timestamp, not overlapped exactly"); + return BatchValidity::Drop; + } + parent_num = l2_safe_head.block_info.number - + (l2_safe_head.block_info.timestamp - self.timestamp()) / cfg.block_time - + 1; + let parent_block = match fetcher.l2_block_info_by_number(parent_num).await { + Ok(block) => block, + Err(e) => { + warn!("failed to fetch L2 block number {parent_num}: {e}"); + // Unable to validate the batch for now. Retry later. + return BatchValidity::Undecided; + } + }; + } + if !self.check_parent_hash(parent_block.block_info.parent_hash) { + warn!( + "parent block number mismatch, expected: {parent_num}, received: {}", + parent_block.block_info.number + ); + return BatchValidity::Drop; + } + + // Filter out batches that were included too late. + if starting_epoch_num + cfg.seq_window_size < inclusion_block.number { + warn!("batch was included too late, sequence window expired"); + return BatchValidity::Drop; + } + + // Check the L1 origin of the batch + if starting_epoch_num > parent_block.l1_origin.number + 1 { + warn!( + "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; + } + + // Verify the l1 origin hash for each l1 block. + // SAFETY: The span batch is not empty so the last element exists. + let end_epoch_num = self.batches.last().unwrap().epoch_num; + let mut origin_checked = false; + // l1Blocks is supplied from batch queue and its length is limited to SequencerWindowSize. + for l1_block in l1_blocks { + if l1_block.number == end_epoch_num { + if !self.check_origin_hash(l1_block.hash) { + warn!( + "batch is for different L1 chain, epoch hash does not match, expected: {}", + l1_block.hash + ); + return BatchValidity::Drop; + } + origin_checked = true; + break; + } + } + if !origin_checked { + info!("need more l1 blocks to check entire origins of span batch"); + return BatchValidity::Undecided; + } + + // Check if the batch is too old. + if starting_epoch_num < parent_block.l1_origin.number { + warn!("dropped batch, epoch is too old, minimum: {}", parent_block.block_info.id()); + return BatchValidity::Drop; + } + + let mut origin_index = 0; + let mut origin_advanced = starting_epoch_num == parent_block.l1_origin.number + 1; + for (i, batch) in self.batches.iter().enumerate() { + if batch.timestamp <= l2_safe_head.block_info.timestamp { + continue; + } + // Find the L1 origin for the batch. + for (j, j_block) in l1_blocks.iter().enumerate().skip(origin_index) { + if batch.epoch_num == j_block.number { + origin_index = j; + break; + } + } + let l1_origin = l1_blocks[origin_index]; + if i > 0 { + origin_advanced = false; + if batch.epoch_num > self.batches[i - 1].epoch_num { + origin_advanced = true; + } + } + let block_timestamp = batch.timestamp; + if block_timestamp < l1_origin.timestamp { + warn!( + "block timestamp is less than L1 origin timestamp, l2_timestamp: {}, l1_timestamp: {}, origin: {}", + block_timestamp, + l1_origin.timestamp, + l1_origin.id() + ); + return BatchValidity::Drop; + } + // Check if we ran out of sequencer time drift + if block_timestamp > l1_origin.timestamp + cfg.max_sequencer_drift { + if batch.transactions.is_empty() { + // If the sequencer is co-operating by producing an empty batch, + // then allow the batch if it was the right thing to do to maintain the L2 time + // >= L1 time invariant. We only check batches that do not + // advance the epoch, to ensure epoch advancement regardless of time drift is + // allowed. + if !origin_advanced { + if origin_index + 1 >= l1_blocks.len() { + info!("without the next L1 origin we cannot determine yet if this empty batch that exceeds the time drift is still valid"); + return BatchValidity::Undecided; + } + if block_timestamp >= l1_blocks[origin_index + 1].timestamp { + // check if the next L1 origin could have been adopted + info!("batch exceeded sequencer time drift without adopting next origin, and next L1 origin would have been valid"); + return BatchValidity::Drop; + } else { + info!("continuing with empty batch before late L1 block to preserve L2 time invariant"); + } + } + } else { + // 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. + warn!( + "batch exceeded sequencer time drift, sequencer must adopt new L1 origin to include transactions again, max_time: {}", + l1_origin.timestamp + cfg.max_sequencer_drift + ); + return BatchValidity::Drop; + } + } + + // Check that the transactions are not empty and do not contain any deposits. + for (tx_index, tx_bytes) in batch.transactions.iter().enumerate() { + if tx_bytes.is_empty() { + warn!( + "transaction data must not be empty, but found empty tx, tx_index: {}", + tx_index + ); + return BatchValidity::Drop; + } + if tx_bytes.0[0] == OpTxType::Deposit as u8 { + warn!("sequencers may not embed any deposits into batch data, but found tx that has one, tx_index: {}", tx_index); + return BatchValidity::Drop; + } + } + } + + // Check overlapped blocks + if self.timestamp() < next_timestamp { + for i in 0..(l2_safe_head.block_info.number - parent_num) { + let safe_block_num = parent_num + i + 1; + let safe_block_payload = match fetcher.payload_by_number(safe_block_num).await { + Ok(p) => p, + Err(e) => { + warn!("failed to fetch payload for block number {safe_block_num}: {e}"); + return BatchValidity::Undecided; + } + }; + let safe_block_txs = &safe_block_payload.execution_payload.transactions; + let batch_txs = &self.batches[i as usize].transactions; + // Execution payload has deposit txs but batch does not. + let deposit_count: usize = safe_block_txs + .iter() + .map(|tx| if tx.0[0] == OpTxType::Deposit as u8 { 1 } else { 0 }) + .sum(); + if safe_block_txs.len() - deposit_count != batch_txs.len() { + warn!( + "overlapped block's tx count does not match, safe_block_txs: {}, batch_txs: {}", + safe_block_txs.len(), + batch_txs.len() + ); + return BatchValidity::Drop; + } + for j in 0..batch_txs.len() { + if safe_block_txs[j + deposit_count] != batch_txs[j].0 { + warn!("overlapped block's transaction does not match"); + return BatchValidity::Drop; + } + } + let safe_block_ref = match safe_block_payload.to_l2_block_ref(cfg) { + Ok(r) => r, + Err(e) => { + warn!("failed to extract L2BlockInfo from execution payload, hash: {}, err: {e}", safe_block_payload.execution_payload.block_hash); + return BatchValidity::Drop; + } + }; + if safe_block_ref.l1_origin.number != self.batches[i as usize].epoch_num { + warn!("overlapped block's L1 origin number does not match"); + return BatchValidity::Drop; + } + } + } + + BatchValidity::Accept } /// Converts the span batch to a raw span batch. @@ -162,3 +428,1070 @@ impl SpanBatch { &self.batches[self.batches.len() - 1 - n] } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::{ + stages::test_utils::{CollectingLayer, TraceStorage}, + traits::test_utils::MockBlockFetcher, + types::{BlockID, Genesis, L2ExecutionPayload, L2ExecutionPayloadEnvelope, RawTransaction}, + }; + use alloy_primitives::{b256, Bytes, B256}; + use op_alloy_consensus::OpTxType; + use tracing::Level; + use tracing_subscriber::{layer::SubscriberExt, util::SubscriberInitExt}; + + #[test] + fn test_timestamp() { + let timestamp = 10; + let first_element = SpanBatchElement { timestamp, ..Default::default() }; + let mut batch = + SpanBatch { batches: vec![first_element, Default::default()], ..Default::default() }; + assert_eq!(batch.timestamp(), timestamp); + } + + #[test] + fn test_starting_epoch_num() { + let epoch_num = 10; + let first_element = SpanBatchElement { epoch_num, ..Default::default() }; + let mut batch = + SpanBatch { batches: vec![first_element, Default::default()], ..Default::default() }; + assert_eq!(batch.starting_epoch_num(), epoch_num); + } + + #[test] + fn test_check_origin_hash() { + let l1_origin_check = FixedBytes::from([17u8; 20]); + let hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let batch = SpanBatch { l1_origin_check, ..Default::default() }; + assert!(batch.check_origin_hash(hash)); + // This hash has 19 matching bytes, the other 13 are zeros. + let invalid = b256!("1111111111111111111111111111111111111100000000000000000000000000"); + assert!(!batch.check_origin_hash(invalid)); + } + + #[test] + fn test_check_parent_hash() { + let parent_check = FixedBytes::from([17u8; 20]); + let hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let batch = SpanBatch { parent_check, ..Default::default() }; + assert!(batch.check_parent_hash(hash)); + // This hash has 19 matching bytes, the other 13 are zeros. + let invalid = b256!("1111111111111111111111111111111111111100000000000000000000000000"); + assert!(!batch.check_parent_hash(invalid)); + } + + #[tokio::test] + async fn test_check_batch_missing_l1_block_input() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig::default(); + let l1_blocks = vec![]; + let l2_safe_head = L2BlockInfo::default(); + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let batch = SpanBatch::default(); + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("missing L1 block input, cannot proceed with batch checking")); + } + + #[tokio::test] + async fn test_check_batches_is_empty() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig::default(); + let l1_blocks = vec![BlockInfo::default()]; + let l2_safe_head = L2BlockInfo::default(); + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let batch = SpanBatch::default(); + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("empty span batch, cannot proceed with batch checking")); + } + + #[tokio::test] + async fn test_eager_block_missing_origins() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig::default(); + let block = BlockInfo { number: 9, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo::default(); + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, ..Default::default() }; + let batch = SpanBatch { batches: vec![first], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::INFO); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "eager batch wants to advance current epoch {}, but could not without more L1 blocks", + block.id() + ); + assert!(logs[0].contains(&str)); + } + + #[tokio::test] + async fn test_check_batch_delta_inactive() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(10), ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 9, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo::default(); + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let batch = SpanBatch { batches: vec![first], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "received SpanBatch (id {}) with L1 origin (timestamp {}) before Delta hard fork", + block.id(), + block.timestamp + ); + assert!(logs[0].contains(&str)); + } + + #[tokio::test] + async fn test_check_batch_out_of_order() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 21, ..Default::default() }; + let batch = SpanBatch { batches: vec![first], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Future + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains( + "received out-of-order batch for future processing after next batch (21 > 20)" + )); + } + + #[tokio::test] + async fn test_check_batch_no_new_blocks() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let batch = SpanBatch { batches: vec![first], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("span batch has no new blocks after safe head")); + } + + #[tokio::test] + async fn test_check_batch_misaligned_timestamp() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 11, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 21, ..Default::default() }; + let batch = SpanBatch { batches: vec![first, second], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("batch has misaligned timestamp, block time is too short")); + } + + #[tokio::test] + async fn test_check_batch_misaligned_without_overlap() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 8, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { batches: vec![first, second], ..Default::default() }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("batch has misaligned timestamp, not overlapped exactly")); + } + + #[tokio::test] + async fn test_check_batch_failed_to_fetch_l2_block() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let mut fetcher = MockBlockFetcher::default(); + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { batches: vec![first, second], ..Default::default() }; + // parent number = 41 - (10 - 10) / 10 - 1 = 40 + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("failed to fetch L2 block number 40: Block not found")); + } + + #[tokio::test] + async fn test_check_batch_parent_hash_fail() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo::default(); + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice( + &b256!("1111111111111111111111111111111111111111000000000000000000000000")[..20], + ), + ..Default::default() + }; + // parent number = 41 - (10 - 10) / 10 - 1 = 40 + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("parent block number mismatch, expected: 40, received: 41")); + } + + #[tokio::test] + async fn test_check_sequence_window_expired() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { delta_time: Some(0), block_time: 10, ..Default::default() }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + ..Default::default() + }; + // parent number = 41 - (10 - 10) / 10 - 1 = 40 + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("batch was included too late, sequence window expired")); + } + + #[tokio::test] + async fn test_starting_epoch_too_far_ahead() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let block = BlockInfo { number: 10, timestamp: 10, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 8, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + ..Default::default() + }; + // parent number = 41 - (10 - 10) / 10 - 1 = 40 + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "batch is for future epoch too far ahead, while it has the next timestamp, so it must be invalid, current_epoch: {}", + block.id(), + ); + assert!(logs[0].contains(&str)); + } + + #[tokio::test] + async fn test_check_batch_epoch_hash_mismatch() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "batch is for different L1 chain, epoch hash does not match, expected: {}", + l1_block_hash, + ); + assert!(logs[0].contains(&str)); + } + + #[tokio::test] + async fn test_need_more_l1_blocks() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 10, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::INFO); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("need more l1 blocks to check entire origins of span batch")); + } + + #[tokio::test] + async fn test_drop_batch_epoch_too_old() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 13, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "dropped batch, epoch is too old, minimum: {}", + l2_safe_head.block_info.id(), + ); + assert!(logs[0].contains(&str)); + } + + // TODO: Test block timestamp less than L1 origin + + // TODO: Test missing l1 origin for empty batch + + #[tokio::test] + async fn test_check_batch_exceeds_max_seq_drif() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + max_sequencer_drift: 0, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let second_block = + BlockInfo { number: 12, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block, second_block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 20, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 10, timestamp: 20, ..Default::default() }; + let third = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second, third], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::INFO); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("batch exceeded sequencer time drift without adopting next origin, and next L1 origin would have been valid")); + } + + #[tokio::test] + async fn test_continuing_with_empty_batch() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + max_sequencer_drift: 0, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let second_block = + BlockInfo { number: 12, timestamp: 21, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block, second_block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 20, transactions: vec![] }; + let second = SpanBatchElement { epoch_num: 10, timestamp: 20, transactions: vec![] }; + let third = SpanBatchElement { epoch_num: 11, timestamp: 20, transactions: vec![] }; + let batch = SpanBatch { + batches: vec![first, second, third], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + txs: SpanBatchTransactions::default(), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Accept + ); + let infos = trace_store.get_by_level(Level::INFO); + assert_eq!(infos.len(), 1); + assert!(infos[0].contains( + "continuing with empty batch before late L1 block to preserve L2 time invariant" + )); + } + + #[tokio::test] + async fn test_check_batch_exceeds_sequencer_time_drift() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + max_sequencer_drift: 0, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let second_block = + BlockInfo { number: 12, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block, second_block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![Default::default()], + }; + let second = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![Default::default()], + }; + let third = SpanBatchElement { + epoch_num: 11, + timestamp: 20, + transactions: vec![Default::default()], + }; + let batch = SpanBatch { + batches: vec![first, second, third], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + txs: SpanBatchTransactions::default(), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("batch exceeded sequencer time drift, sequencer must adopt new L1 origin to include transactions again, max_time: 10")); + } + + #[tokio::test] + async fn test_check_batch_empty_txs() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + max_sequencer_drift: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let second_block = + BlockInfo { number: 12, timestamp: 21, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block, second_block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![Default::default()], + }; + let second = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![Default::default()], + }; + let third = SpanBatchElement { epoch_num: 11, timestamp: 20, transactions: vec![] }; + let batch = SpanBatch { + batches: vec![first, second, third], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + txs: SpanBatchTransactions::default(), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("transaction data must not be empty, but found empty tx")); + } + + #[tokio::test] + async fn test_check_batch_with_deposit_tx() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + max_sequencer_drift: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let second_block = + BlockInfo { number: 12, timestamp: 21, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block, second_block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let filler_bytes = RawTransaction(Bytes::copy_from_slice(&[OpTxType::Eip1559 as u8])); + let first = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![filler_bytes.clone()], + }; + let second = SpanBatchElement { + epoch_num: 10, + timestamp: 20, + transactions: vec![RawTransaction(Bytes::copy_from_slice(&[OpTxType::Deposit as u8]))], + }; + let third = + SpanBatchElement { epoch_num: 11, timestamp: 20, transactions: vec![filler_bytes] }; + let batch = SpanBatch { + batches: vec![first, second, third], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + txs: SpanBatchTransactions::default(), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("sequencers may not embed any deposits into batch data, but found tx that has one, tx_index: 0")); + } + + #[tokio::test] + async fn test_check_batch_failed_to_fetch_payload() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Undecided + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("failed to fetch payload for block number 41: Payload not found")); + } + + // TODO: Test overlap block tx count mismatch + + // TODO: Test overlap block tx doesn't match + + #[tokio::test] + async fn test_check_batch_failed_to_extract_l2_block_info() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let payload = L2ExecutionPayloadEnvelope { + parent_beacon_block_root: None, + execution_payload: L2ExecutionPayload { block_number: 41, ..Default::default() }, + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![payload] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + let str = alloc::format!( + "failed to extract L2BlockInfo from execution payload, hash: {}", + B256::default(), + ); + assert!(logs[0].contains(&str)); + } + + #[tokio::test] + async fn test_overlapped_blocks_origin_mismatch() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let payload_block_hash = + b256!("4444444444444444444444444444444444444444444444444444444444444444"); + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + genesis: Genesis { + l2: BlockID { number: 41, hash: payload_block_hash }, + ..Default::default() + }, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let payload = L2ExecutionPayloadEnvelope { + parent_beacon_block_root: None, + execution_payload: L2ExecutionPayload { + block_number: 41, + block_hash: payload_block_hash, + ..Default::default() + }, + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![payload] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Drop + ); + let logs = trace_store.get_by_level(Level::WARN); + assert_eq!(logs.len(), 1); + assert!(logs[0].contains("overlapped block's L1 origin number does not match")); + } + + #[tokio::test] + async fn test_check_batch_valid_with_genesis_epoch() { + let trace_store: TraceStorage = Default::default(); + let layer = CollectingLayer::new(trace_store.clone()); + tracing_subscriber::Registry::default().with(layer).init(); + + let payload_block_hash = + b256!("4444444444444444444444444444444444444444444444444444444444444444"); + let cfg = RollupConfig { + seq_window_size: 100, + delta_time: Some(0), + block_time: 10, + genesis: Genesis { + l2: BlockID { number: 41, hash: payload_block_hash }, + l1: BlockID { number: 10, ..Default::default() }, + ..Default::default() + }, + ..Default::default() + }; + let l1_block_hash = + b256!("3333333333333333333333333333333333333333000000000000000000000000"); + let block = + BlockInfo { number: 11, timestamp: 10, hash: l1_block_hash, ..Default::default() }; + let l1_blocks = vec![block]; + let parent_hash = b256!("1111111111111111111111111111111111111111000000000000000000000000"); + let l2_safe_head = L2BlockInfo { + block_info: BlockInfo { number: 41, timestamp: 10, parent_hash, ..Default::default() }, + l1_origin: BlockID { number: 9, ..Default::default() }, + ..Default::default() + }; + let inclusion_block = BlockInfo { number: 50, ..Default::default() }; + let l2_block = L2BlockInfo { + block_info: BlockInfo { number: 40, ..Default::default() }, + ..Default::default() + }; + let payload = L2ExecutionPayloadEnvelope { + parent_beacon_block_root: None, + execution_payload: L2ExecutionPayload { + block_number: 41, + block_hash: payload_block_hash, + ..Default::default() + }, + }; + let mut fetcher = MockBlockFetcher { blocks: vec![l2_block], payloads: vec![payload] }; + let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; + let second = SpanBatchElement { epoch_num: 11, timestamp: 20, ..Default::default() }; + let batch = SpanBatch { + batches: vec![first, second], + parent_check: FixedBytes::<20>::from_slice(&parent_hash[..20]), + l1_origin_check: FixedBytes::<20>::from_slice(&l1_block_hash[..20]), + ..Default::default() + }; + assert_eq!( + batch.check_batch(&cfg, &l1_blocks, l2_safe_head, &inclusion_block, &mut fetcher).await, + BatchValidity::Accept + ); + assert!(trace_store.is_empty()); + } +} diff --git a/crates/derive/src/types/batch/span_batch/element.rs b/crates/derive/src/types/batch/span_batch/element.rs index 4be577660..0ff115694 100644 --- a/crates/derive/src/types/batch/span_batch/element.rs +++ b/crates/derive/src/types/batch/span_batch/element.rs @@ -6,7 +6,7 @@ use alloc::vec::Vec; /// A single batch element is similar to the [SingleBatch] type /// but does not contain the parent hash and epoch hash since spans /// do not contain this data for every block in the span. -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Default, Clone, PartialEq, Eq)] pub struct SpanBatchElement { /// The epoch number of the L1 block pub epoch_num: u64, diff --git a/crates/derive/src/types/payload.rs b/crates/derive/src/types/payload.rs index 347646f67..2dda40aed 100644 --- a/crates/derive/src/types/payload.rs +++ b/crates/derive/src/types/payload.rs @@ -48,7 +48,7 @@ impl L2ExecutionPayloadEnvelope { /// The execution payload. #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Default, Clone, PartialEq, Eq)] pub struct L2ExecutionPayload { /// The parent hash. #[cfg_attr(feature = "serde", serde(rename = "parentHash"))]