Commit 210492cd authored by Tei Im's avatar Tei Im Committed by protolambda

Implement span batch atomicity during chain derivation

Add pendingSafeHead to engine queue
Engine queue advances node's safe head once the span batch is fully processed
Batch queue resets cached batches derived from span batch when detecting processing error
parent aec1466f
......@@ -135,6 +135,10 @@ func (s *L2Verifier) L2Safe() eth.L2BlockRef {
return s.derivation.SafeL2Head()
}
func (s *L2Verifier) L2PendingSafe() eth.L2BlockRef {
return s.derivation.PendingSafeL2Head()
}
func (s *L2Verifier) L2Unsafe() eth.L2BlockRef {
return s.derivation.UnsafeL2Head()
}
......
......@@ -2,15 +2,24 @@ package actions
import (
"errors"
"math/big"
"math/rand"
"testing"
"github.com/ethereum-optimism/optimism/op-batcher/compressor"
"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/sync"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/sources"
"github.com/ethereum-optimism/optimism/op-service/testlog"
"github.com/ethereum-optimism/optimism/op-service/testutils"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/stretchr/testify/require"
)
......@@ -166,3 +175,232 @@ func TestEngineP2PSync(gt *testing.T) {
require.Equal(t, sequencer.L2Unsafe().Hash, verifier.EngineSyncTarget().Hash)
}
}
func TestInvalidPayloadInSpanBatch(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
minTs := hexutil.Uint64(0)
// Activate SpanBatch hardfork
dp.DeployConfig.L2GenesisSpanBatchTimeOffset = &minTs
dp.DeployConfig.L2BlockTime = 2
sd := e2eutils.Setup(t, dp, defaultAlloc)
log := testlog.Logger(t, log.LvlInfo)
_, _, miner, sequencer, seqEng, verifier, _, batcher := setupReorgTestActors(t, dp, sd, log)
l2Cl := seqEng.EthClient()
rng := rand.New(rand.NewSource(1234))
signer := types.LatestSigner(sd.L2Cfg.Config)
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
c, e := compressor.NewRatioCompressor(compressor.Config{
TargetFrameSize: 128_000,
TargetNumFrames: 1,
ApproxComprRatio: 1,
})
require.NoError(t, e)
spanBatchBuilder := derive.NewSpanBatchBuilder(sd.RollupCfg.Genesis.L2Time, sd.RollupCfg.L2ChainID)
// Create new span batch channel
channelOut, err := derive.NewChannelOut(derive.SpanBatchType, c, spanBatchBuilder)
require.NoError(t, err)
// Create block A1 ~ A12 for L1 block #0 ~ #2
miner.ActEmptyBlock(t)
miner.ActEmptyBlock(t)
sequencer.ActL1HeadSignal(t)
sequencer.ActBuildToL1HeadUnsafe(t)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
require.NoError(t, err)
if i == 8 {
// Make block A8 as an invalid block
invalidTx := testutils.RandomTx(rng, big.NewInt(100), signer)
block = block.WithBody([]*types.Transaction{block.Transactions()[0], invalidTx}, []*types.Header{})
}
// Add A1 ~ A12 into the channel
_, err = channelOut.AddBlock(block)
require.NoError(t, err)
}
// Submit span batch(A1, ..., A7, invalid A8, A9, ..., A12)
batcher.l2ChannelOut = channelOut
batcher.ActL2ChannelClose(t)
batcher.ActL2BatchSubmit(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
miner.ActL1SafeNext(t)
miner.ActL1FinalizeNext(t)
// After the verifier processed the span batch, only unsafe head should be advanced to A7.
// Safe head is not updated because the span batch is not fully processed.
verifier.ActL1HeadSignal(t)
verifier.ActL2PipelineFull(t)
require.Equal(t, verifier.L2Unsafe().Number, uint64(7))
require.Equal(t, verifier.L2Safe().Number, uint64(0))
// Create new span batch channel
c, e = compressor.NewRatioCompressor(compressor.Config{
TargetFrameSize: 128_000,
TargetNumFrames: 1,
ApproxComprRatio: 1,
})
require.NoError(t, e)
spanBatchBuilder = derive.NewSpanBatchBuilder(sd.RollupCfg.Genesis.L2Time, sd.RollupCfg.L2ChainID)
channelOut, err = derive.NewChannelOut(derive.SpanBatchType, c, spanBatchBuilder)
require.NoError(t, err)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
require.NoError(t, err)
if i == 1 {
// Create valid TX
aliceNonce, err := seqEng.EthClient().PendingNonceAt(t.Ctx(), dp.Addresses.Alice)
require.NoError(t, err)
data := make([]byte, rand.Intn(100))
gas, err := core.IntrinsicGas(data, nil, false, true, true, false)
require.NoError(t, err)
baseFee := seqEng.l2Chain.CurrentBlock().BaseFee
tx := types.MustSignNewTx(dp.Secrets.Alice, signer, &types.DynamicFeeTx{
ChainID: sd.L2Cfg.Config.ChainID,
Nonce: aliceNonce,
GasTipCap: big.NewInt(2 * params.GWei),
GasFeeCap: new(big.Int).Add(new(big.Int).Mul(baseFee, big.NewInt(2)), big.NewInt(2*params.GWei)),
Gas: gas,
To: &dp.Addresses.Bob,
Value: big.NewInt(0),
Data: data,
})
// Create valid new block B1 at the same height as A1
block = block.WithBody([]*types.Transaction{block.Transactions()[0], tx}, []*types.Header{})
}
// Add B1, A2 ~ A12 into the channel
_, err = channelOut.AddBlock(block)
require.NoError(t, err)
}
// Submit span batch(B1, A2, ... A12)
batcher.l2ChannelOut = channelOut
batcher.ActL2ChannelClose(t)
batcher.ActL2BatchSubmit(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
miner.ActL1SafeNext(t)
miner.ActL1FinalizeNext(t)
verifier.ActL1HeadSignal(t)
verifier.ActL2PipelineFull(t)
// verifier should advance its unsafe and safe head to the height of A12.
require.Equal(t, verifier.L2Unsafe().Number, uint64(12))
require.Equal(t, verifier.L2Safe().Number, uint64(12))
}
func TestSpanBatchAtomicity_Consolidation(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
minTs := hexutil.Uint64(0)
// Activate SpanBatch hardfork
dp.DeployConfig.L2GenesisSpanBatchTimeOffset = &minTs
dp.DeployConfig.L2BlockTime = 2
sd := e2eutils.Setup(t, dp, defaultAlloc)
log := testlog.Logger(t, log.LvlInfo)
_, _, miner, sequencer, seqEng, verifier, _, batcher := setupReorgTestActors(t, dp, sd, log)
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), log, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err)
targetHeadNumber := uint64(6) // L1 block time / L2 block time
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
// Create 6 blocks
miner.ActEmptyBlock(t)
sequencer.ActL1HeadSignal(t)
sequencer.ActBuildToL1HeadUnsafe(t)
require.Equal(t, sequencer.L2Unsafe().Number, targetHeadNumber)
// Gossip unsafe blocks to the verifier
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
seqHead, err := seqEngCl.PayloadByNumber(t.Ctx(), i)
require.NoError(t, err)
verifier.ActL2UnsafeGossipReceive(seqHead)(t)
}
verifier.ActL2PipelineFull(t)
// Check if the verifier's unsafe sync is done
require.Equal(t, sequencer.L2Unsafe().Hash, verifier.L2Unsafe().Hash)
// Build and submit a span batch with 6 blocks
batcher.ActSubmitAll(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
// Start verifier safe sync
verifier.ActL1HeadSignal(t)
verifier.l2PipelineIdle = false
for !verifier.l2PipelineIdle {
verifier.ActL2PipelineStep(t)
if verifier.L2PendingSafe().Number < targetHeadNumber {
// If the span batch is not fully processed, the safe head must not advance.
require.Equal(t, verifier.L2Safe().Number, uint64(0))
} else {
// Once the span batch is fully processed, the safe head must advance to the end of span batch.
require.Equal(t, verifier.L2Safe().Number, targetHeadNumber)
require.Equal(t, verifier.L2Safe(), verifier.L2PendingSafe())
}
// The unsafe head must not be changed
require.Equal(t, verifier.L2Unsafe(), sequencer.L2Unsafe())
}
}
func TestSpanBatchAtomicity_ForceAdvance(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
minTs := hexutil.Uint64(0)
// Activate SpanBatch hardfork
dp.DeployConfig.L2GenesisSpanBatchTimeOffset = &minTs
dp.DeployConfig.L2BlockTime = 2
sd := e2eutils.Setup(t, dp, defaultAlloc)
log := testlog.Logger(t, log.LvlInfo)
_, _, miner, sequencer, _, verifier, _, batcher := setupReorgTestActors(t, dp, sd, log)
targetHeadNumber := uint64(6) // L1 block time / L2 block time
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
require.Equal(t, verifier.L2Unsafe().Number, uint64(0))
// Create 6 blocks
miner.ActEmptyBlock(t)
sequencer.ActL1HeadSignal(t)
sequencer.ActBuildToL1HeadUnsafe(t)
require.Equal(t, sequencer.L2Unsafe().Number, targetHeadNumber)
// Build and submit a span batch with 6 blocks
batcher.ActSubmitAll(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
// Start verifier safe sync
verifier.ActL1HeadSignal(t)
verifier.l2PipelineIdle = false
for !verifier.l2PipelineIdle {
verifier.ActL2PipelineStep(t)
if verifier.L2PendingSafe().Number < targetHeadNumber {
// If the span batch is not fully processed, the safe head must not advance.
require.Equal(t, verifier.L2Safe().Number, uint64(0))
} else {
// Once the span batch is fully processed, the safe head must advance to the end of span batch.
require.Equal(t, verifier.L2Safe().Number, targetHeadNumber)
require.Equal(t, verifier.L2Safe(), verifier.L2PendingSafe())
}
// The unsafe head and the pending safe head must be the same
require.Equal(t, verifier.L2Unsafe(), verifier.L2PendingSafe())
}
}
......@@ -33,6 +33,7 @@ type AttributesQueue struct {
builder AttributesBuilder
prev *BatchQueue
batch *SingularBatch
isLastInSpan bool
}
func NewAttributesQueue(log log.Logger, cfg *rollup.Config, builder AttributesBuilder, prev *BatchQueue) *AttributesQueue {
......@@ -48,14 +49,15 @@ func (aq *AttributesQueue) Origin() eth.L1BlockRef {
return aq.prev.Origin()
}
func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2BlockRef) (*eth.PayloadAttributes, error) {
func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2BlockRef) (*AttributesWithParent, error) {
// Get a batch if we need it
if aq.batch == nil {
batch, err := aq.prev.NextBatch(ctx, l2SafeHead)
batch, isLastInSpan, err := aq.prev.NextBatch(ctx, l2SafeHead)
if err != nil {
return nil, err
}
aq.batch = batch
aq.isLastInSpan = isLastInSpan
}
// Actually generate the next attributes
......@@ -63,8 +65,10 @@ func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2
return nil, err
} else {
// Clear out the local state once we will succeed
attr := AttributesWithParent{attrs, l2SafeHead, aq.isLastInSpan}
aq.batch = nil
return attrs, nil
aq.isLastInSpan = false
return &attr, nil
}
}
......
......@@ -89,22 +89,27 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch {
return nextBatch
}
func (bq *BatchQueue) maybeAdvanceEpoch(nextBatch *SingularBatch) {
if len(bq.l1Blocks) == 0 {
return
}
if nextBatch.GetEpochNum() == rollup.Epoch(bq.l1Blocks[0].Number)+1 {
// Advance epoch if necessary
bq.l1Blocks = bq.l1Blocks[1:]
}
}
func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*SingularBatch, error) {
// NextBatch return next valid batch upon the given safe head.
// It also returns the boolean that indicates if the batch is the last block in the batch.
func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*SingularBatch, bool, error) {
if len(bq.nextSpan) > 0 {
if bq.nextSpan[0].Timestamp == safeL2Head.Time+bq.config.BlockTime {
// If there are cached singular batches, pop first one and return.
nextBatch := bq.popNextBatch(safeL2Head)
bq.maybeAdvanceEpoch(nextBatch)
return nextBatch, nil
return nextBatch, len(bq.nextSpan) == 0, nil
} else {
bq.nextSpan = bq.nextSpan[:0]
}
}
// If the epoch is advanced, update bq.l1Blocks
if len(bq.l1Blocks) > 0 && safeL2Head.L1Origin.Number > bq.l1Blocks[0].Number {
for i, l1Block := range bq.l1Blocks {
if safeL2Head.L1Origin.Number == l1Block.Number {
bq.l1Blocks = bq.l1Blocks[i:]
break
}
}
}
// Note: We use the origin that we will have to determine if it's behind. This is important
......@@ -134,7 +139,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
if batch, err := bq.prev.NextBatch(ctx); err == io.EOF {
outOfData = true
} else if err != nil {
return nil, err
return nil, false, err
} else if !originBehind {
bq.AddBatch(ctx, batch, safeL2Head)
}
......@@ -143,20 +148,20 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
// empty the previous stages
if originBehind {
if outOfData {
return nil, io.EOF
return nil, false, io.EOF
} else {
return nil, NotEnoughData
return nil, false, NotEnoughData
}
}
// Finally attempt to derive more batches
batch, err := bq.deriveNextBatch(ctx, outOfData, safeL2Head)
if err == io.EOF && outOfData {
return nil, io.EOF
return nil, false, io.EOF
} else if err == io.EOF {
return nil, NotEnoughData
return nil, false, NotEnoughData
} else if err != nil {
return nil, err
return nil, false, err
}
var nextBatch *SingularBatch
......@@ -164,28 +169,27 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
case SingularBatchType:
singularBatch, ok := batch.(*SingularBatch)
if !ok {
return nil, NewCriticalError(errors.New("failed type assertion to SingularBatch"))
return nil, false, NewCriticalError(errors.New("failed type assertion to SingularBatch"))
}
nextBatch = singularBatch
case SpanBatchType:
spanBatch, ok := batch.(*SpanBatch)
if !ok {
return nil, NewCriticalError(errors.New("failed type assertion to SpanBatch"))
return nil, false, NewCriticalError(errors.New("failed type assertion to SpanBatch"))
}
// If next batch is SpanBatch, convert it to SingularBatches.
singularBatches, err := spanBatch.GetSingularBatches(bq.l1Blocks, safeL2Head)
if err != nil {
return nil, NewCriticalError(err)
return nil, false, NewCriticalError(err)
}
bq.nextSpan = singularBatches
// span-batches are non-empty, so the below pop is safe.
nextBatch = bq.popNextBatch(safeL2Head)
default:
return nil, NewCriticalError(fmt.Errorf("unrecognized batch type: %d", batch.GetBatchType()))
return nil, false, NewCriticalError(fmt.Errorf("unrecognized batch type: %d", batch.GetBatchType()))
}
bq.maybeAdvanceEpoch(nextBatch)
return nextBatch, nil
return nextBatch, len(bq.nextSpan) == 0, nil
}
func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error {
......
This diff is collapsed.
This diff is collapsed.
......@@ -25,17 +25,18 @@ import (
type fakeAttributesQueue struct {
origin eth.L1BlockRef
attrs *eth.PayloadAttributes
islastInSpan bool
}
func (f *fakeAttributesQueue) Origin() eth.L1BlockRef {
return f.origin
}
func (f *fakeAttributesQueue) NextAttributes(_ context.Context, _ eth.L2BlockRef) (*eth.PayloadAttributes, error) {
func (f *fakeAttributesQueue) NextAttributes(_ context.Context, safeHead eth.L2BlockRef) (*AttributesWithParent, error) {
if f.attrs == nil {
return nil, io.EOF
}
return f.attrs, nil
return &AttributesWithParent{f.attrs, safeHead, f.islastInSpan}, nil
}
var _ NextAttributesProvider = (*fakeAttributesQueue)(nil)
......@@ -909,7 +910,7 @@ func TestBlockBuildingRace(t *testing.T) {
GasLimit: &gasLimit,
}
prev := &fakeAttributesQueue{origin: refA, attrs: attrs}
prev := &fakeAttributesQueue{origin: refA, attrs: attrs, islastInSpan: true}
eq := NewEngineQueue(logger, cfg, eng, metrics, prev, l1F, &sync.Config{})
require.ErrorIs(t, eq.Reset(context.Background(), eth.L1BlockRef{}, eth.SystemConfig{}), io.EOF)
......@@ -1078,7 +1079,7 @@ func TestResetLoop(t *testing.T) {
l1F.ExpectL1BlockRefByHash(refA.Hash, refA, nil)
l1F.ExpectL1BlockRefByHash(refA.Hash, refA, nil)
prev := &fakeAttributesQueue{origin: refA, attrs: attrs}
prev := &fakeAttributesQueue{origin: refA, attrs: attrs, islastInSpan: true}
eq := NewEngineQueue(logger, cfg, eng, metrics.NoopMetrics, prev, l1F, &sync.Config{})
eq.unsafeHead = refA2
......
......@@ -51,6 +51,7 @@ type EngineQueueStage interface {
Finalized() eth.L2BlockRef
UnsafeL2Head() eth.L2BlockRef
SafeL2Head() eth.L2BlockRef
PendingSafeL2Head() eth.L2BlockRef
EngineSyncTarget() eth.L2BlockRef
Origin() eth.L1BlockRef
SystemConfig() eth.SystemConfig
......@@ -148,6 +149,10 @@ func (dp *DerivationPipeline) SafeL2Head() eth.L2BlockRef {
return dp.eng.SafeL2Head()
}
func (dp *DerivationPipeline) PendingSafeL2Head() eth.L2BlockRef {
return dp.eng.PendingSafeL2Head()
}
// UnsafeL2Head returns the head of the L2 chain that we are deriving for, this may be past what we derived from L1
func (dp *DerivationPipeline) UnsafeL2Head() eth.L2BlockRef {
return dp.eng.UnsafeL2Head()
......
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