derive

package
v0.0.0-...-70e280b Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Oct 31, 2024 License: MIT, MIT Imports: 38 Imported by: 0

Documentation

Overview

Package derive provides the data transformation functions that take L1 data and turn it into L2 blocks and results. Certain L2 data is also able to turned back into L1 data.

The flow is data is as follows receipts, batches -> eth.PayloadAttributes, by parsing the L1 data and deriving L2 inputs l2.PayloadAttributes -> l2.ExecutionPayload, by running the EVM (using an Execution Engine) L2 block -> Corresponding L1 block info, by parsing the first deposited transaction

The Payload Attributes derivation stage is a pure function. The Execution Payload derivation stage relies on the L2 execution engine to perform the state update. The inversion step is a pure function.

The steps should be kept separate to enable easier testing.

Index

Constants

View Source
const (
	// SingularBatchType is the first version of Batch format, representing a single L2 block.
	SingularBatchType = 0
	// SpanBatchType is the Batch version used after Delta hard fork, representing a span of L2 blocks.
	SpanBatchType = 1
)
View Source
const (
	// BatchDrop indicates that the batch is invalid, and will always be in the future, unless we reorg
	BatchDrop = iota
	// BatchAccept indicates that the batch is valid and should be processed
	BatchAccept
	// BatchUndecided indicates we are lacking L1 information until we can proceed batch filtering
	BatchUndecided
	// BatchFuture indicates that the batch may be valid, but cannot be processed yet and should be checked again later
	BatchFuture
)
View Source
const (
	ZlibCM8  = 8
	ZlibCM15 = 15
)
View Source
const (
	UserDepositSourceDomain    = 0
	L1InfoDepositSourceDomain  = 1
	UpgradeDepositSourceDomain = 2
)
View Source
const (
	L1InfoFuncBedrockSignature = "setL1BlockValues(uint64,uint64,uint256,bytes32,uint64,bytes32,uint256,uint256)"
	L1InfoFuncEcotoneSignature = "setL1BlockValuesEcotone()"
	L1InfoArguments            = 8
	L1InfoBedrockLen           = 4 + 32*L1InfoArguments
	L1InfoEcotoneLen           = 4 + 32*5 // after Ecotone upgrade, args are packed into 5 32-byte slots
)
View Source
const ChannelIDLength = 16

ChannelIDLength defines the length of the channel IDs

View Source
const (
	ChannelVersionBrotli byte = 0x01
)
View Source
const DerivationVersion0 = 0
View Source
const DerivationVersion1 = plasma.TxDataVersion1

DerivationVersion1 is reserved for batcher transactions containing plasma commitments.

View Source
const FrameV0OverHeadSize = 23

FrameV0OverHeadSize is the absolute minimum size of a frame. This is the fixed overhead frame size, calculated as specified in the Frame Format specs: 16 + 2 + 4 + 1 = 23 bytes.

View Source
const MaxFrameLen = 1_000_000

Frames cannot be larger than 1 MB. Data transactions that carry frames are generally not larger than 128 KB due to L1 network conditions, but we leave space to grow larger anyway (gas limit allows for more data).

View Source
const MaxSpanBatchElementCount = 10_000_000

MaxSpanBatchElementCount is the maximum number of blocks, transactions in total, or transaction per block allowed in a span batch.

View Source
const (
	RegolithSystemTxGas = 1_000_000
)
View Source
const UpgradeToFuncSignature = "upgradeTo(address)"

Variables

View Source
var (
	ErrMaxFrameSizeTooSmall    = errors.New("maxSize is too small to fit the fixed frame overhead")
	ErrNotDepositTx            = errors.New("first transaction in block is not a deposit tx")
	ErrTooManyRLPBytes         = errors.New("batch would cause RLP bytes to go over limit")
	ErrChannelOutAlreadyClosed = errors.New("channel-out already closed")
	ErrCompressorFull          = errors.New("compressor is full")
)
View Source
var (
	DepositEventABI      = "TransactionDeposited(address,address,uint256,bytes)"
	DepositEventABIHash  = crypto.Keccak256Hash([]byte(DepositEventABI))
	DepositEventVersion0 = common.Hash{}
)
View Source
var (
	// known address w/ zero txns
	L1BlockDeployerAddress        = common.HexToAddress("0x4210000000000000000000000000000000000000")
	GasPriceOracleDeployerAddress = common.HexToAddress("0x4210000000000000000000000000000000000001")

	EIP4788From = common.HexToAddress("0x0B799C86a49DEeb90402691F1041aa3AF2d3C875")

	UpgradeToFuncBytes4 = crypto.Keccak256([]byte(UpgradeToFuncSignature))[:4]
)
View Source
var (
	ErrTypedTxTooShort = errors.New("typed transaction data too short")

	// NotEnoughData implies that the function currently does not have enough data to progress
	// but if it is retried enough times, it will eventually return a real value or io.EOF
	NotEnoughData = errors.New("not enough data")

	// EngineELSyncing implies that the execution engine is currently in progress of syncing.
	EngineELSyncing = errors.New("engine is performing EL sync")

	// Sentinel errors, use these to get the severity of errors by calling
	// errors.Is(err, ErrTemporary) for example.
	ErrTemporary = NewTemporaryError(nil)
	ErrReset     = NewResetError(nil)
	ErrCritical  = NewCriticalError(nil)
)
View Source
var (
	L1InfoFuncBedrockBytes4 = crypto.Keccak256([]byte(L1InfoFuncBedrockSignature))[:4]
	L1InfoFuncEcotoneBytes4 = crypto.Keccak256([]byte(L1InfoFuncEcotoneSignature))[:4]
	L1InfoDepositerAddress  = common.HexToAddress("0xdeaddeaddeaddeaddeaddeaddeaddeaddead0001")
	L1BlockAddress          = predeploys.L1BlockAddr
)
View Source
var (
	SystemConfigUpdateBatcher           = common.Hash{31: 0}
	SystemConfigUpdateGasConfig         = common.Hash{31: 1}
	SystemConfigUpdateGasLimit          = common.Hash{31: 2}
	SystemConfigUpdateUnsafeBlockSigner = common.Hash{31: 3}
)
View Source
var (
	ConfigUpdateEventABI      = "ConfigUpdate(uint256,uint8,bytes)"
	ConfigUpdateEventABIHash  = crypto.Keccak256Hash([]byte(ConfigUpdateEventABI))
	ConfigUpdateEventVersion0 = common.Hash{}
)
View Source
var CompressionAlgos = []CompressionAlgo{
	Zlib,
	Brotli,
	Brotli9,
	Brotli10,
	Brotli11,
}
View Source
var DuplicateErr = errors.New("duplicate frame")

DuplicateErr is returned when a newly read frame is already known

View Source
var ErrEmptySpanBatch = errors.New("span-batch must not be empty")
View Source
var ErrNoFCUNeeded = errors.New("no FCU call was needed")
View Source
var ErrTooBigSpanBatchSize = errors.New("span batch size limit reached")
View Source
var (
	GasPriceOracleFjordDeployerAddress = common.HexToAddress("0x4210000000000000000000000000000000000002")
)

Functions

func BatchReader

func BatchReader(r io.Reader, maxRLPBytesPerChannel uint64, isFjord bool) (func() (*BatchData, error), error)

BatchReader provides a function that iteratively consumes batches from the reader. The L1Inclusion block is also provided at creation time. Warning: the batch reader can read every batch-type. The caller of the batch-reader should filter the results.

func BlockToSingularBatch

func BlockToSingularBatch(rollupCfg *rollup.Config, block *types.Block) (*SingularBatch, *L1BlockInfo, error)

BlockToSingularBatch transforms a block into a batch object that can easily be RLP encoded.

func DataFromEVMTransactions

func DataFromEVMTransactions(dsCfg DataSourceConfig, batcherAddr common.Address, txs types.Transactions, log log.Logger) []eth.Data

DataFromEVMTransactions filters all of the transactions and returns the calldata from transactions that are sent to the batch inbox address from the batch sender address. This will return an empty array if no valid transactions are found.

func DeriveDeposits

