Kona Book

CI Codecov License Book

warning

kona is in active development, and is not yet ready for use in production. During development, this book will evolve quickly and may contain inaccuracies.

Please open an issue if you find any errors or have any suggestions for improvements, and also feel free to contribute to the project!

Introduction

Originally a suite of portable implementations of the OP Stack rollup state transition, Kona has been extended to be the monorepo for OP Stack types, components, and services built in Rust. Kona provides an ecosystem of extensible, low-level crates that compose into components and services required for the OP Stack.

Protocol crates are no_std compatible for use within the Fault Proof. Types defined in these libraries are shared by other components of the OP Stack as well including the rollup node.

Proof crates are available for developing verifiable Rust programs targeting Fault Proof VMs . These libraries provide tooling and abstractions around low-level syscalls, memory management, and other common structures that authors of verifiable programs will need to interact with. It also provides build pipelines for compiling no_std Rust programs to a format that can be executed by supported Fault Proof VM targets.

Kona is built and maintained by open source contributors and is licensed under the MIT License.

Goals of Kona

1. Composability

Kona provides a common set of tools and abstractions for developing verifiable Rust programs on top of several supported Fault Proof VM targets. This is done to ensure that programs written for one supported FPVM can be easily ported to another supported FPVM, and that the ecosystem of programs built on top of these targets can be easily shared and reused.

2. Safety

Through standardization of these low-level system interfaces and build pipelines, Kona seeks to increase coverage over the low-level operations that are required to build on top of a FPVM.

3. Developer Experience

Building on top of custom Rust targets can be difficult, especially when the target is nascent and tooling is not yet mature. Kona seeks to improve this experience by standardizing and streamlining the process of developing and compiling verifiable Rust programs, targeted at supported FPVMs.

4. Performance

Kona is opinionated in that it favors no_std Rust programs for embedded FPVM development, for both performance and portability. In contrast with alternative approaches, such as the op-program using the Golang MIPS64 target, no_std Rust programs produce much smaller binaries, resulting in fewer instructions that need to be executed on the FPVM. In addition, this offers developers more low-level control over interactions with the FPVM kernel, which can be useful for optimizing performance-critical code.

Development Status

Kona is currently in active development, and is not yet ready for use in production.

Contributing

Contributors are welcome! Please see the contributing guide for more information.

Kona Protocol Libraries

The Kona monorepo contains a set of protocol crates that are designed to be no_std compatible for Kona's fault proof sdk. Protocol crates are built on alloy and op-alloy types.

The following protocol crates are published to crates.io.

kona-hardforks crate kona-registry crate kona-protocol crate kona-genesis crate kona-interop crate kona-derive crate kona-driver crate

At the lowest level, kona-genesis and kona-hardforks expose core genesis and hardfork types.

kona-protocol sits just above kona-genesis, composing genesis types into other core protocol types, as well as many independent protocol types.

More recently, the kona-interop crate was introduced that contains types specific to Interop.

kona-registry contains bindings to the superchain-registry. The registry is available in a no_std environment but requires serde to read serialized configs at compile time. kona-registry uses types defined in kona-genesis to deserialize the superchain registry configs at compile time.

The kona-derive Derivation Pipeline

kona-derive defines an entirely trait-abstracted, no_std derivation pipeline for the OP Stack. It can be used through the Pipeline trait, which is implemented for the concrete DerivationPipeline object.

This document dives into the inner workings of the derivation pipeline, its stages, and how to build and interface with Kona's pipeline. Other documents in this section will provide a comprehensive overview of Derivation Pipeline extensibility including trait-abstracted providers, custom stages, signaling, and hardfork activation including multiplexed stages.

What is a Derivation Pipeline?

Simply put, an OP Stack Derivation Pipeline transforms data on L1 into L2 payload attributes that can be executed to produce the canonical L2 block.

Within a pipeline, there are a set of stages that break up this transformation further. When composed, these stages operate over the input data, sequentially producing payload attributes.

In kona-derive, stages are architected using composition - each sequential stage owns the previous one, forming a stack. For example, let's define stage A as the first stage, accepting raw L1 input data, and stage C produces the pipeline output - payload attributes. Stage B "owns" stage A, and stage C then owns stage B. Using this example, the DerivationPipeline type in kona-derive only holds stage C, since ownership of the other stages is nested within stage C.

note

In a future architecture of the derivation pipeline, stages could be made standalone such that communication between stages happens through channels. In a multi-threaded, non-fault-proof environment, these stages can then run in parallel since stage ownership is decoupled.

Kona's Derivation Pipeline

The top-level stage in kona-derive that produces OpAttributesWithParent is the AttributesQueue.

Post-Holocene (the Holocene hardfork), the following stages are composed by the DerivationPipeline.

Notice, from top to bottom, each stage owns the stage nested below it. Where the L1Traversal stage iterates over L1 data, the AttributesQueue stage produces OpAttributesWithParent, creating a function that transforms L1 data into payload attributes.

The Pipeline interface

Now that we've broken down the stages inside the DerivationPipeline type, let's move up another level to break down how the DerivationPipeline type functions itself. At the highest level, kona-derive defines the interface for working with the pipeline through the Pipeline trait.

Pipeline provides two core methods.

  • peek() -> Option<&OpAttributesWithParent>
  • async step() -> StepResult

Functionally, a pipeline can be "stepped" on, which attempts to derive payload attributes from input data. Steps do not guarantee that payload attributes are produced, they only attempt to advance the stages within the pipeline.

The peek() method provides a way to check if attributes are prepared. Beyond peek() returning Option::Some(&OpAttributesWithParent), the Pipeline extends the Iterator trait, providing a way to consume the generated payload attributes.

Constructing a Derivation Pipeline

kona-derive provides a PipelineBuilder to abstract the complexity of generics away from the downstream consumers. Below we provide an example for using the PipelineBuilder to instantiate a DerivationPipeline.

// Imports
use std::sync::Arc;
use kona_protocol::BlockInfo;
use kona_genesis::RollupConfig;
use kona_providers_alloy::*;

// Use a default rollup config.
let rollup_config = Arc::new(RollupConfig::default());

// Providers are instantiated to with localhost urls (`127.0.0.1`)
let chain_provider =
    AlloyChainProvider::new_http("http://127.0.0.1:8545".try_into().unwrap());
let l2_chain_provider = AlloyL2ChainProvider::new_http(
    "http://127.0.0.1:9545".try_into().unwrap(),
    rollup_config.clone(),
);
let beacon_client = OnlineBeaconClient::new_http("http://127.0.0.1:5555".into());
let blob_provider = OnlineBlobProvider::new(beacon_client, None, None);
let blob_provider = OnlineBlobProviderWithFallback::new(blob_provider, None);
let dap_source =
    EthereumDataSource::new(chain_provider.clone(), blob_provider, &rollup_config);
let builder = StatefulAttributesBuilder::new(
    rollup_config.clone(),
    l2_chain_provider.clone(),
    chain_provider.clone(),
);

// This is the starting L1 block for the pipeline.
//
// To get the starting L1 block for a given L2 block,
// use the `AlloyL2ChainProvider::l2_block_info_by_number`
// method to get the `L2BlockInfo.l1_origin`. This l1_origin
// is the origin that can be passed here.
let origin = BlockInfo::default();

// Build the pipeline using the `PipelineBuilder`.
// Alternatively, use the `new_online_pipeline` helper
// method provided by the `kona-derive-alloy` crate.
let pipeline = PipelineBuilder::new()
   .rollup_config(rollup_config.clone())
   .dap_source(dap_source)
   .l2_chain_provider(l2_chain_provider)
   .chain_provider(chain_provider)
   .builder(builder)
   .origin(origin)
   .build();

assert_eq!(pipeline.rollup_config, rollup_config);
assert_eq!(pipeline.origin(), Some(origin));

Producing Payload Attributes

Since the Pipeline trait extends the Iterator trait, producing OpAttributesWithParent is as simple as as calling Iterator::next() method on the DerivationPipeline.

Extending the example from above, producing the attributes is shown below.

#![allow(unused)]
fn main() {
// Import the iterator trait to show where `.next` is sourced.
use core::iter::Iterator;

// ...
// example from above constructing the pipeline
// ...

let attributes = pipeline.next();

// Since we haven't stepped on the pipeline,
// there shouldn't be any payload attributes prepared.
assert!(attributes.is_none());
}

As demonstrated, the pipeline won't have any payload attributes without having been "stepped" on. Naively, we can continuously step on the pipeline until attributes are ready, and then consume them.

#![allow(unused)]
fn main() {
// Import the iterator trait to show where `.next` is sourced.
use core::iter::Iterator;

// ...
// example from constructing the pipeline
// ...

// Continuously step on the pipeline until attributes are prepared.
let l2_safe_head = L2BlockInfo::default();
loop {
   if matches!(pipeline.step(l2_safe_head).await, StepResult::PreparedAttributes) {
      // The pipeline has successfully prepared payload attributes, break the loop.
      break;
   }
}

// Since the loop is only broken once attributes are prepared,
// this must be `Option::Some`.
let attributes = pipeline.next().expect("Must contain payload attributes");

// The parent of the prepared payload attributes should be
// the l2 safe head that we "stepped on".
assert_eq!(attributes.parent, l2_safe_head);
}

Importantly, the above is not sufficient logic to produce payload attributes and drive the derivation pipeline. There are multiple different StepResults to handle when stepping on the pipeline, including advancing the origin, re-orgs, and pipeline resets. In the next section, pipeline resets are outlined.

For an up-to-date driver that runs the derivation pipeline as part of the fault proof program, reference kona's client driver.

Resets

