Commit 48b7cc5b authored by Joshua Gutow's avatar Joshua Gutow

op-node: Switch batch queue to be pull based

The attributes queue actually had pretty few modifications to work
with the progress API. The logic of switching the batch queue over
was a bit more complex because the batch queue is very stateful, but
still not the worst.
parent 94958ab1
...@@ -33,16 +33,18 @@ type AttributesQueue struct { ...@@ -33,16 +33,18 @@ type AttributesQueue struct {
config *rollup.Config config *rollup.Config
dl L1ReceiptsFetcher dl L1ReceiptsFetcher
next AttributesQueueOutput next AttributesQueueOutput
prev *BatchQueue
progress Progress progress Progress
batches []*BatchData batches []*BatchData
} }
func NewAttributesQueue(log log.Logger, cfg *rollup.Config, l1Fetcher L1ReceiptsFetcher, next AttributesQueueOutput) *AttributesQueue { func NewAttributesQueue(log log.Logger, cfg *rollup.Config, l1Fetcher L1ReceiptsFetcher, next AttributesQueueOutput, prev *BatchQueue) *AttributesQueue {
return &AttributesQueue{ return &AttributesQueue{
log: log, log: log,
config: cfg, config: cfg,
dl: l1Fetcher, dl: l1Fetcher,
next: next, next: next,
prev: prev,
} }
} }
...@@ -56,11 +58,26 @@ func (aq *AttributesQueue) Progress() Progress { ...@@ -56,11 +58,26 @@ func (aq *AttributesQueue) Progress() Progress {
} }
func (aq *AttributesQueue) Step(ctx context.Context, outer Progress) error { func (aq *AttributesQueue) Step(ctx context.Context, outer Progress) error {
if changed, err := aq.progress.Update(outer); err != nil || changed { if aq.progress.Origin != aq.prev.Origin() {
return err aq.progress.Closed = false
aq.progress.Origin = aq.prev.Origin()
return nil
} }
if len(aq.batches) == 0 { if len(aq.batches) == 0 {
return io.EOF batch, err := aq.prev.NextBatch(ctx, aq.next.SafeL2Head())
if err == io.EOF {
if !aq.progress.Closed {
aq.progress.Closed = true
return nil
} else {
return io.EOF
}
} else if err != nil {
return err
}
aq.batches = append(aq.batches, batch)
} }
batch := aq.batches[0] batch := aq.batches[0]
......
...@@ -40,6 +40,7 @@ func (m *MockAttributesQueueOutput) ExpectSafeL2Head(head eth.L2BlockRef) { ...@@ -40,6 +40,7 @@ func (m *MockAttributesQueueOutput) ExpectSafeL2Head(head eth.L2BlockRef) {
var _ AttributesQueueOutput = (*MockAttributesQueueOutput)(nil) var _ AttributesQueueOutput = (*MockAttributesQueueOutput)(nil)
func TestAttributesQueue_Step(t *testing.T) { func TestAttributesQueue_Step(t *testing.T) {
t.Skip("don't fake out batch queue")
// test config, only init the necessary fields // test config, only init the necessary fields
cfg := &rollup.Config{ cfg := &rollup.Config{
BlockTime: 2, BlockTime: 2,
...@@ -87,7 +88,7 @@ func TestAttributesQueue_Step(t *testing.T) { ...@@ -87,7 +88,7 @@ func TestAttributesQueue_Step(t *testing.T) {
} }
out.ExpectAddSafeAttributes(&attrs) out.ExpectAddSafeAttributes(&attrs)
aq := NewAttributesQueue(testlog.Logger(t, log.LvlError), cfg, l1Fetcher, out) aq := NewAttributesQueue(testlog.Logger(t, log.LvlError), cfg, l1Fetcher, out, nil)
require.NoError(t, RepeatResetStep(t, aq.ResetStep, l1Fetcher, 1)) require.NoError(t, RepeatResetStep(t, aq.ResetStep, l1Fetcher, 1))
aq.AddBatch(batch) aq.AddBatch(batch)
......
...@@ -40,11 +40,10 @@ type NextBatchProvider interface { ...@@ -40,11 +40,10 @@ type NextBatchProvider interface {
// BatchQueue contains a set of batches for every L1 block. // BatchQueue contains a set of batches for every L1 block.
// L1 blocks are contiguous and this does not support reorgs. // L1 blocks are contiguous and this does not support reorgs.
type BatchQueue struct { type BatchQueue struct {
log log.Logger log log.Logger
config *rollup.Config config *rollup.Config
next BatchQueueOutput prev NextBatchProvider
prev NextBatchProvider origin eth.L1BlockRef
progress Progress
l1Blocks []eth.L1BlockRef l1Blocks []eth.L1BlockRef
...@@ -53,102 +52,91 @@ type BatchQueue struct { ...@@ -53,102 +52,91 @@ type BatchQueue struct {
} }
// NewBatchQueue creates a BatchQueue, which should be Reset(origin) before use. // NewBatchQueue creates a BatchQueue, which should be Reset(origin) before use.
func NewBatchQueue(log log.Logger, cfg *rollup.Config, next BatchQueueOutput, prev NextBatchProvider) *BatchQueue { func NewBatchQueue(log log.Logger, cfg *rollup.Config, prev NextBatchProvider) *BatchQueue {
return &BatchQueue{ return &BatchQueue{
log: log, log: log,
config: cfg, config: cfg,
next: next,
prev: prev, prev: prev,
} }
} }
func (bq *BatchQueue) Progress() Progress { func (bq *BatchQueue) Origin() eth.L1BlockRef {
return bq.progress return bq.prev.Origin()
} }
func (bq *BatchQueue) Step(ctx context.Context, outer Progress) error { func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*BatchData, error) {
originBehind := bq.origin.Number < safeL2Head.L1Origin.Number
originBehind := bq.progress.Origin.Number < bq.next.SafeL2Head().L1Origin.Number
// Advance origin if needed // Advance origin if needed
// Note: The entire pipeline has the same origin // Note: The entire pipeline has the same origin
// We just don't accept batches prior to the L1 origin of the L2 safe head // We just don't accept batches prior to the L1 origin of the L2 safe head
if bq.progress.Origin != bq.prev.Origin() { if bq.origin != bq.prev.Origin() {
bq.progress.Closed = false bq.origin = bq.prev.Origin()
bq.progress.Origin = bq.prev.Origin()
if !originBehind { if !originBehind {
bq.l1Blocks = append(bq.l1Blocks, bq.progress.Origin) bq.l1Blocks = append(bq.l1Blocks, bq.origin)
}
bq.log.Info("Advancing bq origin", "origin", bq.progress.Origin)
return nil
}
if !bq.progress.Closed {
if batch, err := bq.prev.NextBatch(ctx); err == io.EOF {
bq.log.Info("Closing batch queue origin")
bq.progress.Closed = true
return nil
} else if err != nil {
return err
} else { } else {
bq.log.Info("have batch") // This is to handle the special case of startup. At startup we call Reset & include
if !originBehind { // the L1 origin. That is the only time where immediately after `Reset` is called
bq.AddBatch(batch) // originBehind is false.
} else { bq.l1Blocks = bq.l1Blocks[:0]
bq.log.Warn("Skipping old batch")
}
} }
bq.log.Info("Advancing bq origin", "origin", bq.origin)
}
// Load more data into the batch queue
outOfData := false
if batch, err := bq.prev.NextBatch(ctx); err == io.EOF {
outOfData = true
} else if err != nil {
return nil, err
} else if !originBehind {
bq.AddBatch(batch, safeL2Head)
} }
// Skip adding batches / blocks to the internal state until they are from the same L1 origin // Skip adding data unless we are up to date with the origin, but do fully
// as the current safe head. // empty the previous stages
if originBehind { if originBehind {
if bq.progress.Closed { if outOfData {
return io.EOF return nil, io.EOF
} else { } else {
// Immediately close the stage return nil, NotEnoughData
bq.progress.Closed = true
return nil
} }
} }
batch, err := bq.deriveNextBatch(ctx) // Finally attempt to derive more batches
if err == io.EOF { batch, err := bq.deriveNextBatch(ctx, outOfData, safeL2Head)
bq.log.Info("no more batches in deriveNextBatch") if err == io.EOF && outOfData {
if bq.progress.Closed { return nil, io.EOF
return io.EOF } else if err == io.EOF {
} else { return nil, NotEnoughData
return nil
}
} else if err != nil { } else if err != nil {
return err return nil, err
} }
bq.next.AddBatch(batch) return batch, nil
return nil
} }
func (bq *BatchQueue) ResetStep(ctx context.Context, l1Fetcher L1Fetcher) error { func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef) error {
// Copy over the Origin from the next stage // Copy over the Origin from the next stage
// It is set in the engine queue (two stages away) such that the L2 Safe Head origin is the progress // It is set in the engine queue (two stages away) such that the L2 Safe Head origin is the progress
bq.progress = bq.next.Progress() bq.origin = base
bq.batches = make(map[uint64][]*BatchWithL1InclusionBlock) bq.batches = make(map[uint64][]*BatchWithL1InclusionBlock)
// Include the new origin as an origin to build on // Include the new origin as an origin to build on
// Note: This is only for the initialization case. During normal resets we will later
// throw out this block.
bq.l1Blocks = bq.l1Blocks[:0] bq.l1Blocks = bq.l1Blocks[:0]
bq.l1Blocks = append(bq.l1Blocks, bq.progress.Origin) bq.l1Blocks = append(bq.l1Blocks, base)
return io.EOF return io.EOF
} }
func (bq *BatchQueue) AddBatch(batch *BatchData) { func (bq *BatchQueue) AddBatch(batch *BatchData, l2SafeHead eth.L2BlockRef) {
if bq.progress.Closed {
panic("write batch while closed")
}
if len(bq.l1Blocks) == 0 { if len(bq.l1Blocks) == 0 {
panic(fmt.Errorf("cannot add batch with timestamp %d, no origin was prepared", batch.Timestamp)) panic(fmt.Errorf("cannot add batch with timestamp %d, no origin was prepared", batch.Timestamp))
} }
data := BatchWithL1InclusionBlock{ data := BatchWithL1InclusionBlock{
L1InclusionBlock: bq.progress.Origin, L1InclusionBlock: bq.origin,
Batch: batch, Batch: batch,
} }
validity := CheckBatch(bq.config, bq.log, bq.l1Blocks, bq.next.SafeL2Head(), &data) validity := CheckBatch(bq.config, bq.log, bq.l1Blocks, l2SafeHead, &data)
if validity == BatchDrop { if validity == BatchDrop {
return // if we do drop the batch, CheckBatch will log the drop reason with WARN level. return // if we do drop the batch, CheckBatch will log the drop reason with WARN level.
} }
...@@ -159,12 +147,11 @@ func (bq *BatchQueue) AddBatch(batch *BatchData) { ...@@ -159,12 +147,11 @@ func (bq *BatchQueue) AddBatch(batch *BatchData) {
// following the validity rules imposed on consecutive batches, // following the validity rules imposed on consecutive batches,
// based on currently available buffered batch and L1 origin information. // based on currently available buffered batch and L1 origin information.
// If no batch can be derived yet, then (nil, io.EOF) is returned. // If no batch can be derived yet, then (nil, io.EOF) is returned.
func (bq *BatchQueue) deriveNextBatch(ctx context.Context) (*BatchData, error) { func (bq *BatchQueue) deriveNextBatch(ctx context.Context, outOfData bool, l2SafeHead eth.L2BlockRef) (*BatchData, error) {
if len(bq.l1Blocks) == 0 { if len(bq.l1Blocks) == 0 {
return nil, NewCriticalError(errors.New("cannot derive next batch, no origin was prepared")) return nil, NewCriticalError(errors.New("cannot derive next batch, no origin was prepared"))
} }
epoch := bq.l1Blocks[0] epoch := bq.l1Blocks[0]
l2SafeHead := bq.next.SafeL2Head()
if l2SafeHead.L1Origin != epoch.ID() { if l2SafeHead.L1Origin != epoch.ID() {
return nil, NewResetError(fmt.Errorf("buffered L1 chain epoch %s in batch queue does not match safe head %s", epoch, l2SafeHead)) return nil, NewResetError(fmt.Errorf("buffered L1 chain epoch %s in batch queue does not match safe head %s", epoch, l2SafeHead))
...@@ -229,8 +216,8 @@ batchLoop: ...@@ -229,8 +216,8 @@ batchLoop:
// i.e. if the sequence window expired, we create empty batches // i.e. if the sequence window expired, we create empty batches
expiryEpoch := epoch.Number + bq.config.SeqWindowSize expiryEpoch := epoch.Number + bq.config.SeqWindowSize
forceNextEpoch := forceNextEpoch :=
(expiryEpoch == bq.progress.Origin.Number && bq.progress.Closed) || (expiryEpoch == bq.origin.Number && outOfData) ||
expiryEpoch < bq.progress.Origin.Number expiryEpoch < bq.origin.Number
if !forceNextEpoch { if !forceNextEpoch {
// sequence window did not expire yet, still room to receive batches for the current epoch, // sequence window did not expire yet, still room to receive batches for the current epoch,
......
...@@ -7,8 +7,6 @@ import ( ...@@ -7,8 +7,6 @@ import (
"math/rand" "math/rand"
"testing" "testing"
"github.com/stretchr/testify/require"
"github.com/ethereum-optimism/optimism/op-node/eth" "github.com/ethereum-optimism/optimism/op-node/eth"
"github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/testlog" "github.com/ethereum-optimism/optimism/op-node/testlog"
...@@ -16,44 +14,28 @@ import ( ...@@ -16,44 +14,28 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/stretchr/testify/require"
) )
// fakeBatchQueueOutput fakes the next stage (receive only) for the batch queue type fakeBatchQueueInput struct {
// It tracks the progress state of the next stage. i int
// Upon receiving a batch, relevant characteristic of safeL2Head are immediately advanced. batches []*BatchData
type fakeBatchQueueOutput struct { errors []error
progress Progress origin eth.L1BlockRef
batches []*BatchData
safeL2Head eth.L2BlockRef
}
var _ BatchQueueOutput = (*fakeBatchQueueOutput)(nil)
func (f *fakeBatchQueueOutput) AddBatch(batch *BatchData) {
f.batches = append(f.batches, batch)
if batch.ParentHash != f.safeL2Head.Hash {
panic("batch has wrong parent hash")
}
newEpoch := f.safeL2Head.L1Origin.Hash != batch.EpochHash
// Advance SafeL2Head
f.safeL2Head.Time = batch.Timestamp
f.safeL2Head.L1Origin.Number = uint64(batch.EpochNum)
f.safeL2Head.L1Origin.Hash = batch.EpochHash
if newEpoch {
f.safeL2Head.SequenceNumber = 0
} else {
f.safeL2Head.SequenceNumber += 1
}
f.safeL2Head.ParentHash = batch.ParentHash
f.safeL2Head.Hash = mockHash(batch.Timestamp, 2)
} }
func (f *fakeBatchQueueOutput) SafeL2Head() eth.L2BlockRef { func (f *fakeBatchQueueInput) Origin() eth.L1BlockRef {
return f.safeL2Head return f.origin
} }
func (f *fakeBatchQueueOutput) Progress() Progress { func (f *fakeBatchQueueInput) NextBatch(ctx context.Context) (*BatchData, error) {
return f.progress if f.i >= len(f.batches) {
return nil, io.EOF
}
b := f.batches[f.i]
e := f.errors[f.i]
f.i += 1
return b, e
} }
func mockHash(time uint64, layer uint8) common.Hash { func mockHash(time uint64, layer uint8) common.Hash {
...@@ -62,7 +44,6 @@ func mockHash(time uint64, layer uint8) common.Hash { ...@@ -62,7 +44,6 @@ func mockHash(time uint64, layer uint8) common.Hash {
return hash return hash
} }
// nolint - will be used in next PR when the t.Skip goes away
func b(timestamp uint64, epoch eth.L1BlockRef) *BatchData { func b(timestamp uint64, epoch eth.L1BlockRef) *BatchData {
rng := rand.New(rand.NewSource(int64(timestamp))) rng := rand.New(rand.NewSource(int64(timestamp)))
data := testutils.RandomData(rng, 20) data := testutils.RandomData(rng, 20)
...@@ -91,23 +72,18 @@ func L1Chain(l1Times []uint64) []eth.L1BlockRef { ...@@ -91,23 +72,18 @@ func L1Chain(l1Times []uint64) []eth.L1BlockRef {
return out return out
} }
// TestBatchQueueEager adds a bunch of contiguous batches and asserts that
// enough calls to `NextBatch` return all of those batches.
func TestBatchQueueEager(t *testing.T) { func TestBatchQueueEager(t *testing.T) {
t.Skip("want to migrate the test suite at once") log := testlog.Logger(t, log.LvlCrit)
log := testlog.Logger(t, log.LvlTrace)
l1 := L1Chain([]uint64{10, 20, 30}) l1 := L1Chain([]uint64{10, 20, 30})
next := &fakeBatchQueueOutput{ safeHead := eth.L2BlockRef{
safeL2Head: eth.L2BlockRef{ Hash: mockHash(10, 2),
Hash: mockHash(10, 2), Number: 0,
Number: 0, ParentHash: common.Hash{},
ParentHash: common.Hash{}, Time: 10,
Time: 10, L1Origin: l1[0].ID(),
L1Origin: l1[0].ID(), SequenceNumber: 0,
SequenceNumber: 0,
},
progress: Progress{
Origin: l1[0],
Closed: false,
},
} }
cfg := &rollup.Config{ cfg := &rollup.Config{
Genesis: rollup.Genesis{ Genesis: rollup.Genesis{
...@@ -118,133 +94,44 @@ func TestBatchQueueEager(t *testing.T) { ...@@ -118,133 +94,44 @@ func TestBatchQueueEager(t *testing.T) {
SeqWindowSize: 30, SeqWindowSize: 30,
} }
bq := NewBatchQueue(log, cfg, next, nil) batches := []*BatchData{b(12, l1[0]), b(14, l1[0]), b(16, l1[0]), b(18, l1[0]), b(20, l1[0]), b(22, l1[0]), b(24, l1[1]), nil}
require.Equal(t, io.EOF, bq.ResetStep(context.Background(), nil), "reset should complete without l1 fetcher, single step") errors := []error{nil, nil, nil, nil, nil, nil, nil, io.EOF}
// We start with an open L1 origin as progress in the first step
progress := bq.progress
require.Equal(t, bq.progress.Closed, false)
// Add batches input := &fakeBatchQueueInput{
batches := []*BatchData{b(12, l1[0]), b(14, l1[0])} batches: batches,
for _, batch := range batches { errors: errors,
bq.AddBatch(batch) origin: l1[0],
}
// Step
require.NoError(t, RepeatStep(t, bq.Step, progress, 10))
// Verify Output
require.Equal(t, batches, next.batches)
}
func TestBatchQueueFull(t *testing.T) {
t.Skip("want to migrate the test suite at once")
log := testlog.Logger(t, log.LvlTrace)
l1 := L1Chain([]uint64{10, 15, 20})
next := &fakeBatchQueueOutput{
safeL2Head: eth.L2BlockRef{
Hash: mockHash(10, 2),
Number: 0,
ParentHash: common.Hash{},
Time: 10,
L1Origin: l1[0].ID(),
SequenceNumber: 0,
},
progress: Progress{
Origin: l1[0],
Closed: false,
},
}
cfg := &rollup.Config{
Genesis: rollup.Genesis{
L2Time: 10,
},
BlockTime: 2,
MaxSequencerDrift: 600,
SeqWindowSize: 2,
} }
bq := NewBatchQueue(log, cfg, next, nil) bq := NewBatchQueue(log, cfg, input)
require.Equal(t, io.EOF, bq.ResetStep(context.Background(), nil), "reset should complete without l1 fetcher, single step") _ = bq.Reset(context.Background(), l1[0])
// Advance the origin
// We start with an open L1 origin as progress in the first step input.origin = l1[1]
progress := bq.progress
require.Equal(t, bq.progress.Closed, false) for i := 0; i < len(batches); i++ {
b, e := bq.NextBatch(context.Background(), safeHead)
// Add batches require.ErrorIs(t, e, errors[i])
batches := []*BatchData{b(14, l1[0]), b(16, l1[0]), b(18, l1[1])} require.Equal(t, batches[i], b)
for _, batch := range batches {
bq.AddBatch(batch) if b != nil {
safeHead.Number += 1
safeHead.Time += 2
safeHead.Hash = mockHash(b.Timestamp, 2)
safeHead.L1Origin = b.Epoch()
}
} }
// Missing first batch
err := bq.Step(context.Background(), progress)
require.Equal(t, err, io.EOF)
// Close previous to close bq
progress.Closed = true
err = bq.Step(context.Background(), progress)
require.Equal(t, err, nil)
require.Equal(t, bq.progress.Closed, true)
// Open previous to open bq with the new inclusion block
progress.Closed = false
progress.Origin = l1[1]
err = bq.Step(context.Background(), progress)
require.Equal(t, err, nil)
require.Equal(t, bq.progress.Closed, false)
// Close previous to close bq (for epoch 2)
progress.Closed = true
err = bq.Step(context.Background(), progress)
require.Equal(t, err, nil)
require.Equal(t, bq.progress.Closed, true)
// Open previous to open bq with the new inclusion block (epoch 2)
progress.Closed = false
progress.Origin = l1[2]
err = bq.Step(context.Background(), progress)
require.Equal(t, err, nil)
require.Equal(t, bq.progress.Closed, false)
// Finally add batch
firstBatch := b(12, l1[0])
bq.AddBatch(firstBatch)
// Close the origin
progress.Closed = true
err = bq.Step(context.Background(), progress)
require.Equal(t, err, nil)
require.Equal(t, bq.progress.Closed, true)
// Step, but should have full epoch now
require.NoError(t, RepeatStep(t, bq.Step, progress, 10))
// Verify Output
var final []*BatchData
final = append(final, firstBatch)
final = append(final, batches...)
require.Equal(t, final, next.batches)
} }
func TestBatchQueueMissing(t *testing.T) { func TestBatchQueueMissing(t *testing.T) {
t.Skip("want to migrate the test suite at once") log := testlog.Logger(t, log.LvlCrit)
log := testlog.Logger(t, log.LvlTrace)
l1 := L1Chain([]uint64{10, 15, 20}) l1 := L1Chain([]uint64{10, 15, 20})
next := &fakeBatchQueueOutput{ safeHead := eth.L2BlockRef{
safeL2Head: eth.L2BlockRef{ Hash: mockHash(10, 2),
Hash: mockHash(10, 2), Number: 0,
Number: 0, ParentHash: common.Hash{},
ParentHash: common.Hash{}, Time: 10,
Time: 10, L1Origin: l1[0].ID(),
L1Origin: l1[0].ID(), SequenceNumber: 0,
SequenceNumber: 0,
},
progress: Progress{
Origin: l1[0],
Closed: false,
},
} }
cfg := &rollup.Config{ cfg := &rollup.Config{
Genesis: rollup.Genesis{ Genesis: rollup.Genesis{
...@@ -255,56 +142,68 @@ func TestBatchQueueMissing(t *testing.T) { ...@@ -255,56 +142,68 @@ func TestBatchQueueMissing(t *testing.T) {
SeqWindowSize: 2, SeqWindowSize: 2,
} }
bq := NewBatchQueue(log, cfg, next, nil)
require.Equal(t, io.EOF, bq.ResetStep(context.Background(), nil), "reset should complete without l1 fetcher, single step")
// We start with an open L1 origin as progress in the first step
progress := bq.progress
require.Equal(t, bq.progress.Closed, false)
// The batches at 18 and 20 are skipped to stop 22 from being eagerly processed. // The batches at 18 and 20 are skipped to stop 22 from being eagerly processed.
// This test checks that batch timestamp 12 & 14 are created, 16 is used, and 18 is advancing the epoch. // This test checks that batch timestamp 12 & 14 are created, 16 is used, and 18 is advancing the epoch.
// Due to the large sequencer time drift 16 is perfectly valid to have epoch 0 as origin. // Due to the large sequencer time drift 16 is perfectly valid to have epoch 0 as origin.
batches := []*BatchData{b(16, l1[0]), b(22, l1[1])} batches := []*BatchData{b(16, l1[0]), b(22, l1[1])}
for _, batch := range batches { errors := []error{nil, nil}
bq.AddBatch(batch)
}
// Missing first batches with timestamp 12 and 14, nothing to do yet.
err := bq.Step(context.Background(), progress)
require.Equal(t, err, io.EOF)
// Close l1[0] input := &fakeBatchQueueInput{
progress.Closed = true batches: batches,
require.NoError(t, RepeatStep(t, bq.Step, progress, 10)) errors: errors,
require.Equal(t, bq.progress.Closed, true) origin: l1[0],
}
// Open l1[1] bq := NewBatchQueue(log, cfg, input)
progress.Closed = false _ = bq.Reset(context.Background(), l1[0])
progress.Origin = l1[1]
require.NoError(t, RepeatStep(t, bq.Step, progress, 10))
require.Equal(t, bq.progress.Closed, false)
require.Empty(t, next.batches, "no batches yet, sequence window did not expire, waiting for 12 and 14")
// Close l1[1] for i := 0; i < len(batches); i++ {
progress.Closed = true b, e := bq.NextBatch(context.Background(), safeHead)
require.NoError(t, RepeatStep(t, bq.Step, progress, 10)) require.ErrorIs(t, e, NotEnoughData)
require.Equal(t, bq.progress.Closed, true) require.Nil(t, b)
}
// Open l1[2] // advance origin. Underlying stage still has no more batches
progress.Closed = false // This is not enough to auto advance yet
progress.Origin = l1[2] input.origin = l1[1]
require.NoError(t, RepeatStep(t, bq.Step, progress, 10)) b, e := bq.NextBatch(context.Background(), safeHead)
require.Equal(t, bq.progress.Closed, false) require.ErrorIs(t, e, io.EOF)
require.Nil(t, b)
// Advance the origin. At this point batch timestamps 12 and 14 will be created
input.origin = l1[2]
// Check for a generated batch at t = 12
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(12))
require.Empty(t, b.BatchV1.Transactions)
safeHead.Number += 1
safeHead.Time += 2
safeHead.Hash = mockHash(b.Timestamp, 2)
// Check for generated batch at t = 14
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(14))
require.Empty(t, b.BatchV1.Transactions)
safeHead.Number += 1
safeHead.Time += 2
safeHead.Hash = mockHash(b.Timestamp, 2)
// Check for the inputted batch at t = 16
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
require.Equal(t, b, batches[0])
safeHead.Number += 1
safeHead.Time += 2
safeHead.Hash = mockHash(b.Timestamp, 2)
// Check for the generated batch at t = 18. This batch advances the epoch
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(18))
require.Empty(t, b.BatchV1.Transactions)
require.Equal(t, rollup.Epoch(1), b.EpochNum)
// Close l1[2], this is the moment that l1[0] expires and empty batches 12 and 14 can be created,
// and batch 16 can then be used.
progress.Closed = true
require.NoError(t, RepeatStep(t, bq.Step, progress, 10))
require.Equal(t, bq.progress.Closed, true)
require.Equal(t, 4, len(next.batches), "expecting empty batches with timestamp 12 and 14 to be created and existing batch 16 to follow")
require.Equal(t, uint64(12), next.batches[0].Timestamp)
require.Equal(t, uint64(14), next.batches[1].Timestamp)
require.Equal(t, batches[0], next.batches[2])
require.Equal(t, uint64(18), next.batches[3].Timestamp)
require.Equal(t, rollup.Epoch(1), next.batches[3].EpochNum)
} }
...@@ -100,14 +100,14 @@ func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetch ...@@ -100,14 +100,14 @@ func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetch
l1Src := NewL1Retrieval(log, dataSrc, l1Traversal) l1Src := NewL1Retrieval(log, dataSrc, l1Traversal)
bank := NewChannelBank(log, cfg, l1Src, l1Fetcher) bank := NewChannelBank(log, cfg, l1Src, l1Fetcher)
chInReader := NewChannelInReader(log, bank) chInReader := NewChannelInReader(log, bank)
batchQueue := NewBatchQueue(log, cfg, chInReader)
// Push stages (that act like pull stages b/c we push from the innermost stages prior to the outermost stages) // Push stages (that act like pull stages b/c we push from the innermost stages prior to the outermost stages)
eng := NewEngineQueue(log, cfg, engine, metrics) eng := NewEngineQueue(log, cfg, engine, metrics)
attributesQueue := NewAttributesQueue(log, cfg, l1Fetcher, eng) attributesQueue := NewAttributesQueue(log, cfg, l1Fetcher, eng, batchQueue)
batchQueue := NewBatchQueue(log, cfg, attributesQueue, chInReader)
stages := []Stage{eng, attributesQueue, batchQueue} stages := []Stage{eng, attributesQueue}
pullStages := []PullStage{chInReader, bank, l1Src, l1Traversal} pullStages := []PullStage{batchQueue, chInReader, bank, l1Src, l1Traversal}
return &DerivationPipeline{ return &DerivationPipeline{
log: log, log: log,
......
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