func DeriveDeposits(receipts []*types.Receipt, depositContractAddr common.Address) ([]hexutil.Bytes, error)

func EcotoneNetworkUpgradeTransactions

func EcotoneNetworkUpgradeTransactions() ([]hexutil.Bytes, error)

func FjordNetworkUpgradeTransactions

func FjordNetworkUpgradeTransactions() ([]hexutil.Bytes, error)

FjordNetworkUpgradeTransactions returns the transactions required to upgrade the Fjord network.

func ForceCloseTxData

func ForceCloseTxData(frames []Frame) ([]byte, error)

ForceCloseTxData generates the transaction data for a transaction which will force close a channel. It should be given every frame of that channel which has been submitted on chain. The frames should be given in order that they appear on L1.

func GetBrotliLevel

func GetBrotliLevel(algo CompressionAlgo) int

func L1InfoDeposit

func L1InfoDeposit(rollupCfg *rollup.Config, sysCfg eth.SystemConfig, seqNumber uint64, block eth.BlockInfo, l2BlockTime uint64) (*types.DepositTx, error)

L1InfoDeposit creates a L1 Info deposit transaction based on the L1 block, and the L2 block-height difference with the start of the epoch.

func L1InfoDepositBytes

func L1InfoDepositBytes(rollupCfg *rollup.Config, sysCfg eth.SystemConfig, seqNumber uint64, l1Info eth.BlockInfo, l2BlockTime uint64) ([]byte, error)

L1InfoDepositBytes returns a serialized L1-info attributes transaction.

func L2BlockToBlockRef

func L2BlockToBlockRef(rollupCfg *rollup.Config, block L2BlockRefSource) (eth.L2BlockRef, error)

L2BlockToBlockRef extracts the essential L2BlockRef information from an L2 block ref source, falling back to genesis information if necessary.

func MarshalDepositLogEvent

func MarshalDepositLogEvent(depositContractAddr common.Address, deposit *types.DepositTx) (*types.Log, error)

MarshalDepositLogEvent returns an EVM log entry that encodes a TransactionDeposited event from the deposit contract. This is the reverse of the deposit transaction derivation.

func NewCriticalError

func NewCriticalError(err error) error

NewCriticalError returns a critical error.

func NewError

func NewError(err error, level Level) error

NewError returns a custom Error.

func NewResetError

func NewResetError(err error) error

NewResetError returns a pipeline reset error.

func NewTemporaryError

func NewTemporaryError(err error) error

NewTemporaryError returns a temporary error.

func PayloadToBlockRef

func PayloadToBlockRef(rollupCfg *rollup.Config, payload *eth.ExecutionPayload) (eth.L2BlockRef, error)

PayloadToBlockRef extracts the essential L2BlockRef information from an execution payload, falling back to genesis information if necessary.

func PayloadToSystemConfig

func PayloadToSystemConfig(rollupCfg *rollup.Config, payload *eth.ExecutionPayload) (eth.SystemConfig, error)

func ProcessSystemConfigUpdateLogEvent

func ProcessSystemConfigUpdateLogEvent(destSysCfg *eth.SystemConfig, ev *types.Log, rollupCfg *rollup.Config, l1Time uint64) error

ProcessSystemConfigUpdateLogEvent decodes an EVM log entry emitted by the system config contract and applies it as a system config change.

parse log data for:

event ConfigUpdate(
    uint256 indexed version,
    UpdateType indexed updateType,
    bytes data
);

func ReadTxData

func ReadTxData(r *bytes.Reader) ([]byte, int, error)

ReadTxData reads raw RLP tx data from reader and returns txData and txType

func UnmarshalDepositLogEvent

func UnmarshalDepositLogEvent(ev *types.Log) (*types.DepositTx, error)

UnmarshalDepositLogEvent decodes an EVM log entry emitted by the deposit contract into typed deposit data.

parse log data for:

event TransactionDeposited(
    address indexed from,
    address indexed to,
    uint256 indexed version,
    bytes opaqueData
);

Additionally, the event log-index and

func UpdateSystemConfigWithL1Receipts

func UpdateSystemConfigWithL1Receipts(sysCfg *eth.SystemConfig, receipts []*types.Receipt, cfg *rollup.Config, l1Time uint64) error

UpdateSystemConfigWithL1Receipts filters all L1 receipts to find config updates and applies the config updates to the given sysCfg

func UserDeposits

func UserDeposits(receipts []*types.Receipt, depositContractAddr common.Address) ([]*types.DepositTx, error)

UserDeposits transforms the L2 block-height and L1 receipts into the transaction inputs for a full L2 block

func ValidCompressionAlgo

func ValidCompressionAlgo(value CompressionAlgo) bool

Types

type AttributesBuilder

type AttributesBuilder interface {
	PreparePayloadAttributes(ctx context.Context, l2Parent eth.L2BlockRef, epoch eth.BlockID) (attrs *eth.PayloadAttributes, err error)
}

type AttributesHandler

type AttributesHandler interface {
	// HasAttributes returns if there are any block attributes to process.
	// HasAttributes is for EngineQueue testing only, and can be removed when attribute processing is fully independent.
	HasAttributes() bool
	// SetAttributes overwrites the set of attributes. This may be nil, to clear what may be processed next.
	SetAttributes(attributes *AttributesWithParent)
	// Proceed runs one attempt of processing attributes, if any.
	// Proceed returns io.EOF if there are no attributes to process.
	Proceed(ctx context.Context) error
}

type AttributesQueue

type AttributesQueue struct {
	// contains filtered or unexported fields
}

func NewAttributesQueue

func NewAttributesQueue(log log.Logger, cfg *rollup.Config, builder AttributesBuilder, prev *BatchQueue) *AttributesQueue

func (*AttributesQueue) NextAttributes

func (aq *AttributesQueue) NextAttributes(ctx context.Context, parent eth.L2BlockRef) (*AttributesWithParent, error)

func (*AttributesQueue) Origin

func (aq *AttributesQueue) Origin() eth.L1BlockRef

func (*AttributesQueue) Reset

type AttributesWithParent

type AttributesWithParent struct {
	Attributes   *eth.PayloadAttributes
	Parent       eth.L2BlockRef
	IsLastInSpan bool
}

type BaseChannelCompressor

type BaseChannelCompressor struct {
	CompressorWriter
	// contains filtered or unexported fields
}

func (*BaseChannelCompressor) GetCompressed

func (bcc *BaseChannelCompressor) GetCompressed() *bytes.Buffer

func (*BaseChannelCompressor) Len

func (bcc *BaseChannelCompressor) Len() int

func (*BaseChannelCompressor) Read

func (bcc *BaseChannelCompressor) Read(p []byte) (int, error)

type Batch

type Batch interface {
	GetBatchType() int
	GetTimestamp() uint64
	LogContext(log.Logger) log.Logger
	AsSingularBatch() (*SingularBatch, bool)
	AsSpanBatch() (*SpanBatch, bool)
}

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 BatchData

type BatchData struct {
	ComprAlgo CompressionAlgo
	// contains filtered or unexported fields
}

BatchData is used to represent the typed encoding & decoding. and wraps around a single interface InnerBatchData. Further fields such as cache can be added in the future, without embedding each type of InnerBatchData. Similar design with op-geth's types.Transaction struct.

func NewBatchData

func NewBatchData(inner InnerBatchData) *BatchData

NewBatchData creates a new BatchData

func (*BatchData) DecodeRLP

func (b *BatchData) DecodeRLP(s *rlp.Stream) error

DecodeRLP implements rlp.Decoder

func (*BatchData) EncodeRLP

func (b *BatchData) EncodeRLP(w io.Writer) error

EncodeRLP implements rlp.Encoder

func (*BatchData) GetBatchType

func (bd *BatchData) GetBatchType() uint8

func (*BatchData) MarshalBinary

func (b *BatchData) MarshalBinary() ([]byte, error)

MarshalBinary returns the canonical encoding of the batch.

func (*BatchData) UnmarshalBinary

func (b *BatchData) UnmarshalBinary(data []byte) error

UnmarshalBinary decodes the canonical encoding of batch.

type BatchQueue

type BatchQueue struct {
	// contains filtered or unexported fields
}