When stepping on the DerivationPipeline produces a reset error, the driver of the pipeline must perform a reset on the pipeline. This is done by sending a "signal" through the DerivationPipeline. Below demonstrates this.

#![allow(unused)]
fn main() {
// Import the iterator trait to show where `.next` is sourced.
use core::iter::Iterator;

// ...
// example from constructing the pipeline
// ...

// Continuously step on the pipeline until attributes are prepared.
let l2_safe_head = L2BlockInfo::default();
loop {
   match pipeline.step(l2_safe_head).await {
      StepResult::StepFailed(e) | StepResult::OriginAdvanceErr(e) => {
         match e {
            PipelineErrorKind::Reset(e) => {
               // Get the system config from the provider.
               let system_config = l2_chain_provider
                  .system_config_by_number(
                     l2_safe_head.block_info.number,
                     rollup_config.clone(),
                  )
                  .await?;
               // Reset the pipeline to the initial L2 safe head and L1 origin.
               self.pipeline
                  .signal(
                      ResetSignal {
                          l2_safe_head: l2_safe_head,
                          l1_origin: pipeline
                              .origin()
                              .ok_or_else(|| anyhow!("Missing L1 origin"))?,
                          system_config: Some(system_config),
                      }
                      .signal(),
                  )
                  .await?;
               // ...
            }
            _ => { /* Handling left to the driver */ }
         }
      }
      _ => { /* Handling left to the driver */ }
   }
}
}

Learn More

kona-derive is one implementation of the OP Stack derivation pipeline.

To learn more, it is highly encouraged to read the "first" derivation pipeline written in golang. It is often colloquially referred to as the "reference" implementation and provides the basis for how much of Kona's derivation pipeline was built.

Provenance

The lore do be bountiful.

  • Bard XVIII of the Logic Gates

The kona project spawned out of the need to build a secondary fault proof for the OP Stack. Initially, we sought to re-use magi's derivation pipeline, but the ethereum-rust ecosystem moves quickly and magi was behind by a generation of types - using ethers-rs instead of new alloy types. Additionally, magi's derivation pipeline was not no_std compatible - a hard requirement for running a rust fault proof program on top of the RISCV or MIPS ISAs.

So, @clabby and @refcell stood up kona in a few months.

Trait-abstracted Providers

Kona's derivation pipeline pulls in data from sources that are trait abstracted so the pipeline can be generic over various data sources. Note, "data sources" is used interchangeably with "trait-abstracted providers" for the purpose of this document.

The key traits required for the pipeline are the following.

The kona-derive-alloy crate provides std implementations of these traits using Alloy's reqwest-backed providers.

Provider Usage

Although trait-abstracted Providers are used throughout the pipeline and its stages, the PipelineBuilder makes constructing the pipeline generic over the providers. An example is shown below, where the three required trait implementations are the providers stubbed with todo!().

#![allow(unused)]
fn main() {
use std::sync::Arc;
use kona_genesis::RollupConfig;
use kona_derive::pipeline::PipelineBuilder;
use kona_derive::attributes::StatefulAttributesBuilder;

// The rollup config for your chain.
let cfg = Arc::new(RollupConfig::default());

// Must implement the `ChainProvider` trait.
let chain_provider = todo!("your chain provider");

// Must implement the `L2ChainProvider` trait.
let l2_chain_provider = todo!("your l2 chain provider");

// Must implement the `DataAvailabilityProvider` trait.
let dap = todo!("your data availability provider");

// Generic over the providers.
let attributes = StatefulAttributesBuilder::new(
   cfg.clone(),
   l2_chain_provider.clone(),
   chain_provider.clone(),
);

// Construct a new derivation pipeline.
let pipeline = PipelineBuilder::new()
   .rollup_config(cfg)
   .dap_source(dap)
   .l2_chain_provider(l2_chain_provider)
   .chain_provider(chain_provider)
   .builder(attributes)
   .origin(BlockInfo::default())
   .build();
}

Implementing a Custom Data Availability Provider

Notice

The only required method for the DataAvailabilityProvider trait is the next method.

#![allow(unused)]
fn main() {
use async_trait::async_trait;
use alloy_primitives::Bytes;
use kona_protocol::BlockInfo;
use kona_derive::traits::DataAvailabilityProvider;
use kona_derive::errors::PipelineResult;

/// ExampleAvail
///
/// An example implementation of the `DataAvailabilityProvider` trait.
#[derive(Debug)]
pub struct ExampleAvail {
   // Place your data in here
}

#[async_trait]
impl DataAvailabilityProvider for ExampleAvail {
   type Item = Bytes;

   async fn next(&self, block_ref: &BlockInfo) -> PipelineResult<Self::Item> {
      todo!("return an AsyncIterator implementation here")
   }
}
}

Swapping out a Stage

In the introduction to the derivation pipeline, the derivation pipeline is broken down to demonstrate the composition of stages, forming the transformation function from L1 data into L2 payload attributes.

What makes kona's derivation pipeline extensible is that stages are composed using trait-abstraction. That is, each successive stage composes the previous stage as a generic. As such as long as a stage satisfies two rules, it can be swapped into the pipeline seamlessly.

  1. The stage implements the trait required by the next stage.
  2. The stage uses the same trait for the previous stage as the current stage to be swapped out.

Below provides a concrete example, swapping out the L1Retrieval stage.

Example

In the current, post-Holocene hardfork DerivationPipeline, the bottom three stages of the pipeline are as follows (from top down).

In this set of stages, the L1Traversal stage sits at the bottom. It implements the L1Retrieval trait called the L1RetrievalProvider. This provides generic methods that allow the L1Retrieval stage to call those methods on the generic previous stage that implements this provider trait.

As we go up a level, the same trait abstraction occurs. The L1Retrieval stage implements the provider trait that the FrameQueue stage requires. This trait is the FrameQueueProvider.

Now that we understand the trait abstractions, let's swap out the L1Retrieval stage for a custom DapRetrieval stage.

#![allow(unused)]
fn main() {
// ...
// imports
// ...

// We use the same "L1RetrievalProvider" trait here
// in order to seamlessly use the `L1Traversal`

/// DapRetrieval stage
#[derive(Debug)]
pub struct DapRetrieval<P>
where
    P: L1RetrievalProvider + OriginAdvancer + OriginProvider + SignalReceiver,
{
    /// The previous stage in the pipeline.
    pub prev: P,
    provider: YourDataAvailabilityProvider,
    data: Option<Bytes>,
}

#[async_trait]
impl<P> FrameQueueProvider for DapRetrieval<P>
where
    P: L1RetrievalProvider + OriginAdvancer + OriginProvider + SignalReceiver + Send,
{
    type Item = Bytes;

    async fn next_data(&mut self) -> PipelineResult<Self::Item> {
        if self.data.is_none() {
            let next = self
                .prev
                .next_l1_block()
                .await? // SAFETY: This question mark bubbles up the Eof error.
                .ok_or(PipelineError::MissingL1Data.temp())?;
            self.data = Some(self.provider.get_data(&next).await?);
        }

        match self.data.as_mut().expect("Cannot be None").next().await {
            Ok(data) => Ok(data),
            Err(e) => {
                if let PipelineErrorKind::Temporary(PipelineError::Eof) = e {
                    self.data = None;
                }
                Err(e)
            }
        }
    }
}

// ...
// impl OriginAdvancer for DapRetrieval
// impl OriginProvider for DapRetrieval
// impl SignalReceiver for DapRetrieval
// ..
}

Notice, the L1RetrievalProvider is used as a trait bound so the L1Traversal stage can be used seamlessly as the "prev" stage in the pipeline. Concretely, an instantiation of the DapRetrieval stage could be the following.

DapRetrieval<L1Traversal<..>>

Signals

Understanding signals first require a more in-depth review of the result returned by stepping on the derivation pipeline.

The StepResult

As briefly outlined in the intro, stepping on the derivation pipeline returns a StepResult. Step results provide a an extensible way for pipeline stages to signal different results to the pipeline driver. The variants of StepResult and what they signal include the following.

  • StepResult::PreparedAttributes - signals that payload attributes are ready to be consumed by the pipeline driver.
  • StepResult::AdvancedOrigin - signals that the pipeline has derived all payload attributes for the given L1 block, and the origin of the pipeline was advanced to the next canonical L1 block.
  • StepResult::OriginAdvanceErr(_) - The driver failed to advance the origin of pipeline.
  • StepResult::StepFailed(_) - The step failed.

No action is needed when the prepared attributes step result is received. The pipeline driver may chose to consume the payload attributes how it wishes. Likewise, StepResult::AdvancedOrigin simply notifies the driver that the pipeline advanced its origin - the driver may continue stepping on the pipeline. Now, it becomes more involved with the remaining two variants of StepResult.

When either StepResult::OriginAdvanceErr(_) or StepResult::StepFailed(_) are received, the pipeline driver needs to introspect the error within these variants. Depending on the PipelineErrorKind, the driver may need to send a "signal" down through the pipeline.

The next section goes over pipeline signals by looking at the variants of the PipelineErrorKind and the driver's response.

PipelineErrorKind

There are three variants of the PipelineErrorKind, each groups the inner error based on severity (or how they should be handled).

  • PipelineErrorKind::Temporary - This is an error that's expected, and is temporary. For example, not all channel data has been posted to L1 so the pipeline doesn't have enough data yet to continue deriving payload attributes.
  • PipelineErrorKind::Critical - This is an unexpected error that breaks the derivation pipeline. It should cause the driver to error since this is behavior that is breaking the derivation of payload attributes.
  • PipelineErrorKind::Reset - When this is received, it effectively requests that the driver perform some action on the pipeline. Kona uses message passing so the driver can send a Signal down the pipeline with whatever action that needs to be performed. By allowing both the driver and individual pipeline stages to define their own behaviour around signals, they become very extensible. More on this in a later section.

