engine

package
v1.14.1 Latest Latest
Warning

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

Go to latest
Published: Oct 8, 2025 License: MIT, MIT Imports: 20 Imported by: 8

Documentation

Index

Constants

This section is empty.

Variables

View Source
var ErrEngineSyncing = errors.New("engine is syncing")
View Source
var ErrNoFCUNeeded = errors.New("no FCU call was needed")
View Source
var Kinds = []Kind{
	Geth,
	Reth,
	Erigon,
}
View Source
var ReplaceBlockSource = eth.L1BlockRef{
	Hash:       common.HexToHash("0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff"),
	Number:     ^uint64(0),
	ParentHash: common.Hash{},
	Time:       0,
}

ReplaceBlockSource is a magic value for the "Source" attribute, used when a L2 block is a replacement of an invalidated block. After the replacement has been processed, a reset is performed to derive the next L2 blocks.

Functions

func ForceEngineReset

func ForceEngineReset(ec ResetEngineControl, localUnsafe, crossUnsafe, localSafe, crossSafe, finalized eth.L2BlockRef)

func ValidEngineKind

func ValidEngineKind(value Kind) bool

Types

type AttributesForceResetter added in v1.13.6

type AttributesForceResetter interface {
	ForceReset(ctx context.Context, localUnsafe, crossUnsafe, localSafe, crossSafe, finalized eth.L2BlockRef)
}

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 BuildCancelEvent

type BuildCancelEvent struct {
	Info  eth.PayloadInfo
	Force bool
}

func (BuildCancelEvent) String

func (ev BuildCancelEvent) String() string

type BuildInvalidEvent

type BuildInvalidEvent struct {
	Attributes *derive.AttributesWithParent
	Err        error
}

BuildInvalidEvent is an internal engine event, to post-process upon invalid attributes. Not for temporary processing problems.

func (BuildInvalidEvent) String

func (ev BuildInvalidEvent) String() string

type BuildSealEvent

type BuildSealEvent struct {
	Info         eth.PayloadInfo
	BuildStarted time.Time
	// if payload should be promoted to safe (must also be pending safe, see DerivedFrom)
	Concluding bool
	// payload is promoted to pending-safe if non-zero
	DerivedFrom eth.L1BlockRef
}

func (BuildSealEvent) String

func (ev BuildSealEvent) String() string

type BuildSealedEvent

type BuildSealedEvent struct {
	// if payload should be promoted to (local) safe (must also be pending safe, see DerivedFrom)
	Concluding bool
	// payload is promoted to pending-safe if non-zero
	DerivedFrom  eth.L1BlockRef
	BuildStarted time.Time

	Info     eth.PayloadInfo
	Envelope *eth.ExecutionPayloadEnvelope
	Ref      eth.L2BlockRef
}

BuildSealedEvent is emitted by the engine when a payload finished building, but is not locally inserted as canonical block yet

func (BuildSealedEvent) String

func (ev BuildSealedEvent) String() string

type BuildStartEvent

type BuildStartEvent struct {
	Attributes *derive.AttributesWithParent
}

func (BuildStartEvent) String

func (ev BuildStartEvent) String() string

type BuildStartedEvent

type BuildStartedEvent struct {
	Info eth.PayloadInfo

	BuildStarted time.Time

	Parent eth.L2BlockRef

	// if payload should be promoted to (local) safe (must also be pending safe, see DerivedFrom)
	Concluding bool
	// payload is promoted to pending-safe if non-zero
	DerivedFrom eth.L1BlockRef
}

func (BuildStartedEvent) String

func (ev BuildStartedEvent) String() string

type CrossUpdateHandler added in v1.13.6

type CrossUpdateHandler interface {
	OnCrossUnsafeUpdate(ctx context.Context, crossUnsafe eth.L2BlockRef, localUnsafe eth.L2BlockRef)
	OnCrossSafeUpdate(ctx context.Context, crossSafe eth.L2BlockRef, localSafe eth.L2BlockRef)
}

CrossUpdateHandler handles both cross-unsafe and cross-safe L2 head changes. Nil check required because op-program omits this handler.