BatchQueue contains a set of batches for every L1 block. L1 blocks are contiguous and this does not support reorgs.

func NewBatchQueue

func NewBatchQueue(log log.Logger, cfg *rollup.Config, prev NextBatchProvider, l2 SafeBlockFetcher) *BatchQueue

NewBatchQueue creates a BatchQueue, which should be Reset(origin) before use.

func (*BatchQueue) AddBatch

func (bq *BatchQueue) AddBatch(ctx context.Context, batch Batch, parent eth.L2BlockRef)

func (*BatchQueue) NextBatch

func (bq *BatchQueue) NextBatch(ctx context.Context, parent eth.L2BlockRef) (*SingularBatch, bool, error)

NextBatch return next valid batch upon the given safe head. It also returns the boolean that indicates if the batch is the last block in the batch.

func (*BatchQueue) Origin

func (bq *BatchQueue) Origin() eth.L1BlockRef

func (*BatchQueue) Reset

func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error

type BatchValidity

type BatchValidity uint8

func CheckBatch

func CheckBatch(ctx context.Context, cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef,
	l2SafeHead eth.L2BlockRef, batch *BatchWithL1InclusionBlock, l2Fetcher SafeBlockFetcher,
) BatchValidity

CheckBatch checks if the given batch can be applied on top of the given l2SafeHead, given the contextual L1 blocks the batch was included in. The first entry of the l1Blocks should match the origin of the l2SafeHead. One or more consecutive l1Blocks should be provided. In case of only a single L1 block, the decision whether a batch is valid may have to stay undecided.

type BatchWithL1InclusionBlock

type BatchWithL1InclusionBlock struct {
	Batch
	L1InclusionBlock eth.L1BlockRef
}

type BlobDataSource

type BlobDataSource struct {
	// contains filtered or unexported fields
}

BlobDataSource fetches blobs or calldata as appropriate and transforms them into usable rollup data.

func (*BlobDataSource) Next

func (ds *BlobDataSource) Next(ctx context.Context) (eth.Data, error)

Next returns the next piece of batcher data, or an io.EOF error if no data remains. It returns ResetError if it cannot find the referenced block or a referenced blob, or TemporaryError for any other failure to fetch a block or blob.

type BlockInsertionErrType

type BlockInsertionErrType uint
const (
	// BlockInsertOK indicates that the payload was successfully executed and appended to the canonical chain.
	BlockInsertOK BlockInsertionErrType = iota
	// BlockInsertTemporaryErr indicates that the insertion failed but may succeed at a later time without changes to the payload.
	BlockInsertTemporaryErr
	// BlockInsertPrestateErr indicates that the pre-state to insert the payload could not be prepared, e.g. due to missing chain data.
	BlockInsertPrestateErr
	// BlockInsertPayloadErr indicates that the payload was invalid and cannot become canonical.
	BlockInsertPayloadErr
)

type BrotliCompressor

type BrotliCompressor struct {
	BaseChannelCompressor
}

func (*BrotliCompressor) Reset

func (bc *BrotliCompressor) Reset()

type ByteReader

type ByteReader interface {
	io.Reader
	io.ByteReader
}

type CalldataSource

type CalldataSource struct {
	// contains filtered or unexported fields
}

CalldataSource is a fault tolerant approach to fetching data. The constructor will never fail & it will instead re-attempt the fetcher at a later point.

func (*CalldataSource) Next

func (ds *CalldataSource) Next(ctx context.Context) (eth.Data, error)

Next returns the next piece of data if it has it. If the constructor failed, this will attempt to reinitialize itself. If it cannot find the block it returns a ResetError otherwise it returns a temporary error if fetching the block returns an error.

type Channel

type Channel struct {
	// contains filtered or unexported fields
}

A Channel is a set of batches that are split into at least one, but possibly multiple frames. Frames are allowed to be ingested out of order. Each frame is ingested one by one. Once a frame with `closed` is added to the channel, the channel may mark itself as ready for reading once all intervening frames have been added

func NewChannel

func NewChannel(id ChannelID, openBlock eth.L1BlockRef) *Channel

func (*Channel) AddFrame

func (ch *Channel) AddFrame(frame Frame, l1InclusionBlock eth.L1BlockRef) error

AddFrame adds a frame to the channel. If the frame is not valid for the channel it returns an error. Otherwise the frame is buffered.

func (*Channel) HighestBlock

func (ch *Channel) HighestBlock() eth.L1BlockRef

HighestBlock returns the last L1 block which affect this channel

func (*Channel) IsReady

func (ch *Channel) IsReady() bool

IsReady returns true iff the channel is ready to be read.

func (*Channel) OpenBlockNumber

func (ch *Channel) OpenBlockNumber() uint64

OpenBlockNumber returns the block number of L1 block that contained the first frame for this channel.

func (*Channel) Reader

func (ch *Channel) Reader() io.Reader

Reader returns an io.Reader over the channel data. This panics if it is called while `IsReady` is not true. This function is able to be called multiple times.

func (*Channel) Size

func (ch *Channel) Size() uint64

Size returns the current size of the channel including frame overhead. Reading from the channel does not reduce the size as reading is done on uncompressed data while this size is over compressed data.

type ChannelBank

type ChannelBank struct {
	// contains filtered or unexported fields
}

ChannelBank buffers channel frames, and emits full channel data

func NewChannelBank

func NewChannelBank(log log.Logger, cfg *rollup.Config, prev NextFrameProvider, fetcher L1Fetcher, m Metrics) *ChannelBank

NewChannelBank creates a ChannelBank, which should be Reset(origin) before use.

func (*ChannelBank) IngestFrame

func (cb *ChannelBank) IngestFrame(f Frame)

IngestFrame adds new L1 data to the channel bank. Read() should be called repeatedly first, until everything has been read, before adding new data.

func (*ChannelBank) NextData

func (cb *ChannelBank) NextData(ctx context.Context) ([]byte, error)

NextData pulls the next piece of data from the channel bank. Note that it attempts to pull data out of the channel bank prior to loading data in (unlike most other stages). This is to ensure maintain consistency around channel bank pruning which depends upon the order of operations.

func (*ChannelBank) Origin

func (cb *ChannelBank) Origin() eth.L1BlockRef

func (*ChannelBank) Read

func (cb *ChannelBank) Read() (data []byte, err error)

Read the raw data of the first channel, if it's timed-out or closed. Read returns io.EOF if there is nothing new to read.

func (*ChannelBank) Reset

func (cb *ChannelBank) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error

type ChannelCompressor

type ChannelCompressor interface {
	Write([]byte) (int, error)
	Flush() error
	Close() error
	Reset()
	Len() int
	Read([]byte) (int, error)
	GetCompressed() *bytes.Buffer
}

func NewChannelCompressor

func NewChannelCompressor(algo CompressionAlgo) (ChannelCompressor, error)

type ChannelID

type ChannelID [ChannelIDLength]byte

ChannelID is an opaque identifier for a channel. It is 128 bits to be globally unique.

func (ChannelID) MarshalText

func (id ChannelID) MarshalText() ([]byte, error)

func (ChannelID) String

func (id ChannelID) String() string

func (ChannelID) TerminalString

func (id ChannelID) TerminalString() string

TerminalString implements log.TerminalStringer, formatting a string for console output during logging.

func (*ChannelID) UnmarshalText

func (id *ChannelID) UnmarshalText(text []byte) error

type ChannelInReader

type ChannelInReader struct {
	// contains filtered or unexported fields
}

ChannelInReader reads a batch from the channel This does decompression and limits the max RLP size This is a pure function from the channel, but each channel (or channel fragment) must be tagged with an L1 inclusion block to be passed to the batch queue.

func NewChannelInReader

func NewChannelInReader(cfg *rollup.Config, log log.Logger, prev *ChannelBank, metrics Metrics) *ChannelInReader

NewChannelInReader creates a ChannelInReader, which should be Reset(origin) before use.

func (*ChannelInReader) NextBatch

func (cr *ChannelInReader) NextBatch(ctx context.Context) (Batch, error)

