Commit 43b1c4c0 authored by Minhyuk Kim's avatar Minhyuk Kim Committed by GitHub

Add new sync config: l2.enginekind (#10767)

* Add new syncmode "force-execution-layer"

* Add tests

* Add l2.enginekind flag instead of ForceELSync

* Move l2EngineClientKind to EngineController

* Fix lint

* Rename engine.EngineClientKind to engine.Kind and added engineKind.SupportsPostFinalizationELSync()

* Refactor EL/CL tests in sync_test.go

* Refactor tests in sync_test.go

* Rename engine to ec

* Incorporate enginekind-specific flags into syncConfig, and hide engine kind from the engine controller logic

* Fix tests

---------
Co-authored-by: default avatarprotolambda <proto@protolambda.com>
parent 822df50b
...@@ -37,10 +37,7 @@ import ( ...@@ -37,10 +37,7 @@ import (
type L2Verifier struct { type L2Verifier struct {
log log.Logger log log.Logger
eng interface { eng L2API
engine.Engine
L2BlockRefByNumber(ctx context.Context, num uint64) (eth.L2BlockRef, error)
}
syncStatus driver.SyncStatusTracker syncStatus driver.SyncStatusTracker
...@@ -100,7 +97,7 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri ...@@ -100,7 +97,7 @@ func NewL2Verifier(t Testing, log log.Logger, l1 derive.L1Fetcher, blobsSrc deri
}) })
metrics := &testutils.TestDerivationMetrics{} metrics := &testutils.TestDerivationMetrics{}
ec := engine.NewEngineController(eng, log, metrics, cfg, syncCfg.SyncMode, synchronousEvents) ec := engine.NewEngineController(eng, log, metrics, cfg, syncCfg, synchronousEvents)
engineResetDeriver := engine.NewEngineResetDeriver(ctx, log, cfg, l1, eng, syncCfg, synchronousEvents) engineResetDeriver := engine.NewEngineResetDeriver(ctx, log, cfg, l1, eng, syncCfg, synchronousEvents)
clSync := clsync.NewCLSync(log, cfg, metrics, synchronousEvents) clSync := clsync.NewCLSync(log, cfg, metrics, synchronousEvents)
......
...@@ -5,6 +5,7 @@ import ( ...@@ -5,6 +5,7 @@ import (
"crypto/ecdsa" "crypto/ecdsa"
crand "crypto/rand" crand "crypto/rand"
"fmt" "fmt"
"math/big" "math/big"
"math/rand" "math/rand"
"testing" "testing"
......
...@@ -10,7 +10,7 @@ import ( ...@@ -10,7 +10,7 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/beacon/engine" gethengine "github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
...@@ -23,6 +23,7 @@ import ( ...@@ -23,6 +23,7 @@ import (
engine2 "github.com/ethereum-optimism/optimism/op-node/rollup/engine" engine2 "github.com/ethereum-optimism/optimism/op-node/rollup/engine"
"github.com/ethereum-optimism/optimism/op-node/rollup/event" "github.com/ethereum-optimism/optimism/op-node/rollup/event"
"github.com/ethereum-optimism/optimism/op-node/rollup/sync" "github.com/ethereum-optimism/optimism/op-node/rollup/sync"
plasma "github.com/ethereum-optimism/optimism/op-plasma"
"github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum-optimism/optimism/op-service/sources" "github.com/ethereum-optimism/optimism/op-service/sources"
"github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-service/testlog"
...@@ -585,7 +586,7 @@ func TestBackupUnsafeReorgForkChoiceNotInputError(gt *testing.T) { ...@@ -585,7 +586,7 @@ func TestBackupUnsafeReorgForkChoiceNotInputError(gt *testing.T) {
serverErrCnt := 2 serverErrCnt := 2
for i := 0; i < serverErrCnt; i++ { for i := 0; i < serverErrCnt; i++ {
// mock forkChoiceUpdate failure while restoring previous unsafe chain using backupUnsafe. // mock forkChoiceUpdate failure while restoring previous unsafe chain using backupUnsafe.
seqEng.ActL2RPCFail(t, engine.GenericServerError) seqEng.ActL2RPCFail(t, gethengine.GenericServerError)
// TryBackupUnsafeReorg is called - forkChoiceUpdate returns GenericServerError so retry // TryBackupUnsafeReorg is called - forkChoiceUpdate returns GenericServerError so retry
sequencer.ActL2EventsUntil(t, event.Is[rollup.EngineTemporaryErrorEvent], 100, false) sequencer.ActL2EventsUntil(t, event.Is[rollup.EngineTemporaryErrorEvent], 100, false)
// backupUnsafeHead not emptied yet // backupUnsafeHead not emptied yet
...@@ -608,25 +609,15 @@ func TestBackupUnsafeReorgForkChoiceNotInputError(gt *testing.T) { ...@@ -608,25 +609,15 @@ func TestBackupUnsafeReorgForkChoiceNotInputError(gt *testing.T) {
require.Equal(t, sequencer.L2Safe().Number, uint64(0)) require.Equal(t, sequencer.L2Safe().Number, uint64(0))
} }
// TestELSync tests that a verifier will have the EL import the full chain from the sequencer // builds l2 blocks within the specified range `from` - `to`
// when passed a single unsafe block. op-geth can either snap sync or full sync here. // and performs an EL sync between the sequencer and the verifier,
func TestELSync(gt *testing.T) { // then checks the validity of the payloads within a specified block range.
t := NewDefaultTesting(gt) func PerformELSyncAndCheckPayloads(t Testing, miner *L1Miner, seqEng *L2Engine, sequencer *L2Sequencer, verEng *L2Engine, verifier *L2Verifier, seqEngCl *sources.EngineClient, from, to uint64) {
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams) miner.ActEmptyBlock(t)
sd := e2eutils.Setup(t, dp, defaultAlloc)
log := testlog.Logger(t, log.LevelInfo)
miner, seqEng, sequencer := setupSequencerTest(t, sd, log)
// Enable engine P2P sync
verEng, verifier := setupVerifier(t, sd, log, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), &sync.Config{SyncMode: sync.ELSync})
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), log, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err)
sequencer.ActL2PipelineFull(t) sequencer.ActL2PipelineFull(t)
// Build 10 L1 blocks on the sequencer // Build L1 blocks on the sequencer
for i := 0; i < 10; i++ { for i := from; i < to; i++ {
// Build a L2 block // Build a L2 block
sequencer.ActL2StartBlock(t) sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t) sequencer.ActL2EndBlock(t)
...@@ -638,7 +629,7 @@ func TestELSync(gt *testing.T) { ...@@ -638,7 +629,7 @@ func TestELSync(gt *testing.T) {
// Insert it on the verifier // Insert it on the verifier
seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe) seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err) require.NoError(t, err)
seqStart, err := seqEngCl.PayloadByNumber(t.Ctx(), 1) seqStart, err := seqEngCl.PayloadByNumber(t.Ctx(), from)
require.NoError(t, err) require.NoError(t, err)
verifier.ActL2InsertUnsafePayload(seqHead)(t) verifier.ActL2InsertUnsafePayload(seqHead)(t)
...@@ -654,7 +645,7 @@ func TestELSync(gt *testing.T) { ...@@ -654,7 +645,7 @@ func TestELSync(gt *testing.T) {
// Verify this by checking that the verifier has the correct value for block 1 // Verify this by checking that the verifier has the correct value for block 1
require.Eventually(t, require.Eventually(t,
func() bool { func() bool {
block, err := verifier.eng.L2BlockRefByNumber(t.Ctx(), 1) block, err := verifier.eng.L2BlockRefByNumber(t.Ctx(), from)
if err != nil { if err != nil {
return false return false
} }
...@@ -665,6 +656,69 @@ func TestELSync(gt *testing.T) { ...@@ -665,6 +656,69 @@ func TestELSync(gt *testing.T) {
) )
} }
// verifies that a specific block number on the L2 engine has the expected label.
func VerifyBlock(t Testing, engine L2API, number uint64, label eth.BlockLabel) {
id, err := engine.L2BlockRefByLabel(t.Ctx(), label)
require.NoError(t, err)
require.Equal(t, number, id.Number)
}
// submits batch at a specified block number
func BatchSubmitBlock(t Testing, miner *L1Miner, sequencer *L2Sequencer, verifier *L2Verifier, batcher *L2Batcher, dp *e2eutils.DeployParams, number uint64) {
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
batcher.ActSubmitAll(t)
miner.ActL1StartBlock(number)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
}
// TestELSync tests that a verifier will have the EL import the full chain from the sequencer
// when passed a single unsafe block. op-geth can either snap sync or full sync here.
func TestELSync(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
sd := e2eutils.Setup(t, dp, defaultAlloc)
log := testlog.Logger(t, log.LevelInfo)
miner, seqEng, sequencer := setupSequencerTest(t, sd, log)
// Enable engine P2P sync
verEng, verifier := setupVerifier(t, sd, log, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), &sync.Config{SyncMode: sync.ELSync})
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), log, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err)
PerformELSyncAndCheckPayloads(t, miner, seqEng, sequencer, verEng, verifier, seqEngCl, 0, 10)
}
func PrepareELSyncedNode(t Testing, miner *L1Miner, sequencer *L2Sequencer, seqEng *L2Engine, verifier *L2Verifier, verEng *L2Engine, seqEngCl *sources.EngineClient, batcher *L2Batcher, dp *e2eutils.DeployParams) {
PerformELSyncAndCheckPayloads(t, miner, seqEng, sequencer, verEng, verifier, seqEngCl, 0, 10)
// Despite downloading the blocks, it has not finished finalizing
_, err := verifier.eng.L2BlockRefByLabel(t.Ctx(), "safe")
require.ErrorIs(t, err, ethereum.NotFound)
// Insert a block on the verifier to end snap sync
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err)
verifier.ActL2InsertUnsafePayload(seqHead)(t)
// Check that safe + finalized are there
VerifyBlock(t, verifier.eng, 11, eth.Safe)
VerifyBlock(t, verifier.eng, 11, eth.Finalized)
// Batch submit everything
BatchSubmitBlock(t, miner, sequencer, verifier, batcher, dp, 12)
// Verify that the batch submitted blocks are there now
VerifyBlock(t, sequencer.eng, 12, eth.Safe)
VerifyBlock(t, verifier.eng, 12, eth.Safe)
}
// TestELSyncTransitionstoCL tests that a verifier which starts with EL sync can switch back to a proper CL sync. // TestELSyncTransitionstoCL tests that a verifier which starts with EL sync can switch back to a proper CL sync.
// It takes a sequencer & verifier through the following: // It takes a sequencer & verifier through the following:
// 1. Build 10 unsafe blocks on the sequencer // 1. Build 10 unsafe blocks on the sequencer
...@@ -692,83 +746,7 @@ func TestELSyncTransitionstoCL(gt *testing.T) { ...@@ -692,83 +746,7 @@ func TestELSyncTransitionstoCL(gt *testing.T) {
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), logger, nil, sources.EngineClientDefaultConfig(sd.RollupCfg)) seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), logger, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err) require.NoError(t, err)
miner.ActEmptyBlock(t) PrepareELSyncedNode(t, miner, sequencer, seqEng, verifier, verEng, seqEngCl, batcher, dp)
sequencer.ActL2PipelineFull(t)
// Build 10 L1 blocks on the sequencer
for i := 0; i < 10; i++ {
// Build a L2 block
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
}
// Wait longer to peer. This tests flakes or takes a long time when the op-geth instances are not able to peer.
verEng.AddPeers(seqEng.Enode())
// Insert it on the verifier
seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err)
seqStart, err := seqEngCl.PayloadByNumber(t.Ctx(), 1)
require.NoError(t, err)
verifier.ActL2InsertUnsafePayload(seqHead)(t)
require.Eventually(t,
func() bool {
return seqEng.PeerCount() > 0 && verEng.PeerCount() > 0
},
120*time.Second, 1500*time.Millisecond,
"Sequencer & Verifier must peer with each other for snap sync to work",
)
// Expect snap sync to download & execute the entire chain
// Verify this by checking that the verifier has the correct value for block 1
require.Eventually(t,
func() bool {
block, err := verifier.eng.L2BlockRefByNumber(t.Ctx(), 1)
if err != nil {
return false
}
return seqStart.ExecutionPayload.BlockHash == block.Hash
},
60*time.Second, 1500*time.Millisecond,
"verifier did not snap sync",
)
// Despite downloading the blocks, it has not finished finalizing
_, err = verifier.eng.L2BlockRefByLabel(t.Ctx(), "safe")
require.ErrorIs(t, err, ethereum.NotFound)
// Insert a block on the verifier to end snap sync
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
seqHead, err = seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err)
verifier.ActL2InsertUnsafePayload(seqHead)(t)
// Check that safe + finalized are there
id, err := verifier.eng.L2BlockRefByLabel(t.Ctx(), eth.Safe)
require.Equal(t, uint64(11), id.Number)
require.NoError(t, err)
id, err = verifier.eng.L2BlockRefByLabel(t.Ctx(), eth.Finalized)
require.Equal(t, uint64(11), id.Number)
require.NoError(t, err)
// Batch submit everything
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
batcher.ActSubmitAll(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
// Verify that the batch submitted blocks are there now
id, err = sequencer.eng.L2BlockRefByLabel(t.Ctx(), eth.Safe)
require.NoError(t, err)
require.Equal(t, uint64(12), id.Number)
id, err = verifier.eng.L2BlockRefByLabel(t.Ctx(), eth.Safe)
require.NoError(t, err)
require.Equal(t, uint64(12), id.Number)
// Build another 10 L1 blocks on the sequencer // Build another 10 L1 blocks on the sequencer
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
...@@ -780,7 +758,7 @@ func TestELSyncTransitionstoCL(gt *testing.T) { ...@@ -780,7 +758,7 @@ func TestELSyncTransitionstoCL(gt *testing.T) {
// Now pass payloads to the derivation pipeline // Now pass payloads to the derivation pipeline
// This is a little hacky that we have to manually switch between InsertBlock // This is a little hacky that we have to manually switch between InsertBlock
// and UnsafeGossipReceive in the tests // and UnsafeGossipReceive in the tests
seqHead, err = seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe) seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err) require.NoError(t, err)
verifier.ActL2UnsafeGossipReceive(seqHead)(t) verifier.ActL2UnsafeGossipReceive(seqHead)(t)
verifier.ActL2PipelineFull(t) verifier.ActL2PipelineFull(t)
...@@ -799,27 +777,102 @@ func TestELSyncTransitionstoCL(gt *testing.T) { ...@@ -799,27 +777,102 @@ func TestELSyncTransitionstoCL(gt *testing.T) {
// This was failing prior to PR 9661 because op-node would attempt to immediately insert blocks into the EL inside the engine queue. op-geth // This was failing prior to PR 9661 because op-node would attempt to immediately insert blocks into the EL inside the engine queue. op-geth
// would not be able to fetch the second range of blocks & it would wipe out the unsafe payloads queue because op-node thought that it had a // would not be able to fetch the second range of blocks & it would wipe out the unsafe payloads queue because op-node thought that it had a
// higher unsafe block but op-geth did not. // higher unsafe block but op-geth did not.
id, err = verifier.eng.L2BlockRefByLabel(t.Ctx(), eth.Unsafe) VerifyBlock(t, verifier.eng, 22, eth.Unsafe)
require.NoError(t, err)
require.Equal(t, uint64(22), id.Number)
// Create 1 more block & batch submit everything // Create 1 more block & batch submit everything
sequencer.ActL2StartBlock(t) BatchSubmitBlock(t, miner, sequencer, verifier, batcher, dp, 12)
sequencer.ActL2EndBlock(t)
batcher.ActSubmitAll(t)
miner.ActL1StartBlock(12)(t)
miner.ActL1IncludeTx(dp.Addresses.Batcher)(t)
miner.ActL1EndBlock(t)
sequencer.ActL2PipelineFull(t)
verifier.ActL2PipelineFull(t)
// Verify that the batch submitted blocks are there now // Verify that the batch submitted blocks are there now
id, err = sequencer.eng.L2BlockRefByLabel(t.Ctx(), eth.Safe) VerifyBlock(t, sequencer.eng, 23, eth.Safe)
VerifyBlock(t, verifier.eng, 23, eth.Safe)
}
func TestELSyncTransitionsToCLSyncAfterNodeRestart(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
sd := e2eutils.Setup(t, dp, defaultAlloc)
logger := testlog.Logger(t, log.LevelInfo)
captureLog, captureLogHandler := testlog.CaptureLogger(t, log.LevelInfo)
miner, seqEng, sequencer := setupSequencerTest(t, sd, logger)
batcher := NewL2Batcher(logger, sd.RollupCfg, DefaultBatcherCfg(dp), sequencer.RollupClient(), miner.EthClient(), seqEng.EthClient(), seqEng.EngineClient(t, sd.RollupCfg))
// Enable engine P2P sync
verEng, verifier := setupVerifier(t, sd, captureLog, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), &sync.Config{SyncMode: sync.ELSync})
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), logger, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err)
PrepareELSyncedNode(t, miner, sequencer, seqEng, verifier, verEng, seqEngCl, batcher, dp)
// Create a new verifier which is essentially a new op-node with the sync mode of ELSync and default geth engine kind.
verifier = NewL2Verifier(t, captureLog, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), plasma.Disabled, verifier.eng, sd.RollupCfg, &sync.Config{SyncMode: sync.ELSync}, defaultVerifierCfg().safeHeadListener)
// Build another 10 L1 blocks on the sequencer
for i := 0; i < 10; i++ {
// Build a L2 block
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
}
// Insert new block to the engine and kick off a CL sync
seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err)
verifier.ActL2InsertUnsafePayload(seqHead)(t)
// Verify that the derivation pipeline did not request a sync to the new head. This is the core of the test, but a little fragile.
record := captureLogHandler.FindLog(testlog.NewMessageFilter("Forkchoice requested sync to new head"), testlog.NewAttributesFilter("number", "22"))
require.Nil(t, record, "The verifier should not request to sync to block number 22 because it is in CL mode, not EL mode at this point.")
// Verify that op-node has skipped ELSync and started CL sync because geth has finalized block from ELSync.
record = captureLogHandler.FindLog(testlog.NewMessageFilter("Skipping EL sync and going straight to CL sync because there is a finalized block"))
require.NotNil(t, record, "The verifier should skip EL Sync at this point.")
}
func TestForcedELSyncCLAfterNodeRestart(gt *testing.T) {
t := NewDefaultTesting(gt)
dp := e2eutils.MakeDeployParams(t, defaultRollupTestParams)
sd := e2eutils.Setup(t, dp, defaultAlloc)
logger := testlog.Logger(t, log.LevelInfo)
captureLog, captureLogHandler := testlog.CaptureLogger(t, log.LevelInfo)
miner, seqEng, sequencer := setupSequencerTest(t, sd, logger)
batcher := NewL2Batcher(logger, sd.RollupCfg, DefaultBatcherCfg(dp), sequencer.RollupClient(), miner.EthClient(), seqEng.EthClient(), seqEng.EngineClient(t, sd.RollupCfg))
// Enable engine P2P sync
verEng, verifier := setupVerifier(t, sd, captureLog, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), &sync.Config{SyncMode: sync.ELSync})
seqEngCl, err := sources.NewEngineClient(seqEng.RPCClient(), logger, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, uint64(23), id.Number)
id, err = verifier.eng.L2BlockRefByLabel(t.Ctx(), eth.Safe) PrepareELSyncedNode(t, miner, sequencer, seqEng, verifier, verEng, seqEngCl, batcher, dp)
// Create a new verifier which is essentially a new op-node with the sync mode of ELSync and erigon engine kind.
verifier2 := NewL2Verifier(t, captureLog, miner.L1Client(t, sd.RollupCfg), miner.BlobStore(), plasma.Disabled, verifier.eng, sd.RollupCfg, &sync.Config{SyncMode: sync.ELSync, SupportsPostFinalizationELSync: true}, defaultVerifierCfg().safeHeadListener)
// Build another 10 L1 blocks on the sequencer
for i := 0; i < 10; i++ {
// Build a L2 block
sequencer.ActL2StartBlock(t)
sequencer.ActL2EndBlock(t)
}
// Insert it on the verifier and kick off EL sync.
// Syncing doesn't actually work in test,
// but we can validate the engine is starting EL sync through p2p
seqHead, err := seqEngCl.PayloadByLabel(t.Ctx(), eth.Unsafe)
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, uint64(23), id.Number) verifier2.ActL2InsertUnsafePayload(seqHead)(t)
// Verify that the derivation pipeline did not request a sync to the new head. This is the core of the test, but a little fragile.
record := captureLogHandler.FindLog(testlog.NewMessageFilter("Forkchoice requested sync to new head"), testlog.NewAttributesFilter("number", "22"))
require.NotNil(t, record, "The verifier should request to sync to block number 22 in EL mode")
// Verify that op-node is starting ELSync.
record = captureLogHandler.FindLog(testlog.NewMessageFilter("Skipping EL sync and going straight to CL sync because there is a finalized block"))
require.Nil(t, record, "The verifier should start EL Sync when l2.engineKind is not geth")
record = captureLogHandler.FindLog(testlog.NewMessageFilter("Starting EL sync"))
require.NotNil(t, record, "The verifier should start EL Sync when l2.engineKind is not geth")
} }
func TestInvalidPayloadInSpanBatch(gt *testing.T) { func TestInvalidPayloadInSpanBatch(gt *testing.T) {
......
...@@ -6,6 +6,7 @@ import ( ...@@ -6,6 +6,7 @@ import (
"github.com/urfave/cli/v2" "github.com/urfave/cli/v2"
"github.com/ethereum-optimism/optimism/op-node/rollup/engine"
"github.com/ethereum-optimism/optimism/op-node/rollup/sync" "github.com/ethereum-optimism/optimism/op-node/rollup/sync"
plasma "github.com/ethereum-optimism/optimism/op-plasma" plasma "github.com/ethereum-optimism/optimism/op-plasma"
openum "github.com/ethereum-optimism/optimism/op-service/enum" openum "github.com/ethereum-optimism/optimism/op-service/enum"
...@@ -191,6 +192,17 @@ var ( ...@@ -191,6 +192,17 @@ var (
Value: time.Second * 12, Value: time.Second * 12,
Category: L1RPCCategory, Category: L1RPCCategory,
} }
L2EngineKind = &cli.GenericFlag{
Name: "l2.enginekind",
Usage: "The kind of engine client, used to control the behavior of optimism in respect to different types of engine clients. Valid options: " +
openum.EnumString(engine.Kinds),
EnvVars: prefixEnvVars("L2_ENGINE_KIND"),
Value: func() *engine.Kind {
out := engine.Geth
return &out
}(),
Category: RollupCategory,
}
VerifierL1Confs = &cli.Uint64Flag{ VerifierL1Confs = &cli.Uint64Flag{
Name: "verifier.l1-confs", Name: "verifier.l1-confs",
Usage: "Number of L1 blocks to keep distance from the L1 head before deriving L2 data from. Reorgs are supported, but may be slow to perform.", Usage: "Number of L1 blocks to keep distance from the L1 head before deriving L2 data from. Reorgs are supported, but may be slow to perform.",
...@@ -404,6 +416,7 @@ var optionalFlags = []cli.Flag{ ...@@ -404,6 +416,7 @@ var optionalFlags = []cli.Flag{
ConductorRpcFlag, ConductorRpcFlag,
ConductorRpcTimeoutFlag, ConductorRpcTimeoutFlag,
SafeDBPath, SafeDBPath,
L2EngineKind,
} }
var DeprecatedFlags = []cli.Flag{ var DeprecatedFlags = []cli.Flag{
......
...@@ -110,6 +110,7 @@ func TestEnvVarFormat(t *testing.T) { ...@@ -110,6 +110,7 @@ func TestEnvVarFormat(t *testing.T) {
L2EngineJWTSecret.Name, L2EngineJWTSecret.Name,
L1TrustRPC.Name, L1TrustRPC.Name,
L1RPCProviderKind.Name, L1RPCProviderKind.Name,
L2EngineKind.Name,
SnapshotLog.Name, SnapshotLog.Name,
BackupL2UnsafeSyncRPC.Name, BackupL2UnsafeSyncRPC.Name,
BackupL2UnsafeSyncRPCTrustRPC.Name, BackupL2UnsafeSyncRPCTrustRPC.Name,
......
...@@ -194,7 +194,7 @@ func NewDriver( ...@@ -194,7 +194,7 @@ func NewDriver(
sequencerConfDepth := NewConfDepth(driverCfg.SequencerConfDepth, statusTracker.L1Head, l1) sequencerConfDepth := NewConfDepth(driverCfg.SequencerConfDepth, statusTracker.L1Head, l1)
findL1Origin := NewL1OriginSelector(log, cfg, sequencerConfDepth) findL1Origin := NewL1OriginSelector(log, cfg, sequencerConfDepth)
verifConfDepth := NewConfDepth(driverCfg.VerifierConfDepth, statusTracker.L1Head, l1) verifConfDepth := NewConfDepth(driverCfg.VerifierConfDepth, statusTracker.L1Head, l1)
ec := engine.NewEngineController(l2, log, metrics, cfg, syncCfg.SyncMode, synchronousEvents) ec := engine.NewEngineController(l2, log, metrics, cfg, syncCfg, synchronousEvents)
engineResetDeriver := engine.NewEngineResetDeriver(driverCtx, log, cfg, l1, l2, syncCfg, synchronousEvents) engineResetDeriver := engine.NewEngineResetDeriver(driverCtx, log, cfg, l1, l2, syncCfg, synchronousEvents)
clSync := clsync.NewCLSync(log, cfg, metrics, synchronousEvents) clSync := clsync.NewCLSync(log, cfg, metrics, synchronousEvents)
......
...@@ -48,7 +48,7 @@ type EngineController struct { ...@@ -48,7 +48,7 @@ type EngineController struct {
engine ExecEngine // Underlying execution engine RPC engine ExecEngine // Underlying execution engine RPC
log log.Logger log log.Logger
metrics derive.Metrics metrics derive.Metrics
syncMode sync.Mode syncCfg *sync.Config
syncStatus syncStatusEnum syncStatus syncStatusEnum
chainSpec *rollup.ChainSpec chainSpec *rollup.ChainSpec
rollupCfg *rollup.Config rollupCfg *rollup.Config
...@@ -79,9 +79,9 @@ type EngineController struct { ...@@ -79,9 +79,9 @@ type EngineController struct {
} }
func NewEngineController(engine ExecEngine, log log.Logger, metrics derive.Metrics, func NewEngineController(engine ExecEngine, log log.Logger, metrics derive.Metrics,
rollupCfg *rollup.Config, syncMode sync.Mode, emitter event.Emitter) *EngineController { rollupCfg *rollup.Config, syncCfg *sync.Config, emitter event.Emitter) *EngineController {
syncStatus := syncStatusCL syncStatus := syncStatusCL
if syncMode == sync.ELSync { if syncCfg.SyncMode == sync.ELSync {
syncStatus = syncStatusWillStartEL syncStatus = syncStatusWillStartEL
} }
...@@ -91,7 +91,7 @@ func NewEngineController(engine ExecEngine, log log.Logger, metrics derive.Metri ...@@ -91,7 +91,7 @@ func NewEngineController(engine ExecEngine, log log.Logger, metrics derive.Metri
metrics: metrics, metrics: metrics,
chainSpec: rollup.NewChainSpec(rollupCfg), chainSpec: rollup.NewChainSpec(rollupCfg),
rollupCfg: rollupCfg, rollupCfg: rollupCfg,
syncMode: syncMode, syncCfg: syncCfg,
syncStatus: syncStatus, syncStatus: syncStatus,
clock: clock.SystemClock, clock: clock.SystemClock,
emitter: emitter, emitter: emitter,
...@@ -329,7 +329,7 @@ func (e *EngineController) resetBuildingState() { ...@@ -329,7 +329,7 @@ func (e *EngineController) resetBuildingState() {
// checkNewPayloadStatus checks returned status of engine_newPayloadV1 request for next unsafe payload. // checkNewPayloadStatus checks returned status of engine_newPayloadV1 request for next unsafe payload.
// It returns true if the status is acceptable. // It returns true if the status is acceptable.
func (e *EngineController) checkNewPayloadStatus(status eth.ExecutePayloadStatus) bool { func (e *EngineController) checkNewPayloadStatus(status eth.ExecutePayloadStatus) bool {
if e.syncMode == sync.ELSync { if e.syncCfg.SyncMode == sync.ELSync {
if status == eth.ExecutionValid && e.syncStatus == syncStatusStartedEL { if status == eth.ExecutionValid && e.syncStatus == syncStatusStartedEL {
e.syncStatus = syncStatusFinishedELButNotFinalized e.syncStatus = syncStatusFinishedELButNotFinalized
} }
...@@ -342,7 +342,7 @@ func (e *EngineController) checkNewPayloadStatus(status eth.ExecutePayloadStatus ...@@ -342,7 +342,7 @@ func (e *EngineController) checkNewPayloadStatus(status eth.ExecutePayloadStatus
// checkForkchoiceUpdatedStatus checks returned status of engine_forkchoiceUpdatedV1 request for next unsafe payload. // checkForkchoiceUpdatedStatus checks returned status of engine_forkchoiceUpdatedV1 request for next unsafe payload.
// It returns true if the status is acceptable. // It returns true if the status is acceptable.
func (e *EngineController) checkForkchoiceUpdatedStatus(status eth.ExecutePayloadStatus) bool { func (e *EngineController) checkForkchoiceUpdatedStatus(status eth.ExecutePayloadStatus) bool {
if e.syncMode == sync.ELSync { if e.syncCfg.SyncMode == sync.ELSync {
if status == eth.ExecutionValid && e.syncStatus == syncStatusStartedEL { if status == eth.ExecutionValid && e.syncStatus == syncStatusStartedEL {
e.syncStatus = syncStatusFinishedELButNotFinalized e.syncStatus = syncStatusFinishedELButNotFinalized
} }
...@@ -398,7 +398,7 @@ func (e *EngineController) InsertUnsafePayload(ctx context.Context, envelope *et ...@@ -398,7 +398,7 @@ func (e *EngineController) InsertUnsafePayload(ctx context.Context, envelope *et
if e.syncStatus == syncStatusWillStartEL { if e.syncStatus == syncStatusWillStartEL {
b, err := e.engine.L2BlockRefByLabel(ctx, eth.Finalized) b, err := e.engine.L2BlockRefByLabel(ctx, eth.Finalized)
rollupGenesisIsFinalized := b.Hash == e.rollupCfg.Genesis.L2.Hash rollupGenesisIsFinalized := b.Hash == e.rollupCfg.Genesis.L2.Hash
if errors.Is(err, ethereum.NotFound) || rollupGenesisIsFinalized { if errors.Is(err, ethereum.NotFound) || rollupGenesisIsFinalized || e.syncCfg.SupportsPostFinalizationELSync {
e.syncStatus = syncStatusStartedEL e.syncStatus = syncStatusStartedEL
e.log.Info("Starting EL sync") e.log.Info("Starting EL sync")
e.elStart = e.clock.Now() e.elStart = e.clock.Now()
......
package engine
import "fmt"
// Kind identifies the engine client's kind, used to control the behavior of optimism in different engine clients.
type Kind string
const (
Geth Kind = "geth"
Reth Kind = "reth"
Erigon Kind = "erigon"
)
var Kinds = []Kind{
Geth,
Reth,
Erigon,
}
func (kind Kind) String() string {
return string(kind)
}
func (kind *Kind) Set(value string) error {
if !ValidEngineKind(Kind(value)) {
return fmt.Errorf("unknown engine client kind: %q", value)
}
*kind = Kind(value)
return nil
}
func (kind *Kind) Clone() any {
cpy := *kind
return &cpy
}
func (kind Kind) SupportsPostFinalizationELSync() bool {
switch kind {
case Geth:
return false
case Erigon, Reth:
return true
}
return false
}
func ValidEngineKind(value Kind) bool {
for _, k := range Kinds {
if k == value {
return true
}
}
return false
}
...@@ -70,4 +70,6 @@ type Config struct { ...@@ -70,4 +70,6 @@ type Config struct {
// Note: We probably need to detect the condition that snap sync has not complete when we do a restart prior to running sync-start if we are doing // Note: We probably need to detect the condition that snap sync has not complete when we do a restart prior to running sync-start if we are doing
// snap sync with a genesis finalization data. // snap sync with a genesis finalization data.
SkipSyncStartCheck bool `json:"skip_sync_start_check"` SkipSyncStartCheck bool `json:"skip_sync_start_check"`
SupportsPostFinalizationELSync bool `json:"supports_post_finalization_elsync"`
} }
...@@ -23,6 +23,7 @@ import ( ...@@ -23,6 +23,7 @@ import (
p2pcli "github.com/ethereum-optimism/optimism/op-node/p2p/cli" p2pcli "github.com/ethereum-optimism/optimism/op-node/p2p/cli"
"github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/driver" "github.com/ethereum-optimism/optimism/op-node/rollup/driver"
"github.com/ethereum-optimism/optimism/op-node/rollup/engine"
"github.com/ethereum-optimism/optimism/op-node/rollup/sync" "github.com/ethereum-optimism/optimism/op-node/rollup/sync"
opflags "github.com/ethereum-optimism/optimism/op-service/flags" opflags "github.com/ethereum-optimism/optimism/op-service/flags"
) )
...@@ -269,9 +270,12 @@ func NewSyncConfig(ctx *cli.Context, log log.Logger) (*sync.Config, error) { ...@@ -269,9 +270,12 @@ func NewSyncConfig(ctx *cli.Context, log log.Logger) (*sync.Config, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
engineKind := engine.Kind(ctx.String(flags.L2EngineKind.Name))
cfg := &sync.Config{ cfg := &sync.Config{
SyncMode: mode, SyncMode: mode,
SkipSyncStartCheck: ctx.Bool(flags.SkipSyncStartCheck.Name), SkipSyncStartCheck: ctx.Bool(flags.SkipSyncStartCheck.Name),
SupportsPostFinalizationELSync: engineKind.SupportsPostFinalizationELSync(),
} }
if ctx.Bool(flags.L2EngineSyncEnabled.Name) { if ctx.Bool(flags.L2EngineSyncEnabled.Name) {
cfg.SyncMode = sync.ELSync cfg.SyncMode = sync.ELSync
......
...@@ -39,7 +39,7 @@ func NewDriver(logger log.Logger, cfg *rollup.Config, l1Source derive.L1Fetcher, ...@@ -39,7 +39,7 @@ func NewDriver(logger log.Logger, cfg *rollup.Config, l1Source derive.L1Fetcher,
pipeline := derive.NewDerivationPipeline(logger, cfg, l1Source, l1BlobsSource, plasma.Disabled, l2Source, metrics.NoopMetrics) pipeline := derive.NewDerivationPipeline(logger, cfg, l1Source, l1BlobsSource, plasma.Disabled, l2Source, metrics.NoopMetrics)
pipelineDeriver := derive.NewPipelineDeriver(context.Background(), pipeline, d) pipelineDeriver := derive.NewPipelineDeriver(context.Background(), pipeline, d)
ec := engine.NewEngineController(l2Source, logger, metrics.NoopMetrics, cfg, sync.CLSync, d) ec := engine.NewEngineController(l2Source, logger, metrics.NoopMetrics, cfg, &sync.Config{SyncMode: sync.CLSync}, d)
engineDeriv := engine.NewEngDeriver(logger, context.Background(), cfg, ec, d) engineDeriv := engine.NewEngDeriver(logger, context.Background(), cfg, ec, d)
syncCfg := &sync.Config{SyncMode: sync.CLSync} syncCfg := &sync.Config{SyncMode: sync.CLSync}
engResetDeriv := engine.NewEngineResetDeriver(context.Background(), logger, cfg, l1Source, l2Source, syncCfg, d) engResetDeriv := engine.NewEngineResetDeriver(context.Background(), logger, cfg, l1Source, l2Source, syncCfg, d)
......
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