type Engine

type Engine interface {
	ExecEngine
	derive.L2Source
}

type EngineController

type EngineController struct {

	// For clearing safe head db when EL sync started
	// EngineController is first initialized and used to initialize SyncDeriver.
	// Embed SyncDeriver into EngineController after initializing SyncDeriver
	SyncDeriver SyncDeriver
	// contains filtered or unexported fields
}

func NewEngineController

func NewEngineController(ctx context.Context, engine ExecEngine, log log.Logger, m opmetrics.Metricer,
	rollupCfg *rollup.Config, syncCfg *sync.Config, l1 sync.L1Chain, emitter event.Emitter,
) *EngineController

func (*EngineController) AddUnsafePayload added in v1.14.1

func (e *EngineController) AddUnsafePayload(ctx context.Context, envelope *eth.ExecutionPayloadEnvelope)

AddUnsafePayload schedules an execution payload to be processed, ahead of deriving it from L1.

func (*EngineController) BackupUnsafeL2Head

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

func (*EngineController) CancelBlock added in v1.13.3

func (e *EngineController) CancelBlock(ctx context.Context, id eth.PayloadInfo) error

func (*EngineController) CommitBlock added in v1.13.3

func (*EngineController) Finalized

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

func (*EngineController) ForceReset added in v1.13.6

func (e *EngineController) ForceReset(ctx context.Context, localUnsafe, crossUnsafe, localSafe, crossSafe, finalized eth.L2BlockRef)

ForceReset performs a forced reset to the specified block references, acquiring lock

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) LowestQueuedUnsafeBlock added in v1.14.1

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

LowestQueuedUnsafeBlock retrieves the first queued-up L2 unsafe payload, or a zeroed reference if there is none.

func (*EngineController) OnEvent added in v1.13.6

func (e *EngineController) OnEvent(ctx context.Context, ev event.Event) bool

func (*EngineController) OpenBlock added in v1.13.3

func (e *EngineController) OpenBlock(ctx context.Context, parent eth.BlockID, attrs *eth.PayloadAttributes) (eth.PayloadInfo, error)

func (*EngineController) PendingSafeL2Head

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

func (*EngineController) PromoteFinalized added in v1.13.6

func (e *EngineController) PromoteFinalized(ctx context.Context, ref eth.L2BlockRef)

func (*EngineController) PromoteSafe added in v1.13.6

func (e *EngineController) PromoteSafe(ctx context.Context, ref eth.L2BlockRef, source eth.L1BlockRef)

func (*EngineController) RequestForkchoiceUpdate added in v1.13.6

func (e *EngineController) RequestForkchoiceUpdate(ctx context.Context)

func (*EngineController) RequestPendingSafeUpdate added in v1.13.6

func (e *EngineController) RequestPendingSafeUpdate(ctx context.Context)

func (*EngineController) SafeL2Head

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

func (*EngineController) SealBlock added in v1.13.3

func (*EngineController) SetAttributesResetter added in v1.13.6

func (e *EngineController) SetAttributesResetter(resetter AttributesForceResetter)

SetAttributesResetter sets the attributes component that needs force reset notifications

func (*EngineController) SetBackupUnsafeL2Head

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

SetBackupUnsafeL2Head implements LocalEngineControl.

func (*EngineController) SetCrossUnsafeHead added in v1.9.3

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

SetCrossUnsafeHead the cross-unsafe head.

func (*EngineController) SetCrossUpdateHandler added in v1.13.6

func (e *EngineController) SetCrossUpdateHandler(handler CrossUpdateHandler)

func (*EngineController) SetFinalizedHead

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

SetFinalizedHead implements LocalEngineControl.

func (*EngineController) SetLocalSafeHead added in v1.9.3

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

SetLocalSafeHead sets the local-safe head.

func (*EngineController) SetOriginSelectorResetter added in v1.13.6

func (e *EngineController) SetOriginSelectorResetter(resetter OriginSelectorForceResetter)

SetOriginSelectorResetter sets the origin selector component that needs force reset notifications