The Signal Type

Continuing from the PipelineErrorKind, when the driver receives a PipelineErrorKind::Reset, it needs to send a signal down through the pipeline.

Prior to the Holocene hardfork, the pipeline only needed to be reset when the reset pipeline error was received. Holocene activation rules changed this to require Holocene-specific activation logic internal to the pipeline stages. The way kona's driver handles this activation is by sending a new ActivationSignal if the PipelineErrorKind::Reset type is a ResetError::HoloceneActivation. Otherwise, it will send the ResetSignal.

The last of the three Signal variants is the FlushChannel signal. Similar to ActivationSignal, the flush channel signal is logic introduced post-Holocene. When the driver fails to execute payload attributes and Holocene is active, a FlushChannel signal needs to forwards invalidate the associated batch and channel, and the block is replaced with a deposit-only block.

Extending the Signal Type

To extend the Signal type, all that is needed is to introduce a new variant to the Signal enum.

Once the variant is added, the segments where signals are handled need to be updated. Anywhere the SignalReceiver trait is implemented, handling needs to be updated for the new signal variant. Most notably, this is on the top-level DerivationPipeline type, as well as all the pipeline stages.

An Example

Let's create a new Signal variant that updates the RollupConfig in the L1Traversal stage. Let's call it SetConfig. The signal type would look like the following with this new variant.

#![allow(unused)]
fn main() {
/// A signal to send to the pipeline.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
#[allow(clippy::large_enum_variant)]
pub enum Signal {
    /// Reset the pipeline.
    Reset(ResetSignal),
    /// Hardfork Activation.
    Activation(ActivationSignal),
    /// Flush the currently active channel.
    FlushChannel,
    /// Updates the rollup config in the L1Traversal stage.
    UpdateConfig(ConfigUpdateSignal),
}

/// A signal that updates the `RollupConfig`.
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
pub struct ConfigUpdateSignal(Arc<RollupConfig>);
}

Next, all handling of the Signal type needs to be updated for the new UpdateConfig variant. For the sake of this example, we'll just focus on updating the L1Traversal stage.

#![allow(unused)]
fn main() {
#[async_trait]
impl<F: ChainProvider + Send> SignalReceiver for L1Traversal<F> {
    async fn signal(&mut self, signal: Signal) -> PipelineResult<()> {
        match signal {
            Signal::Reset(ResetSignal { l1_origin, system_config, .. }) |
            Signal::Activation(ActivationSignal { l1_origin, system_config, .. }) => {
                self.block = Some(l1_origin);
                self.done = false;
                self.system_config = system_config.expect("System config must be provided.");
            }
            Signal::UpdateConfig(inner) => {
               self.rollup_config = Arc::clone(&inner.0);
            }
            _ => {}
        }

        Ok(())
    }
}
}

Genesis

kona-genesis crate

The genesis crate contains types related to chain genesis.

This section contains in-depth sections on building with [kona-genesis][genesis] crate types.

Rollup Configs

Rollup configurations are a consensus construct used to configure an Optimism Consensus client. When an OP Stack chain is deployed into production or consensus nodes are configured to sync the chain, certain consensus parameters can be configured. These parameters are defined in the [OP Stack specs][spec-configurability].

Consensus parameters are consumed by OP Stack software through the RollupConfig type defined in the [kona-genesis][genesis] crate.

RollupConfig Type

The [RollupConfig][rc] type is defined in [kona-genesis][genesis].

Rollup configs can be loaded for a given chain id using [kona-registry][registry]. The ROLLUP_CONFIG mapping in the kona-registry provides a mapping from chain ids to rollup config.

System Config

The system configuration is a set of configurable chain parameters defined in a contract on L1. These parameters can be changed through the system config contract, emitting events that are picked up by the [rollup node derivation process][derivation]. To dive deeper into the System Config, visit the [OP Stack Specifications][system-config-specs].

SystemConfig Type

The [SystemConfig][system-config] type is defined in [kona-genesis][genesis].

Parameters defined in the [SystemConfig][system-config] are expected to be updated through L1 receipts, using the [update_with_receipts][update] method.

Holocene Updates

The [Holocene Hardfork][holocene] introduced an update to the [SystemConfig][system-config] type, adding EIP-1559 parameters to the config.

The [SystemConfig][system-config] type in [kona-genesis][genesis] provides a method called [eip_1559_params][eip] that returns the EIP-1559 parameters encoded as a [B64][b64].

Registry

kona-registry crate

kona-registry is a no_std crate that exports rust type definitions for chains in the superchain-registry. These are lazily evaluated statics that provide ChainConfigs, RollupConfigs, and Chain objects for all chains with static definitions in the superchain-registry.

Since it reads static files to read configurations for various chains into instantiated objects, the kona-registry crate requires serde as a dependency.

There are three core statics exposed by the kona-registry.

  • CHAINS: A list of chain objects containing the superchain metadata for this chain.
  • OPCHAINS: A map from chain id to ChainConfig.
  • ROLLUP_CONFIGS: A map from chain id to RollupConfig.

kona-registry exports the complete list of chains within the superchain, as well as each chain's RollupConfigs and ChainConfigs.

Usage

Add the following to your Cargo.toml.

[dependencies]
kona-registry = "0.1.0"

To make kona-registry no_std, toggle default-features off like so.

[dependencies]
kona-registry = { version = "0.1.0", default-features = false }

Below demonstrates getting the RollupConfig for OP Mainnet (Chain ID 10).

#![allow(unused)]
fn main() {
use kona_registry::ROLLUP_CONFIGS;

let op_chain_id = 10;
let op_rollup_config = ROLLUP_CONFIGS.get(&op_chain_id);
println!("OP Mainnet Rollup Config: {:?}", op_rollup_config);
}

A mapping from chain id to ChainConfig is also available.

#![allow(unused)]
fn main() {
use kona_registry::OPCHAINS;

let op_chain_id = 10;
let op_chain_config = OPCHAINS.get(&op_chain_id);
println!("OP Mainnet Chain Config: {:?}", op_chain_config);
}

Interop

kona-interop crate

kona-interop provides core types for the interop protocol.

Hardforks

kona-hardforks crate

Hardforks are consensus layer types of the OP Stack.

kona-hardforks most directly exports the Hardforks type that provides the network upgrade transactions for OP Stack hardforks including the following.

Each hardfork has its own type in kona-hardforks that exposes the network upgrade transactions for that hardfork.

For example, the Ecotone type can be used to retrieve the Ecotone network upgrade transactions through its txs() -> impl Iterator<Bytes> method.

#![allow(unused)]
fn main() {
// Notice, the `Hardfork` trait must be imported in order to
// provide the `txs()` method implemented for the hardfork type.
use kona_hardforks::{Hardfork, Ecotone};
let ecotone_upgrade_txs = Ecotone.txs();
assert_eq!(ecotone_upgrade_txs.collect::<Vec<_>>().len(), 6);
}

Conveniently, the Hardforks type exposes each hardfork type as a field that can be directly referenced without needing to import all the different hardforks.

#![allow(unused)]
fn main() {
use kona_hardforks::{Hardfork, Hardforks};
let ecotone_upgrade_txs = Hardforks::ECOTONE.txs();
assert_eq!(ecotone_upgrade_txs.collect::<Vec<_>>().len(), 6);
}

Protocol

kona-protocol crate

The kona-protocol crate contains types, constants, and methods specific to Optimism derivation and batch-submission.

kona-protocol supports no_std.

Background

Protocol types are primarily used for L2 chain derivation. This section will break down L2 chain derivation as it relates to types defined in kona-protocol - that is, from the raw L2 chain data posted to L1, to the Batch type. And since the Batch type naively breaks up into the payload attributes, once executed, it becomes the canonical L2 block! Note though, this provides an incredibly simplified introduction. It is advised to reference the specs for the most up-to-date information regarding derivation.

The L2 chain is derived from data posted to the L1 chain - either as calldata or blob data. Data is iteratively pulled from each L1 block and translated into the first type defined by kona-protocol: the Frame type.

Frames are parsed from the raw data. Each Frame is a part of a Channel, the next type one level up in deriving L2 blocks. Channels have IDs that frames reference. Frames are added iteratively to the Channel. Once a Channel is ready, it can be used to read a Batch.

Since a Channel stitches together frames, it contains the raw frame data. In order to turn this Channel data into a Batch, it needs to be decompressed using the respective (de)compression algorithm (see the channel specs for more detail on this). Once decompressed, the raw data can be decoded into the Batch type.

Sections

Core Derviation Types (discussed above)

Other Critical Protocol Types

BlockInfo

The BlockInfo type is straightforward, containing the block hash, number, parent hash, and timestamp.

The BlockInfo is a subset of information provided by the block header, used for protocol operations.

L2BlockInfo

The L2BlockInfo extends the BlockInfo type for the canonical L2 chain. It contains the "L1 origin" which is a set of block info for the L1 block that this L2 block "originated".

Similarly to the BlockInfo type, L2BlockInfo is a subset of information provided by a block header, used for protocol operations.

L2BlockInfo provides a from_block_and_gensis method to construct the L2BlockInfo from a block and ChainGenesis.

Frames

Frames are the lowest level data format in the OP Stack protocol.

Where Frames fit in the OP Stack

Transactions posted to the data availability layer of the rollup contain one or multiple Frames. Frames are chunks of raw data that belong to a given Channel, the next, higher up data format in the OP Stack protocol. Importantly, a given transaction can contain a variety of frames from different channels, allowing maximum flexibility when breaking up channels into batcher transactions.