NextBatch pulls out the next batch from the channel if it has it. It returns io.EOF when it cannot make any more progress. It will return a temporary error if it needs to be called again to advance some internal state.

func (*ChannelInReader) NextChannel

func (cr *ChannelInReader) NextChannel()

NextChannel forces the next read to continue with the next channel, resetting any decoding/decompression state to a fresh start.

func (*ChannelInReader) Origin

func (cr *ChannelInReader) Origin() eth.L1BlockRef

func (*ChannelInReader) Reset

func (*ChannelInReader) WriteChannel

func (cr *ChannelInReader) WriteChannel(data []byte) error

TODO: Take full channel for better logging

type ChannelOut

type ChannelOut interface {
	ID() ChannelID
	Reset() error
	AddBlock(*rollup.Config, *types.Block) error
	AddSingularBatch(*SingularBatch, uint64) error
	InputBytes() int
	ReadyBytes() int
	Flush() error
	FullErr() error
	Close() error
	OutputFrame(*bytes.Buffer, uint64) (uint16, error)
}

type CompressionAlgo

type CompressionAlgo string
const (
	// compression algo types
	Zlib     CompressionAlgo = "zlib"
	Brotli   CompressionAlgo = "brotli" // default level
	Brotli9  CompressionAlgo = "brotli-9"
	Brotli10 CompressionAlgo = "brotli-10"
	Brotli11 CompressionAlgo = "brotli-11"
)

func (*CompressionAlgo) Clone

func (algo *CompressionAlgo) Clone() any

func (*CompressionAlgo) IsBrotli

func (algo *CompressionAlgo) IsBrotli() bool

func (*CompressionAlgo) Set

func (algo *CompressionAlgo) Set(value string) error

func (CompressionAlgo) String

func (algo CompressionAlgo) String() string

type Compressor

type Compressor interface {
	// Writer is used to write uncompressed data which will be compressed. Should return
	// ErrCompressorFull if the compressor is full and no more data should be written.
	io.Writer
	// Closer Close function should be called before reading any data.
	io.Closer
	// Reader is used to Read compressed data; should only be called after Close.
	io.Reader
	// Reset will reset all written data
	Reset()
	// Len returns an estimate of the current length of the compressed data; calling Flush will
	// increase the accuracy at the expense of a poorer compression ratio.
	Len() int
	// Flush flushes any uncompressed data to the compression buffer. This will result in a
	// non-optimal compression ratio.
	Flush() error
	// FullErr returns ErrCompressorFull if the compressor is known to be full. Note that
	// calls to Write will fail if an error is returned from this method, but calls to Write
	// can still return ErrCompressorFull even if this does not.
	FullErr() error
}

type CompressorWriter

type CompressorWriter interface {
	Write([]byte) (int, error)
	Flush() error
	Close() error
	Reset(io.Writer)
}

type DataAvailabilitySource

type DataAvailabilitySource interface {
	OpenData(ctx context.Context, ref eth.L1BlockRef, batcherAddr common.Address) (DataIter, error)
}

type DataIter

type DataIter interface {
	Next(ctx context.Context) (eth.Data, error)
}

func NewBlobDataSource

func NewBlobDataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, blobsFetcher L1BlobsFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter

NewBlobDataSource creates a new blob data source.

func NewCalldataSource

func NewCalldataSource(ctx context.Context, log log.Logger, dsCfg DataSourceConfig, fetcher L1TransactionFetcher, ref eth.L1BlockRef, batcherAddr common.Address) DataIter

NewCalldataSource creates a new calldata source. It suppresses errors in fetching the L1 block if they occur. If there is an error, it will attempt to fetch the result on the next call to `Next`.

type DataSourceConfig

type DataSourceConfig struct {
	// contains filtered or unexported fields
}

DataSourceConfig regroups the mandatory rollup.Config fields needed for DataFromEVMTransactions.

type DataSourceFactory

type DataSourceFactory struct {
	// contains filtered or unexported fields
}

DataSourceFactory reads raw transactions from a given block & then filters for batch submitter transactions. This is not a stage in the pipeline, but a wrapper for another stage in the pipeline

func NewDataSourceFactory

func NewDataSourceFactory(log log.Logger, cfg *rollup.Config, fetcher L1Fetcher, blobsFetcher L1BlobsFetcher, plasmaFetcher PlasmaInputFetcher) *DataSourceFactory

func (*DataSourceFactory) OpenData

func (ds *DataSourceFactory) OpenData(ctx context.Context, ref eth.L1BlockRef, batcherAddr common.Address) (DataIter, error)

OpenData returns the appropriate data source for the L1 block `ref`.

type DerivationPipeline

type DerivationPipeline struct {
	// contains filtered or unexported fields
}

DerivationPipeline is updated with new L1 data, and the Step() function can be iterated on to keep the L2 Engine in sync.

func NewDerivationPipeline

func NewDerivationPipeline(log log.Logger, rollupCfg *rollup.Config, l1Fetcher L1Fetcher, l1Blobs L1BlobsFetcher,
	plasma PlasmaInputFetcher, l2Source L2Source, engine LocalEngineControl, metrics Metrics,
	syncCfg *sync.Config, safeHeadListener SafeHeadListener, finalizer FinalizerHooks, attributesHandler AttributesHandler) *DerivationPipeline

func (*DerivationPipeline) EngineReady

func (dp *DerivationPipeline) EngineReady() bool

EngineReady returns true if the engine is ready to be used. When it's being reset its state is inconsistent, and should not be used externally.

func (*DerivationPipeline) Origin

func (dp *DerivationPipeline) Origin() eth.L1BlockRef

Origin is the L1 block of the inner-most stage of the derivation pipeline, i.e. the L1 chain up to and including this point included and/or produced all the safe L2 blocks.

func (*DerivationPipeline) Reset

func (dp *DerivationPipeline) Reset()

func (*DerivationPipeline) Step

func (dp *DerivationPipeline) Step(ctx context.Context) error

Step tries to progress the buffer. An EOF is returned if the pipeline is blocked by waiting for new L1 data. If ctx errors no error is returned, but the step may exit early in a state that can still be continued. Any other error is critical and the derivation pipeline should be reset. An error is expected when the underlying source closes. When Step returns nil, it should be called again, to continue the derivation process.

type Engine

type Engine interface {
	ExecEngine
	L2Source
}

type EngineControl

type EngineControl interface {
	EngineState

	// StartPayload requests the engine to start building a block with the given attributes.
	// If updateSafe, the resulting block will be marked as a safe block.
	StartPayload(ctx context.Context, parent eth.L2BlockRef, attrs *AttributesWithParent, updateSafe bool) (errType BlockInsertionErrType, err error)
	// ConfirmPayload requests the engine to complete the current block. If no block is being built, or if it fails, an error is returned.
	ConfirmPayload(ctx context.Context, agossip async.AsyncGossiper, sequencerConductor conductor.SequencerConductor) (out *eth.ExecutionPayloadEnvelope, errTyp BlockInsertionErrType, err error)
	// CancelPayload requests the engine to stop building the current block without making it canonical.
	// This is optional, as the engine expires building jobs that are left uncompleted, but can still save resources.
	CancelPayload(ctx context.Context, force bool) error
	// BuildingPayload indicates if a payload is being built, and onto which block it is being built, and whether or not it is a safe payload.
	BuildingPayload() (onto eth.L2BlockRef, id eth.PayloadID, safe bool)
}

EngineControl enables other components to build blocks with the Engine, while keeping the forkchoice state and payload-id management internal to avoid state inconsistencies between different users of the EngineControl.

type EngineController

type EngineController struct {
	// contains filtered or unexported fields
}

func NewEngineController

func NewEngineController(engine ExecEngine, log log.Logger, metrics Metrics, rollupCfg *rollup.Config, syncMode sync.Mode) *EngineController

func (*EngineController) BackupUnsafeL2Head

func (e *EngineController) BackupUnsafeL2Head() eth.L2BlockRef

func (*EngineController) BuildingPayload

func (e *EngineController) BuildingPayload() (eth.L2BlockRef, eth.PayloadID, bool)

func (*EngineController) CancelPayload

func (e *EngineController) CancelPayload(ctx context.Context, force bool) error