func (*EngineController) SetPendingSafeL2Head

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

SetPendingSafeL2Head implements LocalEngineControl.

func (*EngineController) SetPipelineResetter added in v1.13.6

func (e *EngineController) SetPipelineResetter(resetter PipelineForceResetter)

SetPipelineResetter sets the pipeline component that needs force reset notifications

func (*EngineController) SetSafeHead

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

SetSafeHead sets the cross-safe head.

func (*EngineController) SetUnsafeHead

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

SetUnsafeHead sets the local-unsafe head.

func (*EngineController) TryBackupUnsafeReorg

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

func (*EngineController) TryUpdateEngine

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

func (*EngineController) TryUpdateLocalSafe added in v1.13.6

func (e *EngineController) TryUpdateLocalSafe(ctx context.Context, ref eth.L2BlockRef, concluding bool, source eth.L1BlockRef)

TryUpdateLocalSafe updates the local safe head if the new reference is newer and concluding, acquiring lock

func (*EngineController) TryUpdatePendingSafe added in v1.13.6

func (e *EngineController) TryUpdatePendingSafe(ctx context.Context, ref eth.L2BlockRef, concluding bool, source eth.L1BlockRef)

TryUpdatePendingSafe updates the pending safe head if the new reference is newer, acquiring lock

func (*EngineController) UnsafeL2Head

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

type EngineResetConfirmedEvent

type EngineResetConfirmedEvent struct {
	LocalUnsafe eth.L2BlockRef
	CrossUnsafe eth.L2BlockRef
	LocalSafe   eth.L2BlockRef
	CrossSafe   eth.L2BlockRef
	Finalized   eth.L2BlockRef
}

func (EngineResetConfirmedEvent) String

func (ev EngineResetConfirmedEvent) String() string

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)
	L2BlockRefByHash(ctx context.Context, hash common.Hash) (eth.L2BlockRef, error)
}

type FinalizedUpdateEvent added in v1.9.5

type FinalizedUpdateEvent struct {
	Ref eth.L2BlockRef
}

FinalizedUpdateEvent signals that a block has been marked as finalized.

func (FinalizedUpdateEvent) String added in v1.9.5

func (ev FinalizedUpdateEvent) String() string

type ForkchoiceUpdateEvent

type ForkchoiceUpdateEvent struct {
	UnsafeL2Head, SafeL2Head, FinalizedL2Head eth.L2BlockRef
}

func (ForkchoiceUpdateEvent) String

func (ev ForkchoiceUpdateEvent) String() string

type InteropInvalidateBlockEvent added in v1.11.0

type InteropInvalidateBlockEvent struct {
	Invalidated eth.BlockRef
	Attributes  *derive.AttributesWithParent
}

InteropInvalidateBlockEvent is emitted when a block needs to be invalidated, and a replacement is needed.

func (InteropInvalidateBlockEvent) String added in v1.11.0

func (ev InteropInvalidateBlockEvent) String() string

type InteropReplacedBlockEvent added in v1.11.0

type InteropReplacedBlockEvent struct {
	Ref      eth.BlockRef
	Envelope *eth.ExecutionPayloadEnvelope
}

InteropReplacedBlockEvent is emitted when a replacement is done.

func (InteropReplacedBlockEvent) String added in v1.11.0

func (ev InteropReplacedBlockEvent) String() string

type InvalidPayloadAttributesEvent

type InvalidPayloadAttributesEvent struct {
	Attributes *derive.AttributesWithParent
	Err        error
}

InvalidPayloadAttributesEvent is a signal to external derivers that the attributes were invalid.

func (InvalidPayloadAttributesEvent) String

type Kind

type Kind string

Kind identifies the engine client's kind, used to control the behavior of optimism in different engine clients.

const (
	Geth   Kind = "geth"
	Reth   Kind = "reth"
	Erigon Kind = "erigon"
)

func (*Kind) Clone

func (kind *Kind) Clone() any

func (*Kind) Set

func (kind *Kind) Set(value string) error

func (Kind) String

func (kind Kind) String() string

