Commit 7719c5a5 authored by Adrian Sutton's avatar Adrian Sutton Committed by GitHub

op-program: Implement initial block derivation step for interop fault proofs (#13675)

* op-service: Define the SuperRoot type

* op-program: Add chain ID to super chain output roots

* Rename

* op-e2e: Add interop fault proofs actions test

* op-program: Update to include chain ID

* op-program: Implement initial block derivation step for interop fault proofs

* op-program: Update to include chain ID

* op-program: Use validate flag correctly.
Rename interop env var.
parent e6c9bb7c
......@@ -6,10 +6,10 @@ import (
"testing"
op_e2e "github.com/ethereum-optimism/optimism/op-e2e"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-e2e/e2eutils/disputegame"
preimage "github.com/ethereum-optimism/optimism/op-preimage"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum/go-ethereum/common"
"github.com/stretchr/testify/require"
)
......@@ -19,10 +19,10 @@ func TestLocalPreimages(t *testing.T) {
tests := []struct {
key preimage.Key
}{
{key: client.L1HeadLocalIndex},
{key: client.L2OutputRootLocalIndex},
{key: client.L2ClaimLocalIndex},
{key: client.L2ClaimBlockNumberLocalIndex},
{key: boot.L1HeadLocalIndex},
{key: boot.L2OutputRootLocalIndex},
{key: boot.L2ClaimLocalIndex},
{key: boot.L2ClaimBlockNumberLocalIndex},
// We don't check client.L2ChainIDLocalIndex because e2e tests use a custom chain configuration
// which requires using a custom chain ID indicator so op-program will load the full rollup config and
// genesis from the preimage oracle
......
package client
package boot
import (
"encoding/binary"
......
package client
package boot
import (
"encoding/binary"
......
package interop
import (
"errors"
"fmt"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/client/claim"
"github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum-optimism/optimism/op-program/client/l1"
"github.com/ethereum-optimism/optimism/op-program/client/l2"
"github.com/ethereum-optimism/optimism/op-program/client/tasks"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
)
var (
ErrIncorrectOutputRootType = errors.New("incorrect output root type")
)
type taskExecutor interface {
RunDerivation(
logger log.Logger,
rollupCfg *rollup.Config,
l2ChainConfig *params.ChainConfig,
l1Head common.Hash,
agreedOutputRoot eth.Bytes32,
claimedBlockNumber uint64,
l1Oracle l1.Oracle,
l2Oracle l2.Oracle) (tasks.DerivationResult, error)
}
func RunInteropProgram(logger log.Logger, bootInfo *boot.BootInfo, l1PreimageOracle l1.Oracle, l2PreimageOracle l2.Oracle, validate bool) error {
return runInteropProgram(logger, bootInfo, l1PreimageOracle, l2PreimageOracle, validate, &interopTaskExecutor{})
}
func runInteropProgram(logger log.Logger, bootInfo *boot.BootInfo, l1PreimageOracle l1.Oracle, l2PreimageOracle l2.Oracle, validate bool, tasks taskExecutor) error {
logger.Info("Interop Program Bootstrapped", "bootInfo", bootInfo)
// For the first step in a timestamp, we would get a SuperRoot as the agreed claim - TransitionStateByRoot will
// automatically convert it to a TransitionState with Step: 0.
transitionState := l2PreimageOracle.TransitionStateByRoot(bootInfo.L2OutputRoot)
if transitionState.Version() != types.IntermediateTransitionVersion {
return fmt.Errorf("%w: %v", ErrIncorrectOutputRootType, transitionState.Version())
}
super, err := eth.UnmarshalSuperRoot(transitionState.SuperRoot)
if err != nil {
return fmt.Errorf("invalid super root: %w", err)
}
if super.Version() != eth.SuperRootVersionV1 {
return fmt.Errorf("%w: %v", ErrIncorrectOutputRootType, super.Version())
}
superRoot := super.(*eth.SuperV1)
claimedBlockNumber, err := bootInfo.RollupConfig.TargetBlockNumber(superRoot.Timestamp + 1)
if err != nil {
return err
}
derivationResult, err := tasks.RunDerivation(
logger,
bootInfo.RollupConfig,
bootInfo.L2ChainConfig,
bootInfo.L1Head,
superRoot.Chains[0].Output,
claimedBlockNumber,
l1PreimageOracle,
l2PreimageOracle,
)
if err != nil {
return err
}
newPendingProgress := make([]types.OptimisticBlock, len(transitionState.PendingProgress)+1)
copy(newPendingProgress, transitionState.PendingProgress)
newPendingProgress[len(newPendingProgress)-1] = types.OptimisticBlock{
BlockHash: derivationResult.BlockHash,
OutputRoot: derivationResult.OutputRoot,
}
finalState := &types.TransitionState{
SuperRoot: transitionState.SuperRoot,
PendingProgress: newPendingProgress,
Step: transitionState.Step + 1,
}
expected, err := finalState.Hash()
if err != nil {
return err
}
if !validate {
return nil
}
return claim.ValidateClaim(logger, derivationResult.SafeHead, eth.Bytes32(bootInfo.L2Claim), eth.Bytes32(expected))
}
type interopTaskExecutor struct {
}
func (t *interopTaskExecutor) RunDerivation(
logger log.Logger,
rollupCfg *rollup.Config,
l2ChainConfig *params.ChainConfig,
l1Head common.Hash,
agreedOutputRoot eth.Bytes32,
claimedBlockNumber uint64,
l1Oracle l1.Oracle,
l2Oracle l2.Oracle) (tasks.DerivationResult, error) {
return tasks.RunDerivation(
logger,
rollupCfg,
l2ChainConfig,
l1Head,
common.Hash(agreedOutputRoot),
claimedBlockNumber,
l1Oracle,
l2Oracle)
}
package interop
import (
"testing"
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum-optimism/optimism/op-program/client/l1"
"github.com/ethereum-optimism/optimism/op-program/client/l2"
"github.com/ethereum-optimism/optimism/op-program/client/l2/test"
"github.com/ethereum-optimism/optimism/op-program/client/tasks"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/testlog"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/stretchr/testify/require"
)
func TestDeriveBlockForFirstChainFromSuperchainRoot(t *testing.T) {
logger := testlog.Logger(t, log.LevelError)
rollupCfg := chaincfg.OPSepolia()
chain1Output := &eth.OutputV0{}
agreedSuperRoot := &eth.SuperV1{
Timestamp: rollupCfg.Genesis.L2Time + 1234,
Chains: []eth.ChainIDAndOutput{{ChainID: rollupCfg.L2ChainID.Uint64(), Output: eth.OutputRoot(chain1Output)}},
}
outputRootHash := common.Hash(eth.SuperRoot(agreedSuperRoot))
l2PreimageOracle, _ := test.NewStubOracle(t)
l2PreimageOracle.TransitionStates[outputRootHash] = &types.TransitionState{SuperRoot: agreedSuperRoot.Marshal()}
tasks := stubTasks{
l2SafeHead: eth.L2BlockRef{
Number: 56,
Hash: common.Hash{0x11},
},
blockHash: common.Hash{0x22},
outputRoot: eth.Bytes32{0x66},
}
expectedIntermediateRoot := &types.TransitionState{
SuperRoot: agreedSuperRoot.Marshal(),
PendingProgress: []types.OptimisticBlock{
{BlockHash: tasks.blockHash, OutputRoot: tasks.outputRoot},
},
Step: 1,
}
expectedClaim, err := expectedIntermediateRoot.Hash()
require.NoError(t, err)
bootInfo := &boot.BootInfo{
L2OutputRoot: outputRootHash,
RollupConfig: rollupCfg,
L2ClaimBlockNumber: agreedSuperRoot.Timestamp + 1,
L2Claim: expectedClaim,
}
err = runInteropProgram(logger, bootInfo, nil, l2PreimageOracle, true, &tasks)
require.NoError(t, err)
}
type stubTasks struct {
l2SafeHead eth.L2BlockRef
blockHash common.Hash
outputRoot eth.Bytes32
err error
}
func (t *stubTasks) RunDerivation(
_ log.Logger,
_ *rollup.Config,
_ *params.ChainConfig,
_ common.Hash,
_ eth.Bytes32,
_ uint64,
_ l1.Oracle,
_ l2.Oracle) (tasks.DerivationResult, error) {
return tasks.DerivationResult{
SafeHead: t.l2SafeHead,
BlockHash: t.blockHash,
OutputRoot: t.outputRoot,
}, t.err
}
package l2
import (
interopTypes "github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
......@@ -14,11 +15,12 @@ const nodeCacheSize = 100_000
const codeCacheSize = 10_000
type CachingOracle struct {
oracle Oracle
blocks *simplelru.LRU[common.Hash, *types.Block]
nodes *simplelru.LRU[common.Hash, []byte]
codes *simplelru.LRU[common.Hash, []byte]
outputs *simplelru.LRU[common.Hash, eth.Output]
oracle Oracle
blocks *simplelru.LRU[common.Hash, *types.Block]
nodes *simplelru.LRU[common.Hash, []byte]
codes *simplelru.LRU[common.Hash, []byte]
outputs *simplelru.LRU[common.Hash, eth.Output]
transitionStates *simplelru.LRU[common.Hash, *interopTypes.TransitionState]
}
func NewCachingOracle(oracle Oracle) *CachingOracle {
......@@ -81,3 +83,13 @@ func (o *CachingOracle) BlockDataByHash(agreedBlockHash, blockHash common.Hash,
o.blocks.Add(blockHash, block)
return block
}
func (o *CachingOracle) TransitionStateByRoot(root common.Hash) *interopTypes.TransitionState {
state, ok := o.transitionStates.Get(root)
if ok {
return state
}
state = o.oracle.TransitionStateByRoot(root)
o.transitionStates.Add(root, state)
return state
}
......@@ -36,20 +36,25 @@ func NewOracleEngine(rollupCfg *rollup.Config, logger log.Logger, backend engine
}
}
func (o *OracleEngine) L2OutputRoot(l2ClaimBlockNum uint64) (eth.Bytes32, error) {
// L2OutputRoot returns the block hash and output root at the specified block number
func (o *OracleEngine) L2OutputRoot(l2ClaimBlockNum uint64) (common.Hash, eth.Bytes32, error) {
outBlock := o.backend.GetHeaderByNumber(l2ClaimBlockNum)
if outBlock == nil {
return eth.Bytes32{}, fmt.Errorf("failed to get L2 block at %d", l2ClaimBlockNum)
return common.Hash{}, eth.Bytes32{}, fmt.Errorf("failed to get L2 block at %d", l2ClaimBlockNum)
}
stateDB, err := o.backend.StateAt(outBlock.Root)
if err != nil {
return eth.Bytes32{}, fmt.Errorf("failed to open L2 state db at block %s: %w", outBlock.Hash(), err)
return common.Hash{}, eth.Bytes32{}, fmt.Errorf("failed to open L2 state db at block %s: %w", outBlock.Hash(), err)
}
withdrawalsTrie, err := stateDB.OpenStorageTrie(predeploys.L2ToL1MessagePasserAddr)
if err != nil {
return eth.Bytes32{}, fmt.Errorf("withdrawals trie unavailable at block %v: %w", outBlock.Hash(), err)
return common.Hash{}, eth.Bytes32{}, fmt.Errorf("withdrawals trie unavailable at block %v: %w", outBlock.Hash(), err)
}
return rollup.ComputeL2OutputRootV0(eth.HeaderBlockInfo(outBlock), withdrawalsTrie.Hash())
output, err := rollup.ComputeL2OutputRootV0(eth.HeaderBlockInfo(outBlock), withdrawalsTrie.Hash())
if err != nil {
return common.Hash{}, eth.Bytes32{}, err
}
return outBlock.Hash(), output, nil
}
func (o *OracleEngine) GetPayload(ctx context.Context, payloadInfo eth.PayloadInfo) (*eth.ExecutionPayloadEnvelope, error) {
......
......@@ -16,6 +16,7 @@ const (
HintL2StateNode = "l2-state-node"
HintL2Output = "l2-output"
HintL2BlockData = "l2-block-data"
HintAgreedPrestate = "agreed-pre-state"
)
type BlockHeaderHint common.Hash
......@@ -73,3 +74,11 @@ func (l L2BlockDataHint) Hint() string {
binary.BigEndian.PutUint64(hintBytes[64:], l.ChainID)
return fmt.Sprintf("%s 0x%s", HintL2BlockData, common.Bytes2Hex(hintBytes))
}
type AgreedPrestateHint common.Hash
var _ preimage.Hint = AgreedPrestateHint{}
func (l AgreedPrestateHint) Hint() string {
return HintAgreedPrestate + " " + (common.Hash)(l).String()
}
......@@ -3,6 +3,7 @@ package l2
import (
"fmt"
interopTypes "github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
......@@ -37,6 +38,8 @@ type Oracle interface {
// BlockDataByHash retrieves the block, including all data used to construct it.
BlockDataByHash(agreedBlockHash, blockHash common.Hash, chainID uint64) *types.Block
TransitionStateByRoot(root common.Hash) *interopTypes.TransitionState
}
// PreimageOracle implements Oracle using by interfacing with the pure preimage.Oracle
......@@ -117,3 +120,13 @@ func (p *PreimageOracle) BlockDataByHash(agreedBlockHash, blockHash common.Hash,
txs := p.LoadTransactions(blockHash, header.TxHash)
return types.NewBlockWithHeader(header).WithBody(types.Body{Transactions: txs})
}
func (p *PreimageOracle) TransitionStateByRoot(root common.Hash) *interopTypes.TransitionState {
p.hint.Hint(AgreedPrestateHint(root))
data := p.oracle.Get(preimage.Keccak256Key(root))
output, err := interopTypes.UnmarshalProofsState(data)
if err != nil {
panic(fmt.Errorf("invalid agreed prestate data for root %s: %w", root, err))
}
return output
}
......@@ -4,10 +4,11 @@ import (
"encoding/binary"
"testing"
interopTypes "github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb"
)
......@@ -19,25 +20,27 @@ type stateOracle interface {
}
type StubBlockOracle struct {
t *testing.T
Blocks map[common.Hash]*types.Block
Outputs map[common.Hash]eth.Output
t *testing.T
Blocks map[common.Hash]*gethTypes.Block
Outputs map[common.Hash]eth.Output
TransitionStates map[common.Hash]*interopTypes.TransitionState
stateOracle
}
func NewStubOracle(t *testing.T) (*StubBlockOracle, *StubStateOracle) {
stateOracle := NewStubStateOracle(t)
blockOracle := StubBlockOracle{
t: t,
Blocks: make(map[common.Hash]*types.Block),
Outputs: make(map[common.Hash]eth.Output),
stateOracle: stateOracle,
t: t,
Blocks: make(map[common.Hash]*gethTypes.Block),
Outputs: make(map[common.Hash]eth.Output),
TransitionStates: make(map[common.Hash]*interopTypes.TransitionState),
stateOracle: stateOracle,
}
return &blockOracle, stateOracle
}
func NewStubOracleWithBlocks(t *testing.T, chain []*types.Block, outputs []eth.Output, db ethdb.Database) *StubBlockOracle {
blocks := make(map[common.Hash]*types.Block, len(chain))
func NewStubOracleWithBlocks(t *testing.T, chain []*gethTypes.Block, outputs []eth.Output, db ethdb.Database) *StubBlockOracle {
blocks := make(map[common.Hash]*gethTypes.Block, len(chain))
for _, block := range chain {
blocks[block.Hash()] = block
}
......@@ -53,7 +56,7 @@ func NewStubOracleWithBlocks(t *testing.T, chain []*types.Block, outputs []eth.O
}
}
func (o StubBlockOracle) BlockByHash(blockHash common.Hash) *types.Block {
func (o StubBlockOracle) BlockByHash(blockHash common.Hash) *gethTypes.Block {
block, ok := o.Blocks[blockHash]
if !ok {
o.t.Fatalf("requested unknown block %s", blockHash)
......@@ -68,8 +71,15 @@ func (o StubBlockOracle) OutputByRoot(root common.Hash) eth.Output {
}
return output
}
func (o StubBlockOracle) TransitionStateByRoot(root common.Hash) *interopTypes.TransitionState {
output, ok := o.TransitionStates[root]
if !ok {
o.t.Fatalf("requested unknown transition state root %s", root)
}
return output
}
func (o StubBlockOracle) BlockDataByHash(agreedBlockHash, blockHash common.Hash, chainID uint64) *types.Block {
func (o StubBlockOracle) BlockDataByHash(agreedBlockHash, blockHash common.Hash, chainID uint64) *gethTypes.Block {
block, ok := o.Blocks[blockHash]
if !ok {
o.t.Fatalf("requested unknown block %s", blockHash)
......
package client
import (
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/client/claim"
"github.com/ethereum-optimism/optimism/op-program/client/l1"
"github.com/ethereum-optimism/optimism/op-program/client/l2"
"github.com/ethereum-optimism/optimism/op-program/client/tasks"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/log"
)
func RunPreInteropProgram(logger log.Logger, bootInfo *boot.BootInfo, l1PreimageOracle *l1.CachingOracle, l2PreimageOracle *l2.CachingOracle) error {
logger.Info("Program Bootstrapped", "bootInfo", bootInfo)
result, err := tasks.RunDerivation(
logger,
bootInfo.RollupConfig,
bootInfo.L2ChainConfig,
bootInfo.L1Head,
bootInfo.L2OutputRoot,
bootInfo.L2ClaimBlockNumber,
l1PreimageOracle,
l2PreimageOracle,
)
if err != nil {
return err
}
return claim.ValidateClaim(logger, result.SafeHead, eth.Bytes32(bootInfo.L2Claim), result.OutputRoot)
}
......@@ -6,11 +6,11 @@ import (
"os"
preimage "github.com/ethereum-optimism/optimism/op-preimage"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/client/claim"
"github.com/ethereum-optimism/optimism/op-program/client/interop"
"github.com/ethereum-optimism/optimism/op-program/client/l1"
"github.com/ethereum-optimism/optimism/op-program/client/l2"
"github.com/ethereum-optimism/optimism/op-program/client/tasks"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/log"
)
......@@ -46,23 +46,9 @@ func RunProgram(logger log.Logger, preimageOracle io.ReadWriter, preimageHinter
l1PreimageOracle := l1.NewCachingOracle(l1.NewPreimageOracle(pClient, hClient))
l2PreimageOracle := l2.NewCachingOracle(l2.NewPreimageOracle(pClient, hClient))
bootInfo := NewBootstrapClient(pClient).BootInfo()
logger.Info("Program Bootstrapped", "bootInfo", bootInfo)
safeHead, outputRoot, err := tasks.RunDerivation(
logger,
bootInfo.RollupConfig,
bootInfo.L2ChainConfig,
bootInfo.L1Head,
bootInfo.L2OutputRoot,
bootInfo.L2ClaimBlockNumber,
l1PreimageOracle,
l2PreimageOracle,
)
if err != nil {
return err
bootInfo := boot.NewBootstrapClient(pClient).BootInfo()
if os.Getenv("OP_PROGRAM_USE_INTEROP") == "true" {
return interop.RunInteropProgram(logger, bootInfo, l1PreimageOracle, l2PreimageOracle, flags == RunProgramFlagValidate)
}
if flags == RunProgramFlagValidate {
return claim.ValidateClaim(logger, safeHead, eth.Bytes32(bootInfo.L2Claim), outputRoot)
}
return nil
return RunPreInteropProgram(logger, bootInfo, l1PreimageOracle, l2PreimageOracle)
}
......@@ -16,7 +16,13 @@ import (
type L2Source interface {
L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (eth.L2BlockRef, error)
L2OutputRoot(uint64) (eth.Bytes32, error)
L2OutputRoot(uint64) (common.Hash, eth.Bytes32, error)
}
type DerivationResult struct {
SafeHead eth.L2BlockRef
BlockHash common.Hash
OutputRoot eth.Bytes32
}
// RunDerivation executes the L2 state transition, given a minimal interface to retrieve data.
......@@ -32,31 +38,35 @@ func RunDerivation(
l2OutputRoot common.Hash,
l2ClaimBlockNum uint64,
l1Oracle l1.Oracle,
l2Oracle l2.Oracle) (eth.L2BlockRef, eth.Bytes32, error) {
l2Oracle l2.Oracle) (DerivationResult, error) {
l1Source := l1.NewOracleL1Client(logger, l1Oracle, l1Head)
l1BlobsSource := l1.NewBlobFetcher(logger, l1Oracle)
engineBackend, err := l2.NewOracleBackedL2Chain(logger, l2Oracle, l1Oracle /* kzg oracle */, l2Cfg, l2OutputRoot)
if err != nil {
return eth.L2BlockRef{}, eth.Bytes32{}, fmt.Errorf("failed to create oracle-backed L2 chain: %w", err)
return DerivationResult{}, fmt.Errorf("failed to create oracle-backed L2 chain: %w", err)
}
l2Source := l2.NewOracleEngine(cfg, logger, engineBackend)
logger.Info("Starting derivation")
d := cldr.NewDriver(logger, cfg, l1Source, l1BlobsSource, l2Source, l2ClaimBlockNum)
if err := d.RunComplete(); err != nil {
return eth.L2BlockRef{}, eth.Bytes32{}, fmt.Errorf("failed to run program to completion: %w", err)
return DerivationResult{}, fmt.Errorf("failed to run program to completion: %w", err)
}
return loadOutputRoot(l2ClaimBlockNum, l2Source)
}
func loadOutputRoot(l2ClaimBlockNum uint64, src L2Source) (eth.L2BlockRef, eth.Bytes32, error) {
func loadOutputRoot(l2ClaimBlockNum uint64, src L2Source) (DerivationResult, error) {
l2Head, err := src.L2BlockRefByLabel(context.Background(), eth.Safe)
if err != nil {
return eth.L2BlockRef{}, eth.Bytes32{}, fmt.Errorf("cannot retrieve safe head: %w", err)
return DerivationResult{}, fmt.Errorf("cannot retrieve safe head: %w", err)
}
outputRoot, err := src.L2OutputRoot(min(l2ClaimBlockNum, l2Head.Number))
blockHash, outputRoot, err := src.L2OutputRoot(min(l2ClaimBlockNum, l2Head.Number))
if err != nil {
return eth.L2BlockRef{}, eth.Bytes32{}, fmt.Errorf("calculate L2 output root: %w", err)
return DerivationResult{}, fmt.Errorf("calculate L2 output root: %w", err)
}
return l2Head, outputRoot, nil
return DerivationResult{
SafeHead: l2Head,
BlockHash: blockHash,
OutputRoot: outputRoot,
}, nil
}
......@@ -6,64 +6,73 @@ import (
"testing"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/stretchr/testify/require"
)
func TestLoadOutputRoot(t *testing.T) {
t.Run("Success", func(t *testing.T) {
expected := eth.Bytes32{0x11}
l2 := &mockL2{
outputRoot: expected,
blockHash: common.Hash{0x24},
outputRoot: eth.Bytes32{0x11},
safeL2: eth.L2BlockRef{Number: 65},
}
safeHead, outputRoot, err := loadOutputRoot(uint64(0), l2)
result, err := loadOutputRoot(uint64(0), l2)
require.NoError(t, err)
require.Equal(t, l2.safeL2, safeHead)
require.Equal(t, expected, outputRoot)
assertDerivationResult(t, result, l2.safeL2, l2.blockHash, l2.outputRoot)
})
t.Run("Success-PriorToSafeHead", func(t *testing.T) {
expected := eth.Bytes32{0x11}
l2 := &mockL2{
blockHash: common.Hash{0x24},
outputRoot: expected,
safeL2: eth.L2BlockRef{
Number: 10,
},
}
safeHead, outputRoot, err := loadOutputRoot(uint64(20), l2)
result, err := loadOutputRoot(uint64(20), l2)
require.NoError(t, err)
require.Equal(t, uint64(10), l2.requestedOutputRoot)
require.Equal(t, l2.safeL2, safeHead)
require.Equal(t, expected, outputRoot)
assertDerivationResult(t, result, l2.safeL2, l2.blockHash, l2.outputRoot)
})
t.Run("Error-SafeHead", func(t *testing.T) {
expectedErr := errors.New("boom")
l2 := &mockL2{
blockHash: common.Hash{0x24},
outputRoot: eth.Bytes32{0x11},
safeL2: eth.L2BlockRef{Number: 10},
safeL2Err: expectedErr,
}
_, _, err := loadOutputRoot(uint64(0), l2)
_, err := loadOutputRoot(uint64(0), l2)
require.ErrorIs(t, err, expectedErr)
})
t.Run("Error-OutputRoot", func(t *testing.T) {
expectedErr := errors.New("boom")
l2 := &mockL2{
blockHash: common.Hash{0x24},
outputRoot: eth.Bytes32{0x11},
outputRootErr: expectedErr,
safeL2: eth.L2BlockRef{Number: 10},
}
_, _, err := loadOutputRoot(uint64(0), l2)
_, err := loadOutputRoot(uint64(0), l2)
require.ErrorIs(t, err, expectedErr)
})
}
func assertDerivationResult(t *testing.T, actual DerivationResult, safeHead eth.L2BlockRef, blockHash common.Hash, outputRoot eth.Bytes32) {
require.Equal(t, safeHead, actual.SafeHead)
require.Equal(t, blockHash, actual.BlockHash)
require.Equal(t, outputRoot, actual.OutputRoot)
}
type mockL2 struct {
safeL2 eth.L2BlockRef
safeL2Err error
blockHash common.Hash
outputRoot eth.Bytes32
outputRootErr error
......@@ -80,12 +89,12 @@ func (m *mockL2) L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (e
return m.safeL2, nil
}
func (m *mockL2) L2OutputRoot(u uint64) (eth.Bytes32, error) {
func (m *mockL2) L2OutputRoot(u uint64) (common.Hash, eth.Bytes32, error) {
m.requestedOutputRoot = u
if m.outputRootErr != nil {
return eth.Bytes32{}, m.outputRootErr
return common.Hash{}, eth.Bytes32{}, m.outputRootErr
}
return m.outputRoot, nil
return m.blockHash, m.outputRoot, nil
}
var _ L2Source = (*mockL2)(nil)
......@@ -9,7 +9,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-program/chainconfig"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/host/config"
"github.com/ethereum-optimism/optimism/op-program/host/types"
oplog "github.com/ethereum-optimism/optimism/op-service/log"
......@@ -181,7 +181,7 @@ func TestL2ChainID(t *testing.T) {
"--rollup.config", rollupCfgFile,
"--l2.genesis", genesisFile,
"--l2.custom"))
require.Equal(t, client.CustomChainIDIndicator, cfg.L2ChainID)
require.Equal(t, boot.CustomChainIDIndicator, cfg.L2ChainID)
})
}
......
......@@ -10,7 +10,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-program/chainconfig"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/host/types"
"github.com/ethereum-optimism/optimism/op-node/rollup"
......@@ -134,7 +134,7 @@ func NewConfig(
_, err := params.LoadOPStackChainConfig(l2ChainID)
if err != nil {
// Unknown chain ID so assume it is custom
l2ChainID = client.CustomChainIDIndicator
l2ChainID = boot.CustomChainIDIndicator
}
return &Config{
L2ChainID: l2ChainID,
......@@ -217,7 +217,7 @@ func NewConfigFromCLI(log log.Logger, ctx *cli.Context) (*Config, error) {
l2ChainID = l2ChainConfig.ChainID.Uint64()
if ctx.Bool(flags.L2Custom.Name) {
log.Warn("Using custom chain configuration via preimage oracle. This is not compatible with on-chain execution.")
l2ChainID = client.CustomChainIDIndicator
l2ChainID = boot.CustomChainIDIndicator
}
}
......
......@@ -8,7 +8,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-program/chainconfig"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/host/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/params"
......@@ -172,7 +172,7 @@ func TestCustomL2ChainID(t *testing.T) {
t.Run("custom", func(t *testing.T) {
customChainConfig := &params.ChainConfig{ChainID: big.NewInt(0x1212121212)}
cfg := NewConfig(validRollupConfig, customChainConfig, validL1Head, validL2Head, validL2OutputRoot, validL2Claim, validL2ClaimBlockNum)
require.Equal(t, cfg.L2ChainID, client.CustomChainIDIndicator)
require.Equal(t, cfg.L2ChainID, boot.CustomChainIDIndicator)
})
}
......
......@@ -9,7 +9,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
preimage "github.com/ethereum-optimism/optimism/op-preimage"
"github.com/ethereum-optimism/optimism/op-program/chainconfig"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/client/l1"
hostcommon "github.com/ethereum-optimism/optimism/op-program/host/common"
"github.com/ethereum-optimism/optimism/op-program/host/config"
......@@ -45,8 +45,8 @@ func TestServerMode(t *testing.T) {
hClient := preimage.NewHintWriter(hintClient)
l1PreimageOracle := l1.NewPreimageOracle(pClient, hClient)
require.Equal(t, l1Head.Bytes(), pClient.Get(client.L1HeadLocalIndex), "Should get l1 head preimages")
require.Equal(t, l2OutputRoot.Bytes(), pClient.Get(client.L2OutputRootLocalIndex), "Should get l2 output root preimages")
require.Equal(t, l1Head.Bytes(), pClient.Get(boot.L1HeadLocalIndex), "Should get l1 head preimages")
require.Equal(t, l2OutputRoot.Bytes(), pClient.Get(boot.L2OutputRootLocalIndex), "Should get l2 output root preimages")
// Should exit when a preimage is unavailable
require.Panics(t, func() {
......
......@@ -4,7 +4,7 @@ import (
"encoding/binary"
"encoding/json"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/host/config"
"github.com/ethereum/go-ethereum/common"
)
......@@ -18,13 +18,13 @@ func NewLocalPreimageSource(config *config.Config) *LocalPreimageSource {
}
var (
l1HeadKey = client.L1HeadLocalIndex.PreimageKey()
l2OutputRootKey = client.L2OutputRootLocalIndex.PreimageKey()
l2ClaimKey = client.L2ClaimLocalIndex.PreimageKey()
l2ClaimBlockNumberKey = client.L2ClaimBlockNumberLocalIndex.PreimageKey()
l2ChainIDKey = client.L2ChainIDLocalIndex.PreimageKey()
l2ChainConfigKey = client.L2ChainConfigLocalIndex.PreimageKey()
rollupKey = client.RollupConfigLocalIndex.PreimageKey()
l1HeadKey = boot.L1HeadLocalIndex.PreimageKey()
l2OutputRootKey = boot.L2OutputRootLocalIndex.PreimageKey()
l2ClaimKey = boot.L2ClaimLocalIndex.PreimageKey()
l2ClaimBlockNumberKey = boot.L2ClaimBlockNumberLocalIndex.PreimageKey()
l2ChainIDKey = boot.L2ChainIDLocalIndex.PreimageKey()
l2ChainConfigKey = boot.L2ChainConfigLocalIndex.PreimageKey()
rollupKey = boot.RollupConfigLocalIndex.PreimageKey()
)
func (s *LocalPreimageSource) Get(key common.Hash) ([]byte, error) {
......@@ -40,12 +40,12 @@ func (s *LocalPreimageSource) Get(key common.Hash) ([]byte, error) {
case l2ChainIDKey:
return binary.BigEndian.AppendUint64(nil, s.config.L2ChainID), nil
case l2ChainConfigKey:
if s.config.L2ChainID != client.CustomChainIDIndicator {
if s.config.L2ChainID != boot.CustomChainIDIndicator {
return nil, ErrNotFound
}
return json.Marshal(s.config.L2ChainConfig)
case rollupKey:
if s.config.L2ChainID != client.CustomChainIDIndicator {
if s.config.L2ChainID != boot.CustomChainIDIndicator {
return nil, ErrNotFound
}
return json.Marshal(s.config.Rollup)
......
......@@ -7,7 +7,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
preimage "github.com/ethereum-optimism/optimism/op-preimage"
"github.com/ethereum-optimism/optimism/op-program/client"
"github.com/ethereum-optimism/optimism/op-program/client/boot"
"github.com/ethereum-optimism/optimism/op-program/host/config"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/params"
......@@ -55,7 +55,7 @@ func TestLocalPreimageSource(t *testing.T) {
func TestGetCustomChainConfigPreimages(t *testing.T) {
cfg := &config.Config{
Rollup: chaincfg.OPSepolia(),
L2ChainID: client.CustomChainIDIndicator,
L2ChainID: boot.CustomChainIDIndicator,
L1Head: common.HexToHash("0x1111"),
L2OutputRoot: common.HexToHash("0x2222"),
L2Claim: common.HexToHash("0x3333"),
......
......@@ -7,6 +7,16 @@ import (
"github.com/stretchr/testify/require"
)
func TestUnmarshalOutput_UnknownVersion(t *testing.T) {
_, err := UnmarshalOutput([]byte{0: 0xA, 32: 0xA})
require.ErrorIs(t, err, ErrInvalidOutputVersion)
}
func TestUnmarshalOutput_TooShortForVersion(t *testing.T) {
_, err := UnmarshalOutput([]byte{0xA})
require.ErrorIs(t, err, ErrInvalidOutput)
}
func TestOutputV0Codec(t *testing.T) {
output := OutputV0{
StateRoot: Bytes32{1, 2, 3},
......@@ -19,8 +29,6 @@ func TestOutputV0Codec(t *testing.T) {
unmarshaledV0 := unmarshaled.(*OutputV0)
require.Equal(t, output, *unmarshaledV0)
_, err = UnmarshalOutput([]byte{0: 0xA, 32: 0xA})
require.ErrorIs(t, err, ErrInvalidOutputVersion)
_, err = UnmarshalOutput([]byte{64: 0xA})
require.ErrorIs(t, err, ErrInvalidOutput)
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment