Commit f768a82e authored by protolambda's avatar protolambda Committed by GitHub

Merge pull request #7615 from ethereum-optimism/span-batch-types

op-node: Span Batch Type, Encoding, and Decoding
parents 0f57bb29 11213e33
...@@ -322,11 +322,11 @@ func FuzzSeqWindowClose(f *testing.F) { ...@@ -322,11 +322,11 @@ func FuzzSeqWindowClose(f *testing.F) {
// Check the timeout // Check the timeout
cb.timeout = timeout cb.timeout = timeout
cb.updateSwTimeout(&derive.BatchData{ cb.updateSwTimeout(derive.NewSingularBatchData(
BatchV1: derive.BatchV1{ derive.SingularBatch{
EpochNum: rollup.Epoch(epochNum), EpochNum: rollup.Epoch(epochNum),
}, },
}) ))
calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin
if timeout > calculatedTimeout && calculatedTimeout != 0 { if timeout > calculatedTimeout && calculatedTimeout != 0 {
cb.checkTimeout(calculatedTimeout) cb.checkTimeout(calculatedTimeout)
...@@ -354,11 +354,11 @@ func FuzzSeqWindowZeroTimeoutClose(f *testing.F) { ...@@ -354,11 +354,11 @@ func FuzzSeqWindowZeroTimeoutClose(f *testing.F) {
// Check the timeout // Check the timeout
cb.timeout = 0 cb.timeout = 0
cb.updateSwTimeout(&derive.BatchData{ cb.updateSwTimeout(derive.NewSingularBatchData(
BatchV1: derive.BatchV1{ derive.SingularBatch{
EpochNum: rollup.Epoch(epochNum), EpochNum: rollup.Epoch(epochNum),
}, },
}) ))
calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin
cb.checkTimeout(calculatedTimeout) cb.checkTimeout(calculatedTimeout)
if cb.timeout != 0 { if cb.timeout != 0 {
......
...@@ -253,7 +253,7 @@ func blockToBatch(block *types.Block) (*derive.BatchData, error) { ...@@ -253,7 +253,7 @@ func blockToBatch(block *types.Block) (*derive.BatchData, error) {
} }
return &derive.BatchData{ return &derive.BatchData{
BatchV1: derive.BatchV1{ SingularBatch: derive.SingularBatch{
ParentHash: block.ParentHash(), ParentHash: block.ParentHash(),
EpochNum: rollup.Epoch(l1Info.Number), EpochNum: rollup.Epoch(l1Info.Number),
EpochHash: l1Info.BlockHash, EpochHash: l1Info.BlockHash,
......
...@@ -9,6 +9,9 @@ import ( ...@@ -9,6 +9,9 @@ import (
"path" "path"
"sort" "sort"
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/cmd/batch_decoder/fetch" "github.com/ethereum-optimism/optimism/op-node/cmd/batch_decoder/fetch"
"github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-node/rollup/derive"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
...@@ -21,7 +24,7 @@ type ChannelWithMetadata struct { ...@@ -21,7 +24,7 @@ type ChannelWithMetadata struct {
InvalidFrames bool `json:"invalid_frames"` InvalidFrames bool `json:"invalid_frames"`
InvalidBatches bool `json:"invalid_batches"` InvalidBatches bool `json:"invalid_batches"`
Frames []FrameWithMetadata `json:"frames"` Frames []FrameWithMetadata `json:"frames"`
Batches []derive.BatchV1 `json:"batches"` Batches []derive.SingularBatch `json:"batches"`
} }
type FrameWithMetadata struct { type FrameWithMetadata struct {
...@@ -65,8 +68,9 @@ func Channels(config Config) { ...@@ -65,8 +68,9 @@ func Channels(config Config) {
for _, frame := range frames { for _, frame := range frames {
framesByChannel[frame.Frame.ID] = append(framesByChannel[frame.Frame.ID], frame) framesByChannel[frame.Frame.ID] = append(framesByChannel[frame.Frame.ID], frame)
} }
cfg := chaincfg.Mainnet
for id, frames := range framesByChannel { for id, frames := range framesByChannel {
ch := processFrames(id, frames) ch := processFrames(cfg, id, frames)
filename := path.Join(config.OutDirectory, fmt.Sprintf("%s.json", id.String())) filename := path.Join(config.OutDirectory, fmt.Sprintf("%s.json", id.String()))
if err := writeChannel(ch, filename); err != nil { if err := writeChannel(ch, filename); err != nil {
log.Fatal(err) log.Fatal(err)
...@@ -84,7 +88,7 @@ func writeChannel(ch ChannelWithMetadata, filename string) error { ...@@ -84,7 +88,7 @@ func writeChannel(ch ChannelWithMetadata, filename string) error {
return enc.Encode(ch) return enc.Encode(ch)
} }
func processFrames(id derive.ChannelID, frames []FrameWithMetadata) ChannelWithMetadata { func processFrames(cfg *rollup.Config, id derive.ChannelID, frames []FrameWithMetadata) ChannelWithMetadata {
ch := derive.NewChannel(id, eth.L1BlockRef{Number: frames[0].InclusionBlock}) ch := derive.NewChannel(id, eth.L1BlockRef{Number: frames[0].InclusionBlock})
invalidFrame := false invalidFrame := false
...@@ -100,17 +104,17 @@ func processFrames(id derive.ChannelID, frames []FrameWithMetadata) ChannelWithM ...@@ -100,17 +104,17 @@ func processFrames(id derive.ChannelID, frames []FrameWithMetadata) ChannelWithM
} }
} }
var batches []derive.BatchV1 var batches []derive.SingularBatch
invalidBatches := false invalidBatches := false
if ch.IsReady() { if ch.IsReady() {
br, err := derive.BatchReader(ch.Reader(), eth.L1BlockRef{}) br, err := derive.BatchReader(cfg, ch.Reader(), eth.L1BlockRef{})
if err == nil { if err == nil {
for batch, err := br(); err != io.EOF; batch, err = br() { for batch, err := br(); err != io.EOF; batch, err = br() {
if err != nil { if err != nil {
fmt.Printf("Error reading batch for channel %v. Err: %v\n", id.String(), err) fmt.Printf("Error reading batch for channel %v. Err: %v\n", id.String(), err)
invalidBatches = true invalidBatches = true
} else { } else {
batches = append(batches, batch.Batch.BatchV1) batches = append(batches, batch.Batch.SingularBatch)
} }
} }
} else { } else {
......
...@@ -42,13 +42,13 @@ func TestAttributesQueue(t *testing.T) { ...@@ -42,13 +42,13 @@ func TestAttributesQueue(t *testing.T) {
safeHead.L1Origin = l1Info.ID() safeHead.L1Origin = l1Info.ID()
safeHead.Time = l1Info.InfoTime safeHead.Time = l1Info.InfoTime
batch := &BatchData{BatchV1{ batch := NewSingularBatchData(SingularBatch{
ParentHash: safeHead.Hash, ParentHash: safeHead.Hash,
EpochNum: rollup.Epoch(l1Info.InfoNum), EpochNum: rollup.Epoch(l1Info.InfoNum),
EpochHash: l1Info.InfoHash, EpochHash: l1Info.InfoHash,
Timestamp: safeHead.Time + cfg.BlockTime, Timestamp: safeHead.Time + cfg.BlockTime,
Transactions: []eth.Data{eth.Data("foobar"), eth.Data("example")}, Transactions: []eth.Data{eth.Data("foobar"), eth.Data("example")},
}} })
parentL1Cfg := eth.SystemConfig{ parentL1Cfg := eth.SystemConfig{
BatcherAddr: common.Address{42}, BatcherAddr: common.Address{42},
......
...@@ -7,48 +7,44 @@ import ( ...@@ -7,48 +7,44 @@ import (
"io" "io"
"sync" "sync"
"github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
) )
// Batch format // Batch format
// first byte is type followed by bytestring. // first byte is type followed by bytestring.
// //
// BatchV1Type := 0
// batchV1 := BatchV1Type ++ RLP([epoch, timestamp, transaction_list]
//
// An empty input is not a valid batch. // An empty input is not a valid batch.
// //
// Note: the type system is based on L1 typed transactions. // Note: the type system is based on L1 typed transactions.
//
// encodeBufferPool holds temporary encoder buffers for batch encoding // encodeBufferPool holds temporary encoder buffers for batch encoding
var encodeBufferPool = sync.Pool{ var encodeBufferPool = sync.Pool{
New: func() any { return new(bytes.Buffer) }, New: func() any { return new(bytes.Buffer) },
} }
const ( const (
BatchV1Type = iota // SingularBatchType is the first version of Batch format, representing a single L2 block.
SingularBatchType = iota
// SpanBatchType is the Batch version used after SpanBatch hard fork, representing a span of L2 blocks.
SpanBatchType
) )
type BatchV1 struct { // Batch contains information to build one or multiple L2 blocks.
ParentHash common.Hash // parent L2 block hash // Batcher converts L2 blocks into Batch and writes encoded bytes to Channel.
EpochNum rollup.Epoch // aka l1 num // Derivation pipeline decodes Batch from Channel, and converts to one or multiple payload attributes.
EpochHash common.Hash // block hash type Batch interface {
Timestamp uint64 GetBatchType() int
// no feeRecipient address input, all fees go to a L2 contract GetTimestamp() uint64
Transactions []hexutil.Bytes LogContext(log.Logger) log.Logger
} }
// BatchData is a composition type that contains raw data of each batch version.
// It has encoding & decoding methods to implement typed encoding.
type BatchData struct { type BatchData struct {
BatchV1 BatchType int
// batches may contain additional data with new upgrades SingularBatch
} RawSpanBatch
func (b *BatchV1) Epoch() eth.BlockID {
return eth.BlockID{Hash: b.EpochHash, Number: uint64(b.EpochNum)}
} }
// EncodeRLP implements rlp.Encoder // EncodeRLP implements rlp.Encoder
...@@ -69,9 +65,18 @@ func (b *BatchData) MarshalBinary() ([]byte, error) { ...@@ -69,9 +65,18 @@ func (b *BatchData) MarshalBinary() ([]byte, error) {
return buf.Bytes(), err return buf.Bytes(), err
} }
// encodeTyped encodes batch type and payload for each batch type.
func (b *BatchData) encodeTyped(buf *bytes.Buffer) error { func (b *BatchData) encodeTyped(buf *bytes.Buffer) error {
buf.WriteByte(BatchV1Type) switch b.BatchType {
return rlp.Encode(buf, &b.BatchV1) case SingularBatchType:
buf.WriteByte(SingularBatchType)
return rlp.Encode(buf, &b.SingularBatch)
case SpanBatchType:
buf.WriteByte(SpanBatchType)
return b.RawSpanBatch.encode(buf)
default:
return fmt.Errorf("unrecognized batch type: %d", b.BatchType)
}
} }
// DecodeRLP implements rlp.Decoder // DecodeRLP implements rlp.Decoder
...@@ -94,14 +99,35 @@ func (b *BatchData) UnmarshalBinary(data []byte) error { ...@@ -94,14 +99,35 @@ func (b *BatchData) UnmarshalBinary(data []byte) error {
return b.decodeTyped(data) return b.decodeTyped(data)
} }
// decodeTyped decodes batch type and payload for each batch type.
func (b *BatchData) decodeTyped(data []byte) error { func (b *BatchData) decodeTyped(data []byte) error {
if len(data) == 0 { if len(data) == 0 {
return fmt.Errorf("batch too short") return fmt.Errorf("batch too short")
} }
switch data[0] { switch data[0] {
case BatchV1Type: case SingularBatchType:
return rlp.DecodeBytes(data[1:], &b.BatchV1) b.BatchType = SingularBatchType
return rlp.DecodeBytes(data[1:], &b.SingularBatch)
case SpanBatchType:
b.BatchType = int(data[0])
return b.RawSpanBatch.decodeBytes(data[1:])
default: default:
return fmt.Errorf("unrecognized batch type: %d", data[0]) return fmt.Errorf("unrecognized batch type: %d", data[0])
} }
} }
// NewSingularBatchData creates new BatchData with SingularBatch
func NewSingularBatchData(singularBatch SingularBatch) *BatchData {
return &BatchData{
BatchType: SingularBatchType,
SingularBatch: singularBatch,
}
}
// NewSpanBatchData creates new BatchData with SpanBatch
func NewSpanBatchData(spanBatch RawSpanBatch) *BatchData {
return &BatchData{
BatchType: SpanBatchType,
RawSpanBatch: spanBatch,
}
}
...@@ -243,15 +243,15 @@ batchLoop: ...@@ -243,15 +243,15 @@ batchLoop:
// batch to ensure that we at least have one batch per epoch. // batch to ensure that we at least have one batch per epoch.
if nextTimestamp < nextEpoch.Time || firstOfEpoch { if nextTimestamp < nextEpoch.Time || firstOfEpoch {
bq.log.Info("Generating next batch", "epoch", epoch, "timestamp", nextTimestamp) bq.log.Info("Generating next batch", "epoch", epoch, "timestamp", nextTimestamp)
return &BatchData{ return NewSingularBatchData(
BatchV1{ SingularBatch{
ParentHash: l2SafeHead.Hash, ParentHash: l2SafeHead.Hash,
EpochNum: rollup.Epoch(epoch.Number), EpochNum: rollup.Epoch(epoch.Number),
EpochHash: epoch.Hash, EpochHash: epoch.Hash,
Timestamp: nextTimestamp, Timestamp: nextTimestamp,
Transactions: nil, Transactions: nil,
}, },
}, nil ), nil
} }
// At this point we have auto generated every batch for the current epoch // At this point we have auto generated every batch for the current epoch
......
...@@ -48,13 +48,13 @@ func mockHash(time uint64, layer uint8) common.Hash { ...@@ -48,13 +48,13 @@ func mockHash(time uint64, layer uint8) common.Hash {
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)
return &BatchData{BatchV1{ return NewSingularBatchData(SingularBatch{
ParentHash: mockHash(timestamp-2, 2), ParentHash: mockHash(timestamp-2, 2),
Timestamp: timestamp, Timestamp: timestamp,
EpochNum: rollup.Epoch(epoch.Number), EpochNum: rollup.Epoch(epoch.Number),
EpochHash: epoch.Hash, EpochHash: epoch.Hash,
Transactions: []hexutil.Bytes{data}, Transactions: []hexutil.Bytes{data},
}} })
} }
func L1Chain(l1Times []uint64) []eth.L1BlockRef { func L1Chain(l1Times []uint64) []eth.L1BlockRef {
...@@ -331,7 +331,7 @@ func TestBatchQueueMissing(t *testing.T) { ...@@ -331,7 +331,7 @@ func TestBatchQueueMissing(t *testing.T) {
b, e = bq.NextBatch(context.Background(), safeHead) b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e) require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(12)) require.Equal(t, b.Timestamp, uint64(12))
require.Empty(t, b.BatchV1.Transactions) require.Empty(t, b.SingularBatch.Transactions)
require.Equal(t, rollup.Epoch(0), b.EpochNum) require.Equal(t, rollup.Epoch(0), b.EpochNum)
safeHead.Number += 1 safeHead.Number += 1
safeHead.Time += 2 safeHead.Time += 2
...@@ -341,7 +341,7 @@ func TestBatchQueueMissing(t *testing.T) { ...@@ -341,7 +341,7 @@ func TestBatchQueueMissing(t *testing.T) {
b, e = bq.NextBatch(context.Background(), safeHead) b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e) require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(14)) require.Equal(t, b.Timestamp, uint64(14))
require.Empty(t, b.BatchV1.Transactions) require.Empty(t, b.SingularBatch.Transactions)
require.Equal(t, rollup.Epoch(0), b.EpochNum) require.Equal(t, rollup.Epoch(0), b.EpochNum)
safeHead.Number += 1 safeHead.Number += 1
safeHead.Time += 2 safeHead.Time += 2
...@@ -367,6 +367,6 @@ func TestBatchQueueMissing(t *testing.T) { ...@@ -367,6 +367,6 @@ func TestBatchQueueMissing(t *testing.T) {
b, e = bq.NextBatch(context.Background(), safeHead) b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e) require.Nil(t, e)
require.Equal(t, b.Timestamp, uint64(18)) require.Equal(t, b.Timestamp, uint64(18))
require.Empty(t, b.BatchV1.Transactions) require.Empty(t, b.SingularBatch.Transactions)
require.Equal(t, rollup.Epoch(1), b.EpochNum) require.Equal(t, rollup.Epoch(1), b.EpochNum)
} }
package derive package derive
import ( import (
"bytes"
"math/big"
"math/rand"
"testing" "testing"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/testutils"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"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/core/types"
"github.com/ethereum/go-ethereum/rlp"
) )
func RandomRawSpanBatch(rng *rand.Rand, chainId *big.Int) *RawSpanBatch {
blockCount := uint64(1 + rng.Int()&0xFF)
originBits := new(big.Int)
for i := 0; i < int(blockCount); i++ {
bit := uint(0)
if testutils.RandomBool(rng) {
bit = uint(1)
}
originBits.SetBit(originBits, i, bit)
}
var blockTxCounts []uint64
totalblockTxCounts := uint64(0)
for i := 0; i < int(blockCount); i++ {
blockTxCount := uint64(rng.Intn(16))
blockTxCounts = append(blockTxCounts, blockTxCount)
totalblockTxCounts += blockTxCount
}
signer := types.NewLondonSigner(chainId)
var txs [][]byte
for i := 0; i < int(totalblockTxCounts); i++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
rawTx, err := tx.MarshalBinary()
if err != nil {
panic("MarshalBinary:" + err.Error())
}
txs = append(txs, rawTx)
}
spanBatchTxs, err := newSpanBatchTxs(txs, chainId)
if err != nil {
panic(err.Error())
}
rawSpanBatch := RawSpanBatch{
spanBatchPrefix: spanBatchPrefix{
relTimestamp: uint64(rng.Uint32()),
l1OriginNum: rng.Uint64(),
parentCheck: testutils.RandomData(rng, 20),
l1OriginCheck: testutils.RandomData(rng, 20),
},
spanBatchPayload: spanBatchPayload{
blockCount: blockCount,
originBits: originBits,
blockTxCounts: blockTxCounts,
txs: spanBatchTxs,
},
}
return &rawSpanBatch
}
func RandomSingularBatch(rng *rand.Rand, txCount int, chainID *big.Int) *SingularBatch {
signer := types.NewLondonSigner(chainID)
baseFee := big.NewInt(rng.Int63n(300_000_000_000))
txsEncoded := make([]hexutil.Bytes, 0, txCount)
// force each tx to have equal chainID
for i := 0; i < txCount; i++ {
tx := testutils.RandomTx(rng, baseFee, signer)
txEncoded, err := tx.MarshalBinary()
if err != nil {
panic("tx Marshal binary" + err.Error())
}
txsEncoded = append(txsEncoded, hexutil.Bytes(txEncoded))
}
return &SingularBatch{
ParentHash: testutils.RandomHash(rng),
EpochNum: rollup.Epoch(1 + rng.Int63n(100_000_000)),
EpochHash: testutils.RandomHash(rng),
Timestamp: uint64(rng.Int63n(2_000_000_000)),
Transactions: txsEncoded,
}
}
func RandomValidConsecutiveSingularBatches(rng *rand.Rand, chainID *big.Int) []*SingularBatch {
blockCount := 2 + rng.Intn(128)
l2BlockTime := uint64(2)
var singularBatches []*SingularBatch
for i := 0; i < blockCount; i++ {
singularBatch := RandomSingularBatch(rng, 1+rng.Intn(8), chainID)
singularBatches = append(singularBatches, singularBatch)
}
l1BlockNum := rng.Uint64()
// make sure oldest timestamp is large enough
singularBatches[0].Timestamp += 256
for i := 0; i < blockCount; i++ {
originChangedBit := rng.Intn(2)
if originChangedBit == 1 {
l1BlockNum++
singularBatches[i].EpochHash = testutils.RandomHash(rng)
} else if i > 0 {
singularBatches[i].EpochHash = singularBatches[i-1].EpochHash
}
singularBatches[i].EpochNum = rollup.Epoch(l1BlockNum)
if i > 0 {
singularBatches[i].Timestamp = singularBatches[i-1].Timestamp + l2BlockTime
}
}
return singularBatches
}
func mockL1Origin(rng *rand.Rand, rawSpanBatch *RawSpanBatch, singularBatches []*SingularBatch) []eth.L1BlockRef {
safeHeadOrigin := testutils.RandomBlockRef(rng)
safeHeadOrigin.Hash = singularBatches[0].EpochHash
safeHeadOrigin.Number = uint64(singularBatches[0].EpochNum)
l1Origins := []eth.L1BlockRef{safeHeadOrigin}
originBitSum := uint64(0)
for i := 0; i < int(rawSpanBatch.blockCount); i++ {
if rawSpanBatch.originBits.Bit(i) == 1 {
l1Origin := testutils.NextRandomRef(rng, l1Origins[originBitSum])
originBitSum++
l1Origin.Hash = singularBatches[i].EpochHash
l1Origin.Number = uint64(singularBatches[i].EpochNum)
l1Origins = append(l1Origins, l1Origin)
}
}
return l1Origins
}
func TestBatchRoundTrip(t *testing.T) { func TestBatchRoundTrip(t *testing.T) {
rng := rand.New(rand.NewSource(0xdeadbeef))
blockTime := uint64(2)
genesisTimestamp := uint64(0)
chainID := new(big.Int).SetUint64(rng.Uint64())
batches := []*BatchData{ batches := []*BatchData{
{ {
BatchV1: BatchV1{ SingularBatch: SingularBatch{
ParentHash: common.Hash{}, ParentHash: common.Hash{},
EpochNum: 0, EpochNum: 0,
Timestamp: 0, Timestamp: 0,
...@@ -20,13 +150,18 @@ func TestBatchRoundTrip(t *testing.T) { ...@@ -20,13 +150,18 @@ func TestBatchRoundTrip(t *testing.T) {
}, },
}, },
{ {
BatchV1: BatchV1{ SingularBatch: SingularBatch{
ParentHash: common.Hash{31: 0x42}, ParentHash: common.Hash{31: 0x42},
EpochNum: 1, EpochNum: 1,
Timestamp: 1647026951, Timestamp: 1647026951,
Transactions: []hexutil.Bytes{[]byte{0, 0, 0}, []byte{0x76, 0xfd, 0x7c}}, Transactions: []hexutil.Bytes{[]byte{0, 0, 0}, []byte{0x76, 0xfd, 0x7c}},
}, },
}, },
NewSingularBatchData(*RandomSingularBatch(rng, 5, chainID)),
NewSingularBatchData(*RandomSingularBatch(rng, 7, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
} }
for i, batch := range batches { for i, batch := range batches {
...@@ -35,6 +170,58 @@ func TestBatchRoundTrip(t *testing.T) { ...@@ -35,6 +170,58 @@ func TestBatchRoundTrip(t *testing.T) {
var dec BatchData var dec BatchData
err = dec.UnmarshalBinary(enc) err = dec.UnmarshalBinary(enc)
assert.NoError(t, err) assert.NoError(t, err)
if dec.BatchType == SpanBatchType {
_, err := dec.RawSpanBatch.derive(blockTime, genesisTimestamp, chainID)
assert.NoError(t, err)
}
assert.Equal(t, batch, &dec, "Batch not equal test case %v", i)
}
}
func TestBatchRoundTripRLP(t *testing.T) {
rng := rand.New(rand.NewSource(0xbeefdead))
blockTime := uint64(2)
genesisTimestamp := uint64(0)
chainID := new(big.Int).SetUint64(rng.Uint64())
batches := []*BatchData{
{
SingularBatch: SingularBatch{
ParentHash: common.Hash{},
EpochNum: 0,
Timestamp: 0,
Transactions: []hexutil.Bytes{},
},
},
{
SingularBatch: SingularBatch{
ParentHash: common.Hash{31: 0x42},
EpochNum: 1,
Timestamp: 1647026951,
Transactions: []hexutil.Bytes{[]byte{0, 0, 0}, []byte{0x76, 0xfd, 0x7c}},
},
},
NewSingularBatchData(*RandomSingularBatch(rng, 5, chainID)),
NewSingularBatchData(*RandomSingularBatch(rng, 7, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
NewSpanBatchData(*RandomRawSpanBatch(rng, chainID)),
}
for i, batch := range batches {
var buf bytes.Buffer
err := batch.EncodeRLP(&buf)
assert.NoError(t, err)
result := buf.Bytes()
var dec BatchData
r := bytes.NewReader(result)
s := rlp.NewStream(r, 0)
err = dec.DecodeRLP(s)
assert.NoError(t, err)
if dec.BatchType == SpanBatchType {
_, err := dec.RawSpanBatch.derive(blockTime, genesisTimestamp, chainID)
assert.NoError(t, err)
}
assert.Equal(t, batch, &dec, "Batch not equal test case %v", i) assert.Equal(t, batch, &dec, "Batch not equal test case %v", i)
} }
} }
...@@ -10,6 +10,7 @@ import ( ...@@ -10,6 +10,7 @@ import (
// FuzzBatchRoundTrip executes a fuzz test similar to TestBatchRoundTrip, which tests that arbitrary BatchData will be // FuzzBatchRoundTrip executes a fuzz test similar to TestBatchRoundTrip, which tests that arbitrary BatchData will be
// encoded and decoded without loss of its original values. // encoded and decoded without loss of its original values.
// Does not test the span batch type because the fuzzer is not aware of the structure of a span batch.
func FuzzBatchRoundTrip(f *testing.F) { func FuzzBatchRoundTrip(f *testing.F) {
f.Fuzz(func(t *testing.T, fuzzedData []byte) { f.Fuzz(func(t *testing.T, fuzzedData []byte) {
// Create our fuzzer wrapper to generate complex values // Create our fuzzer wrapper to generate complex values
...@@ -20,6 +21,10 @@ func FuzzBatchRoundTrip(f *testing.F) { ...@@ -20,6 +21,10 @@ func FuzzBatchRoundTrip(f *testing.F) {
var batchData BatchData var batchData BatchData
typeProvider.Fuzz(&batchData) typeProvider.Fuzz(&batchData)
// force batchdata to only contain singular batch
batchData.BatchType = SingularBatchType
batchData.RawSpanBatch = RawSpanBatch{}
// Encode our batch data // Encode our batch data
enc, err := batchData.MarshalBinary() enc, err := batchData.MarshalBinary()
require.NoError(t, err) require.NoError(t, err)
......
This diff is collapsed.
...@@ -6,6 +6,8 @@ import ( ...@@ -6,6 +6,8 @@ import (
"fmt" "fmt"
"io" "io"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
) )
...@@ -144,7 +146,7 @@ func (ch *Channel) Reader() io.Reader { ...@@ -144,7 +146,7 @@ func (ch *Channel) Reader() io.Reader {
// BatchReader provides a function that iteratively consumes batches from the reader. // BatchReader provides a function that iteratively consumes batches from the reader.
// The L1Inclusion block is also provided at creation time. // The L1Inclusion block is also provided at creation time.
func BatchReader(r io.Reader, l1InclusionBlock eth.L1BlockRef) (func() (BatchWithL1InclusionBlock, error), error) { func BatchReader(cfg *rollup.Config, r io.Reader, l1InclusionBlock eth.L1BlockRef) (func() (BatchWithL1InclusionBlock, error), error) {
// Setup decompressor stage + RLP reader // Setup decompressor stage + RLP reader
zr, err := zlib.NewReader(r) zr, err := zlib.NewReader(r)
if err != nil { if err != nil {
...@@ -157,6 +159,12 @@ func BatchReader(r io.Reader, l1InclusionBlock eth.L1BlockRef) (func() (BatchWit ...@@ -157,6 +159,12 @@ func BatchReader(r io.Reader, l1InclusionBlock eth.L1BlockRef) (func() (BatchWit
L1InclusionBlock: l1InclusionBlock, L1InclusionBlock: l1InclusionBlock,
} }
err := rlpReader.Decode(&ret.Batch) err := rlpReader.Decode(&ret.Batch)
if err != nil {
return ret, err return ret, err
}
if ret.Batch.BatchType == SpanBatchType && !cfg.IsSpanBatch(ret.L1InclusionBlock.Time) {
return ret, fmt.Errorf("cannot accept span-batch in L1 block with time %d", ret.L1InclusionBlock.Time)
}
return ret, nil
}, nil }, nil
} }
...@@ -5,6 +5,8 @@ import ( ...@@ -5,6 +5,8 @@ import (
"context" "context"
"io" "io"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
...@@ -17,6 +19,8 @@ import ( ...@@ -17,6 +19,8 @@ import (
type ChannelInReader struct { type ChannelInReader struct {
log log.Logger log log.Logger
cfg *rollup.Config
nextBatchFn func() (BatchWithL1InclusionBlock, error) nextBatchFn func() (BatchWithL1InclusionBlock, error)
prev *ChannelBank prev *ChannelBank
...@@ -27,8 +31,9 @@ type ChannelInReader struct { ...@@ -27,8 +31,9 @@ type ChannelInReader struct {
var _ ResettableStage = (*ChannelInReader)(nil) var _ ResettableStage = (*ChannelInReader)(nil)
// NewChannelInReader creates a ChannelInReader, which should be Reset(origin) before use. // NewChannelInReader creates a ChannelInReader, which should be Reset(origin) before use.
func NewChannelInReader(log log.Logger, prev *ChannelBank, metrics Metrics) *ChannelInReader { func NewChannelInReader(cfg *rollup.Config, log log.Logger, prev *ChannelBank, metrics Metrics) *ChannelInReader {
return &ChannelInReader{ return &ChannelInReader{
cfg: cfg,
log: log, log: log,
prev: prev, prev: prev,
metrics: metrics, metrics: metrics,
...@@ -41,7 +46,7 @@ func (cr *ChannelInReader) Origin() eth.L1BlockRef { ...@@ -41,7 +46,7 @@ func (cr *ChannelInReader) Origin() eth.L1BlockRef {
// TODO: Take full channel for better logging // TODO: Take full channel for better logging
func (cr *ChannelInReader) WriteChannel(data []byte) error { func (cr *ChannelInReader) WriteChannel(data []byte) error {
if f, err := BatchReader(bytes.NewBuffer(data), cr.Origin()); err == nil { if f, err := BatchReader(cr.cfg, bytes.NewBuffer(data), cr.Origin()); err == nil {
cr.nextBatchFn = f cr.nextBatchFn = f
cr.metrics.RecordChannelInputBytes(len(data)) cr.metrics.RecordChannelInputBytes(len(data))
return nil return nil
......
...@@ -238,15 +238,15 @@ func BlockToBatch(block *types.Block) (*BatchData, L1BlockInfo, error) { ...@@ -238,15 +238,15 @@ func BlockToBatch(block *types.Block) (*BatchData, L1BlockInfo, error) {
return nil, l1Info, fmt.Errorf("could not parse the L1 Info deposit: %w", err) return nil, l1Info, fmt.Errorf("could not parse the L1 Info deposit: %w", err)
} }
return &BatchData{ return NewSingularBatchData(
BatchV1{ SingularBatch{
ParentHash: block.ParentHash(), ParentHash: block.ParentHash(),
EpochNum: rollup.Epoch(l1Info.Number), EpochNum: rollup.Epoch(l1Info.Number),
EpochHash: l1Info.BlockHash, EpochHash: l1Info.BlockHash,
Timestamp: block.Time(), Timestamp: block.Time(),
Transactions: opaqueTxs, Transactions: opaqueTxs,
}, },
}, l1Info, nil ), l1Info, nil
} }
// ForceCloseTxData generates the transaction data for a transaction which will force close // ForceCloseTxData generates the transaction data for a transaction which will force close
......
...@@ -19,6 +19,11 @@ func frameSize(frame Frame) uint64 { ...@@ -19,6 +19,11 @@ func frameSize(frame Frame) uint64 {
const DerivationVersion0 = 0 const DerivationVersion0 = 0
// MaxSpanBatchFieldSize is the maximum amount of bytes that will be read from
// a span batch to decode span batch field. This value cannot be larger than
// MaxRLPBytesPerChannel because single batch cannot be larger than channel size.
const MaxSpanBatchFieldSize = 10_000_000
// MaxChannelBankSize is the amount of memory space, in number of bytes, // MaxChannelBankSize is the amount of memory space, in number of bytes,
// till the bank is pruned by removing channels, // till the bank is pruned by removing channels,
// starting with the oldest channel. // starting with the oldest channel.
......
...@@ -89,7 +89,7 @@ func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetch ...@@ -89,7 +89,7 @@ func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetch
l1Src := NewL1Retrieval(log, dataSrc, l1Traversal) l1Src := NewL1Retrieval(log, dataSrc, l1Traversal)
frameQueue := NewFrameQueue(log, l1Src) frameQueue := NewFrameQueue(log, l1Src)
bank := NewChannelBank(log, cfg, frameQueue, l1Fetcher, metrics) bank := NewChannelBank(log, cfg, frameQueue, l1Fetcher, metrics)
chInReader := NewChannelInReader(log, bank, metrics) chInReader := NewChannelInReader(cfg, log, bank, metrics)
batchQueue := NewBatchQueue(log, cfg, chInReader) batchQueue := NewBatchQueue(log, cfg, chInReader)
attrBuilder := NewFetchingAttributesBuilder(cfg, l1Fetcher, engine) attrBuilder := NewFetchingAttributesBuilder(cfg, l1Fetcher, engine)
attributesQueue := NewAttributesQueue(log, cfg, attrBuilder, batchQueue) attributesQueue := NewAttributesQueue(log, cfg, attrBuilder, batchQueue)
......
package derive
import (
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/log"
)
// Batch format
//
// SingularBatchType := 0
// singularBatch := SingularBatchType ++ RLP([parent_hash, epoch_number, epoch_hash, timestamp, transaction_list])
// SingularBatch is an implementation of Batch interface, containing the input to build one L2 block.
type SingularBatch struct {
ParentHash common.Hash // parent L2 block hash
EpochNum rollup.Epoch // aka l1 num
EpochHash common.Hash // l1 block hash
Timestamp uint64
Transactions []hexutil.Bytes
}
// GetBatchType returns its batch type (batch_version)
func (b *SingularBatch) GetBatchType() int {
return SingularBatchType
}
// GetTimestamp returns its block timestamp
func (b *SingularBatch) GetTimestamp() uint64 {
return b.Timestamp
}
// GetEpochNum returns its epoch number (L1 origin block number)
func (b *SingularBatch) GetEpochNum() rollup.Epoch {
return b.EpochNum
}
// LogContext creates a new log context that contains information of the batch
func (b *SingularBatch) LogContext(log log.Logger) log.Logger {
return log.New(
"batch_timestamp", b.Timestamp,
"parent_hash", b.ParentHash,
"batch_epoch", b.Epoch(),
"txs", len(b.Transactions),
)
}
// Epoch returns a BlockID of its L1 origin.
func (b *SingularBatch) Epoch() eth.BlockID {
return eth.BlockID{Hash: b.EpochHash, Number: uint64(b.EpochNum)}
}
package derive
import (
"math/big"
"math/rand"
"testing"
"github.com/stretchr/testify/assert"
)
func TestSingularBatchForBatchInterface(t *testing.T) {
rng := rand.New(rand.NewSource(0x543331))
chainID := big.NewInt(rng.Int63n(1000))
txCount := 1 + rng.Intn(8)
singularBatch := RandomSingularBatch(rng, txCount, chainID)
assert.Equal(t, SingularBatchType, singularBatch.GetBatchType())
assert.Equal(t, singularBatch.Timestamp, singularBatch.GetTimestamp())
assert.Equal(t, singularBatch.EpochNum, singularBatch.GetEpochNum())
}
This diff is collapsed.
This diff is collapsed.
package derive
import (
"bytes"
"errors"
"fmt"
"io"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
)
type spanBatchTxData interface {
txType() byte // returns the type ID
}
type spanBatchTx struct {
inner spanBatchTxData
}
type spanBatchLegacyTxData struct {
Value *big.Int // wei amount
GasPrice *big.Int // wei per gas
Data []byte
}
func (txData *spanBatchLegacyTxData) txType() byte { return types.LegacyTxType }
type spanBatchAccessListTxData struct {
Value *big.Int // wei amount
GasPrice *big.Int // wei per gas
Data []byte
AccessList types.AccessList // EIP-2930 access list
}
func (txData *spanBatchAccessListTxData) txType() byte { return types.AccessListTxType }
type spanBatchDynamicFeeTxData struct {
Value *big.Int
GasTipCap *big.Int // a.k.a. maxPriorityFeePerGas
GasFeeCap *big.Int // a.k.a. maxFeePerGas
Data []byte
AccessList types.AccessList
}
func (txData *spanBatchDynamicFeeTxData) txType() byte { return types.DynamicFeeTxType }
// Type returns the transaction type.
func (tx *spanBatchTx) Type() uint8 {
return tx.inner.txType()
}
// encodeTyped writes the canonical encoding of a typed transaction to w.
func (tx *spanBatchTx) encodeTyped(w *bytes.Buffer) error {
w.WriteByte(tx.Type())
return rlp.Encode(w, tx.inner)
}
// MarshalBinary returns the canonical encoding of the transaction.
// For legacy transactions, it returns the RLP encoding. For EIP-2718 typed
// transactions, it returns the type and payload.
func (tx *spanBatchTx) MarshalBinary() ([]byte, error) {
if tx.Type() == types.LegacyTxType {
return rlp.EncodeToBytes(tx.inner)
}
var buf bytes.Buffer
err := tx.encodeTyped(&buf)
return buf.Bytes(), err
}
// EncodeRLP implements rlp.Encoder
func (tx *spanBatchTx) EncodeRLP(w io.Writer) error {
if tx.Type() == types.LegacyTxType {
return rlp.Encode(w, tx.inner)
}
// It's an EIP-2718 typed TX envelope.
buf := encodeBufferPool.Get().(*bytes.Buffer)
defer encodeBufferPool.Put(buf)
buf.Reset()
if err := tx.encodeTyped(buf); err != nil {
return err
}
return rlp.Encode(w, buf.Bytes())
}
// setDecoded sets the inner transaction after decoding.
func (tx *spanBatchTx) setDecoded(inner spanBatchTxData, size uint64) {
tx.inner = inner
}
// decodeTyped decodes a typed transaction from the canonical format.
func (tx *spanBatchTx) decodeTyped(b []byte) (spanBatchTxData, error) {
if len(b) <= 1 {
return nil, errors.New("typed transaction too short")
}
switch b[0] {
case types.AccessListTxType:
var inner spanBatchAccessListTxData
err := rlp.DecodeBytes(b[1:], &inner)
if err != nil {
return nil, fmt.Errorf("failed to decode spanBatchAccessListTxData: %w", err)
}
return &inner, nil
case types.DynamicFeeTxType:
var inner spanBatchDynamicFeeTxData
err := rlp.DecodeBytes(b[1:], &inner)
if err != nil {
return nil, fmt.Errorf("failed to decode spanBatchDynamicFeeTxData: %w", err)
}
return &inner, nil
default:
return nil, types.ErrTxTypeNotSupported
}
}
// DecodeRLP implements rlp.Decoder
func (tx *spanBatchTx) DecodeRLP(s *rlp.Stream) error {
kind, size, err := s.Kind()
switch {
case err != nil:
return err
case kind == rlp.List:
// It's a legacy transaction.
var inner spanBatchLegacyTxData
err = s.Decode(&inner)
if err != nil {
return fmt.Errorf("failed to decode spanBatchLegacyTxData: %w", err)
}
tx.setDecoded(&inner, rlp.ListSize(size))
return nil
default:
// It's an EIP-2718 typed TX envelope.
var b []byte
if b, err = s.Bytes(); err != nil {
return err
}
inner, err := tx.decodeTyped(b)
if err != nil {
return err
}
tx.setDecoded(inner, uint64(len(b)))
return nil
}
}
// UnmarshalBinary decodes the canonical encoding of transactions.
// It supports legacy RLP transactions and EIP2718 typed transactions.
func (tx *spanBatchTx) UnmarshalBinary(b []byte) error {
if len(b) > 0 && b[0] > 0x7f {
// It's a legacy transaction.
var data spanBatchLegacyTxData
err := rlp.DecodeBytes(b, &data)
if err != nil {
return fmt.Errorf("failed to decode spanBatchLegacyTxData: %w", err)
}
tx.setDecoded(&data, uint64(len(b)))
return nil
}
// It's an EIP2718 typed transaction envelope.
inner, err := tx.decodeTyped(b)
if err != nil {
return err
}
tx.setDecoded(inner, uint64(len(b)))
return nil
}
// convertToFullTx takes values and convert spanBatchTx to types.Transaction
func (tx *spanBatchTx) convertToFullTx(nonce, gas uint64, to *common.Address, chainID, V, R, S *big.Int) (*types.Transaction, error) {
var inner types.TxData
switch tx.Type() {
case types.LegacyTxType:
batchTxInner := tx.inner.(*spanBatchLegacyTxData)
inner = &types.LegacyTx{
Nonce: nonce,
GasPrice: batchTxInner.GasPrice,
Gas: gas,
To: to,
Value: batchTxInner.Value,
Data: batchTxInner.Data,
V: V,
R: R,
S: S,
}
case types.AccessListTxType:
batchTxInner := tx.inner.(*spanBatchAccessListTxData)
inner = &types.AccessListTx{
ChainID: chainID,
Nonce: nonce,
GasPrice: batchTxInner.GasPrice,
Gas: gas,
To: to,
Value: batchTxInner.Value,
Data: batchTxInner.Data,
AccessList: batchTxInner.AccessList,
V: V,
R: R,
S: S,
}
case types.DynamicFeeTxType:
batchTxInner := tx.inner.(*spanBatchDynamicFeeTxData)
inner = &types.DynamicFeeTx{
ChainID: chainID,
Nonce: nonce,
GasTipCap: batchTxInner.GasTipCap,
GasFeeCap: batchTxInner.GasFeeCap,
Gas: gas,
To: to,
Value: batchTxInner.Value,
Data: batchTxInner.Data,
AccessList: batchTxInner.AccessList,
V: V,
R: R,
S: S,
}
default:
return nil, fmt.Errorf("invalid tx type: %d", tx.Type())
}
return types.NewTx(inner), nil
}
// newSpanBatchTx converts types.Transaction to spanBatchTx
func newSpanBatchTx(tx types.Transaction) (*spanBatchTx, error) {
var inner spanBatchTxData
switch tx.Type() {
case types.LegacyTxType:
inner = &spanBatchLegacyTxData{
GasPrice: tx.GasPrice(),
Value: tx.Value(),
Data: tx.Data(),
}
case types.AccessListTxType:
inner = &spanBatchAccessListTxData{
GasPrice: tx.GasPrice(),
Value: tx.Value(),
Data: tx.Data(),
AccessList: tx.AccessList(),
}
case types.DynamicFeeTxType:
inner = &spanBatchDynamicFeeTxData{
GasTipCap: tx.GasTipCap(),
GasFeeCap: tx.GasFeeCap(),
Value: tx.Value(),
Data: tx.Data(),
AccessList: tx.AccessList(),
}
default:
return nil, fmt.Errorf("invalid tx type: %d", tx.Type())
}
return &spanBatchTx{inner: inner}, nil
}
package derive
import (
"bytes"
"math/big"
"math/rand"
"testing"
"github.com/ethereum-optimism/optimism/op-service/testutils"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/stretchr/testify/assert"
)
func TestSpanBatchTxConvert(t *testing.T) {
rng := rand.New(rand.NewSource(0x1331))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
m := make(map[byte]int)
for i := 0; i < 32; i++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
m[tx.Type()] += 1
v, r, s := tx.RawSignatureValues()
sbtx, err := newSpanBatchTx(*tx)
assert.NoError(t, err)
tx2, err := sbtx.convertToFullTx(tx.Nonce(), tx.Gas(), tx.To(), chainID, v, r, s)
assert.NoError(t, err)
// compare after marshal because we only need inner field of transaction
txEncoded, err := tx.MarshalBinary()
assert.NoError(t, err)
tx2Encoded, err := tx2.MarshalBinary()
assert.NoError(t, err)
assert.Equal(t, txEncoded, tx2Encoded)
}
// make sure every tx type is tested
assert.Positive(t, m[types.LegacyTxType])
assert.Positive(t, m[types.AccessListTxType])
assert.Positive(t, m[types.DynamicFeeTxType])
}
func TestSpanBatchTxRoundTrip(t *testing.T) {
rng := rand.New(rand.NewSource(0x1332))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
m := make(map[byte]int)
for i := 0; i < 32; i++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
m[tx.Type()] += 1
sbtx, err := newSpanBatchTx(*tx)
assert.NoError(t, err)
sbtxEncoded, err := sbtx.MarshalBinary()
assert.NoError(t, err)
var sbtx2 spanBatchTx
err = sbtx2.UnmarshalBinary(sbtxEncoded)
assert.NoError(t, err)
assert.Equal(t, sbtx, &sbtx2)
}
// make sure every tx type is tested
assert.Positive(t, m[types.LegacyTxType])
assert.Positive(t, m[types.AccessListTxType])
assert.Positive(t, m[types.DynamicFeeTxType])
}
func TestSpanBatchTxRoundTripRLP(t *testing.T) {
rng := rand.New(rand.NewSource(0x1333))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
m := make(map[byte]int)
for i := 0; i < 32; i++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
m[tx.Type()] += 1
sbtx, err := newSpanBatchTx(*tx)
assert.NoError(t, err)
var buf bytes.Buffer
err = sbtx.EncodeRLP(&buf)
assert.NoError(t, err)
result := buf.Bytes()
var sbtx2 spanBatchTx
r := bytes.NewReader(result)
rlpReader := rlp.NewStream(r, 0)
err = sbtx2.DecodeRLP(rlpReader)
assert.NoError(t, err)
assert.Equal(t, sbtx, &sbtx2)
}
// make sure every tx type is tested
assert.Positive(t, m[types.LegacyTxType])
assert.Positive(t, m[types.AccessListTxType])
assert.Positive(t, m[types.DynamicFeeTxType])
}
type spanBatchDummyTxData struct{}
func (txData *spanBatchDummyTxData) txType() byte { return types.DepositTxType }
func TestSpanBatchTxInvalidTxType(t *testing.T) {
// span batch never contain deposit tx
depositTx := types.NewTx(&types.DepositTx{})
_, err := newSpanBatchTx(*depositTx)
assert.ErrorContains(t, err, "invalid tx type")
var sbtx spanBatchTx
sbtx.inner = &spanBatchDummyTxData{}
_, err = sbtx.convertToFullTx(0, 0, nil, nil, nil, nil, nil)
assert.ErrorContains(t, err, "invalid tx type")
}
func TestSpanBatchTxDecodeInvalid(t *testing.T) {
var sbtx spanBatchTx
_, err := sbtx.decodeTyped([]byte{})
assert.EqualError(t, err, "typed transaction too short")
tx := types.NewTx(&types.LegacyTx{})
txEncoded, err := tx.MarshalBinary()
assert.NoError(t, err)
// legacy tx is not typed tx
_, err = sbtx.decodeTyped(txEncoded)
assert.EqualError(t, err, types.ErrTxTypeNotSupported.Error())
tx2 := types.NewTx(&types.AccessListTx{})
tx2Encoded, err := tx2.MarshalBinary()
assert.NoError(t, err)
tx2Encoded[0] = types.DynamicFeeTxType
_, err = sbtx.decodeTyped(tx2Encoded)
assert.ErrorContains(t, err, "failed to decode spanBatchDynamicFeeTxData")
tx3 := types.NewTx(&types.DynamicFeeTx{})
tx3Encoded, err := tx3.MarshalBinary()
assert.NoError(t, err)
tx3Encoded[0] = types.AccessListTxType
_, err = sbtx.decodeTyped(tx3Encoded)
assert.ErrorContains(t, err, "failed to decode spanBatchAccessListTxData")
invalidLegacyTxDecoded := []byte{0xFF, 0xFF}
err = sbtx.UnmarshalBinary(invalidLegacyTxDecoded)
assert.ErrorContains(t, err, "failed to decode spanBatchLegacyTxData")
}
This diff is collapsed.
This diff is collapsed.
...@@ -144,7 +144,33 @@ func RandomTx(rng *rand.Rand, baseFee *big.Int, signer types.Signer) *types.Tran ...@@ -144,7 +144,33 @@ func RandomTx(rng *rand.Rand, baseFee *big.Int, signer types.Signer) *types.Tran
gas := params.TxGas + uint64(rng.Int63n(2_000_000)) gas := params.TxGas + uint64(rng.Int63n(2_000_000))
key := InsecureRandomKey(rng) key := InsecureRandomKey(rng)
tip := big.NewInt(rng.Int63n(10 * params.GWei)) tip := big.NewInt(rng.Int63n(10 * params.GWei))
tx, err := types.SignNewTx(key, signer, &types.DynamicFeeTx{
txTypeList := []int{types.LegacyTxType, types.AccessListTxType, types.DynamicFeeTxType}
txType := txTypeList[rng.Intn(len(txTypeList))]
var txData types.TxData
switch txType {
case types.LegacyTxType:
txData = &types.LegacyTx{
Nonce: rng.Uint64(),
GasPrice: new(big.Int).SetUint64(rng.Uint64()),
Gas: gas,
To: RandomTo(rng),
Value: RandomETH(rng, 10),
Data: RandomData(rng, rng.Intn(1000)),
}
case types.AccessListTxType:
txData = &types.AccessListTx{
ChainID: signer.ChainID(),
Nonce: rng.Uint64(),
GasPrice: new(big.Int).SetUint64(rng.Uint64()),
Gas: gas,
To: RandomTo(rng),
Value: RandomETH(rng, 10),
Data: RandomData(rng, rng.Intn(1000)),
AccessList: nil,
}
case types.DynamicFeeTxType:
txData = &types.DynamicFeeTx{
ChainID: signer.ChainID(), ChainID: signer.ChainID(),
Nonce: rng.Uint64(), Nonce: rng.Uint64(),
GasTipCap: tip, GasTipCap: tip,
...@@ -154,7 +180,11 @@ func RandomTx(rng *rand.Rand, baseFee *big.Int, signer types.Signer) *types.Tran ...@@ -154,7 +180,11 @@ func RandomTx(rng *rand.Rand, baseFee *big.Int, signer types.Signer) *types.Tran
Value: RandomETH(rng, 10), Value: RandomETH(rng, 10),
Data: RandomData(rng, rng.Intn(1000)), Data: RandomData(rng, rng.Intn(1000)),
AccessList: nil, AccessList: nil,
}) }
default:
panic("invalid tx type")
}
tx, err := types.SignNewTx(key, signer, txData)
if err != nil { if err != nil {
panic(err) panic(err)
} }
......
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