func (Kind) SupportsPostFinalizationELSync

func (kind Kind) SupportsPostFinalizationELSync() bool

type LocalSafeUpdateEvent added in v1.9.3

type LocalSafeUpdateEvent struct {
	Ref    eth.L2BlockRef
	Source eth.L1BlockRef
}

LocalSafeUpdateEvent signals that a block is now considered to be local-safe.

func (LocalSafeUpdateEvent) String added in v1.9.3

func (ev LocalSafeUpdateEvent) String() string

type Metrics

type Metrics interface {
	RecordUnsafePayloadsBuffer(length uint64, memSize uint64, next eth.BlockID)
}

Metrics interface for CLSync functionality

type OriginSelectorForceResetter added in v1.13.6

type OriginSelectorForceResetter interface {
	ResetOrigins()
}

type PayloadInvalidEvent

type PayloadInvalidEvent struct {
	Envelope *eth.ExecutionPayloadEnvelope
	Err      error
}

func (PayloadInvalidEvent) String

func (ev PayloadInvalidEvent) String() string

type PayloadProcessEvent

type PayloadProcessEvent struct {
	// if payload should be promoted to (local) safe (must also be pending safe, see DerivedFrom)
	Concluding bool
	// payload is promoted to pending-safe if non-zero
	DerivedFrom  eth.L1BlockRef
	BuildStarted time.Time

	Envelope *eth.ExecutionPayloadEnvelope
	Ref      eth.L2BlockRef
}

func (PayloadProcessEvent) String

func (ev PayloadProcessEvent) String() string

type PayloadSealExpiredErrorEvent

type PayloadSealExpiredErrorEvent struct {
	Info eth.PayloadInfo
	Err  error

	Concluding  bool
	DerivedFrom eth.L1BlockRef
}

PayloadSealExpiredErrorEvent identifies a form of failed payload-sealing that is not coupled to the attributes themselves, but rather the build-job process. The user should re-attempt by starting a new build process. The payload-sealing job should not be re-attempted, as it most likely expired, timed out, or referenced an otherwise invalidated block-building job identifier.

func (PayloadSealExpiredErrorEvent) String

type PayloadSealInvalidEvent

type PayloadSealInvalidEvent struct {
	Info eth.PayloadInfo
	Err  error

	Concluding  bool
	DerivedFrom eth.L1BlockRef
}

PayloadSealInvalidEvent identifies a permanent in-consensus problem with the payload sealing.

func (PayloadSealInvalidEvent) String

func (ev PayloadSealInvalidEvent) String() string

type PayloadSuccessEvent

type PayloadSuccessEvent struct {
	// if payload should be promoted to (local) safe (must also be pending safe, see DerivedFrom)
	Concluding bool
	// payload is promoted to pending-safe if non-zero
	DerivedFrom   eth.L1BlockRef
	BuildStarted  time.Time
	InsertStarted time.Time

	Envelope *eth.ExecutionPayloadEnvelope
	Ref      eth.L2BlockRef
}

func (PayloadSuccessEvent) String

func (ev PayloadSuccessEvent) String() string

type PayloadsQueue added in v1.14.1

type PayloadsQueue struct {
	MaxSize uint64

	SizeFn func(p *eth.ExecutionPayloadEnvelope) uint64
	// contains filtered or unexported fields
}

PayloadsQueue buffers payloads by block number. PayloadsQueue is not safe to use concurrently. PayloadsQueue exposes typed Push/Peek/Pop methods to use the queue, without the need to use heap.Push/heap.Pop as caller. PayloadsQueue maintains a MaxSize by counting and tracking sizes of added eth.ExecutionPayload entries. When the size grows too large, the first (lowest block-number) payload is removed from the queue. PayloadsQueue allows entries with same block number, but does not allow duplicate blocks

func NewPayloadsQueue added in v1.14.1

func NewPayloadsQueue(log log.Logger, maxSize uint64, sizeFn func(p *eth.ExecutionPayloadEnvelope) uint64) *PayloadsQueue

func (*PayloadsQueue) DropInapplicableUnsafePayloads added in v1.14.1