func (*EngineController) ConfirmPayload

func (e *EngineController) ConfirmPayload(ctx context.Context, agossip async.AsyncGossiper, sequencerConductor conductor.SequencerConductor) (out *eth.ExecutionPayloadEnvelope, errTyp BlockInsertionErrType, err error)

func (*EngineController) Finalized

func (e *EngineController) Finalized() eth.L2BlockRef

func (*EngineController) InsertUnsafePayload

func (e *EngineController) InsertUnsafePayload(ctx context.Context, envelope *eth.ExecutionPayloadEnvelope, ref eth.L2BlockRef) error

func (*EngineController) IsEngineSyncing

func (e *EngineController) IsEngineSyncing() bool

func (*EngineController) PendingSafeL2Head

func (e *EngineController) PendingSafeL2Head() eth.L2BlockRef

func (*EngineController) ResetBuildingState

func (e *EngineController) ResetBuildingState()

ResetBuildingState implements LocalEngineControl.

func (*EngineController) SafeL2Head

func (e *EngineController) SafeL2Head() eth.L2BlockRef

func (*EngineController) SetBackupUnsafeL2Head

func (e *EngineController) SetBackupUnsafeL2Head(r eth.L2BlockRef, triggerReorg bool)

SetBackupUnsafeL2Head implements LocalEngineControl.

func (*EngineController) SetFinalizedHead

func (e *EngineController) SetFinalizedHead(r eth.L2BlockRef)

SetFinalizedHead implements LocalEngineControl.

func (*EngineController) SetPendingSafeL2Head

func (e *EngineController) SetPendingSafeL2Head(r eth.L2BlockRef)

SetPendingSafeL2Head implements LocalEngineControl.

func (*EngineController) SetSafeHead

func (e *EngineController) SetSafeHead(r eth.L2BlockRef)

SetSafeHead implements LocalEngineControl.

func (*EngineController) SetUnsafeHead

func (e *EngineController) SetUnsafeHead(r eth.L2BlockRef)

SetUnsafeHead implements LocalEngineControl.

func (*EngineController) StartPayload

func (e *EngineController) StartPayload(ctx context.Context, parent eth.L2BlockRef, attrs *AttributesWithParent, updateSafe bool) (errType BlockInsertionErrType, err error)

func (*EngineController) TryBackupUnsafeReorg

func (e *EngineController) TryBackupUnsafeReorg(ctx context.Context) (bool, error)

TryBackupUnsafeReorg attempts to reorg(restore) unsafe head to backupUnsafeHead. If succeeds, update current forkchoice state to the rollup node.

func (*EngineController) TryUpdateEngine

func (e *EngineController) TryUpdateEngine(ctx context.Context) error

TryUpdateEngine attempts to update the engine with the current forkchoice state of the rollup node, this is a no-op if the nodes already agree on the forkchoice state.

func (*EngineController) UnsafeL2Head

func (e *EngineController) UnsafeL2Head() eth.L2BlockRef

type EngineQueue

type EngineQueue struct {
	// contains filtered or unexported fields
}

EngineQueue queues up payload attributes to consolidate or process with the provided Engine

func NewEngineQueue

func NewEngineQueue(log log.Logger, cfg *rollup.Config, l2Source L2Source, engine LocalEngineControl, metrics Metrics,
	prev NextAttributesProvider, l1Fetcher L1Fetcher, syncCfg *sync.Config, safeHeadNotifs SafeHeadListener,
	finalizer FinalizerHooks, attributesHandler AttributesHandler) *EngineQueue

NewEngineQueue creates a new EngineQueue, which should be Reset(origin) before use.

func (*EngineQueue) BackupUnsafeL2Head

func (eq *EngineQueue) BackupUnsafeL2Head() eth.L2BlockRef

func (*EngineQueue) Origin

func (eq *EngineQueue) Origin() eth.L1BlockRef

Origin identifies the L1 chain (incl.) that included and/or produced all the safe L2 blocks.

func (*EngineQueue) Reset

Reset walks the L2 chain backwards until it finds an L2 block whose L1 origin is canonical. The unsafe head is set to the head of the L2 chain, unless the existing safe head is not canonical.

func (*EngineQueue) Step

func (eq *EngineQueue) Step(ctx context.Context) error

func (*EngineQueue) SystemConfig

func (eq *EngineQueue) SystemConfig() eth.SystemConfig

type EngineQueueStage

type EngineQueueStage interface {
	Origin() eth.L1BlockRef
	SystemConfig() eth.SystemConfig
	Step(context.Context) error
}

type EngineState

type EngineState interface {
	Finalized() eth.L2BlockRef
	UnsafeL2Head() eth.L2BlockRef
	SafeL2Head() eth.L2BlockRef
}

EngineState provides a read-only interface of the forkchoice state properties of the L2 Engine.

type Error

type Error struct {
	// contains filtered or unexported fields
}

Error is a wrapper for error, description and a severity level.

func (Error) Error

func (e Error) Error() string

Error satisfies the error interface.

func (Error) Is

func (e Error) Is(target error) bool

Is satisfies the error Unwrap interface.

func (Error) Unwrap

func (e Error) Unwrap() error

Unwrap satisfies the Is/As interface.

type ExecEngine

type ExecEngine interface {
	GetPayload(ctx context.Context, payloadInfo eth.PayloadInfo) (*eth.ExecutionPayloadEnvelope, error)
	ForkchoiceUpdate(ctx context.Context, state *eth.ForkchoiceState, attr *eth.PayloadAttributes) (*eth.ForkchoiceUpdatedResult, error)
	NewPayload(ctx context.Context, payload *eth.ExecutionPayload, parentBeaconBlockRoot *common.Hash) (*eth.PayloadStatusV1, error)
	L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (eth.L2BlockRef, error)
}

type FetchingAttributesBuilder

type FetchingAttributesBuilder struct {
	// contains filtered or unexported fields
}

FetchingAttributesBuilder fetches inputs for the building of L2 payload attributes on the fly.

func (*FetchingAttributesBuilder) PreparePayloadAttributes

func (ba *FetchingAttributesBuilder) PreparePayloadAttributes(ctx context.Context, l2Parent eth.L2BlockRef, epoch eth.BlockID) (attrs *eth.PayloadAttributes, err error)

PreparePayloadAttributes prepares a PayloadAttributes template that is ready to build a L2 block with deposits only, on top of the given l2Parent, with the given epoch as L1 origin. The template defaults to NoTxPool=true, and no sequencer transactions: the caller has to modify the template to add transactions, by setting NoTxPool=false as sequencer, or by appending batch transactions as verifier. The severity of the error is returned; a crit=false error means there was a temporary issue, like a failed RPC or time-out. A crit=true error means the input arguments are inconsistent or invalid.

type FinalizerHooks

type FinalizerHooks interface {
	// OnDerivationL1End remembers the given L1 block,
	// and finalizes any prior data with the latest finality signal based on block height.
	OnDerivationL1End(ctx context.Context, derivedFrom eth.L1BlockRef) error
	// PostProcessSafeL2 remembers the L2 block is derived from the given L1 block, for later finalization.
	PostProcessSafeL2(l2Safe eth.L2BlockRef, derivedFrom eth.L1BlockRef)
	// Reset clear recent state, to adapt to reorgs.
	Reset()
}

type Frame

type Frame struct {
	ID          ChannelID `json:"id"`
	FrameNumber uint16    `json:"frame_number"`
	Data        []byte    `json:"data"`
	IsLast      bool      `json:"is_last"`
}

func ParseFrames

func ParseFrames(data []byte) ([]Frame, error)

ParseFrames parse the on chain serialization of frame(s) in an L1 transaction. Currently only version 0 of the serialization format is supported. All frames must be parsed without error and there must not be any left over data and there must be at least one frame.

func (*Frame) MarshalBinary

func (f *Frame) MarshalBinary(w io.Writer) error

MarshalBinary writes the frame to `w`. It returns any errors encountered while writing, but generally expects the writer very rarely fail.

func (*Frame) UnmarshalBinary

func (f *Frame) UnmarshalBinary(r ByteReader) error

