Commit 96f78bba authored by Inphi's avatar Inphi Committed by GitHub

op-program: Implement consolidation for interop (#13775)

* op-program: Implement consolidation for interop

* address todos; fix claim validation bug

* fix test

* fix todos

* op-program: Fix short-circuit logic in block walk

* op-program: Use l2-chain block fetcher in canon

* op-program: Fix current header in engine_backend

* op-program: Filter hazardrous exec msgs

* cleanup

* fix block cache lookup

* snap to earliest block and update tests
Co-authored-by: default avatarAdrian Sutton <adrian@oplabs.co>

* use adrian's suggestion

* todo formatting

---------
Co-authored-by: default avatarAdrian Sutton <adrian@oplabs.co>
parent 78badddf
......@@ -408,7 +408,6 @@ func TestInteropFaultProofs(gt *testing.T) {
agreedClaim: paddingStep(1023),
disputedClaim: end.Marshal(),
expectValid: true,
skip: true,
},
{
......
package interop
import (
"fmt"
"github.com/ethereum-optimism/optimism/op-program/client/interop/types"
"github.com/ethereum-optimism/optimism/op-program/client/l2"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/cross"
"github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/depset"
"github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/processors"
supervisortypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types"
"github.com/ethereum/go-ethereum/common"
ethtypes "github.com/ethereum/go-ethereum/core/types"
)
func ReceiptsToExecutingMessages(depset depset.ChainIndexFromID, receipts ethtypes.Receipts) ([]*supervisortypes.ExecutingMessage, uint32, error) {
var execMsgs []*supervisortypes.ExecutingMessage
var logCount uint32
for _, rcpt := range receipts {
logCount += uint32(len(rcpt.Logs))
for _, l := range rcpt.Logs {
execMsg, err := processors.DecodeExecutingMessageLog(l, depset)
if err != nil {
return nil, 0, err
}
// TODO: e2e test for both executing and non-executing messages in the logs
if execMsg != nil {
execMsgs = append(execMsgs, execMsg)
}
}
}
return execMsgs, logCount, nil
}
func RunConsolidation(deps ConsolidateCheckDeps,
oracle l2.Oracle,
transitionState *types.TransitionState,
superRoot *eth.SuperV1,
) (eth.Bytes32, error) {
var consolidatedChains []eth.ChainIDAndOutput
for i, chain := range superRoot.Chains {
progress := transitionState.PendingProgress[i]
// TODO(#13776): hint block data execution in case the pending progress is not canonical so we can fetch the correct receipts
block, receipts := oracle.ReceiptsByBlockHash(progress.BlockHash, chain.ChainID)
execMsgs, _, err := ReceiptsToExecutingMessages(deps.DependencySet(), receipts)
if err != nil {
return eth.Bytes32{}, err
}
candidate := supervisortypes.BlockSeal{
Hash: progress.BlockHash,
Number: block.NumberU64(),
Timestamp: block.Time(),
}
if err := checkHazards(deps, candidate, eth.ChainIDFromUInt64(chain.ChainID), execMsgs); err != nil {
// TODO(#13776): replace with deposit-only block if ErrConflict, ErrCycle, or ErrFuture
return eth.Bytes32{}, err
}
consolidatedChains = append(consolidatedChains, eth.ChainIDAndOutput{
ChainID: chain.ChainID,
// TODO(#13776): when applicable, use the deposit-only block output root
Output: progress.OutputRoot,
})
}
consolidatedSuper := &eth.SuperV1{
Timestamp: superRoot.Timestamp + 1,
Chains: consolidatedChains,
}
return eth.SuperRoot(consolidatedSuper), nil
}
type ConsolidateCheckDeps interface {
cross.UnsafeFrontierCheckDeps
cross.CycleCheckDeps
Check(
chain eth.ChainID,
blockNum uint64,
timestamp uint64,
logIdx uint32,
logHash common.Hash,
) (includedIn supervisortypes.BlockSeal, err error)
}
func checkHazards(
deps ConsolidateCheckDeps,
candidate supervisortypes.BlockSeal,
chainID eth.ChainID,
execMsgs []*supervisortypes.ExecutingMessage,
) error {
hazards, err := cross.CrossUnsafeHazards(deps, chainID, candidate, execMsgs)
if err != nil {
return err
}
if err := cross.HazardUnsafeFrontierChecks(deps, hazards); err != nil {
return err
}
if err := cross.HazardCycleChecks(deps.DependencySet(), deps, candidate.Timestamp, hazards); err != nil {
return err
}
return nil
}
type consolidateCheckDeps struct {
oracle l2.Oracle
depset depset.DependencySet
canonBlocks map[uint64]*l2.CanonicalBlockHeaderOracle
}
func newConsolidateCheckDeps(chains []eth.ChainIDAndOutput, oracle l2.Oracle) (*consolidateCheckDeps, error) {
// TODO: handle case where dep set changes in a given timestamp
// TODO: Also replace dep set stubs with the actual dependency set in the RollupConfig.
deps := make(map[eth.ChainID]*depset.StaticConfigDependency)
for i, chain := range chains {
deps[eth.ChainIDFromUInt64(chain.ChainID)] = &depset.StaticConfigDependency{
ChainIndex: supervisortypes.ChainIndex(i),
ActivationTime: 0,
HistoryMinTime: 0,
}
}
canonBlocks := make(map[uint64]*l2.CanonicalBlockHeaderOracle)
for _, chain := range chains {
output := oracle.OutputByRoot(common.Hash(chain.Output), chain.ChainID)
outputV0, ok := output.(*eth.OutputV0)
if !ok {
return nil, fmt.Errorf("unexpected output type: %T", output)
}
head := oracle.BlockByHash(outputV0.BlockHash, chain.ChainID)
blockByHash := func(hash common.Hash) *ethtypes.Block {
return oracle.BlockByHash(hash, chain.ChainID)
}
canonBlocks[chain.ChainID] = l2.NewCanonicalBlockHeaderOracle(head.Header(), blockByHash)
}
depset, err := depset.NewStaticConfigDependencySet(deps)
if err != nil {
return nil, fmt.Errorf("unexpected error: failed to create dependency set: %w", err)
}
return &consolidateCheckDeps{
oracle: oracle,
depset: depset,
canonBlocks: canonBlocks,
}, nil
}
func (d *consolidateCheckDeps) Check(
chain eth.ChainID,
blockNum uint64,
timestamp uint64,
logIdx uint32,
logHash common.Hash,
) (includedIn supervisortypes.BlockSeal, err error) {
// We can assume the oracle has the block the executing message is in
block, err := d.BlockByNumber(d.oracle, blockNum, chain.ToBig().Uint64())
if err != nil {
return supervisortypes.BlockSeal{}, err
}
return supervisortypes.BlockSeal{
Hash: block.Hash(),
Number: block.NumberU64(),
Timestamp: block.Time(),
}, nil
}
func (d *consolidateCheckDeps) IsCrossUnsafe(chainID eth.ChainID, block eth.BlockID) error {
// Assumed to be cross-unsafe. And hazard checks will catch any future blocks prior to calling this
return nil
}
func (d *consolidateCheckDeps) IsLocalUnsafe(chainID eth.ChainID, block eth.BlockID) error {
// Always assumed to be local-unsafe
return nil
}
func (d *consolidateCheckDeps) ParentBlock(chainID eth.ChainID, parentOf eth.BlockID) (parent eth.BlockID, err error) {
block, err := d.BlockByNumber(d.oracle, parentOf.Number-1, chainID.ToBig().Uint64())
if err != nil {
return eth.BlockID{}, err
}
return eth.BlockID{
Hash: block.Hash(),
Number: block.NumberU64(),
}, nil
}
func (d *consolidateCheckDeps) OpenBlock(
chainID eth.ChainID,
blockNum uint64,
) (ref eth.BlockRef, logCount uint32, execMsgs map[uint32]*supervisortypes.ExecutingMessage, err error) {
block, err := d.BlockByNumber(d.oracle, blockNum, chainID.ToBig().Uint64())
if err != nil {
return eth.BlockRef{}, 0, nil, err
}
ref = eth.BlockRef{
Hash: block.Hash(),
Number: block.NumberU64(),
}
_, receipts := d.oracle.ReceiptsByBlockHash(block.Hash(), chainID.ToBig().Uint64())
execs, logCount, err := ReceiptsToExecutingMessages(d.depset, receipts)
if err != nil {
return eth.BlockRef{}, 0, nil, err
}
execMsgs = make(map[uint32]*supervisortypes.ExecutingMessage, len(execs))
for _, exec := range execs {
execMsgs[exec.LogIdx] = exec
}
return ref, uint32(logCount), execMsgs, nil
}
func (d *consolidateCheckDeps) DependencySet() depset.DependencySet {
return d.depset
}
func (d *consolidateCheckDeps) BlockByNumber(oracle l2.Oracle, blockNum uint64, chainID uint64) (*ethtypes.Block, error) {
head := d.canonBlocks[chainID].GetHeaderByNumber(blockNum)
if head == nil {
return nil, fmt.Errorf("head not found for chain %v", chainID)
}
return d.oracle.BlockByHash(head.Hash(), chainID), nil
}
......@@ -26,6 +26,10 @@ var (
InvalidTransitionHash = crypto.Keccak256Hash(InvalidTransition)
)
const (
ConsolidateStep = 1023
)
type taskExecutor interface {
RunDerivation(
logger log.Logger,
......@@ -82,7 +86,22 @@ func stateTransition(logger log.Logger, bootInfo *boot.BootInfoInterop, l1Preima
return common.Hash{}, err
}
expectedPendingProgress = append(expectedPendingProgress, block)
} else if transitionState.Step == ConsolidateStep {
// sanity check
if len(transitionState.PendingProgress) >= ConsolidateStep {
return common.Hash{}, fmt.Errorf("pending progress length does not match the expected step")
}
deps, err := newConsolidateCheckDeps(superRoot.Chains, l2PreimageOracle)
if err != nil {
return common.Hash{}, fmt.Errorf("failed to create consolidate check deps: %w", err)
}
expectedSuperRoot, err := RunConsolidation(deps, l2PreimageOracle, transitionState, superRoot)
if err != nil {
return common.Hash{}, err
}
return common.Hash(expectedSuperRoot), nil
}
finalState := &types.TransitionState{
SuperRoot: transitionState.SuperRoot,
PendingProgress: expectedPendingProgress,
......
......@@ -17,8 +17,10 @@ import (
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/testlog"
"github.com/ethereum/go-ethereum/common"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie"
"github.com/stretchr/testify/require"
)
......@@ -117,6 +119,70 @@ func TestNoOpStep(t *testing.T) {
verifyResult(t, logger, tasksStub, configSource, l2PreimageOracle, outputRootHash, agreedSuperRoot.Timestamp+100000, expectedClaim)
}
func TestDeriveBlockForConsolidateStep(t *testing.T) {
logger := testlog.Logger(t, log.LevelError)
configSource, agreedSuperRoot, tasksStub := setupTwoChains()
block1 := createBlock(1)
block2 := createBlock(2)
output := &eth.OutputV0{BlockHash: block1.Hash()}
agreedTransitionState := &types.TransitionState{
SuperRoot: agreedSuperRoot.Marshal(),
PendingProgress: []types.OptimisticBlock{
{BlockHash: common.Hash{0xaa}, OutputRoot: eth.OutputRoot(output)},
{BlockHash: tasksStub.blockHash, OutputRoot: eth.OutputRoot(output)},
},
Step: ConsolidateStep,
}
outputRootHash := agreedTransitionState.Hash()
l2PreimageOracle, _ := test.NewStubOracle(t)
l2PreimageOracle.TransitionStates[outputRootHash] = agreedTransitionState
l2PreimageOracle.Outputs[common.Hash(eth.OutputRoot(&eth.OutputV0{BlockHash: common.Hash{0x11}}))] = output
l2PreimageOracle.Outputs[common.Hash(eth.OutputRoot(&eth.OutputV0{BlockHash: common.Hash{0x22}}))] = output
l2PreimageOracle.Blocks[output.BlockHash] = block1
l2PreimageOracle.Blocks[common.Hash{0xaa}] = block2
l2PreimageOracle.Blocks[tasksStub.blockHash] = block2
l2PreimageOracle.Receipts[common.Hash{0xaa}] = gethTypes.Receipts{}
l2PreimageOracle.Receipts[tasksStub.blockHash] = gethTypes.Receipts{}
expectedClaim := common.Hash(eth.SuperRoot(&eth.SuperV1{
Timestamp: agreedSuperRoot.Timestamp + 1,
Chains: []eth.ChainIDAndOutput{
{
ChainID: configSource.rollupCfgs[0].L2ChainID.Uint64(),
Output: agreedTransitionState.PendingProgress[0].OutputRoot,
},
{
ChainID: configSource.rollupCfgs[1].L2ChainID.Uint64(),
Output: agreedTransitionState.PendingProgress[1].OutputRoot,
},
},
}))
verifyResult(
t,
logger,
tasksStub,
configSource,
l2PreimageOracle,
outputRootHash,
agreedSuperRoot.Timestamp+100000,
expectedClaim,
)
}
func createBlock(num int64) *gethTypes.Block {
return gethTypes.NewBlock(
&gethTypes.Header{Number: big.NewInt(num)},
nil,
nil,
trie.NewStackTrie(nil),
gethTypes.DefaultBlockConfig,
)
}
func TestTraceExtensionOnceClaimedTimestampIsReached(t *testing.T) {
logger := testlog.Logger(t, log.LevelError)
configSource, agreedSuperRoot, tasksStub := setupTwoChains()
......
......@@ -13,11 +13,13 @@ import (
const blockCacheSize = 3_000
const nodeCacheSize = 100_000
const codeCacheSize = 10_000
const receiptsCacheSize = 100
type CachingOracle struct {
oracle Oracle
blocks *simplelru.LRU[common.Hash, *types.Block]
nodes *simplelru.LRU[common.Hash, []byte]
rcpts *simplelru.LRU[common.Hash, types.Receipts]
codes *simplelru.LRU[common.Hash, []byte]
outputs *simplelru.LRU[common.Hash, eth.Output]
}
......@@ -25,11 +27,13 @@ type CachingOracle struct {
func NewCachingOracle(oracle Oracle) *CachingOracle {
blockLRU, _ := simplelru.NewLRU[common.Hash, *types.Block](blockCacheSize, nil)
nodeLRU, _ := simplelru.NewLRU[common.Hash, []byte](nodeCacheSize, nil)
rcptsLRU, _ := simplelru.NewLRU[common.Hash, types.Receipts](receiptsCacheSize, nil)
codeLRU, _ := simplelru.NewLRU[common.Hash, []byte](codeCacheSize, nil)
outputLRU, _ := simplelru.NewLRU[common.Hash, eth.Output](codeCacheSize, nil)
return &CachingOracle{
oracle: oracle,
blocks: blockLRU,
rcpts: rcptsLRU,
nodes: nodeLRU,
codes: codeLRU,
outputs: outputLRU,
......@@ -46,6 +50,17 @@ func (o *CachingOracle) NodeByHash(nodeHash common.Hash, chainID uint64) []byte
return node
}
func (o *CachingOracle) ReceiptsByBlockHash(blockHash common.Hash, chainID uint64) (*types.Block, types.Receipts) {
rcpts, ok := o.rcpts.Get(blockHash)
if ok {
return o.BlockByHash(blockHash, chainID), rcpts
}
block, rcpts := o.oracle.ReceiptsByBlockHash(blockHash, chainID)
o.blocks.Add(blockHash, block)
o.rcpts.Add(blockHash, rcpts)
return block, rcpts
}
func (o *CachingOracle) CodeByHash(codeHash common.Hash, chainID uint64) []byte {
code, ok := o.codes.Get(codeHash)
if ok {
......
......@@ -51,6 +51,29 @@ func TestNodeByHash(t *testing.T) {
require.Equal(t, node, actual)
}
func TestReceiptsByBlockHash(t *testing.T) {
chainID := uint64(48294)
stub, _ := test.NewStubOracle(t)
oracle := NewCachingOracle(stub)
rng := rand.New(rand.NewSource(1))
block, rcpts := testutils.RandomBlock(rng, 1)
// Initial call retrieves from the stub
stub.Blocks[block.Hash()] = block
stub.Receipts[block.Hash()] = rcpts
actualBlock, actualRcpts := oracle.ReceiptsByBlockHash(block.Hash(), chainID)
require.EqualValues(t, block, actualBlock)
require.EqualValues(t, rcpts, actualRcpts)
// Later calls should retrieve from cache (even if chain ID is different)
delete(stub.Blocks, block.Hash())
delete(stub.Receipts, block.Hash())
actualBlock, actualRcpts = oracle.ReceiptsByBlockHash(block.Hash(), 9982)
require.EqualValues(t, block, actualBlock)
require.EqualValues(t, rcpts, actualRcpts)
}
func TestCodeByHash(t *testing.T) {
stub, stateStub := test.NewStubOracle(t)
oracle := NewCachingOracle(stub)
......
package l2
import (
"fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
)
type BlockByHashFn func(hash common.Hash) *types.Block
type CanonicalBlockHeaderOracle struct {
head *types.Header
hashByNum map[uint64]common.Hash
earliestIndexedBlock *types.Header
blockByHashFn BlockByHashFn
}
func NewCanonicalBlockHeaderOracle(head *types.Header, blockByHashFn BlockByHashFn) *CanonicalBlockHeaderOracle {
return &CanonicalBlockHeaderOracle{
head: head,
hashByNum: map[uint64]common.Hash{
head.Number.Uint64(): head.Hash(),
},
earliestIndexedBlock: head,
blockByHashFn: blockByHashFn,
}
}
func (o *CanonicalBlockHeaderOracle) CurrentHeader() *types.Header {
return o.head
}
// GetHeaderByNumber walks back from the current head to the requested block number
func (o *CanonicalBlockHeaderOracle) GetHeaderByNumber(n uint64) *types.Header {
if o.head.Number.Uint64() < n {
return nil
}
if o.earliestIndexedBlock.Number.Uint64() <= n {
// guaranteed to be cached during lookup
hash, ok := o.hashByNum[n]
if !ok {
panic(fmt.Errorf("block %v was not indexed when earliest block number is %v", n, o.earliestIndexedBlock.Number))
}
return o.blockByHashFn(hash).Header()
}
h := o.earliestIndexedBlock
for h.Number.Uint64() > n {
h = o.blockByHashFn(h.ParentHash).Header()
o.hashByNum[h.Number.Uint64()] = h.Hash()
}
o.earliestIndexedBlock = h
return h
}
func (o *CanonicalBlockHeaderOracle) SetCanonical(head *types.Header) common.Hash {
oldHead := o.head
o.head = head
// Remove canonical hashes after the new header
for n := head.Number.Uint64() + 1; n <= oldHead.Number.Uint64(); n++ {
delete(o.hashByNum, n)
}
// Add new canonical blocks to the block by number cache
// Since the original head is added to the block number cache and acts as the finalized block,
// at some point we must reach the existing canonical chain and can stop updating.
h := o.head
for {
newHash := h.Hash()
prevHash, ok := o.hashByNum[h.Number.Uint64()]
if ok && prevHash == newHash {
// Connected with the existing canonical chain so stop updating
break
}
o.hashByNum[h.Number.Uint64()] = newHash
h = o.blockByHashFn(h.ParentHash).Header()
}
o.earliestIndexedBlock = h
return head.Hash()
}
package l2
import (
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/stretchr/testify/require"
)
func TestCanonicalBlockNumberOracle_GetHeaderByNumber(t *testing.T) {
headBlockNumber := 3
blockCount := 3
chainCfg, blocks, oracle := setupOracle(t, blockCount, headBlockNumber, true)
head := blocks[headBlockNumber].Header()
// Ensure we don't walk back from head on every lookup by failing if a block is loaded multiple times.
requestedBlocks := make(map[common.Hash]bool)
blockByHash := func(hash common.Hash) *types.Block {
if requestedBlocks[hash] {
t.Fatalf("Requested duplicate block: %v", hash)
}
requestedBlocks[hash] = true
return oracle.BlockByHash(hash, chainCfg.ChainID.Uint64())
}
canon := NewCanonicalBlockHeaderOracle(head, blockByHash)
require.Equal(t, head.Hash(), canon.CurrentHeader().Hash())
require.Nil(t, canon.GetHeaderByNumber(4))
oracle.Blocks[blocks[3].Hash()] = blocks[3]
h := canon.GetHeaderByNumber(3)
require.Equal(t, blocks[3].Hash(), h.Hash())
oracle.Blocks[blocks[2].Hash()] = blocks[2]
h = canon.GetHeaderByNumber(2)
require.Equal(t, blocks[2].Hash(), h.Hash())
oracle.Blocks[blocks[1].Hash()] = blocks[1]
h = canon.GetHeaderByNumber(1)
require.Equal(t, blocks[1].Hash(), h.Hash())
oracle.Blocks[blocks[0].Hash()] = blocks[0]
h = canon.GetHeaderByNumber(0)
require.Equal(t, blocks[0].Hash(), h.Hash())
// Test that the block hash is cached. Do not expect oracle requests for any other blocks.
// Allow requesting block 1 again as we're specifically asking for it and only the hash is cached
requestedBlocks[blocks[1].Hash()] = false
oracle.Blocks = map[common.Hash]*types.Block{
blocks[1].Hash(): blocks[1],
}
require.Equal(t, blocks[1].Hash(), canon.GetHeaderByNumber(1).Hash())
}
func TestCanonicalBlockNumberOracle_SetCanonical(t *testing.T) {
headBlockNumber := 3
blockCount := 3
t.Run("set canonical on fork", func(t *testing.T) {
chainCfg, blocks, oracle := setupOracle(t, blockCount, headBlockNumber, true)
head := blocks[headBlockNumber].Header()
blockRequestCount := 0
blockByHash := func(hash common.Hash) *types.Block {
blockRequestCount++
return oracle.BlockByHash(hash, chainCfg.ChainID.Uint64())
}
canon := NewCanonicalBlockHeaderOracle(head, blockByHash)
oracle.Blocks[blocks[2].Hash()] = blocks[2]
oracle.Blocks[blocks[1].Hash()] = blocks[1]
oracle.Blocks[blocks[0].Hash()] = blocks[0]
h := canon.GetHeaderByNumber(0)
require.Equal(t, blocks[0].Hash(), h.Hash())
// Create an alternate block 2
header2b := *blocks[2].Header()
header2b.Time = header2b.Time + 1
block2b := types.NewBlockWithHeader(&header2b)
require.NotEqual(t, blocks[2].Hash(), block2b.Hash())
oracle.Blocks[block2b.Hash()] = block2b
canon.SetCanonical(block2b.Header())
require.Equal(t, block2b.Hash(), canon.CurrentHeader().Hash())
blockRequestCount = 0
require.Nil(t, canon.GetHeaderByNumber(3), "Should have removed block 3 from cache")
require.Equal(t, 0, blockRequestCount, "Should not have needed to fetch a block")
h = canon.GetHeaderByNumber(2)
require.Equal(t, block2b.Hash(), h.Hash(), "Should replace block 2 in cache")
require.Equal(t, 1, blockRequestCount, "Should not have used cache")
blockRequestCount = 0
h = canon.GetHeaderByNumber(1)
require.Equal(t, blocks[1].Hash(), h.Hash(), "Should retain block 1")
require.Equal(t, 1, blockRequestCount, "Should not have used cache")
})
t.Run("set canonical on same chain", func(t *testing.T) {
chainCfg, blocks, oracle := setupOracle(t, blockCount, headBlockNumber, true)
head := blocks[headBlockNumber].Header()
blockByHash := func(hash common.Hash) *types.Block {
return oracle.BlockByHash(hash, chainCfg.ChainID.Uint64())
}
canon := NewCanonicalBlockHeaderOracle(head, blockByHash)
oracle.Blocks[blocks[2].Hash()] = blocks[2]
oracle.Blocks[blocks[1].Hash()] = blocks[1]
oracle.Blocks[blocks[0].Hash()] = blocks[0]
h := canon.GetHeaderByNumber(0)
require.Equal(t, blocks[0].Hash(), h.Hash())
canon.SetCanonical(blocks[2].Header())
require.Equal(t, blocks[2].Hash(), canon.CurrentHeader().Hash())
require.Nil(t, canon.GetHeaderByNumber(3))
// earliest block cache is unchanged.
oracle.Blocks = map[common.Hash]*types.Block{
blocks[1].Hash(): blocks[1],
}
require.Equal(t, blocks[1].Hash(), canon.GetHeaderByNumber(1).Hash())
})
}
......@@ -26,14 +26,11 @@ type OracleBackedL2Chain struct {
chainCfg *params.ChainConfig
engine consensus.Engine
oracleHead *types.Header
head *types.Header
safe *types.Header
finalized *types.Header
vmCfg vm.Config
// Block by number cache
hashByNum map[uint64]common.Hash
earliestIndexedBlock *types.Header
canon *CanonicalBlockHeaderOracle
// Inserted blocks
blocks map[common.Hash]*types.Block
......@@ -53,19 +50,14 @@ func NewOracleBackedL2Chain(logger log.Logger, oracle Oracle, precompileOracle e
}
head := oracle.BlockByHash(outputV0.BlockHash, chainID)
logger.Info("Loaded L2 head", "hash", head.Hash(), "number", head.Number())
return &OracleBackedL2Chain{
chain := &OracleBackedL2Chain{
log: logger,
oracle: oracle,
chainCfg: chainCfg,
engine: beacon.New(nil),
hashByNum: map[uint64]common.Hash{
head.NumberU64(): head.Hash(),
},
earliestIndexedBlock: head.Header(),
// Treat the agreed starting head as finalized - nothing before it can be disputed
head: head.Header(),
safe: head.Header(),
finalized: head.Header(),
oracleHead: head.Header(),
......@@ -74,29 +66,21 @@ func NewOracleBackedL2Chain(logger log.Logger, oracle Oracle, precompileOracle e
vmCfg: vm.Config{
PrecompileOverrides: engineapi.CreatePrecompileOverrides(precompileOracle),
},
}, nil
}
// Use the chain's GetBlockByHash to ensure newly built blocks are visible to the canonical chain
blockByHash := func(hash common.Hash) *types.Block {
return chain.GetBlockByHash(hash)
}
chain.canon = NewCanonicalBlockHeaderOracle(head.Header(), blockByHash)
return chain, nil
}
func (o *OracleBackedL2Chain) CurrentHeader() *types.Header {
return o.head
return o.canon.CurrentHeader()
}
func (o *OracleBackedL2Chain) GetHeaderByNumber(n uint64) *types.Header {
if o.head.Number.Uint64() < n {
return nil
}
hash, ok := o.hashByNum[n]
if ok {
return o.GetHeaderByHash(hash)
}
// Walk back from current head to the requested block number
h := o.head
for h.Number.Uint64() > n {
h = o.GetHeaderByHash(h.ParentHash)
o.hashByNum[h.Number.Uint64()] = h.Hash()
}
o.earliestIndexedBlock = h
return h
return o.canon.GetHeaderByNumber(n)
}
func (o *OracleBackedL2Chain) GetTd(hash common.Hash, number uint64) *big.Int {
......@@ -218,29 +202,7 @@ func (o *OracleBackedL2Chain) AssembleAndInsertBlockWithoutSetHead(processor *en
}
func (o *OracleBackedL2Chain) SetCanonical(head *types.Block) (common.Hash, error) {
oldHead := o.head
o.head = head.Header()
// Remove canonical hashes after the new header
for n := head.NumberU64() + 1; n <= oldHead.Number.Uint64(); n++ {
delete(o.hashByNum, n)
}
// Add new canonical blocks to the block by number cache
// Since the original head is added to the block number cache and acts as the finalized block,
// at some point we must reach the existing canonical chain and can stop updating.
h := o.head
for {
newHash := h.Hash()
prevHash, ok := o.hashByNum[h.Number.Uint64()]
if ok && prevHash == newHash {
// Connected with the existing canonical chain so stop updating
break
}
o.hashByNum[h.Number.Uint64()] = newHash
h = o.GetHeaderByHash(h.ParentHash)
}
return head.Hash(), nil
return o.canon.SetCanonical(head.Header()), nil
}
func (o *OracleBackedL2Chain) SetFinalized(header *types.Header) {
......
......@@ -13,6 +13,7 @@ import (
const (
HintL2BlockHeader = "l2-block-header"
HintL2Transactions = "l2-transactions"
HintL2Receipts = "l2-receipts"
HintL2Code = "l2-code"
HintL2StateNode = "l2-state-node"
HintL2Output = "l2-output"
......@@ -64,6 +65,14 @@ func (l TransactionsHint) Hint() string {
return HintL2Transactions + " " + hexutil.Encode(HashAndChainID(l).Marshal())
}
type ReceiptsHint HashAndChainID
var _ preimage.Hint = ReceiptsHint{}
func (l ReceiptsHint) Hint() string {
return HintL2Receipts + " " + hexutil.Encode(HashAndChainID(l).Marshal())
}
type CodeHint HashAndChainID
var _ preimage.Hint = CodeHint{}
......
......@@ -40,6 +40,8 @@ type Oracle interface {
BlockDataByHash(agreedBlockHash, blockHash common.Hash, chainID uint64) *types.Block
TransitionStateByRoot(root common.Hash) *interopTypes.TransitionState
ReceiptsByBlockHash(blockHash common.Hash, chainID uint64) (*types.Block, types.Receipts)
}
// PreimageOracle implements Oracle using by interfacing with the pure preimage.Oracle
......@@ -152,3 +154,20 @@ func (p *PreimageOracle) TransitionStateByRoot(root common.Hash) *interopTypes.T
}
return output
}
func (p *PreimageOracle) ReceiptsByBlockHash(blockHash common.Hash, chainID uint64) (*types.Block, types.Receipts) {
block := p.BlockByHash(blockHash, chainID)
p.hint.Hint(ReceiptsHint{Hash: blockHash, ChainID: chainID})
opaqueReceipts := mpt.ReadTrie(block.ReceiptHash(), func(key common.Hash) []byte {
return p.oracle.Get(preimage.Keccak256Key(key))
})
txHashes := make([]common.Hash, len(block.Transactions()))
for i, tx := range block.Transactions() {
txHashes[i] = tx.Hash()
}
receipts, err := eth.DecodeRawReceipts(eth.ToBlockID(block), opaqueReceipts, txHashes)
if err != nil {
panic(fmt.Errorf("failed to decode receipts for block %v: %w", block.Hash(), err))
}
return block, receipts
}
......@@ -22,6 +22,7 @@ type stateOracle interface {
type StubBlockOracle struct {
t *testing.T
Blocks map[common.Hash]*gethTypes.Block
Receipts map[common.Hash]gethTypes.Receipts
Outputs map[common.Hash]eth.Output
TransitionStates map[common.Hash]*interopTypes.TransitionState
stateOracle
......@@ -34,6 +35,7 @@ func NewStubOracle(t *testing.T) (*StubBlockOracle, *StubStateOracle) {
Blocks: make(map[common.Hash]*gethTypes.Block),
Outputs: make(map[common.Hash]eth.Output),
TransitionStates: make(map[common.Hash]*interopTypes.TransitionState),
Receipts: make(map[common.Hash]gethTypes.Receipts),
stateOracle: stateOracle,
}
return &blockOracle, stateOracle
......@@ -87,6 +89,14 @@ func (o StubBlockOracle) BlockDataByHash(agreedBlockHash, blockHash common.Hash,
return block
}
func (o StubBlockOracle) ReceiptsByBlockHash(blockHash common.Hash, chainID uint64) (*gethTypes.Block, gethTypes.Receipts) {
receipts, ok := o.Receipts[blockHash]
if !ok {
o.t.Fatalf("requested unknown receipts for block %s", blockHash)
}
return o.BlockByHash(blockHash, chainID), receipts
}
// KvStateOracle loads data from a source ethdb.KeyValueStore
type KvStateOracle struct {
t *testing.T
......
......@@ -91,6 +91,14 @@ func (l *L2Source) CodeByHash(ctx context.Context, hash common.Hash) ([]byte, er
return l.canonicalDebugClient.CodeByHash(ctx, hash)
}
// FetchReceipts implements prefetcher.L2Source.
func (l *L2Source) FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error) {
if l.ExperimentalEnabled() {
return l.experimentalClient.FetchReceipts(ctx, blockHash)
}
return l.canonicalEthClient.FetchReceipts(ctx, blockHash)
}
// NodeByHash implements prefetcher.L2Source.
func (l *L2Source) NodeByHash(ctx context.Context, hash common.Hash) ([]byte, error) {
if l.ExperimentalEnabled() {
......
......@@ -294,6 +294,20 @@ func (p *Prefetcher) prefetch(ctx context.Context, hint string) error {
return fmt.Errorf("failed to fetch L2 state node %s: %w", hash, err)
}
return p.kvStore.Put(preimage.Keccak256Key(hash).PreimageKey(), node)
case l2.HintL2Receipts:
hash, chainID, err := p.parseHashAndChainID("L2 receipts", hintBytes)
if err != nil {
return err
}
source, err := p.l2Sources.ForChainID(chainID)
if err != nil {
return err
}
_, receipts, err := source.FetchReceipts(ctx, hash)
if err != nil {
return fmt.Errorf("failed to fetch L1 block %s receipts: %w", hash, err)
}
return p.storeReceipts(receipts)
case l2.HintL2Code:
hash, chainID, err := p.parseHashAndChainID("L2 code", hintBytes)
if err != nil {
......
......@@ -141,6 +141,16 @@ func (s *RetryingL2Source) CodeByHash(ctx context.Context, hash common.Hash) ([]
})
}
func (s *RetryingL2Source) FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error) {
return retry.Do2(ctx, maxAttempts, s.strategy, func() (eth.BlockInfo, types.Receipts, error) {
i, r, err := s.source.FetchReceipts(ctx, blockHash)
if err != nil {
s.logger.Warn("Failed to fetch receipts", "hash", blockHash, "err", err)
}
return i, r, err
})
}
func (s *RetryingL2Source) OutputByRoot(ctx context.Context, blockRoot common.Hash) (eth.Output, error) {
return retry.Do(ctx, maxAttempts, s.strategy, func() (eth.Output, error) {
o, err := s.source.OutputByRoot(ctx, blockRoot)
......
......@@ -233,6 +233,9 @@ func TestRetryingL2Source(t *testing.T) {
txs := types.Transactions{
&types.Transaction{},
}
rcpts := types.Receipts{
&types.Receipt{},
}
data := []byte{1, 2, 3, 4, 5}
output := &eth.OutputV0{}
wrongOutput := &eth.OutputV0{BlockHash: common.Hash{0x99}}
......@@ -305,6 +308,17 @@ func TestRetryingL2Source(t *testing.T) {
require.Equal(t, data, actual)
})
t.Run("FetchReceipts Success", func(t *testing.T) {
source, mock := createL2Source(t)
defer mock.AssertExpectations(t)
mock.ExpectFetchReceipts(hash, info, rcpts, nil)
actualInfo, actualRcpts, err := source.FetchReceipts(ctx, hash)
require.NoError(t, err)
require.Equal(t, info, actualInfo)
require.Equal(t, rcpts, actualRcpts)
})
t.Run("OutputByRoot Success", func(t *testing.T) {
source, mock := createL2Source(t)
defer mock.AssertExpectations(t)
......@@ -388,6 +402,11 @@ func (m *MockL2Source) CodeByHash(ctx context.Context, hash common.Hash) ([]byte
return out[0].([]byte), *out[1].(*error)
}
func (m *MockL2Source) FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error) {
out := m.Mock.MethodCalled("FetchReceipts", blockHash)
return out[0].(eth.BlockInfo), out[1].(types.Receipts), *out[2].(*error)
}
func (m *MockL2Source) OutputByRoot(ctx context.Context, blockRoot common.Hash) (eth.Output, error) {
out := m.Mock.MethodCalled("OutputByRoot", blockRoot)
return out[0].(eth.Output), *out[1].(*error)
......@@ -410,6 +429,10 @@ func (m *MockL2Source) ExpectCodeByHash(hash common.Hash, code []byte, err error
m.Mock.On("CodeByHash", hash).Once().Return(code, &err)
}
func (m *MockL2Source) ExpectFetchReceipts(blockHash common.Hash, info eth.BlockInfo, rcpts types.Receipts, err error) {
m.Mock.On("FetchReceipts", blockHash).Once().Return(info, rcpts, &err)
}
func (m *MockL2Source) ExpectOutputByRoot(blockHash common.Hash, output eth.Output, err error) {
m.Mock.On("OutputByRoot", blockHash).Once().Return(output, &err)
}
......
......@@ -23,6 +23,7 @@ type L2Source interface {
InfoAndTxsByHash(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Transactions, error)
NodeByHash(ctx context.Context, hash common.Hash) ([]byte, error)
CodeByHash(ctx context.Context, hash common.Hash) ([]byte, error)
FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error)
OutputByRoot(ctx context.Context, blockRoot common.Hash) (eth.Output, error)
OutputByNumber(ctx context.Context, blockNumber uint64) (eth.Output, error)
RollupConfig() *rollup.Config
......
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