Skip to content

Commit

Permalink
feat(client/host): Oracle-backed Blob fetcher
Browse files Browse the repository at this point in the history
Implements the oracle-backed blob fetcher for the client program
  • Loading branch information
clabby committed Jun 16, 2024
1 parent 5d84e79 commit fd50500
Show file tree
Hide file tree
Showing 10 changed files with 247 additions and 37 deletions.
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions bin/host/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ kona-client = { path = "../programs/client", version = "0.1.0" }
kona-common = { path = "../../crates/common", version = "0.0.1" }
kona-preimage = { path = "../../crates/preimage", version = "0.0.1" }
kona-mpt = { path = "../../crates/mpt", version = "0.0.1" }
kona-derive = { path = "../../crates/derive", version = "0.0.1", features = ["online"] }

# external
alloy-provider = { git = "https://github.com/alloy-rs/alloy", rev = "cb95183" }
Expand Down
76 changes: 73 additions & 3 deletions bin/host/src/fetcher/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,13 +3,17 @@
use crate::{kv::KeyValueStore, util};
use alloy_consensus::{Header, TxEnvelope};
use alloy_eips::eip2718::Encodable2718;
use alloy_eips::{eip2718::Encodable2718, eip4844::FIELD_ELEMENTS_PER_BLOB};
use alloy_primitives::{address, keccak256, Address, Bytes, B256};
use alloy_provider::{Provider, ReqwestProvider};
use alloy_rlp::Decodable;
use alloy_rpc_types::{Block, BlockTransactions};
use anyhow::{anyhow, Result};
use kona_client::HintType;
use kona_derive::{
online::{OnlineBeaconClient, OnlineBlobProvider, SimpleSlotDerivation},
types::{BlockInfo, IndexedBlobHash},
};
use kona_preimage::{PreimageKey, PreimageKeyType};
use std::sync::Arc;
use tokio::sync::RwLock;
Expand All @@ -26,6 +30,8 @@ where
kv_store: Arc<RwLock<KV>>,
/// L1 chain provider.
l1_provider: ReqwestProvider,
/// The blob provider
blob_provider: OnlineBlobProvider<OnlineBeaconClient, SimpleSlotDerivation>,
/// L2 chain provider.
/// TODO: OP provider, N = Optimism
l2_provider: ReqwestProvider,
Expand All @@ -43,10 +49,11 @@ where
pub fn new(
kv_store: Arc<RwLock<KV>>,
l1_provider: ReqwestProvider,
blob_provider: OnlineBlobProvider<OnlineBeaconClient, SimpleSlotDerivation>,
l2_provider: ReqwestProvider,
l2_head: B256,
) -> Self {
Self { kv_store, l1_provider, l2_provider, l2_head, last_hint: None }
Self { kv_store, l1_provider, blob_provider, l2_provider, l2_head, last_hint: None }
}

/// Set the last hint to be received.
Expand Down Expand Up @@ -149,7 +156,70 @@ where
.map_err(|e| anyhow!(e))?;
self.store_trie_nodes(raw_receipts.as_slice()).await?;
}
HintType::L1Blob => todo!(),
HintType::L1Blob => {
if hint_data.len() != 48 {
anyhow::bail!("Invalid hint data length: {}", hint_data.len());
}

let hash: B256 = hint_data[0..32]
.as_ref()
.try_into()
.map_err(|e| anyhow!("Failed to convert bytes to B256: {e}"))?;
let index = u64::from_be_bytes(
hint_data[32..40]
.as_ref()
.try_into()
.map_err(|e| anyhow!("Failed to convert bytes to u64: {e}"))?,
);
let timestamp = u64::from_be_bytes(
hint_data[40..48]
.as_ref()
.try_into()
.map_err(|e| anyhow!("Failed to convert bytes to u64: {e}"))?,
);

let partial_block_ref = BlockInfo { timestamp, ..Default::default() };
let indexed_hash = IndexedBlobHash { index: index as usize, hash };

// Fetch the blob sidecar from the blob provider.
let mut sidecars = self
.blob_provider
.fetch_filtered_sidecars(&partial_block_ref, &[indexed_hash])
.await
.map_err(|e| anyhow!("Failed to fetch blob sidecars: {e}"))?;
if sidecars.len() != 1 {
anyhow::bail!("Expected 1 sidecar, got {}", sidecars.len());
}
let sidecar = sidecars.remove(0);

// Acquire a lock on the key-value store and set the preimages.
let mut kv_write_lock = self.kv_store.write().await;

// Set the preimage for the blob commitment.
kv_write_lock.set(
PreimageKey::new(*hash, PreimageKeyType::Sha256).into(),
sidecar.kzg_commitment.to_vec(),
);

// Write all the field elements to the key-value store. There should be 4096.
// The preimage oracle key for each field element is the keccak256 hash of
// `abi.encodePacked(sidecar.KZGCommitment, uint256(i))`
let mut blob_key = [0u8; 80];
blob_key[..48].copy_from_slice(sidecar.kzg_commitment.as_ref());
for i in 0..FIELD_ELEMENTS_PER_BLOB {
blob_key[72..].copy_from_slice(i.to_be_bytes().as_ref());
let blob_key_hash = keccak256(blob_key.as_ref());

kv_write_lock.set(
PreimageKey::new(*blob_key_hash, PreimageKeyType::Keccak256).into(),
blob_key.into(),
);
kv_write_lock.set(
PreimageKey::new(*blob_key_hash, PreimageKeyType::Blob).into(),
sidecar.blob[(i as usize) << 5..(i as usize + 1) << 5].to_vec(),
);
}
}
HintType::L1Precompile => {
// Validate the hint data length.
if hint_data.len() < 20 {
Expand Down
35 changes: 33 additions & 2 deletions bin/host/src/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ use command_fds::{CommandFdExt, FdMapping};
use fetcher::Fetcher;
use futures::FutureExt;
use kona_common::FileDescriptor;
use kona_derive::online::{OnlineBeaconClient, OnlineBlobProvider};
use kona_preimage::{HintReader, OracleServer, PipeHandle};
use kv::KeyValueStore;
use std::{
Expand Down Expand Up @@ -59,10 +60,25 @@ async fn start_server(cfg: HostCli) -> Result<()> {

let kv_store = cfg.construct_kv_store();

let beacon_client = OnlineBeaconClient::new_http(
cfg.l1_beacon_address.clone().expect("Beacon API URL must be set"),
);
let mut blob_provider = OnlineBlobProvider::new(beacon_client, None, None);
blob_provider
.load_configs()
.await
.map_err(|e| anyhow!("Failed to load blob provider configuration: {e}"))?;

let fetcher = (!cfg.is_offline()).then(|| {
let l1_provider = util::http_provider(&cfg.l1_node_address.expect("Provider must be set"));
let l2_provider = util::http_provider(&cfg.l2_node_address.expect("Provider must be set"));
Arc::new(RwLock::new(Fetcher::new(kv_store.clone(), l1_provider, l2_provider, cfg.l2_head)))
Arc::new(RwLock::new(Fetcher::new(
kv_store.clone(),
l1_provider,
blob_provider,
l2_provider,
cfg.l2_head,
)))
});

// Start the server and wait for it to complete.
Expand All @@ -80,12 +96,27 @@ async fn start_server_and_native_client(cfg: HostCli) -> Result<()> {
let (preimage_pipe, hint_pipe, files) = util::create_native_pipes()?;
let kv_store = cfg.construct_kv_store();

let beacon_client = OnlineBeaconClient::new_http(
cfg.l1_beacon_address.clone().expect("Beacon API URL must be set"),
);
let mut blob_provider = OnlineBlobProvider::new(beacon_client, None, None);
blob_provider
.load_configs()
.await
.map_err(|e| anyhow!("Failed to load blob provider configuration: {e}"))?;

let fetcher = (!cfg.is_offline()).then(|| {
let l1_provider =
util::http_provider(cfg.l1_node_address.as_ref().expect("Provider must be set"));
let l2_provider =
util::http_provider(cfg.l2_node_address.as_ref().expect("Provider must be set"));
Arc::new(RwLock::new(Fetcher::new(kv_store.clone(), l1_provider, l2_provider, cfg.l2_head)))
Arc::new(RwLock::new(Fetcher::new(
kv_store.clone(),
l1_provider,
blob_provider,
l2_provider,
cfg.l2_head,
)))
});

// Create the server and start it.
Expand Down
3 changes: 1 addition & 2 deletions bin/programs/client/src/hint.rs
Original file line number Diff line number Diff line change
@@ -1,9 +1,8 @@
//! This module contains the [HintType] enum.
use core::fmt::Display;

use alloc::{string::String, vec::Vec};
use alloy_primitives::hex;
use core::fmt::Display;

/// The [HintType] enum is used to specify the type of hint that was received.
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)]
Expand Down
90 changes: 90 additions & 0 deletions bin/programs/client/src/l1/blob_provider.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
//! Contains the concrete implementation of the [BlobProvider] trait for the client program.
use crate::{CachingOracle, HintType, HINT_WRITER};
use alloc::{boxed::Box, sync::Arc, vec::Vec};
use alloy_consensus::Blob;
use alloy_eips::eip4844::FIELD_ELEMENTS_PER_BLOB;
use alloy_primitives::keccak256;
use async_trait::async_trait;
use kona_derive::{
traits::BlobProvider,
types::{BlobProviderError, IndexedBlobHash},
};
use kona_preimage::{HintWriterClient, PreimageKey, PreimageKeyType, PreimageOracleClient};
use kona_primitives::BlockInfo;

/// An oracle-backed blob provider.
#[derive(Debug)]
pub struct OracleBlobProvider {
oracle: Arc<CachingOracle>,
}

impl OracleBlobProvider {
/// Constructs a new `OracleBlobProvider`.
pub fn new(oracle: Arc<CachingOracle>) -> Self {
Self { oracle }
}

/// Retrieves a blob from the oracle.
///
/// ## Takes
/// - `block_ref`: The block reference.
/// - `blob_hash`: The blob hash.
///
/// ## Returns
/// - `Ok(blob)`: The blob.
/// - `Err(e)`: The blob could not be retrieved.
async fn get_blob(
&self,
block_ref: &BlockInfo,
blob_hash: &IndexedBlobHash,
) -> Result<Blob, BlobProviderError> {
let mut blob_req_meta = [0u8; 48];
blob_req_meta[0..32].copy_from_slice(blob_hash.hash.as_ref());
blob_req_meta[32..40].copy_from_slice((blob_hash.index as u64).to_be_bytes().as_ref());
blob_req_meta[40..48].copy_from_slice(block_ref.timestamp.to_be_bytes().as_ref());

// Send a hint for the blob commitment and field elements.
HINT_WRITER.write(&HintType::L1Blob.encode_with(&[blob_req_meta.as_ref()])).await?;

// Fetch the blob commitment.
let mut commitment = [0u8; 48];
self.oracle
.get_exact(PreimageKey::new(*blob_hash.hash, PreimageKeyType::Sha256), &mut commitment)
.await?;

// Reconstruct the blob from the 4096 field elements.
let mut blob = Blob::default();
let mut field_element_key = [0u8; 80];
field_element_key[..48].copy_from_slice(commitment.as_ref());
for i in 0..FIELD_ELEMENTS_PER_BLOB {
field_element_key[72..].copy_from_slice(i.to_be_bytes().as_ref());

let mut field_element = [0u8; 32];
self.oracle
.get_exact(
PreimageKey::new(*keccak256(field_element_key), PreimageKeyType::Blob),
&mut field_element,
)
.await?;
blob[(i as usize) << 5..(i as usize + 1) << 5].copy_from_slice(field_element.as_ref());
}

Ok(blob)
}
}

#[async_trait]
impl BlobProvider for OracleBlobProvider {
async fn get_blobs(
&mut self,
block_ref: &BlockInfo,
blob_hashes: &[IndexedBlobHash],
) -> Result<Vec<Blob>, BlobProviderError> {
let mut blobs = Vec::with_capacity(blob_hashes.len());
for hash in blob_hashes {
blobs.push(self.get_blob(block_ref, hash).await?);
}
Ok(blobs)
}
}
3 changes: 3 additions & 0 deletions bin/programs/client/src/l1/mod.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,7 @@
//! Contains the L1 constructs of the client program.
mod blob_provider;
pub use blob_provider::OracleBlobProvider;

mod chain_provider;
pub use chain_provider::OracleL1ChainProvider;
2 changes: 1 addition & 1 deletion crates/common/src/io.rs
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ mod native_io {
.map_err(|e| anyhow!("Error writing to buffer to file descriptor: {e}"))?;

// Reset the cursor back to before the data we just wrote for the reader's consumption.
file.seek(SeekFrom::Current(-(buf.len() as i64)))
file.seek(SeekFrom::Current(-(n as i64)))
.map_err(|e| anyhow!("Failed to reset file cursor to 0: {e}"))?;

Ok(n)
Expand Down
2 changes: 1 addition & 1 deletion crates/derive/src/online/beacon_client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ pub trait BeaconClient {
pub struct OnlineBeaconClient {
/// The base URL of the beacon API.
base: String,
/// The inner Ethereum JSON-RPC provider.
/// The inner reqwest client.
inner: Client,
}

Expand Down
Loading

0 comments on commit fd50500

Please sign in to comment.