UnmarshalBinary consumes a full frame from the reader. If `r` fails a read, it returns the error from the reader The reader will be left in a partially read state.

If r doesn't return any bytes, returns io.EOF. If r unexpectedly stops returning data half-way, returns io.ErrUnexpectedEOF.

type FrameQueue

type FrameQueue struct {
	// contains filtered or unexported fields
}

func NewFrameQueue

func NewFrameQueue(log log.Logger, prev NextDataProvider) *FrameQueue

func (*FrameQueue) NextFrame

func (fq *FrameQueue) NextFrame(ctx context.Context) (Frame, error)

func (*FrameQueue) Origin

func (fq *FrameQueue) Origin() eth.L1BlockRef

func (*FrameQueue) Reset

type InnerBatchData

type InnerBatchData interface {
	GetBatchType() int
	// contains filtered or unexported methods
}

InnerBatchData is the underlying data of a BatchData. This is implemented by SingularBatch and RawSpanBatch.

type L1BlobsFetcher

type L1BlobsFetcher interface {
	// GetBlobs fetches blobs that were confirmed in the given L1 block with the given indexed hashes.
	GetBlobs(ctx context.Context, ref eth.L1BlockRef, hashes []eth.IndexedBlobHash) ([]*eth.Blob, error)
}

type L1BlockInfo

type L1BlockInfo struct {
	Number    uint64
	Time      uint64
	BaseFee   *big.Int
	BlockHash common.Hash
	// Not strictly a piece of L1 information. Represents the number of L2 blocks since the start of the epoch,
	// i.e. when the actual L1 info was first introduced.
	SequenceNumber uint64
	// BatcherHash version 0 is just the address with 0 padding to the left.
	BatcherAddr common.Address

	L1FeeOverhead eth.Bytes32 // ignored after Ecotone upgrade
	L1FeeScalar   eth.Bytes32 // ignored after Ecotone upgrade

	BlobBaseFee       *big.Int // added by Ecotone upgrade
	BaseFeeScalar     uint32   // added by Ecotone upgrade
	BlobBaseFeeScalar uint32   // added by Ecotone upgrade
}

L1BlockInfo presents the information stored in a L1Block.setL1BlockValues call

func L1BlockInfoFromBytes

func L1BlockInfoFromBytes(rollupCfg *rollup.Config, l2BlockTime uint64, data []byte) (*L1BlockInfo, error)

L1BlockInfoFromBytes is the inverse of L1InfoDeposit, to see where the L2 chain is derived from

type L1BlockRefByHashFetcher

type L1BlockRefByHashFetcher interface {
	L1BlockRefByHash(context.Context, common.Hash) (eth.L1BlockRef, error)
}

type L1BlockRefByNumberFetcher

type L1BlockRefByNumberFetcher interface {
	L1BlockRefByNumber(context.Context, uint64) (eth.L1BlockRef, error)
	FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error)
}

type L1InfoDepositSource

type L1InfoDepositSource struct {
	L1BlockHash common.Hash
	SeqNumber   uint64
}

func (*L1InfoDepositSource) SourceHash

func (dep *L1InfoDepositSource) SourceHash() common.Hash

type L1ReceiptsFetcher

type L1ReceiptsFetcher interface {
	InfoByHash(ctx context.Context, hash common.Hash) (eth.BlockInfo, error)
	FetchReceipts(ctx context.Context, blockHash common.Hash) (eth.BlockInfo, types.Receipts, error)
}

L1ReceiptsFetcher fetches L1 header info and receipts for the payload attributes derivation (the info tx and deposits)

type L1Retrieval

type L1Retrieval struct {
	// contains filtered or unexported fields
}

func NewL1Retrieval

func NewL1Retrieval(log log.Logger, dataSrc DataAvailabilitySource, prev NextBlockProvider) *L1Retrieval

func (*L1Retrieval) NextData

func (l1r *L1Retrieval) NextData(ctx context.Context) ([]byte, error)

NextData does an action in the L1 Retrieval stage If there is data, it pushes it to the next stage. If there is no more data open ourselves if we are closed or close ourselves if we are open

func (*L1Retrieval) Origin

func (l1r *L1Retrieval) Origin() eth.L1BlockRef

func (*L1Retrieval) Reset

func (l1r *L1Retrieval) Reset(ctx context.Context, base eth.L1BlockRef, sysCfg eth.SystemConfig) error

Reset re-initializes the L1 Retrieval stage to block of it's `next` progress. Note that we open up the `l1r.datas` here because it is required to maintain the internal invariants that later propagate up the derivation pipeline.

type L1TransactionFetcher

type L1TransactionFetcher interface {
	InfoAndTxsByHash(ctx context.Context, hash common.Hash) (eth.BlockInfo, types.Transactions, error)
}

type L1Traversal

type L1Traversal struct {
	// contains filtered or unexported fields
}

func NewL1Traversal

func NewL1Traversal(log log.Logger, cfg *rollup.Config, l1Blocks L1BlockRefByNumberFetcher) *L1Traversal

func (*L1Traversal) AdvanceL1Block

func (l1t *L1Traversal) AdvanceL1Block(ctx context.Context) error

AdvanceL1Block advances the internal state of L1 Traversal

func (*L1Traversal) NextL1Block

func (l1t *L1Traversal) NextL1Block(_ context.Context) (eth.L1BlockRef, error)

NextL1Block returns the next block. It does not advance, but it can only be called once before returning io.EOF

func (*L1Traversal) Origin

func (l1t *L1Traversal) Origin() eth.L1BlockRef

func (*L1Traversal) Reset

func (l1t *L1Traversal) Reset(ctx context.Context, base eth.L1BlockRef, cfg eth.SystemConfig) error

Reset sets the internal L1 block to the supplied base.

func (*L1Traversal) SystemConfig

func (l1c *L1Traversal) SystemConfig() eth.SystemConfig

type L2BlockRefSource

type L2BlockRefSource interface {
	Hash() common.Hash
	ParentHash() common.Hash
	NumberU64() uint64
	Time() uint64
	Transactions() types.Transactions
}

L2BlockRefSource is a source for the generation of a L2BlockRef. E.g. a *types.Block is a L2BlockRefSource.

L2BlockToBlockRef extracts L2BlockRef from a L2BlockRefSource. The first transaction of a source must be a Deposit transaction.

type L2Source

type L2Source interface {
	PayloadByHash(context.Context, common.Hash) (*eth.ExecutionPayloadEnvelope, error)
	PayloadByNumber(context.Context, uint64) (*eth.ExecutionPayloadEnvelope, error)
	L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (eth.L2BlockRef, error)
	L2BlockRefByHash(ctx context.Context, l2Hash common.Hash) (eth.L2BlockRef, error)
	L2BlockRefByNumber(ctx context.Context, num uint64) (eth.L2BlockRef, error)
	SystemConfigL2Fetcher
}

type Level

type Level uint

Level is the severity level of the error.

const (
	// LevelTemporary is a temporary error for example due to an RPC or
	// connection issue, and can be safely ignored and retried by the caller
	LevelTemporary Level = iota
	// LevelReset is a pipeline reset error. It must be treated like a reorg.
	LevelReset
	// LevelCritical is a critical error.
	LevelCritical
)

There are three levels currently, out of which only 2 are being used to classify error by severity. LevelTemporary

func (Level) String

func (lvl Level) String() string

type LocalEngineControl

type LocalEngineControl interface {
	EngineControl
	ResetBuildingState()
	IsEngineSyncing() bool
	TryUpdateEngine(ctx context.Context) error
	TryBackupUnsafeReorg(ctx context.Context) (bool, error)

	PendingSafeL2Head() eth.L2BlockRef
	BackupUnsafeL2Head() eth.L2BlockRef

	SetUnsafeHead(eth.L2BlockRef)
	SetSafeHead(eth.L2BlockRef)
	SetFinalizedHead(eth.L2BlockRef)
	SetPendingSafeL2Head(eth.L2BlockRef)
	SetBackupUnsafeL2Head(block eth.L2BlockRef, triggerReorg bool)
}

type Metrics

