From 465a10bc35a048dd0987bac95ebc13cea59a06c0 Mon Sep 17 00:00:00 2001 From: protolambda <proto@protolambda.com> Date: Sat, 21 Jan 2023 21:05:00 +0100 Subject: [PATCH] op-node: refactor sequencer to use EngineControl, test sequencing with chaos-monkey-like test --- op-e2e/actions/l2_sequencer.go | 65 ++--- op-node/rollup/derive/pipeline.go | 7 - op-node/rollup/driver/driver.go | 21 +- op-node/rollup/driver/sequencer.go | 175 +++++------ op-node/rollup/driver/sequencer_test.go | 368 ++++++++++++++++++++++++ op-node/rollup/driver/state.go | 125 ++------ op-node/rollup/driver/state_tob_test.go | 279 ------------------ 7 files changed, 513 insertions(+), 527 deletions(-) create mode 100644 op-node/rollup/driver/sequencer_test.go delete mode 100644 op-node/rollup/driver/state_tob_test.go diff --git a/op-e2e/actions/l2_sequencer.go b/op-e2e/actions/l2_sequencer.go index 28e231531..a0243473f 100644 --- a/op-e2e/actions/l2_sequencer.go +++ b/op-e2e/actions/l2_sequencer.go @@ -1,37 +1,52 @@ package actions import ( + "context" + "github.com/ethereum/go-ethereum/log" "github.com/stretchr/testify/require" "github.com/ethereum-optimism/optimism/op-node/eth" - "github.com/ethereum-optimism/optimism/op-node/metrics" "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/driver" ) +// MockL1OriginSelector is a shim to override the origin as sequencer, so we can force it to stay on an older origin. +type MockL1OriginSelector struct { + actual *driver.L1OriginSelector + originOverride eth.L1BlockRef // override which origin gets picked +} + +func (m *MockL1OriginSelector) FindL1Origin(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) { + if m.originOverride != (eth.L1BlockRef{}) { + return m.originOverride, nil + } + return m.actual.FindL1Origin(ctx, l1Head, l2Head) +} + // L2Sequencer is an actor that functions like a rollup node, // without the full P2P/API/Node stack, but just the derivation state, and simplified driver with sequencing ability. type L2Sequencer struct { L2Verifier - sequencer *driver.Sequencer - l1OriginSelector *driver.L1OriginSelector - - seqOldOrigin bool // stay on current L1 origin when sequencing a block, unless forced to adopt the next origin + sequencer *driver.Sequencer failL2GossipUnsafeBlock error // mock error + + mockL1OriginSelector *MockL1OriginSelector } func NewL2Sequencer(t Testing, log log.Logger, l1 derive.L1Fetcher, eng L2API, cfg *rollup.Config, seqConfDepth uint64) *L2Sequencer { ver := NewL2Verifier(t, log, l1, eng, cfg) attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, eng) + l1OriginSelector := &MockL1OriginSelector{ + actual: driver.NewL1OriginSelector(log, cfg, l1, seqConfDepth), + } return &L2Sequencer{ L2Verifier: *ver, - sequencer: driver.NewSequencer(log, cfg, eng, ver.derivation, attrBuilder, metrics.NoopMetrics), - l1OriginSelector: driver.NewL1OriginSelector(log, cfg, l1, seqConfDepth), - seqOldOrigin: false, + sequencer: driver.NewSequencer(log, cfg, ver.derivation, attrBuilder, l1OriginSelector), + mockL1OriginSelector: l1OriginSelector, failL2GossipUnsafeBlock: nil, } } @@ -47,22 +62,7 @@ func (s *L2Sequencer) ActL2StartBlock(t Testing) { return } - parent := s.derivation.UnsafeL2Head() - var origin eth.L1BlockRef - if s.seqOldOrigin { - // force old origin, for testing purposes - oldOrigin, err := s.l1.L1BlockRefByHash(t.Ctx(), parent.L1Origin.Hash) - require.NoError(t, err, "failed to get current origin: %s", parent.L1Origin) - origin = oldOrigin - s.seqOldOrigin = false // don't repeat this - } else { - // select origin the real way - l1Origin, err := s.l1OriginSelector.FindL1Origin(t.Ctx(), s.l1State.L1Head(), parent) - require.NoError(t, err) - origin = l1Origin - } - - err := s.sequencer.StartBuildingBlock(t.Ctx(), origin) + err := s.sequencer.StartBuildingBlock(t.Ctx(), s.l1State.L1Head()) require.NoError(t, err, "failed to start block building") s.l2Building = true @@ -76,24 +76,21 @@ func (s *L2Sequencer) ActL2EndBlock(t Testing) { } s.l2Building = false - payload, err := s.sequencer.CompleteBuildingBlock(t.Ctx()) + _, err := s.sequencer.CompleteBuildingBlock(t.Ctx()) // TODO: there may be legitimate temporary errors here, if we mock engine API RPC-failure. // For advanced tests we can catch those and print a warning instead. require.NoError(t, err) - ref, err := derive.PayloadToBlockRef(payload, &s.rollupCfg.Genesis) - require.NoError(t, err, "payload must convert to block ref") - s.derivation.SetUnsafeHead(ref) // TODO: action-test publishing of payload on p2p } // ActL2KeepL1Origin makes the sequencer use the current L1 origin, even if the next origin is available. func (s *L2Sequencer) ActL2KeepL1Origin(t Testing) { - if s.seqOldOrigin { // don't do this twice - t.InvalidAction("already decided to keep old L1 origin") - return - } - s.seqOldOrigin = true + parent := s.derivation.UnsafeL2Head() + // force old origin, for testing purposes + oldOrigin, err := s.l1.L1BlockRefByHash(t.Ctx(), parent.L1Origin.Hash) + require.NoError(t, err, "failed to get current origin: %s", parent.L1Origin) + s.mockL1OriginSelector.originOverride = oldOrigin } // ActBuildToL1Head builds empty blocks until (incl.) the L1 head becomes the L2 origin @@ -109,7 +106,7 @@ func (s *L2Sequencer) ActBuildToL1Head(t Testing) { func (s *L2Sequencer) ActBuildToL1HeadExcl(t Testing) { for { s.ActL2PipelineFull(t) - nextOrigin, err := s.l1OriginSelector.FindL1Origin(t.Ctx(), s.l1State.L1Head(), s.derivation.UnsafeL2Head()) + nextOrigin, err := s.mockL1OriginSelector.FindL1Origin(t.Ctx(), s.l1State.L1Head(), s.derivation.UnsafeL2Head()) require.NoError(t, err) if nextOrigin.Number >= s.l1State.L1Head().Number { break diff --git a/op-node/rollup/derive/pipeline.go b/op-node/rollup/derive/pipeline.go index de4c2ccd5..5de1de719 100644 --- a/op-node/rollup/derive/pipeline.go +++ b/op-node/rollup/derive/pipeline.go @@ -148,13 +148,6 @@ func (dp *DerivationPipeline) BuildingPayload() (onto eth.L2BlockRef, id eth.Pay return dp.eng.BuildingPayload() } -// SetUnsafeHead changes the forkchoice state unsafe head, without changing the engine. -// -// deprecated: use the EngineControl interface instead. -func (dp *DerivationPipeline) SetUnsafeHead(head eth.L2BlockRef) { - dp.eng.SetUnsafeHead(head) -} - // AddUnsafePayload schedules an execution payload to be processed, ahead of deriving it from L1 func (dp *DerivationPipeline) AddUnsafePayload(payload *eth.ExecutionPayload) { dp.eng.AddUnsafePayload(payload) diff --git a/op-node/rollup/driver/driver.go b/op-node/rollup/driver/driver.go index e67b91026..e6fdf7cd9 100644 --- a/op-node/rollup/driver/driver.go +++ b/op-node/rollup/driver/driver.go @@ -14,7 +14,6 @@ import ( type Metrics interface { RecordPipelineReset() - RecordSequencingError() RecordPublishingError() RecordDerivationError() @@ -28,9 +27,8 @@ type Metrics interface { SetDerivationIdle(idle bool) RecordL1ReorgDepth(d uint64) - CountSequencedTxs(count int) - SequencerMetrics + EngineMetrics } type L1Chain interface { @@ -48,7 +46,6 @@ type L2Chain interface { type DerivationPipeline interface { Reset() Step(ctx context.Context) error - SetUnsafeHead(head eth.L2BlockRef) AddUnsafePayload(payload *eth.ExecutionPayload) Finalize(ref eth.L1BlockRef) FinalizedL1() eth.L1BlockRef @@ -68,14 +65,12 @@ type L1StateIface interface { L1Finalized() eth.L1BlockRef } -type L1OriginSelectorIface interface { - FindL1Origin(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) -} - type SequencerIface interface { - StartBuildingBlock(ctx context.Context, l1Origin eth.L1BlockRef) error + StartBuildingBlock(ctx context.Context, l1Head eth.L1BlockRef) error CompleteBuildingBlock(ctx context.Context) (*eth.ExecutionPayload, error) - PlanNextSequencerAction(sequenceErr error) (delay time.Duration, seal bool, onto eth.BlockID) + PlanNextSequencerAction() time.Duration + RunNextSequencerAction(ctx context.Context, l1Head eth.L1BlockRef) *eth.ExecutionPayload + BuildingOnto() eth.L2BlockRef } type Network interface { @@ -90,7 +85,10 @@ func NewDriver(driverCfg *Config, cfg *rollup.Config, l2 L2Chain, l1 L1Chain, ne verifConfDepth := NewConfDepth(driverCfg.VerifierConfDepth, l1State.L1Head, l1) derivationPipeline := derive.NewDerivationPipeline(log, cfg, verifConfDepth, l2, metrics) attrBuilder := derive.NewFetchingAttributesBuilder(cfg, l1, l2) - sequencer := NewSequencer(log, cfg, l2, derivationPipeline, attrBuilder, metrics) + engine := derivationPipeline + meteredEngine := NewMeteredEngine(cfg, engine, metrics, log) + sequencer := NewSequencer(log, cfg, meteredEngine, attrBuilder, findL1Origin) + return &Driver{ l1State: l1State, derivation: derivationPipeline, @@ -106,7 +104,6 @@ func NewDriver(driverCfg *Config, cfg *rollup.Config, l2 L2Chain, l1 L1Chain, ne snapshotLog: snapshotLog, l1: l1, l2: l2, - l1OriginSelector: findL1Origin, sequencer: sequencer, network: network, metrics: metrics, diff --git a/op-node/rollup/driver/sequencer.go b/op-node/rollup/driver/sequencer.go index 0c91f52b8..f2b2ab527 100644 --- a/op-node/rollup/driver/sequencer.go +++ b/op-node/rollup/driver/sequencer.go @@ -2,7 +2,6 @@ package driver import ( "context" - "errors" "fmt" "time" @@ -20,16 +19,8 @@ type Downloader interface { FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error) } -type SequencerMetrics interface { - RecordSequencerBuildingDiffTime(duration time.Duration) - RecordSequencerSealingTime(duration time.Duration) -} - -type EngineState interface { - Finalized() eth.L2BlockRef - UnsafeL2Head() eth.L2BlockRef - SafeL2Head() eth.L2BlockRef - Origin() eth.L1BlockRef +type L1OriginSelectorIface interface { + FindL1Origin(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) } // Sequencer implements the sequencing interface of the driver: it starts and completes block building jobs. @@ -37,40 +28,44 @@ type Sequencer struct { log log.Logger config *rollup.Config - l2 derive.Engine - engineState EngineState + engine derive.EngineControl - attrBuilder derive.AttributesBuilder - buildingOnto eth.L2BlockRef - buildingID eth.PayloadID - buildingStartTime time.Time + attrBuilder derive.AttributesBuilder + l1OriginSelector L1OriginSelectorIface - metrics SequencerMetrics + // timeNow enables sequencer testing to mock the time + timeNow func() time.Time + + nextAction time.Time } -func NewSequencer(log log.Logger, cfg *rollup.Config, l2 derive.Engine, engineState EngineState, attributesBuilder derive.AttributesBuilder, metrics SequencerMetrics) *Sequencer { +func NewSequencer(log log.Logger, cfg *rollup.Config, engine derive.EngineControl, attributesBuilder derive.AttributesBuilder, l1OriginSelector L1OriginSelectorIface) *Sequencer { return &Sequencer{ - log: log, - config: cfg, - l2: l2, - metrics: metrics, - engineState: engineState, - attrBuilder: attributesBuilder, + log: log, + config: cfg, + engine: engine, + timeNow: time.Now, + attrBuilder: attributesBuilder, + l1OriginSelector: l1OriginSelector, } } // StartBuildingBlock initiates a block building job on top of the given L2 head, safe and finalized blocks, and using the provided l1Origin. -func (d *Sequencer) StartBuildingBlock(ctx context.Context, l1Origin eth.L1BlockRef) error { - l2Head := d.engineState.UnsafeL2Head() +func (d *Sequencer) StartBuildingBlock(ctx context.Context, l1Head eth.L1BlockRef) error { + l2Head := d.engine.UnsafeL2Head() + + // Figure out which L1 origin block we're going to be building on top of. + l1Origin, err := d.l1OriginSelector.FindL1Origin(ctx, l1Head, l2Head) + if err != nil { + d.log.Error("Error finding next L1 Origin", "err", err) + return err + } + if !(l2Head.L1Origin.Hash == l1Origin.ParentHash || l2Head.L1Origin.Hash == l1Origin.Hash) { return fmt.Errorf("cannot build new L2 block with L1 origin %s (parent L1 %s) on current L2 head %s with L1 origin %s", l1Origin, l1Origin.ParentHash, l2Head, l2Head.L1Origin) } d.log.Info("creating new block", "parent", l2Head, "l1Origin", l1Origin) - if d.buildingID != (eth.PayloadID{}) { // This may happen when we decide to build a different block in response to a reorg. Or when previous block building failed. - d.log.Warn("did not finish previous block building, starting new building now", "prev_onto", d.buildingOnto, "prev_payload_id", d.buildingID, "new_onto", l2Head) - } - d.buildingStartTime = time.Now() fetchCtx, cancel := context.WithTimeout(ctx, time.Second*20) defer cancel() @@ -90,20 +85,11 @@ func (d *Sequencer) StartBuildingBlock(ctx context.Context, l1Origin eth.L1Block "num", l2Head.Number+1, "time", uint64(attrs.Timestamp), "origin", l1Origin, "origin_time", l1Origin.Time, "noTxPool", attrs.NoTxPool) - // And construct our fork choice state. This is our current fork choice state and will be - // updated as a result of executing the block based on the attributes described above. - fc := eth.ForkchoiceState{ - HeadBlockHash: l2Head.Hash, - SafeBlockHash: d.engineState.SafeL2Head().Hash, - FinalizedBlockHash: d.engineState.Finalized().Hash, - } // Start a payload building process. - id, errTyp, err := derive.StartPayload(ctx, d.l2, fc, attrs) + errTyp, err := d.engine.StartPayload(ctx, l2Head, attrs, false) if err != nil { return fmt.Errorf("failed to start building on top of L2 chain %s, error (%d): %w", l2Head, errTyp, err) } - d.buildingOnto = l2Head - d.buildingID = id return nil } @@ -111,75 +97,90 @@ func (d *Sequencer) StartBuildingBlock(ctx context.Context, l1Origin eth.L1Block // Warning: the safe and finalized L2 blocks as viewed during the initiation of the block building are reused for completion of the block building. // The Execution engine should not change the safe and finalized blocks between start and completion of block building. func (d *Sequencer) CompleteBuildingBlock(ctx context.Context) (*eth.ExecutionPayload, error) { - if d.buildingID == (eth.PayloadID{}) { - return nil, fmt.Errorf("cannot complete payload building: not currently building a payload") - } - sealingStart := time.Now() - - l2Head := d.engineState.UnsafeL2Head() - if d.buildingOnto.Hash != l2Head.Hash { - return nil, fmt.Errorf("engine reorged from %s to %s while building block", d.buildingOnto, l2Head) - } - fc := eth.ForkchoiceState{ - HeadBlockHash: l2Head.Hash, - SafeBlockHash: d.engineState.SafeL2Head().Hash, - FinalizedBlockHash: d.engineState.Finalized().Hash, - } - - // Actually execute the block and add it to the head of the chain. - payload, errTyp, err := derive.ConfirmPayload(ctx, d.log, d.l2, fc, d.buildingID, false) + payload, errTyp, err := d.engine.ConfirmPayload(ctx) if err != nil { - return nil, fmt.Errorf("failed to complete building on top of L2 chain %s, id: %s, error (%d): %w", d.buildingOnto, d.buildingID, errTyp, err) + return nil, fmt.Errorf("failed to complete building block: error (%d): %w", errTyp, err) } - now := time.Now() - sealTime := now.Sub(sealingStart) - buildTime := now.Sub(d.buildingStartTime) - d.metrics.RecordSequencerSealingTime(sealTime) - d.metrics.RecordSequencerBuildingDiffTime(buildTime - time.Duration(d.config.BlockTime)*time.Second) - d.log.Debug("sequenced block", "seal_time", sealTime, "build_time", buildTime) - d.buildingID = eth.PayloadID{} return payload, nil } -// PlanNextSequencerAction returns a desired delay till the next action, and if we should seal the block: -// - true whenever we need to complete a block -// - false whenever we need to start a block -func (d *Sequencer) PlanNextSequencerAction(sequenceErr error) (delay time.Duration, seal bool, onto eth.BlockID) { - blockTime := time.Duration(d.config.BlockTime) * time.Second - head := d.engineState.UnsafeL2Head() +// CancelBuildingBlock cancels the current open block building job. +// This sequencer only maintains one block building job at a time. +func (d *Sequencer) CancelBuildingBlock(ctx context.Context) { + // force-cancel, we can always continue block building, and any error is logged by the engine state + _ = d.engine.CancelPayload(ctx, true) +} - // based on the build error, delay and start over again - if sequenceErr != nil { - if errors.Is(sequenceErr, UninitializedL1StateErr) { - // temporary errors are not so bad, just retry in 500ms - return 500 * time.Millisecond, false, head.ID() - } else { - // we just hit an unknown type of error, delay a re-attempt by as much as a block - return blockTime, false, head.ID() - } +// PlanNextSequencerAction returns a desired delay till the RunNextSequencerAction call. +func (d *Sequencer) PlanNextSequencerAction() time.Duration { + head := d.engine.UnsafeL2Head() + now := d.timeNow() + + buildingOnto, buildingID, _ := d.engine.BuildingPayload() + + // We may have to wait till the next sequencing action, e.g. upon an error. + // If the head changed we need to respond and will not delay the sequencing. + if delay := d.nextAction.Sub(now); delay > 0 && buildingOnto.Hash == head.Hash { + return delay } + blockTime := time.Duration(d.config.BlockTime) * time.Second payloadTime := time.Unix(int64(head.Time+d.config.BlockTime), 0) - remainingTime := time.Until(payloadTime) + remainingTime := payloadTime.Sub(now) // If we started building a block already, and if that work is still consistent, // then we would like to finish it by sealing the block. - if d.buildingID != (eth.PayloadID{}) && d.buildingOnto.Hash == head.Hash { + if buildingID != (eth.PayloadID{}) && buildingOnto.Hash == head.Hash { // if we started building already, then we will schedule the sealing. if remainingTime < sealingDuration { - return 0, true, head.ID() // if there's not enough time for sealing, don't wait. + return 0 // if there's not enough time for sealing, don't wait. } else { // finish with margin of sealing duration before payloadTime - return remainingTime - sealingDuration, true, head.ID() + return remainingTime - sealingDuration } } else { // if we did not yet start building, then we will schedule the start. if remainingTime > blockTime { // if we have too much time, then wait before starting the build - return remainingTime - blockTime, false, head.ID() + return remainingTime - blockTime } else { // otherwise start instantly - return 0, false, head.ID() + return 0 + } + } +} + +// BuildingOnto returns the L2 head reference that the latest block is or was being built on top of. +func (d *Sequencer) BuildingOnto() eth.L2BlockRef { + ref, _, _ := d.engine.BuildingPayload() + return ref +} + +// RunNextSequencerAction starts new block building work, or seals existing work, +// and is best timed by first awaiting the delay returned by PlanNextSequencerAction. +// If a new block is successfully sealed, it will be returned for publishing, nil otherwise. +func (d *Sequencer) RunNextSequencerAction(ctx context.Context, l1Head eth.L1BlockRef) *eth.ExecutionPayload { + if _, buildingID, _ := d.engine.BuildingPayload(); buildingID != (eth.PayloadID{}) { + payload, err := d.CompleteBuildingBlock(ctx) + if err != nil { + d.log.Error("sequencer failed to seal new block", "err", err) + d.nextAction = d.timeNow().Add(time.Second) + if buildingID != (eth.PayloadID{}) { // don't keep stale block building jobs around, try to cancel them + d.CancelBuildingBlock(ctx) + } + return nil + } else { + d.log.Info("sequencer successfully built a new block", "block", payload.ID(), "time", uint64(payload.Timestamp), "txs", len(payload.Transactions)) + return payload + } + } else { + err := d.StartBuildingBlock(ctx, l1Head) + if err != nil { + d.log.Error("sequencer failed to start building new block", "err", err) + d.nextAction = d.timeNow().Add(time.Second) + } else { + d.log.Info("sequencer started building new block", "payload_id", buildingID) } + return nil } } diff --git a/op-node/rollup/driver/sequencer_test.go b/op-node/rollup/driver/sequencer_test.go new file mode 100644 index 000000000..588cf5b88 --- /dev/null +++ b/op-node/rollup/driver/sequencer_test.go @@ -0,0 +1,368 @@ +package driver + +import ( + "context" + crand "crypto/rand" + "encoding/binary" + "errors" + "fmt" + "math/big" + "math/rand" + "testing" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/stretchr/testify/require" + + "github.com/ethereum-optimism/optimism/op-node/eth" + "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-node/rollup/derive" + "github.com/ethereum-optimism/optimism/op-node/testlog" + "github.com/ethereum-optimism/optimism/op-node/testutils" +) + +type FakeEngineControl struct { + finalized eth.L2BlockRef + safe eth.L2BlockRef + unsafe eth.L2BlockRef + + buildingOnto eth.L2BlockRef + buildingID eth.PayloadID + buildingSafe bool + + buildingAttrs *eth.PayloadAttributes + buildingStart time.Time + + cfg *rollup.Config + + timeNow func() time.Time + + makePayload func(onto eth.L2BlockRef, attrs *eth.PayloadAttributes) *eth.ExecutionPayload + + errTyp derive.BlockInsertionErrType + err error + + totalBuildingTime time.Duration + totalBuiltBlocks int + totalTxs int +} + +func (m *FakeEngineControl) avgBuildingTime() time.Duration { + return m.totalBuildingTime / time.Duration(m.totalBuiltBlocks) +} + +func (m *FakeEngineControl) avgTxsPerBlock() float64 { + return float64(m.totalTxs) / float64(m.totalBuiltBlocks) +} + +func (m *FakeEngineControl) StartPayload(ctx context.Context, parent eth.L2BlockRef, attrs *eth.PayloadAttributes, updateSafe bool) (errType derive.BlockInsertionErrType, err error) { + if m.err != nil { + return m.errTyp, m.err + } + m.buildingID = eth.PayloadID{} + _, _ = crand.Read(m.buildingID[:]) + m.buildingOnto = parent + m.buildingSafe = updateSafe + m.buildingAttrs = attrs + m.buildingStart = m.timeNow() + return derive.BlockInsertOK, nil +} + +func (m *FakeEngineControl) ConfirmPayload(ctx context.Context) (out *eth.ExecutionPayload, errTyp derive.BlockInsertionErrType, err error) { + if m.err != nil { + return nil, m.errTyp, m.err + } + buildTime := m.timeNow().Sub(m.buildingStart) + m.totalBuildingTime += buildTime + m.totalBuiltBlocks += 1 + payload := m.makePayload(m.buildingOnto, m.buildingAttrs) + ref, err := derive.PayloadToBlockRef(payload, &m.cfg.Genesis) + if err != nil { + panic(err) + } + m.unsafe = ref + if m.buildingSafe { + m.safe = ref + } + + m.resetBuildingState() + m.totalTxs += len(payload.Transactions) + return payload, derive.BlockInsertOK, nil +} + +func (m *FakeEngineControl) CancelPayload(ctx context.Context, force bool) error { + if force { + m.resetBuildingState() + } + return m.err +} + +func (m *FakeEngineControl) BuildingPayload() (onto eth.L2BlockRef, id eth.PayloadID, safe bool) { + return m.buildingOnto, m.buildingID, m.buildingSafe +} + +func (m *FakeEngineControl) Finalized() eth.L2BlockRef { + return m.finalized +} + +func (m *FakeEngineControl) UnsafeL2Head() eth.L2BlockRef { + return m.unsafe +} + +func (m *FakeEngineControl) SafeL2Head() eth.L2BlockRef { + return m.safe +} + +func (m *FakeEngineControl) resetBuildingState() { + m.buildingID = eth.PayloadID{} + m.buildingOnto = eth.L2BlockRef{} + m.buildingSafe = false + m.buildingAttrs = nil +} + +var _ derive.EngineControl = (*FakeEngineControl)(nil) + +type testAttrBuilderFn func(ctx context.Context, l2Parent eth.L2BlockRef, epoch eth.BlockID) (attrs *eth.PayloadAttributes, err error) + +func (fn testAttrBuilderFn) PreparePayloadAttributes(ctx context.Context, l2Parent eth.L2BlockRef, epoch eth.BlockID) (attrs *eth.PayloadAttributes, err error) { + return fn(ctx, l2Parent, epoch) +} + +var _ derive.AttributesBuilder = (testAttrBuilderFn)(nil) + +type testOriginSelectorFn func(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) + +func (fn testOriginSelectorFn) FindL1Origin(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) { + return fn(ctx, l1Head, l2Head) +} + +var _ L1OriginSelectorIface = (testOriginSelectorFn)(nil) + +// TestSequencerChaosMonkey runs the sequencer in a mocked adversarial environment with +// repeated random errors in dependencies and poor clock timing. +// At the end the health of the chain is checked to show that the sequencer kept the chain in shape. +func TestSequencerChaosMonkey(t *testing.T) { + mockL1Hash := func(num uint64) (out common.Hash) { + out[31] = 1 + binary.BigEndian.PutUint64(out[:], num) + return + } + mockL2Hash := func(num uint64) (out common.Hash) { + out[31] = 2 + binary.BigEndian.PutUint64(out[:], num) + return + } + mockL1ID := func(num uint64) eth.BlockID { + return eth.BlockID{Hash: mockL1Hash(num), Number: num} + } + mockL2ID := func(num uint64) eth.BlockID { + return eth.BlockID{Hash: mockL2Hash(num), Number: num} + } + + rng := rand.New(rand.NewSource(12345)) + + l1Time := uint64(100000) + + // mute errors. We expect a lot of the mocked errors to cause error-logs. We check chain health at the end of the test. + log := testlog.Logger(t, log.LvlCrit) + + cfg := &rollup.Config{ + Genesis: rollup.Genesis{ + L1: mockL1ID(100000), + L2: mockL2ID(200000), + L2Time: l1Time + 300, // L2 may start with a relative old L1 origin and will have to catch it up + SystemConfig: eth.SystemConfig{}, + }, + BlockTime: 2, + MaxSequencerDrift: 30, + } + // keep track of the L1 timestamps we mock because sometimes we only have the L1 hash/num handy + l1Times := map[eth.BlockID]uint64{cfg.Genesis.L1: l1Time} + + genesisL2 := eth.L2BlockRef{ + Hash: cfg.Genesis.L2.Hash, + Number: cfg.Genesis.L2.Number, + ParentHash: mockL2Hash(cfg.Genesis.L2.Number - 1), + Time: cfg.Genesis.L2Time, + L1Origin: cfg.Genesis.L1, + SequenceNumber: 0, + } + // initialize our engine state + engControl := &FakeEngineControl{ + finalized: genesisL2, + safe: genesisL2, + unsafe: genesisL2, + cfg: cfg, + } + + // start wallclock at 5 minutes after the current L2 head. The sequencer has some catching up to do! + clockTime := time.Unix(int64(engControl.unsafe.Time)+5*60, 0) + clockFn := func() time.Time { + return clockTime + } + engControl.timeNow = clockFn + + // mock payload building, we don't need to process any real txs. + engControl.makePayload = func(onto eth.L2BlockRef, attrs *eth.PayloadAttributes) *eth.ExecutionPayload { + txs := make([]eth.Data, 0) + txs = append(txs, attrs.Transactions...) // include deposits + if !attrs.NoTxPool { // if we are allowed to sequence from tx pool, mock some txs + n := rng.Intn(20) + for i := 0; i < n; i++ { + txs = append(txs, []byte(fmt.Sprintf("mock sequenced tx %d", i))) + } + } + return ð.ExecutionPayload{ + ParentHash: onto.Hash, + BlockNumber: eth.Uint64Quantity(onto.Number) + 1, + Timestamp: attrs.Timestamp, + BlockHash: mockL2Hash(onto.Number), + Transactions: txs, + } + } + + // We keep attribute building simple, we don't talk to a real execution engine in this test. + // Sometimes we fake an error in the attributes preparation. + var attrsErr error + attrBuilder := testAttrBuilderFn(func(ctx context.Context, l2Parent eth.L2BlockRef, epoch eth.BlockID) (attrs *eth.PayloadAttributes, err error) { + if attrsErr != nil { + return nil, attrsErr + } + seqNr := l2Parent.SequenceNumber + 1 + if epoch != l2Parent.L1Origin { + seqNr = 0 + } + l1Info := &testutils.MockBlockInfo{ + InfoHash: epoch.Hash, + InfoParentHash: mockL1Hash(epoch.Number - 1), + InfoCoinbase: common.Address{}, + InfoRoot: common.Hash{}, + InfoNum: epoch.Number, + InfoTime: l1Times[epoch], + InfoMixDigest: [32]byte{}, + InfoBaseFee: big.NewInt(1234), + InfoReceiptRoot: common.Hash{}, + } + infoDep, err := derive.L1InfoDepositBytes(seqNr, l1Info, cfg.Genesis.SystemConfig) + require.NoError(t, err) + + testGasLimit := eth.Uint64Quantity(10_000_000) + return ð.PayloadAttributes{ + Timestamp: eth.Uint64Quantity(l2Parent.Time + cfg.BlockTime), + PrevRandao: eth.Bytes32{}, + SuggestedFeeRecipient: common.Address{}, + Transactions: []eth.Data{infoDep}, + NoTxPool: false, + GasLimit: &testGasLimit, + }, nil + }) + + maxL1BlockTimeGap := uint64(100) + // The origin selector just generates random L1 blocks based on RNG + var originErr error + originSelector := testOriginSelectorFn(func(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) { + if originErr != nil { + return eth.L1BlockRef{}, originErr + } + origin := eth.L1BlockRef{ + Hash: mockL1Hash(l2Head.L1Origin.Number), + Number: l2Head.L1Origin.Number, + ParentHash: mockL1Hash(l2Head.L1Origin.Number), + Time: l1Times[l2Head.L1Origin], + } + // randomly make a L1 origin appear, if we can even select it + nextL2Time := l2Head.Time + cfg.BlockTime + if nextL2Time <= origin.Time { + return origin, nil + } + maxTimeIncrement := nextL2Time - origin.Time + if maxTimeIncrement > maxL1BlockTimeGap { + maxTimeIncrement = maxL1BlockTimeGap + } + if rng.Intn(10) == 0 { + nextOrigin := eth.L1BlockRef{ + Hash: mockL1Hash(origin.Number + 1), + Number: origin.Number + 1, + ParentHash: origin.Hash, + Time: origin.Time + 1 + uint64(rng.Int63n(int64(maxTimeIncrement))), + } + l1Times[nextOrigin.ID()] = nextOrigin.Time + return nextOrigin, nil + } else { + return origin, nil + } + }) + + seq := NewSequencer(log, cfg, engControl, attrBuilder, originSelector) + seq.timeNow = clockFn + + l1Head := eth.L1BlockRef{} // TODO this is getting removed + + // try to build 1000 blocks, with 5x as many planning attempts, to handle errors and clock problems + desiredBlocks := 1000 + for i := 0; i < 5*desiredBlocks && engControl.totalBuiltBlocks < desiredBlocks; i++ { + delta := seq.PlanNextSequencerAction() + + x := rng.Float32() + if x < 0.01 { // 1%: mess a lot with the clock: simulate a hang of up to 30 seconds + if i < desiredBlocks/2 { // only in first 50% of blocks to let it heal, hangs take time + delta = time.Duration(rng.Float64() * float64(time.Second*30)) + } + } else if x < 0.1 { // 9%: mess with the timing, -50% to 50% off + delta = time.Duration((0.5 + rng.Float64()) * float64(delta)) + } else if x < 0.5 { + // 40%: mess slightly with the timing, -10% to 10% off + delta = time.Duration((0.9 + rng.Float64()*0.2) * float64(delta)) + } + clockTime = clockTime.Add(delta) + + // reset errors + originErr = nil + attrsErr = nil + engControl.err = nil + engControl.errTyp = derive.BlockInsertOK + + // maybe make something maybe fail, or try a new L1 origin + switch rng.Intn(10) { // 40% chance to fail sequencer action (!!!) + case 0: + originErr = errors.New("mock origin error") + case 1: + attrsErr = errors.New("mock attributes error") + case 2: + engControl.err = errors.New("mock temporary engine error") + engControl.errTyp = derive.BlockInsertTemporaryErr + case 3: + engControl.err = errors.New("mock prestate engine error") + engControl.errTyp = derive.BlockInsertPrestateErr + default: + // no error + } + payload := seq.RunNextSequencerAction(context.Background(), l1Head) + if payload != nil { + require.Equal(t, engControl.UnsafeL2Head().ID(), payload.ID(), "head must stay in sync with emitted payloads") + var tx types.Transaction + require.NoError(t, tx.UnmarshalBinary(payload.Transactions[0])) + info, err := derive.L1InfoDepositTxData(tx.Data()) + require.NoError(t, err) + require.GreaterOrEqual(t, uint64(payload.Timestamp), info.Time, "ensure L2 time >= L1 time") + } + } + + // Now, even though: + // - the start state was behind the wallclock + // - the L1 origin was far behind the L2 + // - we made all components fail at random + // - messed with the clock + // the L2 chain was still built and stats are healthy on average! + l2Head := engControl.UnsafeL2Head() + t.Logf("avg build time: %s, clock timestamp: %d, L2 head time: %d, L1 origin time: %d, avg txs per block: %f", engControl.avgBuildingTime(), clockFn().Unix(), l2Head.Time, l1Times[l2Head.L1Origin], engControl.avgTxsPerBlock()) + require.Equal(t, engControl.totalBuiltBlocks, desiredBlocks, "persist through random errors and build the desired blocks") + require.Equal(t, l2Head.Time, cfg.Genesis.L2Time+uint64(desiredBlocks)*cfg.BlockTime, "reached desired L2 block timestamp") + require.GreaterOrEqual(t, l2Head.Time, l1Times[l2Head.L1Origin], "the L2 time >= the L1 time") + require.Less(t, l2Head.Time-l1Times[l2Head.L1Origin], uint64(100), "The L1 origin time is close to the L2 time") + require.Less(t, clockTime.Sub(time.Unix(int64(l2Head.Time), 0)).Abs(), 2*time.Second, "L2 time is accurate, within 2 seconds of wallclock") + require.Greater(t, engControl.avgBuildingTime(), time.Second, "With 2 second block time and 1 second error backoff and healthy-on-average errors, building time should at least be a second") + require.Greater(t, engControl.avgTxsPerBlock(), 3.0, "We expect at least 1 system tx per block, but with a mocked 0-10 txs we expect an higher avg") +} diff --git a/op-node/rollup/driver/state.go b/op-node/rollup/driver/state.go index c9ef4dd12..18bbca048 100644 --- a/op-node/rollup/driver/state.go +++ b/op-node/rollup/driver/state.go @@ -25,8 +25,6 @@ type SyncStatus = eth.SyncStatus // sealingDuration defines the expected time it takes to seal the block const sealingDuration = time.Millisecond * 50 -var UninitializedL1StateErr = errors.New("the L1 Head in L1 State is not initialized yet") - type Driver struct { l1State L1StateIface @@ -71,11 +69,10 @@ type Driver struct { // L2 Signals: unsafeL2Payloads chan *eth.ExecutionPayload - l1 L1Chain - l2 L2Chain - l1OriginSelector L1OriginSelectorIface - sequencer SequencerIface - network Network // may be nil, network for is optional + l1 L1Chain + l2 L2Chain + sequencer SequencerIface + network Network // may be nil, network for is optional metrics Metrics log log.Logger @@ -142,75 +139,6 @@ func (s *Driver) OnUnsafeL2Payload(ctx context.Context, payload *eth.ExecutionPa } } -// startNewL2Block starts sequencing a new L2 block on top of the unsafe L2 Head. -func (s *Driver) startNewL2Block(ctx context.Context) error { - l2Head := s.derivation.UnsafeL2Head() - - l1Head := s.l1State.L1Head() - if l1Head == (eth.L1BlockRef{}) { - return UninitializedL1StateErr - } - - // Figure out which L1 origin block we're going to be building on top of. - l1Origin, err := s.l1OriginSelector.FindL1Origin(ctx, l1Head, l2Head) - if err != nil { - s.log.Error("Error finding next L1 Origin", "err", err) - return err - } - - // Rollup is configured to not start producing blocks until a specific L1 block has been - // reached. Don't produce any blocks until we're at that genesis block. - if l1Origin.Number < s.config.Genesis.L1.Number { - s.log.Info("Skipping block production because the next L1 Origin is behind the L1 genesis", "next", l1Origin.ID(), "genesis", s.config.Genesis.L1) - return fmt.Errorf("the L1 origin %s cannot be before genesis at %s", l1Origin, s.config.Genesis.L1) - } - - // Should never happen. Sequencer will halt if we get into this situation somehow. - nextL2Time := l2Head.Time + s.config.BlockTime - if nextL2Time < l1Origin.Time { - s.log.Error("Cannot build L2 block for time before L1 origin", - "l2Unsafe", l2Head, "nextL2Time", nextL2Time, "l1Origin", l1Origin, "l1OriginTime", l1Origin.Time) - return fmt.Errorf("cannot build L2 block on top %s for time %d before L1 origin %s at time %d", - l2Head, nextL2Time, l1Origin, l1Origin.Time) - } - - // Start creating the new block. - return s.sequencer.StartBuildingBlock(ctx, l1Origin) -} - -// completeNewBlock completes a previously started L2 block sequencing job. -func (s *Driver) completeNewBlock(ctx context.Context) error { - payload, err := s.sequencer.CompleteBuildingBlock(ctx) - if err != nil { - s.metrics.RecordSequencingError() - s.log.Error("Failed to seal block as sequencer", "err", err) - return err - } - - // Generate an L2 block ref from the payload. - newUnsafeL2Head, err := derive.PayloadToBlockRef(payload, &s.config.Genesis) - if err != nil { - s.metrics.RecordSequencingError() - s.log.Error("Sequenced payload cannot be transformed into valid L2 block reference", "err", err) - return fmt.Errorf("sequenced payload cannot be transformed into valid L2 block reference: %w", err) - } - - // Update our L2 head block based on the new unsafe block we just generated. - s.derivation.SetUnsafeHead(newUnsafeL2Head) - - s.log.Info("Sequenced new l2 block", "l2_unsafe", newUnsafeL2Head, "l1_origin", newUnsafeL2Head.L1Origin, "txs", len(payload.Transactions), "time", newUnsafeL2Head.Time) - s.metrics.CountSequencedTxs(len(payload.Transactions)) - - if s.network != nil { - if err := s.network.PublishL2Payload(ctx, payload); err != nil { - s.log.Warn("failed to publish newly created block", "id", payload.ID(), "err", err) - s.metrics.RecordPublishingError() - // publishing of unsafe data via p2p is optional. Errors are not severe enough to change/halt sequencing but should be logged and metered. - } - } - return nil -} - // the eventLoop responds to L1 changes and internal timers to produce L2 blocks. func (s *Driver) eventLoop() { defer s.wg.Done() @@ -259,34 +187,23 @@ func (s *Driver) eventLoop() { // L1 chain that we need to handle. reqStep() - blockTime := time.Duration(s.config.BlockTime) * time.Second - - var sequenceErr error - var sequenceErrTime time.Time sequencerTimer := time.NewTimer(0) var sequencerCh <-chan time.Time - var sequencingPlannedOnto eth.BlockID - var sequencerSealNext bool planSequencerAction := func() { - delay, seal, onto := s.sequencer.PlanNextSequencerAction(sequenceErr) - if sequenceErr != nil && time.Since(sequenceErrTime) > delay { - sequenceErr = nil - } + delay := s.sequencer.PlanNextSequencerAction() sequencerCh = sequencerTimer.C if len(sequencerCh) > 0 { // empty if not already drained before resetting <-sequencerCh } sequencerTimer.Reset(delay) - sequencingPlannedOnto = onto - sequencerSealNext = seal } for { - // If we are sequencing, update the trigger for the next sequencer action. + // If we are sequencing, and the L1 state is ready, update the trigger for the next sequencer action. // This may adjust at any time based on fork-choice changes or previous errors. - if s.driverConfig.SequencerEnabled && !s.driverConfig.SequencerStopped { + if s.driverConfig.SequencerEnabled && !s.driverConfig.SequencerStopped && s.l1State.L1Head() != (eth.L1BlockRef{}) { // update sequencer time if the head changed - if sequencingPlannedOnto != s.derivation.UnsafeL2Head().ID() { + if s.sequencer.BuildingOnto().ID() != s.derivation.UnsafeL2Head().ID() { planSequencerAction() } } else { @@ -295,22 +212,14 @@ func (s *Driver) eventLoop() { select { case <-sequencerCh: - s.log.Info("sequencing now!", "seal", sequencerSealNext, "idle_derivation", s.idleDerivation) - if sequencerSealNext { - // try to seal the current block task, and allow it to take up to 3 block times. - // If this fails we will simply start a new block building job. - ctx, cancel := context.WithTimeout(ctx, 3*blockTime) - sequenceErr = s.completeNewBlock(ctx) - cancel() - } else { - // Start the block building, don't allow the starting of sequencing to get stuck for more the time of 1 block. - ctx, cancel := context.WithTimeout(ctx, blockTime) - sequenceErr = s.startNewL2Block(ctx) - cancel() - } - if sequenceErr != nil { - s.log.Error("sequencing error", "err", sequenceErr) - sequenceErrTime = time.Now() + payload := s.sequencer.RunNextSequencerAction(ctx, s.l1State.L1Head()) + if s.network != nil && payload != nil { + // Publishing of unsafe data via p2p is optional. + // Errors are not severe enough to change/halt sequencing but should be logged and metered. + if err := s.network.PublishL2Payload(ctx, payload); err != nil { + s.log.Warn("failed to publish newly created block", "id", payload.ID(), "err", err) + s.metrics.RecordPublishingError() + } } planSequencerAction() // schedule the next sequencer action to keep the sequencing looping case payload := <-s.unsafeL2Payloads: @@ -386,8 +295,8 @@ func (s *Driver) eventLoop() { } else { s.log.Info("Sequencer has been started") s.driverConfig.SequencerStopped = false - sequencingPlannedOnto = eth.BlockID{} close(resp.err) + planSequencerAction() // resume sequencing } case respCh := <-s.stopSequencer: if s.driverConfig.SequencerStopped { diff --git a/op-node/rollup/driver/state_tob_test.go b/op-node/rollup/driver/state_tob_test.go deleted file mode 100644 index 251141bc2..000000000 --- a/op-node/rollup/driver/state_tob_test.go +++ /dev/null @@ -1,279 +0,0 @@ -// On develop -package driver - -import ( - "context" - "errors" - "math/big" - "math/rand" - "testing" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/log" - "github.com/stretchr/testify/require" - - "github.com/ethereum-optimism/optimism/op-node/eth" - "github.com/ethereum-optimism/optimism/op-node/metrics" - "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum-optimism/optimism/op-node/rollup/derive" - "github.com/ethereum-optimism/optimism/op-node/testutils" -) - -type TestDummyOutputImpl struct { - willError bool - - cfg *rollup.Config - - l1Origin eth.L1BlockRef - l2Head eth.L2BlockRef -} - -func (d *TestDummyOutputImpl) PlanNextSequencerAction(sequenceErr error) (delay time.Duration, seal bool, onto eth.BlockID) { - return 0, d.l1Origin != (eth.L1BlockRef{}), d.l2Head.ParentID() -} - -func (d *TestDummyOutputImpl) StartBuildingBlock(ctx context.Context, l1Origin eth.L1BlockRef) error { - d.l1Origin = l1Origin - return nil -} - -func (d *TestDummyOutputImpl) CompleteBuildingBlock(ctx context.Context) (*eth.ExecutionPayload, error) { - // If we're meant to error, return one - if d.willError { - return nil, errors.New("the TestDummyOutputImpl.createNewBlock operation failed") - } - info := &testutils.MockBlockInfo{ - InfoHash: d.l1Origin.Hash, - InfoParentHash: d.l1Origin.ParentHash, - InfoCoinbase: common.Address{}, - InfoRoot: common.Hash{}, - InfoNum: d.l1Origin.Number, - InfoTime: d.l1Origin.Time, - InfoMixDigest: [32]byte{}, - InfoBaseFee: big.NewInt(123), - InfoReceiptRoot: common.Hash{}, - } - infoTx, err := derive.L1InfoDepositBytes(d.l2Head.SequenceNumber, info, eth.SystemConfig{}) - if err != nil { - panic(err) - } - payload := eth.ExecutionPayload{ - ParentHash: d.l2Head.Hash, - FeeRecipient: common.Address{}, - StateRoot: eth.Bytes32{}, - ReceiptsRoot: eth.Bytes32{}, - LogsBloom: eth.Bytes256{}, - PrevRandao: eth.Bytes32{}, - BlockNumber: eth.Uint64Quantity(d.l2Head.Number + 1), - GasLimit: 0, - GasUsed: 0, - Timestamp: eth.Uint64Quantity(d.l2Head.Time + d.cfg.BlockTime), - ExtraData: nil, - BaseFeePerGas: eth.Uint256Quantity{}, - BlockHash: common.Hash{123}, - Transactions: []eth.Data{infoTx}, - } - return &payload, nil -} - -var _ SequencerIface = (*TestDummyOutputImpl)(nil) - -type TestDummyDerivationPipeline struct { - DerivationPipeline - l2Head eth.L2BlockRef - l2SafeHead eth.L2BlockRef - l2Finalized eth.L2BlockRef -} - -func (d TestDummyDerivationPipeline) Reset() {} -func (d TestDummyDerivationPipeline) Step(ctx context.Context) error { return nil } -func (d TestDummyDerivationPipeline) SetUnsafeHead(head eth.L2BlockRef) {} -func (d TestDummyDerivationPipeline) AddUnsafePayload(payload *eth.ExecutionPayload) {} -func (d TestDummyDerivationPipeline) Finalized() eth.L2BlockRef { return d.l2Head } -func (d TestDummyDerivationPipeline) SafeL2Head() eth.L2BlockRef { return d.l2SafeHead } -func (d TestDummyDerivationPipeline) UnsafeL2Head() eth.L2BlockRef { return d.l2Finalized } - -type TestDummyL1OriginSelector struct { - retval eth.L1BlockRef -} - -func (l TestDummyL1OriginSelector) FindL1Origin(ctx context.Context, l1Head eth.L1BlockRef, l2Head eth.L2BlockRef) (eth.L1BlockRef, error) { - return l.retval, nil -} - -// TestRejectCreateBlockBadTimestamp tests that a block creation with invalid timestamps will be caught. -// This does not test: -// - The findL1Origin call (it is hardcoded to be the head) -// - The outputInterface used to create a new block from a given payload. -// - The DerivationPipeline setting unsafe head (a mock provider is used to pretend to set it) -// - Metrics (only mocked enough to let the method proceed) -// - Publishing (network is set to nil so publishing won't occur) -func TestRejectCreateBlockBadTimestamp(t *testing.T) { - // Create our random provider - rng := rand.New(rand.NewSource(rand.Int63())) - - // Create our context for methods to execute under - ctx := context.Background() - - // Create our fake L1/L2 heads and link them accordingly - l1HeadRef := testutils.RandomBlockRef(rng) - l2HeadRef := testutils.RandomL2BlockRef(rng) - l2l1OriginBlock := l1HeadRef - l2HeadRef.L1Origin = l2l1OriginBlock.ID() - - // Create a rollup config - cfg := rollup.Config{ - BlockTime: uint64(60), - Genesis: rollup.Genesis{ - L1: l1HeadRef.ID(), - L2: l2HeadRef.ID(), - L2Time: 0x7000, // dummy value - }, - } - - // Patch our timestamp so we fail - l2HeadRef.Time = l2l1OriginBlock.Time - (cfg.BlockTime * 2) - - // Create our outputter - outputProvider := &TestDummyOutputImpl{cfg: &cfg, l2Head: l2HeadRef, willError: false} - - // Create our state - s := Driver{ - l1State: &L1State{ - l1Head: l1HeadRef, - log: log.New(), - metrics: metrics.NoopMetrics, - }, - log: log.New(), - l1OriginSelector: TestDummyL1OriginSelector{retval: l1HeadRef}, - config: &cfg, - sequencer: outputProvider, - derivation: TestDummyDerivationPipeline{}, - metrics: metrics.NoopMetrics, - } - - // Create a new block - // - L2Head's L1Origin, its timestamp should be greater than L1 genesis. - // - L2Head timestamp + BlockTime should be greater than or equal to the L1 Time. - err := s.startNewL2Block(ctx) - if err == nil { - err = s.completeNewBlock(ctx) - } - - // Verify the L1Origin's block number is greater than L1 genesis in our config. - if l2l1OriginBlock.Number < s.config.Genesis.L1.Number { - require.NoError(t, err, "L1Origin block number should be greater than the L1 genesis block number") - } - - // Verify the new L2 block to create will have a time stamp equal or newer than our L1 origin block we derive from. - if l2HeadRef.Time+cfg.BlockTime < l2l1OriginBlock.Time { - // If not, we expect a specific error. - // TODO: This isn't the cleanest, we should construct + compare the whole error message. - require.NotNil(t, err) - require.Contains(t, err.Error(), "cannot build L2 block on top") - require.Contains(t, err.Error(), "for time") - require.Contains(t, err.Error(), "before L1 origin") - return - } - - // If we expected the outputter to error, capture that here - if outputProvider.willError { - require.NotNil(t, err, "outputInterface failed to createNewBlock, so createNewL2Block should also have failed") - return - } - - // Otherwise we should have no error. - require.NoError(t, err, "error raised in TestRejectCreateBlockBadTimestamp") -} - -// FuzzRejectCreateBlockBadTimestamp is a property test derived from the TestRejectCreateBlockBadTimestamp unit test. -// It fuzzes timestamps and block times to find a configuration to violate error checking. -func FuzzRejectCreateBlockBadTimestamp(f *testing.F) { - f.Fuzz(func(t *testing.T, randSeed int64, l2Time uint64, blockTime uint64, forceOutputFail bool, currentL2HeadTime uint64) { - // Create our random provider - rng := rand.New(rand.NewSource(randSeed)) - - // Create our context for methods to execute under - ctx := context.Background() - - // Create our fake L1/L2 heads and link them accordingly - l1HeadRef := testutils.RandomBlockRef(rng) - l2HeadRef := testutils.RandomL2BlockRef(rng) - l2l1OriginBlock := l1HeadRef - l2HeadRef.L1Origin = l2l1OriginBlock.ID() - - // TODO: Cap our block time so it doesn't overflow - if blockTime > 0x100000 { - blockTime = 0x100000 - } - - // Create a rollup config - cfg := rollup.Config{ - BlockTime: blockTime, - Genesis: rollup.Genesis{ - L1: l1HeadRef.ID(), - L2: l2HeadRef.ID(), - L2Time: l2Time, // dummy value - }, - } - - // Patch our timestamp so we fail - l2HeadRef.Time = currentL2HeadTime - - // Create our outputter - outputProvider := &TestDummyOutputImpl{cfg: &cfg, l2Head: l2HeadRef, willError: forceOutputFail} - - // Create our state - s := Driver{ - l1State: &L1State{ - l1Head: l1HeadRef, - log: log.New(), - metrics: metrics.NoopMetrics, - }, - log: log.New(), - l1OriginSelector: TestDummyL1OriginSelector{retval: l1HeadRef}, - config: &cfg, - sequencer: outputProvider, - derivation: TestDummyDerivationPipeline{}, - metrics: metrics.NoopMetrics, - } - - // Create a new block - // - L2Head's L1Origin, its timestamp should be greater than L1 genesis. - // - L2Head timestamp + BlockTime should be greater than or equal to the L1 Time. - err := s.startNewL2Block(ctx) - if err == nil { - err = s.completeNewBlock(ctx) - } - - // Verify the L1Origin's timestamp is greater than L1 genesis in our config. - if l2l1OriginBlock.Number < s.config.Genesis.L1.Number { - require.NoError(t, err) - return - } - - // Verify the new L2 block to create will have a time stamp equal or newer than our L1 origin block we derive from. - if l2HeadRef.Time+cfg.BlockTime < l2l1OriginBlock.Time { - // If not, we expect a specific error. - // TODO: This isn't the cleanest, we should construct + compare the whole error message. - require.NotNil(t, err) - require.Contains(t, err.Error(), "cannot build L2 block on top") - require.Contains(t, err.Error(), "for time") - require.Contains(t, err.Error(), "before L1 origin") - return - } - - // Otherwise we should have no error. - require.Nil(t, err) - - // If we expected the outputter to error, capture that here - if outputProvider.willError { - require.NotNil(t, err, "outputInterface failed to createNewBlock, so createNewL2Block should also have failed") - return - } - - // Otherwise we should have no error. - require.NoError(t, err, "L1Origin block number should be greater than the L1 genesis block number") - }) -} -- 2.23.0