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 { ...@@ -135,6 +135,10 @@ func (s *L2Verifier) L2Safe() eth.L2BlockRef {
return s.derivation.SafeL2Head() return s.derivation.SafeL2Head()
} }
func (s *L2Verifier) L2PendingSafe() eth.L2BlockRef {
return s.derivation.PendingSafeL2Head()
}
func (s *L2Verifier) L2Unsafe() eth.L2BlockRef { func (s *L2Verifier) L2Unsafe() eth.L2BlockRef {
return s.derivation.UnsafeL2Head() return s.derivation.UnsafeL2Head()
} }
......
...@@ -2,15 +2,24 @@ package actions ...@@ -2,15 +2,24 @@ package actions
import ( import (
"errors" "errors"
"math/big"
"math/rand" "math/rand"
"testing" "testing"
"github.com/ethereum-optimism/optimism/op-batcher/compressor"
"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/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/sources" "github.com/ethereum-optimism/optimism/op-service/sources"
"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/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/log"
"github.com/ethereum/go-ethereum/params"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
...@@ -166,3 +175,232 @@ func TestEngineP2PSync(gt *testing.T) { ...@@ -166,3 +175,232 @@ func TestEngineP2PSync(gt *testing.T) {
require.Equal(t, sequencer.L2Unsafe().Hash, verifier.EngineSyncTarget().Hash) 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())
}
}
...@@ -28,11 +28,12 @@ type AttributesBuilder interface { ...@@ -28,11 +28,12 @@ type AttributesBuilder interface {
} }
type AttributesQueue struct { type AttributesQueue struct {
log log.Logger log log.Logger
config *rollup.Config config *rollup.Config
builder AttributesBuilder builder AttributesBuilder
prev *BatchQueue prev *BatchQueue
batch *SingularBatch batch *SingularBatch
isLastInSpan bool
} }
func NewAttributesQueue(log log.Logger, cfg *rollup.Config, builder AttributesBuilder, prev *BatchQueue) *AttributesQueue { func NewAttributesQueue(log log.Logger, cfg *rollup.Config, builder AttributesBuilder, prev *BatchQueue) *AttributesQueue {
...@@ -48,14 +49,15 @@ func (aq *AttributesQueue) Origin() eth.L1BlockRef { ...@@ -48,14 +49,15 @@ func (aq *AttributesQueue) Origin() eth.L1BlockRef {
return aq.prev.Origin() 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 // Get a batch if we need it
if aq.batch == nil { if aq.batch == nil {
batch, err := aq.prev.NextBatch(ctx, l2SafeHead) batch, isLastInSpan, err := aq.prev.NextBatch(ctx, l2SafeHead)
if err != nil { if err != nil {
return nil, err return nil, err
} }
aq.batch = batch aq.batch = batch
aq.isLastInSpan = isLastInSpan
} }
// Actually generate the next attributes // Actually generate the next attributes
...@@ -63,8 +65,10 @@ func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2 ...@@ -63,8 +65,10 @@ func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2
return nil, err return nil, err
} else { } else {
// Clear out the local state once we will succeed // Clear out the local state once we will succeed
attr := AttributesWithParent{attrs, l2SafeHead, aq.isLastInSpan}
aq.batch = nil aq.batch = nil
return attrs, nil aq.isLastInSpan = false
return &attr, nil
} }
} }
......
...@@ -89,22 +89,27 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { ...@@ -89,22 +89,27 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch {
return nextBatch return nextBatch
} }
func (bq *BatchQueue) maybeAdvanceEpoch(nextBatch *SingularBatch) { // NextBatch return next valid batch upon the given safe head.
if len(bq.l1Blocks) == 0 { // It also returns the boolean that indicates if the batch is the last block in the batch.
return func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*SingularBatch, bool, error) {
} if len(bq.nextSpan) > 0 {
if nextBatch.GetEpochNum() == rollup.Epoch(bq.l1Blocks[0].Number)+1 { if bq.nextSpan[0].Timestamp == safeL2Head.Time+bq.config.BlockTime {
// Advance epoch if necessary // If there are cached singular batches, pop first one and return.
bq.l1Blocks = bq.l1Blocks[1:] nextBatch := bq.popNextBatch(safeL2Head)
return nextBatch, len(bq.nextSpan) == 0, nil
} else {
bq.nextSpan = bq.nextSpan[:0]
}
} }
}
func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*SingularBatch, error) { // If the epoch is advanced, update bq.l1Blocks
if len(bq.nextSpan) > 0 { if len(bq.l1Blocks) > 0 && safeL2Head.L1Origin.Number > bq.l1Blocks[0].Number {
// If there are cached singular batches, pop first one and return. for i, l1Block := range bq.l1Blocks {
nextBatch := bq.popNextBatch(safeL2Head) if safeL2Head.L1Origin.Number == l1Block.Number {
bq.maybeAdvanceEpoch(nextBatch) bq.l1Blocks = bq.l1Blocks[i:]
return nextBatch, nil break
}
}
} }
// 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
...@@ -134,7 +139,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) ...@@ -134,7 +139,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
if batch, err := bq.prev.NextBatch(ctx); err == io.EOF { if batch, err := bq.prev.NextBatch(ctx); err == io.EOF {
outOfData = true outOfData = true
} else if err != nil { } else if err != nil {
return nil, err return nil, false, err
} else if !originBehind { } else if !originBehind {
bq.AddBatch(ctx, batch, safeL2Head) bq.AddBatch(ctx, batch, safeL2Head)
} }
...@@ -143,20 +148,20 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) ...@@ -143,20 +148,20 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
// empty the previous stages // empty the previous stages
if originBehind { if originBehind {
if outOfData { if outOfData {
return nil, io.EOF return nil, false, io.EOF
} else { } else {
return nil, NotEnoughData return nil, false, NotEnoughData
} }
} }
// Finally attempt to derive more batches // Finally attempt to derive more batches
batch, err := bq.deriveNextBatch(ctx, outOfData, safeL2Head) batch, err := bq.deriveNextBatch(ctx, outOfData, safeL2Head)
if err == io.EOF && outOfData { if err == io.EOF && outOfData {
return nil, io.EOF return nil, false, io.EOF
} else if err == io.EOF { } else if err == io.EOF {
return nil, NotEnoughData return nil, false, NotEnoughData
} else if err != nil { } else if err != nil {
return nil, err return nil, false, err
} }
var nextBatch *SingularBatch var nextBatch *SingularBatch
...@@ -164,28 +169,27 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) ...@@ -164,28 +169,27 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef)
case SingularBatchType: case SingularBatchType:
singularBatch, ok := batch.(*SingularBatch) singularBatch, ok := batch.(*SingularBatch)
if !ok { 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 nextBatch = singularBatch
case SpanBatchType: case SpanBatchType:
spanBatch, ok := batch.(*SpanBatch) spanBatch, ok := batch.(*SpanBatch)
if !ok { 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. // If next batch is SpanBatch, convert it to SingularBatches.
singularBatches, err := spanBatch.GetSingularBatches(bq.l1Blocks, safeL2Head) singularBatches, err := spanBatch.GetSingularBatches(bq.l1Blocks, safeL2Head)
if err != nil { if err != nil {
return nil, NewCriticalError(err) return nil, false, NewCriticalError(err)
} }
bq.nextSpan = singularBatches bq.nextSpan = singularBatches
// span-batches are non-empty, so the below pop is safe. // span-batches are non-empty, so the below pop is safe.
nextBatch = bq.popNextBatch(safeL2Head) nextBatch = bq.popNextBatch(safeL2Head)
default: 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, len(bq.nextSpan) == 0, nil
return nextBatch, nil
} }
func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error { func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error {
......
This diff is collapsed.
This diff is collapsed.
...@@ -23,19 +23,20 @@ import ( ...@@ -23,19 +23,20 @@ import (
) )
type fakeAttributesQueue struct { type fakeAttributesQueue struct {
origin eth.L1BlockRef origin eth.L1BlockRef
attrs *eth.PayloadAttributes attrs *eth.PayloadAttributes
islastInSpan bool
} }
func (f *fakeAttributesQueue) Origin() eth.L1BlockRef { func (f *fakeAttributesQueue) Origin() eth.L1BlockRef {
return f.origin 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 { if f.attrs == nil {
return nil, io.EOF return nil, io.EOF
} }
return f.attrs, nil return &AttributesWithParent{f.attrs, safeHead, f.islastInSpan}, nil
} }
var _ NextAttributesProvider = (*fakeAttributesQueue)(nil) var _ NextAttributesProvider = (*fakeAttributesQueue)(nil)
...@@ -909,7 +910,7 @@ func TestBlockBuildingRace(t *testing.T) { ...@@ -909,7 +910,7 @@ func TestBlockBuildingRace(t *testing.T) {
GasLimit: &gasLimit, 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{}) eq := NewEngineQueue(logger, cfg, eng, metrics, prev, l1F, &sync.Config{})
require.ErrorIs(t, eq.Reset(context.Background(), eth.L1BlockRef{}, eth.SystemConfig{}), io.EOF) require.ErrorIs(t, eq.Reset(context.Background(), eth.L1BlockRef{}, eth.SystemConfig{}), io.EOF)
...@@ -1078,7 +1079,7 @@ func TestResetLoop(t *testing.T) { ...@@ -1078,7 +1079,7 @@ func TestResetLoop(t *testing.T) {
l1F.ExpectL1BlockRefByHash(refA.Hash, refA, nil) l1F.ExpectL1BlockRefByHash(refA.Hash, refA, nil)
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 := NewEngineQueue(logger, cfg, eng, metrics.NoopMetrics, prev, l1F, &sync.Config{})
eq.unsafeHead = refA2 eq.unsafeHead = refA2
......
...@@ -51,6 +51,7 @@ type EngineQueueStage interface { ...@@ -51,6 +51,7 @@ type EngineQueueStage interface {
Finalized() eth.L2BlockRef Finalized() eth.L2BlockRef
UnsafeL2Head() eth.L2BlockRef UnsafeL2Head() eth.L2BlockRef
SafeL2Head() eth.L2BlockRef SafeL2Head() eth.L2BlockRef
PendingSafeL2Head() eth.L2BlockRef
EngineSyncTarget() eth.L2BlockRef EngineSyncTarget() eth.L2BlockRef
Origin() eth.L1BlockRef Origin() eth.L1BlockRef
SystemConfig() eth.SystemConfig SystemConfig() eth.SystemConfig
...@@ -148,6 +149,10 @@ func (dp *DerivationPipeline) SafeL2Head() eth.L2BlockRef { ...@@ -148,6 +149,10 @@ func (dp *DerivationPipeline) SafeL2Head() eth.L2BlockRef {
return dp.eng.SafeL2Head() 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 // 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 { func (dp *DerivationPipeline) UnsafeL2Head() eth.L2BlockRef {
return dp.eng.UnsafeL2Head() 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