Skip to content

Commit

Permalink
add selector for trusted sync to prevent from executing from an old t…
Browse files Browse the repository at this point in the history
…rusted node (#3336)

* add selector for trusted sync to check that batches  correspond to the supported forkids
  • Loading branch information
joanestebanr committed Feb 21, 2024
1 parent aaad8d7 commit d837dac
Show file tree
Hide file tree
Showing 8 changed files with 217 additions and 9 deletions.
9 changes: 2 additions & 7 deletions etherman/etherman_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,9 +123,6 @@ func TestForcedBatchEvent(t *testing.T) {
assert.Equal(t, auth.From, blocks[0].ForcedBatches[0].Sequencer)
}

// TODO: Review tests with Joan

/*
func TestSequencedBatchesEvent(t *testing.T) {
// Set up testing environment
etherman, ethBackend, auth, _, br := newTestingEnv()
Expand Down Expand Up @@ -165,8 +162,7 @@ func TestSequencedBatchesEvent(t *testing.T) {
}, polygonzkevm.PolygonRollupBaseEtrogBatchData{
Transactions: common.Hex2Bytes(rawTxs),
})
// TODO: Fix params
_, err = etherman.ZkEVM.SequenceBatches(auth, sequences, 0, 0, auth.From)
_, err = etherman.ZkEVM.SequenceBatches(auth, sequences, uint64(time.Now().Unix()), uint64(1), auth.From)
require.NoError(t, err)

// Mine the tx in a block
Expand Down Expand Up @@ -205,7 +201,7 @@ func TestVerifyBatchEvent(t *testing.T) {
Transactions: common.Hex2Bytes(rawTxs),
}
//TODO: Fix params
_, err = etherman.ZkEVM.SequenceBatches(auth, []polygonzkevm.PolygonRollupBaseEtrogBatchData{tx}, 0, 0, auth.From)
_, err = etherman.ZkEVM.SequenceBatches(auth, []polygonzkevm.PolygonRollupBaseEtrogBatchData{tx}, uint64(time.Now().Unix()), uint64(1), auth.From)
require.NoError(t, err)

// Mine the tx in a block
Expand Down Expand Up @@ -233,7 +229,6 @@ func TestVerifyBatchEvent(t *testing.T) {
assert.Equal(t, 0, order[blocks[1].BlockHash][0].Pos)
assert.Equal(t, 0, order[blocks[1].BlockHash][1].Pos)
}
*/

func TestSequenceForceBatchesEvent(t *testing.T) {
// Set up testing environment
Expand Down
2 changes: 2 additions & 0 deletions state/forkid.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@ const (
FORKID_INCABERRY = 6
// FORKID_ETROG is the fork id 7
FORKID_ETROG = 7
// FORKID_ELDERBERRY is the fork id 8
FORKID_ELDERBERRY = 8
)

// ForkIDInterval is a fork id interval
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ func (g *ProcessorL1SequenceBatchesElderberry) sanityCheckTstampLastL2Block(time
return nil
}
lastL2Block := l2blocks[len(l2blocks)-1]
if uint64(lastL2Block.ReceivedAt.Unix()) <= timeLimit {
if uint64(lastL2Block.ReceivedAt.Unix()) > timeLimit {
log.Errorf("The last L2 block timestamp can't be greater than timeLimit. Expected: %d (L1 event), got: %d (last L2Block)", timeLimit, lastL2Block.ReceivedAt.Unix())
return fmt.Errorf("wrong timestamp of last L2 block timestamp with L1 event timestamp")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@ var (
ErrMissingSyncFromL1 = errors.New("must sync from L1")
// ErrFatalDesyncFromL1 is returned when trusted node and permissionless node have different data
ErrFatalDesyncFromL1 = errors.New("fatal situation: the TrustedNode have another data!. Halt or do something")
// ErrCantSyncFromL2 is returned when can't sync from L2, for example the forkid is not supported by L2 sync
ErrCantSyncFromL2 = errors.New("can't sync from L2")
)

// SyncTrustedStateExecutor is the interface that class that synchronize permissionless with a trusted node
Expand Down
84 changes: 84 additions & 0 deletions synchronizer/l2_sync/l2_shared/processor_trusted_batch_selector.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
package l2_shared

/*
This class is a implementation of SyncTrustedStateExecutor that selects the executor to use.
It have a map with the forkID and the executor class to use, if none is available skip trusted sync returning a nil
*/

import (
"context"

"github.com/0xPolygonHermez/zkevm-node/log"
"github.com/0xPolygonHermez/zkevm-node/state"
"github.com/0xPolygonHermez/zkevm-node/synchronizer/common/syncinterfaces"
)

type stateSyncTrustedStateExecutorSelector interface {
GetForkIDInMemory(forkId uint64) *state.ForkIDInterval
GetForkIDByBatchNumber(batchNumber uint64) uint64
}

// SyncTrustedStateExecutorSelector Implements SyncTrustedStateExecutor
type SyncTrustedStateExecutorSelector struct {
state stateSyncTrustedStateExecutorSelector
supportedForks map[uint64]syncinterfaces.SyncTrustedStateExecutor
}

// NewSyncTrustedStateExecutorSelector creates a new SyncTrustedStateExecutorSelector that implements SyncTrustedStateExecutor
func NewSyncTrustedStateExecutorSelector(
supportedForks map[uint64]syncinterfaces.SyncTrustedStateExecutor,
state stateSyncTrustedStateExecutorSelector) *SyncTrustedStateExecutorSelector {
return &SyncTrustedStateExecutorSelector{
supportedForks: supportedForks,
state: state,
}
}

// GetExecutor returns the executor that should be used for the given batch, could be nil
// it returns the executor and the maximum batch number that the executor can process
func (s *SyncTrustedStateExecutorSelector) GetExecutor(latestSyncedBatch uint64, maximumBatchNumberToProcess uint64) (syncinterfaces.SyncTrustedStateExecutor, uint64) {
forkIDForNextBatch := s.state.GetForkIDByBatchNumber(latestSyncedBatch + 1)
executor, ok := s.supportedForks[forkIDForNextBatch]
if !ok {
log.Warnf("No supported sync from Trusted Node for forkID %d", forkIDForNextBatch)
return nil, 0
}
fork := s.state.GetForkIDInMemory(forkIDForNextBatch)
if fork == nil {
log.Errorf("ForkID %d range not available! that is UB", forkIDForNextBatch)
return nil, 0
}

maxCapped := min(maximumBatchNumberToProcess, fork.ToBatchNumber)
log.Debugf("using ForkID %d, lastBatch:%d (maxBatch original:%d capped:%d)", forkIDForNextBatch,
latestSyncedBatch, maximumBatchNumberToProcess, maxCapped)
return executor, maxCapped
}

// SyncTrustedState syncs the trusted state with the permissionless state. In this case
// choose which executor must use
func (s *SyncTrustedStateExecutorSelector) SyncTrustedState(ctx context.Context, latestSyncedBatch uint64, maximumBatchNumberToProcess uint64) error {
executor, maxBatchNumber := s.GetExecutor(latestSyncedBatch, maximumBatchNumberToProcess)
if executor == nil {
log.Warnf("No executor available, skipping SyncTrustedState: latestSyncedBatch:%d, maximumBatchNumberToProcess:%d",
latestSyncedBatch, maximumBatchNumberToProcess)
return syncinterfaces.ErrCantSyncFromL2
}
return executor.SyncTrustedState(ctx, latestSyncedBatch, maxBatchNumber)
}

// CleanTrustedState clean cache of Batches and StateRoot
func (s *SyncTrustedStateExecutorSelector) CleanTrustedState() {
for _, executor := range s.supportedForks {
executor.CleanTrustedState()
}
}

// GetCachedBatch implements syncinterfaces.SyncTrustedStateExecutor. Returns a cached batch
func (s *SyncTrustedStateExecutorSelector) GetCachedBatch(batchNumber uint64) *state.Batch {
executor, _ := s.GetExecutor(batchNumber, 0)
if executor == nil {
return nil
}
return executor.GetCachedBatch(min(batchNumber))
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
package test_l2_shared

import (
"context"
"testing"

"github.com/0xPolygonHermez/zkevm-node/state"
"github.com/0xPolygonHermez/zkevm-node/synchronizer/common/syncinterfaces"
mock_syncinterfaces "github.com/0xPolygonHermez/zkevm-node/synchronizer/common/syncinterfaces/mocks"
"github.com/0xPolygonHermez/zkevm-node/synchronizer/l2_sync/l2_shared"
"github.com/stretchr/testify/require"
)

// Use case 1:
// - Running incaberry mode no forkid7 yet
// expected:
// -

func TestExecutorSelectorFirstConfiguredExecutor(t *testing.T) {
mockIncaberry := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mock1Etrog := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mockState := mock_syncinterfaces.NewStateFullInterface(t)
mockState.EXPECT().GetForkIDByBatchNumber(uint64(1 + 1)).Return(uint64(6))
forkIdInterval := state.ForkIDInterval{
FromBatchNumber: 0,
ToBatchNumber: ^uint64(0),
}
mockState.EXPECT().GetForkIDInMemory(uint64(6)).Return(&forkIdInterval)
sut := l2_shared.NewSyncTrustedStateExecutorSelector(map[uint64]syncinterfaces.SyncTrustedStateExecutor{
uint64(6): mockIncaberry,
uint64(7): mock1Etrog,
}, mockState)

executor, maxBatch := sut.GetExecutor(1, 200)
require.Equal(t, mockIncaberry, executor)
require.Equal(t, uint64(200), maxBatch)
}

func TestExecutorSelectorFirstExecutorCapped(t *testing.T) {
mockIncaberry := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mock1Etrog := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mockState := mock_syncinterfaces.NewStateFullInterface(t)
interval := state.ForkIDInterval{
FromBatchNumber: 1,
ToBatchNumber: 99,
ForkId: 6,
}
mockState.EXPECT().GetForkIDByBatchNumber(uint64(1 + 1)).Return(uint64(6))
mockState.EXPECT().GetForkIDInMemory(uint64(6)).Return(&interval)
sut := l2_shared.NewSyncTrustedStateExecutorSelector(map[uint64]syncinterfaces.SyncTrustedStateExecutor{
uint64(6): mockIncaberry,
uint64(7): mock1Etrog,
}, mockState)

executor, maxBatch := sut.GetExecutor(1, 200)
require.Equal(t, mockIncaberry, executor)
require.Equal(t, uint64(99), maxBatch)
}

func TestExecutorSelectorEtrogBatchForkId7(t *testing.T) {
mockIncaberry := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mock1Etrog := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mockState := mock_syncinterfaces.NewStateFullInterface(t)
interval := state.ForkIDInterval{
FromBatchNumber: 100,
ToBatchNumber: 300,
ForkId: 7,
}
mockState.EXPECT().GetForkIDByBatchNumber(uint64(100 + 1)).Return(uint64(7))
mockState.EXPECT().GetForkIDInMemory(uint64(7)).Return(&interval)
sut := l2_shared.NewSyncTrustedStateExecutorSelector(map[uint64]syncinterfaces.SyncTrustedStateExecutor{
uint64(6): mockIncaberry,
uint64(7): mock1Etrog,
}, mockState)

executor, maxBatch := sut.GetExecutor(100, 200)
require.Equal(t, mockIncaberry, executor)
require.Equal(t, uint64(200), maxBatch)
}

func TestUnsupportedForkId(t *testing.T) {
mockIncaberry := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mock1Etrog := mock_syncinterfaces.NewSyncTrustedStateExecutor(t)
mockState := mock_syncinterfaces.NewStateFullInterface(t)

mockState.EXPECT().GetForkIDByBatchNumber(uint64(100 + 1)).Return(uint64(8))

sut := l2_shared.NewSyncTrustedStateExecutorSelector(map[uint64]syncinterfaces.SyncTrustedStateExecutor{
uint64(6): mockIncaberry,
uint64(7): mock1Etrog,
}, mockState)

executor, _ := sut.GetExecutor(100, 200)
require.Equal(t, nil, executor)

err := sut.SyncTrustedState(context.Background(), 100, 200)
require.ErrorIs(t, err, syncinterfaces.ErrCantSyncFromL2)

}
8 changes: 7 additions & 1 deletion synchronizer/synchronizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,11 @@ func NewSynchronizer(
syncTrustedStateEtrog := l2_sync_etrog.NewSyncTrustedBatchExecutorForEtrog(res.zkEVMClient, res.state, res.state, res,
syncCommon.DefaultTimeProvider{}, L1SyncChecker)

res.syncTrustedStateExecutor = syncTrustedStateEtrog
res.syncTrustedStateExecutor = l2_shared.NewSyncTrustedStateExecutorSelector(map[uint64]syncinterfaces.SyncTrustedStateExecutor{
uint64(state.FORKID_ETROG): syncTrustedStateEtrog,
uint64(state.FORKID_ELDERBERRY): syncTrustedStateEtrog,
}, res.state)

res.l1EventProcessors = defaultsL1EventProcessors(res)
switch cfg.L1SynchronizationMode {
case ParallelMode:
Expand Down Expand Up @@ -383,6 +387,8 @@ func (s *ClientSynchronizer) Sync() error {
}
} else if errors.Is(err, syncinterfaces.ErrMissingSyncFromL1) {
log.Info("Syncing from trusted node need data from L1")
} else if errors.Is(err, syncinterfaces.ErrCantSyncFromL2) {
log.Info("Can't sync from L2, going to sync from L1")
} else {
// We break for resync from Trusted
log.Debug("Sleeping for 1 second to avoid respawn too fast, error: ", err)
Expand Down
20 changes: 20 additions & 0 deletions synchronizer/synchronizer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,13 @@ func TestGivenPermissionlessNodeWhenSyncronizeAgainSameBatchThenUseTheOneInMemor
batch10With3Tx := createBatch(t, lastBatchNumber, 3, ETROG_MODE_FLAG)
previousBatch09 := createBatch(t, lastBatchNumber-1, 1, ETROG_MODE_FLAG)

forkIdInterval := state.ForkIDInterval{
FromBatchNumber: 0,
ToBatchNumber: ^uint64(0),
}
m.State.EXPECT().GetForkIDInMemory(uint64(7)).Return(&forkIdInterval)
m.State.EXPECT().GetForkIDByBatchNumber(lastBatchNumber + 1).Return(uint64(7))

expectedCallsForsyncTrustedState(t, m, sync, nil, batch10With2Tx, previousBatch09, RETRIEVE_BATCH_FROM_DB_FLAG, ETROG_MODE_FLAG)
// Is the first time that appears this batch, so it need to OpenBatch
expectedCallsForOpenBatch(t, m, sync, lastBatchNumber)
Expand Down Expand Up @@ -90,6 +97,13 @@ func TestGivenPermissionlessNodeWhenSyncronizeFirstTimeABatchThenStoreItInALocal
batch10With2Tx := createBatch(t, lastBatchNumber, 2, ETROG_MODE_FLAG)
previousBatch09 := createBatch(t, lastBatchNumber-1, 1, ETROG_MODE_FLAG)

forkIdInterval := state.ForkIDInterval{
FromBatchNumber: 0,
ToBatchNumber: ^uint64(0),
}
m.State.EXPECT().GetForkIDInMemory(uint64(7)).Return(&forkIdInterval)
m.State.EXPECT().GetForkIDByBatchNumber(lastBatchNumber + 1).Return(uint64(7))

// This is a incremental process, permissionless have batch10With1Tx and we add a new block
// but the cache doesnt have this information so it need to get from db
expectedCallsForsyncTrustedState(t, m, sync, batch10With1Tx, batch10With2Tx, previousBatch09, RETRIEVE_BATCH_FROM_DB_FLAG, ETROG_MODE_FLAG)
Expand Down Expand Up @@ -127,6 +141,12 @@ func TestForcedBatchEtrog(t *testing.T) {

// state preparation
ctxMatchBy := mock.MatchedBy(func(ctx context.Context) bool { return ctx != nil })
forkIdInterval := state.ForkIDInterval{
FromBatchNumber: 0,
ToBatchNumber: ^uint64(0),
}
m.State.EXPECT().GetForkIDInMemory(uint64(7)).Return(&forkIdInterval)

m.State.
On("BeginStateTransaction", ctxMatchBy).
Run(func(args mock.Arguments) {
Expand Down

0 comments on commit d837dac

Please sign in to comment.