type Metrics interface {
	RecordL1Ref(name string, ref eth.L1BlockRef)
	RecordL2Ref(name string, ref eth.L2BlockRef)
	RecordChannelInputBytes(inputCompressedBytes int)
	RecordHeadChannelOpened()
	RecordChannelTimedOut()
	RecordFrame()
	RecordDerivedBatches(batchType string)
}

type NextAttributesProvider

type NextAttributesProvider interface {
	Origin() eth.L1BlockRef
	NextAttributes(context.Context, eth.L2BlockRef) (*AttributesWithParent, error)
}

type NextBatchProvider

type NextBatchProvider interface {
	Origin() eth.L1BlockRef
	NextBatch(ctx context.Context) (Batch, error)
}

type NextBlockProvider

type NextBlockProvider interface {
	NextL1Block(context.Context) (eth.L1BlockRef, error)
	Origin() eth.L1BlockRef
	SystemConfig() eth.SystemConfig
}

type NextDataProvider

type NextDataProvider interface {
	NextData(context.Context) ([]byte, error)
	Origin() eth.L1BlockRef
}

type NextFrameProvider

type NextFrameProvider interface {
	NextFrame(ctx context.Context) (Frame, error)
	Origin() eth.L1BlockRef
}

type PlasmaDataSource

type PlasmaDataSource struct {
	// contains filtered or unexported fields
}

PlasmaDataSource is a data source that fetches inputs from a plasma DA provider given their onchain commitments. Same as CalldataSource it will keep attempting to fetch.

func NewPlasmaDataSource

func NewPlasmaDataSource(log log.Logger, src DataIter, l1 L1Fetcher, fetcher PlasmaInputFetcher, id eth.BlockID) *PlasmaDataSource

func (*PlasmaDataSource) Next

func (s *PlasmaDataSource) Next(ctx context.Context) (eth.Data, error)

type PlasmaInputFetcher

type PlasmaInputFetcher interface {
	// GetInput fetches the input for the given commitment at the given block number from the DA storage service.
	GetInput(ctx context.Context, l1 plasma.L1Fetcher, c plasma.CommitmentData, blockId eth.BlockID) (eth.Data, error)
	// AdvanceL1Origin advances the L1 origin to the given block number, syncing the DA challenge events.
	AdvanceL1Origin(ctx context.Context, l1 plasma.L1Fetcher, blockId eth.BlockID) error
	// Reset the challenge origin in case of L1 reorg
	Reset(ctx context.Context, base eth.L1BlockRef, baseCfg eth.SystemConfig) error
}

type RawSpanBatch

type RawSpanBatch struct {
	// contains filtered or unexported fields
}

RawSpanBatch is another representation of SpanBatch, that encodes data according to SpanBatch specs.

func (*RawSpanBatch) GetBatchType

func (b *RawSpanBatch) GetBatchType() int

GetBatchType returns its batch type (batch_version)

func (*RawSpanBatch) ToSpanBatch

func (b *RawSpanBatch) ToSpanBatch(blockTime, genesisTimestamp uint64, chainID *big.Int) (*SpanBatch, error)

ToSpanBatch converts RawSpanBatch to SpanBatch, which implements a wrapper of derive method of RawSpanBatch

type ResettableStage

type ResettableStage interface {
	// Reset resets a pull stage. `base` refers to the L1 Block Reference to reset to, with corresponding configuration.
	Reset(ctx context.Context, base eth.L1BlockRef, baseCfg eth.SystemConfig) error
}

type SafeBlockFetcher

type SafeBlockFetcher interface {
	L2BlockRefByNumber(context.Context, uint64) (eth.L2BlockRef, error)
	PayloadByNumber(context.Context, uint64) (*eth.ExecutionPayloadEnvelope, error)
}

type SafeHeadListener

type SafeHeadListener interface {

	// Enabled reports if this safe head listener is actively using the posted data. This allows the engine queue to
	// optionally skip making calls that may be expensive to prepare.
	// Callbacks may still be made if Enabled returns false but are not guaranteed.
	Enabled() bool

	// SafeHeadUpdated indicates that the safe head has been updated in response to processing batch data
	// The l1Block specified is the first L1 block containing all required batch data to derive newSafeHead
	SafeHeadUpdated(newSafeHead eth.L2BlockRef, l1Block eth.BlockID) error

	// SafeHeadReset indicates that the derivation pipeline reset back to the specified safe head
	// The L1 block that made the new safe head safe is unknown.
	SafeHeadReset(resetSafeHead eth.L2BlockRef) error
}

SafeHeadListener is called when the safe head is updated. The safe head may advance by more than one block in a single update The l1Block specified is the first L1 block that includes sufficient information to derive the new safe head

type SingularBatch

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
}

SingularBatch is an implementation of Batch interface, containing the input to build one L2 block.

func GetSingularBatch

func GetSingularBatch(batchData *BatchData) (*SingularBatch, error)

GetSingularBatch retrieves SingularBatch from batchData

func RandomSingularBatch

func RandomSingularBatch(rng *rand.Rand, txCount int, chainID *big.Int) *SingularBatch

func (*SingularBatch) AsSingularBatch

func (b *SingularBatch) AsSingularBatch() (*SingularBatch, bool)

func (*SingularBatch) AsSpanBatch

func (b *SingularBatch) AsSpanBatch() (*SpanBatch, bool)

func (*SingularBatch) Epoch

func (b *SingularBatch) Epoch() eth.BlockID

Epoch returns a BlockID of its L1 origin.

func (*SingularBatch) GetBatchType

func (b *SingularBatch) GetBatchType() int

GetBatchType returns its batch type (batch_version)

func (*SingularBatch) GetEpochNum

func (b *SingularBatch) GetEpochNum() rollup.Epoch

GetEpochNum returns its epoch number (L1 origin block number)

func (*SingularBatch) GetTimestamp

func (b *SingularBatch) GetTimestamp() uint64

GetTimestamp returns its block timestamp

func (*SingularBatch) LogContext

func (b *SingularBatch) LogContext(log log.Logger) log.Logger

LogContext creates a new log context that contains information of the batch

type SingularChannelOut

type SingularChannelOut struct {
	// contains filtered or unexported fields
}

func NewSingularChannelOut

func NewSingularChannelOut(compress Compressor) (*SingularChannelOut, error)

func (*SingularChannelOut) AddBlock

func (co *SingularChannelOut) AddBlock(rollupCfg *rollup.Config, block *types.Block) error

AddBlock adds a block to the channel. It returns the RLP encoded byte size and an error if there is a problem adding the block. The only sentinel error that it returns is ErrTooManyRLPBytes. If this error is returned, the channel should be closed and a new one should be made.

func (*SingularChannelOut) AddSingularBatch

func (co *SingularChannelOut) AddSingularBatch(batch *SingularBatch, _ uint64) error

AddSingularBatch adds a batch to the channel. It returns the RLP encoded byte size and an error if there is a problem adding the batch. The only sentinel error that it returns is ErrTooManyRLPBytes. If this error is returned, the channel should be closed and a new one should be made.

AddSingularBatch should be used together with BlockToBatch if you need to access the BatchData before adding a block to the channel. It isn't possible to access the batch data with AddBlock.

func (*SingularChannelOut) Close

func (co *SingularChannelOut) Close() error

func (*SingularChannelOut) Flush

func (co *SingularChannelOut) Flush() error

Flush flushes the internal compression stage to the ready buffer. It enables pulling a larger & more complete frame. It reduces the compression efficiency.

func (*SingularChannelOut) FullErr

func (co *SingularChannelOut) FullErr() error

func (*SingularChannelOut) ID

func (co *SingularChannelOut) ID() ChannelID

func (*SingularChannelOut) InputBytes

func (co *SingularChannelOut) InputBytes() int

InputBytes returns the total amount of RLP-encoded input bytes.

func (*SingularChannelOut) OutputFrame

func (co *SingularChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error)

OutputFrame writes a frame to w with a given max size and returns the frame number. Use `ReadyBytes`, `Flush`, and `Close` to modify the ready buffer. Returns an error if the `maxSize` < FrameV0OverHeadSize. Returns io.EOF when the channel is closed & there are no more frames. Returns nil if there is still more buffered data. Returns an error if it ran into an error during processing.