Contents of a Frame

A Frame is comprised of the following items.

  • A ChannelId which is a 16 byte long identifier for the channel that the given frame belongs to.
  • A number that identifies the index of the frame within the channel. Frames are 0-indexed and are bound to u16 size limit.
  • data contains the raw data within the frame.
  • is_last marks if the frame is the last within the channel.

Frame Encoding

When frames are posted through a batcher transaction, they are encoded as a contiguous list with a single byte prefix denoting the derivation version. The encoding can be represented as the following concatenated bytes.

encoded = DERIVATION_VERSION_0 ++ encoded_frame_0 ++ encoded_frame_1 ++ ..

Where DERIVATION_VERSION_0 is a single byte (0x00) indicating the derivation version including how the frames are encoded. Currently, the only supported derivation version is 0.

encoded_frame_0, encoded_frame_1, and so on, are all Frames encoded as raw bytes. A single encoded Frame can be represented by the following concatenation of it's fields.

encoded_frame = channel_id ++ frame_number ++ frame_data_length ++ frame_data ++ is_last

Where ++ represents concatenation. The frame's fields map to it's encoding.

  • channel_id is the 16 byte long Frame::id.
  • frame_number is the 2 byte long (or u16) Frame::number.
  • frame_data_length and frame_data provide the necessary details to decode the Frame::data, where frame_data_length is 4 bytes long (or u32).
  • is_last is a single byte Frame::is_last.

kona's Frame Type

kona-protocol provides the Frame type with a few useful methods. Frames can be encoded and decoded using the Frame::encode and Frame::decode methods. Given the raw batcher transaction data or blob data containing the concatenated derivation version and contiguous list of encoded frames, the Frame::parse_frame and Frame::parse_frames methods provide ways to decode single and multiple frames, respectively.

Channels

Taken from the OP Stack specs, Channels are a set of sequencer batches (for any L2 blocks) compressed together.

Where Channels fit in the OP Stack

L2 transactions are grouped into what are called sequencer batches. In order to obtain a better compression ratio when posting these L2 transactions to the data availability layer, sequencer batches are compressed together into what is called a Channel. This ultimately reduces data availability costs. As previously noted in the Frame section, Channels may not "fit" in a single batcher transaction, posting the data to the data availability layer. In order to accommodate large Channels, a tertiary Frame data type breaks the Channel up into multiple Frames where a batcher transaction then consists of one or multiple Frames.

Contents of a Channel

A Channel is comprised of the following items.

  • A ChannelId which is a 16 byte long identifier for the channel. Notice, Frames also contain a ChannelId, which is the identical to this identifier, since frames "belong" to a given channel.
  • A BlockInfo that marks the L1 block at which the channel is "opened" at.
  • The estimated size of the channel (as a usize) used to drop the channel if there is a data overflow.
  • A boolean if the channel is "closed". This indicates if the last frame has been buffered, and added to the channel.
  • A u16 indicating the highest frame number within the channel.
  • The frame number of the last frame (where is_last set to true).
  • A mapping from Frame number to the Frame itself.
  • A BlockInfo for highest L1 inclusion block that a frame was included in.

Channel Encoding

Channel encoding is even more straightforward than that of a Frame. Simply, a Channel is the concatenated list of encoded Frames.

Since each Frame contains the ChannelId that corresponds to the given Channel, constructing a Channel is as simple as calling the Channel::add_frame method for each of its Frames.

Once the Channel has ingested all of it's Frames, it will be marked as "ready", with the Channel::is_ready method returning true.

The Channel Type

As discussed above, the Channel type is expected to be populated with Frames using its Channel::add_frame method. Below we demonstrate constructing a minimal Channel using a few frames.

#![allow(unused)]
fn main() {
use kona_protocol::{Channel, Frame};

// Construct a channel at the given L1 block.
let id = [0xee; 16];
let block = BlockInfo::default();
let mut channel = Channel::new(id, block);

// The channel will consist of 3 frames.
let frame_0 = Frame { id: [0xee; 16], number: 0, ..Default::default() };
let frame_1 = Frame { id: [0xee; 16], number: 1, ..Default::default() };
let frame_2 = Frame { id: [0xee; 16], number: 2, is_last: true, ..Default::default() };

// Add the frames to the channel.
channel.add_frame(frame_0);
channel.add_frame(frame_1);
channel.add_frame(frame_2);

// Since the last frame was ingested,
// the channel should be ready.
assert!(channel.is_ready());
}

There are a few rules when adding a Frame to a Channel.

Notice, Frames can be added out-of-order so long as the Channel is still open, and the frame hasn't already been added.

Batches

A Batch contains a list of transactions to be included in a specific L2 block. Since the Delta hardfork, there are two Batch types or variants: SingleBatch and SpanBatch.

Where Batches fit in the OP Stack

The Batch is the highest-level data type in the OP Stack derivation process that comes prior to building payload attributes. A Batch is constructed by taking the raw data from a Channel, decompressing it, and decoding the Batch from this decompressed data.

Alternatively, when looking at the Batch type from a batching perspective, and not from the derivation perspective, the Batch type contains a list of L2 transactions and is compressed into the Channel type. In turn, the Channel is split into frames which are posted to the data availability layer through batcher transactions.

Contents of a Batch

A Batch is either a SingleBatch or a SpanBatch, each with their own contents. Below, these types are broken down in their respective sections.

SingleBatch Type

The SingleBatch type contains the following.

  • A BlockHash parent hash that represents the parent L2 block.
  • A u64 epoch number that identifies the epoch for this batch.
  • A BlockHash epoch hash.
  • The timestamp for the batch as a u64.
  • A list of EIP-2718 encoded transactions (represented as Bytes).

In order to validate the SingleBatch once decoded, the SingleBatch::check_batch method should be used, providing the rollup config, l1 blocks, l2 safe head, and inclusion block.

SpanBatch Type

The SpanBatch type (available since the Delta hardfork) comprises the data needed to build a "span" of multiple L2 blocks. It contains the following data.

  • The parent check (the first 20 bytes of the block's parent hash).
  • The l1 origin check (the first 20 bytes of the last block's l1 origin hash).
  • The genesis timestamp.
  • The chain id.
  • A list of SpanBatchElements. These are similar to the SingleBatch type but don't contain the parent hash and epoch hash for this L2 block.
  • Origin bits.
  • Block transaction counts.
  • Span batch transactions which contain information for transactions in a span batch.

Similar to the SingleBatch type discussed above, the SpanBatch type must be validated once decoded. For this, the SpanBatch::check_batch method is available.

After the Holocene hardfork was introduced, span batch validation is greatly simplified to be forwards-invalidating instead of backwards-invalidating, so a new SpanBatch::check_batch_prefix method provides a way to validate each batch as it is loaded, in an iterative fashion.

Batch Encoding

The first byte of the decompressed channel data is the BatchType, which identifies whether the batch is a SingleBatch or a SpanBatch. From there, the respective type is decoded, and derived in the case of the SpanBatch.

The Batch encoding format for the SingleBatch is broken down in the specs.

The Batch Type

The Batch type itself only provides two useful methods.

  • timestamp returns the timestamp of the Batch
  • decode, constructs a new Batch from the provided raw, decompressed batch data and rollup config.

Within each Batch variant, the individual types contain more functionality.

Kona Proof SDK

Welcome to the Kona Proof SDK, a powerful set of libraries designed from first principles to build proofs with the OP Stack STF on top of the OP Stack's FPVMs and other verifiable backends like SP-1, Risc0, Intel TDX, and AMD SEV-SNP. At its core, Kona is built on the principles of modularity, extensibility, and developer empowerment.

A Foundation of Flexibility

The kona repository is more than a fault proof program for the OP Stack — it's an ecosystem of interoperable components, each crafted with reusability and extensibility as primary goals. While we provide Fault Proof VM and "online" backends for key components like kona-derive and kona-executor, the true power of kona lies in its adaptability.

Extend Without Forking

One of Kona's standout features is its ability to support custom features and data sources without requiring you to fork the entire project. Through careful use of Rust's powerful trait system and abstract interfaces, we've created a framework that allows you to plug in your own features and ideas seamlessly.

What You'll Learn

In this section of the developer book, we'll dive deep into the Kona SDK, covering:

  • Building on the FPVM Backend: Learn how to leverage the Fault Proof VM tooling to create your own fault proof programs.
  • Creating Custom Backends: Discover the process of designing and implementing your own backend to run kona-client or a variation of it on different targets.
  • Extending Core Components: Explore techniques for creating new constructs that integrate smoothly with crates like kona-derive and kona-executor.

Whether you're looking to use Kona as-is, extend its functionality, or create entirely new programs based on its libraries, this guide is intended to provide you with the knowledge and tools you need to succeed.

FPVM Backend

📖 Before reading this section of the book, it is advised to read the Fault Proof Program Environment section to familiarize yourself with the PreimageOracle IO pattern.

Kona is effectively split into three parts:

  • OP Stack state transition logic (kona-derive, kona-executor, kona-mpt)
  • OP Stack state transition proof SDK (kona-preimage, kona-proof)
  • Fault Proof VM IO and utilities (kona-std-fpvm, kona-std-fpvm-proc)

This section of the book focuses on the usage of kona-std-fpvm and kona-preimage to facilitate host<->client communication for programs running on top of the FPVM targets.

Host <-> Client Communication API

The FPVM system API is built on several layers. In this document, we'll cover these layers, from lowest-level to highest-level API.

kona-std-fpvm

kona-std-fpvm implements raw syscall dispatch, a default global memory allocator, and a blocking async runtime. kona-std-fpvm relies on a minimal linux backend to function, supporting only the syscalls required to implement the PreimageOracle ABI (read, write, exit_group).

These syscalls are exposed to the user through the io module directly, with each supported platform implementing the BasicKernelInterface trait.

To directly dispatch these syscalls, the io module exposes a safe API:

use kona_std_fpvm::{io, FileDescriptor};

// Print to `stdout`. Infallible, will panic if dispatch fails.
io::print("Hello, world!");

// Print to `stderr`. Infallible, will panic if dispatch fails.
io::print_err("Goodbye, world!");

// Read from or write to a specified file descriptor. Returns a result with the
// return value or syscall errno.
let _ = io::write(FileDescriptor::StdOut, "Hello, world!".as_bytes());
let mut buf = Vec::with_capacity(8);
let _ = io::read(FileDescriptor::StdIn, buf.as_mut_slice());

// Exit the program with a specified exit code.
io::exit(0);

With this library, you can implement a custom host<->client communication protocol, or extend the existing PreimageOracle ABI. However, for most developers, we recommend sticking with kona-preimage when developing programs that target the FPVMs, barring needs like printing directly to stdout.

kona-preimage

kona-preimage is an implementation of the PreimageOracle ABI. This crate enables synchronous communication between the host and client program, described in Host <-> Client Communication in the FPP Dev environment section of the book.

The crate is built around the Channel trait, which serves as a single end of a bidirectional pipe (see: pipe manpage).

Through this handle, the higher-level constructs can read and write data to the counterparty holding on to the other end of the channel, following the protocol below:

sequenceDiagram
    Client->>+Host: Hint preimage (no-op on-chain / read-only mode)
    Host-->>-Client: Hint acknowledgement
    Client-->>+Host: Preimage Request
    Host-->>Host: Prepare Preimage
    Host-->>-Client: Preimage Data

The interfaces of each part of the above protocol are described by the following traits:

Each of these traits, however, can be re-implemented to redefine the host<->client communication protocol if the needs of the consumer are not covered by the to-spec implementations.

kona-proof - Oracle-backed sources (example)

Finally, in kona-proof, implementations of data source traits from kona-derive and kona-executor are provided to pull in untyped data from the host by PreimageKey. These data source traits are covered in more detail within the Custom Backend section, but we'll quickly gloss over them here to build intuition.

Let's take, for example, OracleL1ChainProvider. The ChainProvider trait in kona-derive defines a simple interface for fetching information about the L1 chain. In the OracleL1ChainProvider, this information is pulled in over the PreimageOracle ABI. There are many other examples of these data source traits, namely the L2ChainProvider, BlobProvider, TrieProvider, and TrieHinter, which enable the creation of different data-source backends.

As an example, let's look at OracleL1ChainProvider::header_by_hash, built on top of the CommsClient trait, which is a composition trait of the PreimageOracleClient + HintReaderServer traits outlined above.

#[async_trait]
impl<T: CommsClient + Sync + Send> ChainProvider for OracleL1ChainProvider<T> {
    type Error = anyhow::Error;

    async fn header_by_hash(&mut self, hash: B256) -> Result<Header> {
        // Send a hint for the block header.
        self.oracle.write(&HintType::L1BlockHeader.encode_with(&[hash.as_ref()])).await?;

        // Fetch the header RLP from the oracle.
        let header_rlp =
            self.oracle.get(PreimageKey::new(*hash, PreimageKeyType::Keccak256)).await?;

        // Decode the header RLP into a Header.
        Header::decode(&mut header_rlp.as_slice())
            .map_err(|e| anyhow!("Failed to decode header RLP: {e}"))
    }

    // - snip -
}

In header_by_hash, we use the inner HintWriter to send a hint to the host to prepare the block hash preimage. Then, once we've received an acknowledgement from the host that the preimage has been prepared, we reach out for the RLP (which is the preimage of the hash). After the RLP is received, we decode the Header type, and return it to the user.

Custom Backends

Understanding the OP Stack STF

The OP Stack state transition is comprised of two primary components:

  • The derivation pipeline (kona-derive)
    • Responsible for deriving L2 chain state from the DA layer.
  • The execution engine (kona-executor)
    • Responsible for the execution of transactions and state commitments.
    • Ensures correct application of derived L2 state.

To prove the correctness of the state transition, Kona composes these two components:

  • It combines the derivation of the L2 chain with its execution in the same process.
  • It pulls in necessary data from sources to complete the STF, verifiably unrolling the input commitments along the way.

kona-client serves as an implementation of this process, capable of deriving and executing a single L2 block in a verifiable manner.

📖 Why just a single block by default?

On the OP Stack, we employ an interactive bisection game that narrows in on the disagreed upon block -> block state transition before requiring a fault proof to be ran. Because of this, the default implementation only serves to derive and execute the single block that the participants of the bisection game landed on.

Backend Traits

Covered in the FPVM Backend section of the book, kona-client ships with an implementation of kona-derive and kona-executor's data source traits which pull in data over the PreimageOracle ABI.

However, running kona-client on top of a different verifiable environment, i.e. a zkVM or TEE, is also possible through custom implementations of these data source traits.

op-succinct is an excellent example of both a custom backend and a custom program, implementing both kona-derive and kona-executor's data source traits backed by sp1_lib::io in order to:

  1. Execute kona-client verbatim, proving a single block's derivation and execution on SP-1.
  2. Derive and execute an entire Span Batch worth of L2 blocks, using kona-derive and kona-executor.

This section of the book outlines how you can do the same for a different platform.

Custom kona-derive sources

Before getting started, we need to create custom implementations of the following traits:

TraitDescription
ChainProviderThe ChainProvider trait describes the minimal interface for fetching data from L1 during L2 chain derivation.
L2ChainProviderThe ChainProvider trait describes the minimal interface for fetching data from the safe L2 chain during L2 chain derivation.
BlobProviderThe BlobProvider trait describes an interface for fetching EIP-4844 blobs from the L1 consensus layer during L2 chain derivation.

Once these are implemented, constructing the pipeline is as simple as passing in the data sources to the PipelineBuilder. Keep in mind the requirements for validation of incoming data, depending on your platform. For example, programs targeting zkVMs must constrain that the incoming data is indeed valid, whereas fault proof programs can offload this validation to the on-chain implementation of the host.

let chain_provider = ...;
let l2_chain_provider = ...;
let blob_provider = ...;
let l1_origin = ...;

let cfg = Arc::new(RollupConfig::default());
let attributes = StatefulAttributesBuilder::new(
   cfg.clone(),
   l2_chain_provider.clone(),
   chain_provider.clone(),
);
let dap = EthereumDataSource::new(
   chain_provider.clone(),
   blob_provider,
   cfg.as_ref()
);

// Construct a new derivation pipeline.
let pipeline = PipelineBuilder::new()
   .rollup_config(cfg)
   .dap_source(dap)
   .l2_chain_provider(l2_chain_provider)
   .chain_provider(chain_provider)
   .builder(attributes)
   .origin(l1_origin)
   .build();

From here, a custom derivation driver is needed to produce the desired execution payload(s). An example of this for kona-client can be found in the DerivationDriver.

kona-mpt / kona-executor sources

Before getting started, we need to create custom implementations of the following traits:

TraitDescription
TrieDBFetcherThe TrieDBFetcher trait describes the interface for fetching trie node preimages and chain information while executing a payload on the L2 chain.
TrieDBHinterThe TrieDBHinter trait describes the interface for requesting the host program to prepare trie proof preimages for the client's consumption. For targets with upfront witness generation, i.e. zkVMs, a no-op hinter is exported as NoopTrieDBHinter.

Once we have those, the StatelessL2BlockExecutor can be constructed like so:

#![allow(unused)]
fn main() {
let cfg = RollupConfig::default();
let provider = ...;
let hinter = ...;

let executor = StatelessL2BlockExecutor::builder(&cfg, provider, hinter)
   .with_parent_header(...)
   .build();

let header = executor.execute_payload(...).expect("Failed execution");
}

Bringing it Together

Once your custom backend traits for both kona-derive and kona-executor have been implemented, your final binary may look something like that of kona-client's. Alternatively, if you're looking to prove a wider range of blocks, op-succinct's range program offers a good example of running the pipeline and executor across a string of contiguous blocks.

kona-executor Extensions

The kona-executor crate offers a to-spec, stateless implementation of the OP Stack STF. However, due to the power of revm's Handler abstractions, the logic of the STF can be easily modified.

To register a custom handler, for example to add a custom precompile, modify the behavior of an EVM opcode, or change the fee handling, StatelessL2BlockExecutorBuilder::with_handle_register is your friend. It accepts a KonaHandleRegister, which can be used to take full advantage of revm's Handler API.

Example - Custom Precompile

const MY_PRECOMPILE_ADDRESS: Address = u64_to_address(0xFF);

fn my_precompile(input: &Bytes, gas_limit: u64) -> PrecompileResult {
   Ok(PrecompileOutput::new(50, "hello, world!".as_bytes().into()))
}

fn custom_handle_register<F, H>(
    handler: &mut EvmHandler<'_, (), &mut State<&mut TrieDB<F, H>>>,
) where
   F: TrieProvider,
   H: TrieHinter,
{
   let spec_id = handler.cfg.spec_id;

   handler.pre_execution.load_precompiles = Arc::new(move || {
      let mut ctx_precompiles = spec_to_generic!(spec_id, {
         revm::optimism::load_precompiles::<SPEC, (), &mut State<&mut TrieDB<F, H>>>()
      });

      let precompile = PrecompileWithAddress(
         MY_PRECOMPILE_ADDRESS,
         Precompile::Standard(my_precompile)
      );
      ctx_precompiles.extend([precompile]);

      ctx_precompiles
   });
}

// - snip -

let cfg = RollupConfig::default();
let provider = ...;
let hinter = ...;

let executor = StatelessL2BlockExecutor::builder(&cfg, provider, hinter)
   .with_parent_header(...)
   .with_handle_register(custom_handle_register)
   .build();

Fault Proof Program Development

This chapter provides an overview of Fault Proof Program development on top of the custom FPVM targets supported by Kona.

At a high level, a Fault Proof Program is not much different from a regular no_std Rust program. A custom entrypoint is provided, and the program is compiled down to a custom target, which is then executed on the FPVM.

Fault Proof Programs are structured with 3 stages:

  1. Prologue: The bootstrapping stage, where the program is loaded into memory and the initial state is set up. During this phase, the program's initial state is written to the FPVM's memory, and the program's entrypoint is set.
  2. Execution: The main execution stage, where the program is executed on the FPVM. During this phase, the program's entrypoint is called, and the program is executed until it exits.
  3. Epilogue: The finalization stage, where the program's final state is read from the FPVM's memory. During this phase, the program's final state is inspected and properties of the state transition are verified.

The following sections will provide a more in-depth overview of each of these stages, as well as the tools and abstractions provided by Kona for developing your own Fault Proof Programs.

Environment

Before kicking off the development of your own Fault Proof Program , it's important to understand the environment that your program will be running in.

The FPP runs on top of a custom FPVM target, which is typically a VM with a modified subset of an existing reduced instruction set architecture and a subset of Linux syscalls. The FPVM is designed to execute verifiable programs, and commonly modifies the instruction set it is derived from as well as the internal representation of memory to support verifiable memory access, client (program) communication with the host (the FPVM), and other implementation-specific features.

Host <-> Client Communication

While the program is running on top of the FPVM, it is considered to be in the client role, while the VM is in the host role. The only way for the client and host to communicate with one another is synchronously through the Preimage ABI (specification).

In order for the client to read from the host, the read and write syscalls are modified within the FPVM to allow the client to request preparation of and read foreign data.

Reading

When the client wants to read data from the host, it must first send a "hint" to the host through the hint file descriptor, which signals a request for the host to prepare the data for reading. The host will then prepare the data, and send a hint acknowledgement back to the client. The client can then read the data from the host through the designated file descriptor.

The preparation step ("hinting") is an optimization that allows the host to know ahead of time the intents of the client and the data it requires for execution. This can allow for lazy loading of data, and also prevent the need for unnecessary allocations within the host's memory. This step is a no-op on-chain, and is only ran locally when the host is the native implementation of the FPVM.

sequenceDiagram
    Client->>+Host: Hint preimage (no-op on-chain / read-only mode)
    Host-->>-Client: Hint acknowledgement
    Client-->>+Host: Preimage Request
    Host-->>Host: Prepare Preimage
    Host-->>-Client: Preimage Data

Full Example

Below, we have a full architecture diagram of the op-program (source: fault proof specs), the reference implementation for the OP Stack's Fault Proof Program, which has the objective of verifying claims about the state of an OP Stack layer two.

op-program-architecture

In this program, execution and derivation of the L2 chain is performed within it, and ultimately the claimed state of the L2 chain is verified in the prologue stage.

It communicates with the host for two reasons:

  1. To request preparation of L1 and L2 state data preimages.
  2. To read the L1 and L2 state data preimages that were prepared after the above requests.

The host is responsible for:

  1. Preparing and maintaining a store of the L1 and L2 state data preimages, as well as localized bootstrap k/v pairs.
  2. Providing the L1 and L2 state data preimages to the client for reading.

Other programs (clients) may have different requirements for communication with the host, but the above is a common pattern for programs built on top of a FPVMs. In general:

  1. The client program is a state machine that is responsible for bootstrapping itself from the inputs, executing the program logic, and verifying the outcome.
  2. The host is responsible for providing the client with data it wasn't bootstrapped with, and for executing the program itself.

Supported Targets

Kona seeks to support all FPVM targets that LLVM and rustc can offer introductory support for. Below is a matrix of features that Kona offers for each FPVM target:

TargetBuild PipelineIOmalloc
cannon & cannon-rs✅✅✅
asterisc✅✅✅

If there is a feature that you would like to see supported, please open an issue or consider contributing!

Asterisc (RISC-V)

Asterisc is based off of the rv64gc target architecture, which defines the following extensions:

  • RV32I support - 32 bit base instruction set
    • FENCE, ECALL, EBREAK are hardwired to implement a minimal subset of systemcalls of the linux kernel
      • Work in progress. All syscalls used by the Golang risc64 runtime.
  • RV64I support
  • RV32M+RV64M: Multiplication support
  • RV32A+RV64A: Atomics support
  • RV{32,64}{D,F,Q}: no-op: No floating points support (since no IEEE754 determinism with rounding modes etc., nor worth the complexity)
  • Zifencei: FENCE.I no-op: No need for FENCE.I
  • Zicsr: no-op: some support for Control-and-status registers may come later though.
  • Ztso: no-op: no need for Total Store Ordering
  • other: revert with error code on unrecognized instructions

asterisc supports a plethora of syscalls, documented in the repository. kona offers an interface for programs to directly invoke a select few syscalls:

  1. EXIT - Terminate the process with the provided exit code.
  2. WRITE - Write the passed buffer to the passed file descriptor.
  3. READ - Read the specified number of bytes from the passed file descriptor.

Cannon (MIPS64r2)

Cannon is based off of the mips64r2 target architecture, specified in MIPS® Architecture For Programmers Volume II-A: The MIPS64® Instruction Set Reference Manual

Syscalls

Syscalls supported by cannon can be found within the cannon specification here.

Prologue

The prologue stage of the program is commonly responsible for bootstrapping the program with inputs from an external source, pulled in through the Host <-> Client communication implementation.

As a rule of thumb, the prologue implementation should be kept minimal, and should not do much more than establish the inputs for the execution phase.

Example

As an example, the prologue stage of the kona-client program runs through several steps:

  1. Pull in the boot information over the Preimage Oracle ABI, containing:
    • The L1 head hash containing all data required to reproduce the L2 safe chain at the claimed block height.
    • The latest finalized L2 output root.
    • The L2 output root claim.
    • The block number of the L2 output root claim.
    • The L2 chain ID.
  2. Pull in the RollupConfig and L2ChainConfig corresponding to the passed L2 chain ID.
  3. Validate these values.
  4. Pass the boot information to the execution phase.

Execution

The execution phase of the program is commonly the heaviest portion of the fault proof program, where the computation that is being verified is performed.

This phase consumes the outputs of the prologue phase, and performs the bulk of the verifiable computation. After execution has concluded, the outputs are passed along to the epilogue phase for final verification.

Example

At a high-level, in the kona-client program, the execution phase:

  1. Derives the inputs to the L2 derivation pipeline by unrolling the L1 head hash fetched in the epilogue.
  2. Passes the inputs to the L2 derivation pipeline, producing the L2 execution payloads required to reproduce the L2 safe chain at the claimed height.
  3. Executes the payloads produced by the L2 derivation pipeline, producing the L2 output root at the L2 claim height.

Epilogue

The epilogue stage of the program is intended to perform the final validation on the outputs from the execution phase. In most programs, this entails comparing the outputs of the execution phase to portions of the bootstrap data made available during the prologue phase.

Generally, this phase should consist almost entirely of validation steps.

Example

In the kona-client program, the prologue phase only contains two directives:

  1. Validate that the L2 safe chain could be produced at the claimed L2 block height.
  2. The constructed output root is equivalent to the claimed L2 output root.

Examples

Examples for working with kona crates.

Loading a Rollup Config from a Chain ID

In this section, the code examples demonstrate loading the rollup config for the given L2 Chain ID.

Let's load the Rollup Config for OP Mainnet which hash chain id 10.

#![allow(unused)]
fn main() {
use kona_registry::ROLLUP_CONFIGS;
use kona_genesis::OP_MAINNET_CHAIN_ID;

// Load a rollup config from the chain id.
let op_mainnet_config = ROLLUP_CONFIGS.get(&OP_MAINNET_CHAIN_ID).expect("infallible");

// The chain id should match the hardcoded chain id.
assert_eq!(op_mainnet_config.chain_id, OP_MAINNET_CHAIN_ID);
}

Available Configs

kona-registry dynamically provides all rollup configs from the superchain-registry for their respective chain ids. Note though, that this requires serde since it deserializes the rollup configs dynamically from json files.

Transform Frames into a Batch

note

This example performs the reverse transformation as the batch-to-frames example.

caution

Steps and handling of types with respect to chain tip, ordering of frames, re-orgs, and more are not covered by this example. This example solely demonstrates the most trivial way to transform individual [Frame][frame]s into a [Batch][batch] type.

This example walks through transforming [Frame][frame]s into the [Batch][batch] types.

Walkthrough

The high level transformation is the following.

raw bytes[] -> frames[] -> channel -> decompressed channel data -> Batch

Given the raw, batch-submitted frame data as bytes (read in with the [hex! macro][hex]), the first step is to decode the frame data into [Frame][frame]s using [Frame::decode][decode-frame]. Once all the [Frame][frame]s are decoded, the [Channel][channel] can be constructed using the [ChannelId][channel-id] of the first frame.

note

[Frame][frame]s may also be added to a [Channel][channel] once decoded with the [Channel::add_frame][add-frame] method.

When the [Channel][channel] is [Channel::is_ready()][is-ready], the frame data can taken from the [Channel][channel] using [Channel::frame_data()][frame-data]. This data is represented as [Bytes][bytes] and needs to be decompressed using the respective compression algorithm depending on which hardforks are activated (using the RollupConfig). For the sake of this example, brotli is used (which was activated in the [Fjord hardfork][fjord]). Decompressed brotli bytes can then be passed right into [Batch::decode][decode-batch] to wind up with the example's desired [Batch][batch].

Running this example:

  • Clone the examples repository: git clone git@github.com:op-rs/kona.git
  • Run: cargo run --example frames_to_batch
//! This example decodes raw [Frame]s and reads them into a [Channel] and into a [SingleBatch].

use alloy_consensus::{SignableTransaction, TxEip1559, TxEnvelope};
use alloy_eips::eip2718::{Decodable2718, Encodable2718};
use alloy_primitives::{Address, BlockHash, Bytes, PrimitiveSignature, U256, hex};
use kona_genesis::RollupConfig;
use kona_protocol::{Batch, BlockInfo, Channel, Frame, SingleBatch, decompress_brotli};

fn main() {
    // Raw frame data taken from the `encode_channel` example.
    let first_frame = hex!(
        "60d54f49b71978b1b09288af847b11d200000000004d1b1301f82f0f6c3734f4821cd090ef3979d71a98e7e483b1dccdd525024c0ef16f425c7b4976a7acc0c94a0514b72c096d4dcc52f0b22dae193c70c86d0790a304a08152c8250031d091063ea000"
    );
    let second_frame = hex!(
        "60d54f49b71978b1b09288af847b11d2000100000046b00d00005082edde7ccf05bded2004462b5e80e1c42cd08e307f5baac723b22864cc6cd01ddde84efc7c018d7ada56c2fa8e3c5bedd494c3a7a884439d5771afcecaf196cb3801"
    );

    // Decode the raw frames.
    let decoded_first = Frame::decode(&first_frame).expect("decodes frame").1;
    let decoded_second = Frame::decode(&second_frame).expect("decodes frame").1;

    // Create a channel.
    let id = decoded_first.id;
    let open_block = BlockInfo::default();
    let mut channel = Channel::new(id, open_block);

    // Add the frames to the channel.
    let l1_inclusion_block = BlockInfo::default();
    channel.add_frame(decoded_first, l1_inclusion_block).expect("adds frame");
    channel.add_frame(decoded_second, l1_inclusion_block).expect("adds frame");

    // Get the frame data from the channel.
    let frame_data = channel.frame_data().expect("some frame data");
    println!("Frame data: {}", hex::encode(&frame_data));

    // Decompress the frame data with brotli.
    let config = RollupConfig::default();
    let max = config.max_rlp_bytes_per_channel(open_block.timestamp) as usize;
    let decompressed = decompress_brotli(&frame_data, max).expect("decompresses brotli");
    println!("Decompressed frame data: {}", hex::encode(&decompressed));

    // Decode the single batch from the decompressed data.
    let batch = Batch::decode(&mut decompressed.as_slice(), &config).expect("batch decodes");
    assert_eq!(
        batch,
        Batch::Single(SingleBatch {
            parent_hash: BlockHash::ZERO,
            epoch_num: 1,
            epoch_hash: BlockHash::ZERO,
            timestamp: 1,
            transactions: example_transactions(),
        })
    );

    println!("Successfully decoded frames into a Batch");
}

fn example_transactions() -> Vec<Bytes> {
    let mut transactions = Vec::new();

    // First Transaction in the batch.
    let tx = TxEip1559 {
        chain_id: 10u64,
        nonce: 2,
        max_fee_per_gas: 3,
        max_priority_fee_per_gas: 4,
        gas_limit: 5,
        to: Address::left_padding_from(&[6]).into(),
        value: U256::from(7_u64),
        input: vec![8].into(),
        access_list: Default::default(),
    };
    let sig = PrimitiveSignature::test_signature();
    let tx_signed = tx.into_signed(sig);
    let envelope: TxEnvelope = tx_signed.into();
    let encoded = envelope.encoded_2718();
    transactions.push(encoded.clone().into());
    let mut slice = encoded.as_slice();
    let decoded = TxEnvelope::decode_2718(&mut slice).unwrap();
    assert!(matches!(decoded, TxEnvelope::Eip1559(_)));

    // Second transaction in the batch.
    let tx = TxEip1559 {
        chain_id: 10u64,
        nonce: 2,
        max_fee_per_gas: 3,
        max_priority_fee_per_gas: 4,
        gas_limit: 5,
        to: Address::left_padding_from(&[7]).into(),
        value: U256::from(7_u64),
        input: vec![8].into(),
        access_list: Default::default(),
    };
    let sig = PrimitiveSignature::test_signature();
    let tx_signed = tx.into_signed(sig);
    let envelope: TxEnvelope = tx_signed.into();
    let encoded = envelope.encoded_2718();
    transactions.push(encoded.clone().into());
    let mut slice = encoded.as_slice();
    let decoded = TxEnvelope::decode_2718(&mut slice).unwrap();
    assert!(matches!(decoded, TxEnvelope::Eip1559(_)));

    transactions
}

Transform a Batch into Frames

note

This example performs the reverse transformation as the frames-to-batch example.

caution

Steps and handling of types with respect to chain tip, ordering of frames, re-orgs, and more are not covered by this example. This example solely demonstrates the most trivial way to transform an individual [Batch][batch] into [Frame][frame]s.

This example walks through transforming a [Batch][batch] into [Frame][frame]s.

Effectively, this example demonstrates the encoding process from an L2 batch into the serialized bytes that are posted to the data availability layer.

Walkthrough

The high level transformation is the following.

Batch -> decompressed batch data -> ChannelOut -> frames[] -> bytes[]

Given the [Batch][batch], the first step to encode the batch using the [Batch::encode()][encode-batch] method. The output bytes need to then be compressed prior to adding them to the [ChannelOut][channel-out].

note

The [ChannelOut][channel-out] type also provides a method for adding the [Batch][batch] itself, handling encoding and compression, but this method is not available yet.

Once compressed using the [compress_brotli][compress-brotli] method, the compressed bytes can be added to a newly constructed [ChannelOut][channel-out]. As long as the [ChannelOut][channel-out] has [ready_bytes()][ready-bytes], [Frame][frame]s can be constructed using the [ChannelOut::output_frame()][output-frame] method, specifying the maximum frame size.

Once [Frame][frame]s are returned from the [ChannelOut][channel-out], they can be [Frame::encode][encode-frame] into raw, serialized data ready to be batch-submitted to the data-availability layer.

Running this example:

  • Clone the examples repository: git clone git@github.com:op-rs/kona.git
  • Run: cargo run --example batch_to_frames
#![allow(unused)]
fn main() {
{{#include ../../../crates/protocol/protocol/examples/batch_to_frames.rs}}
}

Create a L1BlockInfoTx Variant for a new Hardfork

This example walks through creating a variant of the L1BlockInfoTx for a new Hardfork.

note

This example is very verbose. To grok required changes, view this PR diff which introduces Isthmus hardfork changes to the L1BlockInfoTx with a new variant.

Required Genesis Updates

The first updates that need to be made are to kona-genesis types, namely the RollupConfig and HardForkConfig.

First, add a timestamp field to the RollupConfig. Let's use the hardfork name "Glacier" as an example.

#![allow(unused)]
fn main() {
pub struct RollupConfig {
   ...
   /// `glacier_time` sets the activation time for the Glacier network upgrade.
   /// Active if `glacier_time` != None && L2 block timestamp >= Some(glacier_time), inactive
   /// otherwise.
   #[cfg_attr(feature = "serde", serde(skip_serializing_if = "Option::is_none"))]
   pub glacier_time: Option<u64>,
   ...
}
}

Add an accessor on the RollupConfig to provide a way of checking whether the "Glacier" hardfork is active for a given timestamp. Also update the prior hardfork accessor to call this method (let's use "Isthmus" as the prior hardfork).

#![allow(unused)]
fn main() {
    /// Returns true if Isthmus is active at the given timestamp.
    pub fn is_isthmus_active(&self, timestamp: u64) -> bool {
        self.isthmus_time.map_or(false, |t| timestamp >= t) || self.is_glacier_active(timestamp)
    }

    /// Returns true if Glacier is active at the given timestamp.
    pub fn is_glacier_active(&self, timestamp: u64) -> bool {
        self.glacier_time.map_or(false, |t| timestamp >= t)
    }
}

Lastly, add the "Glacier" timestamp to the HardForkConfig.

#![allow(unused)]
fn main() {
pub struct HardForkConfig {
    ...
    /// Glacier hardfork activation time
    pub glacier_time: Option<u64>,
}
}

Protocol Changes

Introduce a new glacier.rs module containing a L1BlockInfoGlacier type in kona_genesis::info module.

This should include a few methods used in the L1BlockInfoTx later.

#![allow(unused)]
fn main() {
    pub fn encode_calldata(&self) -> Bytes { ... }

    pub fn decode_calldata(r: &[u8]) -> Result<Self, DecodeError> { ... }
}

Use other hardfork variants like the L1BlockInfoEcotone for reference.

Next, add the new "Glacier" variant to the L1BlockInfoTx.

#![allow(unused)]
fn main() {
pub enum L1BlockInfoTx {
   ...
   Glacier(L1BlockInfoGlacier)
}
}

Update L1BlockInfoTx::try_new to construct the L1BlockInfoGlacier if the hardfork is active using the RollupConfig::is_glacier_active.

Also, be sure to update L1BlockInfoTx::decode_calldata with the new variant decoding, as well as other L1BlockInfoTx methods.

Once some tests are added surrounding the decoding and encoding of the new L1BlockInfoGlacier variant, all required changes are complete!

Now, this example PR diff introducing the Isthmus changes should make sense, since it effectively implements the above changes for the Isthmus hardfork (replacing "Glacier" with "Isthmus"). Notice, Isthmus introduces some new "operator fee" fields as part of it's L1BlockInfoIsthmus type. Some new error variants to the BlockInfoError are needed as well.

Request For Comment [RFC]

Documents in this section are in the request-for-comment stage.

To comment on these documents, open an issue in the kona repository and provide detail on the changes you're requesting.

Once the document has been reviewed, they will be moved to the archives.

Umbrella Crates

info

TL;DR, this is a proposal to introduce tiny crates inside each container directory (e.g. crates/protocol/) to re-export all crates contained in that directory.

Context

Repository Structure

Kona now has a monorepo structure that merged maili and hilo crates into kona. This introduces a number of higher-level directories that hold a variety of crates themselves. As of the time at which this document was written the kona repository loosely looks like the following.

bins/
  -> client/
  -> host/
crates/
  -> protocol/
     -> genesis/
     -> protocol/
     -> derive/
     -> driver/
     -> interop/
     -> registry/
  -> proof/
     -> mpt/
     -> executor/
     -> proof/
     -> proof-interop/
     -> preimage/
     -> std-fpvm/
     -> std-fpvm-proc/
  -> node/
     -> net/
     -> rpc/
     -> engine/
  -> providers/
     -> providers-alloy/
     -> providers-local/
  -> utilities/
     -> serde/

Within crates, the protocol, proof, node, providers, and utilities directories all contain crates, and are not crates themselves - only directories.

Publishing Crates

When crates in kona are published, they are all published individually, with no way to add all kona crates as a dependency. This make discoverability difficult without accurate and up-to-date documentation, adding overhead.

Problem

As a monorepo, kona will likely have a growing number of crates that will make it increasingly difficult to discover new kona crates and manage kona as a dependency, for downstream consumers.

Additionally, each crate has it's own independent version, which makes it more nuanced to manage kona dependencies and less clear which crate versions are compatible.

Considered Options

Single Umbrella Crate

One option that would make kona crates the easiest to consume is to provide a single umbrella crate that lives at the top-level (e.g. crates/umbrella/Cargo.toml).

This crate could simply be called kona and re-export all crates in the kona monorepo under various feature flags, with propagating std and serde feature flags.

Tradeoffs

The benefit of this option is providing a single crate to consume all of kona, with the downside of having to manage all the various feature flags and crate re-exports in the single umbrella crate.

Grouped Umbrella Crates

In each of the crates/ sub-directories, provide an umbrella crate that exports all crates within that subdirectory.

For example, in the crates/protocol/ subdirectory, an umbrella crate would re-export all crates in crates/protocol/. It could be called kona-umbrella-protocol or some other name to make it easily discoverable.

Tradeoffs

While this simplifies updates when adding or removing a re-export, it introduces n-1 additional crates as the single umbrella crate, where n is the number of sub-directories in crates/. These many umbrella crates also make kona less easily consumed by downstream users of kona as opposed to the singular umbrella crate.

Top-level Umbrella with Subdirectory Umbrellas (Combined)

Effectively, this option is to combined the previous two options into one.

In this configuration, the top-level umbrella crate could just re-export each of the umbrella crates in the sub-directory.

Tradeoffs

Unfortunately this option now introduces n + 1 number of crates where n is the number of subdirectories in crates/, but it still only requires updates to the subdirectory umbrellas when a crate is added or removed.

The benefit of this is option is the top-level umbrella crate is very much simplified, since it only needs to re-export the n umbrella crates and not every crate in the workspace. It also provides the single consumable kona crate for downstream users that greatly simplifies managing kona as a dependency.

Proposed Solution

???

Looking for comments/thoughts here before landing on a proposed solution.

Archived Documents

These are archived documents that have been implemented or are no longer relevant.

Documents

DocumentDateDecision
Monorepo02/19/2025Approved

Monorepo Project

This is a document outlining merging multiple external repositories into kona to create a rust monorepo for Optimism.

Provenance

Let's rewind the clock to when kona was just being started in the spring of 2024. What little optimism rust types existed were siloed in applications like op-reth and revm. Library code that could have been re-used was unfortunately placed in an std environment that kona cannot use because the kona fault proof program is built for a minimal instruction set.

Effectively, kona needed a bunch of Optimism-specific and slightly-modified Ethereum types that were available in a no_std environment.

As development started, types were defined in kona. Mostly jammed into the kona-derive crate for use in derivation, these types were now being duplicated across a number of rust repos just to support no_std.

Enter op-alloy. The first effort to de-duplicate types between reth, revm, and kona (as well as periphery applications like magi). Rather than keeping types in kona, op-alloy was introduced as a shared place for no_std compatible optimism rust types. This worked well as a common place to contribute and decoupled the rapid development in kona from the more stable definition of Optimism rust types.

Fast forward to 2025, interop started seriously picking up momentum as a critical project in the Optimism Rust world. Interop introduces a whole host of new Optimism-specific types that really should live in a shared library. But op-alloy was already becoming quite large beyond the minimal, de-duplicated set of types originally intended.

This led to OP folks splitting out Optimism-specific types from op-alloy into a new repo called maili. What was not foreseen was the grievance with yet-another level in dependency chain for Optimism rust projects. Now, downstream optimism rust projects could have to import all of these crates just to construct an OP Stack service:

  • op-alloy
  • maili
  • op-revm
  • op-reth
  • kona

And those are just the Optimism-specific crates, let alone Ethereum crates like alloy.

Why?

The current dependency chain is ever growing. A small change in op-alloy cascades into the following.

  • Release op-alloy crates
  • Update maili with op-alloy version and changes.
  • Release maili crates.
  • Update kona to work with both op-alloy and maili crates.
  • Release kona.
  • If something breaks in kona or downstream consumers, repeat.

To iterate faster without needing to manage releases or change dependencies into git refs, this doc proposes a monorepo structure that pulls maili into kona, while retaining maili crate names and versioning.

We propose gone are the days of releasing a single version for all crates. With a larger, more extensive kona monorepo, crates will need to manage their own semver.

To re-iterate: the key takeaway here is current downstream consumers of maili crates will not have to change a thing. Crates pulled into kona will still be published under maili- prefixed crate names. How this is managed while keeping the kona- prefix crate naming consistent is discussed below.

Proposed Repository Structure

The new repository structure would look as follows.

crates/
  proof/
    mpt/
    executor/
    preimage/
    fpvm/
    fpvm-proc/
    proof
    proof-interop/
  protocol/
    derive/
    driver/
    interop/
    genesis/   <-- Has Maili Shadow
    registry/  <-- Has Maili Shadow
    protocol/  <-- Has Maili Shadow
  services/
    rpc/       <-- Has Maili Shadow
    net/
    providers-alloy/
    providers-local/
  utilities/
    serde/     <-- Has Maili Shadow
  cli/
    ...

info

Crates denoted with <-- Has Maili Shadow are ported from maili, but contain a nested crate with the maili- prefix instead of kona- prefix. These crates re-export their kona- equivalent crates. This allows downstream users to not need to change their dependencies to keep using maili- crates! Eventually once the kona monorepo matures, and downstream consumers use kona- crates instead of maili-, these can be removed.

Maili Shadow Example

Let's look at crates/protocol/genesis.

This crate will have a Cargo.toml that defines itself as kona-genesis.

The contents of the crates/protocol/genesis directory will be

../genesis/
  README.md
  Cargo.toml  <-- package.name = "kona-genesis"
  src/
    ..   <-- current contents of `maili-genesis`, ported
  maili/
    Cargo.toml  <-- package.name = "maili-genesis"
    src/
      lib.rs  <-- Re-exports `kona-genesis`

This structure allows us to seamlessly remain backwards compatible, while being able to work in the new kona- crates without requiring heavy lifting to support maili- crates.

Glossary

This document contains definitions for terms used throughout the Kona book.

Fault Proof VM

A Fault Proof VM is a virtual machine, commonly supporting a subset of the Linux kernel's syscalls and a modified subset of an existing reduced instruction set architecture, that is designed to execute verifiable programs.

Full specification for the cannon & cannon-rs FPVMs, as an example, is available in the Optimism Monorepo.

Fault Proof Program

A Fault Proof Program is a program, commonly written in a general-purpose language such as Golang, C, or Rust, that may be compiled down to a compatible Fault Proof VM target and provably executed on that target VM.

Examples of Fault Proof Programs include the OP Program, which runs on top of cannon, cannon-rs, and asterisc to verify a claim about the state of an OP Stack layer two.

Preimage ABI

The Preimage ABI is a specification for a synchronous communication protocol between a client and a host that is used to request and read data from the host's datastore. Full specifications for the Preimage ABI are available in the Optimism Monorepo.

Contributing

Thank you for wanting to contribute! Before contributing to this repository, please read through this document and discuss the change you wish to make via issue.

Dependencies

Before working with this repository locally, you'll need to install several dependencies:

Optional

Pull Request Process

  1. Before anything, create an issue to discuss the change you're wanting to make, if it is significant or changes functionality. Feel free to skip this step for trivial changes.
  2. Once your change is implemented, ensure that all checks are passing before creating a PR. The full CI pipeline can be run locally via the justfiles in the repository.
  3. Make sure to update any documentation that has gone stale as a result of the change, in the README files, the [book][book], and in rustdoc comments.
  4. Once you have sign-off from a maintainer, you may merge your pull request yourself if you have permissions to do so. If not, the maintainer who approves your pull request will add it to the merge queue.