Commit 18f64536 authored by Adrian Sutton's avatar Adrian Sutton Committed by GitHub

op-program: Prune L1 blocks after advancing origin (#11530)

* op-program: Retry pruning L1 blocks when advancing origin to include the parent L1 origin

Add e2e test to reproduce corner case when deploying a chain at L1 genesis.

* Trim once, after any new origin is added.

* Add unit test.

* Add action test

* Review feedback.

* Improve test assertion
parent 3e68cf01
package actions
import (
"testing"
altda "github.com/ethereum-optimism/optimism/op-alt-da"
batcherFlags "github.com/ethereum-optimism/optimism/op-batcher/flags"
"github.com/ethereum-optimism/optimism/op-e2e/e2eutils"
"github.com/ethereum-optimism/optimism/op-node/node/safedb"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive"
"github.com/ethereum-optimism/optimism/op-service/sources"
"github.com/ethereum-optimism/optimism/op-service/testlog"
"github.com/ethereum/go-ethereum/log"
"github.com/stretchr/testify/require"
)
// TestDeriveChainFromNearL1Genesis tests a corner case where when the derivation pipeline starts, the
// safe head has an L1 origin of block 1. The derivation then starts with pipeline origin of L1 genesis,
// just one block prior to the origin of the safe head.
// This is a regression test, previously the pipeline encountered got stuck in a reset loop with the error:
// buffered L1 chain epoch %s in batch queue does not match safe head origin %s
func TestDeriveChainFromNearL1Genesis(gt *testing.T) {
t := NewDefaultTesting(gt)
p := &e2eutils.TestParams{
MaxSequencerDrift: 20, // larger than L1 block time we simulate in this test (12)
SequencerWindowSize: 24,
ChannelTimeout: 20,
L1BlockTime: 12,
}
dp := e2eutils.MakeDeployParams(t, p)
// do not activate Delta hardfork for verifier
applyDeltaTimeOffset(dp, nil)
sd := e2eutils.Setup(t, dp, defaultAlloc)
logger := testlog.Logger(t, log.LevelInfo)
miner, seqEngine, sequencer := setupSequencerTest(t, sd, logger)
miner.ActEmptyBlock(t)
require.EqualValues(gt, 1, miner.l1Chain.CurrentBlock().Number.Uint64())
ref, err := derive.L2BlockToBlockRef(sequencer.rollupCfg, seqEngine.l2Chain.Genesis())
require.NoError(gt, err)
require.EqualValues(gt, 0, ref.L1Origin.Number)
sequencer.ActL1HeadSignal(t)
sequencer.ActBuildToL1Head(t)
l2BlockNum := seqEngine.l2Chain.CurrentBlock().Number.Uint64()
ref, err = derive.L2BlockToBlockRef(sequencer.rollupCfg, seqEngine.l2Chain.GetBlockByNumber(l2BlockNum))
require.NoError(gt, err)
require.EqualValues(gt, 1, ref.L1Origin.Number)
miner.ActEmptyBlock(t)
rollupSeqCl := sequencer.RollupClient()
// Force batcher to submit SingularBatches to L1.
batcher := NewL2Batcher(logger, sd.RollupCfg, &BatcherCfg{
MinL1TxSize: 0,
MaxL1TxSize: 128_000,
BatcherKey: dp.Secrets.Batcher,
DataAvailabilityType: batcherFlags.CalldataType,
}, rollupSeqCl, miner.EthClient(), seqEngine.EthClient(), seqEngine.EngineClient(t, sd.RollupCfg))
batcher.ActSubmitAll(t)
require.EqualValues(gt, l2BlockNum, batcher.l2BufferedBlock.Number)
// confirm batch on L1
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
bl := miner.l1Chain.CurrentBlock()
logger.Info("Produced L1 block with batch",
"num", miner.l1Chain.CurrentBlock().Number.Uint64(),
"txs", len(miner.l1Chain.GetBlockByHash(bl.Hash()).Transactions()))
// Process batches so safe head updates
sequencer.ActL1HeadSignal(t)
sequencer.ActL2PipelineFull(t)
require.EqualValues(gt, l2BlockNum, seqEngine.l2Chain.CurrentSafeBlock().Number.Uint64())
// Finalize L1 and process so L2 finalized updates
miner.ActL1Safe(t, miner.l1Chain.CurrentBlock().Number.Uint64())
miner.ActL1Finalize(t, miner.l1Chain.CurrentBlock().Number.Uint64())
sequencer.ActL1SafeSignal(t)
sequencer.ActL1FinalizedSignal(t)
sequencer.ActL2PipelineFull(t)
require.EqualValues(gt, l2BlockNum, seqEngine.l2Chain.CurrentFinalBlock().Number.Uint64())
// Create a new verifier using the existing engine so it already has the safe and finalized heads set.
// This is the same situation as if op-node restarted at this point.
l2Cl, err := sources.NewEngineClient(seqEngine.RPCClient(), logger, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(gt, err)
verifier := NewL2Verifier(t, logger, sequencer.l1, miner.BlobStore(), altda.Disabled, l2Cl, sequencer.rollupCfg, sequencer.syncCfg, safedb.Disabled)
verifier.ActL2PipelineFull(t) // Should not get stuck in a reset loop forever
require.EqualValues(gt, l2BlockNum, seqEngine.l2Chain.CurrentSafeBlock().Number.Uint64())
require.EqualValues(gt, l2BlockNum, seqEngine.l2Chain.CurrentFinalBlock().Number.Uint64())
syncStatus := verifier.syncStatus.SyncStatus()
require.EqualValues(gt, l2BlockNum, syncStatus.SafeL2.Number)
require.EqualValues(gt, l2BlockNum, syncStatus.FinalizedL2.Number)
}
...@@ -6,6 +6,7 @@ import ( ...@@ -6,6 +6,7 @@ import (
"slices" "slices"
"time" "time"
"github.com/ethereum-optimism/optimism/cannon/mipsevm"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/types" "github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
"github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
...@@ -98,6 +99,10 @@ func (c *ClaimHelper) RequireCorrectOutputRoot(ctx context.Context) { ...@@ -98,6 +99,10 @@ func (c *ClaimHelper) RequireCorrectOutputRoot(ctx context.Context) {
c.require.Equalf(expected, c.claim, "Should have correct output root in claim %v and position %v", c.Index, c.Position) c.require.Equalf(expected, c.claim, "Should have correct output root in claim %v and position %v", c.Index, c.Position)
} }
func (c *ClaimHelper) RequireInvalidStatusCode() {
c.require.Equal(byte(mipsevm.VMStatusInvalid), c.claim[0], "should have had invalid status code")
}
func (c *ClaimHelper) Attack(ctx context.Context, value common.Hash, opts ...MoveOpt) *ClaimHelper { func (c *ClaimHelper) Attack(ctx context.Context, value common.Hash, opts ...MoveOpt) *ClaimHelper {
c.game.Attack(ctx, c.Index, value, opts...) c.game.Attack(ctx, c.Index, value, opts...)
return c.WaitForCounterClaim(ctx) return c.WaitForCounterClaim(ctx)
......
...@@ -910,3 +910,45 @@ func TestOutputCannonHonestSafeTraceExtension_InvalidRoot(t *testing.T) { ...@@ -910,3 +910,45 @@ func TestOutputCannonHonestSafeTraceExtension_InvalidRoot(t *testing.T) {
game.LogGameData(ctx) game.LogGameData(ctx)
require.EqualValues(t, gameTypes.GameStatusChallengerWon, game.Status(ctx)) require.EqualValues(t, gameTypes.GameStatusChallengerWon, game.Status(ctx))
} }
func TestAgreeFirstBlockWithOriginOf1(t *testing.T) {
op_e2e.InitParallel(t, op_e2e.UsesCannon)
ctx := context.Background()
sys, _ := StartFaultDisputeSystem(t)
t.Cleanup(sys.Close)
rollupClient := sys.RollupClient("sequencer")
blockNum := uint64(0)
limit := uint64(100)
for ; blockNum <= limit; blockNum++ {
require.NoError(t, wait.ForBlock(ctx, sys.NodeClient("sequencer"), blockNum))
output, err := rollupClient.OutputAtBlock(ctx, blockNum)
require.NoError(t, err)
if output.BlockRef.L1Origin.Number == 1 {
break
}
}
require.Less(t, blockNum, limit)
// Create a dispute game with a dishonest claim @ L2 block #4
disputeGameFactory := disputegame.NewFactoryHelper(t, ctx, sys)
// Make the agreed block the first one with L1 origin of block 1 so the claim is blockNum+1
game := disputeGameFactory.StartOutputCannonGame(ctx, "sequencer", blockNum+1, common.Hash{0xCA, 0xFE})
require.NotNil(t, game)
outputRootClaim := game.DisputeLastBlock(ctx)
game.LogGameData(ctx)
honestChallenger := game.StartChallenger(ctx, "HonestActor", challenger.WithPrivKey(sys.Cfg.Secrets.Alice))
// Wait for the honest challenger to dispute the outputRootClaim. This creates a root of an execution game that we challenge by coercing
// a step at a preimage trace index.
outputRootClaim = outputRootClaim.WaitForCounterClaim(ctx)
game.LogGameData(ctx)
// Should claim output root is invalid, but actually panics.
outputRootClaim.RequireInvalidStatusCode()
// The above method already verified the image was uploaded and step called successfully
// So we don't waste time resolving the game - that's tested elsewhere.
require.NoError(t, honestChallenger.Close())
}
...@@ -109,21 +109,6 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, parent eth.L2BlockRef) (*Si ...@@ -109,21 +109,6 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, parent eth.L2BlockRef) (*Si
} }
} }
// If the epoch is advanced, update bq.l1Blocks
// Advancing epoch must be done after the pipeline successfully apply the entire span batch to the chain.
// Because the span batch can be reverted during processing the batch, then we must preserve existing l1Blocks
// to verify the epochs of the next candidate batch.
if len(bq.l1Blocks) > 0 && parent.L1Origin.Number > bq.l1Blocks[0].Number {
for i, l1Block := range bq.l1Blocks {
if parent.L1Origin.Number == l1Block.Number {
bq.l1Blocks = bq.l1Blocks[i:]
bq.log.Debug("Advancing internal L1 blocks", "next_epoch", bq.l1Blocks[0].ID(), "next_epoch_time", bq.l1Blocks[0].Time)
break
}
}
// If we can't find the origin of parent block, we have to advance bq.origin.
}
// Note: We use the origin that we will have to determine if it's behind. This is important // Note: We use the origin that we will have to determine if it's behind. This is important
// because it's the future origin that gets saved into the l1Blocks array. // because it's the future origin that gets saved into the l1Blocks array.
// We always update the origin of this stage if it is not the same so after the update code // We always update the origin of this stage if it is not the same so after the update code
...@@ -146,6 +131,21 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, parent eth.L2BlockRef) (*Si ...@@ -146,6 +131,21 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, parent eth.L2BlockRef) (*Si
bq.log.Info("Advancing bq origin", "origin", bq.origin, "originBehind", originBehind) bq.log.Info("Advancing bq origin", "origin", bq.origin, "originBehind", originBehind)
} }
// If the epoch is advanced, update bq.l1Blocks
// Advancing epoch must be done after the pipeline successfully apply the entire span batch to the chain.
// Because the span batch can be reverted during processing the batch, then we must preserve existing l1Blocks
// to verify the epochs of the next candidate batch.
if len(bq.l1Blocks) > 0 && parent.L1Origin.Number > bq.l1Blocks[0].Number {
for i, l1Block := range bq.l1Blocks {
if parent.L1Origin.Number == l1Block.Number {
bq.l1Blocks = bq.l1Blocks[i:]
bq.log.Debug("Advancing internal L1 blocks", "next_epoch", bq.l1Blocks[0].ID(), "next_epoch_time", bq.l1Blocks[0].Time)
break
}
}
// If we can't find the origin of parent block, we have to advance bq.origin.
}
// Load more data into the batch queue // Load more data into the batch queue
outOfData := false outOfData := false
if batch, err := bq.prev.NextBatch(ctx); err == io.EOF { if batch, err := bq.prev.NextBatch(ctx); err == io.EOF {
......
...@@ -147,6 +147,7 @@ func TestBatchQueue(t *testing.T) { ...@@ -147,6 +147,7 @@ func TestBatchQueue(t *testing.T) {
{"BatchQueueMissing", BatchQueueMissing}, {"BatchQueueMissing", BatchQueueMissing},
{"BatchQueueAdvancedEpoch", BatchQueueAdvancedEpoch}, {"BatchQueueAdvancedEpoch", BatchQueueAdvancedEpoch},
{"BatchQueueShuffle", BatchQueueShuffle}, {"BatchQueueShuffle", BatchQueueShuffle},
{"BatchQueueResetOneBlockBeforeOrigin", BatchQueueResetOneBlockBeforeOrigin},
} }
for _, test := range tests { for _, test := range tests {
test := test test := test
...@@ -224,6 +225,67 @@ func BatchQueueNewOrigin(t *testing.T, batchType int) { ...@@ -224,6 +225,67 @@ func BatchQueueNewOrigin(t *testing.T, batchType int) {
require.Equal(t, l1[2], bq.origin) require.Equal(t, l1[2], bq.origin)
} }
// BatchQueueResetOneBlockBeforeOrigin tests that the batch queue properly
// prunes the l1Block recorded as part of a reset when the starting origin
// is exactly one block prior to the safe head origin.
func BatchQueueResetOneBlockBeforeOrigin(t *testing.T, batchType int) {
log := testlog.Logger(t, log.LevelTrace)
l1 := L1Chain([]uint64{10, 15, 20, 25})
safeHead := eth.L2BlockRef{
Hash: mockHash(10, 2),
Number: 0,
ParentHash: common.Hash{},
Time: 20,
L1Origin: l1[1].ID(),
SequenceNumber: 0,
}
cfg := &rollup.Config{
Genesis: rollup.Genesis{
L2Time: 10,
},
BlockTime: 2,
MaxSequencerDrift: 600,
SeqWindowSize: 2,
DeltaTime: getDeltaTime(batchType),
}
input := &fakeBatchQueueInput{
batches: []Batch{nil},
errors: []error{io.EOF},
origin: l1[0],
}
bq := NewBatchQueue(log, cfg, input, nil)
_ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{})
require.Equal(t, []eth.L1BlockRef{l1[0]}, bq.l1Blocks)
// Prev Origin: 0; Safehead Origin: 1; Internal Origin: 0
// Should return no data but keep the same origin
data, _, err := bq.NextBatch(context.Background(), safeHead)
require.Nil(t, data)
require.Equal(t, io.EOF, err)
require.Equal(t, []eth.L1BlockRef{l1[0]}, bq.l1Blocks)
require.Equal(t, l1[0], bq.origin)
// Prev Origin: 1; Safehead Origin: 1; Internal Origin: 0
// Should record new l1 origin in l1blocks, prune block 0 and advance internal origin
input.origin = l1[1]
data, _, err = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, data)
require.Equalf(t, io.EOF, err, "expected io.EOF but got %v", err)
require.Equal(t, []eth.L1BlockRef{l1[1]}, bq.l1Blocks)
require.Equal(t, l1[1], bq.origin)
// Prev Origin: 2; Safehead Origin: 1; Internal Origin: 1
// Should add to l1Blocks + advance internal origin
input.origin = l1[2]
data, _, err = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, data)
require.Equal(t, io.EOF, err)
require.Equal(t, []eth.L1BlockRef{l1[1], l1[2]}, bq.l1Blocks)
require.Equal(t, l1[2], bq.origin)
}
// BatchQueueEager adds a bunch of contiguous batches and asserts that // BatchQueueEager adds a bunch of contiguous batches and asserts that
// enough calls to `NextBatch` return all of those batches. // enough calls to `NextBatch` return all of those batches.
func BatchQueueEager(t *testing.T, batchType int) { func BatchQueueEager(t *testing.T, batchType int) {
......
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