From d38ee5932da6c9d667f2267646f81e839b1fe3c3 Mon Sep 17 00:00:00 2001
From: Ivan Litteri <67517699+ilitteri@users.noreply.github.com>
Date: Thu, 20 Mar 2025 16:48:16 +0000
Subject: [PATCH 01/29] refactor(l2): add flag for setting sponsor private key
 (#2281)

**Motivation**

<!-- Why does this pull request exist? What are its goals? -->

The current implementation requires a `.env` file to exist and the
execution panics if this file does not exist. Nevertheless, this has a
purpose of being. As this feature should be used in `l2` it is assumed
that there's a `.env` file and that is ok because it should. This PR
intends to add a second path for setting the sponsor pk without needing
a `.env`.

**Description**

Add a flag `--sponsor-private-key` as a second option for setting this
value.
---
 cmd/ethrex/Cargo.toml      |  2 +-
 cmd/ethrex/cli.rs          |  6 ++++++
 cmd/ethrex/initializers.rs | 10 ++++++++--
 cmd/ethrex/utils.rs        | 15 +++++++++++++++
 4 files changed, 30 insertions(+), 3 deletions(-)

diff --git a/cmd/ethrex/Cargo.toml b/cmd/ethrex/Cargo.toml
index f9dac59fc..79ae1b121 100644
--- a/cmd/ethrex/Cargo.toml
+++ b/cmd/ethrex/Cargo.toml
@@ -20,7 +20,7 @@ hex.workspace = true
 tracing.workspace = true
 tracing-subscriber.workspace = true
 k256.workspace = true
-clap = { version = "4.3", features = ["derive"] }
+clap = { version = "4.3", features = ["derive", "env"] }
 clap_complete = "4.5.17"
 eyre = "0.6.12"
 directories = "5.0.1"
diff --git a/cmd/ethrex/cli.rs b/cmd/ethrex/cli.rs
index 3678234c9..c3c29a032 100644
--- a/cmd/ethrex/cli.rs
+++ b/cmd/ethrex/cli.rs
@@ -14,6 +14,9 @@ use crate::{
     DEFAULT_DATADIR,
 };
 
+#[cfg(feature = "l2")]
+use secp256k1::SecretKey;
+
 pub const VERSION_STRING: &str = env!("CARGO_PKG_VERSION");
 
 #[allow(clippy::upper_case_acronyms)]
@@ -180,6 +183,7 @@ impl Default for Options {
     }
 }
 
+#[cfg(feature = "l2")]
 #[derive(ClapParser)]
 pub struct L2Options {
     #[arg(
@@ -189,6 +193,8 @@ pub struct L2Options {
         help_heading = "L2 options"
     )]
     pub sponsorable_addresses_file_path: Option<String>,
+    #[arg(long, value_parser = utils::parse_private_key, env = "SPONSOR_PRIVATE_KEY", help = "The private key of ethrex L2 transactions sponsor.", help_heading = "L2 options")]
+    pub sponsor_private_key: Option<SecretKey>,
 }
 
 #[cfg(feature = "based")]
diff --git a/cmd/ethrex/initializers.rs b/cmd/ethrex/initializers.rs
index 6c00ce41b..2390e518f 100644
--- a/cmd/ethrex/initializers.rs
+++ b/cmd/ethrex/initializers.rs
@@ -127,7 +127,7 @@ pub fn init_rpc_api(
         #[cfg(feature = "l2")]
         get_valid_delegation_addresses(l2_opts),
         #[cfg(feature = "l2")]
-        get_sponsor_pk(),
+        get_sponsor_pk(l2_opts),
     )
     .into_future();
 
@@ -360,7 +360,13 @@ pub fn get_valid_delegation_addresses(l2_opts: &L2Options) -> Vec<Address> {
 }
 
 #[cfg(feature = "l2")]
