Commit 445a3d40 authored by Brian Bland's avatar Brian Bland Committed by GitHub

feat(sequencer): Origin Selector asynchronously prefetches the next origin from events (#12134)

* Sequencer: Origin Selector optimistically prefetches the next origin in background

* L1OriginSelector erases cached state on reset

* L1OriginSelector attempts to fetch on ForkchoiceUpdateEvent

* Move to a fully event-driven model, no extra goroutines

* Add missing test comment

* Minor cleanup, more tests

* Tune the context timeouts
parent 73038c88
...@@ -56,8 +56,9 @@ func NewL2Sequencer(t Testing, log log.Logger, l1 derive.L1Fetcher, blobSrc deri ...@@ -56,8 +56,9 @@ func NewL2Sequencer(t Testing, log log.Logger, l1 derive.L1Fetcher, blobSrc deri
ver := NewL2Verifier(t, log, l1, blobSrc, altDASrc, eng, cfg, &sync.Config{}, safedb.Disabled, interopBackend) ver := NewL2Verifier(t, log, l1, blobSrc, altDASrc, eng, cfg, &sync.Config{}, safedb.Disabled, interopBackend)
attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, eng) attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, eng)
seqConfDepthL1 := confdepth.NewConfDepth(seqConfDepth, ver.syncStatus.L1Head, l1) seqConfDepthL1 := confdepth.NewConfDepth(seqConfDepth, ver.syncStatus.L1Head, l1)
originSelector := sequencing.NewL1OriginSelector(t.Ctx(), log, cfg, seqConfDepthL1)
l1OriginSelector := &MockL1OriginSelector{ l1OriginSelector := &MockL1OriginSelector{
actual: sequencing.NewL1OriginSelector(log, cfg, seqConfDepthL1), actual: originSelector,
} }
metr := metrics.NoopMetrics metr := metrics.NoopMetrics
seqStateListener := node.DisabledConfigPersistence{} seqStateListener := node.DisabledConfigPersistence{}
...@@ -78,6 +79,7 @@ func NewL2Sequencer(t Testing, log log.Logger, l1 derive.L1Fetcher, blobSrc deri ...@@ -78,6 +79,7 @@ func NewL2Sequencer(t Testing, log log.Logger, l1 derive.L1Fetcher, blobSrc deri
}, },
} }
ver.eventSys.Register("sequencer", seq, opts) ver.eventSys.Register("sequencer", seq, opts)
ver.eventSys.Register("origin-selector", originSelector, opts)
require.NoError(t, seq.Init(t.Ctx(), true)) require.NoError(t, seq.Init(t.Ctx(), true))
return &L2Sequencer{ return &L2Sequencer{
L2Verifier: ver, L2Verifier: ver,
......
...@@ -245,7 +245,8 @@ func NewDriver( ...@@ -245,7 +245,8 @@ func NewDriver(
asyncGossiper := async.NewAsyncGossiper(driverCtx, network, log, metrics) asyncGossiper := async.NewAsyncGossiper(driverCtx, network, log, metrics)
attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, l2) attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, l2)
sequencerConfDepth := confdepth.NewConfDepth(driverCfg.SequencerConfDepth, statusTracker.L1Head, l1) sequencerConfDepth := confdepth.NewConfDepth(driverCfg.SequencerConfDepth, statusTracker.L1Head, l1)
findL1Origin := sequencing.NewL1OriginSelector(log, cfg, sequencerConfDepth) findL1Origin := sequencing.NewL1OriginSelector(driverCtx, log, cfg, sequencerConfDepth)
sys.Register("origin-selector", findL1Origin, opts)
sequencer = sequencing.NewSequencer(driverCtx, log, cfg, attrBuilder, findL1Origin, sequencer = sequencing.NewSequencer(driverCtx, log, cfg, attrBuilder, findL1Origin,
sequencerStateListener, sequencerConductor, asyncGossiper, metrics) sequencerStateListener, sequencerConductor, asyncGossiper, metrics)
sys.Register("sequencer", sequencer, opts) sys.Register("sequencer", sequencer, opts)
......
...@@ -4,6 +4,7 @@ import ( ...@@ -4,6 +4,7 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"sync"
"time" "time"
"github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum"
...@@ -11,6 +12,8 @@ import ( ...@@ -11,6 +12,8 @@ import (
"github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-node/rollup/derive"
"github.com/ethereum-optimism/optimism/op-node/rollup/engine"
"github.com/ethereum-optimism/optimism/op-node/rollup/event"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
) )
...@@ -20,15 +23,23 @@ type L1Blocks interface { ...@@ -20,15 +23,23 @@ type L1Blocks interface {
} }
type L1OriginSelector struct { type L1OriginSelector struct {
ctx context.Context
log log.Logger log log.Logger
cfg *rollup.Config cfg *rollup.Config
spec *rollup.ChainSpec spec *rollup.ChainSpec
l1 L1Blocks l1 L1Blocks
// Internal cache of L1 origins for faster access.
currentOrigin eth.L1BlockRef
nextOrigin eth.L1BlockRef
mu sync.Mutex
} }
func NewL1OriginSelector(log log.Logger, cfg *rollup.Config, l1 L1Blocks) *L1OriginSelector { func NewL1OriginSelector(ctx context.Context, log log.Logger, cfg *rollup.Config, l1 L1Blocks) *L1OriginSelector {
return &L1OriginSelector{ return &L1OriginSelector{
ctx: ctx,
log: log, log: log,
cfg: cfg, cfg: cfg,
spec: rollup.NewChainSpec(cfg), spec: rollup.NewChainSpec(cfg),
...@@ -36,62 +47,162 @@ func NewL1OriginSelector(log log.Logger, cfg *rollup.Config, l1 L1Blocks) *L1Ori ...@@ -36,62 +47,162 @@ func NewL1OriginSelector(log log.Logger, cfg *rollup.Config, l1 L1Blocks) *L1Ori
} }
} }
func (los *L1OriginSelector) OnEvent(ev event.Event) bool {
switch x := ev.(type) {
case engine.ForkchoiceUpdateEvent:
los.onForkchoiceUpdate(x.UnsafeL2Head)
case rollup.ResetEvent:
los.reset()
default:
return false
}
return true
}
// FindL1Origin determines what the next L1 Origin should be. // FindL1Origin determines what the next L1 Origin should be.
// The L1 Origin is either the L2 Head's Origin, or the following L1 block // The L1 Origin is either the L2 Head's Origin, or the following L1 block
// if the next L2 block's time is greater than or equal to the L2 Head's Origin. // if the next L2 block's time is greater than or equal to the L2 Head's Origin.
func (los *L1OriginSelector) FindL1Origin(ctx context.Context, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) { func (los *L1OriginSelector) FindL1Origin(ctx context.Context, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) {
// Grab a reference to the current L1 origin block. This call is by hash and thus easily cached. currentOrigin, nextOrigin, err := los.CurrentAndNextOrigin(ctx, l2Head)
currentOrigin, err := los.l1.L1BlockRefByHash(ctx, l2Head.L1Origin.Hash)
if err != nil { if err != nil {
return eth.L1BlockRef{}, err return eth.L1BlockRef{}, err
} }
// If the next L2 block time is greater than the next origin block's time, we can choose to
// start building on top of the next origin. Sequencer implementation has some leeway here and
// could decide to continue to build on top of the previous origin until the Sequencer runs out
// of slack. For simplicity, we implement our Sequencer to always start building on the latest
// L1 block when we can.
if nextOrigin != (eth.L1BlockRef{}) && l2Head.Time+los.cfg.BlockTime >= nextOrigin.Time {
return nextOrigin, nil
}
msd := los.spec.MaxSequencerDrift(currentOrigin.Time) msd := los.spec.MaxSequencerDrift(currentOrigin.Time)
log := los.log.New("current", currentOrigin, "current_time", currentOrigin.Time, log := los.log.New("current", currentOrigin, "current_time", currentOrigin.Time,
"l2_head", l2Head, "l2_head_time", l2Head.Time, "max_seq_drift", msd) "l2_head", l2Head, "l2_head_time", l2Head.Time, "max_seq_drift", msd)
seqDrift := l2Head.Time + los.cfg.BlockTime - currentOrigin.Time pastSeqDrift := l2Head.Time+los.cfg.BlockTime-currentOrigin.Time > msd
// If we are past the sequencer depth, we may want to advance the origin, but need to still // If we are not past the max sequencer drift, we can just return the current origin.
// check the time of the next origin. if !pastSeqDrift {
pastSeqDrift := seqDrift > msd return currentOrigin, nil
if pastSeqDrift {
log.Warn("Next L2 block time is past the sequencer drift + current origin time")
seqDrift = msd
} }
// Calculate the maximum time we can spend attempting to fetch the next L1 origin block. // Otherwise, we need to find the next L1 origin block in order to continue producing blocks.
// Time spent fetching this information is time not spent building the next L2 block, so log.Warn("Next L2 block time is past the sequencer drift + current origin time")
// we generally prioritize keeping this value small, allowing for a nonzero failure rate.
// As the next L2 block time approaches the max sequencer drift, increase our tolerance for
// slower L1 fetches in order to avoid falling too far behind.
fetchTimeout := time.Second + (9*time.Second*time.Duration(seqDrift))/time.Duration(msd)
fetchCtx, cancel := context.WithTimeout(ctx, fetchTimeout)
defer cancel()
// Attempt to find the next L1 origin block, where the next origin is the immediate child of if nextOrigin == (eth.L1BlockRef{}) {
// the current origin block. fetchCtx, cancel := context.WithTimeout(ctx, 10*time.Second)
// The L1 source can be shimmed to hide new L1 blocks and enforce a sequencer confirmation distance. defer cancel()
nextOrigin, err := los.l1.L1BlockRefByNumber(fetchCtx, currentOrigin.Number+1)
if err != nil { // If the next origin is not set, we need to fetch it now.
if pastSeqDrift { nextOrigin, err = los.fetch(fetchCtx, currentOrigin.Number+1)
if err != nil {
return eth.L1BlockRef{}, fmt.Errorf("cannot build next L2 block past current L1 origin %s by more than sequencer time drift, and failed to find next L1 origin: %w", currentOrigin, err) return eth.L1BlockRef{}, fmt.Errorf("cannot build next L2 block past current L1 origin %s by more than sequencer time drift, and failed to find next L1 origin: %w", currentOrigin, err)
} }
}
// If the next origin is ahead of the L2 head, we must return the current origin.
if l2Head.Time+los.cfg.BlockTime < nextOrigin.Time {
return currentOrigin, nil
}
return nextOrigin, nil
}
func (los *L1OriginSelector) CurrentAndNextOrigin(ctx context.Context, l2Head eth.L2BlockRef) (eth.L1BlockRef, eth.L1BlockRef, error) {
los.mu.Lock()
defer los.mu.Unlock()
if l2Head.L1Origin == los.currentOrigin.ID() {
// Most likely outcome: the L2 head is still on the current origin.
} else if l2Head.L1Origin == los.nextOrigin.ID() {
// If the L2 head has progressed to the next origin, update the current and next origins.
los.currentOrigin = los.nextOrigin
los.nextOrigin = eth.L1BlockRef{}
} else {
// If for some reason the L2 head is not on the current or next origin, we need to find the
// current origin block and reset the next origin.
// This is most likely to occur on the first block after a restart.
// Grab a reference to the current L1 origin block. This call is by hash and thus easily cached.
currentOrigin, err := los.l1.L1BlockRefByHash(ctx, l2Head.L1Origin.Hash)
if err != nil {
return eth.L1BlockRef{}, eth.L1BlockRef{}, err
}
los.currentOrigin = currentOrigin
los.nextOrigin = eth.L1BlockRef{}
}
return los.currentOrigin, los.nextOrigin, nil
}
func (los *L1OriginSelector) maybeSetNextOrigin(nextOrigin eth.L1BlockRef) {
los.mu.Lock()
defer los.mu.Unlock()
// Set the next origin if it is the immediate child of the current origin.
if nextOrigin.ParentHash == los.currentOrigin.Hash {
los.nextOrigin = nextOrigin
}
}
func (los *L1OriginSelector) onForkchoiceUpdate(unsafeL2Head eth.L2BlockRef) {
// Only allow a relatively small window for fetching the next origin, as this is performed
// on a best-effort basis.
ctx, cancel := context.WithTimeout(los.ctx, 500*time.Millisecond)
defer cancel()
currentOrigin, nextOrigin, err := los.CurrentAndNextOrigin(ctx, unsafeL2Head)
if err != nil {
log.Error("Failed to get current and next L1 origin on forkchoice update", "err", err)
return
}
los.tryFetchNextOrigin(ctx, currentOrigin, nextOrigin)
}
// tryFetchNextOrigin schedules a fetch for the next L1 origin block if it is not already set.
// This method always closes the channel, even if the next origin is already set.
func (los *L1OriginSelector) tryFetchNextOrigin(ctx context.Context, currentOrigin, nextOrigin eth.L1BlockRef) {
// If the next origin is already set, we don't need to do anything.
if nextOrigin != (eth.L1BlockRef{}) {
return
}
// If the current origin is not set, we can't schedule the next origin check.
if currentOrigin == (eth.L1BlockRef{}) {
return
}
if _, err := los.fetch(ctx, currentOrigin.Number+1); err != nil {
if errors.Is(err, ethereum.NotFound) { if errors.Is(err, ethereum.NotFound) {
log.Debug("No next L1 block found, repeating current origin") log.Debug("No next potential L1 origin found")
} else { } else {
log.Error("Failed to get next origin. Falling back to current origin", "err", err) log.Error("Failed to get next origin", "err", err)
} }
return currentOrigin, nil
} }
}
// If the next L2 block time is greater than the next origin block's time, we can choose to func (los *L1OriginSelector) fetch(ctx context.Context, number uint64) (eth.L1BlockRef, error) {
// start building on top of the next origin. Sequencer implementation has some leeway here and // Attempt to find the next L1 origin block, where the next origin is the immediate child of
// could decide to continue to build on top of the previous origin until the Sequencer runs out // the current origin block.
// of slack. For simplicity, we implement our Sequencer to always start building on the latest // The L1 source can be shimmed to hide new L1 blocks and enforce a sequencer confirmation distance.
// L1 block when we can. nextOrigin, err := los.l1.L1BlockRefByNumber(ctx, number)
if l2Head.Time+los.cfg.BlockTime >= nextOrigin.Time { if err != nil {
return nextOrigin, nil return eth.L1BlockRef{}, err
} }
return currentOrigin, nil los.maybeSetNextOrigin(nextOrigin)
return nextOrigin, nil
}
func (los *L1OriginSelector) reset() {
los.mu.Lock()
defer los.mu.Unlock()
los.currentOrigin = eth.L1BlockRef{}
los.nextOrigin = eth.L1BlockRef{}
} }
...@@ -2,10 +2,12 @@ package sequencing ...@@ -2,10 +2,12 @@ package sequencing
import ( import (
"context" "context"
"errors"
"testing" "testing"
"github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/confdepth" "github.com/ethereum-optimism/optimism/op-node/rollup/confdepth"
"github.com/ethereum-optimism/optimism/op-node/rollup/engine"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-service/testlog"
"github.com/ethereum-optimism/optimism/op-service/testutils" "github.com/ethereum-optimism/optimism/op-service/testutils"
...@@ -15,14 +17,186 @@ import ( ...@@ -15,14 +17,186 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
// TestOriginSelectorFetchCurrentError ensures that the origin selector
// returns an error when it cannot fetch the current origin and has no
// internal cached state.
func TestOriginSelectorFetchCurrentError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 500,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
Time: 25,
ParentHash: a.Hash,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 24,
}
l1.ExpectL1BlockRefByHash(a.Hash, eth.L1BlockRef{}, errors.New("test error"))
s := NewL1OriginSelector(ctx, log, cfg, l1)
_, err := s.FindL1Origin(ctx, l2Head)
require.ErrorContains(t, err, "test error")
// The same outcome occurs when the cached origin is different from that of the L2 head.
l1.ExpectL1BlockRefByHash(a.Hash, eth.L1BlockRef{}, errors.New("test error"))
s = NewL1OriginSelector(ctx, log, cfg, l1)
s.currentOrigin = b
_, err = s.FindL1Origin(ctx, l2Head)
require.ErrorContains(t, err, "test error")
}
// TestOriginSelectorFetchNextError ensures that the origin selector
// gracefully handles an error when fetching the next origin from the
// forkchoice update event.
func TestOriginSelectorFetchNextError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 500,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 24,
}
s := NewL1OriginSelector(ctx, log, cfg, l1)
s.currentOrigin = a
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
l1.ExpectL1BlockRefByNumber(b.Number, eth.L1BlockRef{}, ethereum.NotFound)
handled := s.OnEvent(engine.ForkchoiceUpdateEvent{UnsafeL2Head: l2Head})
require.True(t, handled)
l1.ExpectL1BlockRefByNumber(b.Number, eth.L1BlockRef{}, errors.New("test error"))
handled = s.OnEvent(engine.ForkchoiceUpdateEvent{UnsafeL2Head: l2Head})
require.True(t, handled)
// The next origin should still be `a` because the fetch failed.
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
}
// TestOriginSelectorAdvances ensures that the origin selector // TestOriginSelectorAdvances ensures that the origin selector
// advances the origin // advances the origin with the internal cache
// //
// There are 2 L1 blocks at time 20 & 25. The L2 Head is at time 24. // There are 3 L1 blocks at times 20, 22, 24. The L2 Head is at time 24.
// The next L2 time is 26 which is after the next L1 block time. There // The next L2 time is 26 which is after the next L1 block time. There
// is no conf depth to stop the origin selection so block `b` should // is no conf depth to stop the origin selection so block `b` should
// be the next L1 origin // be the next L1 origin, and then block `c` is the subsequent L1 origin.
func TestOriginSelectorAdvances(t *testing.T) { func TestOriginSelectorAdvances(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 500,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
Time: 22,
ParentHash: a.Hash,
}
c := eth.L1BlockRef{
Hash: common.Hash{'c'},
Number: 12,
Time: 24,
ParentHash: b.Hash,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 24,
}
s := NewL1OriginSelector(ctx, log, cfg, l1)
s.currentOrigin = a
s.nextOrigin = b
// Trigger the background fetch via a forkchoice update.
// This should be a no-op because the next origin is already cached.
handled := s.OnEvent(engine.ForkchoiceUpdateEvent{UnsafeL2Head: l2Head})
require.True(t, handled)
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, b, next)
l2Head = eth.L2BlockRef{
L1Origin: b.ID(),
Time: 26,
}
// The origin is still `b` because the next origin has not been fetched yet.
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, b, next)
l1.ExpectL1BlockRefByNumber(c.Number, c, nil)
// Trigger the background fetch via a forkchoice update.
// This will actually fetch the next origin because the internal cache is empty.
handled = s.OnEvent(engine.ForkchoiceUpdateEvent{UnsafeL2Head: l2Head})
require.True(t, handled)
// The next origin should be `c` now.
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, c, next)
}
// TestOriginSelectorHandlesReset ensures that the origin selector
// resets its internal cached state on derivation pipeline resets.
func TestOriginSelectorHandlesReset(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 500, MaxSequencerDrift: 500,
...@@ -46,11 +220,81 @@ func TestOriginSelectorAdvances(t *testing.T) { ...@@ -46,11 +220,81 @@ func TestOriginSelectorAdvances(t *testing.T) {
Time: 24, Time: 24,
} }
s := NewL1OriginSelector(ctx, log, cfg, l1)
s.currentOrigin = a
s.nextOrigin = b
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, b, next)
// Trigger the pipeline reset
handled := s.OnEvent(rollup.ResetEvent{})
require.True(t, handled)
// The next origin should be `a` now, but we need to fetch it
// because the internal cache was reset.
l1.ExpectL1BlockRefByHash(a.Hash, a, nil) l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
}
// TestOriginSelectorFetchesNextOrigin ensures that the origin selector
// fetches the next origin when a fcu is received and the internal cache is empty
//
// The next L2 time is 26 which is after the next L1 block time. There
// is no conf depth to stop the origin selection so block `b` will
// be the next L1 origin as soon as it is fetched.
func TestOriginSelectorFetchesNextOrigin(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 500,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
Time: 25,
ParentHash: a.Hash,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 24,
}
// This is called as part of the background prefetch job
l1.ExpectL1BlockRefByNumber(b.Number, b, nil) l1.ExpectL1BlockRefByNumber(b.Number, b, nil)
s := NewL1OriginSelector(log, cfg, l1) s := NewL1OriginSelector(ctx, log, cfg, l1)
next, err := s.FindL1Origin(context.Background(), l2Head) s.currentOrigin = a
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
// Selection is stable until the next origin is fetched
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
// Trigger the background fetch via a forkchoice update
handled := s.OnEvent(engine.ForkchoiceUpdateEvent{UnsafeL2Head: l2Head})
require.True(t, handled)
// The next origin should be `b` now.
next, err = s.FindL1Origin(ctx, l2Head)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, b, next) require.Equal(t, b, next)
} }
...@@ -64,6 +308,9 @@ func TestOriginSelectorAdvances(t *testing.T) { ...@@ -64,6 +308,9 @@ func TestOriginSelectorAdvances(t *testing.T) {
// but it should select block `a` because the L2 block time must be ahead // but it should select block `a` because the L2 block time must be ahead
// of the the timestamp of it's L1 origin. // of the the timestamp of it's L1 origin.
func TestOriginSelectorRespectsOriginTiming(t *testing.T) { func TestOriginSelectorRespectsOriginTiming(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 500, MaxSequencerDrift: 500,
...@@ -87,15 +334,61 @@ func TestOriginSelectorRespectsOriginTiming(t *testing.T) { ...@@ -87,15 +334,61 @@ func TestOriginSelectorRespectsOriginTiming(t *testing.T) {
Time: 22, Time: 22,
} }
l1.ExpectL1BlockRefByHash(a.Hash, a, nil) s := NewL1OriginSelector(ctx, log, cfg, l1)
l1.ExpectL1BlockRefByNumber(b.Number, b, nil) s.currentOrigin = a
s.nextOrigin = b
s := NewL1OriginSelector(log, cfg, l1) next, err := s.FindL1Origin(ctx, l2Head)
next, err := s.FindL1Origin(context.Background(), l2Head)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, a, next) require.Equal(t, a, next)
} }
// TestOriginSelectorRespectsSeqDrift
//
// There are 2 L1 blocks at time 20 & 25. The L2 Head is at time 27.
// The next L2 time is 29. The sequencer drift is 8 so the L2 head is
// valid with origin `a`, but the next L2 block is not valid with origin `b.`
// This is because 29 (next L2 time) > 20 (origin) + 8 (seq drift) => invalid block.
// The origin selector does not yet know about block `b` so it should wait for the
// background fetch to complete synchronously.
func TestOriginSelectorRespectsSeqDrift(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 8,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
Time: 25,
ParentHash: a.Hash,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 27,
}
l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
l1.ExpectL1BlockRefByNumber(b.Number, b, nil)
s := NewL1OriginSelector(ctx, log, cfg, l1)
next, err := s.FindL1Origin(ctx, l2Head)
require.NoError(t, err)
require.Equal(t, b, next)
}
// TestOriginSelectorRespectsConfDepth ensures that the origin selector // TestOriginSelectorRespectsConfDepth ensures that the origin selector
// will respect the confirmation depth requirement // will respect the confirmation depth requirement
// //
...@@ -104,6 +397,9 @@ func TestOriginSelectorRespectsOriginTiming(t *testing.T) { ...@@ -104,6 +397,9 @@ func TestOriginSelectorRespectsOriginTiming(t *testing.T) {
// as the origin, however block `b` is the L1 Head & the sequencer // as the origin, however block `b` is the L1 Head & the sequencer
// needs to wait until that block is confirmed enough before advancing. // needs to wait until that block is confirmed enough before advancing.
func TestOriginSelectorRespectsConfDepth(t *testing.T) { func TestOriginSelectorRespectsConfDepth(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 500, MaxSequencerDrift: 500,
...@@ -127,11 +423,11 @@ func TestOriginSelectorRespectsConfDepth(t *testing.T) { ...@@ -127,11 +423,11 @@ func TestOriginSelectorRespectsConfDepth(t *testing.T) {
Time: 27, Time: 27,
} }
l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
confDepthL1 := confdepth.NewConfDepth(10, func() eth.L1BlockRef { return b }, l1) confDepthL1 := confdepth.NewConfDepth(10, func() eth.L1BlockRef { return b }, l1)
s := NewL1OriginSelector(log, cfg, confDepthL1) s := NewL1OriginSelector(ctx, log, cfg, confDepthL1)
s.currentOrigin = a
next, err := s.FindL1Origin(context.Background(), l2Head) next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, a, next) require.Equal(t, a, next)
} }
...@@ -147,6 +443,9 @@ func TestOriginSelectorRespectsConfDepth(t *testing.T) { ...@@ -147,6 +443,9 @@ func TestOriginSelectorRespectsConfDepth(t *testing.T) {
// This is because 29 (next L2 time) > 20 (origin) + 8 (seq drift) => invalid block. // This is because 29 (next L2 time) > 20 (origin) + 8 (seq drift) => invalid block.
// We maintain confirmation distance, even though we would shift to the next origin if we could. // We maintain confirmation distance, even though we would shift to the next origin if we could.
func TestOriginSelectorStrictConfDepth(t *testing.T) { func TestOriginSelectorStrictConfDepth(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 8, MaxSequencerDrift: 8,
...@@ -172,9 +471,9 @@ func TestOriginSelectorStrictConfDepth(t *testing.T) { ...@@ -172,9 +471,9 @@ func TestOriginSelectorStrictConfDepth(t *testing.T) {
l1.ExpectL1BlockRefByHash(a.Hash, a, nil) l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
confDepthL1 := confdepth.NewConfDepth(10, func() eth.L1BlockRef { return b }, l1) confDepthL1 := confdepth.NewConfDepth(10, func() eth.L1BlockRef { return b }, l1)
s := NewL1OriginSelector(log, cfg, confDepthL1) s := NewL1OriginSelector(ctx, log, cfg, confDepthL1)
_, err := s.FindL1Origin(context.Background(), l2Head) _, err := s.FindL1Origin(ctx, l2Head)
require.ErrorContains(t, err, "sequencer time drift") require.ErrorContains(t, err, "sequencer time drift")
} }
...@@ -187,6 +486,9 @@ func u64ptr(n uint64) *uint64 { ...@@ -187,6 +486,9 @@ func u64ptr(n uint64) *uint64 {
// This time the same L1 origin is returned if no new L1 head is seen, instead of an error, // This time the same L1 origin is returned if no new L1 head is seen, instead of an error,
// because the Fjord max sequencer drift is higher. // because the Fjord max sequencer drift is higher.
func TestOriginSelector_FjordSeqDrift(t *testing.T) { func TestOriginSelector_FjordSeqDrift(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 8, MaxSequencerDrift: 8,
...@@ -205,13 +507,12 @@ func TestOriginSelector_FjordSeqDrift(t *testing.T) { ...@@ -205,13 +507,12 @@ func TestOriginSelector_FjordSeqDrift(t *testing.T) {
Time: 27, // next L2 block time would be past pre-Fjord seq drift Time: 27, // next L2 block time would be past pre-Fjord seq drift
} }
l1.ExpectL1BlockRefByHash(a.Hash, a, nil) s := NewL1OriginSelector(ctx, log, cfg, l1)
l1.ExpectL1BlockRefByNumber(a.Number+1, eth.L1BlockRef{}, ethereum.NotFound) s.currentOrigin = a
s := NewL1OriginSelector(log, cfg, l1)
l1O, err := s.FindL1Origin(context.Background(), l2Head) next, err := s.FindL1Origin(ctx, l2Head)
require.NoError(t, err, "with Fjord activated, have increased max seq drift") require.NoError(t, err, "with Fjord activated, have increased max seq drift")
require.Equal(t, a, l1O) require.Equal(t, a, next)
} }
// TestOriginSelectorSeqDriftRespectsNextOriginTime // TestOriginSelectorSeqDriftRespectsNextOriginTime
...@@ -221,6 +522,53 @@ func TestOriginSelector_FjordSeqDrift(t *testing.T) { ...@@ -221,6 +522,53 @@ func TestOriginSelector_FjordSeqDrift(t *testing.T) {
// drift, the origin should remain on block `a` because the next origin's // drift, the origin should remain on block `a` because the next origin's
// time is greater than the next L2 time. // time is greater than the next L2 time.
func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) { func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 8,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
a := eth.L1BlockRef{
Hash: common.Hash{'a'},
Number: 10,
Time: 20,
}
b := eth.L1BlockRef{
Hash: common.Hash{'b'},
Number: 11,
Time: 100,
ParentHash: a.Hash,
}
l2Head := eth.L2BlockRef{
L1Origin: a.ID(),
Time: 27,
}
s := NewL1OriginSelector(ctx, log, cfg, l1)
s.currentOrigin = a
s.nextOrigin = b
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err)
require.Equal(t, a, next)
}
// TestOriginSelectorSeqDriftRespectsNextOriginTimeNoCache
//
// There are 2 L1 blocks at time 20 & 100. The L2 Head is at time 27.
// The next L2 time is 29. Even though the next L2 time is past the seq
// drift, the origin should remain on block `a` because the next origin's
// time is greater than the next L2 time.
// The L1OriginSelector does not have the next origin cached, and must fetch it
// because the max sequencer drift has been exceeded.
func TestOriginSelectorSeqDriftRespectsNextOriginTimeNoCache(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 8, MaxSequencerDrift: 8,
...@@ -244,11 +592,12 @@ func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) { ...@@ -244,11 +592,12 @@ func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) {
Time: 27, Time: 27,
} }
l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
l1.ExpectL1BlockRefByNumber(b.Number, b, nil) l1.ExpectL1BlockRefByNumber(b.Number, b, nil)
s := NewL1OriginSelector(log, cfg, l1) s := NewL1OriginSelector(ctx, log, cfg, l1)
next, err := s.FindL1Origin(context.Background(), l2Head) s.currentOrigin = a
next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, a, next) require.Equal(t, a, next)
} }
...@@ -263,6 +612,9 @@ func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) { ...@@ -263,6 +612,9 @@ func TestOriginSelectorSeqDriftRespectsNextOriginTime(t *testing.T) {
// Due to a conf depth of 2, block `b` is not immediately visible, // Due to a conf depth of 2, block `b` is not immediately visible,
// and the origin selection should fail until it is visible, by waiting for block `c`. // and the origin selection should fail until it is visible, by waiting for block `c`.
func TestOriginSelectorHandlesLateL1Blocks(t *testing.T) { func TestOriginSelectorHandlesLateL1Blocks(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit) log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{ cfg := &rollup.Config{
MaxSequencerDrift: 8, MaxSequencerDrift: 8,
...@@ -300,23 +652,43 @@ func TestOriginSelectorHandlesLateL1Blocks(t *testing.T) { ...@@ -300,23 +652,43 @@ func TestOriginSelectorHandlesLateL1Blocks(t *testing.T) {
// l2 head does not change, so we start at the same origin again and again until we meet the conf depth // l2 head does not change, so we start at the same origin again and again until we meet the conf depth
l1.ExpectL1BlockRefByHash(a.Hash, a, nil) l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
l1.ExpectL1BlockRefByHash(a.Hash, a, nil)
l1.ExpectL1BlockRefByNumber(b.Number, b, nil) l1.ExpectL1BlockRefByNumber(b.Number, b, nil)
l1Head := b l1Head := b
confDepthL1 := confdepth.NewConfDepth(2, func() eth.L1BlockRef { return l1Head }, l1) confDepthL1 := confdepth.NewConfDepth(2, func() eth.L1BlockRef { return l1Head }, l1)
s := NewL1OriginSelector(log, cfg, confDepthL1) s := NewL1OriginSelector(ctx, log, cfg, confDepthL1)
_, err := s.FindL1Origin(context.Background(), l2Head) _, err := s.FindL1Origin(ctx, l2Head)
require.ErrorContains(t, err, "sequencer time drift") require.ErrorContains(t, err, "sequencer time drift")
l1Head = c l1Head = c
_, err = s.FindL1Origin(context.Background(), l2Head) _, err = s.FindL1Origin(ctx, l2Head)
require.ErrorContains(t, err, "sequencer time drift") require.ErrorContains(t, err, "sequencer time drift")
l1Head = d l1Head = d
next, err := s.FindL1Origin(context.Background(), l2Head) next, err := s.FindL1Origin(ctx, l2Head)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, a, next, "must stay on a because the L1 time may not be higher than the L2 time") require.Equal(t, a, next, "must stay on a because the L1 time may not be higher than the L2 time")
} }
// TestOriginSelectorMiscEvent ensures that the origin selector ignores miscellaneous events,
// but instead returns false to indicate that the event was not handled.
func TestOriginSelectorMiscEvent(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log := testlog.Logger(t, log.LevelCrit)
cfg := &rollup.Config{
MaxSequencerDrift: 8,
BlockTime: 2,
}
l1 := &testutils.MockL1Source{}
defer l1.AssertExpectations(t)
s := NewL1OriginSelector(ctx, log, cfg, l1)
// This event is not handled
handled := s.OnEvent(rollup.L1TemporaryErrorEvent{})
require.False(t, handled)
}
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