Commit c9677e41 authored by Axel Kingsley's avatar Axel Kingsley Committed by GitHub

op-batcher: stateful span batches & blind compressor (#9954)

* Add Benchmark for AddSingularBatch

* update compressor configs ; address PR comments

* Add b.N

* Export RandomSingularBatch through batch_test_util.go

* measure only the final batch ; other organizational improvements

* Add Benchmark for ToRawSpanBatch

* update tests

* minor fixup

* Add Benchmark for adding *All* Span Batches

* comment fixups

* narrow tests to only test span batches that won't exceed RLP limit

* Stateful Span Batches

* Blind Compressor

* final fixes

* add peek helper function

* Address PR Comments
parent 919df965
......@@ -83,11 +83,11 @@ func NewChannelBuilder(cfg ChannelConfig, rollupCfg rollup.Config, latestL1Origi
if err != nil {
return nil, err
}
var spanBatchBuilder *derive.SpanBatchBuilder
var spanBatch *derive.SpanBatch
if cfg.BatchType == derive.SpanBatchType {
spanBatchBuilder = derive.NewSpanBatchBuilder(rollupCfg.Genesis.L2Time, rollupCfg.L2ChainID)
spanBatch = derive.NewSpanBatch(rollupCfg.Genesis.L2Time, rollupCfg.L2ChainID)
}
co, err := derive.NewChannelOut(cfg.BatchType, c, spanBatchBuilder)
co, err := derive.NewChannelOut(cfg.BatchType, c, spanBatch)
if err != nil {
return nil, err
}
......
......@@ -7,6 +7,7 @@ import (
"math/big"
"math/rand"
"testing"
"time"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive"
......@@ -83,8 +84,9 @@ func newMiniL2BlockWithNumberParentAndL1Information(numTx int, l2Number *big.Int
// which is presumably ErrTooManyRLPBytes.
func addTooManyBlocks(cb *ChannelBuilder) error {
rng := rand.New(rand.NewSource(1234))
t := time.Now()
for i := 0; i < 10_000; i++ {
block := dtest.RandomL2BlockWithChainId(rng, 1000, defaultTestRollupConfig.L2ChainID)
block := dtest.RandomL2BlockWithChainIdAndTime(rng, 1000, defaultTestRollupConfig.L2ChainID, t.Add(time.Duration(i)*time.Second))
_, err := cb.AddBlock(block)
if err != nil {
return err
......@@ -511,8 +513,9 @@ func ChannelBuilder_OutputFramesMaxFrameIndex(t *testing.T, batchType uint) {
require.NoError(t, err)
require.False(t, cb.IsFull())
require.Equal(t, 0, cb.PendingFrames())
for {
a := dtest.RandomL2BlockWithChainId(rng, 1000, defaultTestRollupConfig.L2ChainID)
ti := time.Now()
for i := 0; ; i++ {
a := dtest.RandomL2BlockWithChainIdAndTime(rng, 1000, defaultTestRollupConfig.L2ChainID, ti.Add(time.Duration(i)*time.Second))
_, err = cb.AddBlock(a)
if cb.IsFull() {
fullErr := cb.FullErr()
......@@ -702,9 +705,10 @@ func ChannelBuilder_PendingFrames_TotalFrames(t *testing.T, batchType uint) {
require.Zero(cb.PendingFrames())
require.Zero(cb.TotalFrames())
ti := time.Now()
// fill up
for {
block := dtest.RandomL2BlockWithChainId(rng, 4, defaultTestRollupConfig.L2ChainID)
for i := 0; ; i++ {
block := dtest.RandomL2BlockWithChainIdAndTime(rng, 4, defaultTestRollupConfig.L2ChainID, ti.Add(time.Duration(i)*time.Second))
_, err := cb.AddBlock(block)
if cb.IsFull() {
break
......@@ -734,10 +738,10 @@ func ChannelBuilder_InputBytes(t *testing.T, batchType uint) {
rng := rand.New(rand.NewSource(4982432))
cfg := defaultTestChannelConfig()
cfg.BatchType = batchType
var spanBatchBuilder *derive.SpanBatchBuilder
var spanBatch *derive.SpanBatch
if batchType == derive.SpanBatchType {
chainId := big.NewInt(1234)
spanBatchBuilder = derive.NewSpanBatchBuilder(uint64(0), chainId)
spanBatch = derive.NewSpanBatch(uint64(0), chainId)
}
cb, err := NewChannelBuilder(cfg, defaultTestRollupConfig, latestL1BlockOrigin)
require.NoError(err)
......@@ -745,15 +749,17 @@ func ChannelBuilder_InputBytes(t *testing.T, batchType uint) {
require.Zero(cb.InputBytes())
var l int
ti := time.Now()
for i := 0; i < 5; i++ {
block := dtest.RandomL2BlockWithChainId(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID)
block := dtest.RandomL2BlockWithChainIdAndTime(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID, ti.Add(time.Duration(i)*time.Second))
if batchType == derive.SingularBatchType {
l += blockBatchRlpSize(t, block)
} else {
singularBatch, l1Info, err := derive.BlockToSingularBatch(&defaultTestRollupConfig, block)
require.NoError(err)
spanBatchBuilder.AppendSingularBatch(singularBatch, l1Info.SequenceNumber)
rawSpanBatch, err := spanBatchBuilder.GetRawSpanBatch()
err = spanBatch.AppendSingularBatch(singularBatch, l1Info.SequenceNumber)
require.NoError(err)
rawSpanBatch, err := spanBatch.ToRawSpanBatch()
require.NoError(err)
batch := derive.NewBatchData(rawSpanBatch)
var buf bytes.Buffer
......@@ -779,8 +785,9 @@ func ChannelBuilder_OutputBytes(t *testing.T, batchType uint) {
require.Zero(cb.OutputBytes())
for {
block := dtest.RandomL2BlockWithChainId(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID)
ti := time.Now()
for i := 0; ; i++ {
block := dtest.RandomL2BlockWithChainIdAndTime(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID, ti.Add(time.Duration(i)*time.Second))
_, err := cb.AddBlock(block)
if errors.Is(err, derive.ErrCompressorFull) {
break
......
package compressor
import (
"bytes"
"compress/zlib"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive"
)
// BlindCompressor is a simple compressor that blindly compresses data
// the only way to know if the target size has been reached is to first flush the buffer
// and then check the length of the compressed data
type BlindCompressor struct {
config Config
inputBytes int
buf bytes.Buffer
compress *zlib.Writer
}
// NewBlindCompressor creates a new derive.Compressor implementation that compresses
func NewBlindCompressor(config Config) (derive.Compressor, error) {
c := &BlindCompressor{
config: config,
}
compress, err := zlib.NewWriterLevel(&c.buf, zlib.BestCompression)
if err != nil {
return nil, err
}
c.compress = compress
return c, nil
}
func (t *BlindCompressor) Write(p []byte) (int, error) {
if err := t.FullErr(); err != nil {
return 0, err
}
t.inputBytes += len(p)
return t.compress.Write(p)
}
func (t *BlindCompressor) Close() error {
return t.compress.Close()
}
func (t *BlindCompressor) Read(p []byte) (int, error) {
return t.buf.Read(p)
}
func (t *BlindCompressor) Reset() {
t.buf.Reset()
t.compress.Reset(&t.buf)
t.inputBytes = 0
}
func (t *BlindCompressor) Len() int {
return t.buf.Len()
}
func (t *BlindCompressor) Flush() error {
return t.compress.Flush()
}
// FullErr returns an error if the target output size has been reached.
// Flush *must* be called before this method to ensure the buffer is up to date
func (t *BlindCompressor) FullErr() error {
if uint64(t.Len()) >= t.config.TargetOutputSize {
return derive.ErrCompressorFull
}
return nil
}
package compressor_test
import (
"testing"
"github.com/ethereum-optimism/optimism/op-batcher/compressor"
"github.com/stretchr/testify/require"
)
func TestBlindCompressorLimit(t *testing.T) {
bc, err := compressor.NewBlindCompressor(compressor.Config{
TargetOutputSize: 10,
})
require.NoError(t, err)
// write far too much data to the compressor, but never flush
for i := 0; i < 100; i++ {
_, err := bc.Write([]byte("hello"))
require.NoError(t, err)
require.NoError(t, bc.FullErr())
}
// finally flush the compressor and see that it is full
bc.Flush()
require.Error(t, bc.FullErr())
// write a little more data to the compressor and see that it is still full
_, err = bc.Write([]byte("hello"))
require.Error(t, err)
}
......@@ -10,6 +10,7 @@ const (
RatioKind = "ratio"
ShadowKind = "shadow"
NoneKind = "none"
BlindKind = "blind"
// CloseOverheadZlib is the number of final bytes a [zlib.Writer] call writes
// to the output buffer.
......@@ -20,6 +21,7 @@ var Kinds = map[string]FactoryFunc{
RatioKind: NewRatioCompressor,
ShadowKind: NewShadowCompressor,
NoneKind: NewNonCompressor,
BlindKind: NewBlindCompressor,
}
var KindKeys []string
......
......@@ -189,14 +189,13 @@ func (s *L2Batcher) Buffer(t Testing) error {
if s.l2BatcherCfg.GarbageCfg != nil {
ch, err = NewGarbageChannelOut(s.l2BatcherCfg.GarbageCfg)
} else {
c, e := compressor.NewRatioCompressor(compressor.Config{
c, e := compressor.NewBlindCompressor(compressor.Config{
TargetOutputSize: batcher.MaxDataSize(1, s.l2BatcherCfg.MaxL1TxSize),
ApproxComprRatio: 1,
})
require.NoError(t, e, "failed to create compressor")
var batchType uint = derive.SingularBatchType
var spanBatchBuilder *derive.SpanBatchBuilder = nil
var spanBatch *derive.SpanBatch
if s.l2BatcherCfg.ForceSubmitSingularBatch && s.l2BatcherCfg.ForceSubmitSpanBatch {
t.Fatalf("ForceSubmitSingularBatch and ForceSubmitSpanBatch cannot be set to true at the same time")
......@@ -205,9 +204,9 @@ func (s *L2Batcher) Buffer(t Testing) error {
} else if s.l2BatcherCfg.ForceSubmitSpanBatch || s.rollupCfg.IsDelta(block.Time()) {
// If both ForceSubmitSingularBatch and ForceSubmitSpanbatch are false, use SpanBatch automatically if Delta HF is activated.
batchType = derive.SpanBatchType
spanBatchBuilder = derive.NewSpanBatchBuilder(s.rollupCfg.Genesis.L2Time, s.rollupCfg.L2ChainID)
spanBatch = derive.NewSpanBatch(s.rollupCfg.Genesis.L2Time, s.rollupCfg.L2ChainID)
}
ch, err = derive.NewChannelOut(batchType, c, spanBatchBuilder)
ch, err = derive.NewChannelOut(batchType, c, spanBatch)
}
require.NoError(t, err, "failed to create channel")
s.l2ChannelOut = ch
......
......@@ -26,6 +26,17 @@ import (
"github.com/stretchr/testify/require"
)
func newSpanChannelOut(t StatefulTesting, e e2eutils.SetupData) derive.ChannelOut {
c, err := compressor.NewBlindCompressor(compressor.Config{
TargetOutputSize: 128_000,
})
require.NoError(t, err)
spanBatch := derive.NewSpanBatch(e.RollupCfg.Genesis.L2Time, e.RollupCfg.L2ChainID)
channelOut, err := derive.NewChannelOut(derive.SpanBatchType, c, spanBatch)
require.NoError(t, err)
return channelOut
}
// TestSyncBatchType run each sync test case in singular batch mode and span batch mode.
func TestSyncBatchType(t *testing.T) {
tests := []struct {
......@@ -211,15 +222,7 @@ func TestBackupUnsafe(gt *testing.T) {
require.Equal(t, verifier.L2Unsafe().Number, uint64(5))
require.Equal(t, verifier.L2Safe().Number, uint64(0))
c, e := compressor.NewRatioCompressor(compressor.Config{
TargetOutputSize: 128_000,
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)
channelOut := newSpanChannelOut(t, *sd)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
......@@ -382,15 +385,7 @@ func TestBackupUnsafeReorgForkChoiceInputError(gt *testing.T) {
require.Equal(t, verifier.L2Unsafe().Number, uint64(5))
require.Equal(t, verifier.L2Safe().Number, uint64(0))
c, e := compressor.NewRatioCompressor(compressor.Config{
TargetOutputSize: 128_000,
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)
channelOut := newSpanChannelOut(t, *sd)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
......@@ -529,15 +524,7 @@ func TestBackupUnsafeReorgForkChoiceNotInputError(gt *testing.T) {
require.Equal(t, verifier.L2Unsafe().Number, uint64(5))
require.Equal(t, verifier.L2Safe().Number, uint64(0))
c, e := compressor.NewRatioCompressor(compressor.Config{
TargetOutputSize: 128_000,
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)
channelOut := newSpanChannelOut(t, *sd)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
......@@ -866,15 +853,7 @@ func TestInvalidPayloadInSpanBatch(gt *testing.T) {
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
c, e := compressor.NewRatioCompressor(compressor.Config{
TargetOutputSize: 128_000,
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)
channelOut := newSpanChannelOut(t, *sd)
// Create block A1 ~ A12 for L1 block #0 ~ #2
miner.ActEmptyBlock(t)
......@@ -913,15 +892,7 @@ func TestInvalidPayloadInSpanBatch(gt *testing.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{
TargetOutputSize: 128_000,
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)
channelOut = newSpanChannelOut(t, *sd)
for i := uint64(1); i <= sequencer.L2Unsafe().Number; i++ {
block, err := l2Cl.BlockByNumber(t.Ctx(), new(big.Int).SetUint64(i))
......
......@@ -25,8 +25,12 @@ var (
nc, _ = compressor.NewNonCompressor(compressor.Config{
TargetOutputSize: 100_000_000_000,
})
bc, _ = compressor.NewBlindCompressor(compressor.Config{
TargetOutputSize: 100_000_000_000,
})
compressors = map[string]derive.Compressor{
"BlindCompressor": bc,
"NonCompressor": nc,
"RatioCompressor": rc,
"ShadowCompressor": sc,
......@@ -107,8 +111,8 @@ func BenchmarkFinalBatchChannelOut(b *testing.B) {
// don't measure the setup time
b.StopTimer()
compressors[tc.compKey].Reset()
spanBatchBuilder := derive.NewSpanBatchBuilder(0, chainID)
cout, _ := derive.NewChannelOut(tc.BatchType, compressors[tc.compKey], spanBatchBuilder)
spanBatch := derive.NewSpanBatch(uint64(0), chainID)
cout, _ := derive.NewChannelOut(tc.BatchType, compressors[tc.compKey], spanBatch)
// add all but the final batch to the channel out
for i := 0; i < tc.BatchCount-1; i++ {
_, err := cout.AddSingularBatch(batches[i], 0)
......@@ -170,8 +174,8 @@ func BenchmarkAllBatchesChannelOut(b *testing.B) {
// don't measure the setup time
b.StopTimer()
compressors[tc.compKey].Reset()
spanBatchBuilder := derive.NewSpanBatchBuilder(0, chainID)
cout, _ := derive.NewChannelOut(tc.BatchType, compressors[tc.compKey], spanBatchBuilder)
spanBatch := derive.NewSpanBatch(0, chainID)
cout, _ := derive.NewChannelOut(tc.BatchType, compressors[tc.compKey], spanBatch)
b.StartTimer()
// add all batches to the channel out
for i := 0; i < tc.BatchCount; i++ {
......@@ -219,12 +223,13 @@ func BenchmarkGetRawSpanBatch(b *testing.B) {
for bn := 0; bn < b.N; bn++ {
// don't measure the setup time
b.StopTimer()
spanBatchBuilder := derive.NewSpanBatchBuilder(0, chainID)
spanBatch := derive.NewSpanBatch(uint64(0), chainID)
for i := 0; i < tc.BatchCount; i++ {
spanBatchBuilder.AppendSingularBatch(batches[i], 0)
err := spanBatch.AppendSingularBatch(batches[i], 0)
require.NoError(b, err)
}
b.StartTimer()
_, err := spanBatchBuilder.GetRawSpanBatch()
_, err := spanBatch.ToRawSpanBatch()
require.NoError(b, err)
}
})
......
......@@ -66,7 +66,7 @@ func buildSpanBatches(t *testing.T, parent *eth.L2BlockRef, singularBatches []*S
var spanBatches []Batch
idx := 0
for _, count := range blockCounts {
span := NewSpanBatch(singularBatches[idx : idx+count])
span := initializedSpanBatch(singularBatches[idx:idx+count], uint64(0), chainId)
spanBatches = append(spanBatches, span)
idx += count
}
......@@ -767,7 +767,7 @@ func TestBatchQueueOverlappingSpanBatch(t *testing.T) {
var inputBatches []Batch
batchSize := 3
for i := 0; i < len(expectedOutputBatches)-batchSize; i++ {
inputBatches = append(inputBatches, NewSpanBatch(expectedOutputBatches[i:i+batchSize]))
inputBatches = append(inputBatches, initializedSpanBatch(expectedOutputBatches[i:i+batchSize], uint64(0), chainId))
}
inputBatches = append(inputBatches, nil)
// inputBatches:
......@@ -872,12 +872,12 @@ func TestBatchQueueComplex(t *testing.T) {
inputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF}
// batches will be returned by fakeBatchQueueInput
inputBatches := []Batch{
NewSpanBatch(expectedOutputBatches[0:2]), // [6, 8] - no overlap
expectedOutputBatches[2], // [10] - no overlap
NewSpanBatch(expectedOutputBatches[1:4]), // [8, 10, 12] - overlapped blocks: 8 or 8, 10
expectedOutputBatches[4], // [14] - no overlap
NewSpanBatch(expectedOutputBatches[4:6]), // [14, 16] - overlapped blocks: nothing or 14
NewSpanBatch(expectedOutputBatches[6:9]), // [18, 20, 22] - no overlap
initializedSpanBatch(expectedOutputBatches[0:2], uint64(0), chainId), // [6, 8] - no overlap
expectedOutputBatches[2], // [10] - no overlap
initializedSpanBatch(expectedOutputBatches[1:4], uint64(0), chainId), // [8, 10, 12] - overlapped blocks: 8 or 8, 10
expectedOutputBatches[4], // [14] - no overlap
initializedSpanBatch(expectedOutputBatches[4:6], uint64(0), chainId), // [14, 16] - overlapped blocks: nothing or 14
initializedSpanBatch(expectedOutputBatches[6:9], uint64(0), chainId), // [18, 20, 22] - no overlap
}
// Shuffle the order of input batches
......@@ -979,7 +979,7 @@ func TestBatchQueueResetSpan(t *testing.T) {
}
input := &fakeBatchQueueInput{
batches: []Batch{NewSpanBatch(singularBatches)},
batches: []Batch{initializedSpanBatch(singularBatches, uint64(0), chainId)},
errors: []error{nil},
origin: l1[2],
}
......
This diff is collapsed.
......@@ -63,12 +63,12 @@ type ChannelOut interface {
OutputFrame(*bytes.Buffer, uint64) (uint16, error)
}
func NewChannelOut(batchType uint, compress Compressor, spanBatchBuilder *SpanBatchBuilder) (ChannelOut, error) {
func NewChannelOut(batchType uint, compress Compressor, spanBatch *SpanBatch) (ChannelOut, error) {
switch batchType {
case SingularBatchType:
return NewSingularChannelOut(compress)
case SpanBatchType:
return NewSpanChannelOut(compress, spanBatchBuilder)
return NewSpanChannelOut(compress, spanBatch)
default:
return nil, fmt.Errorf("unrecognized batch type: %d", batchType)
}
......
......@@ -8,7 +8,6 @@ import (
"fmt"
"io"
"math/big"
"sort"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
......@@ -411,9 +410,16 @@ func singularBatchToElement(singularBatch *SingularBatch) *SpanBatchElement {
// SpanBatch is an implementation of Batch interface,
// containing the input to build a span of L2 blocks in derived form (SpanBatchElement)
type SpanBatch struct {
ParentCheck [20]byte // First 20 bytes of the first block's parent hash
L1OriginCheck [20]byte // First 20 bytes of the last block's L1 origin hash
Batches []*SpanBatchElement // List of block input in derived form
ParentCheck [20]byte // First 20 bytes of the first block's parent hash
L1OriginCheck [20]byte // First 20 bytes of the last block's L1 origin hash
GenesisTimestamp uint64
ChainID *big.Int
Batches []*SpanBatchElement // List of block input in derived form
// caching
originBits *big.Int
blockTxCounts []uint64
sbtxs *spanBatchTxs
}
// spanBatchMarshaling is a helper type used for JSON marshaling.
......@@ -493,60 +499,73 @@ func (b *SpanBatch) GetBlockCount() int {
return len(b.Batches)
}
func (b *SpanBatch) peek(n int) *SpanBatchElement { return b.Batches[len(b.Batches)-1-n] }
// AppendSingularBatch appends a SingularBatch into the span batch
// updates l1OriginCheck or parentCheck if needed.
func (b *SpanBatch) AppendSingularBatch(singularBatch *SingularBatch) {
if len(b.Batches) == 0 {
copy(b.ParentCheck[:], singularBatch.ParentHash.Bytes()[:20])
func (b *SpanBatch) AppendSingularBatch(singularBatch *SingularBatch, seqNum uint64) error {
// if this new element is not ordered with respect to the last element, panic
if len(b.Batches) > 0 && b.peek(0).Timestamp > singularBatch.Timestamp {
panic("span batch is not ordered")
}
// always append the new batch and set the L1 origin check
b.Batches = append(b.Batches, singularBatchToElement(singularBatch))
// always update the L1 origin check
copy(b.L1OriginCheck[:], singularBatch.EpochHash.Bytes()[:20])
// if there is only one batch, initialize the ParentCheck
// and set the epochBit based on the seqNum
epochBit := uint(0)
if len(b.Batches) == 1 {
if seqNum == 0 {
epochBit = 1
}
copy(b.ParentCheck[:], singularBatch.ParentHash.Bytes()[:20])
} else {
// if there is more than one batch, set the epochBit based on the last two batches
if b.peek(1).EpochNum < b.peek(0).EpochNum {
epochBit = 1
}
}
// set the respective bit in the originBits
b.originBits.SetBit(b.originBits, len(b.Batches)-1, epochBit)
// update the blockTxCounts cache with the latest batch's tx count
b.blockTxCounts = append(b.blockTxCounts, uint64(len(b.peek(0).Transactions)))
// add the new txs to the sbtxs
newTxs := make([][]byte, 0, len(b.peek(0).Transactions))
for i := 0; i < len(b.peek(0).Transactions); i++ {
newTxs = append(newTxs, b.peek(0).Transactions[i])
}
// add the new txs to the sbtxs
// this is the only place where we can get an error
return b.sbtxs.AddTxs(newTxs, b.ChainID)
}
// ToRawSpanBatch merges SingularBatch List and initialize single RawSpanBatch
func (b *SpanBatch) ToRawSpanBatch(originChangedBit uint, genesisTimestamp uint64, chainID *big.Int) (*RawSpanBatch, error) {
func (b *SpanBatch) ToRawSpanBatch() (*RawSpanBatch, error) {
if len(b.Batches) == 0 {
return nil, errors.New("cannot merge empty singularBatch list")
}
raw := RawSpanBatch{}
// Sort by timestamp of L2 block
sort.Slice(b.Batches, func(i, j int) bool {
return b.Batches[i].Timestamp < b.Batches[j].Timestamp
})
// spanBatchPrefix
span_start := b.Batches[0]
span_end := b.Batches[len(b.Batches)-1]
raw.relTimestamp = span_start.Timestamp - genesisTimestamp
raw.l1OriginNum = uint64(span_end.EpochNum)
raw.parentCheck = b.ParentCheck
raw.l1OriginCheck = b.L1OriginCheck
// spanBatchPayload
raw.blockCount = uint64(len(b.Batches))
raw.originBits = new(big.Int)
raw.originBits.SetBit(raw.originBits, 0, originChangedBit)
for i := 1; i < len(b.Batches); i++ {
bit := uint(0)
if b.Batches[i-1].EpochNum < b.Batches[i].EpochNum {
bit = 1
}
raw.originBits.SetBit(raw.originBits, i, bit)
}
var blockTxCounts []uint64
var txs [][]byte
for _, batch := range b.Batches {
blockTxCount := uint64(len(batch.Transactions))
blockTxCounts = append(blockTxCounts, blockTxCount)
for _, rawTx := range batch.Transactions {
txs = append(txs, rawTx)
}
}
raw.blockTxCounts = blockTxCounts
stxs, err := newSpanBatchTxs(txs, chainID)
if err != nil {
return nil, err
}
raw.txs = stxs
return &raw, nil
return &RawSpanBatch{
spanBatchPrefix: spanBatchPrefix{
relTimestamp: span_start.Timestamp - b.GenesisTimestamp,
l1OriginNum: uint64(span_end.EpochNum),
parentCheck: b.ParentCheck,
l1OriginCheck: b.L1OriginCheck,
},
spanBatchPayload: spanBatchPayload{
blockCount: uint64(len(b.Batches)),
originBits: b.originBits,
blockTxCounts: b.blockTxCounts,
txs: b.sbtxs,
},
}, nil
}
// GetSingularBatches converts SpanBatchElements after L2 safe head to SingularBatches.
......@@ -582,17 +601,15 @@ func (b *SpanBatch) GetSingularBatches(l1Origins []eth.L1BlockRef, l2SafeHead et
}
// NewSpanBatch converts given singularBatches into SpanBatchElements, and creates a new SpanBatch.
func NewSpanBatch(singularBatches []*SingularBatch) *SpanBatch {
spanBatch := &SpanBatch{}
if len(singularBatches) == 0 {
return spanBatch
func NewSpanBatch(genesisTimestamp uint64, chainID *big.Int) *SpanBatch {
// newSpanBatchTxs can't fail with empty txs
sbtxs, _ := newSpanBatchTxs([][]byte{}, chainID)
return &SpanBatch{
GenesisTimestamp: genesisTimestamp,
ChainID: chainID,
originBits: big.NewInt(0),
sbtxs: sbtxs,
}
copy(spanBatch.ParentCheck[:], singularBatches[0].ParentHash.Bytes()[:20])
copy(spanBatch.L1OriginCheck[:], singularBatches[len(singularBatches)-1].EpochHash.Bytes()[:20])
for _, singularBatch := range singularBatches {
spanBatch.Batches = append(spanBatch.Batches, singularBatchToElement(singularBatch))
}
return spanBatch
}
// DeriveSpanBatch derives SpanBatch from BatchData.
......@@ -605,49 +622,6 @@ func DeriveSpanBatch(batchData *BatchData, blockTime, genesisTimestamp uint64, c
return rawSpanBatch.ToSpanBatch(blockTime, genesisTimestamp, chainID)
}
// SpanBatchBuilder is a utility type to build a SpanBatch by adding a SingularBatch one by one.
// makes easier to stack SingularBatches and convert to RawSpanBatch for encoding.
type SpanBatchBuilder struct {
genesisTimestamp uint64
chainID *big.Int
spanBatch *SpanBatch
originChangedBit uint
}
func NewSpanBatchBuilder(genesisTimestamp uint64, chainID *big.Int) *SpanBatchBuilder {
return &SpanBatchBuilder{
genesisTimestamp: genesisTimestamp,
chainID: chainID,
spanBatch: &SpanBatch{},
}
}
func (b *SpanBatchBuilder) AppendSingularBatch(singularBatch *SingularBatch, seqNum uint64) {
if b.GetBlockCount() == 0 {
b.originChangedBit = 0
if seqNum == 0 {
b.originChangedBit = 1
}
}
b.spanBatch.AppendSingularBatch(singularBatch)
}
func (b *SpanBatchBuilder) GetRawSpanBatch() (*RawSpanBatch, error) {
raw, err := b.spanBatch.ToRawSpanBatch(b.originChangedBit, b.genesisTimestamp, b.chainID)
if err != nil {
return nil, err
}
return raw, nil
}
func (b *SpanBatchBuilder) GetBlockCount() int {
return len(b.spanBatch.Batches)
}
func (b *SpanBatchBuilder) Reset() {
b.spanBatch = &SpanBatch{}
}
// ReadTxData reads raw RLP tx data from reader and returns txData and txType
func ReadTxData(r *bytes.Reader) ([]byte, int, error) {
var txData []byte
......
......@@ -18,6 +18,29 @@ import (
"github.com/ethereum-optimism/optimism/op-service/testutils"
)
// initializedSpanBatch creates a new SpanBatch with given SingularBatches.
// It is used *only* in tests to create a SpanBatch with given SingularBatches as a convenience.
// It will also ignore any errors that occur during AppendSingularBatch.
// Tests should manually set the first bit of the originBits if needed using SetFirstOriginChangedBit
func initializedSpanBatch(singularBatches []*SingularBatch, genesisTimestamp uint64, chainID *big.Int) *SpanBatch {
spanBatch := NewSpanBatch(genesisTimestamp, chainID)
if len(singularBatches) == 0 {
return spanBatch
}
for i := 0; i < len(singularBatches); i++ {
if err := spanBatch.AppendSingularBatch(singularBatches[i], uint64(i)); err != nil {
continue
}
}
return spanBatch
}
// setFirstOriginChangedBit sets the first bit of the originBits to the given value
// used for testing when a Span Batch is made with InitializedSpanBatch, which doesn't have a sequence number
func (b *SpanBatch) setFirstOriginChangedBit(bit uint) {
b.originBits.SetBit(b.originBits, 0, bit)
}
func TestSpanBatchForBatchInterface(t *testing.T) {
rng := rand.New(rand.NewSource(0x5432177))
chainID := big.NewInt(rng.Int63n(1000))
......@@ -27,7 +50,7 @@ func TestSpanBatchForBatchInterface(t *testing.T) {
safeL2Head := testutils.RandomL2BlockRef(rng)
safeL2Head.Hash = common.BytesToHash(singularBatches[0].ParentHash[:])
spanBatch := NewSpanBatch(singularBatches)
spanBatch := initializedSpanBatch(singularBatches, uint64(0), chainID)
// check interface method implementations except logging
require.Equal(t, SpanBatchType, spanBatch.GetBatchType())
......@@ -322,9 +345,10 @@ func TestSpanBatchDerive(t *testing.T) {
safeL2Head.Hash = common.BytesToHash(singularBatches[0].ParentHash[:])
genesisTimeStamp := 1 + singularBatches[0].Timestamp - 128
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
spanBatch := initializedSpanBatch(singularBatches, genesisTimeStamp, chainID)
// set originChangedBit to match the original test implementation
spanBatch.setFirstOriginChangedBit(uint(originChangedBit))
rawSpanBatch, err := spanBatch.ToRawSpanBatch()
require.NoError(t, err)
spanBatchDerived, err := rawSpanBatch.derive(l2BlockTime, genesisTimeStamp, chainID)
......@@ -354,14 +378,15 @@ func TestSpanBatchAppend(t *testing.T) {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
// initialize empty span batch
spanBatch := NewSpanBatch([]*SingularBatch{})
spanBatch := initializedSpanBatch([]*SingularBatch{}, uint64(0), chainID)
L := 2
for i := 0; i < L; i++ {
spanBatch.AppendSingularBatch(singularBatches[i])
err := spanBatch.AppendSingularBatch(singularBatches[i], uint64(i))
require.NoError(t, err)
}
// initialize with two singular batches
spanBatch2 := NewSpanBatch(singularBatches[:L])
spanBatch2 := initializedSpanBatch(singularBatches[:L], uint64(0), chainID)
require.Equal(t, spanBatch, spanBatch2)
}
......@@ -376,9 +401,10 @@ func TestSpanBatchMerge(t *testing.T) {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
blockCount := len(singularBatches)
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
spanBatch := initializedSpanBatch(singularBatches, genesisTimeStamp, chainID)
// set originChangedBit to match the original test implementation
spanBatch.setFirstOriginChangedBit(uint(originChangedBit))
rawSpanBatch, err := spanBatch.ToRawSpanBatch()
require.NoError(t, err)
// check span batch prefix
......@@ -389,7 +415,7 @@ func TestSpanBatchMerge(t *testing.T) {
// check span batch payload
require.Equal(t, int(rawSpanBatch.blockCount), len(singularBatches))
require.Equal(t, rawSpanBatch.originBits.Bit(0), originChangedBit)
require.Equal(t, rawSpanBatch.originBits.Bit(0), uint(originChangedBit))
for i := 1; i < blockCount; i++ {
if rawSpanBatch.originBits.Bit(i) == 1 {
require.Equal(t, singularBatches[i].EpochNum, singularBatches[i-1].EpochNum+1)
......@@ -421,9 +447,10 @@ func TestSpanBatchToSingularBatch(t *testing.T) {
safeL2Head.Time = singularBatches[0].Timestamp - 2
genesisTimeStamp := 1 + singularBatches[0].Timestamp - 128
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
spanBatch := initializedSpanBatch(singularBatches, genesisTimeStamp, chainID)
// set originChangedBit to match the original test implementation
spanBatch.setFirstOriginChangedBit(uint(originChangedBit))
rawSpanBatch, err := spanBatch.ToRawSpanBatch()
require.NoError(t, err)
l1Origins := mockL1Origin(rng, rawSpanBatch, singularBatches)
......@@ -492,49 +519,6 @@ func TestSpanBatchReadTxDataInvalid(t *testing.T) {
require.ErrorContains(t, err, "tx RLP prefix type must be list")
}
func TestSpanBatchBuilder(t *testing.T) {
rng := rand.New(rand.NewSource(0xbab1bab1))
chainID := new(big.Int).SetUint64(rng.Uint64())
for originChangedBit := 0; originChangedBit < 2; originChangedBit++ {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
safeL2Head := testutils.RandomL2BlockRef(rng)
if originChangedBit == 0 {
safeL2Head.Hash = common.BytesToHash(singularBatches[0].ParentHash[:])
}
genesisTimeStamp := 1 + singularBatches[0].Timestamp - 128
var seqNum uint64 = 1
if originChangedBit == 1 {
seqNum = 0
}
spanBatchBuilder := NewSpanBatchBuilder(genesisTimeStamp, chainID)
require.Equal(t, 0, spanBatchBuilder.GetBlockCount())
for i := 0; i < len(singularBatches); i++ {
spanBatchBuilder.AppendSingularBatch(singularBatches[i], seqNum)
require.Equal(t, i+1, spanBatchBuilder.GetBlockCount())
require.Equal(t, singularBatches[0].ParentHash.Bytes()[:20], spanBatchBuilder.spanBatch.ParentCheck[:])
require.Equal(t, singularBatches[i].EpochHash.Bytes()[:20], spanBatchBuilder.spanBatch.L1OriginCheck[:])
}
rawSpanBatch, err := spanBatchBuilder.GetRawSpanBatch()
require.NoError(t, err)
// compare with rawSpanBatch not using spanBatchBuilder
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch2, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
require.NoError(t, err)
require.Equal(t, rawSpanBatch2, rawSpanBatch)
spanBatchBuilder.Reset()
require.Equal(t, 0, spanBatchBuilder.GetBlockCount())
}
}
func TestSpanBatchMaxTxData(t *testing.T) {
rng := rand.New(rand.NewSource(0x177288))
......
......@@ -396,32 +396,42 @@ func isProtectedV(v uint64, txType int) bool {
}
func newSpanBatchTxs(txs [][]byte, chainID *big.Int) (*spanBatchTxs, error) {
sbtxs := &spanBatchTxs{
contractCreationBits: big.NewInt(0),
yParityBits: big.NewInt(0),
txSigs: []spanBatchSignature{},
txNonces: []uint64{},
txGases: []uint64{},
txTos: []common.Address{},
txDatas: []hexutil.Bytes{},
txTypes: []int{},
protectedBits: big.NewInt(0),
}
if err := sbtxs.AddTxs(txs, chainID); err != nil {
return nil, err
}
return sbtxs, nil
}
func (sbtx *spanBatchTxs) AddTxs(txs [][]byte, chainID *big.Int) error {
totalBlockTxCount := uint64(len(txs))
var txSigs []spanBatchSignature
var txTos []common.Address
var txNonces []uint64
var txGases []uint64
var txDatas []hexutil.Bytes
var txTypes []int
contractCreationBits := new(big.Int)
yParityBits := new(big.Int)
protectedBits := new(big.Int)
totalLegacyTxCount := uint64(0)
offset := sbtx.totalBlockTxCount
for idx := 0; idx < int(totalBlockTxCount); idx++ {
var tx types.Transaction
if err := tx.UnmarshalBinary(txs[idx]); err != nil {
return nil, errors.New("failed to decode tx")
return errors.New("failed to decode tx")
}
if tx.Type() == types.LegacyTxType {
protectedBit := uint(0)
if tx.Protected() {
protectedBit = uint(1)
}
protectedBits.SetBit(protectedBits, int(totalLegacyTxCount), protectedBit)
totalLegacyTxCount++
sbtx.protectedBits.SetBit(sbtx.protectedBits, int(sbtx.totalLegacyTxCount), protectedBit)
sbtx.totalLegacyTxCount++
}
if tx.Protected() && tx.ChainId().Cmp(chainID) != 0 {
return nil, fmt.Errorf("protected tx has chain ID %d, but expected chain ID %d", tx.ChainId(), chainID)
return fmt.Errorf("protected tx has chain ID %d, but expected chain ID %d", tx.ChainId(), chainID)
}
var txSig spanBatchSignature
v, r, s := tx.RawSignatureValues()
......@@ -430,42 +440,31 @@ func newSpanBatchTxs(txs [][]byte, chainID *big.Int) (*spanBatchTxs, error) {
txSig.v = v.Uint64()
txSig.r = R
txSig.s = S
txSigs = append(txSigs, txSig)
sbtx.txSigs = append(sbtx.txSigs, txSig)
contractCreationBit := uint(1)
if tx.To() != nil {
txTos = append(txTos, *tx.To())
sbtx.txTos = append(sbtx.txTos, *tx.To())
contractCreationBit = uint(0)
}
contractCreationBits.SetBit(contractCreationBits, idx, contractCreationBit)
sbtx.contractCreationBits.SetBit(sbtx.contractCreationBits, idx+int(offset), contractCreationBit)
yParityBit, err := convertVToYParity(txSig.v, int(tx.Type()))
if err != nil {
return nil, err
return err
}
yParityBits.SetBit(yParityBits, idx, yParityBit)
txNonces = append(txNonces, tx.Nonce())
txGases = append(txGases, tx.Gas())
sbtx.yParityBits.SetBit(sbtx.yParityBits, idx+int(offset), yParityBit)
sbtx.txNonces = append(sbtx.txNonces, tx.Nonce())
sbtx.txGases = append(sbtx.txGases, tx.Gas())
stx, err := newSpanBatchTx(tx)
if err != nil {
return nil, err
return err
}
txData, err := stx.MarshalBinary()
if err != nil {
return nil, err
return err
}
txDatas = append(txDatas, txData)
txTypes = append(txTypes, int(tx.Type()))
}
return &spanBatchTxs{
totalBlockTxCount: totalBlockTxCount,
contractCreationBits: contractCreationBits,
yParityBits: yParityBits,
txSigs: txSigs,
txNonces: txNonces,
txGases: txGases,
txTos: txTos,
txDatas: txDatas,
txTypes: txTypes,
protectedBits: protectedBits,
totalLegacyTxCount: totalLegacyTxCount,
}, nil
sbtx.txDatas = append(sbtx.txDatas, txData)
sbtx.txTypes = append(sbtx.txTypes, int(tx.Type()))
}
sbtx.totalBlockTxCount += totalBlockTxCount
return nil
}
......@@ -302,6 +302,31 @@ func TestSpanBatchTxsTxDatas(t *testing.T) {
require.Equal(t, txDatas, sbt.txDatas)
require.Equal(t, txTypes, sbt.txTypes)
}
func TestSpanBatchTxsAddTxs(t *testing.T) {
rng := rand.New(rand.NewSource(0x1234))
chainID := big.NewInt(rng.Int63n(1000))
// make batches to extract txs from
batches := RandomValidConsecutiveSingularBatches(rng, chainID)
allTxs := [][]byte{}
iterativeSBTX, err := newSpanBatchTxs([][]byte{}, chainID)
require.NoError(t, err)
for i := 0; i < len(batches); i++ {
// explicitly extract txs due to mismatch of [][]byte to []hexutil.Bytes
txs := [][]byte{}
for j := 0; j < len(batches[i].Transactions); j++ {
txs = append(txs, batches[i].Transactions[j])
}
err = iterativeSBTX.AddTxs(txs, chainID)
require.NoError(t, err)
allTxs = append(allTxs, txs...)
}
fullSBTX, err := newSpanBatchTxs(allTxs, chainID)
require.NoError(t, err)
require.Equal(t, iterativeSBTX, fullSBTX)
}
func TestSpanBatchTxsRecoverV(t *testing.T) {
rng := rand.New(rand.NewSource(0x123))
......
......@@ -23,8 +23,8 @@ type SpanChannelOut struct {
compress Compressor
// closed indicates if the channel is closed
closed bool
// spanBatchBuilder contains information requires to build SpanBatch
spanBatchBuilder *SpanBatchBuilder
// spanBatch is the batch being built
spanBatch *SpanBatch
// reader contains compressed data for making output frames
reader *bytes.Buffer
}
......@@ -33,14 +33,14 @@ func (co *SpanChannelOut) ID() ChannelID {
return co.id
}
func NewSpanChannelOut(compress Compressor, spanBatchBuilder *SpanBatchBuilder) (*SpanChannelOut, error) {
func NewSpanChannelOut(compress Compressor, spanBatch *SpanBatch) (*SpanChannelOut, error) {
c := &SpanChannelOut{
id: ChannelID{},
frame: 0,
rlpLength: 0,
compress: compress,
spanBatchBuilder: spanBatchBuilder,
reader: &bytes.Buffer{},
id: ChannelID{},
frame: 0,
rlpLength: 0,
compress: compress,
spanBatch: spanBatch,
reader: &bytes.Buffer{},
}
_, err := rand.Read(c.id[:])
if err != nil {
......@@ -56,7 +56,7 @@ func (co *SpanChannelOut) Reset() error {
co.compress.Reset()
co.reader.Reset()
co.closed = false
co.spanBatchBuilder.Reset()
co.spanBatch = NewSpanBatch(co.spanBatch.GenesisTimestamp, co.spanBatch.ChainID)
_, err := rand.Read(co.id[:])
return err
}
......@@ -100,9 +100,11 @@ func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch, seqNum uint64)
}
var buf bytes.Buffer
// Append Singular batch to its span batch builder
co.spanBatchBuilder.AppendSingularBatch(batch, seqNum)
if err := co.spanBatch.AppendSingularBatch(batch, seqNum); err != nil {
return 0, fmt.Errorf("failed to append SingularBatch to SpanBatch: %w", err)
}
// Convert Span batch to RawSpanBatch
rawSpanBatch, err := co.spanBatchBuilder.GetRawSpanBatch()
rawSpanBatch, err := co.spanBatch.ToRawSpanBatch()
if err != nil {
return 0, fmt.Errorf("failed to convert SpanBatch into RawSpanBatch: %w", err)
}
......@@ -117,12 +119,9 @@ func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch, seqNum uint64)
}
co.rlpLength = buf.Len()
if co.spanBatchBuilder.GetBlockCount() > 1 {
// Flush compressed data into reader to preserve current result.
// If the channel is full after this block is appended, we should use preserved data.
if err := co.compress.Flush(); err != nil {
return 0, fmt.Errorf("failed to flush compressor: %w", err)
}
// If the channel is full after this block is appended, we should use preserved data.
// so copy the compressed data to reader
if len(co.spanBatch.Batches) > 1 {
_, err = io.Copy(co.reader, co.compress)
if err != nil {
// Must reset reader to avoid partial output
......@@ -135,9 +134,13 @@ func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch, seqNum uint64)
co.compress.Reset()
// Avoid using io.Copy here, because we need all or nothing
written, err := co.compress.Write(buf.Bytes())
// Always flush (for BlindCompressor to check if it's full)
if err := co.compress.Flush(); err != nil {
return 0, fmt.Errorf("failed to flush compressor: %w", err)
}
if co.compress.FullErr() != nil {
err = co.compress.FullErr()
if co.spanBatchBuilder.GetBlockCount() == 1 {
if len(co.spanBatch.Batches) == 1 {
// Do not return ErrCompressorFull for the first block in the batch
// In this case, reader must be empty. then the contents of compressor will be copied to reader when the channel is closed.
err = nil
......
......@@ -3,6 +3,7 @@ package test
import (
"math/big"
"math/rand"
"time"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive"
......@@ -14,9 +15,8 @@ import (
// RandomL2Block returns a random block whose first transaction is a random pre-Ecotone upgrade
// L1 Info Deposit transaction.
func RandomL2Block(rng *rand.Rand, txCount int) (*types.Block, []*types.Receipt) {
l1Block := types.NewBlock(testutils.RandomHeader(rng),
nil, nil, nil, trie.NewStackTrie(nil))
func RandomL2Block(rng *rand.Rand, txCount int, t time.Time) (*types.Block, []*types.Receipt) {
l1Block := types.NewBlock(testutils.RandomHeader(rng), nil, nil, nil, trie.NewStackTrie(nil))
rollupCfg := rollup.Config{}
if testutils.RandomBool(rng) {
t := uint64(0)
......@@ -26,12 +26,21 @@ func RandomL2Block(rng *rand.Rand, txCount int) (*types.Block, []*types.Receipt)
if err != nil {
panic("L1InfoDeposit: " + err.Error())
}
return testutils.RandomBlockPrependTxs(rng, txCount, types.NewTx(l1InfoTx))
if t.IsZero() {
return testutils.RandomBlockPrependTxs(rng, txCount, types.NewTx(l1InfoTx))
} else {
return testutils.RandomBlockPrependTxsWithTime(rng, txCount, uint64(t.Unix()), types.NewTx(l1InfoTx))
}
}
func RandomL2BlockWithChainId(rng *rand.Rand, txCount int, chainId *big.Int) *types.Block {
return RandomL2BlockWithChainIdAndTime(rng, txCount, chainId, time.Time{})
}
func RandomL2BlockWithChainIdAndTime(rng *rand.Rand, txCount int, chainId *big.Int, t time.Time) *types.Block {
signer := types.NewLondonSigner(chainId)
block, _ := RandomL2Block(rng, 0)
block, _ := RandomL2Block(rng, 0, t)
txs := []*types.Transaction{block.Transactions()[0]} // L1 info deposit TX
for i := 0; i < txCount; i++ {
txs = append(txs, testutils.RandomTx(rng, big.NewInt(int64(rng.Uint32())), signer))
......
......@@ -250,7 +250,7 @@ func RandomReceipt(rng *rand.Rand, signer types.Signer, tx *types.Transaction, t
}
}
func RandomHeader(rng *rand.Rand) *types.Header {
func RandomHeaderWithTime(rng *rand.Rand, t uint64) *types.Header {
return &types.Header{
ParentHash: RandomHash(rng),
UncleHash: types.EmptyUncleHash,
......@@ -263,7 +263,7 @@ func RandomHeader(rng *rand.Rand) *types.Header {
Number: big.NewInt(1 + rng.Int63n(100_000_000)),
GasLimit: 0,
GasUsed: 0,
Time: uint64(rng.Int63n(2_000_000_000)),
Time: t,
Extra: RandomData(rng, rng.Intn(33)),
MixDigest: common.Hash{},
Nonce: types.BlockNonce{},
......@@ -271,15 +271,17 @@ func RandomHeader(rng *rand.Rand) *types.Header {
}
}
func RandomHeader(rng *rand.Rand) *types.Header {
t := uint64(rng.Int63n(2_000_000_000))
return RandomHeaderWithTime(rng, t)
}
func RandomBlock(rng *rand.Rand, txCount uint64) (*types.Block, []*types.Receipt) {
return RandomBlockPrependTxs(rng, int(txCount))
}
// RandomBlockPrependTxs returns a random block with txCount randomly generated
// transactions and additionally the transactions ptxs prepended. So the total
// number of transactions is len(ptxs) + txCount.
func RandomBlockPrependTxs(rng *rand.Rand, txCount int, ptxs ...*types.Transaction) (*types.Block, []*types.Receipt) {
header := RandomHeader(rng)
func RandomBlockPrependTxsWithTime(rng *rand.Rand, txCount int, t uint64, ptxs ...*types.Transaction) (*types.Block, []*types.Receipt) {
header := RandomHeaderWithTime(rng, t)
signer := types.NewLondonSigner(big.NewInt(rng.Int63n(1000)))
txs := make([]*types.Transaction, 0, txCount+len(ptxs))
txs = append(txs, ptxs...)
......@@ -312,6 +314,14 @@ func RandomBlockPrependTxs(rng *rand.Rand, txCount int, ptxs ...*types.Transacti
return block, receipts
}
// RandomBlockPrependTxs returns a random block with txCount randomly generated
// transactions and additionally the transactions ptxs prepended. So the total
// number of transactions is len(ptxs) + txCount.
func RandomBlockPrependTxs(rng *rand.Rand, txCount int, ptxs ...*types.Transaction) (*types.Block, []*types.Receipt) {
t := uint64(rng.Int63n(2_000_000_000))
return RandomBlockPrependTxsWithTime(rng, txCount, t, ptxs...)
}
func RandomOutputResponse(rng *rand.Rand) *eth.OutputResponse {
return &eth.OutputResponse{
Version: eth.Bytes32(RandomHash(rng)),
......
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