func (*SingularChannelOut) ReadyBytes

func (co *SingularChannelOut) ReadyBytes() int

ReadyBytes returns the number of bytes that the channel out can immediately output into a frame. Use `Flush` or `Close` to move data from the compression buffer into the ready buffer if more bytes are needed. Add blocks may add to the ready buffer, but it is not guaranteed due to the compression stage.

func (*SingularChannelOut) Reset

func (co *SingularChannelOut) Reset() error

type SpanBatch

type SpanBatch struct {
	ParentCheck      [20]byte // First 20 bytes of the first block's parent hash
	L1OriginCheck    [20]byte // First 20 bytes of the last block's L1 origin hash
	GenesisTimestamp uint64
	ChainID          *big.Int
	Batches          []*SpanBatchElement // List of block input in derived form
	// contains filtered or unexported fields
}

SpanBatch is an implementation of Batch interface, containing the input to build a span of L2 blocks in derived form (SpanBatchElement)

func DeriveSpanBatch

func DeriveSpanBatch(batchData *BatchData, blockTime, genesisTimestamp uint64, chainID *big.Int) (*SpanBatch, error)

DeriveSpanBatch derives SpanBatch from BatchData.

func NewSpanBatch

func NewSpanBatch(genesisTimestamp uint64, chainID *big.Int) *SpanBatch

NewSpanBatch converts given singularBatches into SpanBatchElements, and creates a new SpanBatch.

func (*SpanBatch) AppendSingularBatch

func (b *SpanBatch) AppendSingularBatch(singularBatch *SingularBatch, seqNum uint64) error

AppendSingularBatch appends a SingularBatch into the span batch updates l1OriginCheck or parentCheck if needed.

func (*SpanBatch) AsSingularBatch

func (b *SpanBatch) AsSingularBatch() (*SingularBatch, bool)

func (*SpanBatch) AsSpanBatch

func (b *SpanBatch) AsSpanBatch() (*SpanBatch, bool)

func (*SpanBatch) CheckOriginHash

func (b *SpanBatch) CheckOriginHash(hash common.Hash) bool

CheckOriginHash checks if the l1OriginCheck matches the first 20 bytes of given hash, probably L1 block hash from the current canonical L1 chain.

func (*SpanBatch) CheckParentHash

func (b *SpanBatch) CheckParentHash(hash common.Hash) bool

CheckParentHash checks if the parentCheck matches the first 20 bytes of given hash, probably the current L2 safe head.

func (*SpanBatch) GetBatchType

func (b *SpanBatch) GetBatchType() int

GetBatchType returns its batch type (batch_version)

func (*SpanBatch) GetBlockCount

func (b *SpanBatch) GetBlockCount() int

GetBlockCount returns the number of blocks in the span

func (*SpanBatch) GetBlockEpochNum

func (b *SpanBatch) GetBlockEpochNum(i int) uint64

GetBlockEpochNum returns the epoch number(L1 origin block number) of the block at the given index in the span.

func (*SpanBatch) GetBlockTimestamp

func (b *SpanBatch) GetBlockTimestamp(i int) uint64

GetBlockTimestamp returns the timestamp of the block at the given index in the span.

func (*SpanBatch) GetBlockTransactions

func (b *SpanBatch) GetBlockTransactions(i int) []hexutil.Bytes

GetBlockTransactions returns the encoded transactions of the block at the given index in the span.

func (*SpanBatch) GetSingularBatches

func (b *SpanBatch) GetSingularBatches(l1Origins []eth.L1BlockRef, l2SafeHead eth.L2BlockRef) ([]*SingularBatch, error)

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 (*SpanBatch) GetStartEpochNum

func (b *SpanBatch) GetStartEpochNum() rollup.Epoch

GetStartEpochNum returns epoch number(L1 origin block number) of the first block in the span

func (*SpanBatch) GetTimestamp

func (b *SpanBatch) GetTimestamp() uint64

GetTimestamp returns timestamp of the first block in the span

func (*SpanBatch) LogContext

func (b *SpanBatch) LogContext(log log.Logger) log.Logger

LogContext creates a new log context that contains information of the batch

func (*SpanBatch) MarshalJSON

func (b *SpanBatch) MarshalJSON() ([]byte, error)

func (*SpanBatch) ToRawSpanBatch

func (b *SpanBatch) ToRawSpanBatch() (*RawSpanBatch, error)

ToRawSpanBatch merges SingularBatch List and initialize single RawSpanBatch

type SpanBatchElement

type SpanBatchElement struct {
	EpochNum     rollup.Epoch // aka l1 num
	Timestamp    uint64
	Transactions []hexutil.Bytes
}

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 SpanChannelOut

type SpanChannelOut struct {
	// contains filtered or unexported fields
}

func NewSpanChannelOut

func NewSpanChannelOut(genesisTimestamp uint64, chainID *big.Int, targetOutputSize uint64, compressionAlgo CompressionAlgo) (*SpanChannelOut, error)

func (*SpanChannelOut) AddBlock

func (co *SpanChannelOut) AddBlock(rollupCfg *rollup.Config, block *types.Block) error

AddBlock adds a block to the channel. returns an error if there is a problem adding the block. The only sentinel error that it returns is ErrTooManyRLPBytes. If this error is returned, the channel should be closed and a new one should be made.

func (*SpanChannelOut) AddSingularBatch

func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch, seqNum uint64) error

AddSingularBatch adds a SingularBatch to the channel, compressing the data if necessary. if the new batch would make the channel exceed the target size, the last batch is reverted, and the compression happens on the previous RLP buffer instead if the input is too small to need compression, data is accumulated but not compressed

func (*SpanChannelOut) Close

func (co *SpanChannelOut) Close() error

func (*SpanChannelOut) Flush

func (co *SpanChannelOut) Flush() error

Flush implements the Channel Out Span Channel Out manages the flushing of the compressor internally, so this is a no-op

func (*SpanChannelOut) FullErr

func (co *SpanChannelOut) FullErr() error

func (*SpanChannelOut) ID

func (co *SpanChannelOut) ID() ChannelID

func (*SpanChannelOut) InputBytes

func (co *SpanChannelOut) InputBytes() int

InputBytes returns the total amount of RLP-encoded input bytes.

func (*SpanChannelOut) OutputFrame

func (co *SpanChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error)

OutputFrame writes a frame to w with a given max size and returns the frame number. Use `ReadyBytes`, `Flush`, and `Close` to modify the ready buffer. Returns an error if the `maxSize` < FrameV0OverHeadSize. Returns io.EOF when the channel is closed & there are no more frames. Returns nil if there is still more buffered data. Returns an error if it ran into an error during processing.

func (*SpanChannelOut) ReadyBytes

func (co *SpanChannelOut) ReadyBytes() int

ReadyBytes returns the total amount of compressed bytes that are ready to be output. Span Channel Out does not provide early output, so this will always be 0 until the channel is closed or full

func (*SpanChannelOut) Reset

func (co *SpanChannelOut) Reset() error

type SystemConfigL2Fetcher

type SystemConfigL2Fetcher interface {
	SystemConfigByL2Hash(ctx context.Context, hash common.Hash) (eth.SystemConfig, error)
}

type UpgradeDepositSource

type UpgradeDepositSource struct {
	Intent string
}

UpgradeDepositSource implements the translation of upgrade-tx identity information to a deposit source-hash, which makes the deposit uniquely identifiable. System-upgrade transactions have their own domain for source-hashes, to not conflict with user-deposits or deposited L1 information. The intent identifies the upgrade-tx uniquely, in a human-readable way.

func (*UpgradeDepositSource) SourceHash

func (dep *UpgradeDepositSource) SourceHash() common.Hash

type UserDepositSource

type UserDepositSource struct {
	L1BlockHash common.Hash
	LogIndex    uint64
}

func (*UserDepositSource) SourceHash

func (dep *UserDepositSource) SourceHash() common.Hash

type ZlibCompressor

type ZlibCompressor struct {
	BaseChannelCompressor
}

func (*ZlibCompressor) Reset

func (zc *ZlibCompressor) Reset()

Directories

Path Synopsis

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL