Commit d6e1259b authored by Changwan Park's avatar Changwan Park Committed by Tei Im

Implement Span batch encoding and decoding

Define SpanBatch and related types
Rename BatchV1 to SingularBatch
Add unit test cases
parent cb42a610
......@@ -322,11 +322,11 @@ func FuzzSeqWindowClose(f *testing.F) {
// Check the timeout
cb.timeout = timeout
cb.updateSwTimeout(&derive.BatchData{
BatchV1: derive.BatchV1{
cb.updateSwTimeout(derive.NewSingularBatchData(
derive.SingularBatch{
EpochNum: rollup.Epoch(epochNum),
},
})
))
calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin
if timeout > calculatedTimeout && calculatedTimeout != 0 {
cb.checkTimeout(calculatedTimeout)
......@@ -354,11 +354,11 @@ func FuzzSeqWindowZeroTimeoutClose(f *testing.F) {
// Check the timeout
cb.timeout = 0
cb.updateSwTimeout(&derive.BatchData{
BatchV1: derive.BatchV1{
cb.updateSwTimeout(derive.NewSingularBatchData(
derive.SingularBatch{
EpochNum: rollup.Epoch(epochNum),
},
})
))
calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin
cb.checkTimeout(calculatedTimeout)
if cb.timeout != 0 {
......
......@@ -253,7 +253,7 @@ func blockToBatch(block *types.Block) (*derive.BatchData, error) {
}
return &derive.BatchData{
BatchV1: derive.BatchV1{
SingularBatch: derive.SingularBatch{
ParentHash: block.ParentHash(),
EpochNum: rollup.Epoch(l1Info.Number),
EpochHash: l1Info.BlockHash,
......
......@@ -16,12 +16,12 @@ import (
)
type ChannelWithMetadata struct {
ID derive.ChannelID `json:"id"`
IsReady bool `json:"is_ready"`
InvalidFrames bool `json:"invalid_frames"`
InvalidBatches bool `json:"invalid_batches"`
Frames []FrameWithMetadata `json:"frames"`
Batches []derive.BatchV1 `json:"batches"`
ID derive.ChannelID `json:"id"`
IsReady bool `json:"is_ready"`
InvalidFrames bool `json:"invalid_frames"`
InvalidBatches bool `json:"invalid_batches"`
Frames []FrameWithMetadata `json:"frames"`
Batches []derive.SingularBatch `json:"batches"`
}
type FrameWithMetadata struct {
......@@ -100,7 +100,7 @@ func processFrames(id derive.ChannelID, frames []FrameWithMetadata) ChannelWithM
}
}
var batches []derive.BatchV1
var batches []derive.SingularBatch
invalidBatches := false
if ch.IsReady() {
br, err := derive.BatchReader(ch.Reader(), eth.L1BlockRef{})
......@@ -110,7 +110,7 @@ func processFrames(id derive.ChannelID, frames []FrameWithMetadata) ChannelWithM
fmt.Printf("Error reading batch for channel %v. Err: %v\n", id.String(), err)
invalidBatches = true
} else {
batches = append(batches, batch.Batch.BatchV1)
batches = append(batches, batch.Batch.SingularBatch)
}
}
} else {
......
......@@ -42,13 +42,13 @@ func TestAttributesQueue(t *testing.T) {
safeHead.L1Origin = l1Info.ID()
safeHead.Time = l1Info.InfoTime
batch := &BatchData{BatchV1{
batch := NewSingularBatchData(SingularBatch{
ParentHash: safeHead.Hash,
EpochNum: rollup.Epoch(l1Info.InfoNum),
EpochHash: l1Info.InfoHash,
Timestamp: safeHead.Time + cfg.BlockTime,
Transactions: []eth.Data{eth.Data("foobar"), eth.Data("example")},
}}
})
parentL1Cfg := eth.SystemConfig{
BatcherAddr: common.Address{42},
......
......@@ -7,48 +7,44 @@ import (
"io"
"sync"
"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"
"github.com/ethereum/go-ethereum/rlp"
)
// Batch format
// first byte is type followed by bytestring.
//
// BatchV1Type := 0
// batchV1 := BatchV1Type ++ RLP([epoch, timestamp, transaction_list]
//
// An empty input is not a valid batch.
//
// Note: the type system is based on L1 typed transactions.
//
// encodeBufferPool holds temporary encoder buffers for batch encoding
var encodeBufferPool = sync.Pool{
New: func() any { return new(bytes.Buffer) },
}
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 {
ParentHash common.Hash // parent L2 block hash
EpochNum rollup.Epoch // aka l1 num
EpochHash common.Hash // block hash
Timestamp uint64
// no feeRecipient address input, all fees go to a L2 contract
Transactions []hexutil.Bytes
// Batch contains information to build one or multiple L2 blocks.
// Batcher converts L2 blocks into Batch and writes encoded bytes to Channel.
// Derivation pipeline decodes Batch from Channel, and converts to one or multiple payload attributes.
type Batch interface {
GetBatchType() int
GetTimestamp() uint64
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 {
BatchV1
// batches may contain additional data with new upgrades
}
func (b *BatchV1) Epoch() eth.BlockID {
return eth.BlockID{Hash: b.EpochHash, Number: uint64(b.EpochNum)}
BatchType int
SingularBatch
RawSpanBatch
}
// EncodeRLP implements rlp.Encoder
......@@ -69,9 +65,18 @@ func (b *BatchData) MarshalBinary() ([]byte, error) {
return buf.Bytes(), err
}
// encodeTyped encodes batch type and payload for each batch type.
func (b *BatchData) encodeTyped(buf *bytes.Buffer) error {
buf.WriteByte(BatchV1Type)
return rlp.Encode(buf, &b.BatchV1)
switch b.BatchType {
case SingularBatchType:
buf.WriteByte(SingularBatchType)
return rlp.Encode(buf, &b.SingularBatch)
case SpanBatchType:
buf.WriteByte(byte(b.BatchType))
return b.RawSpanBatch.encode(buf)
default:
return fmt.Errorf("unrecognized batch type: %d", b.BatchType)
}
}
// DecodeRLP implements rlp.Decoder
......@@ -94,14 +99,35 @@ func (b *BatchData) UnmarshalBinary(data []byte) error {
return b.decodeTyped(data)
}
// decodeTyped decodes batch type and payload for each batch type.
func (b *BatchData) decodeTyped(data []byte) error {
if len(data) == 0 {
return fmt.Errorf("batch too short")
}
switch data[0] {
case BatchV1Type:
return rlp.DecodeBytes(data[1:], &b.BatchV1)
case SingularBatchType:
b.BatchType = SingularBatchType
return rlp.DecodeBytes(data[1:], &b.SingularBatch)
case SpanBatchType:
b.BatchType = int(data[0])
return b.RawSpanBatch.decodeBytes(data[1:])
default:
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:
// batch to ensure that we at least have one batch per epoch.
if nextTimestamp < nextEpoch.Time || firstOfEpoch {
bq.log.Info("Generating next batch", "epoch", epoch, "timestamp", nextTimestamp)
return &BatchData{
BatchV1{
return NewSingularBatchData(
SingularBatch{
ParentHash: l2SafeHead.Hash,
EpochNum: rollup.Epoch(epoch.Number),
EpochHash: epoch.Hash,
Timestamp: nextTimestamp,
Transactions: nil,
},
}, nil
), nil
}
// 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 {
func b(timestamp uint64, epoch eth.L1BlockRef) *BatchData {
rng := rand.New(rand.NewSource(int64(timestamp)))
data := testutils.RandomData(rng, 20)
return &BatchData{BatchV1{
return NewSingularBatchData(SingularBatch{
ParentHash: mockHash(timestamp-2, 2),
Timestamp: timestamp,
EpochNum: rollup.Epoch(epoch.Number),
EpochHash: epoch.Hash,
Transactions: []hexutil.Bytes{data},
}}
})
}
func L1Chain(l1Times []uint64) []eth.L1BlockRef {
......@@ -331,7 +331,7 @@ func TestBatchQueueMissing(t *testing.T) {
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
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)
safeHead.Number += 1
safeHead.Time += 2
......@@ -341,7 +341,7 @@ func TestBatchQueueMissing(t *testing.T) {
b, e = bq.NextBatch(context.Background(), safeHead)
require.Nil(t, e)
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)
safeHead.Number += 1
safeHead.Time += 2
......@@ -367,6 +367,6 @@ func TestBatchQueueMissing(t *testing.T) {
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.Empty(t, b.SingularBatch.Transactions)
require.Equal(t, rollup.Epoch(1), b.EpochNum)
}
package derive
import (
"bytes"
"math/big"
"math/rand"
"testing"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/testutils"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/stretchr/testify/assert"
"github.com/ethereum/go-ethereum/common"
"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) {
rng := rand.New(rand.NewSource(0xdeadbeef))
blockTime := uint64(2)
genesisTimestamp := uint64(0)
chainID := new(big.Int).SetUint64(rng.Uint64())
batches := []*BatchData{
{
BatchV1: BatchV1{
SingularBatch: SingularBatch{
ParentHash: common.Hash{},
EpochNum: 0,
Timestamp: 0,
......@@ -20,13 +150,18 @@ func TestBatchRoundTrip(t *testing.T) {
},
},
{
BatchV1: BatchV1{
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 {
......@@ -35,6 +170,58 @@ func TestBatchRoundTrip(t *testing.T) {
var dec BatchData
err = dec.UnmarshalBinary(enc)
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)
}
}
......@@ -10,6 +10,7 @@ import (
// FuzzBatchRoundTrip executes a fuzz test similar to TestBatchRoundTrip, which tests that arbitrary BatchData will be
// encoded and decoded without loss of its original values.
// Does not test span batch because fuzzer is not aware structure of span batch.
func FuzzBatchRoundTrip(f *testing.F) {
f.Fuzz(func(t *testing.T, fuzzedData []byte) {
// Create our fuzzer wrapper to generate complex values
......@@ -20,6 +21,10 @@ func FuzzBatchRoundTrip(f *testing.F) {
var batchData BatchData
typeProvider.Fuzz(&batchData)
// force batchdata to only contain singular batch
batchData.BatchType = SingularBatchType
batchData.RawSpanBatch = RawSpanBatch{}
// Encode our batch data
enc, err := batchData.MarshalBinary()
require.NoError(t, err)
......
......@@ -174,13 +174,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A1.Time,
Transactions: nil,
}},
}),
},
Expected: BatchUndecided,
},
......@@ -190,13 +190,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A1.Time + 1, // 1 too high
Transactions: nil,
}},
}),
},
Expected: BatchFuture,
},
......@@ -206,13 +206,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A0.Time, // repeating the same time
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -222,13 +222,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A1.Time - 1, // block time is 2, so this is 1 too low
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -238,13 +238,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: testutils.RandomHash(rng),
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A1.Time,
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -254,13 +254,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1F, // included in 5th block after epoch of batch, while seq window is 4
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
Timestamp: l2A1.Time,
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -270,13 +270,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2B0, // we already moved on to B
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1C,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2B0.Hash, // build on top of safe head to continue
EpochNum: rollup.Epoch(l2A3.L1Origin.Number), // epoch A is no longer valid
EpochHash: l2A3.L1Origin.Hash,
Timestamp: l2B0.Time + conf.BlockTime, // pass the timestamp check to get too epoch check
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -286,13 +286,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1C,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2B0.ParentHash,
EpochNum: rollup.Epoch(l2B0.L1Origin.Number),
EpochHash: l2B0.L1Origin.Hash,
Timestamp: l2B0.Time,
Transactions: nil,
}},
}),
},
Expected: BatchUndecided,
},
......@@ -302,13 +302,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1D,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2B0.ParentHash,
EpochNum: rollup.Epoch(l1C.Number), // invalid, we need to adopt epoch B before C
EpochHash: l1C.Hash,
Timestamp: l2B0.Time,
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -318,13 +318,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1C,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2B0.ParentHash,
EpochNum: rollup.Epoch(l2B0.L1Origin.Number),
EpochHash: l1A.Hash, // invalid, epoch hash should be l1B
Timestamp: l2B0.Time,
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......@@ -334,13 +334,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
ParentHash: l2A4.ParentHash,
EpochNum: rollup.Epoch(l2A4.L1Origin.Number),
EpochHash: l2A4.L1Origin.Hash,
Timestamp: l2A4.Time,
Transactions: []hexutil.Bytes{[]byte("sequencer should not include this tx")},
}},
}),
},
Expected: BatchDrop,
},
......@@ -350,13 +350,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2X0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1Z,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2Y0.ParentHash,
EpochNum: rollup.Epoch(l2Y0.L1Origin.Number),
EpochHash: l2Y0.L1Origin.Hash,
Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time
Transactions: []hexutil.Bytes{[]byte("sequencer should not include this tx")},
}},
}),
},
Expected: BatchDrop,
},
......@@ -366,13 +366,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1BLate,
Batch: &BatchData{BatchV1{ // l2A4 time < l1BLate time, so we cannot adopt origin B yet
Batch: NewSingularBatchData(SingularBatch{ // l2A4 time < l1BLate time, so we cannot adopt origin B yet
ParentHash: l2A4.ParentHash,
EpochNum: rollup.Epoch(l2A4.L1Origin.Number),
EpochHash: l2A4.L1Origin.Hash,
Timestamp: l2A4.Time,
Transactions: nil,
}},
}),
},
Expected: BatchAccept, // accepted because empty & preserving L2 time invariant
},
......@@ -382,13 +382,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2X0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1Z,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2Y0.ParentHash,
EpochNum: rollup.Epoch(l2Y0.L1Origin.Number),
EpochHash: l2Y0.L1Origin.Hash,
Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time
Transactions: nil,
}},
}),
},
Expected: BatchAccept, // accepted because empty & still advancing epoch
},
......@@ -398,13 +398,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
ParentHash: l2A4.ParentHash,
EpochNum: rollup.Epoch(l2A4.L1Origin.Number),
EpochHash: l2A4.L1Origin.Hash,
Timestamp: l2A4.Time,
Transactions: nil,
}},
}),
},
Expected: BatchUndecided, // we have to wait till the next epoch is in sight to check the time
},
......@@ -414,13 +414,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1C,
Batch: &BatchData{BatchV1{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0
ParentHash: l2A4.ParentHash,
EpochNum: rollup.Epoch(l2A4.L1Origin.Number),
EpochHash: l2A4.L1Origin.Hash,
Timestamp: l2A4.Time,
Transactions: nil,
}},
}),
},
Expected: BatchDrop, // dropped because it could have advanced the epoch to B
},
......@@ -430,7 +430,7 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
......@@ -438,7 +438,7 @@ func TestValidBatch(t *testing.T) {
Transactions: []hexutil.Bytes{
[]byte{}, // empty tx data
},
}},
}),
},
Expected: BatchDrop,
},
......@@ -448,7 +448,7 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
......@@ -456,7 +456,7 @@ func TestValidBatch(t *testing.T) {
Transactions: []hexutil.Bytes{
[]byte{types.DepositTxType, 0}, // piece of data alike to a deposit
},
}},
}),
},
Expected: BatchDrop,
},
......@@ -466,7 +466,7 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A0,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2A1.ParentHash,
EpochNum: rollup.Epoch(l2A1.L1Origin.Number),
EpochHash: l2A1.L1Origin.Hash,
......@@ -475,7 +475,7 @@ func TestValidBatch(t *testing.T) {
[]byte{0x02, 0x42, 0x13, 0x37},
[]byte{0x02, 0xde, 0xad, 0xbe, 0xef},
},
}},
}),
},
Expected: BatchAccept,
},
......@@ -485,7 +485,7 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A3,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1C,
Batch: &BatchData{BatchV1{
Batch: NewSingularBatchData(SingularBatch{
ParentHash: l2B0.ParentHash,
EpochNum: rollup.Epoch(l2B0.L1Origin.Number),
EpochHash: l2B0.L1Origin.Hash,
......@@ -494,7 +494,7 @@ func TestValidBatch(t *testing.T) {
[]byte{0x02, 0x42, 0x13, 0x37},
[]byte{0x02, 0xde, 0xad, 0xbe, 0xef},
},
}},
}),
},
Expected: BatchAccept,
},
......@@ -504,13 +504,13 @@ func TestValidBatch(t *testing.T) {
L2SafeHead: l2A2,
Batch: BatchWithL1InclusionBlock{
L1InclusionBlock: l1B,
Batch: &BatchData{BatchV1{ // we build l2B0', which starts a new epoch too early
Batch: NewSingularBatchData(SingularBatch{ // we build l2B0', which starts a new epoch too early
ParentHash: l2A2.Hash,
EpochNum: rollup.Epoch(l2B0.L1Origin.Number),
EpochHash: l2B0.L1Origin.Hash,
Timestamp: l2A2.Time + conf.BlockTime,
Transactions: nil,
}},
}),
},
Expected: BatchDrop,
},
......
......@@ -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 &BatchData{
BatchV1{
return NewSingularBatchData(
SingularBatch{
ParentHash: block.ParentHash(),
EpochNum: rollup.Epoch(l1Info.Number),
EpochHash: l1Info.BlockHash,
Timestamp: block.Time(),
Transactions: opaqueTxs,
},
}, l1Info, nil
), l1Info, nil
}
// ForceCloseTxData generates the transaction data for a transaction which will force close
......
......@@ -19,6 +19,11 @@ func frameSize(frame Frame) uint64 {
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,
// till the bank is pruned by removing channels,
// starting with the oldest channel.
......
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([epoch, 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())
}
package derive
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"io"
"math/big"
"sort"
"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"
"github.com/ethereum/go-ethereum/rlp"
)
// Batch format
//
// SpanBatchType := 1
// spanBatch := SpanBatchType ++ prefix ++ payload
// prefix := rel_timestamp ++ l1_origin_num ++ parent_check ++ l1_origin_check
// payload := payload = block_count ++ origin_bits ++ block_tx_counts ++ txs
// txs = contract_creation_bits ++ y_parity_bits ++ tx_sigs ++ tx_tos ++ tx_datas ++ tx_nonces ++ tx_gases
var ErrTooBigSpanBatchFieldSize = errors.New("batch would cause field bytes to go over limit")
type spanBatchPrefix struct {
relTimestamp uint64 // Relative timestamp of the first block
l1OriginNum uint64 // L1 origin number
parentCheck []byte // First 20 bytes of the first block's parent hash
l1OriginCheck []byte // First 20 bytes of the last block's L1 origin hash
}
type spanBatchPayload struct {
blockCount uint64 // Number of L2 block in the span
originBits *big.Int // Bitlist of blockCount bits. Each bit indicates if the L1 origin is changed at the L2 block.
blockTxCounts []uint64 // List of transaction counts for each L2 block
txs *spanBatchTxs // Transactions encoded in SpanBatch specs
}
// RawSpanBatch is another representation of SpanBatch, that encodes data according to SpanBatch specs.
type RawSpanBatch struct {
spanBatchPrefix
spanBatchPayload
}
// decodeOriginBits parses data into bp.originBits
// originBits is bitlist right-padded to a multiple of 8 bits
func (bp *spanBatchPayload) decodeOriginBits(r *bytes.Reader) error {
originBitBufferLen := bp.blockCount / 8
if bp.blockCount%8 != 0 {
originBitBufferLen++
}
// avoid out of memory before allocation
if originBitBufferLen > MaxSpanBatchFieldSize {
return ErrTooBigSpanBatchFieldSize
}
originBitBuffer := make([]byte, originBitBufferLen)
_, err := io.ReadFull(r, originBitBuffer)
if err != nil {
return fmt.Errorf("failed to read origin bits: %w", err)
}
originBits := new(big.Int)
for i := 0; i < int(bp.blockCount); i += 8 {
end := i + 8
if end < int(bp.blockCount) {
end = int(bp.blockCount)
}
bits := originBitBuffer[i/8]
for j := i; j < end; j++ {
bit := uint((bits >> (j - i)) & 1)
originBits.SetBit(originBits, j, bit)
}
}
bp.originBits = originBits
return nil
}
// decodeRelTimestamp parses data into bp.relTimestamp
func (bp *spanBatchPrefix) decodeRelTimestamp(r *bytes.Reader) error {
relTimestamp, err := binary.ReadUvarint(r)
if err != nil {
return fmt.Errorf("failed to read rel timestamp: %w", err)
}
bp.relTimestamp = relTimestamp
return nil
}
// decodeL1OriginNum parses data into bp.l1OriginNum
func (bp *spanBatchPrefix) decodeL1OriginNum(r *bytes.Reader) error {
L1OriginNum, err := binary.ReadUvarint(r)
if err != nil {
return fmt.Errorf("failed to read l1 origin num: %w", err)
}
bp.l1OriginNum = L1OriginNum
return nil
}
// decodeParentCheck parses data into bp.parentCheck
func (bp *spanBatchPrefix) decodeParentCheck(r *bytes.Reader) error {
bp.parentCheck = make([]byte, 20)
_, err := io.ReadFull(r, bp.parentCheck)
if err != nil {
return fmt.Errorf("failed to read parent check: %w", err)
}
return nil
}
// decodeL1OriginCheck parses data into bp.decodeL1OriginCheck
func (bp *spanBatchPrefix) decodeL1OriginCheck(r *bytes.Reader) error {
bp.l1OriginCheck = make([]byte, 20)
_, err := io.ReadFull(r, bp.l1OriginCheck)
if err != nil {
return fmt.Errorf("failed to read l1 origin check: %w", err)
}
return nil
}
// decodePrefix parses data into bp.spanBatchPrefix
func (bp *spanBatchPrefix) decodePrefix(r *bytes.Reader) error {
if err := bp.decodeRelTimestamp(r); err != nil {
return err
}
if err := bp.decodeL1OriginNum(r); err != nil {
return err
}
if err := bp.decodeParentCheck(r); err != nil {
return err
}
if err := bp.decodeL1OriginCheck(r); err != nil {
return err
}
return nil
}
// decodeBlockCount parses data into bp.blockCount
func (bp *spanBatchPayload) decodeBlockCount(r *bytes.Reader) error {
blockCount, err := binary.ReadUvarint(r)
bp.blockCount = blockCount
if err != nil {
return fmt.Errorf("failed to read block count: %w", err)
}
return nil
}
// decodeBlockTxCounts parses data into bp.blockTxCounts
// and sets bp.txs.totalBlockTxCount as sum(bp.blockTxCounts)
func (bp *spanBatchPayload) decodeBlockTxCounts(r *bytes.Reader) error {
var blockTxCounts []uint64
for i := 0; i < int(bp.blockCount); i++ {
blockTxCount, err := binary.ReadUvarint(r)
if err != nil {
return fmt.Errorf("failed to read block tx count: %w", err)
}
blockTxCounts = append(blockTxCounts, blockTxCount)
}
bp.blockTxCounts = blockTxCounts
return nil
}
// decodeTxs parses data into bp.txs
func (bp *spanBatchPayload) decodeTxs(r *bytes.Reader) error {
if bp.txs == nil {
bp.txs = &spanBatchTxs{}
}
if bp.blockTxCounts == nil {
return errors.New("failed to read txs: blockTxCounts not set")
}
totalBlockTxCount := uint64(0)
for i := 0; i < len(bp.blockTxCounts); i++ {
totalBlockTxCount += bp.blockTxCounts[i]
}
bp.txs.totalBlockTxCount = totalBlockTxCount
if err := bp.txs.decode(r); err != nil {
return err
}
return nil
}
// decodePayload parses data into bp.spanBatchPayload
func (bp *spanBatchPayload) decodePayload(r *bytes.Reader) error {
if err := bp.decodeBlockCount(r); err != nil {
return err
}
if err := bp.decodeOriginBits(r); err != nil {
return err
}
if err := bp.decodeBlockTxCounts(r); err != nil {
return err
}
if err := bp.decodeTxs(r); err != nil {
return err
}
return nil
}
// decodeBytes parses data into b from data
func (b *RawSpanBatch) decodeBytes(data []byte) error {
r := bytes.NewReader(data)
if err := b.decodePrefix(r); err != nil {
return err
}
if err := b.decodePayload(r); err != nil {
return err
}
return nil
}
// encodeRelTimestamp encodes bp.relTimestamp
func (bp *spanBatchPrefix) encodeRelTimestamp(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
n := binary.PutUvarint(buf[:], bp.relTimestamp)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write rel timestamp: %w", err)
}
return nil
}
// encodeL1OriginNum encodes bp.l1OriginNum
func (bp *spanBatchPrefix) encodeL1OriginNum(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
n := binary.PutUvarint(buf[:], bp.l1OriginNum)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write l1 origin number: %w", err)
}
return nil
}
// encodeParentCheck encodes bp.parentCheck
func (bp *spanBatchPrefix) encodeParentCheck(w io.Writer) error {
if _, err := w.Write(bp.parentCheck); err != nil {
return fmt.Errorf("cannot write parent check: %w", err)
}
return nil
}
// encodeL1OriginCheck encodes bp.l1OriginCheck
func (bp *spanBatchPrefix) encodeL1OriginCheck(w io.Writer) error {
if _, err := w.Write(bp.l1OriginCheck); err != nil {
return fmt.Errorf("cannot write l1 origin check: %w", err)
}
return nil
}
// encodePrefix encodes spanBatchPrefix
func (bp *spanBatchPrefix) encodePrefix(w io.Writer) error {
if err := bp.encodeRelTimestamp(w); err != nil {
return err
}
if err := bp.encodeL1OriginNum(w); err != nil {
return err
}
if err := bp.encodeParentCheck(w); err != nil {
return err
}
if err := bp.encodeL1OriginCheck(w); err != nil {
return err
}
return nil
}
// encodeOriginBits encodes bp.originBits
// originBits is bitlist right-padded to a multiple of 8 bits
func (bp *spanBatchPayload) encodeOriginBits(w io.Writer) error {
originBitBufferLen := bp.blockCount / 8
if bp.blockCount%8 != 0 {
originBitBufferLen++
}
originBitBuffer := make([]byte, originBitBufferLen)
for i := 0; i < int(bp.blockCount); i += 8 {
end := i + 8
if end < int(bp.blockCount) {
end = int(bp.blockCount)
}
var bits uint = 0
for j := i; j < end; j++ {
bits |= bp.originBits.Bit(j) << (j - i)
}
originBitBuffer[i/8] = byte(bits)
}
if _, err := w.Write(originBitBuffer); err != nil {
return fmt.Errorf("cannot write origin bits: %w", err)
}
return nil
}
// encodeBlockCount encodes bp.blockCount
func (bp *spanBatchPayload) encodeBlockCount(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
n := binary.PutUvarint(buf[:], bp.blockCount)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write block count: %w", err)
}
return nil
}
// encodeBlockTxCounts encodes bp.blockTxCounts
func (bp *spanBatchPayload) encodeBlockTxCounts(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
for _, blockTxCount := range bp.blockTxCounts {
n := binary.PutUvarint(buf[:], blockTxCount)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write block tx count: %w", err)
}
}
return nil
}
// encodeTxs encodes bp.txs
func (bp *spanBatchPayload) encodeTxs(w io.Writer) error {
if bp.txs == nil {
return errors.New("cannot write txs: txs not set")
}
if err := bp.txs.encode(w); err != nil {
return err
}
return nil
}
// encodePayload encodes spanBatchPayload
func (bp *spanBatchPayload) encodePayload(w io.Writer) error {
if err := bp.encodeBlockCount(w); err != nil {
return err
}
if err := bp.encodeOriginBits(w); err != nil {
return err
}
if err := bp.encodeBlockTxCounts(w); err != nil {
return err
}
if err := bp.encodeTxs(w); err != nil {
return err
}
return nil
}
// encode writes the byte encoding of SpanBatch to Writer stream
func (b *RawSpanBatch) encode(w io.Writer) error {
if err := b.encodePrefix(w); err != nil {
return err
}
if err := b.encodePayload(w); err != nil {
return err
}
return nil
}
// encodeBytes returns the byte encoding of SpanBatch
func (b *RawSpanBatch) encodeBytes() ([]byte, error) {
buf := encodeBufferPool.Get().(*bytes.Buffer)
defer encodeBufferPool.Put(buf)
buf.Reset()
if err := b.encode(buf); err != nil {
return []byte{}, err
}
return buf.Bytes(), nil
}
// derive converts RawSpanBatch into SpanBatch, which has a list of spanBatchElement.
// We need chain config constants to derive values for making payload attributes.
func (b *RawSpanBatch) derive(blockTime, genesisTimestamp uint64, chainID *big.Int) (*SpanBatch, error) {
blockOriginNums := make([]uint64, b.blockCount)
l1OriginBlockNumber := b.l1OriginNum
for i := int(b.blockCount) - 1; i >= 0; i-- {
blockOriginNums[i] = l1OriginBlockNumber
if b.originBits.Bit(i) == 1 && i > 0 {
l1OriginBlockNumber--
}
}
b.txs.recoverV(chainID)
fullTxs, err := b.txs.fullTxs(chainID)
if err != nil {
return nil, err
}
spanBatch := SpanBatch{
parentCheck: b.parentCheck,
l1OriginCheck: b.l1OriginCheck,
}
txIdx := 0
for i := 0; i < int(b.blockCount); i++ {
batch := spanBatchElement{}
batch.Timestamp = genesisTimestamp + b.relTimestamp + blockTime*uint64(i)
batch.EpochNum = rollup.Epoch(blockOriginNums[i])
for j := 0; j < int(b.blockTxCounts[i]); j++ {
batch.Transactions = append(batch.Transactions, fullTxs[txIdx])
txIdx++
}
spanBatch.batches = append(spanBatch.batches, &batch)
}
return &spanBatch, nil
}
// spanBatchElement is a derived form of input to build a L2 block.
// similar to SingularBatch, but does not have ParentHash and EpochHash
// because Span batch spec does not contain parent hash and epoch hash of every block in the span.
type spanBatchElement struct {
EpochNum rollup.Epoch // aka l1 num
Timestamp uint64
Transactions []hexutil.Bytes
}
// singularBatchToElement converts a SingularBatch to a spanBatchElement
func singularBatchToElement(singularBatch *SingularBatch) *spanBatchElement {
return &spanBatchElement{
EpochNum: singularBatch.EpochNum,
Timestamp: singularBatch.Timestamp,
Transactions: singularBatch.Transactions,
}
}
// 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 []byte // First 20 bytes of the first block's parent hash
l1OriginCheck []byte // First 20 bytes of the last block's L1 origin hash
batches []*spanBatchElement // List of block input in derived form
}
// GetBatchType returns its batch type (batch_version)
func (b *SpanBatch) GetBatchType() int {
return SpanBatchType
}
// GetTimestamp returns timestamp of the first block in the span
func (b *SpanBatch) GetTimestamp() uint64 {
return b.batches[0].Timestamp
}
// LogContext creates a new log context that contains information of the batch
func (b *SpanBatch) LogContext(log log.Logger) log.Logger {
return log.New(
"batch_timestamp", b.batches[0].Timestamp,
"parent_check", hexutil.Encode(b.parentCheck),
"origin_check", hexutil.Encode(b.l1OriginCheck),
"start_epoch_number", b.GetStartEpochNum(),
"end_epoch_number", b.GetBlockEpochNum(len(b.batches)-1),
"block_count", len(b.batches),
)
}
// GetStartEpochNum returns epoch number(L1 origin block number) of the first block in the span
func (b *SpanBatch) GetStartEpochNum() rollup.Epoch {
return b.batches[0].EpochNum
}
// CheckOriginHash checks if the l1OriginCheck matches the first 20 bytes of given hash, probably L1 block hash from the current canonical L1 chain.
func (b *SpanBatch) CheckOriginHash(hash common.Hash) bool {
return bytes.Equal(b.l1OriginCheck, hash.Bytes()[:20])
}
// CheckParentHash checks if the parentCheck matches the first 20 bytes of given hash, probably the current L2 safe head.
func (b *SpanBatch) CheckParentHash(hash common.Hash) bool {
return bytes.Equal(b.parentCheck, hash.Bytes()[:20])
}
// GetBlockEpochNum returns the epoch number(L1 origin block number) of the block at the given index in the span.
func (b *SpanBatch) GetBlockEpochNum(i int) uint64 {
return uint64(b.batches[i].EpochNum)
}
// GetBlockTimestamp returns the timestamp of the block at the given index in the span.
func (b *SpanBatch) GetBlockTimestamp(i int) uint64 {
return b.batches[i].Timestamp
}
// GetBlockTransactions returns the encoded transactions of the block at the given index in the span.
func (b *SpanBatch) GetBlockTransactions(i int) []hexutil.Bytes {
return b.batches[i].Transactions
}
// GetBlockCount returns the number of blocks in the span
func (b *SpanBatch) GetBlockCount() int {
return len(b.batches)
}
// 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 {
b.parentCheck = singularBatch.ParentHash.Bytes()[:20]
}
b.batches = append(b.batches, singularBatchToElement(singularBatch))
b.l1OriginCheck = singularBatch.EpochHash.Bytes()[:20]
}
// ToRawSpanBatch merges SingularBatch List and initialize single RawSpanBatch
func (b *SpanBatch) ToRawSpanBatch(originChangedBit uint, genesisTimestamp uint64, chainID *big.Int) (*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 = make([]byte, 20)
copy(raw.parentCheck, b.parentCheck)
raw.l1OriginCheck = make([]byte, 20)
copy(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
}
// GetSingularBatches converts spanBatchElements after L2 safe head to SingularBatches.
// Since spanBatchElement does not contain EpochHash, set EpochHash from the given L1 blocks.
// The result SingularBatches do not contain ParentHash yet. It must be set by BatchQueue.
func (b *SpanBatch) GetSingularBatches(l1Origins []eth.L1BlockRef, l2SafeHead eth.L2BlockRef) ([]*SingularBatch, error) {
var singularBatches []*SingularBatch
originIdx := 0
for _, batch := range b.batches {
if batch.Timestamp <= l2SafeHead.Time {
continue
}
singularBatch := SingularBatch{
EpochNum: batch.EpochNum,
Timestamp: batch.Timestamp,
Transactions: batch.Transactions,
}
originFound := false
for i := originIdx; i < len(l1Origins); i++ {
if l1Origins[i].Number == uint64(batch.EpochNum) {
originIdx = i
singularBatch.EpochHash = l1Origins[i].Hash
originFound = true
break
}
}
if !originFound {
return nil, fmt.Errorf("unable to find L1 origin for the epoch number: %d", batch.EpochNum)
}
singularBatches = append(singularBatches, &singularBatch)
}
return singularBatches, nil
}
// NewSpanBatch converts given singularBatches into spanBatchElements, and creates a new SpanBatch.
func NewSpanBatch(singularBatches []*SingularBatch) *SpanBatch {
if len(singularBatches) == 0 {
return &SpanBatch{}
}
spanBatch := SpanBatch{
parentCheck: singularBatches[0].ParentHash.Bytes()[:20],
l1OriginCheck: singularBatches[len(singularBatches)-1].EpochHash.Bytes()[:20],
}
for _, singularBatch := range singularBatches {
spanBatch.batches = append(spanBatch.batches, singularBatchToElement(singularBatch))
}
return &spanBatch
}
// 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 {
parentEpoch uint64
genesisTimestamp uint64
chainID *big.Int
spanBatch *SpanBatch
}
func NewSpanBatchBuilder(parentEpoch uint64, genesisTimestamp uint64, chainID *big.Int) *SpanBatchBuilder {
return &SpanBatchBuilder{
parentEpoch: parentEpoch,
genesisTimestamp: genesisTimestamp,
chainID: chainID,
spanBatch: &SpanBatch{},
}
}
func (b *SpanBatchBuilder) AppendSingularBatch(singularBatch *SingularBatch) {
b.spanBatch.AppendSingularBatch(singularBatch)
}
func (b *SpanBatchBuilder) GetRawSpanBatch() (*RawSpanBatch, error) {
originChangedBit := 0
if uint64(b.spanBatch.GetStartEpochNum()) != b.parentEpoch {
originChangedBit = 1
}
raw, err := b.spanBatch.ToRawSpanBatch(uint(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
offset, err := r.Seek(0, io.SeekCurrent)
if err != nil {
return nil, 0, fmt.Errorf("failed to seek tx reader: %w", err)
}
b, err := r.ReadByte()
if err != nil {
return nil, 0, fmt.Errorf("failed to read tx initial byte: %w", err)
}
txType := byte(0)
if int(b) <= 0x7F {
// EIP-2718: non legacy tx so write tx type
txType = byte(b)
txData = append(txData, txType)
} else {
// legacy tx: seek back single byte to read prefix again
_, err = r.Seek(offset, io.SeekStart)
if err != nil {
return nil, 0, fmt.Errorf("failed to seek tx reader: %w", err)
}
}
// avoid out of memory before allocation
s := rlp.NewStream(r, MaxSpanBatchFieldSize)
var txPayload []byte
kind, _, err := s.Kind()
switch {
case err != nil:
if errors.Is(err, rlp.ErrValueTooLarge) {
return nil, 0, ErrTooBigSpanBatchFieldSize
}
return nil, 0, fmt.Errorf("failed to read tx RLP prefix: %w", err)
case kind == rlp.List:
if txPayload, err = s.Raw(); err != nil {
return nil, 0, fmt.Errorf("failed to read tx RLP payload: %w", err)
}
default:
return nil, 0, errors.New("tx RLP prefix type must be list")
}
txData = append(txData, txPayload...)
return txData, int(txType), nil
}
package derive
import (
"bytes"
"math/big"
"math/rand"
"testing"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/testutils"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/stretchr/testify/assert"
)
func TestSpanBatchForBatchInterface(t *testing.T) {
rng := rand.New(rand.NewSource(0x5432177))
chainID := big.NewInt(rng.Int63n(1000))
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
blockCount := len(singularBatches)
safeL2Head := testutils.RandomL2BlockRef(rng)
safeL2Head.Hash = common.BytesToHash(singularBatches[0].ParentHash[:])
spanBatch := NewSpanBatch(singularBatches)
// check interface method implementations except logging
assert.Equal(t, SpanBatchType, spanBatch.GetBatchType())
assert.Equal(t, singularBatches[0].Timestamp, spanBatch.GetTimestamp())
assert.Equal(t, singularBatches[0].EpochNum, spanBatch.GetStartEpochNum())
assert.True(t, spanBatch.CheckOriginHash(singularBatches[blockCount-1].EpochHash))
assert.True(t, spanBatch.CheckParentHash(singularBatches[0].ParentHash))
}
func TestSpanBatchOriginBits(t *testing.T) {
rng := rand.New(rand.NewSource(0x77665544))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
blockCount := rawSpanBatch.blockCount
var buf bytes.Buffer
err := rawSpanBatch.encodeOriginBits(&buf)
assert.NoError(t, err)
// originBit field is fixed length: single bit
originBitBufferLen := blockCount / 8
if blockCount%8 != 0 {
originBitBufferLen++
}
assert.Equal(t, buf.Len(), int(originBitBufferLen))
result := buf.Bytes()
var sb RawSpanBatch
sb.blockCount = blockCount
r := bytes.NewReader(result)
err = sb.decodeOriginBits(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.originBits, sb.originBits)
}
func TestSpanBatchPrefix(t *testing.T) {
rng := rand.New(rand.NewSource(0x44775566))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
// only compare prefix
rawSpanBatch.spanBatchPayload = spanBatchPayload{}
var buf bytes.Buffer
err := rawSpanBatch.encodePrefix(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodePrefix(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch, &sb)
}
func TestSpanBatchRelTimestamp(t *testing.T) {
rng := rand.New(rand.NewSource(0x44775566))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeRelTimestamp(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodeRelTimestamp(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.relTimestamp, sb.relTimestamp)
}
func TestSpanBatchL1OriginNum(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556688))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeL1OriginNum(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodeL1OriginNum(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.l1OriginNum, sb.l1OriginNum)
}
func TestSpanBatchParentCheck(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556689))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeParentCheck(&buf)
assert.NoError(t, err)
// parent check field is fixed length: 20 bytes
assert.Equal(t, buf.Len(), 20)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodeParentCheck(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.parentCheck, sb.parentCheck)
}
func TestSpanBatchL1OriginCheck(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556690))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeL1OriginCheck(&buf)
assert.NoError(t, err)
// l1 origin check field is fixed length: 20 bytes
assert.Equal(t, buf.Len(), 20)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodeL1OriginCheck(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.l1OriginCheck, sb.l1OriginCheck)
}
func TestSpanBatchPayload(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556691))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodePayload(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodePayload(r)
assert.NoError(t, err)
sb.txs.recoverV(chainID)
assert.Equal(t, rawSpanBatch.spanBatchPayload, sb.spanBatchPayload)
}
func TestSpanBatchBlockCount(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556691))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeBlockCount(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
err = sb.decodeBlockCount(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.blockCount, sb.blockCount)
}
func TestSpanBatchBlockTxCounts(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556692))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeBlockTxCounts(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
sb.blockCount = rawSpanBatch.blockCount
err = sb.decodeBlockTxCounts(r)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch.blockTxCounts, sb.blockTxCounts)
}
func TestSpanBatchTxs(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556693))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
var buf bytes.Buffer
err := rawSpanBatch.encodeTxs(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sb RawSpanBatch
sb.blockTxCounts = rawSpanBatch.blockTxCounts
err = sb.decodeTxs(r)
assert.NoError(t, err)
sb.txs.recoverV(chainID)
assert.Equal(t, rawSpanBatch.txs, sb.txs)
}
func TestSpanBatchRoundTrip(t *testing.T) {
rng := rand.New(rand.NewSource(0x77556694))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
result, err := rawSpanBatch.encodeBytes()
assert.NoError(t, err)
var sb RawSpanBatch
err = sb.decodeBytes(result)
assert.NoError(t, err)
sb.txs.recoverV(chainID)
assert.Equal(t, rawSpanBatch, &sb)
}
func TestSpanBatchDerive(t *testing.T) {
rng := rand.New(rand.NewSource(0xbab0bab0))
chainID := new(big.Int).SetUint64(rng.Uint64())
l2BlockTime := uint64(2)
for originChangedBit := 0; originChangedBit < 2; originChangedBit++ {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
safeL2Head := testutils.RandomL2BlockRef(rng)
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)
assert.NoError(t, err)
spanBatchDerived, err := rawSpanBatch.derive(l2BlockTime, genesisTimeStamp, chainID)
assert.NoError(t, err)
blockCount := len(singularBatches)
assert.Equal(t, safeL2Head.Hash.Bytes()[:20], spanBatchDerived.parentCheck)
assert.Equal(t, singularBatches[blockCount-1].Epoch().Hash.Bytes()[:20], spanBatchDerived.l1OriginCheck)
assert.Equal(t, len(singularBatches), int(rawSpanBatch.blockCount))
for i := 1; i < len(singularBatches); i++ {
assert.Equal(t, spanBatchDerived.batches[i].Timestamp, spanBatchDerived.batches[i-1].Timestamp+l2BlockTime)
}
for i := 0; i < len(singularBatches); i++ {
assert.Equal(t, singularBatches[i].EpochNum, spanBatchDerived.batches[i].EpochNum)
assert.Equal(t, singularBatches[i].Timestamp, spanBatchDerived.batches[i].Timestamp)
assert.Equal(t, singularBatches[i].Transactions, spanBatchDerived.batches[i].Transactions)
}
}
}
func TestSpanBatchAppend(t *testing.T) {
rng := rand.New(rand.NewSource(0x44337711))
chainID := new(big.Int).SetUint64(rng.Uint64())
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
// initialize empty span batch
spanBatch := NewSpanBatch([]*SingularBatch{})
L := 2
for i := 0; i < L; i++ {
spanBatch.AppendSingularBatch(singularBatches[i])
}
// initialize with two singular batches
spanBatch2 := NewSpanBatch(singularBatches[:L])
assert.Equal(t, spanBatch, spanBatch2)
}
func TestSpanBatchMerge(t *testing.T) {
rng := rand.New(rand.NewSource(0x73314433))
genesisTimeStamp := rng.Uint64()
chainID := new(big.Int).SetUint64(rng.Uint64())
for originChangedBit := 0; originChangedBit < 2; originChangedBit++ {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
blockCount := len(singularBatches)
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
assert.NoError(t, err)
// check span batch prefix
assert.Equal(t, rawSpanBatch.relTimestamp, singularBatches[0].Timestamp-genesisTimeStamp, "invalid relative timestamp")
assert.Equal(t, rollup.Epoch(rawSpanBatch.l1OriginNum), singularBatches[blockCount-1].EpochNum)
assert.Equal(t, rawSpanBatch.parentCheck, singularBatches[0].ParentHash.Bytes()[:20], "invalid parent check")
assert.Equal(t, rawSpanBatch.l1OriginCheck, singularBatches[blockCount-1].EpochHash.Bytes()[:20], "invalid l1 origin check")
// check span batch payload
assert.Equal(t, int(rawSpanBatch.blockCount), len(singularBatches))
assert.Equal(t, rawSpanBatch.originBits.Bit(0), originChangedBit)
for i := 1; i < blockCount; i++ {
if rawSpanBatch.originBits.Bit(i) == 1 {
assert.Equal(t, singularBatches[i].EpochNum, singularBatches[i-1].EpochNum+1)
} else {
assert.Equal(t, singularBatches[i].EpochNum, singularBatches[i-1].EpochNum)
}
}
for i := 0; i < len(singularBatches); i++ {
txCount := len(singularBatches[i].Transactions)
assert.Equal(t, txCount, int(rawSpanBatch.blockTxCounts[i]))
}
// check invariants
endEpochNum := rawSpanBatch.l1OriginNum
assert.Equal(t, endEpochNum, uint64(singularBatches[blockCount-1].EpochNum))
// we do not check txs field because it has to be derived to be compared
}
}
func TestSpanBatchToSingularBatch(t *testing.T) {
rng := rand.New(rand.NewSource(0xbab0bab1))
chainID := new(big.Int).SetUint64(rng.Uint64())
for originChangedBit := 0; originChangedBit < 2; originChangedBit++ {
singularBatches := RandomValidConsecutiveSingularBatches(rng, chainID)
safeL2Head := testutils.RandomL2BlockRef(rng)
safeL2Head.Hash = common.BytesToHash(singularBatches[0].ParentHash[:])
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)
assert.NoError(t, err)
l1Origins := mockL1Origin(rng, rawSpanBatch, singularBatches)
singularBatches2, err := spanBatch.GetSingularBatches(l1Origins, safeL2Head)
assert.NoError(t, err)
// GetSingularBatches does not fill in parent hash of singular batches
// empty out parent hash for comparison
for i := 0; i < len(singularBatches); i++ {
singularBatches[i].ParentHash = common.Hash{}
}
// check parent hash is empty
for i := 0; i < len(singularBatches2); i++ {
assert.Equal(t, singularBatches2[i].ParentHash, common.Hash{})
}
assert.Equal(t, singularBatches, singularBatches2)
}
}
func TestSpanBatchReadTxData(t *testing.T) {
rng := rand.New(rand.NewSource(0x109550))
chainID := new(big.Int).SetUint64(rng.Uint64())
txCount := 64
signer := types.NewLondonSigner(chainID)
var rawTxs [][]byte
var txs []*types.Transaction
m := make(map[byte]int)
for i := 0; i < txCount; i++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
m[tx.Type()] += 1
rawTx, err := tx.MarshalBinary()
assert.NoError(t, err)
rawTxs = append(rawTxs, rawTx)
txs = append(txs, tx)
}
for i := 0; i < txCount; i++ {
r := bytes.NewReader(rawTxs[i])
_, txType, err := ReadTxData(r)
assert.NoError(t, err)
assert.Equal(t, int(txs[i].Type()), txType)
}
// 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 TestSpanBatchReadTxDataInvalid(t *testing.T) {
dummy, err := rlp.EncodeToBytes("dummy")
assert.NoError(t, err)
// test non list rlp decoding
r := bytes.NewReader(dummy)
_, _, err = ReadTxData(r)
assert.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
parentEpoch := uint64(singularBatches[0].EpochNum)
if originChangedBit == 1 {
parentEpoch -= 1
}
spanBatchBuilder := NewSpanBatchBuilder(parentEpoch, genesisTimeStamp, chainID)
assert.Equal(t, 0, spanBatchBuilder.GetBlockCount())
for i := 0; i < len(singularBatches); i++ {
spanBatchBuilder.AppendSingularBatch(singularBatches[i])
assert.Equal(t, i+1, spanBatchBuilder.GetBlockCount())
assert.Equal(t, singularBatches[0].ParentHash.Bytes()[:20], spanBatchBuilder.spanBatch.parentCheck)
assert.Equal(t, singularBatches[i].EpochHash.Bytes()[:20], spanBatchBuilder.spanBatch.l1OriginCheck)
}
rawSpanBatch, err := spanBatchBuilder.GetRawSpanBatch()
assert.NoError(t, err)
// compare with rawSpanBatch not using spanBatchBuilder
spanBatch := NewSpanBatch(singularBatches)
originChangedBit := uint(originChangedBit)
rawSpanBatch2, err := spanBatch.ToRawSpanBatch(originChangedBit, genesisTimeStamp, chainID)
assert.NoError(t, err)
assert.Equal(t, rawSpanBatch2, rawSpanBatch)
spanBatchBuilder.Reset()
assert.Equal(t, 0, spanBatchBuilder.GetBlockCount())
}
}
func TestSpanBatchMaxTxData(t *testing.T) {
rng := rand.New(rand.NewSource(0x177288))
invalidTx := types.NewTx(&types.DynamicFeeTx{
Data: testutils.RandomData(rng, MaxSpanBatchFieldSize+1),
})
txEncoded, err := invalidTx.MarshalBinary()
assert.NoError(t, err)
r := bytes.NewReader(txEncoded)
_, _, err = ReadTxData(r)
assert.ErrorIs(t, err, ErrTooBigSpanBatchFieldSize)
}
func TestSpanBatchMaxOriginBitsLength(t *testing.T) {
var sb RawSpanBatch
sb.blockCount = 0xFFFFFFFFFFFFFFFF
r := bytes.NewReader([]byte{})
err := sb.decodeOriginBits(r)
assert.ErrorIs(t, err, ErrTooBigSpanBatchFieldSize)
}
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-node/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")
}
package derive
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"io"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types"
"github.com/holiman/uint256"
)
type spanBatchTxs struct {
// this field must be manually set
totalBlockTxCount uint64
// 7 fields
contractCreationBits *big.Int
yParityBits *big.Int
txSigs []spanBatchSignature
txNonces []uint64
txGases []uint64
txTos []common.Address
txDatas []hexutil.Bytes
txTypes []int
}
type spanBatchSignature struct {
v uint64
r *uint256.Int
s *uint256.Int
}
// contractCreationBits is bitlist right-padded to a multiple of 8 bits
func (btx *spanBatchTxs) encodeContractCreationBits(w io.Writer) error {
contractCreationBitBufferLen := btx.totalBlockTxCount / 8
if btx.totalBlockTxCount%8 != 0 {
contractCreationBitBufferLen++
}
contractCreationBitBuffer := make([]byte, contractCreationBitBufferLen)
for i := 0; i < int(btx.totalBlockTxCount); i += 8 {
end := i + 8
if end < int(btx.totalBlockTxCount) {
end = int(btx.totalBlockTxCount)
}
var bits uint = 0
for j := i; j < end; j++ {
bits |= btx.contractCreationBits.Bit(j) << (j - i)
}
contractCreationBitBuffer[i/8] = byte(bits)
}
if _, err := w.Write(contractCreationBitBuffer); err != nil {
return fmt.Errorf("cannot write contract creation bits: %w", err)
}
return nil
}
// contractCreationBits is bitlist right-padded to a multiple of 8 bits
func (btx *spanBatchTxs) decodeContractCreationBits(r *bytes.Reader) error {
contractCreationBitBufferLen := btx.totalBlockTxCount / 8
if btx.totalBlockTxCount%8 != 0 {
contractCreationBitBufferLen++
}
// avoid out of memory before allocation
if contractCreationBitBufferLen > MaxSpanBatchFieldSize {
return ErrTooBigSpanBatchFieldSize
}
contractCreationBitBuffer := make([]byte, contractCreationBitBufferLen)
_, err := io.ReadFull(r, contractCreationBitBuffer)
if err != nil {
return fmt.Errorf("failed to read contract creation bits: %w", err)
}
contractCreationBits := new(big.Int)
for i := 0; i < int(btx.totalBlockTxCount); i += 8 {
end := i + 8
if end < int(btx.totalBlockTxCount) {
end = int(btx.totalBlockTxCount)
}
bits := contractCreationBitBuffer[i/8]
for j := i; j < end; j++ {
bit := uint((bits >> (j - i)) & 1)
contractCreationBits.SetBit(contractCreationBits, j, bit)
}
}
btx.contractCreationBits = contractCreationBits
return nil
}
func (btx *spanBatchTxs) contractCreationCount() uint64 {
if btx.contractCreationBits == nil {
panic("contract creation bits not set")
}
var result uint64 = 0
for i := 0; i < int(btx.totalBlockTxCount); i++ {
bit := btx.contractCreationBits.Bit(i)
if bit == 1 {
result++
}
}
return result
}
// yParityBits is bitlist right-padded to a multiple of 8 bits
func (btx *spanBatchTxs) encodeYParityBits(w io.Writer) error {
yParityBitBufferLen := btx.totalBlockTxCount / 8
if btx.totalBlockTxCount%8 != 0 {
yParityBitBufferLen++
}
yParityBitBuffer := make([]byte, yParityBitBufferLen)
for i := 0; i < int(btx.totalBlockTxCount); i += 8 {
end := i + 8
if end < int(btx.totalBlockTxCount) {
end = int(btx.totalBlockTxCount)
}
var bits uint = 0
for j := i; j < end; j++ {
bits |= btx.yParityBits.Bit(j) << (j - i)
}
yParityBitBuffer[i/8] = byte(bits)
}
if _, err := w.Write(yParityBitBuffer); err != nil {
return fmt.Errorf("cannot write y parity bits: %w", err)
}
return nil
}
func (btx *spanBatchTxs) encodeTxSigsRS(w io.Writer) error {
for _, txSig := range btx.txSigs {
rBuf := txSig.r.Bytes32()
if _, err := w.Write(rBuf[:]); err != nil {
return fmt.Errorf("cannot write tx sig r: %w", err)
}
sBuf := txSig.s.Bytes32()
if _, err := w.Write(sBuf[:]); err != nil {
return fmt.Errorf("cannot write tx sig s: %w", err)
}
}
return nil
}
func (btx *spanBatchTxs) encodeTxNonces(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
for _, txNonce := range btx.txNonces {
n := binary.PutUvarint(buf[:], txNonce)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write tx nonce: %w", err)
}
}
return nil
}
func (btx *spanBatchTxs) encodeTxGases(w io.Writer) error {
var buf [binary.MaxVarintLen64]byte
for _, txGas := range btx.txGases {
n := binary.PutUvarint(buf[:], txGas)
if _, err := w.Write(buf[:n]); err != nil {
return fmt.Errorf("cannot write tx gas: %w", err)
}
}
return nil
}
func (btx *spanBatchTxs) encodeTxTos(w io.Writer) error {
for _, txTo := range btx.txTos {
if _, err := w.Write(txTo.Bytes()); err != nil {
return fmt.Errorf("cannot write tx to address: %w", err)
}
}
return nil
}
func (btx *spanBatchTxs) encodeTxDatas(w io.Writer) error {
for _, txData := range btx.txDatas {
if _, err := w.Write(txData); err != nil {
return fmt.Errorf("cannot write tx data: %w", err)
}
}
return nil
}
// yParityBits is bitlist right-padded to a multiple of 8 bits
func (btx *spanBatchTxs) decodeYParityBits(r *bytes.Reader) error {
yParityBitBufferLen := btx.totalBlockTxCount / 8
if btx.totalBlockTxCount%8 != 0 {
yParityBitBufferLen++
}
// avoid out of memory before allocation
if yParityBitBufferLen > MaxSpanBatchFieldSize {
return ErrTooBigSpanBatchFieldSize
}
yParityBitBuffer := make([]byte, yParityBitBufferLen)
_, err := io.ReadFull(r, yParityBitBuffer)
if err != nil {
return fmt.Errorf("failed to read y parity bits: %w", err)
}
yParityBits := new(big.Int)
for i := 0; i < int(btx.totalBlockTxCount); i += 8 {
end := i + 8
if end < int(btx.totalBlockTxCount) {
end = int(btx.totalBlockTxCount)
}
bits := yParityBitBuffer[i/8]
for j := i; j < end; j++ {
bit := uint((bits >> (j - i)) & 1)
yParityBits.SetBit(yParityBits, j, bit)
}
}
btx.yParityBits = yParityBits
return nil
}
func (btx *spanBatchTxs) decodeTxSigsRS(r *bytes.Reader) error {
var txSigs []spanBatchSignature
var sigBuffer [32]byte
for i := 0; i < int(btx.totalBlockTxCount); i++ {
var txSig spanBatchSignature
_, err := io.ReadFull(r, sigBuffer[:])
if err != nil {
return fmt.Errorf("failed to read tx sig r: %w", err)
}
txSig.r, _ = uint256.FromBig(new(big.Int).SetBytes(sigBuffer[:]))
_, err = io.ReadFull(r, sigBuffer[:])
if err != nil {
return fmt.Errorf("failed to read tx sig s: %w", err)
}
txSig.s, _ = uint256.FromBig(new(big.Int).SetBytes(sigBuffer[:]))
txSigs = append(txSigs, txSig)
}
btx.txSigs = txSigs
return nil
}
func (btx *spanBatchTxs) decodeTxNonces(r *bytes.Reader) error {
var txNonces []uint64
for i := 0; i < int(btx.totalBlockTxCount); i++ {
txNonce, err := binary.ReadUvarint(r)
if err != nil {
return fmt.Errorf("failed to read tx nonce: %w", err)
}
txNonces = append(txNonces, txNonce)
}
btx.txNonces = txNonces
return nil
}
func (btx *spanBatchTxs) decodeTxGases(r *bytes.Reader) error {
var txGases []uint64
for i := 0; i < int(btx.totalBlockTxCount); i++ {
txGas, err := binary.ReadUvarint(r)
if err != nil {
return fmt.Errorf("failed to read tx gas: %w", err)
}
txGases = append(txGases, txGas)
}
btx.txGases = txGases
return nil
}
func (btx *spanBatchTxs) decodeTxTos(r *bytes.Reader) error {
var txTos []common.Address
txToBuffer := make([]byte, common.AddressLength)
contractCreationCount := btx.contractCreationCount()
for i := 0; i < int(btx.totalBlockTxCount-contractCreationCount); i++ {
_, err := io.ReadFull(r, txToBuffer)
if err != nil {
return fmt.Errorf("failed to read tx to address: %w", err)
}
txTos = append(txTos, common.BytesToAddress(txToBuffer))
}
btx.txTos = txTos
return nil
}
func (btx *spanBatchTxs) decodeTxDatas(r *bytes.Reader) error {
var txDatas []hexutil.Bytes
var txTypes []int
// Do not need txDataHeader because RLP byte stream already includes length info
for i := 0; i < int(btx.totalBlockTxCount); i++ {
txData, txType, err := ReadTxData(r)
if err != nil {
return err
}
txDatas = append(txDatas, txData)
txTypes = append(txTypes, txType)
}
btx.txDatas = txDatas
btx.txTypes = txTypes
return nil
}
func (btx *spanBatchTxs) recoverV(chainID *big.Int) {
if len(btx.txTypes) != len(btx.txSigs) {
panic("tx type length and tx sigs length mismatch")
}
for idx, txType := range btx.txTypes {
bit := uint64(btx.yParityBits.Bit(idx))
var v uint64
switch txType {
case types.LegacyTxType:
// EIP155
v = chainID.Uint64()*2 + 35 + bit
case types.AccessListTxType:
v = bit
case types.DynamicFeeTxType:
v = bit
default:
panic(fmt.Sprintf("invalid tx type: %d", txType))
}
btx.txSigs[idx].v = v
}
}
func (btx *spanBatchTxs) encode(w io.Writer) error {
if err := btx.encodeContractCreationBits(w); err != nil {
return err
}
if err := btx.encodeYParityBits(w); err != nil {
return err
}
if err := btx.encodeTxSigsRS(w); err != nil {
return err
}
if err := btx.encodeTxTos(w); err != nil {
return err
}
if err := btx.encodeTxDatas(w); err != nil {
return err
}
if err := btx.encodeTxNonces(w); err != nil {
return err
}
if err := btx.encodeTxGases(w); err != nil {
return err
}
return nil
}
func (btx *spanBatchTxs) decode(r *bytes.Reader) error {
if err := btx.decodeContractCreationBits(r); err != nil {
return err
}
if err := btx.decodeYParityBits(r); err != nil {
return err
}
if err := btx.decodeTxSigsRS(r); err != nil {
return err
}
if err := btx.decodeTxTos(r); err != nil {
return err
}
if err := btx.decodeTxDatas(r); err != nil {
return err
}
if err := btx.decodeTxNonces(r); err != nil {
return err
}
if err := btx.decodeTxGases(r); err != nil {
return err
}
return nil
}
func (btx *spanBatchTxs) fullTxs(chainID *big.Int) ([][]byte, error) {
var txs [][]byte
toIdx := 0
for idx := 0; idx < int(btx.totalBlockTxCount); idx++ {
var stx spanBatchTx
if err := stx.UnmarshalBinary(btx.txDatas[idx]); err != nil {
return nil, err
}
nonce := btx.txNonces[idx]
gas := btx.txGases[idx]
var to *common.Address = nil
bit := btx.contractCreationBits.Bit(idx)
if bit == 0 {
if len(btx.txTos) <= toIdx {
return nil, errors.New("tx to not enough")
}
to = &btx.txTos[toIdx]
toIdx++
}
v := new(big.Int).SetUint64(btx.txSigs[idx].v)
r := btx.txSigs[idx].r.ToBig()
s := btx.txSigs[idx].s.ToBig()
tx, err := stx.convertToFullTx(nonce, gas, to, chainID, v, r, s)
if err != nil {
return nil, err
}
encodedTx, err := tx.MarshalBinary()
if err != nil {
return nil, err
}
txs = append(txs, encodedTx)
}
return txs, nil
}
func convertVToYParity(v uint64, txType int) uint {
var yParityBit uint
switch txType {
case types.LegacyTxType:
// EIP155: v = 2 * chainID + 35 + yParity
// v - 35 = yParity (mod 2)
yParityBit = uint((v - 35) & 1)
case types.AccessListTxType:
yParityBit = uint(v)
case types.DynamicFeeTxType:
yParityBit = uint(v)
default:
panic(fmt.Sprintf("invalid tx type: %d", txType))
}
return yParityBit
}
func newSpanBatchTxs(txs [][]byte, chainID *big.Int) (*spanBatchTxs, 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)
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")
}
var txSig spanBatchSignature
v, r, s := tx.RawSignatureValues()
R, _ := uint256.FromBig(r)
S, _ := uint256.FromBig(s)
txSig.v = v.Uint64()
txSig.r = R
txSig.s = S
txSigs = append(txSigs, txSig)
contractCreationBit := uint(1)
if tx.To() != nil {
txTos = append(txTos, *tx.To())
contractCreationBit = uint(0)
}
contractCreationBits.SetBit(contractCreationBits, idx, contractCreationBit)
yParityBit := convertVToYParity(txSig.v, int(tx.Type()))
yParityBits.SetBit(yParityBits, idx, yParityBit)
txNonces = append(txNonces, tx.Nonce())
txGases = append(txGases, tx.Gas())
stx, err := newSpanBatchTx(tx)
if err != nil {
return nil, err
}
txData, err := stx.MarshalBinary()
if err != nil {
return nil, 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,
}, nil
}
package derive
import (
"bytes"
"math/big"
"math/rand"
"testing"
"github.com/ethereum-optimism/optimism/op-node/testutils"
"github.com/ethereum/go-ethereum/core/types"
"github.com/holiman/uint256"
"github.com/stretchr/testify/assert"
)
func TestSpanBatchTxsContractCreationBits(t *testing.T) {
rng := rand.New(rand.NewSource(0x1234567))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
contractCreationBits := rawSpanBatch.txs.contractCreationBits
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.contractCreationBits = contractCreationBits
sbt.totalBlockTxCount = totalBlockTxCount
var buf bytes.Buffer
err := sbt.encodeContractCreationBits(&buf)
assert.NoError(t, err)
// contractCreationBit field is fixed length: single bit
contractCreationBitBufferLen := totalBlockTxCount / 8
if totalBlockTxCount%8 != 0 {
contractCreationBitBufferLen++
}
assert.Equal(t, buf.Len(), int(contractCreationBitBufferLen))
result := buf.Bytes()
sbt.contractCreationBits = nil
r := bytes.NewReader(result)
err = sbt.decodeContractCreationBits(r)
assert.NoError(t, err)
assert.Equal(t, contractCreationBits, sbt.contractCreationBits)
}
func TestSpanBatchTxsContractCreationCount(t *testing.T) {
rng := rand.New(rand.NewSource(0x1337))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
contractCreationBits := rawSpanBatch.txs.contractCreationBits
contractCreationCount := rawSpanBatch.txs.contractCreationCount()
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.contractCreationBits = contractCreationBits
sbt.totalBlockTxCount = totalBlockTxCount
var buf bytes.Buffer
err := sbt.encodeContractCreationBits(&buf)
assert.NoError(t, err)
result := buf.Bytes()
sbt.contractCreationBits = nil
r := bytes.NewReader(result)
err = sbt.decodeContractCreationBits(r)
assert.NoError(t, err)
assert.Equal(t, contractCreationCount, sbt.contractCreationCount())
}
func TestSpanBatchTxsYParityBits(t *testing.T) {
rng := rand.New(rand.NewSource(0x7331))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
yParityBits := rawSpanBatch.txs.yParityBits
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.yParityBits = yParityBits
sbt.totalBlockTxCount = totalBlockTxCount
var buf bytes.Buffer
err := sbt.encodeYParityBits(&buf)
assert.NoError(t, err)
// yParityBit field is fixed length: single bit
yParityBitBufferLen := totalBlockTxCount / 8
if totalBlockTxCount%8 != 0 {
yParityBitBufferLen++
}
assert.Equal(t, buf.Len(), int(yParityBitBufferLen))
result := buf.Bytes()
sbt.yParityBits = nil
r := bytes.NewReader(result)
err = sbt.decodeYParityBits(r)
assert.NoError(t, err)
assert.Equal(t, yParityBits, sbt.yParityBits)
}
func TestSpanBatchTxsTxSigs(t *testing.T) {
rng := rand.New(rand.NewSource(0x73311337))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
txSigs := rawSpanBatch.txs.txSigs
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.totalBlockTxCount = totalBlockTxCount
sbt.txSigs = txSigs
var buf bytes.Buffer
err := sbt.encodeTxSigsRS(&buf)
assert.NoError(t, err)
// txSig field is fixed length: 32 byte + 32 byte = 64 byte
assert.Equal(t, buf.Len(), 64*int(totalBlockTxCount))
result := buf.Bytes()
sbt.txSigs = nil
r := bytes.NewReader(result)
err = sbt.decodeTxSigsRS(r)
assert.NoError(t, err)
// v field is not set
for i := 0; i < int(totalBlockTxCount); i++ {
assert.Equal(t, txSigs[i].r, sbt.txSigs[i].r)
assert.Equal(t, txSigs[i].s, sbt.txSigs[i].s)
}
}
func TestSpanBatchTxsTxNonces(t *testing.T) {
rng := rand.New(rand.NewSource(0x123456))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
txNonces := rawSpanBatch.txs.txNonces
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.totalBlockTxCount = totalBlockTxCount
sbt.txNonces = txNonces
var buf bytes.Buffer
err := sbt.encodeTxNonces(&buf)
assert.NoError(t, err)
result := buf.Bytes()
sbt.txNonces = nil
r := bytes.NewReader(result)
err = sbt.decodeTxNonces(r)
assert.NoError(t, err)
assert.Equal(t, txNonces, sbt.txNonces)
}
func TestSpanBatchTxsTxGases(t *testing.T) {
rng := rand.New(rand.NewSource(0x12345))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
txGases := rawSpanBatch.txs.txGases
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.totalBlockTxCount = totalBlockTxCount
sbt.txGases = txGases
var buf bytes.Buffer
err := sbt.encodeTxGases(&buf)
assert.NoError(t, err)
result := buf.Bytes()
sbt.txGases = nil
r := bytes.NewReader(result)
err = sbt.decodeTxGases(r)
assert.NoError(t, err)
assert.Equal(t, txGases, sbt.txGases)
}
func TestSpanBatchTxsTxTos(t *testing.T) {
rng := rand.New(rand.NewSource(0x54321))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
txTos := rawSpanBatch.txs.txTos
contractCreationBits := rawSpanBatch.txs.contractCreationBits
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.txTos = txTos
// creation bits and block tx count must be se to decode tos
sbt.contractCreationBits = contractCreationBits
sbt.totalBlockTxCount = totalBlockTxCount
var buf bytes.Buffer
err := sbt.encodeTxTos(&buf)
assert.NoError(t, err)
// to field is fixed length: 20 bytes
assert.Equal(t, buf.Len(), 20*len(txTos))
result := buf.Bytes()
sbt.txTos = nil
r := bytes.NewReader(result)
err = sbt.decodeTxTos(r)
assert.NoError(t, err)
assert.Equal(t, txTos, sbt.txTos)
}
func TestSpanBatchTxsTxDatas(t *testing.T) {
rng := rand.New(rand.NewSource(0x1234))
chainID := big.NewInt(rng.Int63n(1000))
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
txDatas := rawSpanBatch.txs.txDatas
txTypes := rawSpanBatch.txs.txTypes
totalBlockTxCount := rawSpanBatch.txs.totalBlockTxCount
var sbt spanBatchTxs
sbt.totalBlockTxCount = totalBlockTxCount
sbt.txDatas = txDatas
var buf bytes.Buffer
err := sbt.encodeTxDatas(&buf)
assert.NoError(t, err)
result := buf.Bytes()
sbt.txDatas = nil
sbt.txTypes = nil
r := bytes.NewReader(result)
err = sbt.decodeTxDatas(r)
assert.NoError(t, err)
assert.Equal(t, txDatas, sbt.txDatas)
assert.Equal(t, txTypes, sbt.txTypes)
}
func TestSpanBatchTxsRecoverV(t *testing.T) {
rng := rand.New(rand.NewSource(0x123))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
totalblockTxCount := rng.Intn(100)
var spanBatchTxs spanBatchTxs
var txTypes []int
var txSigs []spanBatchSignature
var originalVs []uint64
yParityBits := new(big.Int)
for idx := 0; idx < totalblockTxCount; idx++ {
tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer)
txTypes = append(txTypes, int(tx.Type()))
var txSig spanBatchSignature
v, r, s := tx.RawSignatureValues()
// Do not fill in txSig.V
txSig.r, _ = uint256.FromBig(r)
txSig.s, _ = uint256.FromBig(s)
txSigs = append(txSigs, txSig)
originalVs = append(originalVs, v.Uint64())
yParityBit := convertVToYParity(v.Uint64(), int(tx.Type()))
yParityBits.SetBit(yParityBits, idx, yParityBit)
}
spanBatchTxs.yParityBits = yParityBits
spanBatchTxs.txSigs = txSigs
spanBatchTxs.txTypes = txTypes
// recover txSig.v
spanBatchTxs.recoverV(chainID)
var recoveredVs []uint64
for _, txSig := range spanBatchTxs.txSigs {
recoveredVs = append(recoveredVs, txSig.v)
}
assert.Equal(t, originalVs, recoveredVs, "recovered v mismatch")
}
func TestSpanBatchTxsRoundTrip(t *testing.T) {
rng := rand.New(rand.NewSource(0x73311337))
chainID := big.NewInt(rng.Int63n(1000))
for i := 0; i < 4; i++ {
rawSpanBatch := RandomRawSpanBatch(rng, chainID)
sbt := rawSpanBatch.txs
totalBlockTxCount := sbt.totalBlockTxCount
var buf bytes.Buffer
err := sbt.encode(&buf)
assert.NoError(t, err)
result := buf.Bytes()
r := bytes.NewReader(result)
var sbt2 spanBatchTxs
sbt2.totalBlockTxCount = totalBlockTxCount
err = sbt2.decode(r)
assert.NoError(t, err)
sbt2.recoverV(chainID)
assert.Equal(t, sbt, &sbt2)
}
}
func TestSpanBatchTxsRoundTripFullTxs(t *testing.T) {
rng := rand.New(rand.NewSource(0x13377331))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
for i := 0; i < 4; i++ {
totalblockTxCounts := uint64(1 + rng.Int()&0xFF)
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()
assert.NoError(t, err)
txs = append(txs, rawTx)
}
sbt, err := newSpanBatchTxs(txs, chainID)
assert.NoError(t, err)
txs2, err := sbt.fullTxs(chainID)
assert.NoError(t, err)
assert.Equal(t, txs, txs2)
}
}
func TestSpanBatchTxsRecoverVInvalidTxType(t *testing.T) {
defer func() {
if r := recover(); r == nil {
t.Errorf("The code did not panic")
}
}()
rng := rand.New(rand.NewSource(0x321))
chainID := big.NewInt(rng.Int63n(1000))
var sbt spanBatchTxs
sbt.txTypes = []int{types.DepositTxType}
sbt.txSigs = []spanBatchSignature{{v: 0, r: nil, s: nil}}
sbt.yParityBits = new(big.Int)
// expect panic
sbt.recoverV(chainID)
}
func TestSpanBatchTxsFullTxNotEnoughTxTos(t *testing.T) {
rng := rand.New(rand.NewSource(0x13572468))
chainID := big.NewInt(rng.Int63n(1000))
signer := types.NewLondonSigner(chainID)
totalblockTxCounts := uint64(1 + rng.Int()&0xFF)
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()
assert.NoError(t, err)
txs = append(txs, rawTx)
}
sbt, err := newSpanBatchTxs(txs, chainID)
assert.NoError(t, err)
// drop single to field
sbt.txTos = sbt.txTos[:len(sbt.txTos)-2]
_, err = sbt.fullTxs(chainID)
assert.EqualError(t, err, "tx to not enough")
}
func TestSpanBatchTxsMaxContractCreationBitsLength(t *testing.T) {
var sbt spanBatchTxs
sbt.totalBlockTxCount = 0xFFFFFFFFFFFFFFFF
r := bytes.NewReader([]byte{})
err := sbt.decodeContractCreationBits(r)
assert.ErrorIs(t, err, ErrTooBigSpanBatchFieldSize)
}
func TestSpanBatchTxsMaxYParityBitsLength(t *testing.T) {
var sb RawSpanBatch
sb.blockCount = 0xFFFFFFFFFFFFFFFF
r := bytes.NewReader([]byte{})
err := sb.decodeOriginBits(r)
assert.ErrorIs(t, err, ErrTooBigSpanBatchFieldSize)
}
......@@ -144,17 +144,47 @@ func RandomTx(rng *rand.Rand, baseFee *big.Int, signer types.Signer) *types.Tran
gas := params.TxGas + uint64(rng.Int63n(2_000_000))
key := InsecureRandomKey(rng)
tip := big.NewInt(rng.Int63n(10 * params.GWei))
tx, err := types.SignNewTx(key, signer, &types.DynamicFeeTx{
ChainID: signer.ChainID(),
Nonce: rng.Uint64(),
GasTipCap: tip,
GasFeeCap: new(big.Int).Add(baseFee, tip),
Gas: gas,
To: RandomTo(rng),
Value: RandomETH(rng, 10),
Data: RandomData(rng, rng.Intn(1000)),
AccessList: nil,
})
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(),
Nonce: rng.Uint64(),
GasTipCap: tip,
GasFeeCap: new(big.Int).Add(baseFee, tip),
Gas: gas,
To: RandomTo(rng),
Value: RandomETH(rng, 10),
Data: RandomData(rng, rng.Intn(1000)),
AccessList: nil,
}
default:
panic("invalid tx type")
}
tx, err := types.SignNewTx(key, signer, txData)
if err != nil {
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