-pub fn get_sponsor_pk() -> SecretKey {
+pub fn get_sponsor_pk(opts: &L2Options) -> SecretKey {
+    if let Some(pk) = opts.sponsor_private_key {
+        return pk;
+    }
+
+    warn!("Sponsor private key not provided. Trying to read from the .env file.");
+
     if let Err(e) = read_env_file() {
         panic!("Failed to read .env file: {e}");
     }
diff --git a/cmd/ethrex/utils.rs b/cmd/ethrex/utils.rs
index cde665f3a..fe683a882 100644
--- a/cmd/ethrex/utils.rs
+++ b/cmd/ethrex/utils.rs
@@ -5,6 +5,9 @@ use ethrex_common::types::{Block, Genesis};
 use ethrex_p2p::{kademlia::KademliaTable, sync::SyncMode, types::Node};
 use ethrex_rlp::decode::RLPDecode;
 use ethrex_vm::backends::EvmEngine;
+use hex::FromHexError;
+#[cfg(feature = "l2")]
+use secp256k1::SecretKey;
 use std::{
     fs::File,
     io,
@@ -118,3 +121,15 @@ pub fn read_known_peers(file_path: PathBuf) -> Result<Vec<Node>, serde_json::Err
 
     serde_json::from_reader(file)
 }
+
+#[cfg(feature = "l2")]
+pub fn parse_private_key(s: &str) -> eyre::Result<SecretKey> {
+    Ok(SecretKey::from_slice(&parse_hex(s)?)?)
+}
+
+pub fn parse_hex(s: &str) -> eyre::Result<Bytes, FromHexError> {
+    match s.strip_prefix("0x") {
+        Some(s) => hex::decode(s).map(Into::into),
+        None => hex::decode(s).map(Into::into),
+    }
+}

From d964a2fece5ad19273b02aa5081b6a85609437dc Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Manuel=20I=C3=B1aki=20Bilbao?=
 <manuel.bilbao@lambdaclass.com>
Date: Thu, 20 Mar 2025 15:21:32 -0300
Subject: [PATCH 02/29] chore(core): add `rust-toolchain.toml` (#2278)

**Motivation**

<!-- Why does this pull request exist? What are its goals? -->
We have a pinned version of Rust in the CI and also in `.tool-versions`
(for `asdf`) but not for `rustup`. We encountered ourselves running
different versions of Rust, with different results, specially when
running tools like Clippy

**Description**

<!-- A clear and concise general description of the changes this PR
introduces -->
Added a `rust-toolchain.toml` file with the pinned version of Rust so
it's evaluated by default when using `rustup`. As a side effect, needed
to change the way Pico CLI is installed in the CI.
---
 .github/workflows/ci_l2_prover_nightly.yaml | 10 ++--------
 rust-toolchain.toml                         |  3 +++
 2 files changed, 5 insertions(+), 8 deletions(-)
 create mode 100644 rust-toolchain.toml

diff --git a/.github/workflows/ci_l2_prover_nightly.yaml b/.github/workflows/ci_l2_prover_nightly.yaml
index c8fefb757..1d3d8bc58 100644
--- a/.github/workflows/ci_l2_prover_nightly.yaml
+++ b/.github/workflows/ci_l2_prover_nightly.yaml
@@ -35,10 +35,7 @@ jobs:
         uses: Swatinem/rust-cache@v2
       # https://pico-docs.brevis.network/getting-started/installation
       - name: Install pico-cli
-        uses: baptiste0928/cargo-install@v3
-        with:
-          crate: pico-cli
-          git: https://github.com/brevis-network/pico
+        run: cargo +nightly install --git https://github.com/brevis-network/pico pico-cli
       - name: ${{ matrix.action.command }} Command
         run: cargo +nightly-2024-11-27 ${{ matrix.action.command }} ${{ matrix.action.args }}
 
@@ -56,10 +53,7 @@ jobs:
           components: rust-src
       # https://pico-docs.brevis.network/getting-started/installation
       - name: Install pico-cli
-        uses: baptiste0928/cargo-install@v3
-        with:
-          crate: pico-cli
-          git: https://github.com/brevis-network/pico
+        run: cargo +nightly install --git https://github.com/brevis-network/pico pico-cli
       - name: Build
         run: |
           cd crates/l2/prover
diff --git a/rust-toolchain.toml b/rust-toolchain.toml
new file mode 100644
index 000000000..5f3ff1776
--- /dev/null
+++ b/rust-toolchain.toml
@@ -0,0 +1,3 @@
+[toolchain]
+channel = "1.82.0"
+profile = "default"

From 31dd81a4a8a26640c365a1eb58180f98d4f663c2 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Avila=20Gast=C3=B3n?=
 <72628438+avilagaston9@users.noreply.github.com>
Date: Thu, 20 Mar 2025 15:25:53 -0300
Subject: [PATCH 03/29] fix(l1): enable CORS for rpc endpoints (#2275)

**Motivation**

To be used with different applications

**Description**

Adds a permissive CORS layer using
[axum](https://docs.rs/axum/latest/axum/middleware/index.html) +
[tower-http](https://docs.rs/tower-http/0.6.2/tower_http/cors/index.html).
- All request headers allowed.
- All methods allowed.
- All origins allowed.
- All headers exposed.

Closes None
---
 Cargo.lock                       | 15 +++++++++++++++
 crates/networking/rpc/Cargo.toml |  1 +
 crates/networking/rpc/rpc.rs     |  8 ++++++++
 3 files changed, 24 insertions(+)

diff --git a/Cargo.lock b/Cargo.lock
index b2a5372ac..4abb97957 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -3445,6 +3445,7 @@ dependencies = [
  "thiserror 2.0.11",
  "tokio",
  "tokio-util",
+ "tower-http",
  "tracing",
  "tracing-subscriber 0.3.19",
 ]
@@ -9994,6 +9995,20 @@ dependencies = [
  "tracing",
 ]
 
+[[package]]
+name = "tower-http"
+version = "0.6.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "403fa3b783d4b626a8ad51d766ab03cb6d2dbfc46b1c5d4448395e6628dc9697"
+dependencies = [
+ "bitflags 2.8.0",
+ "bytes",
+ "http",
+ "pin-project-lite",
+ "tower-layer",
+ "tower-service",
+]
+
 [[package]]
 name = "tower-layer"
 version = "0.3.3"
diff --git a/crates/networking/rpc/Cargo.toml b/crates/networking/rpc/Cargo.toml
index c41f40bec..d38fcdb2e 100644
--- a/crates/networking/rpc/Cargo.toml
+++ b/crates/networking/rpc/Cargo.toml
@@ -7,6 +7,7 @@ edition = "2021"
 
 [dependencies]
 axum.workspace = true
+tower-http = { version = "0.6.2", features = ["cors"] }
 serde = { version = "1.0.203", features = ["derive"] }
 serde_json = "1.0.117"
 tokio.workspace = true
diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs
index a1fd733ce..d9a489b42 100644
--- a/crates/networking/rpc/rpc.rs
+++ b/crates/networking/rpc/rpc.rs
@@ -48,6 +48,7 @@ use std::{
     time::Duration,
 };
 use tokio::{net::TcpListener, sync::Mutex as TokioMutex};
+use tower_http::cors::CorsLayer;
 use tracing::info;
 use types::transaction::SendRawTransactionRequest;
 use utils::{
@@ -210,8 +211,15 @@ pub async fn start_api(
         }
     });
 
+    // All request headers allowed.
+    // All methods allowed.
+    // All origins allowed.
+    // All headers exposed.
+    let cors = CorsLayer::permissive();
+
     let http_router = Router::new()
         .route("/", post(handle_http_request))
+        .layer(cors)
         .with_state(service_context.clone());
     let http_listener = TcpListener::bind(http_addr).await.unwrap();
 

From 4f7024cdd9997138bb88ddb94f5721d0343ad95c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Thu, 20 Mar 2025 16:04:14 -0300
Subject: [PATCH 04/29] fix(l2): make TCP connection async (#2280)

**Motivation**

The prover server-client TCP connection uses blocking primitive from the
standard library, so whenever one of the processes is expecting a
connection they don't yield control to the runtime and all other
processes get blocked (because tokio's scheduler is cooperative).

This PR replaces these primitives with tokio's async ones.

Closes #1983
Closes #2019
---
 crates/l2/prover/src/prover_client.rs | 43 +++++++++++++----------
 crates/l2/sequencer/errors.rs         |  2 ++
 crates/l2/sequencer/prover_server.rs  | 49 +++++++++++++++------------
 3 files changed, 56 insertions(+), 38 deletions(-)

diff --git a/crates/l2/prover/src/prover_client.rs b/crates/l2/prover/src/prover_client.rs
index 3af47cdf5..d39861655 100644
--- a/crates/l2/prover/src/prover_client.rs
+++ b/crates/l2/prover/src/prover_client.rs
@@ -3,12 +3,12 @@ use ethrex_l2::{
     sequencer::prover_server::ProofData,
     utils::{config::prover_client::ProverClientConfig, prover::proving_systems::ProofCalldata},
 };
-use std::{
-    io::{BufReader, BufWriter},
+use std::time::Duration;
+use tokio::{
+    io::{AsyncReadExt, AsyncWriteExt},
     net::TcpStream,
-    time::Duration,
+    time::sleep,
 };
-use tokio::time::sleep;
 use tracing::{debug, error, info, warn};
 use zkvm_interface::io::ProgramInput;
 
@@ -38,14 +38,15 @@ impl ProverClient {
     pub async fn start(&self) {
         // Build the prover depending on the prover_type passed as argument.
         loop {
-            match self.request_new_input() {
+            match self.request_new_input().await {
                 // If we get the input
                 Ok(prover_data) => {
                     // Generate the Proof
                     match prove(prover_data.input).and_then(to_calldata) {
                         Ok(proving_output) => {
-                            if let Err(e) =
-                                self.submit_proof(prover_data.block_number, proving_output)
+                            if let Err(e) = self
+                                .submit_proof(prover_data.block_number, proving_output)
+                                .await
                             {
                                 // TODO: Retry?
                                 warn!("Failed to submit proof: {e}");
@@ -63,10 +64,11 @@ impl ProverClient {
         }
     }
 
-    fn request_new_input(&self) -> Result<ProverData, String> {
+    async fn request_new_input(&self) -> Result<ProverData, String> {
         // Request the input with the correct block_number
         let request = ProofData::request();
         let response = connect_to_prover_server_wr(&self.prover_server_endpoint, &request)
+            .await
             .map_err(|e| format!("Failed to get Response: {e}"))?;
 
         match response {
@@ -95,10 +97,15 @@ impl ProverClient {
         }
     }
 
-    fn submit_proof(&self, block_number: u64, proving_output: ProofCalldata) -> Result<(), String> {
+    async fn submit_proof(
+        &self,
+        block_number: u64,
+        proving_output: ProofCalldata,
+    ) -> Result<(), String> {
         let submit = ProofData::submit(block_number, proving_output);
 
         let submit_ack = connect_to_prover_server_wr(&self.prover_server_endpoint, &submit)
+            .await
             .map_err(|e| format!("Failed to get SubmitAck: {e}"))?;
 
         match submit_ack {
@@ -111,17 +118,19 @@ impl ProverClient {
     }
 }
 
-fn connect_to_prover_server_wr(
+async fn connect_to_prover_server_wr(
     addr: &str,
     write: &ProofData,
 ) -> Result<ProofData, Box<dyn std::error::Error>> {
-    let stream = TcpStream::connect(addr)?;
-    let buf_writer = BufWriter::new(&stream);
+    let mut stream = TcpStream::connect(addr).await?;
     debug!("Connection established!");
-    serde_json::ser::to_writer(buf_writer, write)?;
-    stream.shutdown(std::net::Shutdown::Write)?;
 
-    let buf_reader = BufReader::new(&stream);
-    let response: ProofData = serde_json::de::from_reader(buf_reader)?;
-    Ok(response)
+    stream.write_all(&serde_json::to_vec(&write)?).await?;
+    stream.shutdown().await?;
+
+    let mut buffer = Vec::new();
+    stream.read_to_end(&mut buffer).await?;
+
+    let response: Result<ProofData, _> = serde_json::from_slice(&buffer);
+    Ok(response?)
 }
diff --git a/crates/l2/sequencer/errors.rs b/crates/l2/sequencer/errors.rs
index 9832b0cea..a5d21be35 100644
--- a/crates/l2/sequencer/errors.rs
+++ b/crates/l2/sequencer/errors.rs
@@ -56,6 +56,8 @@ pub enum ProverServerError {
     SaveStateError(#[from] SaveStateError),
     #[error("Failed to encode calldata: {0}")]
     CalldataEncodeError(#[from] CalldataEncodeError),
+    #[error("ProverServer failed when (de)serializing JSON: {0}")]
+    JsonError(#[from] serde_json::Error),
 }
 
 #[derive(Debug, thiserror::Error)]
diff --git a/crates/l2/sequencer/prover_server.rs b/crates/l2/sequencer/prover_server.rs
index 373d15f27..b3663d5e6 100644
--- a/crates/l2/sequencer/prover_server.rs
+++ b/crates/l2/sequencer/prover_server.rs
@@ -26,11 +26,12 @@ use secp256k1::SecretKey;
 use serde::{Deserialize, Serialize};
 use std::{
     fmt::Debug,
-    io::{BufReader, BufWriter, Write},
-    net::{IpAddr, Shutdown, TcpListener, TcpStream},
+    net::IpAddr,
     sync::mpsc::{self, Receiver},
     time::Duration,
 };
+use tokio::io::{AsyncReadExt, AsyncWriteExt};
+use tokio::net::{TcpListener, TcpStream};
 use tokio::{
     signal::unix::{signal, SignalKind},
     time::sleep,
@@ -207,21 +208,23 @@ impl ProverServer {
         let mut sigint = signal(SignalKind::interrupt())?;
         sigint.recv().await.ok_or(SigIntError::Recv)?;
         tx.send(()).map_err(SigIntError::Send)?;
-        TcpStream::connect(format!("{}:{}", config.listen_ip, config.listen_port))?
-            .shutdown(Shutdown::Both)
+        TcpStream::connect(format!("{}:{}", config.listen_ip, config.listen_port))
+            .await?
+            .shutdown()
+            .await
             .map_err(SigIntError::Shutdown)?;
 
         Ok(())
     }
 
     pub async fn start(&mut self, rx: Receiver<()>) -> Result<(), ProverServerError> {
-        let listener = TcpListener::bind(format!("{}:{}", self.ip, self.port))?;
+        let listener = TcpListener::bind(format!("{}:{}", self.ip, self.port)).await?;
 
         info!("Starting TCP server at {}:{}", self.ip, self.port);
 
-        for stream in listener.incoming() {
-            match stream {
-                Ok(stream) => {
+        loop {
+            match listener.accept().await {
+                Ok((stream, _)) => {
                     debug!("Connection established!");
 
                     if let Ok(()) = rx.try_recv() {
@@ -242,7 +245,8 @@ impl ProverServer {
     }
 
     async fn handle_connection(&mut self, mut stream: TcpStream) -> Result<(), ProverServerError> {
-        let buf_reader = BufReader::new(&stream);
+        let mut buffer = Vec::new();
+        stream.read_to_end(&mut buffer).await?;
 
         let last_verified_block =
             EthClient::get_last_verified_block(&self.eth_client, self.on_chain_proposer_address)
@@ -274,11 +278,11 @@ impl ProverServer {
             tx_submitted = true;
         }
 
-        let data: Result<ProofData, _> = serde_json::de::from_reader(buf_reader);
+        let data: Result<ProofData, _> = serde_json::from_slice(&buffer);
         match data {
             Ok(ProofData::Request) => {
                 if let Err(e) = self
-                    .handle_request(&stream, block_to_verify, tx_submitted)
+                    .handle_request(&mut stream, block_to_verify, tx_submitted)
                     .await
                 {
                     warn!("Failed to handle request: {e}");
@@ -288,7 +292,7 @@ impl ProverServer {
                 block_number,
                 calldata,
             }) => {
-                self.handle_submit(&mut stream, block_number)?;
+                self.handle_submit(&mut stream, block_number).await?;
 
                 // Avoid storing a proof of a future block_number
                 // CHECK: maybe we would like to store all the proofs given the case in which
@@ -338,7 +342,7 @@ impl ProverServer {
 
     async fn handle_request(
         &self,
-        stream: &TcpStream,
+        stream: &mut TcpStream,
         block_number: u64,
         tx_submitted: bool,
     ) -> Result<(), ProverServerError> {
@@ -361,12 +365,15 @@ impl ProverServer {
             response
         };
 
-        let writer = BufWriter::new(stream);
-        serde_json::to_writer(writer, &response)
-            .map_err(|e| ProverServerError::ConnectionError(e.into()))
+        let buffer = serde_json::to_vec(&response)?;
+        stream
+            .write_all(&buffer)
+            .await
+            .map_err(ProverServerError::ConnectionError)?;
+        Ok(())
     }
 
-    fn handle_submit(
+    async fn handle_submit(
         &self,
         stream: &mut TcpStream,
         block_number: u64,
@@ -374,12 +381,12 @@ impl ProverServer {
         debug!("Submit received for BlockNumber: {block_number}");
 
         let response = ProofData::submit_ack(block_number);
-        let json_string = serde_json::to_string(&response)
-            .map_err(|e| ProverServerError::Custom(format!("serde_json::to_string(): {e}")))?;
+
+        let buffer = serde_json::to_vec(&response)?;
         stream
-            .write_all(json_string.as_bytes())
+            .write_all(&buffer)
+            .await
             .map_err(ProverServerError::ConnectionError)?;
-
         Ok(())
     }
 

From 862fb49e6143e5bdc1f3aa8939a95dff4038e5f2 Mon Sep 17 00:00:00 2001
From: fmoletta <99273364+fmoletta@users.noreply.github.com>
Date: Fri, 21 Mar 2025 10:40:23 -0300
Subject: [PATCH 05/29] fix(l1): fix unending storage healer process in snap
 sync (#2287)

**Motivation**
There is currently a bug in snap sync. When a state sync becomes stale,
the snap sync cycle is aborted but the storage healer process is left
hanging instead if signaling it to end and waiting for it to finish. The
loop condition of the storage healer is also not properly set, keeping
it alive even after the end signal if it still has paths to heal. This
PR fixes both of this problems
<!-- Why does this pull request exist? What are its goals? -->

**Description**
* Fix loop condition in storage healer
* End storage healer if state sync aborts due to stale pivot
<!-- A clear and concise general description of the changes this PR
introduces -->

<!-- Link to issues: Resolves #111, Resolves #222 -->

Closes #issue_number
---
 crates/networking/p2p/sync.rs                 | 2 ++
 crates/networking/p2p/sync/storage_healing.rs | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)

diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs
index 682614e6a..4f53c687a 100644
--- a/crates/networking/p2p/sync.rs
+++ b/crates/networking/p2p/sync.rs
@@ -529,6 +529,8 @@ impl SyncManager {
             .await?;
             if stale_pivot {
                 warn!("Stale Pivot, aborting state sync");
+                storage_healer_sender.send(vec![]).await?;
+                storage_healer_handler.await??;
                 return Ok(false);
             }
         }
diff --git a/crates/networking/p2p/sync/storage_healing.rs b/crates/networking/p2p/sync/storage_healing.rs
index 16cf830de..d414f9ffe 100644
--- a/crates/networking/p2p/sync/storage_healing.rs
+++ b/crates/networking/p2p/sync/storage_healing.rs
@@ -36,7 +36,7 @@ pub(crate) async fn storage_healer(
     // alive until the end signal so we don't lose queued messages
     let mut stale = false;
     let mut incoming = true;
-    while incoming || !pending_paths.is_empty() {
+    while incoming {
         // If we have enough pending storages to fill a batch
         // or if we have no more incoming batches, spawn a fetch process
         // If the pivot became stale don't process anything and just save incoming requests

From ab751f0470192a2120b27f9ef207ff5e06c4676f Mon Sep 17 00:00:00 2001
From: fmoletta <99273364+fmoletta@users.noreply.github.com>
Date: Fri, 21 Mar 2025 11:54:55 -0300
Subject: [PATCH 06/29] feat(l1): write multiple account's storage batches in
 the same db txn (#2270)

**Motivation**
When measuring time taken by each task during snap sync I noticed that a
lot of time was spent writing the storage ranges obtained from peers to
the DB snapshot. It would take anywhere from 3 to over 10 seconds to
write all the ranges to the DB (around 300 storage ranges per request).
This PR modifies the insertion logic to write all 300 ranges in the same
DB transaction, reducing the time taken to write all the ranges to the
DB to 10 milliseconds or less
<!-- Why does this pull request exist? What are its goals? -->

**Description**
* Add `write_storage_snapshot_batches` method to `Store`, which can
write multiple batches from different accounts on the same txn
* Write all storage ranges received from peers in a single DB txn using
the method above on the storage fetcher (snap sync)
<!-- A clear and concise general description of the changes this PR
introduces -->

<!-- Link to issues: Resolves #111, Resolves #222 -->

Closes: None, but helps speed up snap sync
---
 crates/networking/p2p/sync/storage_fetcher.rs | 10 ++++----
 crates/storage/api.rs                         |  8 +++++++
 crates/storage/store.rs                       | 11 +++++++++
 crates/storage/store_db/in_memory.rs          | 18 ++++++++++++++
 crates/storage/store_db/libmdbx.rs            | 22 +++++++++++++++++
 crates/storage/store_db/redb.rs               | 24 +++++++++++++++++++
 6 files changed, 87 insertions(+), 6 deletions(-)

diff --git a/crates/networking/p2p/sync/storage_fetcher.rs b/crates/networking/p2p/sync/storage_fetcher.rs
index b925b3248..eb6fb07bd 100644
--- a/crates/networking/p2p/sync/storage_fetcher.rs
+++ b/crates/networking/p2p/sync/storage_fetcher.rs
@@ -151,12 +151,10 @@ async fn fetch_storage_batch(
             // The incomplete range is not the first, we cannot asume it is a large trie, so lets add it back to the queue
         }
         // Store the storage ranges & rebuild the storage trie for each account
-        for (keys, values) in keys.into_iter().zip(values.into_iter()) {
-            let (account_hash, expected_root) = batch.remove(0);
-            // Write storage to snapshot
-            store.write_snapshot_storage_batch(account_hash, keys, values)?;
-            complete_storages.push((account_hash, expected_root));
-        }
+        let filled_storages: Vec<(H256, H256)> = batch.drain(..values.len()).collect();
+        let account_hashes: Vec<H256> = filled_storages.iter().map(|(hash, _)| *hash).collect();
+        complete_storages.extend(filled_storages);
+        store.write_snapshot_storage_batches(account_hashes, keys, values)?;
         // Send complete storages to the rebuilder
         storage_trie_rebuilder_sender
             .send(complete_storages)
diff --git a/crates/storage/api.rs b/crates/storage/api.rs
index 8630ee7c3..ab6f302aa 100644
--- a/crates/storage/api.rs
+++ b/crates/storage/api.rs
@@ -271,6 +271,14 @@ pub trait StoreEngine: Debug + Send + Sync + RefUnwindSafe {
         storage_values: Vec<U256>,
     ) -> Result<(), StoreError>;
 
+    /// Write multiple storage batches belonging to different accounts into the current storage snapshot
+    fn write_snapshot_storage_batches(
+        &self,
+        account_hashes: Vec<H256>,
+        storage_keys: Vec<Vec<H256>>,
+        storage_values: Vec<Vec<U256>>,
+    ) -> Result<(), StoreError>;
+
     /// Set the latest root of the rebuilt state trie and the last downloaded hashes from each segment
     fn set_state_trie_rebuild_checkpoint(
         &self,
diff --git a/crates/storage/store.rs b/crates/storage/store.rs
index 92007f222..48c68a3cb 100644
--- a/crates/storage/store.rs
+++ b/crates/storage/store.rs
@@ -957,6 +957,17 @@ impl Store {
             .write_snapshot_storage_batch(account_hash, storage_keys, storage_values)
     }
 
+    /// Write multiple storage batches belonging to different accounts into the current storage snapshot
+    pub fn write_snapshot_storage_batches(
+        &self,
+        account_hashes: Vec<H256>,
+        storage_keys: Vec<Vec<H256>>,
+        storage_values: Vec<Vec<U256>>,
+    ) -> Result<(), StoreError> {
+        self.engine
+            .write_snapshot_storage_batches(account_hashes, storage_keys, storage_values)
+    }
+
     /// Clears all checkpoint data created during the last snap sync
     pub fn clear_snap_state(&self) -> Result<(), StoreError> {
         self.engine.clear_snap_state()
diff --git a/crates/storage/store_db/in_memory.rs b/crates/storage/store_db/in_memory.rs
index 2dbfff95e..589922064 100644
--- a/crates/storage/store_db/in_memory.rs
+++ b/crates/storage/store_db/in_memory.rs
@@ -480,6 +480,24 @@ impl StoreEngine for Store {
             .extend(storage_keys.into_iter().zip(storage_values));
         Ok(())
     }
+    fn write_snapshot_storage_batches(
+        &self,
+        account_hashes: Vec<H256>,
+        storage_keys: Vec<Vec<H256>>,
+        storage_values: Vec<Vec<U256>>,
+    ) -> Result<(), StoreError> {
+        for (account_hash, (storage_keys, storage_values)) in account_hashes
+            .into_iter()
+            .zip(storage_keys.into_iter().zip(storage_values.into_iter()))
+        {
+            self.inner()
+                .storage_snapshot
+                .entry(account_hash)
+                .or_default()
+                .extend(storage_keys.into_iter().zip(storage_values));
+        }
+        Ok(())
+    }
 
     fn set_state_trie_rebuild_checkpoint(
         &self,
diff --git a/crates/storage/store_db/libmdbx.rs b/crates/storage/store_db/libmdbx.rs
index 365e575ef..3d13bb034 100644
--- a/crates/storage/store_db/libmdbx.rs
+++ b/crates/storage/store_db/libmdbx.rs
@@ -595,6 +595,28 @@ impl StoreEngine for Store {
         txn.commit().map_err(StoreError::LibmdbxError)
     }
 
+    fn write_snapshot_storage_batches(
+        &self,
+        account_hashes: Vec<H256>,
+        storage_keys: Vec<Vec<H256>>,
+        storage_values: Vec<Vec<U256>>,
+    ) -> Result<(), StoreError> {
+        let txn = self
+            .db
+            .begin_readwrite()
+            .map_err(StoreError::LibmdbxError)?;
+        for (account_hash, (storage_keys, storage_values)) in account_hashes
+            .into_iter()
+            .zip(storage_keys.into_iter().zip(storage_values.into_iter()))
+        {
+            for (key, value) in storage_keys.into_iter().zip(storage_values.into_iter()) {
+                txn.upsert::<StorageSnapShot>(account_hash.into(), (key.into(), value.into()))
+                    .map_err(StoreError::LibmdbxError)?;
+            }
+        }
+        txn.commit().map_err(StoreError::LibmdbxError)
+    }
+
     fn set_state_trie_rebuild_checkpoint(
         &self,
         checkpoint: (H256, [H256; STATE_TRIE_SEGMENTS]),
diff --git a/crates/storage/store_db/redb.rs b/crates/storage/store_db/redb.rs
index 29d803d2e..b4e06fcc0 100644
--- a/crates/storage/store_db/redb.rs
+++ b/crates/storage/store_db/redb.rs
@@ -775,6 +775,30 @@ impl StoreEngine for RedBStore {
         write_tx.commit()?;
         Ok(())
     }
+    fn write_snapshot_storage_batches(
+        &self,
+        account_hashes: Vec<H256>,
+        storage_keys: Vec<Vec<H256>>,
+        storage_values: Vec<Vec<U256>>,
+    ) -> Result<(), StoreError> {
+        let write_tx = self.db.begin_write()?;
+        {
+            let mut table = write_tx.open_multimap_table(STORAGE_SNAPSHOT_TABLE)?;
+            for (account_hash, (storage_keys, storage_values)) in account_hashes
+                .into_iter()
+                .zip(storage_keys.into_iter().zip(storage_values.into_iter()))
+            {
+                for (key, value) in storage_keys.into_iter().zip(storage_values.into_iter()) {
+                    table.insert(
+                        <H256 as Into<AccountHashRLP>>::into(account_hash),
+                        (key.0, value.to_big_endian()),
+                    )?;
+                }
+            }
+        }
+        write_tx.commit()?;
+        Ok(())
+    }
 
     fn set_state_trie_rebuild_checkpoint(
         &self,

From c753e2c1ce96049b72e30ddd0b751983ef036fed Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Thu, 20 Mar 2025 15:42:50 -0300
Subject: [PATCH 07/29] reintroduce exec proof

---
 crates/l2/prover/Dockerfile          | 59 +++++++++++++++++++
 crates/l2/sequencer/prover_server.rs | 87 ++++++++++++++++------------
 2 files changed, 110 insertions(+), 36 deletions(-)
 create mode 100644 crates/l2/prover/Dockerfile

diff --git a/crates/l2/prover/Dockerfile b/crates/l2/prover/Dockerfile
new file mode 100644
index 000000000..af780c00f
--- /dev/null
+++ b/crates/l2/prover/Dockerfile
@@ -0,0 +1,59 @@
+ARG RUSTFLAGS
+ARG PROVER
+
+FROM rust:1.82 AS chef
+
+RUN apt-get update && apt-get install -y \
+  curl \
+  git \
+  pkg-config \
+  clang \
+  build-essential \
+  libclang-dev \
+  libc6 \
+  libssl-dev \
+  ca-certificates \
+  && rm -rf /var/lib/apt/lists/*
+RUN cargo install cargo-chef
+RUN curl -L https://sp1.succinct.xyz | bash && ~/.sp1/bin/sp1up --version 4.1.0
+
+ENV PATH="/root/.sp1/bin:$PATH"
+ENV PATH="/root/.cargo/bin:$PATH"
+
+RUN cargo prove install-toolchain
+
+WORKDIR /ethrex
+
+FROM chef AS planner
+ENV RUSTFLAGS=$RUSTFLAGS
+
+COPY . .
+# Determine the crates that need to be built from dependencies
+RUN cargo chef prepare --bin ethrex_prover --recipe-path recipe.json
+
+FROM chef AS builder
+ENV RUSTFLAGS=$RUSTFLAGS
+COPY --from=planner /ethrex/recipe.json recipe.json
+# Build dependencies only, these remained cached
+RUN cargo chef cook --release --recipe-path recipe.json --manifest-path crates/l2/prover/Cargo.toml --bin ethrex_prover --features sp1
+
+COPY . .
+RUN cargo build --release --manifest-path crates/l2/prover/Cargo.toml --features sp1
+
+FROM --platform=${BUILDPLATFORM} ubuntu:24.04
+ENV RUSTFLAGS=$RUSTFLAGS
+WORKDIR /usr/local/bin
+
+# install basic packages
+RUN apt-get update && apt-get -y install curl git
+
+# install rust
+RUN curl https://sh.rustup.rs -sSf | bash -s -- -y
+ENV PATH="/root/.cargo/bin:${PATH}"
+
+# Check cargo is visible
+RUN cargo --help
+
+COPY --from=builder ethrex/target/release/ethrex_prover .
+
+ENTRYPOINT [ "./ethrex_prover" ]
diff --git a/crates/l2/sequencer/prover_server.rs b/crates/l2/sequencer/prover_server.rs
index 8d0cd2eb6..ee2b1abb9 100644
--- a/crates/l2/sequencer/prover_server.rs
+++ b/crates/l2/sequencer/prover_server.rs
@@ -155,27 +155,30 @@ impl ProverServer {
         .await?;
 
         let mut needed_proof_types = vec![];
-
-        for (key, addr) in verifier_contracts {
-            if addr == DEV_MODE_ADDRESS {
-                continue;
-            } else {
-                match key.as_str() {
-                    R0VERIFIER => {
-                        info!("RISC0 proof needed");
-                        needed_proof_types.push(ProverType::RISC0);
-                    }
-                    SP1VERIFIER => {
-                        info!("SP1 proof needed");
-                        needed_proof_types.push(ProverType::SP1);
-                    }
-                    PICOVERIFIER => {
-                        info!("PICO proof needed");
-                        needed_proof_types.push(ProverType::Pico);
+        if !config.dev_mode {
+            for (key, addr) in verifier_contracts {
+                if addr == DEV_MODE_ADDRESS {
+                    continue;
+                } else {
+                    match key.as_str() {
+                        "R0VERIFIER()" => {
+                            info!("RISC0 proof needed");
+                            needed_proof_types.push(ProverType::RISC0);
+                        }
+                        "SP1VERIFIER()" => {
+                            info!("SP1 proof needed");
+                            needed_proof_types.push(ProverType::SP1);
+                        }
+                        "PICOVERIFIER()" => {
+                            info!("PICO proof needed");
+                            needed_proof_types.push(ProverType::Pico);
+                        }
+                        _ => unreachable!("There shouldn't be a value different than the used backends/verifiers R0VERIFIER|SP1VERIFER|PICOVERIFIER."),
                     }
-                    _ => unreachable!("There shouldn't be a value different than the used backends/verifiers R0VERIFIER|SP1VERIFER|PICOVERIFIER."),
                 }
             }
+        } else {
+            needed_proof_types.push(ProverType::Exec);
         }
 
         Ok(Self {
@@ -280,6 +283,7 @@ impl ProverServer {
                     error!("Failed to accept connection: {}", e);
                 }
             }
+            warn!("end incoming");
         }
         Ok(())
     }
@@ -360,26 +364,23 @@ impl ProverServer {
                     return Ok(());
                 }
 
-                // The ProverType::Exec doesn't generate a real proof, we don't need to save it.
-                if calldata.prover_type != ProverType::Exec {
-                    // Check if we have the proof for that ProverType
-                    let has_proof = match block_number_has_state_file(
-                        StateFileType::Proof(calldata.prover_type),
-                        block_number,
-                    ) {
-                        Ok(has_proof) => has_proof,
-                        Err(e) => {
-                            let error = format!("{e}");
-                            if !error.contains("No such file or directory") {
-                                return Err(e.into());
-                            }
-                            false
+                // Check if we have the proof for that ProverType
+                let has_proof = match block_number_has_state_file(
+                    StateFileType::Proof(calldata.prover_type),
+                    block_number,
+                ) {
+                    Ok(has_proof) => has_proof,
+                    Err(e) => {
+                        let error = format!("{e}");
+                        if !error.contains("No such file or directory") {
+                            return Err(e.into());
                         }
-                    };
-                    // If we don't have it, insert it.
-                    if !has_proof {
-                        write_state(block_number, &StateType::Proof(calldata))?;
+                        false
                     }
+                };
+                // If we don't have it, insert it.
+                if !has_proof {
+                    write_state(block_number, &StateType::Proof(calldata))?;
                 }
 
                 // Then if we have all the proofs, we send the transaction in the next `handle_connection` call.
@@ -485,6 +486,20 @@ impl ProverServer {
         // the structure has to match the one defined in the OnChainProposer.sol contract.
         // It may cause some issues, but the ethrex_prover_lib cannot be imported,
         // this approach is straight-forward for now.
+        let exec_proof = {
+            if self.needed_proof_types.contains(&ProverType::Exec) {
+                let exec_proof = read_proof(block_number, StateFileType::Proof(ProverType::Exec))?;
+                if exec_proof.prover_type != ProverType::Exec {
+                    return Err(ProverServerError::Custom(
+                        "exec Proof isn't present".to_string(),
+                    ));
+                }
+                exec_proof.calldata
+            } else {
+                ProverType::Exec.empty_calldata()
+            }
+        };
+
         let risc0_proof = {
             if self.needed_proof_types.contains(&ProverType::RISC0) {
                 let risc0_proof =

From 7c5a9ac94f585be224a31e2feadeb3f8d8e8f889 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 14:58:34 -0300
Subject: [PATCH 08/29] add test-tmux as integration test

---
 .github/workflows/ci_l2.yaml | 2 +-
 crates/l2/Makefile           | 5 +++++
 2 files changed, 6 insertions(+), 1 deletion(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index fffa32337..f7b4f8a58 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -35,4 +35,4 @@ jobs:
         run: |
           cd crates/l2
           cp config_example.toml config.toml
-          make ci_test
+          make test-tmux
diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index f5e8be9ac..a833796a3 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -180,6 +180,11 @@ ci_test: ## 🚧 Runs the L2's integration test, used by the github's CI
 test: ## 🚧 Runs the L2's integration test, run `make init` and in a new terminal make test
 	CONFIG_FILE=config.toml cargo test l2 --release -- --nocapture --test-threads=1 || (echo "The tests have failed.\n Is the L2 running? To start it, run:\n make rm-db-l1; make rm-db-l2; make restart" ; exit 1)
 
+test-tmux: down rm-db-l1 rm-db-l2 init-local-l1 deploy-l1 update-system-contracts
+	tmux new-session -d -s 'ethrex-l2' make init-l2 \; set -w remain-on-exit on
+	tmux split-window -h 'make init-prover;bash -i'
+	tmux split-window -v 'cargo test l2 --release -- --nocapture --test-threads=1'
+	tmux attach
 
 # Purge L2's state
 UNAME_S:=$(shell uname -s)

From b334a9f355f3380f2e9c1c89f726c45de0323e23 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 15:11:31 -0300
Subject: [PATCH 09/29] Revert "add test-tmux as integration test"

This reverts commit 208fe3a7fb4c26921ffa757eadc8d27d84d830ce.
---
 .github/workflows/ci_l2.yaml | 2 +-
 crates/l2/Makefile           | 5 -----
 2 files changed, 1 insertion(+), 6 deletions(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index f7b4f8a58..fffa32337 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -35,4 +35,4 @@ jobs:
         run: |
           cd crates/l2
           cp config_example.toml config.toml
-          make test-tmux
+          make ci_test
diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index a833796a3..f5e8be9ac 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -180,11 +180,6 @@ ci_test: ## 🚧 Runs the L2's integration test, used by the github's CI
 test: ## 🚧 Runs the L2's integration test, run `make init` and in a new terminal make test
 	CONFIG_FILE=config.toml cargo test l2 --release -- --nocapture --test-threads=1 || (echo "The tests have failed.\n Is the L2 running? To start it, run:\n make rm-db-l1; make rm-db-l2; make restart" ; exit 1)
 
-test-tmux: down rm-db-l1 rm-db-l2 init-local-l1 deploy-l1 update-system-contracts
-	tmux new-session -d -s 'ethrex-l2' make init-l2 \; set -w remain-on-exit on
-	tmux split-window -h 'make init-prover;bash -i'
-	tmux split-window -v 'cargo test l2 --release -- --nocapture --test-threads=1'
-	tmux attach
 
 # Purge L2's state
 UNAME_S:=$(shell uname -s)

From 0d1cfa2fb9468caee0335bcd7a91d5d06365fe9b Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 15:25:56 -0300
Subject: [PATCH 10/29] modify integration test, add build-prover

---
 .github/workflows/ci_l2.yaml |  8 +++++++-
 crates/l2/Makefile           | 16 +++++++++++-----
 2 files changed, 18 insertions(+), 6 deletions(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index fffa32337..70a1e5033 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -31,8 +31,14 @@ jobs:
       - name: Caching
         uses: Swatinem/rust-cache@v2
 
+      - name: Build prover
+        run: |
+          cd crates/l2
+          PROVER=sp1 make build-prover
+
       - name: Run L2 integration test
         run: |
+          target/release
           cd crates/l2
           cp config_example.toml config.toml
-          make ci_test
+          ../../target/release/ethrex_prover & make ci_test
diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index f5e8be9ac..bfb4a1769 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -143,11 +143,11 @@ down-l2: ## πŸ›‘ Shuts down the L2 Lambda ethrex Client
 
 restart-l2: down-l2 rm-db-l2 init-l2 ## πŸ”„ Restarts the L2 Lambda ethrex Client
 
-init-prover: ## πŸš€ Initializes the Prover
-	@if [ -z "$$T" ]; then \
-		echo "Error: ProverType (T) is missing."; \
+build-prover:
+	@if [ -z "$$PROVER" ]; then \
+		echo "Error: ProverType (PROVER) is missing."; \
 		echo "Please provide it as an argument:"; \
-		echo "make init-prover T=<prover_type: (risc0, sp1)> <G=true>."; \
+		echo "make init-prover PROVER=<prover_type: (risc0, sp1, pico, exec)> <G=true>"; \
 		echo "The prover can also be run with GPU (G)"; \
 		exit 1; \
 	fi; \
@@ -157,7 +157,8 @@ init-prover: ## πŸš€ Initializes the Prover
 	else \
 		GPU=",gpu"; \
 	fi; \
-	cargo run --release --features "$$T$$GPU,l2" --manifest-path ./prover/Cargo.toml --bin ethrex_prover -- $$T
+
+	RUSTFLAGS='-C target-cpu=native' cargo build --release --features "$$PROVER$$GPU,l2" --manifest-path ./prover/Cargo.toml --bin ethrex_prover; \
 
 rm-db-l2: ## πŸ›‘ Removes the DB used by the L2
 	cargo run --release --manifest-path ../../Cargo.toml --bin ethrex -- removedb --datadir ${ethrex_L2_DEV_LIBMDBX}
@@ -180,6 +181,11 @@ ci_test: ## 🚧 Runs the L2's integration test, used by the github's CI
 test: ## 🚧 Runs the L2's integration test, run `make init` and in a new terminal make test
 	CONFIG_FILE=config.toml cargo test l2 --release -- --nocapture --test-threads=1 || (echo "The tests have failed.\n Is the L2 running? To start it, run:\n make rm-db-l1; make rm-db-l2; make restart" ; exit 1)
 
+test-tmux: down rm-db-l1 rm-db-l2 init-local-l1 deploy-l1 update-system-contracts
+	tmux new-session -d -s 'ethrex-l2' make init-l2 \; set -w remain-on-exit on
+	tmux split-window -h 'make init-prover;bash -i'
+	tmux split-window -v 'cargo test l2 --release -- --nocapture --test-threads=1'
+	tmux attach
 
 # Purge L2's state
 UNAME_S:=$(shell uname -s)

From e7ab83e304e933ef8970400415f91e3d18ebbd77 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 15:48:48 -0300
Subject: [PATCH 11/29] add toolchain install to job

---
 .github/workflows/ci_l2.yaml | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index 70a1e5033..018db91aa 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -31,6 +31,11 @@ jobs:
       - name: Caching
         uses: Swatinem/rust-cache@v2
 
+      - name: RISC-V SP1 toolchain install
+        run: |
+          curl -L https://sp1.succinct.xyz | bash
+          ~/.sp1/bin/sp1up --version 4.1.0
+
       - name: Build prover
         run: |
           cd crates/l2

From d481d7f17c8843b51651e5ba46390f5444498998 Mon Sep 17 00:00:00 2001
From: Ivan Litteri <67517699+ilitteri@users.noreply.github.com>
Date: Fri, 21 Mar 2025 19:42:06 +0000
Subject: [PATCH 12/29] feat(l2): `restart-testnet` target (#2293)

**Motivation**

Having a target for restarting the L2 deployment on a testnet.
---
 crates/l2/Makefile | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index 46dd36cb1..7e2283d94 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -1,6 +1,7 @@
 .PHONY: help init down clean restart cli update-cli-contracts init-local-l1 init-l1 down-local-l1 \
 		restart-local-l1 rm-db-l1 clean-contract-deps restart-contract-deps deploy-l1 init-l2 down-l2 \
-		restart-l2 init-prover rm-db-l2 purge_prover_state ci_test test init-testnet deploy-l1-testnet
+		restart-l2 init-prover rm-db-l2 purge_prover_state ci_test test init-testnet deploy-l1-testnet \
+		restart-testnet
 
 .DEFAULT_GOAL := help
 
@@ -22,6 +23,10 @@ clean: clean-contract-deps ## 🧹 Cleans the localnet
 
 restart: restart-local-l1 deploy-l1 restart-l2 ## πŸ”„ Restarts the localnet
 
+## Same as restart but for testnet deployment. The local database is cleaned and the contracts are deployed again.
+restart-testnet:
+	down-l2 down-metrics rm-db-l2 deploy-l1-testnet update-system-contracts init-l2
+
 # CLI
 cli: ## πŸ› οΈ Installs the L2 Lambda ethrex CLI
 	cd ${ethrex_PATH}/cmd/ethrex_l2 && \

From a52148f65c0164594fb852b0daea2bc2b1e841d5 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 16:45:05 -0300
Subject: [PATCH 13/29] fix job

---
 .github/workflows/ci_l2.yaml | 1 -
 1 file changed, 1 deletion(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index 018db91aa..b8194dae4 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -43,7 +43,6 @@ jobs:
 
       - name: Run L2 integration test
         run: |
-          target/release
           cd crates/l2
           cp config_example.toml config.toml
           ../../target/release/ethrex_prover & make ci_test

From d8c0352bb884998ae7e2fcbe6913672ebffbdfd1 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Fri, 21 Mar 2025 17:18:21 -0300
Subject: [PATCH 14/29] disable dev mode and deploy sp1

---
 crates/l2/config_example.toml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/crates/l2/config_example.toml b/crates/l2/config_example.toml
index c71f5921d..67f13662e 100644
--- a/crates/l2/config_example.toml
+++ b/crates/l2/config_example.toml
@@ -13,7 +13,7 @@ risc0_contract_verifier = "0x00000000000000000000000000000000000000AA"
 # If set to 0xAA skip proof verification.
 # Only use in dev mode.
 sp1_contract_verifier = "0x00000000000000000000000000000000000000AA"
-sp1_deploy_verifier = false
+sp1_deploy_verifier = true
 # If set to false, the salt will be randomized.
 salt_is_zero = true
 
@@ -54,5 +54,5 @@ listen_port = 3900
 # The prover_server is in charge of verifying the zkProofs.
 verifier_address = "0xE25583099BA105D9ec0A67f5Ae86D90e50036425"
 verifier_private_key = "0x39725efee3fb28614de3bacaffe4cc4bd8c436257e2c8bb887c4b5c4be45e76d"
-dev_mode = true
+dev_mode = false
 dev_interval_ms = 5000

From 9b0c70f3121eac4dcf86a3fd62220281cfa697cc Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Manuel=20I=C3=B1aki=20Bilbao?=
 <manuel.bilbao@lambdaclass.com>
Date: Fri, 21 Mar 2025 17:42:37 -0300
Subject: [PATCH 15/29] feat(l2): add state reconstruction command (#2204)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

**Motivation**

<!-- Why does this pull request exist? What are its goals? -->
We need a way to reconstruct the chain state in case of a failure or
even if someone want to _trustlessly_ access the state.

**Description**

<!-- A clear and concise general description of the changes this PR
introduces -->
This PR introduces a new ethrex_l2 CLI command, `stack reconstruct`,
that takes downloaded blobs from L1 and reconstruct the blocks based on
its info, storing the state in a Libmdbx store. The blobs can be
downloaded using the `stack blobs-saver` command.
At this stage, the command is able to successfully reconstruct the chain
state and continue to produce blocks.
Note that, as we send state diffs and not transactions to L1, some data
(i.e., transactions history, receipts) will not be accessible in a
reconstructed network.

<!-- Link to issues: Resolves #111, Resolves #222 -->

Closes #1103

---------

Co-authored-by: Avila GastΓ³n <72628438+avilagaston9@users.noreply.github.com>
---
 Cargo.lock                                    |   1 +
 cmd/ethrex_l2/src/commands/stack.rs           |  77 +++-
 crates/blockchain/fork_choice.rs              |  29 +-
 crates/common/types/blobs_bundle.rs           |  17 +-
 crates/common/types/constants.rs              |   3 +
 crates/l2/Cargo.toml                          |   1 +
 .../l2/contracts/src/l1/OnChainProposer.sol   |  13 +-
 .../src/l1/interfaces/IOnChainProposer.sol    |   5 -
 crates/l2/docs/state_diffs.md                 |  12 +-
 crates/l2/sequencer/errors.rs                 |   3 +
 crates/l2/sequencer/l1_committer.rs           |   9 +-
 crates/l2/sequencer/state_diff.rs             | 347 ++++++++++++++++--
 crates/networking/rpc/clients/eth/mod.rs      |  12 -
 crates/networking/rpc/engine/fork_choice.rs   |  31 +-
 crates/storage/store.rs                       |  43 ++-
 15 files changed, 501 insertions(+), 102 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 4abb97957..ea2fe0f6c 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -3279,6 +3279,7 @@ dependencies = [
  "ethrex-rpc",
  "ethrex-sdk",
  "ethrex-storage",
+ "ethrex-trie",
  "ethrex-vm",
  "hex",
  "jsonwebtoken",
diff --git a/cmd/ethrex_l2/src/commands/stack.rs b/cmd/ethrex_l2/src/commands/stack.rs
index 02995c037..83d793607 100644
--- a/cmd/ethrex_l2/src/commands/stack.rs
+++ b/cmd/ethrex_l2/src/commands/stack.rs
@@ -1,16 +1,22 @@
 use crate::{config::EthrexL2Config, utils::config::confirm};
 use clap::Subcommand;
-use ethrex_common::{Address, U256};
+use ethrex_common::{
+    types::{bytes_from_blob, BlockHeader, BYTES_PER_BLOB},
+    Address, U256,
+};
+use ethrex_l2::sequencer::state_diff::StateDiff;
 use ethrex_rpc::{
     clients::{beacon::BeaconClient, eth::BlockByNumber},
     EthClient,
 };
+use ethrex_storage::{EngineType, Store};
 use eyre::{ContextCompat, OptionExt};
+use itertools::Itertools;
 use keccak_hash::keccak;
 use reqwest::Url;
 use secp256k1::SecretKey;
 use std::{
-    fs::create_dir_all,
+    fs::{create_dir_all, read_dir},
     path::{Path, PathBuf},
     thread::sleep,
     time::Duration,
@@ -93,6 +99,17 @@ pub(crate) enum Command {
         #[clap(short = 'b', long)]
         l1_beacon_rpc: Url,
     },
+    #[clap(about = "Reconstructs the L2 state from L1 blobs.")]
+    Reconstruct {
+        #[clap(short = 'g', long, help = "The genesis file for the L2 network.")]
+        genesis: PathBuf,
+        #[clap(short = 'b', long, help = "The directory to read the blobs from.")]
+        blobs_dir: PathBuf,
+        #[clap(short = 's', long, help = "The path to the store.")]
+        store_path: PathBuf,
+        #[clap(short = 'c', long, help = "Address of the L2 proposer coinbase")]
+        coinbase: Address,
+    },
 }
 
 impl Command {
@@ -277,6 +294,62 @@ impl Command {
                     current_block += U256::one();
                 }
             }
+            Command::Reconstruct {
+                genesis,
+                blobs_dir,
+                store_path,
+                coinbase,
+            } => {
+                let store = Store::new_from_genesis(
+                    store_path.to_str().expect("Invalid store path"),
+                    EngineType::Libmdbx,
+                    genesis.to_str().expect("Invalid genesis path"),
+                )?;
+
+                let genesis_header = store.get_block_header(0)?.expect("Genesis block not found");
+                let genesis_block_hash = genesis_header.compute_block_hash();
+
+                let mut new_trie = store
+                    .state_trie(genesis_block_hash)?
+                    .expect("Cannot open state trie");
+                let mut last_number = 0;
+                let mut last_hash = genesis_block_hash;
+
+                let files: Vec<std::fs::DirEntry> = read_dir(blobs_dir)?.try_collect()?;
+                for file in files.into_iter().sorted_by_key(|f| f.file_name()) {
+                    let blob = std::fs::read(file.path())?;
+
+                    if blob.len() != BYTES_PER_BLOB {
+                        panic!("Invalid blob size");
+                    }
+
+                    let blob = bytes_from_blob(blob.into());
+                    let state_diff = StateDiff::decode(&blob)?;
+                    let account_updates = state_diff.to_account_updates(&new_trie)?;
+
+                    new_trie = store
+                        .apply_account_updates_from_trie(new_trie, &account_updates)
+                        .expect("Error applying account updates");
+
+                    let new_block = BlockHeader {
+                        coinbase,
+                        number: last_number + 1,
+                        parent_hash: last_hash,
+                        state_root: new_trie.hash().expect("Error committing state"),
+                        ..state_diff.header
+                    };
+                    let new_block_hash = new_block.compute_block_hash();
+
+                    store.add_block_header(new_block_hash, new_block)?;
+                    store.add_block_number(new_block_hash, last_number + 1)?;
+                    store.set_canonical_block(last_number + 1, new_block_hash)?;
+
+                    last_number += 1;
+                    last_hash = new_block_hash;
+                }
+
+                store.update_latest_block_number(last_number)?;
+            }
         }
         Ok(())
     }
diff --git a/crates/blockchain/fork_choice.rs b/crates/blockchain/fork_choice.rs
index 0bf65b6f3..01a23fe12 100644
--- a/crates/blockchain/fork_choice.rs
+++ b/crates/blockchain/fork_choice.rs
@@ -1,5 +1,5 @@
 use ethrex_common::{
-    types::{Block, BlockHash, BlockHeader, BlockNumber},
+    types::{BlockHash, BlockHeader, BlockNumber},
     H256,
 };
 use ethrex_storage::{error::StoreError, Store};
@@ -29,18 +29,18 @@ pub fn apply_fork_choice(
     }
 
     let finalized_res = if !finalized_hash.is_zero() {
-        store.get_block_by_hash(finalized_hash)?
+        store.get_block_header_by_hash(finalized_hash)?
     } else {
         None
     };
 
     let safe_res = if !safe_hash.is_zero() {
-        store.get_block_by_hash(safe_hash)?
+        store.get_block_header_by_hash(safe_hash)?
     } else {
         None
     };
 
-    let head_res = store.get_block_by_hash(head_hash)?;
+    let head_res = store.get_block_header_by_hash(head_hash)?;
 
     if !safe_hash.is_zero() {
         check_order(&safe_res, &head_res)?;
@@ -50,12 +50,10 @@ pub fn apply_fork_choice(
         check_order(&finalized_res, &safe_res)?;
     }
 
-    let Some(head_block) = head_res else {
+    let Some(head) = head_res else {
         return Err(InvalidForkChoice::Syncing);
     };
 
-    let head = head_block.header;
-
     let latest = store.get_latest_block_number()?;
 
     // If the head block is an already present head ancestor, skip the update.
@@ -77,8 +75,7 @@ pub fn apply_fork_choice(
     };
 
     // Check that finalized and safe blocks are part of the new canonical chain.
-    if let Some(ref finalized_block) = finalized_res {
-        let finalized = &finalized_block.header;
+    if let Some(ref finalized) = finalized_res {
         if !((is_canonical(store, finalized.number, finalized_hash)?
             && finalized.number <= link_block_number)
             || (finalized.number == head.number && finalized_hash == head_hash)
@@ -91,8 +88,7 @@ pub fn apply_fork_choice(
         };
     }
 
-    if let Some(ref safe_block) = safe_res {
-        let safe = &safe_block.header;
+    if let Some(ref safe) = safe_res {
         if !((is_canonical(store, safe.number, safe_hash)? && safe.number <= link_block_number)
             || (safe.number == head.number && safe_hash == head_hash)
             || new_canonical_blocks.contains(&(safe.number, safe_hash)))
@@ -119,10 +115,10 @@ pub fn apply_fork_choice(
     // Make head canonical and label all special blocks correctly.
     store.set_canonical_block(head.number, head_hash)?;
     if let Some(finalized) = finalized_res {
-        store.update_finalized_block_number(finalized.header.number)?;
+        store.update_finalized_block_number(finalized.number)?;
     }
     if let Some(safe) = safe_res {
-        store.update_safe_block_number(safe.header.number)?;
+        store.update_safe_block_number(safe.number)?;
     }
     store.update_latest_block_number(head.number)?;
     store.update_sync_status(true)?;
@@ -131,14 +127,17 @@ pub fn apply_fork_choice(
 }
 
 // Checks that block 1 is prior to block 2 and that if the second is present, the first one is too.
-fn check_order(block_1: &Option<Block>, block_2: &Option<Block>) -> Result<(), InvalidForkChoice> {
+fn check_order(
+    block_1: &Option<BlockHeader>,
+    block_2: &Option<BlockHeader>,
+) -> Result<(), InvalidForkChoice> {
     // We don't need to perform the check if the hashes are null
     match (block_1, block_2) {
         (None, Some(_)) => Err(InvalidForkChoice::ElementNotFound(
             error::ForkChoiceElement::Finalized,
         )),
         (Some(b1), Some(b2)) => {
-            if b1.header.number > b2.header.number {
+            if b1.number > b2.number {
                 Err(InvalidForkChoice::Unordered)
             } else {
                 Ok(())
diff --git a/crates/common/types/blobs_bundle.rs b/crates/common/types/blobs_bundle.rs
index 0aeb0af47..4acbdd1f6 100644
--- a/crates/common/types/blobs_bundle.rs
+++ b/crates/common/types/blobs_bundle.rs
@@ -19,7 +19,7 @@ use ethrex_rlp::{
 };
 use serde::{Deserialize, Serialize};
 
-use super::BYTES_PER_BLOB;
+use super::{BYTES_PER_BLOB, SAFE_BYTES_PER_BLOB};
 
 pub type Bytes48 = [u8; 48];
 pub type Blob = [u8; BYTES_PER_BLOB];
@@ -47,7 +47,7 @@ pub fn blob_from_bytes(bytes: Bytes) -> Result<Blob, BlobsBundleError> {
     // This functions moved from `l2/utils/eth_client/transaction.rs`
     // We set the first byte of every 32-bytes chunk to 0x00
     // so it's always under the field module.
-    if bytes.len() > BYTES_PER_BLOB * 31 / 32 {
+    if bytes.len() > SAFE_BYTES_PER_BLOB {
         return Err(BlobsBundleError::BlobDataInvalidBytesLength);
     }
 
@@ -63,6 +63,19 @@ pub fn blob_from_bytes(bytes: Bytes) -> Result<Blob, BlobsBundleError> {
     Ok(buf)
 }
 
+pub fn bytes_from_blob(blob: Bytes) -> [u8; SAFE_BYTES_PER_BLOB] {
+    let mut buf = [0u8; SAFE_BYTES_PER_BLOB];
+    buf.copy_from_slice(
+        &blob
+            .chunks(32)
+            .map(|x| x[1..].to_vec())
+            .collect::<Vec<_>>()
+            .concat(),
+    );
+
+    buf
+}
+
 fn kzg_commitment_to_versioned_hash(data: &Commitment) -> H256 {
     use k256::sha2::Digest;
     let mut versioned_hash: [u8; 32] = k256::sha2::Sha256::digest(data).into();
diff --git a/crates/common/types/constants.rs b/crates/common/types/constants.rs
index 0e195cb38..5ee347346 100644
--- a/crates/common/types/constants.rs
+++ b/crates/common/types/constants.rs
@@ -14,3 +14,6 @@ pub const VERSIONED_HASH_VERSION_KZG: u8 = 0x01; // Defined in [EIP-4844](https:
 pub const BYTES_PER_FIELD_ELEMENT: usize = 32;
 pub const FIELD_ELEMENTS_PER_BLOB: usize = 4096;
 pub const BYTES_PER_BLOB: usize = BYTES_PER_FIELD_ELEMENT * FIELD_ELEMENTS_PER_BLOB;
+/// The maximum number of bytes that can be "safely" stored in a blob. This is, prepend
+/// a zero byte for every 32 bytes of data to ensure they not exceed the field modulus.
+pub const SAFE_BYTES_PER_BLOB: usize = BYTES_PER_BLOB * 31 / 32;
diff --git a/crates/l2/Cargo.toml b/crates/l2/Cargo.toml
index dfcfa3ebe..8c8d9b01a 100644
--- a/crates/l2/Cargo.toml
+++ b/crates/l2/Cargo.toml
@@ -18,6 +18,7 @@ ethrex-rlp.workspace = true
 ethrex-rpc.workspace = true
 ethrex-blockchain = { workspace = true, features = ["l2"] }
 ethrex-storage.workspace = true
+ethrex-trie.workspace = true
 ethrex-vm = { workspace = true, features = ["l2"] }
 ethrex-dev = { path = "../../crates/blockchain/dev", default-features = false }
 ethrex-metrics = { path = "../blockchain/metrics", default-features = false }
diff --git a/crates/l2/contracts/src/l1/OnChainProposer.sol b/crates/l2/contracts/src/l1/OnChainProposer.sol
index 5ed845d08..2053c5131 100644
--- a/crates/l2/contracts/src/l1/OnChainProposer.sol
+++ b/crates/l2/contracts/src/l1/OnChainProposer.sol
@@ -38,13 +38,6 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
     /// @dev This is crucial for ensuring that only subsequents blocks are committed in the contract.
     uint256 public lastCommittedBlock;
 
-    /// @notice The next block to commit.
-    /// @dev This variable holds the block number of the next block to commit.
-    /// @dev `nextBlockToCommit` should be equal to `lastCommittedBlock` + 1.
-    /// @dev Only the block with the block number equal to `nextBlockToCommit` will be committed.
-    /// @dev This variable is called by the `l1_committer.rs`.
-    uint256 public nextBlockToCommit;
-
     /// @dev The sequencer addresses that are authorized to commit and verify blocks.
     mapping(address _authorizedAddress => bool)
         public authorizedSequencerAddresses;
@@ -149,10 +142,7 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
         bytes32 depositLogs
     ) external override onlySequencer {
         require(
-            blockNumber == nextBlockToCommit ||
-                (blockNumber == 0 &&
-                    lastCommittedBlock == 0 &&
-                    nextBlockToCommit == 0),
+            blockNumber == lastCommittedBlock + 1,
             "OnChainProposer: blockNumber is not the immediate successor of lastCommittedBlock"
         );
         require(
@@ -180,7 +170,6 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
             depositLogs
         );
         lastCommittedBlock = blockNumber;
-        nextBlockToCommit = blockNumber + 1;
         emit BlockCommitted(commitment);
     }
 
diff --git a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
index 1f38ef777..d50e182b5 100644
--- a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
+++ b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
@@ -10,11 +10,6 @@ interface IOnChainProposer {
     /// @return The latest committed block number as a uint256.
     function lastCommittedBlock() external view returns (uint256);
 
-    /// @notice The next block number to commit.
-    /// @dev This value should be equal to `lastCommittedBlock() + 1`.
-    /// @return The next block number to commit as a uint256.
-    function nextBlockToCommit() external view returns (uint256);
-
     /// @notice The latest verified block number.
     /// @return The latest verified block number as a uint256.
     function lastVerifiedBlock() external view returns (uint256);
diff --git a/crates/l2/docs/state_diffs.md b/crates/l2/docs/state_diffs.md
index f8b78c262..ab9a67f40 100644
--- a/crates/l2/docs/state_diffs.md
+++ b/crates/l2/docs/state_diffs.md
@@ -20,8 +20,11 @@ After executing each L2 block, the EVM will return the following data:
 
 The full state diff sent on every block will then be a sequence of bytes encoded as follows. We use the notation `un` for a sequence of `n` bits, so `u16` is a 16-bit sequence and `u96` a 96-bit one, we don’t really care about signedness here; if we don’t specify it, the value is of variable length and a field before it specifies it.
 
-- The first byte is a `u8`: the version header. For now it should always be zero, but we reserve it for future changes to the encoding/compression format.
-- Next come the `ModifiedAccounts` list. The first two bytes (`u16`) are the amount of element it has, followed by its entries. Each entry correspond to an altered address and has the form:
+- The first byte is a `u8`: the version header. For now it should always be one, but we reserve it for future changes to the encoding/compression format.
+- Next come the block header info:
+  - The `tx_root` and `receipts_root` are `u256` values.
+  - The `gas_limit`, `gas_used`, `timestamp`, and `base_fee_per_gas` are `u64` values.
+- Next the `ModifiedAccounts` list. The first two bytes (`u16`) are the amount of element it has, followed by its entries. Each entry correspond to an altered address and has the form:
   - The first byte is the `type` of the modification. The value is a `u8`, constrained to the range `[1; 23]`, computed by adding the following values:
     - `1` if the balance of the EOA/contract was modified.
     - `2` if the nonce of the EOA/contract was modified.
@@ -33,7 +36,7 @@ The full state diff sent on every block will then be a sequence of bytes encoded
   - If the nonce was modified (i.e. `type & 0x02 == 2`), the next 2 bytes, a `u16`, is the increase in the nonce.
   - If the storage was modified (i.e. `type & 0x04 == 4`), the next 2 bytes, a `u16`, is the number of storage slots modified. Then come the sequence of `(key_u256, new_value_u256)` key value pairs with the modified slots.
   - If the contract was created and the bytecode is previously unknown (i.e. `type & 0x08 == 8`), the next 2 bytes, a `u16`, is the length of the bytecode in bytes. Then come the bytecode itself.
-  - If the contract was created and the bytecode is previously known (i.e. `type & 0x10 == 16`), the next 20 bytes, a `u160`, is the hash of the bytecode of the contract.
+  - If the contract was created and the bytecode is previously known (i.e. `type & 0x10 == 16`), the next 32 bytes, a `u256`, is the hash of the bytecode of the contract.
   - Note that values `8` and `16` are mutually exclusive, and if `type` is greater or equal to `4`, then the address is a contract. Each address can only appear once in the list.
 - Next the `WithdrawalLogs` field:
     - First two bytes are the number of entries, then come the tuples `(to_u160, amount_u256, tx_hash_u256)`.
@@ -45,6 +48,9 @@ To recap, using `||` for byte concatenation and `[]` for optional parameters, th
 
 ```jsx
 version_header_u8 ||
+// Block Header info
+tx_root_u256 || receipts_root_u256 ||
+gas_limit_u64 || gas_used_u64 || timestamp_u64 || base_fee_per_gas_u64
 // Modified Accounts
 number_of_modified_accounts_u16 ||
 (
diff --git a/crates/l2/sequencer/errors.rs b/crates/l2/sequencer/errors.rs
index a5d21be35..5b70547e2 100644
--- a/crates/l2/sequencer/errors.rs
+++ b/crates/l2/sequencer/errors.rs
@@ -9,6 +9,7 @@ use ethrex_l2_sdk::merkle_tree::MerkleError;
 use ethrex_rpc::clients::eth::errors::{CalldataEncodeError, EthClientError};
 use ethrex_rpc::clients::EngineClientError;
 use ethrex_storage::error::StoreError;
+use ethrex_trie::TrieError;
 use ethrex_vm::EvmError;
 use tokio::task::JoinError;
 
@@ -168,6 +169,8 @@ pub enum StateDiffError {
     EmptyAccountDiff,
     #[error("The length of the vector is too big to fit in u16: {0}")]
     LengthTooBig(#[from] core::num::TryFromIntError),
+    #[error("DB Error: {0}")]
+    DbError(#[from] TrieError),
 }
 
 #[derive(Debug, thiserror::Error)]
diff --git a/crates/l2/sequencer/l1_committer.rs b/crates/l2/sequencer/l1_committer.rs
index d21a28201..bd244a1a5 100644
--- a/crates/l2/sequencer/l1_committer.rs
+++ b/crates/l2/sequencer/l1_committer.rs
@@ -85,9 +85,11 @@ impl Committer {
     }
 
     async fn main_logic(&mut self) -> Result<(), CommitterError> {
-        let block_number =
-            EthClient::get_next_block_to_commit(&self.eth_client, self.on_chain_proposer_address)
-                .await?;
+        let block_number = 1 + EthClient::get_last_committed_block(
+            &self.eth_client,
+            self.on_chain_proposer_address,
+        )
+        .await?;
 
         let Some(block_to_commit_body) = self
             .store
@@ -309,6 +311,7 @@ impl Committer {
         let state_diff = StateDiff {
             modified_accounts,
             version: StateDiff::default().version,
+            header: block.header.clone(),
             withdrawal_logs: withdrawals
                 .iter()
                 .map(|(hash, tx)| WithdrawalLog {
diff --git a/crates/l2/sequencer/state_diff.rs b/crates/l2/sequencer/state_diff.rs
index 6deb90475..6443c716d 100644
--- a/crates/l2/sequencer/state_diff.rs
+++ b/crates/l2/sequencer/state_diff.rs
@@ -2,6 +2,10 @@ use std::collections::HashMap;
 
 use bytes::Bytes;
 use ethereum_types::{Address, H256, U256};
+use ethrex_common::types::{AccountInfo, AccountState, BlockHeader};
+use ethrex_rlp::decode::RLPDecode;
+use ethrex_storage::{hash_address, AccountUpdate};
+use ethrex_trie::Trie;
 
 use super::errors::StateDiffError;
 
@@ -9,11 +13,12 @@ use super::errors::StateDiffError;
 pub struct AccountStateDiff {
     pub new_balance: Option<U256>,
     pub nonce_diff: u16,
-    pub storage: Vec<(H256, U256)>,
+    pub storage: HashMap<H256, U256>,
     pub bytecode: Option<Bytes>,
     pub bytecode_hash: Option<H256>,
 }
 
+#[derive(Clone, Copy)]
 pub enum AccountStateDiffType {
     NewBalance = 1,
     NonceDiff = 2,
@@ -39,6 +44,7 @@ pub struct DepositLog {
 #[derive(Clone)]
 pub struct StateDiff {
     pub version: u8,
+    pub header: BlockHeader,
     pub modified_accounts: HashMap<Address, AccountStateDiff>,
     pub withdrawal_logs: Vec<WithdrawalLog>,
     pub deposit_logs: Vec<DepositLog>,
@@ -59,17 +65,6 @@ impl TryFrom<u8> for AccountStateDiffType {
     }
 }
 
-impl Default for StateDiff {
-    fn default() -> Self {
-        StateDiff {
-            version: 1,
-            modified_accounts: HashMap::new(),
-            withdrawal_logs: Vec::new(),
-            deposit_logs: Vec::new(),
-        }
-    }
-}
-
 impl From<AccountStateDiffType> for u8 {
     fn from(value: AccountStateDiffType) -> Self {
         match value {
@@ -82,19 +77,47 @@ impl From<AccountStateDiffType> for u8 {
     }
 }
 
+impl AccountStateDiffType {
+    // Checks if the type is present in the given value
+    pub fn is_in(&self, value: u8) -> bool {
+        value & u8::from(*self) == u8::from(*self)
+    }
+}
+
+impl Default for StateDiff {
+    fn default() -> Self {
+        StateDiff {
+            version: 1,
+            header: BlockHeader::default(),
+            modified_accounts: HashMap::new(),
+            withdrawal_logs: Vec::new(),
+            deposit_logs: Vec::new(),
+        }
+    }
+}
+
 impl StateDiff {
     pub fn encode(&self) -> Result<Bytes, StateDiffError> {
         if self.version != 1 {
             return Err(StateDiffError::UnsupportedVersion(self.version));
         }
+
+        let mut encoded: Vec<u8> = Vec::new();
+        encoded.push(self.version);
+
+        // Header fields
+        encoded.extend(self.header.transactions_root.0);
+        encoded.extend(self.header.receipts_root.0);
+        encoded.extend(self.header.gas_limit.to_be_bytes());
+        encoded.extend(self.header.gas_used.to_be_bytes());
+        encoded.extend(self.header.timestamp.to_be_bytes());
+        encoded.extend(self.header.base_fee_per_gas.unwrap_or(0).to_be_bytes());
+
         let modified_accounts_len: u16 = self
             .modified_accounts
             .len()
             .try_into()
             .map_err(StateDiffError::from)?;
-
-        let mut encoded: Vec<u8> = Vec::new();
-        encoded.push(self.version);
         encoded.extend(modified_accounts_len.to_be_bytes());
 
         for (address, diff) in &self.modified_accounts {
@@ -104,12 +127,16 @@ impl StateDiff {
             encoded.extend(diff_encoded);
         }
 
+        let withdrawal_len: u16 = self.withdrawal_logs.len().try_into()?;
+        encoded.extend(withdrawal_len.to_be_bytes());
         for withdrawal in self.withdrawal_logs.iter() {
             encoded.extend(withdrawal.address.0);
             encoded.extend_from_slice(&withdrawal.amount.to_big_endian());
             encoded.extend(&withdrawal.tx_hash.0);
         }
 
+        let deposits_len: u16 = self.deposit_logs.len().try_into()?;
+        encoded.extend(deposits_len.to_be_bytes());
         for deposit in self.deposit_logs.iter() {
             encoded.extend(deposit.address.0);
             encoded.extend_from_slice(&deposit.amount.to_big_endian());
@@ -118,8 +145,124 @@ impl StateDiff {
         Ok(Bytes::from(encoded))
     }
 
-    pub fn decode() -> Result<Self, String> {
-        unimplemented!()
+    pub fn decode(bytes: &[u8]) -> Result<Self, StateDiffError> {
+        let mut decoder = Decoder::new(bytes);
+
+        let version = decoder.get_u8()?;
+        if version != 0x01 {
+            return Err(StateDiffError::UnsupportedVersion(version));
+        }
+
+        // Header fields
+        let transactions_root = decoder.get_h256()?;
+        let receipts_root = decoder.get_h256()?;
+        let gas_limit = decoder.get_u64()?;
+        let gas_used = decoder.get_u64()?;
+        let timestamp = decoder.get_u64()?;
+        let base_fee_per_gas = decoder.get_u64()?;
+
+        // Accounts diff
+        let modified_accounts_len = decoder.get_u16()?;
+
+        let mut modified_accounts = HashMap::with_capacity(modified_accounts_len.into());
+        for _ in 0..modified_accounts_len {
+            let next_bytes = bytes.get(decoder.consumed()..).ok_or(
+                StateDiffError::FailedToSerializeStateDiff("Not enough bytes".to_string()),
+            )?;
+            let (bytes_read, address, account_diff) = AccountStateDiff::decode(next_bytes)?;
+            decoder.advance(bytes_read);
+            modified_accounts.insert(address, account_diff);
+        }
+
+        let withdrawal_logs_len = decoder.get_u16()?;
+
+        let mut withdrawal_logs = Vec::with_capacity(withdrawal_logs_len.into());
+        for _ in 0..withdrawal_logs_len {
+            let address = decoder.get_address()?;
+            let amount = decoder.get_u256()?;
+            let tx_hash = decoder.get_h256()?;
+
+            withdrawal_logs.push(WithdrawalLog {
+                address,
+                amount,
+                tx_hash,
+            });
+        }
+
+        let deposit_logs_len = decoder.get_u16()?;
+
+        let mut deposit_logs = Vec::with_capacity(deposit_logs_len.into());
+        for _ in 0..deposit_logs_len {
+            let address = decoder.get_address()?;
+            let amount = decoder.get_u256()?;
+
+            deposit_logs.push(DepositLog {
+                address,
+                amount,
+                nonce: Default::default(),
+            });
+        }
+
+        Ok(Self {
+            header: BlockHeader {
+                transactions_root,
+                receipts_root,
+                gas_limit,
+                gas_used,
+                timestamp,
+                base_fee_per_gas: Some(base_fee_per_gas),
+                ..Default::default()
+            },
+            version,
+            modified_accounts,
+            withdrawal_logs,
+            deposit_logs,
+        })
+    }
+
+    pub fn to_account_updates(
+        &self,
+        prev_state: &Trie,
+    ) -> Result<Vec<AccountUpdate>, StateDiffError> {
+        let mut account_updates = Vec::new();
+
+        for (address, diff) in &self.modified_accounts {
+            let account_state = match prev_state
+                .get(&hash_address(address))
+                .map_err(StateDiffError::DbError)?
+            {
+                Some(rlp) => AccountState::decode(&rlp)
+                    .map_err(|e| StateDiffError::FailedToDeserializeStateDiff(e.to_string()))?,
+                None => AccountState::default(),
+            };
+
+            let balance = diff.new_balance.unwrap_or(account_state.balance);
+            let nonce = account_state.nonce + u64::from(diff.nonce_diff);
+            let bytecode_hash = diff.bytecode_hash.unwrap_or(account_state.code_hash);
+
+            let account_info = if diff.new_balance.is_some()
+                || diff.nonce_diff != 0
+                || diff.bytecode_hash.is_some()
+            {
+                Some(AccountInfo {
+                    balance,
+                    nonce,
+                    code_hash: bytecode_hash,
+                })
+            } else {
+                None
+            };
+
+            account_updates.push(AccountUpdate {
+                address: *address,
+                removed: false,
+                info: account_info,
+                code: diff.bytecode.clone(),
+                added_storage: diff.storage.clone(),
+            });
+        }
+
+        Ok(account_updates)
     }
 }
 
@@ -161,11 +304,7 @@ impl AccountStateDiff {
 
         if let Some(bytecode) = &self.bytecode {
             let r_type: u8 = AccountStateDiffType::Bytecode.into();
-            let bytecode_len: u16 = self
-                .storage
-                .len()
-                .try_into()
-                .map_err(StateDiffError::from)?;
+            let bytecode_len: u16 = bytecode.len().try_into().map_err(StateDiffError::from)?;
             r#type += r_type;
             encoded.extend(bytecode_len.to_be_bytes());
             encoded.extend(bytecode);
@@ -183,4 +322,168 @@ impl AccountStateDiff {
 
         Ok((r#type, Bytes::from(encoded)))
     }
+
+    /// Returns a tuple of the number of bytes read, the address of the account
+    /// and the decoded `AccountStateDiff`
+    pub fn decode(bytes: &[u8]) -> Result<(usize, Address, Self), StateDiffError> {
+        let mut decoder = Decoder::new(bytes);
+
+        let update_type = decoder.get_u8()?;
+
+        let address = decoder.get_address()?;
+
+        let new_balance = if AccountStateDiffType::NewBalance.is_in(update_type) {
+            Some(decoder.get_u256()?)
+        } else {
+            None
+        };
+
+        let nonce_diff = if AccountStateDiffType::NonceDiff.is_in(update_type) {
+            Some(decoder.get_u16()?)
+        } else {
+            None
+        };
+
+        let mut storage_diff = HashMap::new();
+        if AccountStateDiffType::Storage.is_in(update_type) {
+            let storage_slots_updated = decoder.get_u16()?;
+            storage_diff.reserve(storage_slots_updated.into());
+
+            for _ in 0..storage_slots_updated {
+                let key = decoder.get_h256()?;
+                let new_value = decoder.get_u256()?;
+
+                storage_diff.insert(key, new_value);
+            }
+        }
+
+        let bytecode = if AccountStateDiffType::Bytecode.is_in(update_type) {
+            let bytecode_len = decoder.get_u16()?;
+            Some(decoder.get_bytes(bytecode_len.into())?)
+        } else {
+            None
+        };
+
+        let bytecode_hash = if AccountStateDiffType::BytecodeHash.is_in(update_type) {
+            Some(decoder.get_h256()?)
+        } else {
+            None
+        };
+
+        Ok((
+            decoder.consumed(),
+            address,
+            AccountStateDiff {
+                new_balance,
+                nonce_diff: nonce_diff.unwrap_or(0),
+                storage: storage_diff,
+                bytecode,
+                bytecode_hash,
+            },
+        ))
+    }
+}
+
+struct Decoder {
+    bytes: Bytes,
+    offset: usize,
+}
+
+impl Decoder {
+    fn new(bytes: &[u8]) -> Self {
+        Decoder {
+            bytes: Bytes::copy_from_slice(bytes),
+            offset: 0,
+        }
+    }
+
+    fn consumed(&self) -> usize {
+        self.offset
+    }
+
+    fn advance(&mut self, size: usize) {
+        self.offset += size;
+    }
+
+    fn get_address(&mut self) -> Result<Address, StateDiffError> {
+        let res = Address::from_slice(self.bytes.get(self.offset..self.offset + 20).ok_or(
+            StateDiffError::FailedToDeserializeStateDiff("Not enough bytes".to_string()),
+        )?);
+        self.offset += 20;
+
+        Ok(res)
+    }
+
+    fn get_u256(&mut self) -> Result<U256, StateDiffError> {
+        let res = U256::from_big_endian(self.bytes.get(self.offset..self.offset + 32).ok_or(
+            StateDiffError::FailedToDeserializeStateDiff("Not enough bytes".to_string()),
+        )?);
+        self.offset += 32;
+
+        Ok(res)
+    }
+
+    fn get_h256(&mut self) -> Result<H256, StateDiffError> {
+        let res = H256::from_slice(self.bytes.get(self.offset..self.offset + 32).ok_or(
+            StateDiffError::FailedToDeserializeStateDiff("Not enough bytes".to_string()),
+        )?);
+        self.offset += 32;
+
+        Ok(res)
+    }
+
+    fn get_u8(&mut self) -> Result<u8, StateDiffError> {
+        let res =
+            self.bytes
+                .get(self.offset)
+                .ok_or(StateDiffError::FailedToDeserializeStateDiff(
+                    "Not enough bytes".to_string(),
+                ))?;
+        self.offset += 1;
+
+        Ok(*res)
+    }
+
+    fn get_u16(&mut self) -> Result<u16, StateDiffError> {
+        let res = u16::from_be_bytes(
+            self.bytes
+                .get(self.offset..self.offset + 2)
+                .ok_or(StateDiffError::FailedToDeserializeStateDiff(
+                    "Not enough bytes".to_string(),
+                ))?
+                .try_into()
+                .map_err(|_| {
+                    StateDiffError::FailedToDeserializeStateDiff("Cannot parse u16".to_string())
+                })?,
+        );
+        self.offset += 2;
+
+        Ok(res)
+    }
+
+    fn get_u64(&mut self) -> Result<u64, StateDiffError> {
+        let res = u64::from_be_bytes(
+            self.bytes
+                .get(self.offset..self.offset + 8)
+                .ok_or(StateDiffError::FailedToDeserializeStateDiff(
+                    "Not enough bytes".to_string(),
+                ))?
+                .try_into()
+                .map_err(|_| {
+                    StateDiffError::FailedToDeserializeStateDiff("Cannot parse u64".to_string())
+                })?,
+        );
+        self.offset += 8;
+
+        Ok(res)
+    }
+
+    fn get_bytes(&mut self, size: usize) -> Result<Bytes, StateDiffError> {
+        let res = self.bytes.get(self.offset..self.offset + size).ok_or(
+            StateDiffError::FailedToDeserializeStateDiff("Not enough bytes".to_string()),
+        )?;
+        self.offset += size;
+
+        Ok(Bytes::copy_from_slice(res))
+    }
 }
diff --git a/crates/networking/rpc/clients/eth/mod.rs b/crates/networking/rpc/clients/eth/mod.rs
index f5004772c..5ec69054f 100644
--- a/crates/networking/rpc/clients/eth/mod.rs
+++ b/crates/networking/rpc/clients/eth/mod.rs
@@ -510,18 +510,6 @@ impl EthClient {
         }
     }
 
-    pub async fn get_next_block_to_commit(
-        eth_client: &EthClient,
-        on_chain_proposer_address: Address,
-    ) -> Result<u64, EthClientError> {
-        Self::_call_block_variable(
-            eth_client,
-            b"nextBlockToCommit()",
-            on_chain_proposer_address,
-        )
-        .await
-    }
-
     /// Fetches a block from the Ethereum blockchain by its number or the latest/earliest/pending block.
     /// If no `block_number` is provided, get the latest.
     pub async fn get_block_by_number(
diff --git a/crates/networking/rpc/engine/fork_choice.rs b/crates/networking/rpc/engine/fork_choice.rs
index 652b949c9..1399a84c7 100644
--- a/crates/networking/rpc/engine/fork_choice.rs
+++ b/crates/networking/rpc/engine/fork_choice.rs
@@ -263,19 +263,26 @@ fn handle_forkchoice(
         Ok(head) => {
             // Remove included transactions from the mempool after we accept the fork choice
             // TODO(#797): The remove of transactions from the mempool could be incomplete (i.e. REORGS)
-            if let Ok(Some(block)) = context.storage.get_block_by_hash(head.compute_block_hash()) {
-                for tx in &block.body.transactions {
-                    context
-                        .blockchain
-                        .remove_transaction_from_pool(&tx.compute_hash())
-                        .map_err(|err| RpcErr::Internal(err.to_string()))?;
+            match context.storage.get_block_by_hash(head.compute_block_hash()) {
+                Ok(Some(block)) => {
+                    for tx in &block.body.transactions {
+                        context
+                            .blockchain
+                            .remove_transaction_from_pool(&tx.compute_hash())
+                            .map_err(|err| RpcErr::Internal(err.to_string()))?;
+                    }
                 }
-            } else {
-                return Err(RpcErr::Internal(
-                    "Failed to get block by hash to remove transactions from the mempool"
-                        .to_string(),
-                ));
-            }
+                Ok(None) => {
+                    warn!("Couldn't get block by hash to remove transactions from the mempool. This is expected in a reconstruted network")
+                }
+                Err(_) => {
+                    return Err(RpcErr::Internal(
+                        "Failed to get block by hash to remove transactions from the mempool"
+                            .to_string(),
+                    ))
+                }
+            };
+
             Ok((
                 Some(head),
                 ForkChoiceResponse::from(PayloadStatus::valid_with_hash(
diff --git a/crates/storage/store.rs b/crates/storage/store.rs
index 48c68a3cb..ddb51f744 100644
--- a/crates/storage/store.rs
+++ b/crates/storage/store.rs
@@ -312,9 +312,19 @@ impl Store {
         block_hash: BlockHash,
         account_updates: &[AccountUpdate],
     ) -> Result<Option<H256>, StoreError> {
-        let Some(mut state_trie) = self.state_trie(block_hash)? else {
+        let Some(state_trie) = self.state_trie(block_hash)? else {
             return Ok(None);
         };
+
+        let mut state_trie = self.apply_account_updates_from_trie(state_trie, account_updates)?;
+        Ok(Some(state_trie.hash()?))
+    }
+
+    pub fn apply_account_updates_from_trie(
+        &self,
+        mut state_trie: Trie,
+        account_updates: &[AccountUpdate],
+    ) -> Result<Trie, StoreError> {
         for update in account_updates.iter() {
             let hashed_address = hash_address(&update.address);
             if update.removed {
@@ -355,7 +365,8 @@ impl Store {
                 state_trie.insert(hashed_address, account_state.encode_to_vec())?;
             }
         }
-        Ok(Some(state_trie.hash()?))
+
+        Ok(state_trie)
     }
 
     /// Adds all genesis accounts and returns the genesis block's state_root
@@ -603,7 +614,7 @@ impl Store {
         self.engine.unset_canonical_block(number)
     }
 
-    // Obtain the storage trie for the given block
+    /// Obtain the storage trie for the given block
     pub fn state_trie(&self, block_hash: BlockHash) -> Result<Option<Trie>, StoreError> {
         let Some(header) = self.get_block_header_by_hash(block_hash)? else {
             return Ok(None);
@@ -611,7 +622,7 @@ impl Store {
         Ok(Some(self.engine.open_state_trie(header.state_root)))
     }
 
-    // Obtain the storage trie for the given account on the given block
+    /// Obtain the storage trie for the given account on the given block
     pub fn storage_trie(
         &self,
         block_hash: BlockHash,
@@ -645,11 +656,7 @@ impl Store {
         let Some(state_trie) = self.state_trie(block_hash)? else {
             return Ok(None);
         };
-        let hashed_address = hash_address(&address);
-        let Some(encoded_state) = state_trie.get(&hashed_address)? else {
-            return Ok(None);
-        };
-        Ok(Some(AccountState::decode(&encoded_state)?))
+        self.get_account_state_from_trie(&state_trie, address)
     }
 
     pub fn get_account_state_by_hash(
@@ -660,6 +667,14 @@ impl Store {
         let Some(state_trie) = self.state_trie(block_hash)? else {
             return Ok(None);
         };
+        self.get_account_state_from_trie(&state_trie, address)
+    }
+
+    pub fn get_account_state_from_trie(
+        &self,
+        state_trie: &Trie,
+        address: Address,
+    ) -> Result<Option<AccountState>, StoreError> {
         let hashed_address = hash_address(&address);
         let Some(encoded_state) = state_trie.get(&hashed_address)? else {
             return Ok(None);
@@ -842,16 +857,16 @@ impl Store {
         self.engine.open_state_trie(*EMPTY_TRIE_HASH)
     }
 
-    /// Methods exclusive for trie management during snap-syncing
+    // Methods exclusive for trie management during snap-syncing
 
-    // Obtain a state trie from the given state root
-    // Doesn't check if the state root is valid
+    /// Obtain a state trie from the given state root.
+    /// Doesn't check if the state root is valid
     pub fn open_state_trie(&self, state_root: H256) -> Trie {
         self.engine.open_state_trie(state_root)
     }
 
-    // Obtain a storage trie from the given address and storage_root
-    // Doesn't check if the account is stored
+    /// Obtain a storage trie from the given address and storage_root.
+    /// Doesn't check if the account is stored
     pub fn open_storage_trie(&self, account_hash: H256, storage_root: H256) -> Trie {
         self.engine.open_storage_trie(account_hash, storage_root)
     }

From 92cd758fd30447b82b6fffa13351772b50d6a165 Mon Sep 17 00:00:00 2001
From: Ivan Litteri <67517699+ilitteri@users.noreply.github.com>
Date: Fri, 21 Mar 2025 21:09:42 +0000
Subject: [PATCH 16/29] fix(l2): use absolute path for `.env` file (#2295)

**Motivation**

Running the stack outside of `crates/l2` fails because the `.env` file
path is set to its relative form.

**Description**

Use the `.env` file absolute path.
---
 crates/l2/contracts/deployer.rs |  4 ++--
 crates/l2/parse_toml.rs         |  6 +++---
 crates/l2/utils/config/mod.rs   | 29 +++++++++++++++++++----------
 3 files changed, 24 insertions(+), 15 deletions(-)

diff --git a/crates/l2/contracts/deployer.rs b/crates/l2/contracts/deployer.rs
index 928c019f4..df21e7311 100644
--- a/crates/l2/contracts/deployer.rs
+++ b/crates/l2/contracts/deployer.rs
@@ -3,7 +3,7 @@ use colored::Colorize;
 use ethereum_types::{Address, H160, H256};
 use ethrex_common::U256;
 use ethrex_l2::utils::config::errors;
-use ethrex_l2::utils::config::{read_env_as_lines, read_env_file, write_env};
+use ethrex_l2::utils::config::{read_env_as_lines, read_env_file, write_env_file};
 use ethrex_l2::utils::test_data_io::read_genesis_file;
 use ethrex_l2_sdk::calldata::{encode_calldata, Value};
 use ethrex_l2_sdk::get_address_from_secret_key;
@@ -166,7 +166,7 @@ If running locally, a reasonable value would be CONFIG_FILE=config.toml",
         }
         wr_lines.push(line);
     }
-    write_env(wr_lines).map_err(DeployError::EnvFileError)?;
+    write_env_file(wr_lines).map_err(DeployError::EnvFileError)?;
     Ok(())
 }
 
diff --git a/crates/l2/parse_toml.rs b/crates/l2/parse_toml.rs
index 54a793398..08d13ad7d 100644
--- a/crates/l2/parse_toml.rs
+++ b/crates/l2/parse_toml.rs
@@ -1,4 +1,4 @@
-use crate::errors::*;
+use crate::{errors::*, utils};
 use serde::Deserialize;
 use std::fs::OpenOptions;
 use std::io::Write;
@@ -243,12 +243,12 @@ impl L2Config {
 }
 
 pub fn write_to_env(config: String) -> Result<(), ConfigError> {
-    let env_file_name = std::env::var("ENV_FILE").unwrap_or(".env".to_string());
+    let env_file_path = utils::config::get_env_file_path();
     let env_file = OpenOptions::new()
         .write(true)
         .create(true)
         .truncate(true)
-        .open(env_file_name);
+        .open(env_file_path);
     match env_file {
         Ok(mut file) => {
             file.write_all(&config.into_bytes()).map_err(|_| {
diff --git a/crates/l2/utils/config/mod.rs b/crates/l2/utils/config/mod.rs
index 4b60deca8..5df646f97 100644
--- a/crates/l2/utils/config/mod.rs
+++ b/crates/l2/utils/config/mod.rs
@@ -1,4 +1,7 @@
-use std::io::{BufRead, Write};
+use std::{
+    io::{BufRead, Write},
+    path::PathBuf,
+};
 
 use tracing::{debug, info};
 pub mod block_producer;
@@ -11,9 +14,8 @@ pub mod prover_server;
 pub mod errors;
 
 pub fn read_env_file() -> Result<(), errors::ConfigError> {
-    let env_file_name = std::env::var("ENV_FILE").unwrap_or(".env".to_string());
-    let env_file_path = open_readable(env_file_name)?;
-    let reader = std::io::BufReader::new(env_file_path);
+    let env_file = open_env_file()?;
+    let reader = std::io::BufReader::new(env_file);
 
     for line in reader.lines() {
         let line = line?;
@@ -41,14 +43,14 @@ pub fn read_env_file() -> Result<(), errors::ConfigError> {
 
 pub fn read_env_as_lines(
 ) -> Result<std::io::Lines<std::io::BufReader<std::fs::File>>, errors::ConfigError> {
-    let env_file_path = std::env::var("ENV_FILE").unwrap_or(".env".to_owned());
-    let env_file = open_readable(env_file_path)?;
+    let env_file = open_env_file()?;
     let reader = std::io::BufReader::new(env_file);
 
     Ok(reader.lines())
 }
 
-fn open_readable(path: String) -> std::io::Result<std::fs::File> {
+fn open_env_file() -> std::io::Result<std::fs::File> {
+    let path = get_env_file_path();
     match std::fs::File::open(path) {
         Ok(file) => Ok(file),
         Err(err) if err.kind() == std::io::ErrorKind::NotFound => {
@@ -59,12 +61,19 @@ fn open_readable(path: String) -> std::io::Result<std::fs::File> {
     }
 }
 
-pub fn write_env(lines: Vec<String>) -> Result<(), errors::ConfigError> {
-    let env_file_name = std::env::var("ENV_FILE").unwrap_or(".env".to_string());
+pub fn get_env_file_path() -> PathBuf {
+    match std::env::var("ENV_FILE") {
+        Ok(env_file_path) => PathBuf::from(env_file_path),
+        Err(_) => PathBuf::from(env!("CARGO_MANIFEST_DIR")).join(".env"),
+    }
+}
+
+pub fn write_env_file(lines: Vec<String>) -> Result<(), errors::ConfigError> {
+    let path = get_env_file_path();
     let env_file = match std::fs::OpenOptions::new()
         .write(true)
         .truncate(true)
-        .open(&env_file_name)
+        .open(path)
     {
         Ok(file) => file,
         Err(err) if err.kind() == std::io::ErrorKind::NotFound => {

From fb42b34d1841339632e02aa5e6c7de0c38c5cfe3 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 11:56:34 -0300
Subject: [PATCH 17/29] test run prover only

---
 .github/workflows/ci_l2.yaml | 2 +-
 crates/l2/Makefile           | 2 --
 2 files changed, 1 insertion(+), 3 deletions(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index b8194dae4..dce0bf643 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -45,4 +45,4 @@ jobs:
         run: |
           cd crates/l2
           cp config_example.toml config.toml
-          ../../target/release/ethrex_prover & make ci_test
+          make ci_test & ../../target/release/ethrex_prover
diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index bfb4a1769..96d563f52 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -175,8 +175,6 @@ ci_test: ## 🚧 Runs the L2's integration test, used by the github's CI
 	touch .env
 	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} down
 	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} up -d --build
-	cargo test l2 --release -- --nocapture --test-threads=1
-	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} down
 
 test: ## 🚧 Runs the L2's integration test, run `make init` and in a new terminal make test
 	CONFIG_FILE=config.toml cargo test l2 --release -- --nocapture --test-threads=1 || (echo "The tests have failed.\n Is the L2 running? To start it, run:\n make rm-db-l1; make rm-db-l2; make restart" ; exit 1)

From 54571d980ee7c010b16bff82e72a9c3a618ae3e7 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 12:11:36 -0300
Subject: [PATCH 18/29] add env example

---
 .github/workflows/ci_l2.yaml |  1 +
 crates/l2/.env.example       | 37 ++++++++++++++++++++++++++++++++++++
 2 files changed, 38 insertions(+)
 create mode 100644 crates/l2/.env.example

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index dce0bf643..a6b937fb2 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -45,4 +45,5 @@ jobs:
         run: |
           cd crates/l2
           cp config_example.toml config.toml
+          cp .env.example .env
           make ci_test & ../../target/release/ethrex_prover
diff --git a/crates/l2/.env.example b/crates/l2/.env.example
new file mode 100644
index 000000000..60cfe6d8b
--- /dev/null
+++ b/crates/l2/.env.example
@@ -0,0 +1,37 @@
+DEPLOYER_ADDRESS=0x3d1e15a1a55578f7c920884a9943b3b35d0d885b
+DEPLOYER_PRIVATE_KEY=0x385c546456b6a603a1cfcaa9ec9494ba4832da08dd6bcf4de9a71e4a01b74924
+DEPLOYER_RISC0_CONTRACT_VERIFIER=0x00000000000000000000000000000000000000AA
+DEPLOYER_SP1_CONTRACT_VERIFIER=0x6db7edccfc1140ffe44e842017af815ec9ce2db7
+DEPLOYER_PICO_CONTRACT_VERIFIER=0x00000000000000000000000000000000000000aa
+DEPLOYER_SP1_DEPLOY_VERIFIER=true
+DEPLOYER_PICO_DEPLOY_VERIFIER=false
+DEPLOYER_SALT_IS_ZERO=true
+
+ETH_RPC_URL=http://localhost:8545
+
+ENGINE_API_RPC_URL=http://localhost:8552
+ENGINE_API_JWT_PATH=./jwt.hex
+
+L1_WATCHER_BRIDGE_ADDRESS=0xdc1cf336da5ccc868a9b455a9d313b0f6885bcbc
+L1_WATCHER_CHECK_INTERVAL_MS=1000
+L1_WATCHER_MAX_BLOCK_STEP=5000
+L1_WATCHER_L2_PROPOSER_PRIVATE_KEY=0x385c546456b6a603a1cfcaa9ec9494ba4832da08dd6bcf4de9a71e4a01b74924
+
+PROPOSER_INTERVAL_MS=5000
+PROPOSER_COINBASE_ADDRESS=0x0007a881CD95B1484fca47615B64803dad620C8d
+
+COMMITTER_ON_CHAIN_PROPOSER_ADDRESS=0xc2fa2dba759d3bf748e9a4805b749b6db6133eb6
+COMMITTER_L1_ADDRESS=0x3d1e15a1a55578f7c920884a9943b3b35d0d885b
+COMMITTER_L1_PRIVATE_KEY=0x385c546456b6a603a1cfcaa9ec9494ba4832da08dd6bcf4de9a71e4a01b74924
+COMMITTER_INTERVAL_MS=1000
+COMMITTER_ARBITRARY_BASE_BLOB_GAS_PRICE=1000000000
+
+PROVER_CLIENT_PROVER_SERVER_ENDPOINT=localhost:3900
+PROVER_CLIENT_INTERVAL_MS=5000
+
+PROVER_SERVER_LISTEN_IP=127.0.0.1
+PROVER_SERVER_LISTEN_PORT=3900
+PROVER_SERVER_VERIFIER_ADDRESS=0xE25583099BA105D9ec0A67f5Ae86D90e50036425
+PROVER_SERVER_VERIFIER_PRIVATE_KEY=0x39725efee3fb28614de3bacaffe4cc4bd8c436257e2c8bb887c4b5c4be45e76d
+PROVER_SERVER_DEV_MODE=false
+PROVER_SERVER_DEV_INTERVAL_MS=5000

From 16d661f562c4fd2fe16e0ff086dec9efc9961917 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:00:13 -0300
Subject: [PATCH 19/29] replace back build-prover with init-prover

---
 crates/l2/Makefile | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index 96d563f52..4006f5c97 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -143,7 +143,7 @@ down-l2: ## πŸ›‘ Shuts down the L2 Lambda ethrex Client
 
 restart-l2: down-l2 rm-db-l2 init-l2 ## πŸ”„ Restarts the L2 Lambda ethrex Client
 
-build-prover:
+init-prover:
 	@if [ -z "$$PROVER" ]; then \
 		echo "Error: ProverType (PROVER) is missing."; \
 		echo "Please provide it as an argument:"; \
@@ -158,7 +158,7 @@ build-prover:
 		GPU=",gpu"; \
 	fi; \
 
-	RUSTFLAGS='-C target-cpu=native' cargo build --release --features "$$PROVER$$GPU,l2" --manifest-path ./prover/Cargo.toml --bin ethrex_prover; \
+	RUSTFLAGS='-C target-cpu=native' cargo run --release --features "$$PROVER$$GPU,l2" --manifest-path ./prover/Cargo.toml --bin ethrex_prover; \
 
 rm-db-l2: ## πŸ›‘ Removes the DB used by the L2
 	cargo run --release --manifest-path ../../Cargo.toml --bin ethrex -- removedb --datadir ${ethrex_L2_DEV_LIBMDBX}

From 2bb8e30596541611b0a5a94badc9dd8d7745e01c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:00:21 -0300
Subject: [PATCH 20/29] add init-tmux

---
 crates/l2/Makefile | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index 4006f5c97..2fa035dde 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -13,6 +13,11 @@ help: ## πŸ“š Show help for each of the Makefile recipes
 
 init: init-local-l1 deploy-l1 update-system-contracts init-l2 ## πŸš€ Initializes a localnet with Lambda ethrex client as both L1 and L2
 
+init-tmux: down rm-db-l1 rm-db-l2 init-local-l1 deploy-l1 update-system-contracts
+	tmux new-session -d -s 'ethrex-l2' make init-l2 \; set -w remain-on-exit on
+	tmux split-window -h 'make init-prover;bash -i'
+	tmux attach
+
 ## Same as init but does not do deposits for rich accounts since that doesn't make sense for deployments to devnets/testnets i.e Sepolia
 init-testnet: init-local-l1 deploy-l1-testnet update-system-contracts  init-l2
 

From 289f86cf3d05b175e9552090b51f447507f5447e Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:00:32 -0300
Subject: [PATCH 21/29] revert ci_test changes

---
 crates/l2/Makefile | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)

diff --git a/crates/l2/Makefile b/crates/l2/Makefile
index 2fa035dde..db86611f2 100644
--- a/crates/l2/Makefile
+++ b/crates/l2/Makefile
@@ -180,16 +180,12 @@ ci_test: ## 🚧 Runs the L2's integration test, used by the github's CI
 	touch .env
 	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} down
 	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} up -d --build
+	cargo test l2 --release -- --nocapture --test-threads=1
+	CI_ETHREX_WORKDIR=${CI_ETHREX_WORKDIR} docker compose -f ${ethrex_L2_DOCKER_COMPOSE_PATH} down
 
 test: ## 🚧 Runs the L2's integration test, run `make init` and in a new terminal make test
 	CONFIG_FILE=config.toml cargo test l2 --release -- --nocapture --test-threads=1 || (echo "The tests have failed.\n Is the L2 running? To start it, run:\n make rm-db-l1; make rm-db-l2; make restart" ; exit 1)
 
-test-tmux: down rm-db-l1 rm-db-l2 init-local-l1 deploy-l1 update-system-contracts
-	tmux new-session -d -s 'ethrex-l2' make init-l2 \; set -w remain-on-exit on
-	tmux split-window -h 'make init-prover;bash -i'
-	tmux split-window -v 'cargo test l2 --release -- --nocapture --test-threads=1'
-	tmux attach
-
 # Purge L2's state
 UNAME_S:=$(shell uname -s)
 # This directory is set by crates/l2/utils/prover/save_state.rs -> const DEFAULT_DATADIR

From 3e3e92a8f1cf90ebb40d63b6b9eb3898cfdcbf2c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:00:54 -0300
Subject: [PATCH 22/29] run test with exec prover

---
 crates/l2/docker-compose-l2.yaml | 15 +++++++++++++++
 crates/l2/prover/Dockerfile      | 20 ++------------------
 2 files changed, 17 insertions(+), 18 deletions(-)

diff --git a/crates/l2/docker-compose-l2.yaml b/crates/l2/docker-compose-l2.yaml
index a0ce0a6ed..30d77cae8 100644
--- a/crates/l2/docker-compose-l2.yaml
+++ b/crates/l2/docker-compose-l2.yaml
@@ -55,3 +55,18 @@ services:
     depends_on:
       contract_deployer:
         condition: service_completed_successfully
+
+  ethrex_prover:
+    restart: always
+    container_name: ethrex_prover_client
+    image: "ethrex_prover"
+    build:
+      context: ../../
+      dockerfile: ./crates/l2/prover/Dockerfile
+    environment:
+      RUSTFLAGS: '-C target-cpu=native'
+    volumes:
+      - ./.env:/.env:ro
+    depends_on:
+      contract_deployer:
+        condition: service_completed_successfully
diff --git a/crates/l2/prover/Dockerfile b/crates/l2/prover/Dockerfile
index af780c00f..5c30d10c7 100644
--- a/crates/l2/prover/Dockerfile
+++ b/crates/l2/prover/Dockerfile
@@ -1,6 +1,3 @@
-ARG RUSTFLAGS
-ARG PROVER
-
 FROM rust:1.82 AS chef
 
 RUN apt-get update && apt-get install -y \
@@ -15,13 +12,10 @@ RUN apt-get update && apt-get install -y \
   ca-certificates \
   && rm -rf /var/lib/apt/lists/*
 RUN cargo install cargo-chef
-RUN curl -L https://sp1.succinct.xyz | bash && ~/.sp1/bin/sp1up --version 4.1.0
 
 ENV PATH="/root/.sp1/bin:$PATH"
 ENV PATH="/root/.cargo/bin:$PATH"
 
-RUN cargo prove install-toolchain
-
 WORKDIR /ethrex
 
 FROM chef AS planner
@@ -35,25 +29,15 @@ FROM chef AS builder
 ENV RUSTFLAGS=$RUSTFLAGS
 COPY --from=planner /ethrex/recipe.json recipe.json
 # Build dependencies only, these remained cached
-RUN cargo chef cook --release --recipe-path recipe.json --manifest-path crates/l2/prover/Cargo.toml --bin ethrex_prover --features sp1
+RUN cargo chef cook --release --recipe-path recipe.json --manifest-path crates/l2/prover/Cargo.toml --bin ethrex_prover --features exec
 
 COPY . .
-RUN cargo build --release --manifest-path crates/l2/prover/Cargo.toml --features sp1
+RUN cargo build --release --manifest-path crates/l2/prover/Cargo.toml --features exec
 
 FROM --platform=${BUILDPLATFORM} ubuntu:24.04
 ENV RUSTFLAGS=$RUSTFLAGS
 WORKDIR /usr/local/bin
 
-# install basic packages
-RUN apt-get update && apt-get -y install curl git
-
-# install rust
-RUN curl https://sh.rustup.rs -sSf | bash -s -- -y
-ENV PATH="/root/.cargo/bin:${PATH}"
-
-# Check cargo is visible
-RUN cargo --help
-
 COPY --from=builder ethrex/target/release/ethrex_prover .
 
 ENTRYPOINT [ "./ethrex_prover" ]

From 10c831d60a8d0e5f70f3b1367917ba567ff5701d Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:09:30 -0300
Subject: [PATCH 23/29] fix job

---
 .github/workflows/ci_l2.yaml | 13 +------------
 1 file changed, 1 insertion(+), 12 deletions(-)

diff --git a/.github/workflows/ci_l2.yaml b/.github/workflows/ci_l2.yaml
index a6b937fb2..fffa32337 100644
--- a/.github/workflows/ci_l2.yaml
+++ b/.github/workflows/ci_l2.yaml
@@ -31,19 +31,8 @@ jobs:
       - name: Caching
         uses: Swatinem/rust-cache@v2
 
-      - name: RISC-V SP1 toolchain install
-        run: |
-          curl -L https://sp1.succinct.xyz | bash
-          ~/.sp1/bin/sp1up --version 4.1.0
-
-      - name: Build prover
-        run: |
-          cd crates/l2
-          PROVER=sp1 make build-prover
-
       - name: Run L2 integration test
         run: |
           cd crates/l2
           cp config_example.toml config.toml
-          cp .env.example .env
-          make ci_test & ../../target/release/ethrex_prover
+          make ci_test

From d6c4a729dbbf8a4a00cedd0d8ab286c75a890b99 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Mon, 24 Mar 2025 15:26:58 -0300
Subject: [PATCH 24/29] remove unused env

---
 crates/l2/prover/Dockerfile | 1 -
 1 file changed, 1 deletion(-)

diff --git a/crates/l2/prover/Dockerfile b/crates/l2/prover/Dockerfile
index 5c30d10c7..8dd8a7c3d 100644
--- a/crates/l2/prover/Dockerfile
+++ b/crates/l2/prover/Dockerfile
@@ -19,7 +19,6 @@ ENV PATH="/root/.cargo/bin:$PATH"
 WORKDIR /ethrex
 
 FROM chef AS planner
-ENV RUSTFLAGS=$RUSTFLAGS
 
 COPY . .
 # Determine the crates that need to be built from dependencies

From 5d09c0f9dbd6dee8742153589af4c06e8b5a981e Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Wed, 12 Mar 2025 17:17:36 -0300
Subject: [PATCH 25/29] add deposit hash calculation

---
 Cargo.lock                                    |   1 +
 crates/l2/prover/zkvm/interface/Cargo.toml    |   1 +
 .../l2/prover/zkvm/interface/sp1/Cargo.lock   | 137 +++++++++---------
 .../l2/prover/zkvm/interface/sp1/Cargo.toml   |   1 +
 .../l2/prover/zkvm/interface/sp1/src/main.rs  |  29 +++-
 crates/l2/prover/zkvm/interface/src/lib.rs    |  73 ++++++++++
 6 files changed, 171 insertions(+), 71 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index ea2fe0f6c..8f080996e 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -11060,6 +11060,7 @@ dependencies = [
  "ethrex-storage",
  "ethrex-trie",
  "ethrex-vm",
+ "keccak-hash",
  "pico-cli",
  "risc0-build 1.2.4",
  "serde",
diff --git a/crates/l2/prover/zkvm/interface/Cargo.toml b/crates/l2/prover/zkvm/interface/Cargo.toml
index d1b53a643..8b3e68b70 100644
--- a/crates/l2/prover/zkvm/interface/Cargo.toml
+++ b/crates/l2/prover/zkvm/interface/Cargo.toml
@@ -14,6 +14,7 @@ ethrex-vm = { path = "../../../../vm", default-features = false }
 ethrex-rlp = { path = "../../../../common/rlp", default-features = false }
 ethrex-storage = { path = "../../../../storage", default-features = false }
 ethrex-trie = { path = "../../../../common/trie", default-features = false }
+keccak-hash = "0.11.0"
 
 # Temporarily pin version because 0.11.1 breaks compilation
 # alloy-rpc-types-engine = "=0.11.0"
diff --git a/crates/l2/prover/zkvm/interface/sp1/Cargo.lock b/crates/l2/prover/zkvm/interface/sp1/Cargo.lock
index 2dc5e2235..4e3c6128e 100644
--- a/crates/l2/prover/zkvm/interface/sp1/Cargo.lock
+++ b/crates/l2/prover/zkvm/interface/sp1/Cargo.lock
@@ -1,6 +1,6 @@
 # This file is automatically @generated by Cargo.
 # It is not intended for manual editing.
-version = 3
+version = 4
 
 [[package]]
 name = "addchain"
@@ -124,9 +124,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-json-abi"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "ec80745c33797e8baf547a8cfeb850e60d837fe9b9e67b3f579c1fcd26f527e9"
+checksum = "df4054f177d1600f17e2bc152f6a927592641b19861e6005cc51bdf7d4fa27a6"
 dependencies = [
  "alloy-primitives",
  "alloy-sol-type-parser",
@@ -149,9 +149,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-primitives"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "eacedba97e65cdc7ab592f2b22ef5d3ab8d60b2056bc3a6e6363577e8270ec6f"
+checksum = "7283185baefbe66136649dc316c9dcc6f0e9f1d635ae19783615919f83bc298a"
 dependencies = [
  "alloy-rlp",
  "bytes",
@@ -243,9 +243,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-sol-macro"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3637022e781bc73a9e300689cd91105a0e6be00391dd4e2110a71cc7e9f20a94"
+checksum = "f99b007e002f1082b28827cc47d9c72562d412a98c06f29aa438118ff3036c43"
 dependencies = [
  "alloy-sol-macro-expander",
  "alloy-sol-macro-input",
@@ -257,9 +257,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-sol-macro-expander"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3b9bd22d0bba90e40f40c625c33d39afb7d62b22192476a2ce1dcf8409dce880"
+checksum = "6c0a9cb9b1afbcd3325e0fff9fdf98e6d095643fae9e5584e80597f0b79b6d6e"
 dependencies = [
  "alloy-sol-macro-input",
  "const-hex",
@@ -275,9 +275,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-sol-macro-input"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "05ae4646e8123ec2fd10f9c22e361ffe4365c42811431829c2eabae528546bcc"
+checksum = "530c4863e707b95f99b37792cdfa94d30004ec552aed41e200a1d9264d44e669"
 dependencies = [
  "const-hex",
  "dunce",
@@ -291,9 +291,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-sol-type-parser"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "488a747fdcefeec5c1ed5aa9e08becd775106777fdeae2a35730729fc8a95910"
+checksum = "74b210dd863afa9da93c488601a1f23bee1e3ce47e15519582320c205645a7a0"
 dependencies = [
  "serde",
  "winnow",
@@ -301,9 +301,9 @@ dependencies = [
 
 [[package]]
 name = "alloy-sol-types"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "767957235807b021126dca1598ac3ef477007eace07961607dc5f490550909c7"
+checksum = "4f5ff802859e2797d022dc812b5b4ee40d829e0fb446c269a87826c7f0021976"
 dependencies = [
  "alloy-json-abi",
  "alloy-primitives",
@@ -722,9 +722,9 @@ checksum = "8b96ec4966b5813e2c0507c1f86115c8c5abaadc3980879c3424042a02fd1ad3"
 
 [[package]]
 name = "cc"
-version = "1.2.16"
+version = "1.2.17"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "be714c154be609ec7f5dad223a33bf1482fff90472de28f7362806e6d4832b8c"
+checksum = "1fcb57c740ae1daf453ae85f16e37396f672b039e00d9d866e07ddb24e328e3a"
 dependencies = [
  "shlex",
 ]
@@ -1003,9 +1003,9 @@ dependencies = [
 
 [[package]]
 name = "deranged"
-version = "0.4.0"
+version = "0.4.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "9c9e6a11ca8224451684bc0d7d5a7adbf8f2fd6887261a1cfc3c0432f9d4068e"
+checksum = "28cfac68e08048ae1883171632c2aef3ebc555621ae56fbccce1cbf22dd7f058"
 dependencies = [
  "powerfmt",
  "serde",
@@ -1591,14 +1591,15 @@ dependencies = [
 
 [[package]]
 name = "iana-time-zone"
-version = "0.1.61"
+version = "0.1.62"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220"
+checksum = "b2fd658b06e56721792c5df4475705b6cda790e9298d19d2f8af083457bcd127"
 dependencies = [
  "android_system_properties",
  "core-foundation-sys",
  "iana-time-zone-haiku",
  "js-sys",
+ "log",
  "wasm-bindgen",
  "windows-core",
 ]
@@ -1738,9 +1739,9 @@ dependencies = [
 
 [[package]]
 name = "k256"
-version = "0.13.3"
+version = "0.13.4"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "956ff9b67e26e1a6a866cb758f12c6f8746208489e3e4a4b5580802f2f0a587b"
+checksum = "f6e3919bbaa2945715f0bb6d3934a173d1e9a59ac23767fbaaef277265a7411b"
 dependencies = [
  "cfg-if",
  "ecdsa",
@@ -1892,9 +1893,9 @@ checksum = "fe7db12097d22ec582439daf8618b8fdd1a7bef6270e9af3b1ebcd30893cf413"
 
 [[package]]
 name = "log"
-version = "0.4.26"
+version = "0.4.27"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "30bde2b3dc3671ae49d8e2e9f044c7c005836e7a023ee57cffa25ab82764bb9e"
+checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94"
 
 [[package]]
 name = "macro-string"
@@ -2051,9 +2052,9 @@ dependencies = [
 
 [[package]]
 name = "p3-baby-bear"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "080896e9d09e9761982febafe3b3da5cbf320e32f0c89b6e2e01e875129f4c2d"
+checksum = "8cb78ed180f93f297480787f344d8191893942ad454404b5f5695f8bb7db9caf"
 dependencies = [
  "num-bigint 0.4.6",
  "p3-field",
@@ -2066,9 +2067,9 @@ dependencies = [
 
 [[package]]
 name = "p3-dft"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "292e97d02d4c38d8b306c2b8c0428bf15f4d32a11a40bcf80018f675bf33267e"
+checksum = "c0790c522c282efcce3a2fcee4bc3f08f85554c3186af6ae2b80609af85e7c4b"
 dependencies = [
  "p3-field",
  "p3-matrix",
@@ -2079,9 +2080,9 @@ dependencies = [
 
 [[package]]
 name = "p3-field"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "f91d8e5f9ede1171adafdb0b6a0df1827fbd4eb6a6217bfa36374e5d86248757"
+checksum = "0f0042a53ff45c6fbd48e7809ae9da442a72619fe5199a7608ae0176848f59e0"
 dependencies = [
  "itertools 0.12.1",
  "num-bigint 0.4.6",
@@ -2093,9 +2094,9 @@ dependencies = [
 
 [[package]]
 name = "p3-matrix"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "98bf2c7680b8e906a5e147fe4ceb05a11cc9fa35678aa724333bcb35c72483c1"
+checksum = "1534966235a3f1d18060c3c30f19dde660ce3a6b5b3971aa7167f606c5e28216"
 dependencies = [
  "itertools 0.12.1",
  "p3-field",
@@ -2108,15 +2109,15 @@ dependencies = [
 
 [[package]]
 name = "p3-maybe-rayon"
-version = "0.2.0"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3925562a4c03183eafc92fd07b19f65ac6cb4b48d68c3920ce58d9bee6efe362"
+checksum = "6f1c9f082bdcc02ca518889a6de856c1e2457c5aece70a480b8903f499c23021"
 
 [[package]]
 name = "p3-mds"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "706cea48976f54702dc68dffa512684c1304d1a3606cadea423cfe0b1ee25134"
+checksum = "21e6eb5891c3f395a761a6451386aba4549ec4cd2a2dd0ed66fd86edbdbdfd22"
 dependencies = [
  "itertools 0.12.1",
  "p3-dft",
@@ -2129,9 +2130,9 @@ dependencies = [
 
 [[package]]
 name = "p3-poseidon2"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a2ce5f5ec7f1ba3a233a671621029def7bd416e7c51218c9d1167d21602cf312"
+checksum = "a24abe96fff52632152b167b95a0f0ab0805502d98f858dbac792aeabf24fa39"
 dependencies = [
  "gcd",
  "p3-field",
@@ -2143,9 +2144,9 @@ dependencies = [
 
 [[package]]
 name = "p3-symmetric"
-version = "0.2.0-succinct"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "2f29dc5bb6c99d3de75869d5c086874b64890280eeb7d3e068955f939e219253"
+checksum = "b479c0a51ba4cd432c77a8e7bedbacf94426ba6ee2639578d589d8f724f25589"
 dependencies = [
  "itertools 0.12.1",
  "p3-field",
@@ -2154,9 +2155,9 @@ dependencies = [
 
 [[package]]
 name = "p3-util"
-version = "0.2.0"
+version = "0.2.1-succinct"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "88dd5ca3eb6ff33cb20084778c32a6d68064a1913b4632437408c5a1098408b3"
+checksum = "c93e679f867938dbfa5b40361e8642fd21986a7273eeb4cad2bad1b8a3a3d03b"
 dependencies = [
  "serde",
 ]
@@ -2454,9 +2455,9 @@ checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c"
 
 [[package]]
 name = "revm"
-version = "19.6.0"
+version = "19.7.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7b906766b7ba049b515848952b5ae74f363d456e98de2021048a513e442b4f42"
+checksum = "c175ecec83bba464aa8406502fe5bf670491c2ace81a153264891d43bc7fa332"
 dependencies = [
  "auto_impl",
  "cfg-if",
@@ -2497,9 +2498,9 @@ dependencies = [
 
 [[package]]
 name = "revm-precompile"
-version = "16.1.0"
+version = "16.2.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6caa1a7ff2cc4a09a263fcf9de99151706f323d30f33d519ed329f017a02b046"
+checksum = "99743c3a2cac341084cc15ac74286c4bf34a0941ebf60aa420cfdb9f81f72f9f"
 dependencies = [
  "aurora-engine-modexp",
  "c-kzg",
@@ -2715,7 +2716,7 @@ dependencies = [
 [[package]]
 name = "secp256k1"
 version = "0.29.1"
-source = "git+https://github.com/sp1-patches/rust-secp256k1?tag=patch-0.29.1-sp1-4.0.0#d3a398cfc7928cdb9b92ea2ab45232e6468ef230"
+source = "git+https://github.com/sp1-patches/rust-secp256k1?tag=patch-0.29.1-sp1-4.0.0#9b40a0c114a0c52334afbd37f84465a64564b4ac"
 dependencies = [
  "cfg-if",
  "ecdsa",
@@ -2727,7 +2728,7 @@ dependencies = [
 [[package]]
 name = "secp256k1-sys"
 version = "0.10.0"
-source = "git+https://github.com/sp1-patches/rust-secp256k1?tag=patch-0.29.1-sp1-4.0.0#d3a398cfc7928cdb9b92ea2ab45232e6468ef230"
+source = "git+https://github.com/sp1-patches/rust-secp256k1?tag=patch-0.29.1-sp1-4.0.0#9b40a0c114a0c52334afbd37f84465a64564b4ac"
 dependencies = [
  "cc",
 ]
@@ -2894,9 +2895,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b"
 
 [[package]]
 name = "sp1-lib"
-version = "4.1.3"
+version = "4.1.4"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a062a18747775aa84ed9776549ebd94f4cb5c05a2ffaa4f41a0cf6d49b0bee7f"
+checksum = "dcfd37d1d409446577a070f17b8e125cd82c39d78abc512977dd4857677c738b"
 dependencies = [
  "bincode",
  "elliptic-curve",
@@ -2906,9 +2907,9 @@ dependencies = [
 
 [[package]]
 name = "sp1-primitives"
-version = "4.1.3"
+version = "4.1.4"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "f33a3021e4775b92020f82502b3a4f8dbecd2be375fae2c9a831df120fe4e10a"
+checksum = "5943e519bd51b8c5c76813948acb544283a2eee5aead41fe9c63bf6425fcad18"
 dependencies = [
  "bincode",
  "hex",
@@ -2924,9 +2925,9 @@ dependencies = [
 
 [[package]]
 name = "sp1-zkvm"
-version = "4.1.3"
+version = "4.1.4"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3923200d2de0a1e96590b3b9e968db282b2856c088c1ab164229d36d8f3192ff"
+checksum = "e5085a98fa0cf2a1e570dfd5347eb3b86df23e24d3b684e53b6fff36309c4758"
 dependencies = [
  "cfg-if",
  "getrandom 0.2.15",
@@ -3024,9 +3025,9 @@ dependencies = [
 
 [[package]]
 name = "syn-solidity"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "d975606bae72d8aad5b07d9342465e123a2cccf53a5a735aedf81ca92a709ecb"
+checksum = "36dbbf0d465ab9fdfea3093e755ae8839bdc1263dbe18d35064d02d6060f949e"
 dependencies = [
  "paste",
  "proc-macro2",
@@ -3042,9 +3043,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369"
 
 [[package]]
 name = "tempfile"
-version = "3.19.0"
+version = "3.19.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "488960f40a3fd53d72c2a29a58722561dee8afdd175bd88e3db4677d7b2ba600"
+checksum = "7437ac7763b9b123ccf33c338a5cc1bac6f69b45a136c19bdd8a65e3916435bf"
 dependencies = [
  "fastrand",
  "getrandom 0.3.2",
@@ -3084,9 +3085,9 @@ dependencies = [
 
 [[package]]
 name = "time"
-version = "0.3.40"
+version = "0.3.41"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "9d9c75b47bdff86fa3334a3db91356b8d7d86a9b839dab7d0bdc5c3d3a077618"
+checksum = "8a7619e19bc266e0f9c5e6686659d394bc57973859340060a69221e57dbc0c40"
 dependencies = [
  "deranged",
  "itoa",
@@ -3105,9 +3106,9 @@ checksum = "c9e9a38711f559d9e3ce1cdb06dd7c5b8ea546bc90052da6d06bb76da74bb07c"
 
 [[package]]
 name = "time-macros"
-version = "0.2.21"
+version = "0.2.22"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "29aa485584182073ed57fd5004aa09c371f021325014694e432313345865fd04"
+checksum = "3526739392ec93fd8b359c8e98514cb3e8e021beb4e5f597b00a0221f8ed8a49"
 dependencies = [
  "num-conv",
  "time-core",
@@ -3473,18 +3474,18 @@ dependencies = [
 
 [[package]]
 name = "zerocopy"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "fd97444d05a4328b90e75e503a34bad781f14e28a823ad3557f0750df1ebcbc6"
+checksum = "2586fea28e186957ef732a5f8b3be2da217d65c5969d4b1e17f973ebbe876879"
 dependencies = [
  "zerocopy-derive",
 ]
 
 [[package]]
 name = "zerocopy-derive"
-version = "0.8.23"
+version = "0.8.24"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6352c01d0edd5db859a63e2605f4ea3183ddbd15e2c4a9e7d32184df75e4f154"
+checksum = "a996a8f63c5c4448cd959ac1bab0aaa3306ccfd060472f85943ee0750f0169be"
 dependencies = [
  "proc-macro2",
  "quote",
@@ -3515,6 +3516,7 @@ dependencies = [
 name = "zkvm-sp1-program"
 version = "0.1.0"
 dependencies = [
+ "cfg-if",
  "ethrex-blockchain",
  "ethrex-common",
  "ethrex-rlp",
@@ -3532,6 +3534,7 @@ dependencies = [
  "ethrex-storage",
  "ethrex-trie",
  "ethrex-vm",
+ "keccak-hash",
  "serde",
  "serde_json",
  "serde_with",
diff --git a/crates/l2/prover/zkvm/interface/sp1/Cargo.toml b/crates/l2/prover/zkvm/interface/sp1/Cargo.toml
index 84b508e6d..ac3d218df 100644
--- a/crates/l2/prover/zkvm/interface/sp1/Cargo.toml
+++ b/crates/l2/prover/zkvm/interface/sp1/Cargo.toml
@@ -8,6 +8,7 @@ edition = "2021"
 [dependencies]
 sp1-zkvm = "4.1.0"
 zkvm_interface = { path = "../" }
+cfg-if = "1.0.0"
 
 ethrex-common = { path = "../../../../../common", default-features = false }
 ethrex-rlp = { path = "../../../../../common/rlp" }
diff --git a/crates/l2/prover/zkvm/interface/sp1/src/main.rs b/crates/l2/prover/zkvm/interface/sp1/src/main.rs
index 3b98973ee..1341d26ea 100644
--- a/crates/l2/prover/zkvm/interface/sp1/src/main.rs
+++ b/crates/l2/prover/zkvm/interface/sp1/src/main.rs
@@ -7,6 +7,9 @@ use zkvm_interface::{
     trie::{update_tries, verify_db},
 };
 
+#[cfg(feature = "l2")]
+use zkvm_interface::deposits::{get_block_deposits, get_deposit_hash};
+
 sp1_zkvm::entrypoint!(main);
 
 pub fn main() {
@@ -59,8 +62,26 @@ pub fn main() {
         panic!("invalid final state trie");
     }
 
-    sp1_zkvm::io::commit(&ProgramOutput {
-        initial_state_hash,
-        final_state_hash,
-    });
+    cfg_if::cfg_if! {
+        if #[cfg(feature = "l2")] {
+            let deposits = get_block_deposits(&block);
+            let deposit_logs_hash = get_deposit_hash(
+                deposits
+                    .iter()
+                    .filter_map(|tx| tx.get_deposit_hash())
+                    .collect(),
+            ).expect("failed to calculate deposit logs hash");
+
+            sp1_zkvm::io::commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+                deposit_logs_hash
+            });
+        } else {
+            sp1_zkvm::io::commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+            });
+        }
+    }
 }
diff --git a/crates/l2/prover/zkvm/interface/src/lib.rs b/crates/l2/prover/zkvm/interface/src/lib.rs
index 19e85a3b0..39614892b 100644
--- a/crates/l2/prover/zkvm/interface/src/lib.rs
+++ b/crates/l2/prover/zkvm/interface/src/lib.rs
@@ -50,6 +50,9 @@ pub mod io {
         pub initial_state_hash: H256,
         /// final state trie root hash
         pub final_state_hash: H256,
+        #[cfg(feature = "l2")]
+        /// deposit_logs_hash
+        pub deposit_logs_hash: H256,
     }
 
     impl ProgramOutput {
@@ -247,3 +250,73 @@ pub mod trie {
         Ok(())
     }
 }
+
+#[cfg(feature = "l2")]
+pub mod deposits {
+    // This module was based on the L1 committer.
+    // TODO: We should move this to some kind of "common" library for the L2, but the zkvm programs
+    // can't depend on ethrex-l2 because of incompatible dependencies.
+
+    use ethrex_common::{
+        types::{Block, PrivilegedL2Transaction, Transaction},
+        Address, U256,
+    };
+    use keccak_hash::{keccak, H256};
+
+    #[derive(Debug, thiserror::Error)]
+    pub enum DepositError {
+        #[error("Failed to decode deposit hash")]
+        FailedToDecodeHash,
+        #[error("Length does not fit in u16")]
+        LengthTooLarge(#[from] std::num::TryFromIntError),
+    }
+
+    #[derive(Clone)]
+    pub struct DepositLog {
+        pub address: Address,
+        pub amount: U256,
+        pub nonce: u64,
+    }
+
+    pub fn get_block_deposits(block: &Block) -> Vec<PrivilegedL2Transaction> {
+        let deposits = block
+            .body
+            .transactions
+            .iter()
+            .filter_map(|tx| match tx {
+                Transaction::PrivilegedL2Transaction(tx) => Some(tx.clone()),
+                _ => None,
+            })
+            .collect();
+
+        deposits
+    }
+
+    pub fn get_deposit_hash(deposit_hashes: Vec<H256>) -> Result<H256, DepositError> {
+        if !deposit_hashes.is_empty() {
+            let deposit_hashes_len: u16 = deposit_hashes
+                .len()
+                .try_into()
+                .map_err(DepositError::from)?;
+            Ok(H256::from_slice(
+                [
+                    &deposit_hashes_len.to_be_bytes(),
+                    keccak(
+                        deposit_hashes
+                            .iter()
+                            .map(H256::as_bytes)
+                            .collect::<Vec<&[u8]>>()
+                            .concat(),
+                    )
+                    .as_bytes()
+                    .get(2..32)
+                    .ok_or(DepositError::FailedToDecodeHash)?,
+                ]
+                .concat()
+                .as_slice(),
+            ))
+        } else {
+            Ok(H256::zero())
+        }
+    }
+}

From eaa41a1258ced39724730fc6b63fc7c93b27b348 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Wed, 12 Mar 2025 17:30:47 -0300
Subject: [PATCH 26/29] add deposits to the rest of backends

---
 .../l2/prover/zkvm/interface/pico/Cargo.toml  |  1 +
 .../l2/prover/zkvm/interface/pico/src/main.rs | 26 +++++++++++++---
 .../l2/prover/zkvm/interface/risc0/Cargo.toml |  1 +
 .../prover/zkvm/interface/risc0/src/main.rs   | 31 ++++++++++++++++---
 4 files changed, 50 insertions(+), 9 deletions(-)

diff --git a/crates/l2/prover/zkvm/interface/pico/Cargo.toml b/crates/l2/prover/zkvm/interface/pico/Cargo.toml
index 3479002fe..3525ea080 100644
--- a/crates/l2/prover/zkvm/interface/pico/Cargo.toml
+++ b/crates/l2/prover/zkvm/interface/pico/Cargo.toml
@@ -8,6 +8,7 @@ edition = "2024"
 [dependencies]
 pico-sdk = { git = "https://github.com/brevis-network/pico" }
 zkvm_interface = { path = "../" }
+cfg-if = "1.0.0"
 
 ethrex-common = { path = "../../../../../common", default-features = false }
 ethrex-rlp = { path = "../../../../../common/rlp" }
diff --git a/crates/l2/prover/zkvm/interface/pico/src/main.rs b/crates/l2/prover/zkvm/interface/pico/src/main.rs
index f1b4d3c3b..a543f9d9d 100644
--- a/crates/l2/prover/zkvm/interface/pico/src/main.rs
+++ b/crates/l2/prover/zkvm/interface/pico/src/main.rs
@@ -61,8 +61,26 @@ pub fn main() {
         panic!("invalid final state trie");
     }
 
-    commit(&ProgramOutput {
-        initial_state_hash,
-        final_state_hash,
-    });
+    cfg_if::cfg_if! {
+        if #[cfg(feature = "l2")] {
+            let deposits = get_block_deposits(&block);
+            let deposit_logs_hash = get_deposit_hash(
+                deposits
+                    .iter()
+                    .filter_map(|tx| tx.get_deposit_hash())
+                    .collect(),
+            ).expect("failed to calculate deposit logs hash");
+
+            commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+                deposit_logs_hash
+            });
+        } else {
+            commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+            });
+        }
+    }
 }
diff --git a/crates/l2/prover/zkvm/interface/risc0/Cargo.toml b/crates/l2/prover/zkvm/interface/risc0/Cargo.toml
index d45914805..c41ef5766 100644
--- a/crates/l2/prover/zkvm/interface/risc0/Cargo.toml
+++ b/crates/l2/prover/zkvm/interface/risc0/Cargo.toml
@@ -8,6 +8,7 @@ edition = "2021"
 [dependencies]
 risc0-zkvm = { version = "1.2.2", default-features = false, features = ["std"] }
 zkvm_interface = { path = "../" }
+cfg-if = "1.0.0"
 
 ethrex-common = { path = "../../../../../common", default-features = false }
 ethrex-rlp = { path = "../../../../../common/rlp" }
diff --git a/crates/l2/prover/zkvm/interface/risc0/src/main.rs b/crates/l2/prover/zkvm/interface/risc0/src/main.rs
index e73ca2b27..7b8eb5f7c 100644
--- a/crates/l2/prover/zkvm/interface/risc0/src/main.rs
+++ b/crates/l2/prover/zkvm/interface/risc0/src/main.rs
@@ -1,13 +1,16 @@
 use risc0_zkvm::guest::env;
 
 use ethrex_blockchain::{validate_block, validate_gas_used};
-use ethrex_vm::{backends::revm::db::EvmState, backends::revm::REVM};
+use ethrex_vm::{backends::revm::REVM, backends::revm::db::EvmState};
 
 use zkvm_interface::{
     io::{ProgramInput, ProgramOutput},
     trie::{update_tries, verify_db},
 };
 
+#[cfg(feature = "l2")]
+use zkvm_interface::deposits::{get_block_deposits, get_deposit_hash};
+
 fn main() {
     let ProgramInput {
         block,
@@ -58,8 +61,26 @@ fn main() {
         panic!("invalid final state trie");
     }
 
-    env::commit(&ProgramOutput {
-        initial_state_hash,
-        final_state_hash,
-    });
+    cfg_if::cfg_if! {
+        if #[cfg(feature = "l2")] {
+            let deposits = get_block_deposits(&block);
+            let deposit_logs_hash = get_deposit_hash(
+                deposits
+                    .iter()
+                    .filter_map(|tx| tx.get_deposit_hash())
+                    .collect(),
+            ).expect("failed to calculate deposit logs hash");
+
+            env::commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+                deposit_logs_hash
+            });
+        } else {
+            env::commit(&ProgramOutput {
+                initial_state_hash,
+                final_state_hash,
+            });
+        }
+    }
 }

From 16162ffc1449fc51542b99369ae0df5b5c88e400 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Wed, 12 Mar 2025 17:36:30 -0300
Subject: [PATCH 27/29] Update lib.rs

---
 crates/l2/prover/zkvm/interface/src/lib.rs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/crates/l2/prover/zkvm/interface/src/lib.rs b/crates/l2/prover/zkvm/interface/src/lib.rs
index 39614892b..379d216b0 100644
--- a/crates/l2/prover/zkvm/interface/src/lib.rs
+++ b/crates/l2/prover/zkvm/interface/src/lib.rs
@@ -51,7 +51,7 @@ pub mod io {
         /// final state trie root hash
         pub final_state_hash: H256,
         #[cfg(feature = "l2")]
-        /// deposit_logs_hash
+        /// hash of all the deposit logs made in executed block
         pub deposit_logs_hash: H256,
     }
 

From 7d95fe1837ccfd90fcd312f7f2b3a33970bee943 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Thu, 13 Mar 2025 12:30:06 -0300
Subject: [PATCH 28/29] add deposits check on public inputs

---
 .../l2/contracts/src/l1/OnChainProposer.sol   | 24 +++++++++++++++++--
 .../src/l1/interfaces/IOnChainProposer.sol    |  2 +-
 crates/l2/prover/src/backends/risc0.rs        |  6 ++---
 3 files changed, 26 insertions(+), 6 deletions(-)

diff --git a/crates/l2/contracts/src/l1/OnChainProposer.sol b/crates/l2/contracts/src/l1/OnChainProposer.sol
index e7653ca83..1e131aead 100644
--- a/crates/l2/contracts/src/l1/OnChainProposer.sol
+++ b/crates/l2/contracts/src/l1/OnChainProposer.sol
@@ -184,7 +184,7 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
         //risc0
         bytes calldata risc0BlockProof,
         bytes32 risc0ImageId,
-        bytes32 risc0JournalDigest,
+        bytes calldata risc0Journal,
         //sp1
         bytes32 sp1ProgramVKey,
         bytes calldata sp1PublicValues,
@@ -206,6 +206,26 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
             "OnChainProposer: block not committed"
         );
 
+        // Check committed desposits were validated
+
+        // TODO: unify public inputs in single parameter
+        bytes32 risc0DepositsLogHash = bytes32(risc0Journal[64:96]);
+        bytes32 sp1DepositsLogHash = bytes32(sp1PublicValues[64:96]);
+        bytes32 picoDepositsLogHash = bytes32(picoPublicValues[64:96]);
+
+        require(
+            blockCommitments[blockNumber].depositLogs == risc0DepositsLogHash,
+            "OnChainProposer: wrong deposits log hash for risc0 public inputs"
+        );
+        require(
+            blockCommitments[blockNumber].depositLogs == sp1DepositsLogHash,
+            "OnChainProposer: wrong deposits log hash for sp1 public inputs"
+        );
+        require(
+            blockCommitments[blockNumber].depositLogs == picoDepositsLogHash,
+            "OnChainProposer: wrong deposits log hash for pico public inputs"
+        );
+
         if (PICOVERIFIER != DEV_MODE) {
             // If the verification fails, it will revert.
             IPicoVerifier(PICOVERIFIER).verifyPicoProof(
@@ -220,7 +240,7 @@ contract OnChainProposer is IOnChainProposer, ReentrancyGuard {
             IRiscZeroVerifier(R0VERIFIER).verify(
                 risc0BlockProof,
                 risc0ImageId,
-                risc0JournalDigest
+                sha256(risc0Journal)
             );
         }
 
diff --git a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
index 05606b3f8..72c86c70c 100644
--- a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
+++ b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
@@ -72,7 +72,7 @@ interface IOnChainProposer {
         //risc0
         bytes calldata risc0BlockProof,
         bytes32 risc0ImageId,
-        bytes32 risc0JournalDigest,
+        bytes calldata risc0Journal,
         //sp1
         bytes32 sp1ProgramVKey,
         bytes calldata sp1PublicValues,
diff --git a/crates/l2/prover/src/backends/risc0.rs b/crates/l2/prover/src/backends/risc0.rs
index d2355f978..d77b49bfc 100644
--- a/crates/l2/prover/src/backends/risc0.rs
+++ b/crates/l2/prover/src/backends/risc0.rs
@@ -46,7 +46,7 @@ pub fn verify(receipt: &Receipt) -> Result<(), Box<dyn std::error::Error>> {
 pub fn to_calldata(receipt: Receipt) -> Result<ProofCalldata, Box<dyn std::error::Error>> {
     let seal = encode_seal(&receipt)?;
     let image_id = ZKVM_RISC0_PROGRAM_ID;
-    let journal_digest = receipt.journal.digest().as_bytes().to_vec();
+    let journal = receipt.journal.bytes;
 
     // convert image_id into bytes
     let image_id = {
@@ -59,11 +59,11 @@ pub fn to_calldata(receipt: Receipt) -> Result<ProofCalldata, Box<dyn std::error
 
     // bytes calldata seal,
     // bytes32 imageId,
-    // bytes32 journalDigest
+    // bytes journal
     let calldata = vec![
         Value::Bytes(seal.into()),
         Value::FixedBytes(image_id.into()),
-        Value::FixedBytes(journal_digest.into()),
+        Value::Bytes(journal.into()),
     ];
 
     Ok(ProofCalldata {

From f4dafce9fd314de33274b71101dda79941f951ca Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Est=C3=A9fano=20Bargas?= <estefano.bargas@fing.edu.uy>
Date: Thu, 13 Mar 2025 12:30:54 -0300
Subject: [PATCH 29/29] fix docs

---
 crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
index 72c86c70c..030c9715c 100644
--- a/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
+++ b/crates/l2/contracts/src/l1/interfaces/IOnChainProposer.sol
@@ -58,7 +58,7 @@ interface IOnChainProposer {
     /// ----------------------------------------------------------------------
     /// @param risc0BlockProof is the proof of the block to be verified.
     /// @param risc0ImageId Digest of the zkVM imageid.
-    /// @param risc0JournalDigest Digest of the public_inputs aka journal
+    /// @param risc0Journal public_inputs aka journal
     /// ----------------------------------------------------------------------
     /// @param sp1ProgramVKey Public verifying key
     /// @param sp1PublicValues Values used to perform the execution