Commit 31653e5e authored by protolambda's avatar protolambda Committed by GitHub

op-node: change finalizer to use events (#10972)

parent e449dd28
...@@ -17,6 +17,7 @@ import ( ...@@ -17,6 +17,7 @@ import (
batcherFlags "github.com/ethereum-optimism/optimism/op-batcher/flags" batcherFlags "github.com/ethereum-optimism/optimism/op-batcher/flags"
"github.com/ethereum-optimism/optimism/op-e2e/e2eutils" "github.com/ethereum-optimism/optimism/op-e2e/e2eutils"
"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/finality"
"github.com/ethereum-optimism/optimism/op-node/rollup/sync" "github.com/ethereum-optimism/optimism/op-node/rollup/sync"
"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"
...@@ -246,7 +247,7 @@ func L2Finalization(gt *testing.T, deltaTimeOffset *hexutil.Uint64) { ...@@ -246,7 +247,7 @@ func L2Finalization(gt *testing.T, deltaTimeOffset *hexutil.Uint64) {
// If we get this false signal, we shouldn't finalize the L2 chain. // If we get this false signal, we shouldn't finalize the L2 chain.
altBlock4 := sequencer.SyncStatus().SafeL1 altBlock4 := sequencer.SyncStatus().SafeL1
altBlock4.Hash = common.HexToHash("0xdead") altBlock4.Hash = common.HexToHash("0xdead")
sequencer.finalizer.Finalize(t.Ctx(), altBlock4) sequencer.synchronousEvents.Emit(finality.FinalizeL1Event{FinalizedL1: altBlock4})
sequencer.ActL2PipelineFull(t) sequencer.ActL2PipelineFull(t)
require.Equal(t, uint64(3), sequencer.SyncStatus().FinalizedL1.Number) require.Equal(t, uint64(3), sequencer.SyncStatus().FinalizedL1.Number)
require.Equal(t, heightToSubmit, sequencer.SyncStatus().FinalizedL2.Number, "unknown/bad finalized L1 blocks are ignored") require.Equal(t, heightToSubmit, sequencer.SyncStatus().FinalizedL2.Number, "unknown/bad finalized L1 blocks are ignored")
......
...@@ -96,6 +96,7 @@ func RunProposerTest(gt *testing.T, deltaTimeOffset *hexutil.Uint64) { ...@@ -96,6 +96,7 @@ func RunProposerTest(gt *testing.T, deltaTimeOffset *hexutil.Uint64) {
sequencer.ActL2PipelineFull(t) sequencer.ActL2PipelineFull(t)
sequencer.ActL1SafeSignal(t) sequencer.ActL1SafeSignal(t)
sequencer.ActL1FinalizedSignal(t) sequencer.ActL1FinalizedSignal(t)
sequencer.ActL2PipelineFull(t)
require.Equal(t, sequencer.SyncStatus().UnsafeL2, sequencer.SyncStatus().FinalizedL2) require.Equal(t, sequencer.SyncStatus().UnsafeL2, sequencer.SyncStatus().FinalizedL2)
require.True(t, proposer.CanPropose(t)) require.True(t, proposer.CanPropose(t))
......
...@@ -98,9 +98,9 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri ...@@ -98,9 +98,9 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri
var finalizer driver.Finalizer var finalizer driver.Finalizer
if cfg.PlasmaEnabled() { if cfg.PlasmaEnabled() {
finalizer = finality.NewPlasmaFinalizer(log, cfg, l1, ec, plasmaSrc) finalizer = finality.NewPlasmaFinalizer(ctx, log, cfg, l1, synchronousEvents, plasmaSrc)
} else { } else {
finalizer = finality.NewFinalizer(log, cfg, l1, ec) finalizer = finality.NewFinalizer(ctx, log, cfg, l1, synchronousEvents)
} }
attributesHandler := attributes.NewAttributesHandler(log, cfg, ctx, eng, synchronousEvents) attributesHandler := attributes.NewAttributesHandler(log, cfg, ctx, eng, synchronousEvents)
...@@ -153,6 +153,7 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri ...@@ -153,6 +153,7 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri
clSync, clSync,
pipelineDeriver, pipelineDeriver,
attributesHandler, attributesHandler,
finalizer,
} }
t.Cleanup(rollupNode.rpc.Stop) t.Cleanup(rollupNode.rpc.Stop)
...@@ -288,13 +289,15 @@ func (s *L2Verifier) ActL1FinalizedSignal(t Testing) { ...@@ -288,13 +289,15 @@ func (s *L2Verifier) ActL1FinalizedSignal(t Testing) {
finalized, err := s.l1.L1BlockRefByLabel(t.Ctx(), eth.Finalized) finalized, err := s.l1.L1BlockRefByLabel(t.Ctx(), eth.Finalized)
require.NoError(t, err) require.NoError(t, err)
s.l1State.HandleNewL1FinalizedBlock(finalized) s.l1State.HandleNewL1FinalizedBlock(finalized)
s.finalizer.Finalize(t.Ctx(), finalized) s.synchronousEvents.Emit(finality.FinalizeL1Event{FinalizedL1: finalized})
} }
func (s *L2Verifier) OnEvent(ev rollup.Event) { func (s *L2Verifier) OnEvent(ev rollup.Event) {
switch x := ev.(type) { switch x := ev.(type) {
case rollup.L1TemporaryErrorEvent:
s.log.Warn("L1 temporary error", "err", x.Err)
case rollup.EngineTemporaryErrorEvent: case rollup.EngineTemporaryErrorEvent:
s.log.Warn("Derivation process temporary error", "err", x.Err) s.log.Warn("Engine temporary error", "err", x.Err)
if errors.Is(x.Err, sync.WrongChainErr) { // action-tests don't back off on temporary errors. Avoid a bad genesis setup from looping. if errors.Is(x.Err, sync.WrongChainErr) { // action-tests don't back off on temporary errors. Avoid a bad genesis setup from looping.
panic(fmt.Errorf("genesis setup issue: %w", x.Err)) panic(fmt.Errorf("genesis setup issue: %w", x.Err))
} }
......
...@@ -147,8 +147,9 @@ func (eq *AttributesHandler) consolidateNextSafeAttributes(attributes *derive.At ...@@ -147,8 +147,9 @@ func (eq *AttributesHandler) consolidateNextSafeAttributes(attributes *derive.At
return return
} }
eq.emitter.Emit(engine.PromotePendingSafeEvent{ eq.emitter.Emit(engine.PromotePendingSafeEvent{
Ref: ref, Ref: ref,
Safe: attributes.IsLastInSpan, Safe: attributes.IsLastInSpan,
DerivedFrom: attributes.DerivedFrom,
}) })
} }
......
...@@ -25,6 +25,13 @@ func TestAttributesHandler(t *testing.T) { ...@@ -25,6 +25,13 @@ func TestAttributesHandler(t *testing.T) {
rng := rand.New(rand.NewSource(1234)) rng := rand.New(rand.NewSource(1234))
refA := testutils.RandomBlockRef(rng) refA := testutils.RandomBlockRef(rng)
refB := eth.L1BlockRef{
Hash: testutils.RandomHash(rng),
Number: refA.Number + 1,
ParentHash: refA.Hash,
Time: refA.Time + 12,
}
aL1Info := &testutils.MockBlockInfo{ aL1Info := &testutils.MockBlockInfo{
InfoParentHash: refA.ParentHash, InfoParentHash: refA.ParentHash,
InfoNum: refA.Number, InfoNum: refA.Number,
...@@ -263,6 +270,7 @@ func TestAttributesHandler(t *testing.T) { ...@@ -263,6 +270,7 @@ func TestAttributesHandler(t *testing.T) {
Attributes: attrA1.Attributes, // attributes will match, passing consolidation Attributes: attrA1.Attributes, // attributes will match, passing consolidation
Parent: attrA1.Parent, Parent: attrA1.Parent,
IsLastInSpan: lastInSpan, IsLastInSpan: lastInSpan,
DerivedFrom: refB,
} }
emitter.ExpectOnce(derive.ConfirmReceivedAttributesEvent{}) emitter.ExpectOnce(derive.ConfirmReceivedAttributesEvent{})
emitter.ExpectOnce(engine.PendingSafeRequestEvent{}) emitter.ExpectOnce(engine.PendingSafeRequestEvent{})
...@@ -274,8 +282,9 @@ func TestAttributesHandler(t *testing.T) { ...@@ -274,8 +282,9 @@ func TestAttributesHandler(t *testing.T) {
l2.ExpectPayloadByNumber(refA1.Number, payloadA1, nil) l2.ExpectPayloadByNumber(refA1.Number, payloadA1, nil)
emitter.ExpectOnce(engine.PromotePendingSafeEvent{ emitter.ExpectOnce(engine.PromotePendingSafeEvent{
Ref: refA1, Ref: refA1,
Safe: lastInSpan, // last in span becomes safe instantaneously Safe: lastInSpan, // last in span becomes safe instantaneously
DerivedFrom: refB,
}) })
ah.OnEvent(engine.PendingSafeUpdateEvent{ ah.OnEvent(engine.PendingSafeUpdateEvent{
PendingSafe: refA0, PendingSafe: refA0,
......
...@@ -9,7 +9,9 @@ import ( ...@@ -9,7 +9,9 @@ import (
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
) )
type DeriverIdleEvent struct{} type DeriverIdleEvent struct {
Origin eth.L1BlockRef
}
func (d DeriverIdleEvent) String() string { func (d DeriverIdleEvent) String() string {
return "derivation-idle" return "derivation-idle"
...@@ -84,10 +86,10 @@ func (d *PipelineDeriver) OnEvent(ev rollup.Event) { ...@@ -84,10 +86,10 @@ func (d *PipelineDeriver) OnEvent(ev rollup.Event) {
attrib, err := d.pipeline.Step(d.ctx, x.PendingSafe) attrib, err := d.pipeline.Step(d.ctx, x.PendingSafe)
if err == io.EOF { if err == io.EOF {
d.pipeline.log.Debug("Derivation process went idle", "progress", d.pipeline.Origin(), "err", err) d.pipeline.log.Debug("Derivation process went idle", "progress", d.pipeline.Origin(), "err", err)
d.emitter.Emit(DeriverIdleEvent{}) d.emitter.Emit(DeriverIdleEvent{Origin: d.pipeline.Origin()})
} else if err != nil && errors.Is(err, EngineELSyncing) { } else if err != nil && errors.Is(err, EngineELSyncing) {
d.pipeline.log.Debug("Derivation process went idle because the engine is syncing", "progress", d.pipeline.Origin(), "err", err) d.pipeline.log.Debug("Derivation process went idle because the engine is syncing", "progress", d.pipeline.Origin(), "err", err)
d.emitter.Emit(DeriverIdleEvent{}) d.emitter.Emit(DeriverIdleEvent{Origin: d.pipeline.Origin()})
} else if err != nil && errors.Is(err, ErrReset) { } else if err != nil && errors.Is(err, ErrReset) {
d.emitter.Emit(rollup.ResetEvent{Err: err}) d.emitter.Emit(rollup.ResetEvent{Err: err})
} else if err != nil && errors.Is(err, ErrTemporary) { } else if err != nil && errors.Is(err, ErrTemporary) {
......
...@@ -91,9 +91,8 @@ type AttributesHandler interface { ...@@ -91,9 +91,8 @@ type AttributesHandler interface {
} }
type Finalizer interface { type Finalizer interface {
Finalize(ctx context.Context, ref eth.L1BlockRef)
FinalizedL1() eth.L1BlockRef FinalizedL1() eth.L1BlockRef
engine.FinalizerHooks rollup.Deriver
} }
type PlasmaIface interface { type PlasmaIface interface {
...@@ -186,9 +185,9 @@ func NewDriver( ...@@ -186,9 +185,9 @@ func NewDriver(
var finalizer Finalizer var finalizer Finalizer
if cfg.PlasmaEnabled() { if cfg.PlasmaEnabled() {
finalizer = finality.NewPlasmaFinalizer(log, cfg, l1, ec, plasma) finalizer = finality.NewPlasmaFinalizer(driverCtx, log, cfg, l1, synchronousEvents, plasma)
} else { } else {
finalizer = finality.NewFinalizer(log, cfg, l1, ec) finalizer = finality.NewFinalizer(driverCtx, log, cfg, l1, synchronousEvents)
} }
attributesHandler := attributes.NewAttributesHandler(log, cfg, driverCtx, l2, synchronousEvents) attributesHandler := attributes.NewAttributesHandler(log, cfg, driverCtx, l2, synchronousEvents)
...@@ -254,6 +253,7 @@ func NewDriver( ...@@ -254,6 +253,7 @@ func NewDriver(
clSync, clSync,
pipelineDeriver, pipelineDeriver,
attributesHandler, attributesHandler,
finalizer,
} }
return driver return driver
......
...@@ -18,6 +18,7 @@ import ( ...@@ -18,6 +18,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/rollup/conductor" "github.com/ethereum-optimism/optimism/op-node/rollup/conductor"
"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/engine"
"github.com/ethereum-optimism/optimism/op-node/rollup/finality"
"github.com/ethereum-optimism/optimism/op-node/rollup/sync" "github.com/ethereum-optimism/optimism/op-node/rollup/sync"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
) )
...@@ -312,9 +313,7 @@ func (s *Driver) eventLoop() { ...@@ -312,9 +313,7 @@ func (s *Driver) eventLoop() {
// no step, justified L1 information does not do anything for L2 derivation or status // no step, justified L1 information does not do anything for L2 derivation or status
case newL1Finalized := <-s.l1FinalizedSig: case newL1Finalized := <-s.l1FinalizedSig:
s.l1State.HandleNewL1FinalizedBlock(newL1Finalized) s.l1State.HandleNewL1FinalizedBlock(newL1Finalized)
ctx, cancel := context.WithTimeout(s.driverCtx, time.Second*5) s.Emit(finality.FinalizeL1Event{FinalizedL1: newL1Finalized})
s.Finalizer.Finalize(ctx, newL1Finalized)
cancel()
reqStep() // we may be able to mark more L2 data as finalized now reqStep() // we may be able to mark more L2 data as finalized now
case <-s.sched.NextDelayedStep(): case <-s.sched.NextDelayedStep():
s.Emit(StepAttemptEvent{}) s.Emit(StepAttemptEvent{})
...@@ -396,8 +395,7 @@ type SyncDeriver struct { ...@@ -396,8 +395,7 @@ type SyncDeriver struct {
Finalizer Finalizer Finalizer Finalizer
SafeHeadNotifs rollup.SafeHeadListener // notified when safe head is updated SafeHeadNotifs rollup.SafeHeadListener // notified when safe head is updated
lastNotifiedSafeHead eth.L2BlockRef
CLSync CLSync CLSync CLSync
...@@ -428,8 +426,11 @@ func (s *SyncDeriver) OnEvent(ev rollup.Event) { ...@@ -428,8 +426,11 @@ func (s *SyncDeriver) OnEvent(ev rollup.Event) {
s.onStepEvent() s.onStepEvent()
case rollup.ResetEvent: case rollup.ResetEvent:
s.onResetEvent(x) s.onResetEvent(x)
case rollup.L1TemporaryErrorEvent:
s.Log.Warn("L1 temporary error", "err", x.Err)
s.Emitter.Emit(StepReqEvent{})
case rollup.EngineTemporaryErrorEvent: case rollup.EngineTemporaryErrorEvent:
s.Log.Warn("Derivation process temporary error", "err", x.Err) s.Log.Warn("Engine temporary error", "err", x.Err)
// Make sure that for any temporarily failed attributes we retry processing. // Make sure that for any temporarily failed attributes we retry processing.
s.Emitter.Emit(engine.PendingSafeRequestEvent{}) s.Emitter.Emit(engine.PendingSafeRequestEvent{})
...@@ -445,6 +446,19 @@ func (s *SyncDeriver) OnEvent(ev rollup.Event) { ...@@ -445,6 +446,19 @@ func (s *SyncDeriver) OnEvent(ev rollup.Event) {
// If there is more data to process, // If there is more data to process,
// continue derivation quickly // continue derivation quickly
s.Emitter.Emit(StepReqEvent{ResetBackoff: true}) s.Emitter.Emit(StepReqEvent{ResetBackoff: true})
case engine.SafeDerivedEvent:
s.onSafeDerivedBlock(x)
}
}
func (s *SyncDeriver) onSafeDerivedBlock(x engine.SafeDerivedEvent) {
if s.SafeHeadNotifs != nil && s.SafeHeadNotifs.Enabled() {
if err := s.SafeHeadNotifs.SafeHeadUpdated(x.Safe, x.DerivedFrom.ID()); err != nil {
// At this point our state is in a potentially inconsistent state as we've updated the safe head
// in the execution client but failed to post process it. Reset the pipeline so the safe head rolls back
// a little (it always rolls back at least 1 block) and then it will retry storing the entry
s.Emitter.Emit(rollup.ResetEvent{Err: fmt.Errorf("safe head notifications failed: %w", err)})
}
} }
} }
...@@ -457,7 +471,7 @@ func (s *SyncDeriver) onEngineConfirmedReset(x engine.EngineResetConfirmedEvent) ...@@ -457,7 +471,7 @@ func (s *SyncDeriver) onEngineConfirmedReset(x engine.EngineResetConfirmedEvent)
s.Log.Error("Failed to warn safe-head notifier of safe-head reset", "safe", x.Safe) s.Log.Error("Failed to warn safe-head notifier of safe-head reset", "safe", x.Safe)
return return
} }
if s.SafeHeadNotifs.Enabled() && x.Safe.Number == s.Config.Genesis.L2.Number && x.Safe.Hash == s.Config.Genesis.L2.Hash { if s.SafeHeadNotifs.Enabled() && x.Safe.ID() == s.Config.Genesis.L2 {
// The rollup genesis block is always safe by definition. So if the pipeline resets this far back we know // The rollup genesis block is always safe by definition. So if the pipeline resets this far back we know
// we will process all safe head updates and can record genesis as always safe from L1 genesis. // we will process all safe head updates and can record genesis as always safe from L1 genesis.
// Note that it is not safe to use cfg.Genesis.L1 here as it is the block immediately before the L2 genesis // Note that it is not safe to use cfg.Genesis.L1 here as it is the block immediately before the L2 genesis
...@@ -483,7 +497,7 @@ func (s *SyncDeriver) onStepEvent() { ...@@ -483,7 +497,7 @@ func (s *SyncDeriver) onStepEvent() {
// where some things are triggered through events, and some through this synchronous step function. // where some things are triggered through events, and some through this synchronous step function.
// We just translate the results into their equivalent events, // We just translate the results into their equivalent events,
// to merge the error-handling with that of the new event-based system. // to merge the error-handling with that of the new event-based system.
err := s.SyncStep(s.Ctx) err := s.SyncStep()
if err != nil && errors.Is(err, derive.EngineELSyncing) { if err != nil && errors.Is(err, derive.EngineELSyncing) {
s.Log.Debug("Derivation process went idle because the engine is syncing", "unsafe_head", s.Engine.UnsafeL2Head(), "err", err) s.Log.Debug("Derivation process went idle because the engine is syncing", "unsafe_head", s.Engine.UnsafeL2Head(), "err", err)
s.Emitter.Emit(ResetStepBackoffEvent{}) s.Emitter.Emit(ResetStepBackoffEvent{})
...@@ -504,14 +518,13 @@ func (s *SyncDeriver) onStepEvent() { ...@@ -504,14 +518,13 @@ func (s *SyncDeriver) onStepEvent() {
func (s *SyncDeriver) onResetEvent(x rollup.ResetEvent) { func (s *SyncDeriver) onResetEvent(x rollup.ResetEvent) {
// If the system corrupts, e.g. due to a reorg, simply reset it // If the system corrupts, e.g. due to a reorg, simply reset it
s.Log.Warn("Deriver system is resetting", "err", x.Err) s.Log.Warn("Deriver system is resetting", "err", x.Err)
s.Finalizer.Reset()
s.Emitter.Emit(StepReqEvent{}) s.Emitter.Emit(StepReqEvent{})
s.Emitter.Emit(engine.ResetEngineRequestEvent{}) s.Emitter.Emit(engine.ResetEngineRequestEvent{})
} }
// SyncStep performs the sequence of encapsulated syncing steps. // SyncStep performs the sequence of encapsulated syncing steps.
// Warning: this sequence will be broken apart as outlined in op-node derivers design doc. // Warning: this sequence will be broken apart as outlined in op-node derivers design doc.
func (s *SyncDeriver) SyncStep(ctx context.Context) error { func (s *SyncDeriver) SyncStep() error {
if err := s.Drain(); err != nil { if err := s.Drain(); err != nil {
return err return err
} }
...@@ -533,25 +546,6 @@ func (s *SyncDeriver) SyncStep(ctx context.Context) error { ...@@ -533,25 +546,6 @@ func (s *SyncDeriver) SyncStep(ctx context.Context) error {
// Any now processed forkchoice updates will trigger CL-sync payload processing, if any payload is queued up. // Any now processed forkchoice updates will trigger CL-sync payload processing, if any payload is queued up.
derivationOrigin := s.Derivation.Origin()
if s.SafeHeadNotifs != nil && s.SafeHeadNotifs.Enabled() && s.Derivation.DerivationReady() &&
s.lastNotifiedSafeHead != s.Engine.SafeL2Head() {
s.lastNotifiedSafeHead = s.Engine.SafeL2Head()
// make sure we track the last L2 safe head for every new L1 block
if err := s.SafeHeadNotifs.SafeHeadUpdated(s.lastNotifiedSafeHead, derivationOrigin.ID()); err != nil {
// At this point our state is in a potentially inconsistent state as we've updated the safe head
// in the execution client but failed to post process it. Reset the pipeline so the safe head rolls back
// a little (it always rolls back at least 1 block) and then it will retry storing the entry
return derive.NewResetError(fmt.Errorf("safe head notifications failed: %w", err))
}
}
s.Finalizer.PostProcessSafeL2(s.Engine.SafeL2Head(), derivationOrigin)
// try to finalize the L2 blocks we have synced so far (no-op if L1 finality is behind)
if err := s.Finalizer.OnDerivationL1End(ctx, derivationOrigin); err != nil {
return fmt.Errorf("finalizer OnDerivationL1End error: %w", err)
}
// Since we don't force attributes to be processed at this point, // Since we don't force attributes to be processed at this point,
// we cannot safely directly trigger the derivation, as that may generate new attributes that // we cannot safely directly trigger the derivation, as that may generate new attributes that
// conflict with what attributes have not been applied yet. // conflict with what attributes have not been applied yet.
......
...@@ -62,14 +62,25 @@ func (ev PendingSafeUpdateEvent) String() string { ...@@ -62,14 +62,25 @@ func (ev PendingSafeUpdateEvent) String() string {
// PromotePendingSafeEvent signals that a block can be marked as pending-safe, and/or safe. // PromotePendingSafeEvent signals that a block can be marked as pending-safe, and/or safe.
type PromotePendingSafeEvent struct { type PromotePendingSafeEvent struct {
Ref eth.L2BlockRef Ref eth.L2BlockRef
Safe bool Safe bool
DerivedFrom eth.L1BlockRef
} }
func (ev PromotePendingSafeEvent) String() string { func (ev PromotePendingSafeEvent) String() string {
return "promote-pending-safe" return "promote-pending-safe"
} }
// SafeDerivedEvent signals that a block was determined to be safe, and derived from the given L1 block
type SafeDerivedEvent struct {
Safe eth.L2BlockRef
DerivedFrom eth.L1BlockRef
}
func (ev SafeDerivedEvent) String() string {
return "safe-derived"
}
type ProcessAttributesEvent struct { type ProcessAttributesEvent struct {
Attributes *derive.AttributesWithParent Attributes *derive.AttributesWithParent
} }
...@@ -123,6 +134,15 @@ func (ev EngineResetConfirmedEvent) String() string { ...@@ -123,6 +134,15 @@ func (ev EngineResetConfirmedEvent) String() string {
return "engine-reset-confirmed" return "engine-reset-confirmed"
} }
// PromoteFinalizedEvent signals that a block can be marked as finalized.
type PromoteFinalizedEvent struct {
Ref eth.L2BlockRef
}
func (ev PromoteFinalizedEvent) String() string {
return "promote-finalized"
}
type EngDeriver struct { type EngDeriver struct {
log log.Logger log log.Logger
cfg *rollup.Config cfg *rollup.Config
...@@ -217,7 +237,7 @@ func (d *EngDeriver) OnEvent(ev rollup.Event) { ...@@ -217,7 +237,7 @@ func (d *EngDeriver) OnEvent(ev rollup.Event) {
log.Debug("Reset of Engine is completed", log.Debug("Reset of Engine is completed",
"safeHead", x.Safe, "unsafe", x.Unsafe, "safe_timestamp", x.Safe.Time, "safeHead", x.Safe, "unsafe", x.Unsafe, "safe_timestamp", x.Safe.Time,
"unsafe_timestamp", x.Unsafe.Time) "unsafe_timestamp", x.Unsafe.Time)
d.emitter.Emit(EngineResetConfirmedEvent{}) d.emitter.Emit(EngineResetConfirmedEvent(x))
case ProcessAttributesEvent: case ProcessAttributesEvent:
d.onForceNextSafeAttributes(x.Attributes) d.onForceNextSafeAttributes(x.Attributes)
case PendingSafeRequestEvent: case PendingSafeRequestEvent:
...@@ -233,7 +253,18 @@ func (d *EngDeriver) OnEvent(ev rollup.Event) { ...@@ -233,7 +253,18 @@ func (d *EngDeriver) OnEvent(ev rollup.Event) {
} }
if x.Safe && x.Ref.Number > d.ec.SafeL2Head().Number { if x.Safe && x.Ref.Number > d.ec.SafeL2Head().Number {
d.ec.SetSafeHead(x.Ref) d.ec.SetSafeHead(x.Ref)
d.emitter.Emit(SafeDerivedEvent{Safe: x.Ref, DerivedFrom: x.DerivedFrom})
}
case PromoteFinalizedEvent:
if x.Ref.Number < d.ec.Finalized().Number {
d.log.Error("Cannot rewind finality,", "ref", x.Ref, "finalized", d.ec.Finalized())
return
}
if x.Ref.Number > d.ec.SafeL2Head().Number {
d.log.Error("Block must be safe before it can be finalized", "ref", x.Ref, "safe", d.ec.SafeL2Head())
return
} }
d.ec.SetFinalizedHead(x.Ref)
} }
} }
...@@ -301,6 +332,7 @@ func (eq *EngDeriver) onForceNextSafeAttributes(attributes *derive.AttributesWit ...@@ -301,6 +332,7 @@ func (eq *EngDeriver) onForceNextSafeAttributes(attributes *derive.AttributesWit
eq.ec.SetPendingSafeL2Head(ref) eq.ec.SetPendingSafeL2Head(ref)
if attributes.IsLastInSpan { if attributes.IsLastInSpan {
eq.ec.SetSafeHead(ref) eq.ec.SetSafeHead(ref)
eq.emitter.Emit(SafeDerivedEvent{Safe: ref, DerivedFrom: attributes.DerivedFrom})
} }
eq.emitter.Emit(PendingSafeUpdateEvent{ eq.emitter.Emit(PendingSafeUpdateEvent{
PendingSafe: eq.ec.PendingSafeL2Head(), PendingSafe: eq.ec.PendingSafeL2Head(),
......
...@@ -20,6 +20,21 @@ func (fn EmitterFunc) Emit(ev Event) { ...@@ -20,6 +20,21 @@ func (fn EmitterFunc) Emit(ev Event) {
fn(ev) fn(ev)
} }
// L1TemporaryErrorEvent identifies a temporary issue with the L1 data.
type L1TemporaryErrorEvent struct {
Err error
}
var _ Event = L1TemporaryErrorEvent{}
func (ev L1TemporaryErrorEvent) String() string {
return "l1-temporary-error"
}
// EngineTemporaryErrorEvent identifies a temporary processing issue.
// It applies to both L1 and L2 data, often inter-related.
// This scope will be reduced over time, to only capture L2-engine specific temporary errors.
// See L1TemporaryErrorEvent for L1 related temporary errors.
type EngineTemporaryErrorEvent struct { type EngineTemporaryErrorEvent struct {
Err error Err error
} }
......
...@@ -4,11 +4,13 @@ import ( ...@@ -4,11 +4,13 @@ import (
"context" "context"
"fmt" "fmt"
"sync" "sync"
"time"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"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-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
) )
...@@ -68,10 +70,17 @@ type Finalizer struct { ...@@ -68,10 +70,17 @@ type Finalizer struct {
log log.Logger log log.Logger
ctx context.Context
emitter rollup.EventEmitter
// finalizedL1 is the currently perceived finalized L1 block. // finalizedL1 is the currently perceived finalized L1 block.
// This may be ahead of the current traversed origin when syncing. // This may be ahead of the current traversed origin when syncing.
finalizedL1 eth.L1BlockRef finalizedL1 eth.L1BlockRef
// lastFinalizedL2 maintains how far we finalized, so we don't have to emit re-attempts.
lastFinalizedL2 eth.L2BlockRef
// triedFinalizeAt tracks at which L1 block number we last tried to finalize during sync. // triedFinalizeAt tracks at which L1 block number we last tried to finalize during sync.
triedFinalizeAt uint64 triedFinalizeAt uint64
...@@ -82,20 +91,19 @@ type Finalizer struct { ...@@ -82,20 +91,19 @@ type Finalizer struct {
finalityLookback uint64 finalityLookback uint64
l1Fetcher FinalizerL1Interface l1Fetcher FinalizerL1Interface
ec FinalizerEngine
} }
func NewFinalizer(log log.Logger, cfg *rollup.Config, l1Fetcher FinalizerL1Interface, ec FinalizerEngine) *Finalizer { func NewFinalizer(ctx context.Context, log log.Logger, cfg *rollup.Config, l1Fetcher FinalizerL1Interface, emitter rollup.EventEmitter) *Finalizer {
lookback := calcFinalityLookback(cfg) lookback := calcFinalityLookback(cfg)
return &Finalizer{ return &Finalizer{
ctx: ctx,
log: log, log: log,
finalizedL1: eth.L1BlockRef{}, finalizedL1: eth.L1BlockRef{},
triedFinalizeAt: 0, triedFinalizeAt: 0,
finalityData: make([]FinalityData, 0, lookback), finalityData: make([]FinalityData, 0, lookback),
finalityLookback: lookback, finalityLookback: lookback,
l1Fetcher: l1Fetcher, l1Fetcher: l1Fetcher,
ec: ec, emitter: emitter,
} }
} }
...@@ -108,8 +116,39 @@ func (fi *Finalizer) FinalizedL1() (out eth.L1BlockRef) { ...@@ -108,8 +116,39 @@ func (fi *Finalizer) FinalizedL1() (out eth.L1BlockRef) {
return return
} }
// Finalize applies a L1 finality signal, without any fork-choice or L2 state changes. type FinalizeL1Event struct {
func (fi *Finalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) { FinalizedL1 eth.L1BlockRef
}
func (ev FinalizeL1Event) String() string {
return "finalized-l1"
}
type TryFinalizeEvent struct{}
func (ev TryFinalizeEvent) String() string {
return "try-finalize"
}
func (fi *Finalizer) OnEvent(ev rollup.Event) {
switch x := ev.(type) {
case FinalizeL1Event:
fi.onL1Finalized(x.FinalizedL1)
case engine.SafeDerivedEvent:
fi.onDerivedSafeBlock(x.Safe, x.DerivedFrom)
case derive.DeriverIdleEvent:
fi.onDerivationIdle(x.Origin)
case rollup.ResetEvent:
fi.onReset()
case TryFinalizeEvent:
fi.tryFinalize()
case engine.ForkchoiceUpdateEvent:
fi.lastFinalizedL2 = x.FinalizedL2Head
}
}
// onL1Finalized applies a L1 finality signal
func (fi *Finalizer) onL1Finalized(l1Origin eth.L1BlockRef) {
fi.mu.Lock() fi.mu.Lock()
defer fi.mu.Unlock() defer fi.mu.Unlock()
prevFinalizedL1 := fi.finalizedL1 prevFinalizedL1 := fi.finalizedL1
...@@ -127,13 +166,11 @@ func (fi *Finalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) { ...@@ -127,13 +166,11 @@ func (fi *Finalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) {
fi.finalizedL1 = l1Origin fi.finalizedL1 = l1Origin
} }
// remnant of finality in EngineQueue: the finalization work does not inherit a context from the caller. // when the L1 change we can suggest to try to finalize, as the pre-condition for L2 finality has now changed
if err := fi.tryFinalize(ctx); err != nil { fi.emitter.Emit(TryFinalizeEvent{})
fi.log.Warn("received L1 finalization signal, but was unable to determine and apply L2 finality", "err", err)
}
} }
// OnDerivationL1End is called when a L1 block has been fully exhausted (i.e. no more L2 blocks to derive from). // onDerivationIdle is called when the pipeline is exhausted of new data (i.e. no more L2 blocks to derive from).
// //
// Since finality applies to all L2 blocks fully derived from the same block, // Since finality applies to all L2 blocks fully derived from the same block,
// it optimal to only check after the derivation from the L1 block has been exhausted. // it optimal to only check after the derivation from the L1 block has been exhausted.
...@@ -141,24 +178,27 @@ func (fi *Finalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) { ...@@ -141,24 +178,27 @@ func (fi *Finalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) {
// This will look at what has been buffered so far, // This will look at what has been buffered so far,
// sanity-check we are on the finalizing L1 chain, // sanity-check we are on the finalizing L1 chain,
// and finalize any L2 blocks that were fully derived from known finalized L1 blocks. // and finalize any L2 blocks that were fully derived from known finalized L1 blocks.
func (fi *Finalizer) OnDerivationL1End(ctx context.Context, derivedFrom eth.L1BlockRef) error { func (fi *Finalizer) onDerivationIdle(derivedFrom eth.L1BlockRef) {
fi.mu.Lock() fi.mu.Lock()
defer fi.mu.Unlock() defer fi.mu.Unlock()
if fi.finalizedL1 == (eth.L1BlockRef{}) { if fi.finalizedL1 == (eth.L1BlockRef{}) {
return nil // if no L1 information is finalized yet, then skip this return // if no L1 information is finalized yet, then skip this
} }
// If we recently tried finalizing, then don't try again just yet, but traverse more of L1 first. // If we recently tried finalizing, then don't try again just yet, but traverse more of L1 first.
if fi.triedFinalizeAt != 0 && derivedFrom.Number <= fi.triedFinalizeAt+finalityDelay { if fi.triedFinalizeAt != 0 && derivedFrom.Number <= fi.triedFinalizeAt+finalityDelay {
return nil return
} }
fi.log.Info("processing L1 finality information", "l1_finalized", fi.finalizedL1, "derived_from", derivedFrom, "previous", fi.triedFinalizeAt) fi.log.Debug("processing L1 finality information", "l1_finalized", fi.finalizedL1, "derived_from", derivedFrom, "previous", fi.triedFinalizeAt)
fi.triedFinalizeAt = derivedFrom.Number fi.triedFinalizeAt = derivedFrom.Number
return fi.tryFinalize(ctx) fi.emitter.Emit(TryFinalizeEvent{})
} }
func (fi *Finalizer) tryFinalize(ctx context.Context) error { func (fi *Finalizer) tryFinalize() {
// default to keep the same finalized block fi.mu.Lock()
finalizedL2 := fi.ec.Finalized() defer fi.mu.Unlock()
// overwritten if we finalize
finalizedL2 := fi.lastFinalizedL2 // may be zeroed if nothing was finalized since startup.
var finalizedDerivedFrom eth.BlockID var finalizedDerivedFrom eth.BlockID
// go through the latest inclusion data, and find the last L2 block that was derived from a finalized L1 block // go through the latest inclusion data, and find the last L2 block that was derived from a finalized L1 block
for _, fd := range fi.finalityData { for _, fd := range fi.finalityData {
...@@ -169,37 +209,41 @@ func (fi *Finalizer) tryFinalize(ctx context.Context) error { ...@@ -169,37 +209,41 @@ func (fi *Finalizer) tryFinalize(ctx context.Context) error {
} }
} }
if finalizedDerivedFrom != (eth.BlockID{}) { if finalizedDerivedFrom != (eth.BlockID{}) {
ctx, cancel := context.WithTimeout(fi.ctx, time.Second*10)
defer cancel()
// Sanity check the finality signal of L1. // Sanity check the finality signal of L1.
// Even though the signal is trusted and we do the below check also, // Even though the signal is trusted and we do the below check also,
// the signal itself has to be canonical to proceed. // the signal itself has to be canonical to proceed.
// TODO(#10724): This check could be removed if the finality signal is fully trusted, and if tests were more flexible for this case. // TODO(#10724): This check could be removed if the finality signal is fully trusted, and if tests were more flexible for this case.
signalRef, err := fi.l1Fetcher.L1BlockRefByNumber(ctx, fi.finalizedL1.Number) signalRef, err := fi.l1Fetcher.L1BlockRefByNumber(ctx, fi.finalizedL1.Number)
if err != nil { if err != nil {
return derive.NewTemporaryError(fmt.Errorf("failed to check if on finalizing L1 chain, could not fetch block %d: %w", fi.finalizedL1.Number, err)) fi.emitter.Emit(rollup.L1TemporaryErrorEvent{Err: fmt.Errorf("failed to check if on finalizing L1 chain, could not fetch block %d: %w", fi.finalizedL1.Number, err)})
return
} }
if signalRef.Hash != fi.finalizedL1.Hash { if signalRef.Hash != fi.finalizedL1.Hash {
return derive.NewResetError(fmt.Errorf("need to reset, we assumed %s is finalized, but canonical chain is %s", fi.finalizedL1, signalRef)) fi.emitter.Emit(rollup.ResetEvent{Err: fmt.Errorf("need to reset, we assumed %s is finalized, but canonical chain is %s", fi.finalizedL1, signalRef)})
return
} }
// Sanity check we are indeed on the finalizing chain, and not stuck on something else. // Sanity check we are indeed on the finalizing chain, and not stuck on something else.
// We assume that the block-by-number query is consistent with the previously received finalized chain signal // We assume that the block-by-number query is consistent with the previously received finalized chain signal
derivedRef, err := fi.l1Fetcher.L1BlockRefByNumber(ctx, finalizedDerivedFrom.Number) derivedRef, err := fi.l1Fetcher.L1BlockRefByNumber(ctx, finalizedDerivedFrom.Number)
if err != nil { if err != nil {
return derive.NewTemporaryError(fmt.Errorf("failed to check if on finalizing L1 chain, could not fetch block %d: %w", finalizedDerivedFrom.Number, err)) fi.emitter.Emit(rollup.L1TemporaryErrorEvent{Err: fmt.Errorf("failed to check if on finalizing L1 chain, could not fetch block %d: %w", finalizedDerivedFrom.Number, err)})
return
} }
if derivedRef.Hash != finalizedDerivedFrom.Hash { if derivedRef.Hash != finalizedDerivedFrom.Hash {
return derive.NewResetError(fmt.Errorf("need to reset, we are on %s, not on the finalizing L1 chain %s (towards %s)", fi.emitter.Emit(rollup.ResetEvent{Err: fmt.Errorf("need to reset, we are on %s, not on the finalizing L1 chain %s (towards %s)",
finalizedDerivedFrom, derivedRef, fi.finalizedL1)) finalizedDerivedFrom, derivedRef, fi.finalizedL1)})
return
} }
fi.emitter.Emit(engine.PromoteFinalizedEvent{Ref: finalizedL2})
fi.ec.SetFinalizedHead(finalizedL2)
} }
return nil
} }
// PostProcessSafeL2 buffers the L1 block the safe head was fully derived from, // onDerivedSafeBlock buffers the L1 block the safe head was fully derived from,
// to finalize it once the derived-from L1 block, or a later L1 block, finalizes. // to finalize it once the derived-from L1 block, or a later L1 block, finalizes.
func (fi *Finalizer) PostProcessSafeL2(l2Safe eth.L2BlockRef, derivedFrom eth.L1BlockRef) { func (fi *Finalizer) onDerivedSafeBlock(l2Safe eth.L2BlockRef, derivedFrom eth.L1BlockRef) {
fi.mu.Lock() fi.mu.Lock()
defer fi.mu.Unlock() defer fi.mu.Unlock()
// remember the last L2 block that we fully derived from the given finality data // remember the last L2 block that we fully derived from the given finality data
...@@ -225,9 +269,9 @@ func (fi *Finalizer) PostProcessSafeL2(l2Safe eth.L2BlockRef, derivedFrom eth.L1 ...@@ -225,9 +269,9 @@ func (fi *Finalizer) PostProcessSafeL2(l2Safe eth.L2BlockRef, derivedFrom eth.L1
} }
} }
// Reset clears the recent history of safe-L2 blocks used for finalization, // onReset clears the recent history of safe-L2 blocks used for finalization,
// to avoid finalizing any reorged-out L2 blocks. // to avoid finalizing any reorged-out L2 blocks.
func (fi *Finalizer) Reset() { func (fi *Finalizer) onReset() {
fi.mu.Lock() fi.mu.Lock()
defer fi.mu.Unlock() defer fi.mu.Unlock()
fi.finalityData = fi.finalityData[:0] fi.finalityData = fi.finalityData[:0]
......
...@@ -13,25 +13,12 @@ import ( ...@@ -13,25 +13,12 @@ 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-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"
) )
type fakeEngine struct {
finalized eth.L2BlockRef
}
func (f *fakeEngine) Finalized() eth.L2BlockRef {
return f.finalized
}
func (f *fakeEngine) SetFinalizedHead(ref eth.L2BlockRef) {
f.finalized = ref
}
var _ FinalizerEngine = (*fakeEngine)(nil)
func TestEngineQueue_Finalize(t *testing.T) { func TestEngineQueue_Finalize(t *testing.T) {
rng := rand.New(rand.NewSource(1234)) rng := rand.New(rand.NewSource(1234))
...@@ -80,12 +67,12 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -80,12 +67,12 @@ func TestEngineQueue_Finalize(t *testing.T) {
ParentHash: refG.Hash, ParentHash: refG.Hash,
Time: refG.Time + l1Time, Time: refG.Time + l1Time,
} }
refI := eth.L1BlockRef{ //refI := eth.L1BlockRef{
Hash: testutils.RandomHash(rng), // Hash: testutils.RandomHash(rng),
Number: refH.Number + 1, // Number: refH.Number + 1,
ParentHash: refH.Hash, // ParentHash: refH.Hash,
Time: refH.Time + l1Time, // Time: refH.Time + l1Time,
} //}
refA0 := eth.L2BlockRef{ refA0 := eth.L2BlockRef{
Hash: testutils.RandomHash(rng), Hash: testutils.RandomHash(rng),
...@@ -203,22 +190,29 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -203,22 +190,29 @@ func TestEngineQueue_Finalize(t *testing.T) {
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil)
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil)
ec := &fakeEngine{} emitter := &testutils.MockEmitter{}
ec.SetFinalizedHead(refA1) fi := NewFinalizer(context.Background(), logger, &rollup.Config{}, l1F, emitter)
fi := NewFinalizer(logger, &rollup.Config{}, l1F, ec)
// now say C1 was included in D and became the new safe head // now say C1 was included in D and became the new safe head
fi.PostProcessSafeL2(refC1, refD) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC1, DerivedFrom: refD})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refD)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refD})
emitter.AssertExpectations(t)
// now say D0 was included in E and became the new safe head // now say D0 was included in E and became the new safe head
fi.PostProcessSafeL2(refD0, refE) fi.OnEvent(engine.SafeDerivedEvent{Safe: refD0, DerivedFrom: refE})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
emitter.AssertExpectations(t)
// let's finalize D from which we fully derived C1, but not D0
fi.Finalize(context.Background(), refD) // Let's finalize D from which we fully derived C1, but not D0
require.Equal(t, refC1, ec.Finalized(), "C1 was included in finalized D, and should now be finalized, as finality signal is instantly picked up") // This will trigger an attempt of L2 finalization.
emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(FinalizeL1Event{FinalizedL1: refD})
emitter.AssertExpectations(t)
// C1 was included in finalized D, and should now be finalized
emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refC1})
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
}) })
// Finality signal is received, but couldn't immediately be checked // Finality signal is received, but couldn't immediately be checked
...@@ -230,25 +224,37 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -230,25 +224,37 @@ func TestEngineQueue_Finalize(t *testing.T) {
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // to check finality signal l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // to check finality signal
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // to check what was derived from (same in this case) l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // to check what was derived from (same in this case)
ec := &fakeEngine{} emitter := &testutils.MockEmitter{}
ec.SetFinalizedHead(refA1) fi := NewFinalizer(context.Background(), logger, &rollup.Config{}, l1F, emitter)
fi := NewFinalizer(logger, &rollup.Config{}, l1F, ec)
// now say C1 was included in D and became the new safe head // now say C1 was included in D and became the new safe head
fi.PostProcessSafeL2(refC1, refD) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC1, DerivedFrom: refD})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refD)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refD})
emitter.AssertExpectations(t)
// now say D0 was included in E and became the new safe head // now say D0 was included in E and became the new safe head
fi.PostProcessSafeL2(refD0, refE) fi.OnEvent(engine.SafeDerivedEvent{Safe: refD0, DerivedFrom: refE})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
emitter.AssertExpectations(t)
// let's finalize D from which we fully derived C1, but not D0 // let's finalize D from which we fully derived C1, but not D0
fi.Finalize(context.Background(), refD) emitter.ExpectOnce(TryFinalizeEvent{})
require.Equal(t, refA1, ec.Finalized(), "C1 was included in finalized D, but finality could not be verified yet, due to temporary test error") fi.OnEvent(FinalizeL1Event{FinalizedL1: refD})
emitter.AssertExpectations(t)
require.NoError(t, fi.OnDerivationL1End(context.Background(), refF)) // C1 was included in finalized D, but finality could not be verified yet, due to temporary test error
require.Equal(t, refC1, ec.Finalized(), "C1 was included in finalized D, and should now be finalized, as check can succeed when revisited") emitter.ExpectOnceType("L1TemporaryErrorEvent")
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
// upon the next signal we should schedule a finalization re-attempt
emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(derive.DeriverIdleEvent{Origin: refF})
emitter.AssertExpectations(t)
// C1 was included in finalized D, and should now be finalized, as check can succeed when revisited
emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refC1})
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
}) })
// Test that finality progression can repeat a few times. // Test that finality progression can repeat a few times.
...@@ -257,43 +263,80 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -257,43 +263,80 @@ func TestEngineQueue_Finalize(t *testing.T) {
l1F := &testutils.MockL1Source{} l1F := &testutils.MockL1Source{}
defer l1F.AssertExpectations(t) defer l1F.AssertExpectations(t)
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) emitter := &testutils.MockEmitter{}
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) fi := NewFinalizer(context.Background(), logger, &rollup.Config{}, l1F, emitter)
l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil)
l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil)
l1F.ExpectL1BlockRefByNumber(refH.Number, refH, nil)
l1F.ExpectL1BlockRefByNumber(refH.Number, refH, nil)
ec := &fakeEngine{}
ec.SetFinalizedHead(refA1)
fi := NewFinalizer(logger, &rollup.Config{}, l1F, ec) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC1, DerivedFrom: refD})
fi.OnEvent(derive.DeriverIdleEvent{Origin: refD})
emitter.AssertExpectations(t)
fi.PostProcessSafeL2(refC1, refD) fi.OnEvent(engine.SafeDerivedEvent{Safe: refD0, DerivedFrom: refE})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refD)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
emitter.AssertExpectations(t)
fi.PostProcessSafeL2(refD0, refE) // L1 finality signal will trigger L2 finality attempt
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(FinalizeL1Event{FinalizedL1: refD})
emitter.AssertExpectations(t)
fi.Finalize(context.Background(), refD) // C1 was included in D, and should be finalized now
require.NoError(t, fi.OnDerivationL1End(context.Background(), refF)) emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refC1})
require.Equal(t, refC1, ec.Finalized(), "C1 was included in D, and should be finalized now") l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil)
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil)
fi.Finalize(context.Background(), refE) fi.OnEvent(TryFinalizeEvent{})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refG)) emitter.AssertExpectations(t)
require.Equal(t, refD0, ec.Finalized(), "D0 was included in E, and should be finalized now") l1F.AssertExpectations(t)
fi.PostProcessSafeL2(refD1, refH) // Another L1 finality event, trigger L2 finality attempt again
fi.PostProcessSafeL2(refE0, refH) emitter.ExpectOnce(TryFinalizeEvent{})
fi.PostProcessSafeL2(refE1, refH) fi.OnEvent(FinalizeL1Event{FinalizedL1: refE})
fi.PostProcessSafeL2(refF0, refH) emitter.AssertExpectations(t)
fi.PostProcessSafeL2(refF1, refH)
require.NoError(t, fi.OnDerivationL1End(context.Background(), refH))
require.Equal(t, refD0, ec.Finalized(), "D1-F1 were included in L1 blocks that have not been finalized yet")
fi.Finalize(context.Background(), refH) // D0 was included in E, and should be finalized now
require.NoError(t, fi.OnDerivationL1End(context.Background(), refI)) emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refD0})
require.Equal(t, refF1, ec.Finalized(), "F1 should be finalized now") l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil)
l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil)
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
l1F.AssertExpectations(t)
// D0 is still there in the buffer, and may be finalized again, if it were not for the latest forkchoice update.
fi.OnEvent(engine.ForkchoiceUpdateEvent{FinalizedL2Head: refD0})
emitter.AssertExpectations(t) // should trigger no events
// we expect a finality attempt, since we have not idled on something yet
emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(derive.DeriverIdleEvent{Origin: refG})
emitter.AssertExpectations(t)
fi.OnEvent(engine.SafeDerivedEvent{Safe: refD1, DerivedFrom: refH})
fi.OnEvent(engine.SafeDerivedEvent{Safe: refE0, DerivedFrom: refH})
fi.OnEvent(engine.SafeDerivedEvent{Safe: refE1, DerivedFrom: refH})
fi.OnEvent(engine.SafeDerivedEvent{Safe: refF0, DerivedFrom: refH})
fi.OnEvent(engine.SafeDerivedEvent{Safe: refF1, DerivedFrom: refH})
emitter.AssertExpectations(t) // above updates add data, but no attempt is made until idle or L1 signal
// We recently finalized already, and there is no new L1 finality data
fi.OnEvent(derive.DeriverIdleEvent{Origin: refH})
emitter.AssertExpectations(t)
// D1-F1 were included in L1 blocks that have not been finalized yet.
// D0 is known to be finalized already.
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
// Now L1 block H is actually finalized, and we can proceed with another attempt
emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(FinalizeL1Event{FinalizedL1: refH})
emitter.AssertExpectations(t)
// F1 should be finalized now, since it was included in H
emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refF1})
l1F.ExpectL1BlockRefByNumber(refH.Number, refH, nil)
l1F.ExpectL1BlockRefByNumber(refH.Number, refH, nil)
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
l1F.AssertExpectations(t)
}) })
// In this test the finality signal is for a block more than // In this test the finality signal is for a block more than
...@@ -305,22 +348,28 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -305,22 +348,28 @@ func TestEngineQueue_Finalize(t *testing.T) {
l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // check the signal l1F.ExpectL1BlockRefByNumber(refD.Number, refD, nil) // check the signal
l1F.ExpectL1BlockRefByNumber(refC.Number, refC, nil) // check what we derived the L2 block from l1F.ExpectL1BlockRefByNumber(refC.Number, refC, nil) // check what we derived the L2 block from
ec := &fakeEngine{} emitter := &testutils.MockEmitter{}
ec.SetFinalizedHead(refA1) fi := NewFinalizer(context.Background(), logger, &rollup.Config{}, l1F, emitter)
fi := NewFinalizer(logger, &rollup.Config{}, l1F, ec)
// now say B1 was included in C and became the new safe head // now say B1 was included in C and became the new safe head
fi.PostProcessSafeL2(refB1, refC) fi.OnEvent(engine.SafeDerivedEvent{Safe: refB1, DerivedFrom: refC})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refC)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refC})
emitter.AssertExpectations(t)
// now say C0 was included in E and became the new safe head // now say C0 was included in E and became the new safe head
fi.PostProcessSafeL2(refC0, refE) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC0, DerivedFrom: refE})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
emitter.AssertExpectations(t)
// let's finalize D, from which we fully derived B1, but not C0 (referenced L1 origin in L2 block != inclusion of L2 block in L1 chain) // let's finalize D, from which we fully derived B1, but not C0 (referenced L1 origin in L2 block != inclusion of L2 block in L1 chain)
fi.Finalize(context.Background(), refD) emitter.ExpectOnce(TryFinalizeEvent{})
require.Equal(t, refB1, ec.Finalized(), "B1 was included in finalized D, and should now be finalized") fi.OnEvent(FinalizeL1Event{FinalizedL1: refD})
emitter.AssertExpectations(t)
// B1 was included in finalized D, and should now be finalized
emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refB1})
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
}) })
// Test that reorg race condition is handled. // Test that reorg race condition is handled.
...@@ -335,14 +384,13 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -335,14 +384,13 @@ func TestEngineQueue_Finalize(t *testing.T) {
l1F.ExpectL1BlockRefByNumber(refF.Number, refF, nil) // check signal l1F.ExpectL1BlockRefByNumber(refF.Number, refF, nil) // check signal
l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil) // post-reorg l1F.ExpectL1BlockRefByNumber(refE.Number, refE, nil) // post-reorg
ec := &fakeEngine{} emitter := &testutils.MockEmitter{}
ec.SetFinalizedHead(refA1) fi := NewFinalizer(context.Background(), logger, &rollup.Config{}, l1F, emitter)
fi := NewFinalizer(logger, &rollup.Config{}, l1F, ec)
// now say B1 was included in C and became the new safe head // now say B1 was included in C and became the new safe head
fi.PostProcessSafeL2(refB1, refC) fi.OnEvent(engine.SafeDerivedEvent{Safe: refB1, DerivedFrom: refC})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refC)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refC})
emitter.AssertExpectations(t)
// temporary fork of the L1, and derived safe L2 blocks from. // temporary fork of the L1, and derived safe L2 blocks from.
refC0Alt := eth.L2BlockRef{ refC0Alt := eth.L2BlockRef{
...@@ -367,34 +415,56 @@ func TestEngineQueue_Finalize(t *testing.T) { ...@@ -367,34 +415,56 @@ func TestEngineQueue_Finalize(t *testing.T) {
ParentHash: refC.Hash, ParentHash: refC.Hash,
Time: refC.Time + l1Time, Time: refC.Time + l1Time,
} }
fi.PostProcessSafeL2(refC0Alt, refDAlt) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC0Alt, DerivedFrom: refDAlt})
fi.PostProcessSafeL2(refC1Alt, refDAlt) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC1Alt, DerivedFrom: refDAlt})
// We get an early finality signal for F, of the chain that did not include refC0Alt and refC1Alt, // We get an early finality signal for F, of the chain that did not include refC0Alt and refC1Alt,
// as L1 block F does not build on DAlt. // as L1 block F does not build on DAlt.
// The finality signal was for a new chain, while derivation is on an old stale chain. // The finality signal was for a new chain, while derivation is on an old stale chain.
// It should be detected that C0Alt and C1Alt cannot actually be finalized, // It should be detected that C0Alt and C1Alt cannot actually be finalized,
// even though they are older than the latest finality signal. // even though they are older than the latest finality signal.
fi.Finalize(context.Background(), refF) emitter.ExpectOnce(TryFinalizeEvent{})
require.Equal(t, refA1, ec.Finalized(), "cannot verify refC0Alt and refC1Alt, and refB1 is older and not checked") fi.OnEvent(FinalizeL1Event{FinalizedL1: refF})
emitter.AssertExpectations(t)
// cannot verify refC0Alt and refC1Alt, and refB1 is older and not checked
emitter.ExpectOnceType("ResetEvent")
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t) // no change in finality
// And process DAlt, still stuck on old chain. // And process DAlt, still stuck on old chain.
require.ErrorIs(t, derive.ErrReset, fi.OnDerivationL1End(context.Background(), refDAlt))
require.Equal(t, refA1, ec.Finalized(), "no new finalized L2 blocks after early finality signal with stale chain") emitter.ExpectOnce(TryFinalizeEvent{})
require.Equal(t, refF, fi.FinalizedL1(), "remember the new finality signal for later however") fi.OnEvent(derive.DeriverIdleEvent{Origin: refDAlt})
emitter.AssertExpectations(t)
// no new finalized L2 blocks after early finality signal with stale chain
emitter.ExpectOnceType("ResetEvent")
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
// Now reset, because of the reset error // Now reset, because of the reset error
fi.Reset() fi.OnEvent(rollup.ResetEvent{})
require.Equal(t, refF, fi.FinalizedL1(), "remember the new finality signal for later however")
// And process the canonical chain, with empty block D (no post-processing of canonical C0 blocks yet) // And process the canonical chain, with empty block D (no post-processing of canonical C0 blocks yet)
require.NoError(t, fi.OnDerivationL1End(context.Background(), refD)) emitter.ExpectOnce(TryFinalizeEvent{})
fi.OnEvent(derive.DeriverIdleEvent{Origin: refD})
emitter.AssertExpectations(t)
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t) // no new finality
// Include C0 in E // Include C0 in E
fi.PostProcessSafeL2(refC0, refE) fi.OnEvent(engine.SafeDerivedEvent{Safe: refC0, DerivedFrom: refE})
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
// Due to the "finalityDelay" we don't repeat finality checks shortly after one another. // Due to the "finalityDelay" we don't repeat finality checks shortly after one another,
require.Equal(t, refA1, ec.Finalized()) // and don't expect a finality attempt.
emitter.AssertExpectations(t)
// if we reset the attempt, then we can finalize however. // if we reset the attempt, then we can finalize however.
fi.triedFinalizeAt = 0 fi.triedFinalizeAt = 0
require.NoError(t, fi.OnDerivationL1End(context.Background(), refE)) emitter.ExpectOnce(TryFinalizeEvent{})
require.Equal(t, refC0, ec.Finalized()) fi.OnEvent(derive.DeriverIdleEvent{Origin: refE})
emitter.AssertExpectations(t)
emitter.ExpectOnce(engine.PromoteFinalizedEvent{Ref: refC0})
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
}) })
} }
...@@ -26,17 +26,17 @@ type PlasmaFinalizer struct { ...@@ -26,17 +26,17 @@ type PlasmaFinalizer struct {
backend PlasmaBackend backend PlasmaBackend
} }
func NewPlasmaFinalizer(log log.Logger, cfg *rollup.Config, func NewPlasmaFinalizer(ctx context.Context, log log.Logger, cfg *rollup.Config,
l1Fetcher FinalizerL1Interface, ec FinalizerEngine, l1Fetcher FinalizerL1Interface, emitter rollup.EventEmitter,
backend PlasmaBackend) *PlasmaFinalizer { backend PlasmaBackend) *PlasmaFinalizer {
inner := NewFinalizer(log, cfg, l1Fetcher, ec) inner := NewFinalizer(ctx, log, cfg, l1Fetcher, emitter)
// In alt-da mode, the finalization signal is proxied through the plasma manager. // In alt-da mode, the finalization signal is proxied through the plasma manager.
// Finality signal will come from the DA contract or L1 finality whichever is last. // Finality signal will come from the DA contract or L1 finality whichever is last.
// The plasma module will then call the inner.Finalize function when applicable. // The plasma module will then call the inner.Finalize function when applicable.
backend.OnFinalizedHeadSignal(func(ref eth.L1BlockRef) { backend.OnFinalizedHeadSignal(func(ref eth.L1BlockRef) {
inner.Finalize(context.Background(), ref) // plasma backend context passing can be improved inner.OnEvent(FinalizeL1Event{FinalizedL1: ref})
}) })
return &PlasmaFinalizer{ return &PlasmaFinalizer{
...@@ -45,6 +45,11 @@ func NewPlasmaFinalizer(log log.Logger, cfg *rollup.Config, ...@@ -45,6 +45,11 @@ func NewPlasmaFinalizer(log log.Logger, cfg *rollup.Config,
} }
} }
func (fi *PlasmaFinalizer) Finalize(ctx context.Context, l1Origin eth.L1BlockRef) { func (fi *PlasmaFinalizer) OnEvent(ev rollup.Event) {
fi.backend.Finalize(l1Origin) switch x := ev.(type) {
case FinalizeL1Event:
fi.backend.Finalize(x.FinalizedL1)
default:
fi.Finalizer.OnEvent(ev)
}
} }
...@@ -11,6 +11,8 @@ import ( ...@@ -11,6 +11,8 @@ import (
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"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/engine"
plasma "github.com/ethereum-optimism/optimism/op-plasma" plasma "github.com/ethereum-optimism/optimism/op-plasma"
"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"
...@@ -83,9 +85,6 @@ func TestPlasmaFinalityData(t *testing.T) { ...@@ -83,9 +85,6 @@ func TestPlasmaFinalityData(t *testing.T) {
SequenceNumber: 1, SequenceNumber: 1,
} }
ec := &fakeEngine{}
ec.SetFinalizedHead(refA1)
// Simulate plasma finality by waiting for the finalized-inclusion // Simulate plasma finality by waiting for the finalized-inclusion
// of a commitment to turn into undisputed finalized data. // of a commitment to turn into undisputed finalized data.
commitmentInclusionFinalized := eth.L1BlockRef{} commitmentInclusionFinalized := eth.L1BlockRef{}
...@@ -95,7 +94,9 @@ func TestPlasmaFinalityData(t *testing.T) { ...@@ -95,7 +94,9 @@ func TestPlasmaFinalityData(t *testing.T) {
}, },
forwardTo: nil, forwardTo: nil,
} }
fi := NewPlasmaFinalizer(logger, cfg, l1F, ec, plasmaBackend)
emitter := &testutils.MockEmitter{}
fi := NewPlasmaFinalizer(context.Background(), logger, cfg, l1F, emitter, plasmaBackend)
require.NotNil(t, plasmaBackend.forwardTo, "plasma backend must have access to underlying standard finalizer") require.NotNil(t, plasmaBackend.forwardTo, "plasma backend must have access to underlying standard finalizer")
require.Equal(t, expFinalityLookback, cap(fi.finalityData)) require.Equal(t, expFinalityLookback, cap(fi.finalityData))
...@@ -107,7 +108,9 @@ func TestPlasmaFinalityData(t *testing.T) { ...@@ -107,7 +108,9 @@ func TestPlasmaFinalityData(t *testing.T) {
// and post processing. // and post processing.
for i := uint64(0); i < 200; i++ { for i := uint64(0); i < 200; i++ {
if i == 10 { // finalize a L1 commitment if i == 10 { // finalize a L1 commitment
fi.Finalize(context.Background(), l1parent) fi.OnEvent(FinalizeL1Event{FinalizedL1: l1parent})
emitter.AssertExpectations(t) // no events emitted upon L1 finality
require.Equal(t, l1parent, commitmentInclusionFinalized, "plasma backend received L1 signal")
} }
previous := l1parent previous := l1parent
...@@ -127,24 +130,56 @@ func TestPlasmaFinalityData(t *testing.T) { ...@@ -127,24 +130,56 @@ func TestPlasmaFinalityData(t *testing.T) {
L1Origin: previous.ID(), // reference previous origin, not the block the batch was included in L1Origin: previous.ID(), // reference previous origin, not the block the batch was included in
SequenceNumber: j, SequenceNumber: j,
} }
fi.PostProcessSafeL2(l2parent, l1parent) fi.OnEvent(engine.SafeDerivedEvent{Safe: l2parent, DerivedFrom: l1parent})
emitter.AssertExpectations(t)
} }
require.NoError(t, fi.OnDerivationL1End(context.Background(), l1parent)) // might trigger finalization attempt, if expired finality delay
emitter.ExpectMaybeRun(func(ev rollup.Event) {
require.IsType(t, TryFinalizeEvent{}, ev)
})
fi.OnEvent(derive.DeriverIdleEvent{})
emitter.AssertExpectations(t)
// clear expectations
emitter.Mock.ExpectedCalls = nil
// no L2 finalize event, as no L1 finality signal has been forwarded by plasma backend yet
fi.OnEvent(TryFinalizeEvent{})
emitter.AssertExpectations(t)
// Pretend to be the plasma backend,
// send the original finalization signal to the underlying finalizer,
// now that we are sure the commitment itself is not just finalized,
// but the referenced data cannot be disputed anymore.
plasmaFinalization := commitmentInclusionFinalized.Number + cfg.PlasmaConfig.DAChallengeWindow plasmaFinalization := commitmentInclusionFinalized.Number + cfg.PlasmaConfig.DAChallengeWindow
if i == plasmaFinalization { if commitmentInclusionFinalized != (eth.L1BlockRef{}) && l1parent.Number == plasmaFinalization {
// Pretend to be the plasma backend, // When the signal is forwarded, a finalization attempt will be scheduled
// send the original finalization signal to the underlying finalizer, emitter.ExpectOnce(TryFinalizeEvent{})
// now that we are sure the commitment itself is not just finalized,
// but the referenced data cannot be disputed anymore.
plasmaBackend.forwardTo(commitmentInclusionFinalized) plasmaBackend.forwardTo(commitmentInclusionFinalized)
} emitter.AssertExpectations(t)
// The next time OnDerivationL1End is called, after the finality signal was triggered by plasma backend, require.Equal(t, commitmentInclusionFinalized, fi.finalizedL1, "finality signal now made its way in regular finalizer")
// we should have a finalized L2 block.
// The L1 origin of the simulated L2 blocks lags 1 behind the block the L2 block is included in on L1. // As soon as a finalization attempt is made, after the finality signal was triggered by plasma backend,
// So to check the L2 finality progress, we check if the next L1 block after the L1 origin // we should get an attempt to get a finalized L2 block.
// of the safe block matches that of the finalized L1 block. // In this test the L1 origin of the simulated L2 blocks lags 1 behind the block the L2 block is included in on L1.
if i == plasmaFinalization+1 { // So to check the L2 finality progress, we check if the next L1 block after the L1 origin
require.Equal(t, plasmaFinalization, ec.Finalized().L1Origin.Number+1) // of the safe block matches that of the finalized L1 block.
l1F.ExpectL1BlockRefByNumber(commitmentInclusionFinalized.Number, commitmentInclusionFinalized, nil)
l1F.ExpectL1BlockRefByNumber(commitmentInclusionFinalized.Number, commitmentInclusionFinalized, nil)
var finalizedL2 eth.L2BlockRef
emitter.ExpectOnceRun(func(ev rollup.Event) {
if x, ok := ev.(engine.PromoteFinalizedEvent); ok {
finalizedL2 = x.Ref
} else {
t.Fatalf("expected L2 finalization, but got: %s", ev)
}
})
fi.OnEvent(TryFinalizeEvent{})
l1F.AssertExpectations(t)
emitter.AssertExpectations(t)
require.Equal(t, commitmentInclusionFinalized.Number, finalizedL2.L1Origin.Number+1)
// Confirm finalization, so there will be no repeats of the PromoteFinalizedEvent
fi.OnEvent(engine.ForkchoiceUpdateEvent{FinalizedL2Head: finalizedL2})
emitter.AssertExpectations(t)
} }
} }
......
...@@ -68,6 +68,9 @@ func (d *ProgramDeriver) OnEvent(ev rollup.Event) { ...@@ -68,6 +68,9 @@ func (d *ProgramDeriver) OnEvent(ev rollup.Event) {
case rollup.ResetEvent: case rollup.ResetEvent:
d.closing = true d.closing = true
d.result = fmt.Errorf("unexpected reset error: %w", x.Err) d.result = fmt.Errorf("unexpected reset error: %w", x.Err)
case rollup.L1TemporaryErrorEvent:
d.closing = true
d.result = fmt.Errorf("unexpected L1 error: %w", x.Err)
case rollup.EngineTemporaryErrorEvent: case rollup.EngineTemporaryErrorEvent:
// (Legacy case): While most temporary errors are due to requests for external data failing which can't happen, // (Legacy case): While most temporary errors are due to requests for external data failing which can't happen,
// they may also be returned due to other events like channels timing out so need to be handled // they may also be returned due to other events like channels timing out so need to be handled
......
...@@ -120,8 +120,16 @@ func TestProgramDeriver(t *testing.T) { ...@@ -120,8 +120,16 @@ func TestProgramDeriver(t *testing.T) {
require.True(t, p.closing) require.True(t, p.closing)
require.NotNil(t, p.result) require.NotNil(t, p.result)
}) })
// on temporary error: continue derivation. // on L1 temporary error: stop with error
t.Run("temp error event", func(t *testing.T) { t.Run("L1 temporary error event", func(t *testing.T) {
p, m := newProgram(t, 1000)
p.OnEvent(rollup.L1TemporaryErrorEvent{Err: errors.New("temp test err")})
m.AssertExpectations(t)
require.True(t, p.closing)
require.NotNil(t, p.result)
})
// on engine temporary error: continue derivation (because legacy, not all connection related)
t.Run("engine temp error event", func(t *testing.T) {
p, m := newProgram(t, 1000) p, m := newProgram(t, 1000)
m.ExpectOnce(engine.PendingSafeRequestEvent{}) m.ExpectOnce(engine.PendingSafeRequestEvent{})
p.OnEvent(rollup.EngineTemporaryErrorEvent{Err: errors.New("temp test err")}) p.OnEvent(rollup.EngineTemporaryErrorEvent{Err: errors.New("temp test err")})
......
...@@ -18,10 +18,22 @@ func (m *MockEmitter) ExpectOnce(expected rollup.Event) { ...@@ -18,10 +18,22 @@ func (m *MockEmitter) ExpectOnce(expected rollup.Event) {
m.Mock.On("Emit", expected).Once() m.Mock.On("Emit", expected).Once()
} }
func (m *MockEmitter) ExpectMaybeRun(fn func(ev rollup.Event)) {
m.Mock.On("Emit", mock.Anything).Maybe().Run(func(args mock.Arguments) {
fn(args.Get(0).(rollup.Event))
})
}
func (m *MockEmitter) ExpectOnceType(typ string) { func (m *MockEmitter) ExpectOnceType(typ string) {
m.Mock.On("Emit", mock.AnythingOfType(typ)).Once() m.Mock.On("Emit", mock.AnythingOfType(typ)).Once()
} }
func (m *MockEmitter) ExpectOnceRun(fn func(ev rollup.Event)) {
m.Mock.On("Emit", mock.Anything).Once().Run(func(args mock.Arguments) {
fn(args.Get(0).(rollup.Event))
})
}
func (m *MockEmitter) AssertExpectations(t mock.TestingT) { func (m *MockEmitter) AssertExpectations(t mock.TestingT) {
m.Mock.AssertExpectations(t) m.Mock.AssertExpectations(t)
} }
......
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