diff --git a/Cargo.lock b/Cargo.lock index e81ad352..dee7789f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2477,15 +2477,10 @@ dependencies = [ name = "kona-primitives" version = "0.0.2" dependencies = [ - "alloy-consensus", "alloy-eips", "alloy-primitives", - "alloy-rlp", "anyhow", "c-kzg", - "op-alloy-consensus", - "op-alloy-genesis", - "op-alloy-protocol", "revm", "serde", "serde_json", diff --git a/bin/client/src/l2/chain_provider.rs b/bin/client/src/l2/chain_provider.rs index 79694539..83c8c5b4 100644 --- a/bin/client/src/l2/chain_provider.rs +++ b/bin/client/src/l2/chain_provider.rs @@ -8,10 +8,9 @@ use alloy_primitives::{Address, Bytes, B256}; use alloy_rlp::Decodable; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use kona_derive::traits::L2ChainProvider; +use kona_derive::{block::OpBlock, traits::L2ChainProvider}; use kona_mpt::{OrderedListWalker, TrieDBFetcher, TrieDBHinter}; use kona_preimage::{CommsClient, PreimageKey, PreimageKeyType}; -use kona_primitives::{L2ExecutionPayloadEnvelope, OpBlock}; use op_alloy_consensus::OpTxEnvelope; use op_alloy_genesis::{RollupConfig, SystemConfig}; use op_alloy_protocol::L2BlockInfo; @@ -73,13 +72,13 @@ impl L2ChainProvider for OracleL2ChainProvider async fn l2_block_info_by_number(&mut self, number: u64) -> Result { // Get the payload at the given block number. - let payload = self.payload_by_number(number).await?; + let payload = self.block_by_number(number).await?; // Construct the system config from the payload. payload.to_l2_block_ref(&self.boot_info.rollup_config).map_err(Into::into) } - async fn payload_by_number(&mut self, number: u64) -> Result { + async fn block_by_number(&mut self, number: u64) -> Result { // Fetch the header for the given block number. let header @ Header { transactions_root, timestamp, .. } = self.header_by_number(number).await?; @@ -104,7 +103,7 @@ impl L2ChainProvider for OracleL2ChainProvider withdrawals: self.boot_info.rollup_config.is_canyon_active(timestamp).then(Vec::new), ..Default::default() }; - Ok(optimism_block.into()) + Ok(optimism_block) } async fn system_config_by_number( @@ -113,7 +112,7 @@ impl L2ChainProvider for OracleL2ChainProvider rollup_config: Arc, ) -> Result { // Get the payload at the given block number. - let payload = self.payload_by_number(number).await?; + let payload = self.block_by_number(number).await?; // Construct the system config from the payload. payload.to_system_config(rollup_config.as_ref()).map_err(Into::into) diff --git a/crates/derive/src/batch/span_batch/batch.rs b/crates/derive/src/batch/span_batch/batch.rs index b1c5492d..fd7a8285 100644 --- a/crates/derive/src/batch/span_batch/batch.rs +++ b/crates/derive/src/batch/span_batch/batch.rs @@ -1,6 +1,7 @@ //! The Span Batch Type use alloc::vec::Vec; +use alloy_eips::eip2718::Encodable2718; use alloy_primitives::FixedBytes; use op_alloy_consensus::OpTxType; use op_alloy_genesis::RollupConfig; @@ -277,20 +278,18 @@ impl SpanBatch { 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 { + let safe_block_payload = match fetcher.block_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 safe_block_txs = &safe_block_payload.body; 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(); + let deposit_count: usize = + safe_block_txs.iter().map(|tx| if tx.is_deposit() { 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: {}", @@ -300,7 +299,9 @@ impl SpanBatch { return BatchValidity::Drop; } for j in 0..batch_txs.len() { - if safe_block_txs[j + deposit_count] != batch_txs[j].0 { + let mut buf = Vec::new(); + safe_block_txs[j + deposit_count].encode_2718(&mut buf); + if buf != batch_txs[j].0 { warn!("overlapped block's transaction does not match"); return BatchValidity::Drop; } @@ -308,7 +309,7 @@ impl SpanBatch { 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); + warn!("failed to extract L2BlockInfo from execution payload, hash: {}, err: {e}", safe_block_payload.header.hash_slow()); return BatchValidity::Drop; } }; @@ -417,13 +418,14 @@ impl SpanBatch { mod tests { use super::*; use crate::{ + block::OpBlock, stages::test_utils::{CollectingLayer, TraceStorage}, traits::test_utils::TestL2ChainProvider, }; use alloc::vec; + use alloy_consensus::Header; use alloy_eips::BlockNumHash; use alloy_primitives::{b256, Bytes, B256}; - use kona_primitives::{L2ExecutionPayload, L2ExecutionPayloadEnvelope}; use op_alloy_consensus::OpTxType; use op_alloy_genesis::ChainGenesis; use tracing::Level; @@ -1407,13 +1409,11 @@ mod tests { l1_origin: BlockNumHash { number: 9, ..Default::default() }, ..Default::default() }; - let payload = L2ExecutionPayloadEnvelope { - parent_beacon_block_root: None, - execution_payload: L2ExecutionPayload { block_number: 41, ..Default::default() }, - }; + let block = + OpBlock { header: Header { number: 41, ..Default::default() }, ..Default::default() }; let mut fetcher = TestL2ChainProvider { blocks: vec![l2_block], - payloads: vec![payload], + op_blocks: vec![block], ..Default::default() }; let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; @@ -1477,17 +1477,17 @@ mod tests { l1_origin: BlockNumHash { number: 9, ..Default::default() }, ..Default::default() }; - let payload = L2ExecutionPayloadEnvelope { - parent_beacon_block_root: None, - execution_payload: L2ExecutionPayload { - block_number: 41, - block_hash: payload_block_hash, + let block = OpBlock { + header: Header { + number: 41, + // TODO: correct hash ..Default::default() }, + ..Default::default() }; let mut fetcher = TestL2ChainProvider { blocks: vec![l2_block], - payloads: vec![payload], + op_blocks: vec![block], ..Default::default() }; let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; @@ -1553,17 +1553,17 @@ mod tests { l1_origin: BlockNumHash { number: 9, ..Default::default() }, ..Default::default() }; - let payload = L2ExecutionPayloadEnvelope { - parent_beacon_block_root: None, - execution_payload: L2ExecutionPayload { - block_number: 41, - block_hash: payload_block_hash, + let block = OpBlock { + header: Header { + number: 41, + // TODO: correct hash ..Default::default() }, + ..Default::default() }; let mut fetcher = TestL2ChainProvider { blocks: vec![l2_block], - payloads: vec![payload], + op_blocks: vec![block], ..Default::default() }; let first = SpanBatchElement { epoch_num: 10, timestamp: 10, ..Default::default() }; diff --git a/crates/derive/src/block.rs b/crates/derive/src/block.rs new file mode 100644 index 00000000..eb847f69 --- /dev/null +++ b/crates/derive/src/block.rs @@ -0,0 +1,166 @@ +//! This module contains the various Block types. + +use alloc::vec::Vec; +use alloy_consensus::{Header, TxEnvelope}; +use alloy_eips::eip4895::Withdrawal; +use alloy_primitives::B256; +use alloy_rlp::{RlpDecodable, RlpEncodable}; +use op_alloy_consensus::OpTxEnvelope; +use op_alloy_genesis::{RollupConfig, SystemConfig}; +use op_alloy_protocol::{ + block_info::DecodeError, BlockInfo, L1BlockInfoBedrock, L1BlockInfoEcotone, L1BlockInfoTx, + L2BlockInfo, +}; +use thiserror::Error; + +/// Ethereum full block. +/// +/// Withdrawals can be optionally included at the end of the RLP encoded message. +/// +/// Taken from [reth-primitives](https://github.com/paradigmxyz/reth) +#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)] +#[rlp(trailing)] +pub struct Block { + /// Block header. + pub header: Header, + /// Transactions in this block. + pub body: Vec, + /// Ommers/uncles header. + pub ommers: Vec
, + /// Block withdrawals. + pub withdrawals: Option>, +} + +/// An error encountered during [OpBlock] conversion. +#[derive(Error, Debug)] +pub enum OpBlockConversionError { + /// Invalid genesis hash. + #[error("Invalid genesis hash. Expected {0}, got {1}")] + InvalidGenesisHash(B256, B256), + /// Invalid transaction type. + #[error("First payload transaction has unexpected type: {0}")] + InvalidTxType(u8), + /// L1 Info error + #[error(transparent)] + L1InfoError(#[from] DecodeError), + /// Missing system config in genesis block. + #[error("Missing system config in genesis block")] + MissingSystemConfigGenesis, + /// Empty transactions. + #[error("Empty transactions in payload. Block hash: {0}")] + EmptyTransactions(B256), +} + +/// OP Stack full block. +/// +/// Withdrawals can be optionally included at the end of the RLP encoded message. +/// +/// Taken from [reth-primitives](https://github.com/paradigmxyz/reth) +#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)] +#[rlp(trailing)] +pub struct OpBlock { + /// Block header. + pub header: Header, + /// Transactions in this block. + pub body: Vec, + /// Ommers/uncles header. + pub ommers: Vec
, + /// Block withdrawals. + pub withdrawals: Option>, +} + +impl OpBlock { + /// Converts the [OpBlock] to an [L2BlockInfo], by checking against the L1 + /// information transaction or the genesis block. + pub fn to_l2_block_ref( + &self, + rollup_config: &RollupConfig, + ) -> Result { + let (l1_origin, sequence_number) = if self.header.number == rollup_config.genesis.l2.number + { + if self.header.hash_slow() != rollup_config.genesis.l2.hash { + return Err(OpBlockConversionError::InvalidGenesisHash( + rollup_config.genesis.l2.hash, + self.header.hash_slow(), + )); + } + (rollup_config.genesis.l1, 0) + } else { + if self.body.is_empty() { + return Err(OpBlockConversionError::EmptyTransactions(self.header.hash_slow())); + } + + let OpTxEnvelope::Deposit(ref tx) = self.body[0] else { + return Err(OpBlockConversionError::InvalidTxType(self.body[0].tx_type() as u8)); + }; + + let l1_info = L1BlockInfoTx::decode_calldata(tx.input.as_ref())?; + (l1_info.id(), l1_info.sequence_number()) + }; + + Ok(L2BlockInfo { + block_info: BlockInfo { + hash: self.header.hash_slow(), + number: self.header.number, + parent_hash: self.header.parent_hash, + timestamp: self.header.timestamp, + }, + l1_origin, + seq_num: sequence_number, + }) + } + + /// Converts the [OpBlock] to a partial [SystemConfig]. + pub fn to_system_config( + &self, + rollup_config: &RollupConfig, + ) -> Result { + if self.header.number == rollup_config.genesis.l2.number { + if self.header.hash_slow() != rollup_config.genesis.l2.hash { + return Err(OpBlockConversionError::InvalidGenesisHash( + rollup_config.genesis.l2.hash, + self.header.hash_slow(), + )); + } + return rollup_config + .genesis + .system_config + .ok_or(OpBlockConversionError::MissingSystemConfigGenesis); + } + + if self.body.is_empty() { + return Err(OpBlockConversionError::EmptyTransactions(self.header.hash_slow())); + } + let OpTxEnvelope::Deposit(ref tx) = self.body[0] else { + return Err(OpBlockConversionError::InvalidTxType(self.body[0].tx_type() as u8)); + }; + + let l1_info = L1BlockInfoTx::decode_calldata(tx.input.as_ref())?; + let l1_fee_scalar = match l1_info { + L1BlockInfoTx::Bedrock(L1BlockInfoBedrock { l1_fee_scalar, .. }) => l1_fee_scalar, + L1BlockInfoTx::Ecotone(L1BlockInfoEcotone { + base_fee_scalar, + blob_base_fee_scalar, + .. + }) => { + // Translate Ecotone values back into encoded scalar if needed. + // We do not know if it was derived from a v0 or v1 scalar, + // but v1 is fine, a 0 blob base fee has the same effect. + let mut buf = B256::ZERO; + buf[0] = 0x01; + buf[24..28].copy_from_slice(blob_base_fee_scalar.to_be_bytes().as_ref()); + buf[28..32].copy_from_slice(base_fee_scalar.to_be_bytes().as_ref()); + buf.into() + } + }; + + Ok(SystemConfig { + batcher_address: l1_info.batcher_address(), + overhead: l1_info.l1_fee_overhead(), + scalar: l1_fee_scalar, + gas_limit: self.header.gas_limit as u64, + base_fee_scalar: None, + blob_base_fee_scalar: None, + }) + } +} diff --git a/crates/derive/src/lib.rs b/crates/derive/src/lib.rs index 7bfb6219..8d01260b 100644 --- a/crates/derive/src/lib.rs +++ b/crates/derive/src/lib.rs @@ -10,6 +10,7 @@ extern crate alloc; mod macros; pub mod batch; +pub mod block; pub mod errors; pub mod params; pub mod pipeline; diff --git a/crates/derive/src/online/alloy_providers.rs b/crates/derive/src/online/alloy_providers.rs index 0a6e4caf..e90e1516 100644 --- a/crates/derive/src/online/alloy_providers.rs +++ b/crates/derive/src/online/alloy_providers.rs @@ -9,12 +9,14 @@ use alloy_rlp::{Buf, Decodable}; use alloy_transport::{RpcError, TransportErrorKind, TransportResult}; use async_trait::async_trait; use core::{num::NonZeroUsize, str::FromStr}; -use kona_primitives::{Block, L2ExecutionPayloadEnvelope, OpBlock}; use lru::LruCache; use op_alloy_genesis::{RollupConfig, SystemConfig}; use op_alloy_protocol::{BlockInfo, L2BlockInfo}; -use crate::traits::{ChainProvider, L2ChainProvider}; +use crate::{ + block::{Block, OpBlock}, + traits::{ChainProvider, L2ChainProvider}, +}; const CACHE_SIZE: usize = 16; @@ -270,8 +272,8 @@ pub struct AlloyL2ChainProvider { inner: ReqwestProvider, /// The rollup configuration. rollup_config: Arc, - /// `payload_by_number` LRU cache. - payload_by_number_cache: LruCache, + /// `block_by_number` LRU cache. + block_by_number_cache: LruCache, /// `l2_block_info_by_number` LRU cache. l2_block_info_by_number_cache: LruCache, /// `system_config_by_l2_hash` LRU cache. @@ -284,7 +286,7 @@ impl AlloyL2ChainProvider { Self { inner, rollup_config, - payload_by_number_cache: LruCache::new(NonZeroUsize::new(CACHE_SIZE).unwrap()), + block_by_number_cache: LruCache::new(NonZeroUsize::new(CACHE_SIZE).unwrap()), l2_block_info_by_number_cache: LruCache::new(NonZeroUsize::new(CACHE_SIZE).unwrap()), system_config_by_number_cache: LruCache::new(NonZeroUsize::new(CACHE_SIZE).unwrap()), } @@ -337,18 +339,18 @@ impl L2ChainProvider for AlloyL2ChainProvider { return Ok(*l2_block_info); } - let payload = match self.payload_by_number(number).await { + let block = match self.block_by_number(number).await { Ok(p) => p, Err(e) => { crate::timer!(DISCARD, timer); crate::inc!( PROVIDER_ERRORS, - &["l2_chain_provider", "l2_block_info_by_number", "payload_by_number"] + &["l2_chain_provider", "l2_block_info_by_number", "block_by_number"] ); return Err(e); } }; - let l2_block_info = match payload.to_l2_block_ref(self.rollup_config.as_ref()) { + let l2_block_info = match block.to_l2_block_ref(self.rollup_config.as_ref()) { Ok(b) => b, Err(e) => { crate::timer!(DISCARD, timer); @@ -363,19 +365,16 @@ impl L2ChainProvider for AlloyL2ChainProvider { Ok(l2_block_info) } - async fn payload_by_number( - &mut self, - number: u64, - ) -> Result { - crate::inc!(PROVIDER_CALLS, &["l2_chain_provider", "payload_by_number"]); + async fn block_by_number(&mut self, number: u64) -> Result { + crate::inc!(PROVIDER_CALLS, &["l2_chain_provider", "block_by_number"]); crate::timer!( START, PROVIDER_RESPONSE_TIME, - &["l2_chain_provider", "payload_by_number"], + &["l2_chain_provider", "block_by_number"], timer ); - if let Some(payload) = self.payload_by_number_cache.get(&number) { - return Ok(payload.clone()); + if let Some(block) = self.block_by_number_cache.get(&number) { + return Ok(block.clone()); } let raw_block: TransportResult = @@ -386,7 +385,7 @@ impl L2ChainProvider for AlloyL2ChainProvider { crate::timer!(DISCARD, timer); crate::inc!( PROVIDER_ERRORS, - &["l2_chain_provider", "payload_by_number", "debug_getRawBlock"] + &["l2_chain_provider", "block_by_number", "debug_getRawBlock"] ); return Err(e); } @@ -395,14 +394,12 @@ impl L2ChainProvider for AlloyL2ChainProvider { Ok(b) => b, Err(e) => { crate::timer!(DISCARD, timer); - crate::inc!(PROVIDER_ERRORS, &["l2_chain_provider", "payload_by_number", "decode"]); + crate::inc!(PROVIDER_ERRORS, &["l2_chain_provider", "block_by_number", "decode"]); return Err(RpcError::LocalUsageError(Box::new(e))); } }; - let payload_envelope: L2ExecutionPayloadEnvelope = block.into(); - - self.payload_by_number_cache.put(number, payload_envelope.clone()); - Ok(payload_envelope) + self.block_by_number_cache.put(number, block.clone()); + Ok(block) } async fn system_config_by_number( @@ -421,18 +418,18 @@ impl L2ChainProvider for AlloyL2ChainProvider { return Ok(*system_config); } - let envelope = match self.payload_by_number(number).await { + let block = match self.block_by_number(number).await { Ok(e) => e, Err(e) => { crate::timer!(DISCARD, timer); crate::inc!( PROVIDER_ERRORS, - &["l2_chain_provider", "system_config_by_number", "payload_by_number"] + &["l2_chain_provider", "system_config_by_number", "block_by_number"] ); return Err(e); } }; - let sys_config = match envelope.to_system_config(&rollup_config) { + let sys_config = match block.to_system_config(&rollup_config) { Ok(s) => s, Err(e) => { crate::timer!(DISCARD, timer); diff --git a/crates/derive/src/online/pipeline.rs b/crates/derive/src/online/pipeline.rs index c3586504..9213c01d 100644 --- a/crates/derive/src/online/pipeline.rs +++ b/crates/derive/src/online/pipeline.rs @@ -8,6 +8,7 @@ use super::{ use alloc::sync::Arc; use op_alloy_genesis::RollupConfig; use op_alloy_protocol::BlockInfo; + // Pipeline internal stages aren't re-exported at the module-level. use crate::stages::{ AttributesQueue, BatchQueue, ChannelBank, ChannelReader, FrameQueue, L1Retrieval, L1Traversal, diff --git a/crates/derive/src/stages/batch_queue.rs b/crates/derive/src/stages/batch_queue.rs index 3ab20313..6f80f032 100644 --- a/crates/derive/src/stages/batch_queue.rs +++ b/crates/derive/src/stages/batch_queue.rs @@ -444,6 +444,7 @@ where mod tests { use super::*; use crate::{ + block::OpBlock, stages::{ channel_reader::BatchReader, test_utils::{CollectingLayer, MockBatchQueueProvider, TraceStorage}, @@ -451,11 +452,11 @@ mod tests { traits::test_utils::TestL2ChainProvider, }; use alloc::vec; - use alloy_eips::BlockNumHash; + use alloy_consensus::Header; + use alloy_eips::{eip2718::Decodable2718, BlockNumHash}; use alloy_primitives::{address, b256, Address, Bytes, TxKind, B256, U256}; use alloy_rlp::{BytesMut, Encodable}; - use kona_primitives::{L2ExecutionPayload, L2ExecutionPayloadEnvelope}; - use op_alloy_consensus::{OpTxType, TxDeposit}; + use op_alloy_consensus::{OpTxEnvelope, OpTxType, TxDeposit}; use op_alloy_genesis::ChainGenesis; use op_alloy_protocol::{L1BlockInfoBedrock, L1BlockInfoTx}; use tracing::Level; @@ -604,27 +605,35 @@ mod tests { }, ..Default::default() }; - let payload = L2ExecutionPayloadEnvelope { - parent_beacon_block_root: None, - execution_payload: L2ExecutionPayload { - block_number: 8, - block_hash: payload_block_hash, - transactions: batch_txs, + let batch_txs = batch_txs + .into_iter() + .map(|tx| OpTxEnvelope::decode_2718(&mut &tx[..]).unwrap()) + .collect(); + let second_batch_txs = second_batch_txs + .into_iter() + .map(|tx| OpTxEnvelope::decode_2718(&mut &tx[..]).unwrap()) + .collect(); + let block = OpBlock { + header: Header { + number: 8, + // TODO: fix hash ..Default::default() }, + body: batch_txs, + ..Default::default() }; - let second = L2ExecutionPayloadEnvelope { - parent_beacon_block_root: None, - execution_payload: L2ExecutionPayload { - block_number: 9, - block_hash: payload_block_hash, - transactions: second_batch_txs, + let second = OpBlock { + header: Header { + number: 9, + // TODO: fix hash ..Default::default() }, + body: second_batch_txs, + ..Default::default() }; let fetcher = TestL2ChainProvider { blocks: vec![block_nine, block_seven], - payloads: vec![payload, second], + op_blocks: vec![block, second], ..Default::default() }; let mut bq = BatchQueue::new(cfg, mock, fetcher); diff --git a/crates/derive/src/stages/test_utils/sys_config_fetcher.rs b/crates/derive/src/stages/test_utils/sys_config_fetcher.rs index b11dd054..136d7bc4 100644 --- a/crates/derive/src/stages/test_utils/sys_config_fetcher.rs +++ b/crates/derive/src/stages/test_utils/sys_config_fetcher.rs @@ -1,11 +1,10 @@ //! Implements a mock [L2SystemConfigFetcher] for testing. -use crate::traits::L2ChainProvider; +use crate::{block::OpBlock, traits::L2ChainProvider}; use alloc::{boxed::Box, sync::Arc}; use anyhow::Result; use async_trait::async_trait; use hashbrown::HashMap; -use kona_primitives::L2ExecutionPayloadEnvelope; use op_alloy_genesis::{RollupConfig, SystemConfig}; use op_alloy_protocol::L2BlockInfo; @@ -47,7 +46,7 @@ impl L2ChainProvider for MockSystemConfigL2Fetcher { unimplemented!() } - async fn payload_by_number(&mut self, _: u64) -> Result { + async fn block_by_number(&mut self, _: u64) -> Result { unimplemented!() } } diff --git a/crates/derive/src/traits/providers.rs b/crates/derive/src/traits/providers.rs index 282067ee..cc7f16cc 100644 --- a/crates/derive/src/traits/providers.rs +++ b/crates/derive/src/traits/providers.rs @@ -1,10 +1,10 @@ use core::fmt::Display; +use crate::block::OpBlock; use alloc::{boxed::Box, string::ToString, sync::Arc, vec::Vec}; use alloy_consensus::{Header, Receipt, TxEnvelope}; use alloy_primitives::B256; use async_trait::async_trait; -use kona_primitives::L2ExecutionPayloadEnvelope; use op_alloy_genesis::{RollupConfig, SystemConfig}; use op_alloy_protocol::{BlockInfo, L2BlockInfo}; @@ -42,12 +42,9 @@ pub trait L2ChainProvider { /// Errors if the block does not exist. async fn l2_block_info_by_number(&mut 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( - &mut self, - number: u64, - ) -> Result; + /// Returns the block for a given number. + /// Errors if no block is available for the given block number. + async fn block_by_number(&mut self, number: u64) -> Result; /// Returns the [SystemConfig] by L2 number. async fn system_config_by_number( diff --git a/crates/derive/src/traits/test_utils.rs b/crates/derive/src/traits/test_utils.rs index 2ed33b35..b07666bf 100644 --- a/crates/derive/src/traits/test_utils.rs +++ b/crates/derive/src/traits/test_utils.rs @@ -1,6 +1,7 @@ //! Test Utilities for derive traits use crate::{ + block::OpBlock, errors::{BlobProviderError, PipelineError, PipelineResult}, traits::{ AsyncIterator, BlobProvider, ChainProvider, DataAvailabilityProvider, L2ChainProvider, @@ -14,7 +15,7 @@ use anyhow::Result; use async_trait::async_trait; use core::fmt::Debug; use hashbrown::HashMap; -use kona_primitives::{IndexedBlobHash, L2ExecutionPayloadEnvelope}; +use kona_primitives::IndexedBlobHash; use op_alloy_genesis::{RollupConfig, SystemConfig}; use op_alloy_protocol::{BlockInfo, L2BlockInfo}; @@ -220,8 +221,8 @@ pub struct TestL2ChainProvider { pub blocks: Vec, /// Short circuit the block return to be the first block. pub short_circuit: bool, - /// Payloads - pub payloads: Vec, + /// Blocks + pub op_blocks: Vec, /// System configs pub system_configs: HashMap, } @@ -230,10 +231,10 @@ impl TestL2ChainProvider { /// Creates a new [MockBlockFetcher] with the given origin and batches. pub fn new( blocks: Vec, - payloads: Vec, + op_blocks: Vec, system_configs: HashMap, ) -> Self { - Self { blocks, short_circuit: false, payloads, system_configs } + Self { blocks, short_circuit: false, op_blocks, system_configs } } } @@ -252,12 +253,12 @@ impl L2ChainProvider for TestL2ChainProvider { .ok_or_else(|| anyhow::anyhow!("Block not found")) } - async fn payload_by_number(&mut self, number: u64) -> Result { - self.payloads + async fn block_by_number(&mut self, number: u64) -> Result { + self.op_blocks .iter() - .find(|p| p.execution_payload.block_number == number) + .find(|p| p.header.number == number) .cloned() - .ok_or_else(|| anyhow::anyhow!("Payload not found")) + .ok_or_else(|| anyhow::anyhow!("L2 Block not found")) } async fn system_config_by_number( diff --git a/crates/primitives/Cargo.toml b/crates/primitives/Cargo.toml index e86406fd..381c8e13 100644 --- a/crates/primitives/Cargo.toml +++ b/crates/primitives/Cargo.toml @@ -15,12 +15,7 @@ thiserror.workspace = true # Alloy alloy-eips.workspace = true -alloy-consensus.workspace = true -alloy-rlp = { workspace = true, features = ["derive"] } alloy-primitives = { workspace = true, features = ["rlp"] } -op-alloy-consensus.workspace = true -op-alloy-protocol.workspace = true -op-alloy-genesis.workspace = true # `serde` feature dependencies serde = { workspace = true, optional = true } @@ -36,14 +31,11 @@ serde_json.workspace = true [features] default = ["serde"] -serde = ["dep:serde", "op-alloy-consensus/serde", "op-alloy-protocol/serde", "op-alloy-genesis/serde"] +serde = ["dep:serde"] online = [ "dep:c-kzg", "dep:sha2", "dep:revm", "dep:tracing", "revm/default", - "op-alloy-consensus/std", - "op-alloy-protocol/std", - "op-alloy-genesis/std", ] diff --git a/crates/primitives/src/block.rs b/crates/primitives/src/block.rs deleted file mode 100644 index 1432878c..00000000 --- a/crates/primitives/src/block.rs +++ /dev/null @@ -1,63 +0,0 @@ -//! This module contains the various Block types. - -use alloc::vec::Vec; -use alloy_consensus::{Header, TxEnvelope}; -use alloy_eips::eip4895::Withdrawal; -use alloy_rlp::{RlpDecodable, RlpEncodable}; -use op_alloy_consensus::OpTxEnvelope; - -#[cfg(feature = "serde")] -use serde::{Deserialize, Serialize}; - -/// The Block Kind -/// -/// The block kinds are: -/// - `Earliest`: The earliest known block. -/// - `Latest`: The latest pending block. -/// - `Finalized`: The latest finalized block. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Debug, Clone, Copy, Eq, PartialEq)] -pub enum BlockKind { - /// The earliest known block. - Earliest, - /// The latest pending block. - Latest, - /// The latest finalized block. - Finalized, -} - -/// Ethereum full block. -/// -/// Withdrawals can be optionally included at the end of the RLP encoded message. -/// -/// Taken from [reth-primitives](https://github.com/paradigmxyz/reth) -#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)] -#[rlp(trailing)] -pub struct Block { - /// Block header. - pub header: Header, - /// Transactions in this block. - pub body: Vec, - /// Ommers/uncles header. - pub ommers: Vec
, - /// Block withdrawals. - pub withdrawals: Option>, -} - -/// OP Stack full block. -/// -/// Withdrawals can be optionally included at the end of the RLP encoded message. -/// -/// Taken from [reth-primitives](https://github.com/paradigmxyz/reth) -#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)] -#[rlp(trailing)] -pub struct OpBlock { - /// Block header. - pub header: Header, - /// Transactions in this block. - pub body: Vec, - /// Ommers/uncles header. - pub ommers: Vec
, - /// Block withdrawals. - pub withdrawals: Option>, -} diff --git a/crates/primitives/src/lib.rs b/crates/primitives/src/lib.rs index bf28f003..70737250 100644 --- a/crates/primitives/src/lib.rs +++ b/crates/primitives/src/lib.rs @@ -7,14 +7,6 @@ extern crate alloc; -pub mod block; -pub use block::{Block, BlockKind, OpBlock}; - -pub mod payload; -pub use payload::{ - L2ExecutionPayload, L2ExecutionPayloadEnvelope, PAYLOAD_MEM_FIXED_COST, PAYLOAD_TX_MEM_OVERHEAD, -}; - pub mod blob; pub use blob::{BlobData, BlobDecodingError, IndexedBlobHash}; diff --git a/crates/primitives/src/payload.rs b/crates/primitives/src/payload.rs deleted file mode 100644 index 2b2aab8b..00000000 --- a/crates/primitives/src/payload.rs +++ /dev/null @@ -1,298 +0,0 @@ -//! Contains the execution payload type. - -use alloc::vec::Vec; -use alloy_eips::{ - eip2718::{Decodable2718, Encodable2718}, - eip4895::Withdrawal, -}; -use alloy_primitives::{Address, Bloom, Bytes, B256}; -use alloy_rlp::Encodable; -use op_alloy_consensus::{OpTxEnvelope, OpTxType}; -use op_alloy_genesis::{RollupConfig, SystemConfig}; -use op_alloy_protocol::{ - block_info::DecodeError, BlockInfo, L1BlockInfoBedrock, L1BlockInfoEcotone, L1BlockInfoTx, - L2BlockInfo, -}; -use thiserror::Error; - -/// Fixed and variable memory costs for a payload. -/// ~1000 bytes per payload, with some margin for overhead like map data. -pub const PAYLOAD_MEM_FIXED_COST: u64 = 1000; - -/// Memory overhead per payload transaction. -/// 24 bytes per tx overhead (size of slice header in memory). -pub const PAYLOAD_TX_MEM_OVERHEAD: u64 = 24; - -use super::OpBlock; - -#[cfg(feature = "serde")] -use serde::{Deserialize, Serialize}; - -/// Envelope wrapping the [L2ExecutionPayload]. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct L2ExecutionPayloadEnvelope { - /// Parent beacon block root. - #[cfg_attr(feature = "serde", serde(rename = "parentBeaconBlockRoot"))] - pub parent_beacon_block_root: Option, - /// The inner execution payload. - #[cfg_attr(feature = "serde", serde(rename = "executionPayload"))] - pub execution_payload: L2ExecutionPayload, -} - -impl L2ExecutionPayloadEnvelope { - /// Returns the payload memory size. - pub fn mem_size(&self) -> u64 { - let mut out = PAYLOAD_MEM_FIXED_COST; - for tx in &self.execution_payload.transactions { - out += tx.len() as u64 + PAYLOAD_TX_MEM_OVERHEAD; - } - out - } -} - -/// The execution payload. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Debug, Default, Clone, PartialEq, Eq)] -pub struct L2ExecutionPayload { - /// The parent hash. - #[cfg_attr(feature = "serde", serde(rename = "parentHash"))] - pub parent_hash: B256, - /// The coinbase address. - #[cfg_attr(feature = "serde", serde(rename = "feeRecipient"))] - pub fee_recipient: Address, - /// The state root. - #[cfg_attr(feature = "serde", serde(rename = "stateRoot"))] - pub state_root: B256, - /// The transactions root. - #[cfg_attr(feature = "serde", serde(rename = "receiptsRoot"))] - pub receipts_root: B256, - /// The logs bloom. - #[cfg_attr(feature = "serde", serde(rename = "logsBloom"))] - pub logs_bloom: Bloom, - /// The mix hash. - #[cfg_attr(feature = "serde", serde(rename = "prevRandao"))] - pub prev_randao: B256, - /// The difficulty. - #[cfg_attr(feature = "serde", serde(rename = "blockNumber"))] - pub block_number: u64, - /// The gas limit. - #[cfg_attr(feature = "serde", serde(rename = "gasLimit"))] - pub gas_limit: u128, - /// The gas used. - #[cfg_attr(feature = "serde", serde(rename = "gasUsed"))] - pub gas_used: u128, - /// The timestamp. - #[cfg_attr(feature = "serde", serde(rename = "timestamp"))] - pub timestamp: u64, - /// The extra data. - #[cfg_attr(feature = "serde", serde(rename = "extraData"))] - pub extra_data: Bytes, - /// Base fee per gas. - #[cfg_attr( - feature = "serde", - serde(rename = "baseFeePerGas", skip_serializing_if = "Option::is_none") - )] - pub base_fee_per_gas: Option, - /// Block hash. - #[cfg_attr(feature = "serde", serde(rename = "blockHash"))] - pub block_hash: B256, - /// The transactions. - #[cfg_attr(feature = "serde", serde(rename = "transactions"))] - pub transactions: Vec, - /// The deserialized transactions. - #[cfg_attr(feature = "serde", serde(skip))] - pub deserialized_transactions: Vec, - /// The withdrawals. - #[cfg_attr( - feature = "serde", - serde(rename = "withdrawals", skip_serializing_if = "Option::is_none") - )] - pub withdrawals: Option>, - /// The blob gas used. - #[cfg_attr( - feature = "serde", - serde(rename = "blobGasUsed", skip_serializing_if = "Option::is_none") - )] - pub blob_gas_used: Option, - /// The excess blob gas. - #[cfg_attr( - feature = "serde", - serde(rename = "excessBlobGas", skip_serializing_if = "Option::is_none") - )] - pub excess_blob_gas: Option, -} - -/// An error encountered during [L2ExecutionPayloadEnvelope] conversion. -#[derive(Error, Debug)] -pub enum PayloadConversionError { - /// Invalid genesis hash. - #[error("Invalid genesis hash. Expected {0}, got {1}")] - InvalidGenesisHash(B256, B256), - /// Invalid transaction type. - #[error("First payload transaction has unexpected type: {0}")] - InvalidTxType(u8), - /// L1 Info error - #[error(transparent)] - L1InfoError(#[from] DecodeError), - /// EIP 2718 RLP error - #[error("EIP 2718 RLP error: {0}")] - Eip2718Error(alloy_eips::eip2718::Eip2718Error), - /// Missing system config in genesis block. - #[error("Missing system config in genesis block")] - MissingSystemConfigGenesis, - /// Empty transactions. - #[error("Empty transactions in payload. Block hash: {0}")] - EmptyTransactions(B256), -} - -impl L2ExecutionPayloadEnvelope { - /// Converts the [L2ExecutionPayloadEnvelope] to an [L2BlockInfo], by checking against the L1 - /// information transaction or the genesis block. - pub fn to_l2_block_ref( - &self, - rollup_config: &RollupConfig, - ) -> Result { - let L2ExecutionPayloadEnvelope { execution_payload, .. } = self; - - let (l1_origin, sequence_number) = if execution_payload.block_number == - rollup_config.genesis.l2.number - { - if execution_payload.block_hash != rollup_config.genesis.l2.hash { - return Err(PayloadConversionError::InvalidGenesisHash( - rollup_config.genesis.l2.hash, - execution_payload.block_hash, - )); - } - (rollup_config.genesis.l1, 0) - } else { - if execution_payload.transactions.is_empty() { - return Err(PayloadConversionError::EmptyTransactions(execution_payload.block_hash)); - } - - let ty = execution_payload.transactions[0][0]; - if ty != OpTxType::Deposit as u8 { - return Err(PayloadConversionError::InvalidTxType(ty)); - } - let tx = OpTxEnvelope::decode_2718(&mut execution_payload.transactions[0].as_ref()) - .map_err(PayloadConversionError::Eip2718Error)?; - - let OpTxEnvelope::Deposit(tx) = tx else { - return Err(PayloadConversionError::InvalidTxType(tx.tx_type() as u8)); - }; - - let l1_info = L1BlockInfoTx::decode_calldata(tx.input.as_ref())?; - (l1_info.id(), l1_info.sequence_number()) - }; - - Ok(L2BlockInfo { - block_info: BlockInfo { - hash: execution_payload.block_hash, - number: execution_payload.block_number, - parent_hash: execution_payload.parent_hash, - timestamp: execution_payload.timestamp, - }, - l1_origin, - seq_num: sequence_number, - }) - } - - /// Converts the [L2ExecutionPayloadEnvelope] to a partial [SystemConfig]. - pub fn to_system_config( - &self, - rollup_config: &RollupConfig, - ) -> Result { - let L2ExecutionPayloadEnvelope { execution_payload, .. } = self; - - if execution_payload.block_number == rollup_config.genesis.l2.number { - if execution_payload.block_hash != rollup_config.genesis.l2.hash { - return Err(PayloadConversionError::InvalidGenesisHash( - rollup_config.genesis.l2.hash, - execution_payload.block_hash, - )); - } - return rollup_config - .genesis - .system_config - .ok_or(PayloadConversionError::MissingSystemConfigGenesis); - } - - if execution_payload.transactions.is_empty() { - return Err(PayloadConversionError::EmptyTransactions(execution_payload.block_hash)); - } - let ty = execution_payload.transactions[0][0]; - if ty != OpTxType::Deposit as u8 { - return Err(PayloadConversionError::InvalidTxType(ty)); - } - let tx = OpTxEnvelope::decode_2718(&mut execution_payload.transactions[0].as_ref()) - .map_err(PayloadConversionError::Eip2718Error)?; - - let OpTxEnvelope::Deposit(tx) = tx else { - return Err(PayloadConversionError::InvalidTxType(tx.tx_type() as u8)); - }; - - let l1_info = L1BlockInfoTx::decode_calldata(tx.input.as_ref())?; - let l1_fee_scalar = match l1_info { - L1BlockInfoTx::Bedrock(L1BlockInfoBedrock { l1_fee_scalar, .. }) => l1_fee_scalar, - L1BlockInfoTx::Ecotone(L1BlockInfoEcotone { - base_fee_scalar, - blob_base_fee_scalar, - .. - }) => { - // Translate Ecotone values back into encoded scalar if needed. - // We do not know if it was derived from a v0 or v1 scalar, - // but v1 is fine, a 0 blob base fee has the same effect. - let mut buf = B256::ZERO; - buf[0] = 0x01; - buf[24..28].copy_from_slice(blob_base_fee_scalar.to_be_bytes().as_ref()); - buf[28..32].copy_from_slice(base_fee_scalar.to_be_bytes().as_ref()); - buf.into() - } - }; - - Ok(SystemConfig { - batcher_address: l1_info.batcher_address(), - overhead: l1_info.l1_fee_overhead(), - scalar: l1_fee_scalar, - gas_limit: execution_payload.gas_limit as u64, - base_fee_scalar: None, - blob_base_fee_scalar: None, - }) - } -} - -impl From for L2ExecutionPayloadEnvelope { - fn from(block: OpBlock) -> Self { - let OpBlock { header, body, withdrawals, .. } = block; - Self { - execution_payload: L2ExecutionPayload { - parent_hash: header.parent_hash, - fee_recipient: header.beneficiary, - state_root: header.state_root, - receipts_root: header.receipts_root, - logs_bloom: header.logs_bloom, - prev_randao: header.difficulty.into(), - block_number: header.number, - gas_limit: header.gas_limit, - gas_used: header.gas_used, - timestamp: header.timestamp, - extra_data: header.extra_data.clone(), - base_fee_per_gas: header.base_fee_per_gas, - block_hash: header.hash_slow(), - deserialized_transactions: body.clone(), - transactions: body - .into_iter() - .map(|tx| { - let mut buf = Vec::with_capacity(tx.length()); - tx.encode_2718(&mut buf); - buf.into() - }) - .collect(), - withdrawals, - blob_gas_used: header.blob_gas_used, - excess_blob_gas: header.excess_blob_gas, - }, - parent_beacon_block_root: header.parent_beacon_block_root, - } - } -}