func (pq *PayloadsQueue) DropInapplicableUnsafePayloads(event ForkchoiceUpdateEvent)

func (*PayloadsQueue) Len added in v1.14.1

func (upq *PayloadsQueue) Len() int

func (*PayloadsQueue) MemSize added in v1.14.1

func (upq *PayloadsQueue) MemSize() uint64

func (*PayloadsQueue) Peek added in v1.14.1

Peek retrieves the payload with the lowest block number from the queue in O(1), or nil if the queue is empty.

func (*PayloadsQueue) Pop added in v1.14.1

Pop removes the payload with the lowest block number from the queue in O(log(N)), and may return nil if the queue is empty.

func (*PayloadsQueue) Push added in v1.14.1

Push adds the payload to the queue, in O(log(N)).

Don't DoS ourselves by buffering too many unsafe payloads. If the queue size after pushing exceed the allowed memory, then pop payloads until memory is not exceeding anymore.

We prefer higher block numbers over lower block numbers, since lower block numbers are more likely to be conflicts and/or read from L1 sooner. The higher payload block numbers can be preserved, and once L1 contents meets these, they can all be processed in order.

type PendingSafeUpdateEvent

type PendingSafeUpdateEvent struct {
	PendingSafe eth.L2BlockRef
	Unsafe      eth.L2BlockRef // tip, added to the signal, to determine if there are existing blocks to consolidate
}

func (PendingSafeUpdateEvent) String

func (ev PendingSafeUpdateEvent) String() string

type PipelineForceResetter added in v1.13.6

type PipelineForceResetter interface {
	ResetPipeline()
}

type PromoteCrossUnsafeEvent added in v1.9.3

type PromoteCrossUnsafeEvent struct {
	Ref eth.L2BlockRef
}

PromoteCrossUnsafeEvent signals that the given block may be promoted to cross-unsafe.

func (PromoteCrossUnsafeEvent) String added in v1.9.3

func (ev PromoteCrossUnsafeEvent) String() string

type ResetEngineControl

type ResetEngineControl interface {
	SetUnsafeHead(eth.L2BlockRef)
	SetCrossUnsafeHead(ref eth.L2BlockRef)
	SetLocalSafeHead(ref eth.L2BlockRef)
	SetSafeHead(eth.L2BlockRef)
	SetFinalizedHead(eth.L2BlockRef)
	SetBackupUnsafeL2Head(block eth.L2BlockRef, triggerReorg bool)
	SetPendingSafeL2Head(eth.L2BlockRef)
}

type ResetEngineRequestEvent

type ResetEngineRequestEvent struct {
}

ResetEngineRequestEvent requests the EngineController to walk the L2 chain backwards until it finds a plausible unsafe head, and find an L2 safe block that is guaranteed to still be from the L1 chain. This event is not used in interop.

func (ResetEngineRequestEvent) String

func (ev ResetEngineRequestEvent) String() string

type RollupAPI added in v1.13.3

type RollupAPI interface {
	apis.BuildAPI
	apis.CommitAPI
}

RollupAPI is the API we serve as rollup-node to interact with the execution engine and forkchoice state.

type SafeDerivedEvent

type SafeDerivedEvent struct {
	Safe   eth.L2BlockRef
	Source eth.L1BlockRef
}

SafeDerivedEvent signals that a block was determined to be safe, and derived from the given L1 block. This is signaled upon procedural call of PromoteSafe method

func (SafeDerivedEvent) String

func (ev SafeDerivedEvent) String() string

type SyncDeriver added in v1.13.6

type SyncDeriver interface {
	OnELSyncStarted()
}

type UnsafeUpdateEvent added in v1.9.3

type UnsafeUpdateEvent struct {
	Ref eth.L2BlockRef
}

UnsafeUpdateEvent signals that the given block is now considered safe. This is pre-forkchoice update; the change may not be reflected yet in the EL.

func (UnsafeUpdateEvent) String added in v1.9.3

func (ev UnsafeUpdateEvent) String() string

Jump to

Keyboard shortcuts

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