From eb5636f0a2953a540be024b9e72a46f14bc8e2d4 Mon Sep 17 00:00:00 2001 From: joanestebanr <129153821+joanestebanr@users.noreply.github.com> Date: Thu, 30 May 2024 22:25:34 +0200 Subject: [PATCH 1/2] Add param ExecuteBatchNoCountersFlag to synchronizer --- state/batchV2.go | 16 +++++++++- .../etrog/processor_l1_sequence_batches.go | 17 +++++++++-- .../processor_l1_sequence_batches_test.go | 6 ++-- .../mocks/state_full_interface.go | 29 ++++++++++--------- synchronizer/common/syncinterfaces/state.go | 2 +- synchronizer/config.go | 4 +++ synchronizer/default_l1processors.go | 4 +-- synchronizer/synchronizer.go | 2 +- synchronizer/synchronizer_test.go | 2 +- 9 files changed, 56 insertions(+), 26 deletions(-) diff --git a/state/batchV2.go b/state/batchV2.go index 7674347e6e..b38ef07ec6 100644 --- a/state/batchV2.go +++ b/state/batchV2.go @@ -35,6 +35,7 @@ type ProcessingContextV2 struct { GlobalExitRoot common.Hash // GlobalExitRoot is not use for execute but use to OpenBatch (data on DB) ExecutionMode uint64 ClosingReason ClosingReason + NoCountersFlag bool } // ProcessBatchV2 processes a batch for forkID >= ETROG @@ -98,7 +99,8 @@ func (s *State) ProcessBatchV2(ctx context.Context, request ProcessRequest, upda } // ExecuteBatchV2 is used by the synchronizer to reprocess batches to compare generated state root vs stored one -func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) { +func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]L1DataV2, timestampLimit time.Time, + updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, noCountersFlag bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) { if dbTx == nil { return nil, ErrDBTxNil } @@ -116,6 +118,11 @@ func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, L1InfoTreeRoot updateMT = cTrue } + NoCountersMT := uint32(cFalse) + if noCountersFlag { + log.Warnf("ExecuteBatchV2: NoCountersFlag is true for batch %d", batch.BatchNumber) + NoCountersMT = cTrue + } // Create Batch processBatchRequest := &executor.ProcessBatchRequestV2{ OldBatchNum: batch.BatchNumber - 1, @@ -131,6 +138,7 @@ func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, L1InfoTreeRoot ForkId: forkId, ContextId: uuid.NewString(), SkipVerifyL1InfoRoot: skipVerifyL1InfoRoot, + NoCounters: NoCountersMT, } if forcedBlockHashL1 != nil { @@ -217,6 +225,11 @@ func (s *State) processBatchV2(ctx context.Context, processingCtx *ProcessingCon } else { timestampLimitUnix = uint64(time.Now().Unix()) } + NoCountersMT := uint32(cFalse) + if processingCtx.NoCountersFlag { + log.Warnf("processBatchV2: NoCountersFlag is true for batch %d", lastBatch.BatchNumber) + NoCountersMT = cTrue + } // Create Batch processBatchRequest := &executor.ProcessBatchRequestV2{ OldBatchNum: lastBatch.BatchNumber - 1, @@ -231,6 +244,7 @@ func (s *State) processBatchV2(ctx context.Context, processingCtx *ProcessingCon ContextId: uuid.NewString(), SkipVerifyL1InfoRoot: processingCtx.SkipVerifyL1InfoRoot, L1InfoRoot: processingCtx.L1InfoRoot.Bytes(), + NoCounters: NoCountersMT, } if processingCtx.ForcedBlockHashL1 != nil { diff --git a/synchronizer/actions/etrog/processor_l1_sequence_batches.go b/synchronizer/actions/etrog/processor_l1_sequence_batches.go index 2f91d8a9d9..2812a2487b 100644 --- a/synchronizer/actions/etrog/processor_l1_sequence_batches.go +++ b/synchronizer/actions/etrog/processor_l1_sequence_batches.go @@ -25,7 +25,7 @@ type stateProcessSequenceBatches interface { GetNextForcedBatches(ctx context.Context, nextForcedBatches int, dbTx pgx.Tx) ([]state.ForcedBatch, error) GetBatchByNumber(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*state.Batch, error) ProcessAndStoreClosedBatchV2(ctx context.Context, processingCtx state.ProcessingContextV2, dbTx pgx.Tx, caller metrics.CallerLabel) (common.Hash, uint64, string, error) - ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) + ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, noCountersFlag bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) AddAccumulatedInputHash(ctx context.Context, batchNum uint64, accInputHash common.Hash, dbTx pgx.Tx) error ResetTrustedState(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) error AddSequence(ctx context.Context, sequence state.Sequence, dbTx pgx.Tx) error @@ -41,6 +41,11 @@ type syncProcessSequenceBatchesInterface interface { CleanTrustedState() } +// ProcessorConfig configuration for ProcessorL1SequenceBatchesEtrog +type ProcessorConfig struct { + NoCounters bool +} + // ProcessorL1SequenceBatchesEtrog implements L1EventProcessor type ProcessorL1SequenceBatchesEtrog struct { actions.ProcessorBase[ProcessorL1SequenceBatchesEtrog] @@ -48,13 +53,15 @@ type ProcessorL1SequenceBatchesEtrog struct { sync syncProcessSequenceBatchesInterface timeProvider syncCommon.TimeProvider halter syncinterfaces.CriticalErrorHandler + cfg ProcessorConfig } // NewProcessorL1SequenceBatches returns instance of a processor for SequenceBatchesOrder func NewProcessorL1SequenceBatches(state stateProcessSequenceBatches, sync syncProcessSequenceBatchesInterface, timeProvider syncCommon.TimeProvider, - halter syncinterfaces.CriticalErrorHandler) *ProcessorL1SequenceBatchesEtrog { + halter syncinterfaces.CriticalErrorHandler, + cfg ProcessorConfig) *ProcessorL1SequenceBatchesEtrog { return &ProcessorL1SequenceBatchesEtrog{ ProcessorBase: actions.ProcessorBase[ProcessorL1SequenceBatchesEtrog]{ SupportedEvent: []etherman.EventOrder{etherman.SequenceBatchesOrder, etherman.InitialSequenceBatchesOrder}, @@ -63,6 +70,7 @@ func NewProcessorL1SequenceBatches(state stateProcessSequenceBatches, sync: sync, timeProvider: timeProvider, halter: halter, + cfg: cfg, } } @@ -158,6 +166,7 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con ForcedBlockHashL1: forcedBlockHashL1, SkipVerifyL1InfoRoot: 1, ClosingReason: state.SyncL1EventSequencedForcedBatchClosingReason, + NoCountersFlag: p.cfg.NoCounters, } } else if sbatch.PolygonRollupBaseEtrogBatchData.ForcedTimestamp > 0 && sbatch.BatchNumber == 1 { // This is the initial batch (injected) log.Debug("Processing initial batch") @@ -175,6 +184,7 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con ForcedBlockHashL1: forcedBlockHashL1, SkipVerifyL1InfoRoot: 1, ClosingReason: state.SyncL1EventInitialBatchClosingReason, + NoCountersFlag: p.cfg.NoCounters, } } else { var maxGER common.Hash @@ -200,6 +210,7 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con SkipVerifyL1InfoRoot: 1, GlobalExitRoot: batch.GlobalExitRoot, ClosingReason: state.SyncL1EventSequencedBatchClosingReason, + NoCountersFlag: p.cfg.NoCounters, } if batch.GlobalExitRoot == (common.Hash{}) { if len(leaves) > 0 { @@ -264,7 +275,7 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con } } // Reprocess batch to compare the stateRoot with tBatch.StateRoot and get accInputHash - batchRespose, err := p.state.ExecuteBatchV2(ctx, batch, processCtx.L1InfoRoot, leaves, *processCtx.Timestamp, false, processCtx.SkipVerifyL1InfoRoot, processCtx.ForcedBlockHashL1, dbTx) + batchRespose, err := p.state.ExecuteBatchV2(ctx, batch, processCtx.L1InfoRoot, leaves, *processCtx.Timestamp, false, processCtx.SkipVerifyL1InfoRoot, processCtx.ForcedBlockHashL1, p.cfg.NoCounters, dbTx) if err != nil { log.Errorf("error executing L1 batch: %+v, error: %v", batch, err) rollbackErr := dbTx.Rollback(ctx) diff --git a/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go b/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go index 3b30db1c50..6aa2361d6d 100644 --- a/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go +++ b/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go @@ -54,7 +54,7 @@ func createMocks(t *testing.T) *mocksEtrogProcessorL1 { func createSUT(mocks *mocksEtrogProcessorL1) *ProcessorL1SequenceBatchesEtrog { return NewProcessorL1SequenceBatches(mocks.State, mocks.Synchronizer, - mocks.TimeProvider, mocks.CriticalErrorHandler) + mocks.TimeProvider, mocks.CriticalErrorHandler, ProcessorConfig{}) } func TestL1SequenceBatchesNoData(t *testing.T) { @@ -175,7 +175,7 @@ func TestL1SequenceBatchesTrustedBatchSequencedThatAlreadyExistsMismatch(t *test mocks := createMocks(t) CriticalErrorHandlerPanic := CriticalErrorHandlerPanic{} sut := NewProcessorL1SequenceBatches(mocks.State, mocks.Synchronizer, - mocks.TimeProvider, CriticalErrorHandlerPanic) + mocks.TimeProvider, CriticalErrorHandlerPanic, ProcessorConfig{}) ctx := context.Background() batch := newStateBatch(3) l1InfoRoot := common.HexToHash(hashExamplesValues[0]) @@ -248,7 +248,7 @@ func expectationsProcessAndStoreClosedBatchV2(t *testing.T, mocks *mocksEtrogPro func expectationsForExecution(t *testing.T, mocks *mocksEtrogProcessorL1, ctx context.Context, sequencedBatch etherman.SequencedBatch, timestampLimit time.Time, response *executor.ProcessBatchResponseV2) { mocks.State.EXPECT().ExecuteBatchV2(ctx, mock.Anything, *sequencedBatch.L1InfoRoot, mock.Anything, timestampLimit, false, - uint32(1), (*common.Hash)(nil), mocks.DbTx).Return(response, nil) + uint32(1), (*common.Hash)(nil), false, mocks.DbTx).Return(response, nil) } func newProcessBatchResponseV2(batch *state.Batch) *executor.ProcessBatchResponseV2 { diff --git a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go index 248c817fa6..3e96086cc0 100644 --- a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go +++ b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go @@ -695,9 +695,9 @@ func (_c *StateFullInterface_ExecuteBatch_Call) RunAndReturn(run func(context.Co return _c } -// ExecuteBatchV2 provides a mock function with given fields: ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx -func (_m *StateFullInterface) ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) { - ret := _m.Called(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx) +// ExecuteBatchV2 provides a mock function with given fields: ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx +func (_m *StateFullInterface) ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, noCountersFlag bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) { + ret := _m.Called(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx) if len(ret) == 0 { panic("no return value specified for ExecuteBatchV2") @@ -705,19 +705,19 @@ func (_m *StateFullInterface) ExecuteBatchV2(ctx context.Context, batch state.Ba var r0 *executor.ProcessBatchResponseV2 var r1 error - if rf, ok := ret.Get(0).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, pgx.Tx) (*executor.ProcessBatchResponseV2, error)); ok { - return rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx) + if rf, ok := ret.Get(0).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, bool, pgx.Tx) (*executor.ProcessBatchResponseV2, error)); ok { + return rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx) } - if rf, ok := ret.Get(0).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, pgx.Tx) *executor.ProcessBatchResponseV2); ok { - r0 = rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx) + if rf, ok := ret.Get(0).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, bool, pgx.Tx) *executor.ProcessBatchResponseV2); ok { + r0 = rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(*executor.ProcessBatchResponseV2) } } - if rf, ok := ret.Get(1).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, pgx.Tx) error); ok { - r1 = rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx) + if rf, ok := ret.Get(1).(func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, bool, pgx.Tx) error); ok { + r1 = rf(ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx) } else { r1 = ret.Error(1) } @@ -739,14 +739,15 @@ type StateFullInterface_ExecuteBatchV2_Call struct { // - updateMerkleTree bool // - skipVerifyL1InfoRoot uint32 // - forcedBlockHashL1 *common.Hash +// - noCountersFlag bool // - dbTx pgx.Tx -func (_e *StateFullInterface_Expecter) ExecuteBatchV2(ctx interface{}, batch interface{}, L1InfoTreeRoot interface{}, l1InfoTreeData interface{}, timestampLimit interface{}, updateMerkleTree interface{}, skipVerifyL1InfoRoot interface{}, forcedBlockHashL1 interface{}, dbTx interface{}) *StateFullInterface_ExecuteBatchV2_Call { - return &StateFullInterface_ExecuteBatchV2_Call{Call: _e.mock.On("ExecuteBatchV2", ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, dbTx)} +func (_e *StateFullInterface_Expecter) ExecuteBatchV2(ctx interface{}, batch interface{}, L1InfoTreeRoot interface{}, l1InfoTreeData interface{}, timestampLimit interface{}, updateMerkleTree interface{}, skipVerifyL1InfoRoot interface{}, forcedBlockHashL1 interface{}, noCountersFlag interface{}, dbTx interface{}) *StateFullInterface_ExecuteBatchV2_Call { + return &StateFullInterface_ExecuteBatchV2_Call{Call: _e.mock.On("ExecuteBatchV2", ctx, batch, L1InfoTreeRoot, l1InfoTreeData, timestampLimit, updateMerkleTree, skipVerifyL1InfoRoot, forcedBlockHashL1, noCountersFlag, dbTx)} } -func (_c *StateFullInterface_ExecuteBatchV2_Call) Run(run func(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, dbTx pgx.Tx)) *StateFullInterface_ExecuteBatchV2_Call { +func (_c *StateFullInterface_ExecuteBatchV2_Call) Run(run func(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, noCountersFlag bool, dbTx pgx.Tx)) *StateFullInterface_ExecuteBatchV2_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(state.Batch), args[2].(common.Hash), args[3].(map[uint32]state.L1DataV2), args[4].(time.Time), args[5].(bool), args[6].(uint32), args[7].(*common.Hash), args[8].(pgx.Tx)) + run(args[0].(context.Context), args[1].(state.Batch), args[2].(common.Hash), args[3].(map[uint32]state.L1DataV2), args[4].(time.Time), args[5].(bool), args[6].(uint32), args[7].(*common.Hash), args[8].(bool), args[9].(pgx.Tx)) }) return _c } @@ -756,7 +757,7 @@ func (_c *StateFullInterface_ExecuteBatchV2_Call) Return(_a0 *executor.ProcessBa return _c } -func (_c *StateFullInterface_ExecuteBatchV2_Call) RunAndReturn(run func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, pgx.Tx) (*executor.ProcessBatchResponseV2, error)) *StateFullInterface_ExecuteBatchV2_Call { +func (_c *StateFullInterface_ExecuteBatchV2_Call) RunAndReturn(run func(context.Context, state.Batch, common.Hash, map[uint32]state.L1DataV2, time.Time, bool, uint32, *common.Hash, bool, pgx.Tx) (*executor.ProcessBatchResponseV2, error)) *StateFullInterface_ExecuteBatchV2_Call { _c.Call.Return(run) return _c } diff --git a/synchronizer/common/syncinterfaces/state.go b/synchronizer/common/syncinterfaces/state.go index 41d7acbedc..efb44772b1 100644 --- a/synchronizer/common/syncinterfaces/state.go +++ b/synchronizer/common/syncinterfaces/state.go @@ -48,7 +48,7 @@ type StateFullInterface interface { StoreTransaction(ctx context.Context, batchNumber uint64, processedTx *state.ProcessTransactionResponse, coinbase common.Address, timestamp uint64, egpLog *state.EffectiveGasPriceLog, globalExitRoot, blockInfoRoot common.Hash, dbTx pgx.Tx) (*state.L2Header, error) GetStateRootByBatchNumber(ctx context.Context, batchNum uint64, dbTx pgx.Tx) (common.Hash, error) ExecuteBatch(ctx context.Context, batch state.Batch, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponse, error) - ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) + ExecuteBatchV2(ctx context.Context, batch state.Batch, L1InfoTreeRoot common.Hash, l1InfoTreeData map[uint32]state.L1DataV2, timestampLimit time.Time, updateMerkleTree bool, skipVerifyL1InfoRoot uint32, forcedBlockHashL1 *common.Hash, noCountersFlag bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) GetLastVerifiedBatch(ctx context.Context, dbTx pgx.Tx) (*state.VerifiedBatch, error) GetLastVirtualBatchNum(ctx context.Context, dbTx pgx.Tx) (uint64, error) AddSequence(ctx context.Context, sequence state.Sequence, dbTx pgx.Tx) error diff --git a/synchronizer/config.go b/synchronizer/config.go index 4ca64414f7..e69312e70a 100644 --- a/synchronizer/config.go +++ b/synchronizer/config.go @@ -33,6 +33,10 @@ type Config struct { L1ParallelSynchronization L1ParallelSynchronizationConfig // L2Synchronization Configuration for L2 synchronization L2Synchronization l2_sync.Config `mapstructure:"L2Synchronization"` + + // ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set + // this a very dangerous option, if you are not sure set to FALSE + ExecuteBatchNoCountersFlag bool `mapstructure:"ExecuteBatchWithNoCounters"` } // L1BlockCheckConfig Configuration for L1 Block Checker diff --git a/synchronizer/default_l1processors.go b/synchronizer/default_l1processors.go index 671d28cea7..1360f348d8 100644 --- a/synchronizer/default_l1processors.go +++ b/synchronizer/default_l1processors.go @@ -9,7 +9,7 @@ import ( "github.com/0xPolygonHermez/zkevm-node/synchronizer/common" ) -func defaultsL1EventProcessors(sync *ClientSynchronizer, l2Blockchecker *actions.CheckL2BlockHash) *processor_manager.L1EventProcessors { +func defaultsL1EventProcessors(sync *ClientSynchronizer, l2Blockchecker *actions.CheckL2BlockHash, cfg Config) *processor_manager.L1EventProcessors { p := processor_manager.NewL1EventProcessorsBuilder() p.Register(incaberry.NewProcessorL1GlobalExitRoot(sync.state)) p.Register(actions.NewCheckL2BlockDecorator(incaberry.NewProcessorL1SequenceBatches(sync.state, sync.etherMan, sync.pool, sync.eventLog, sync), l2Blockchecker)) @@ -17,7 +17,7 @@ func defaultsL1EventProcessors(sync *ClientSynchronizer, l2Blockchecker *actions p.Register(actions.NewCheckL2BlockDecorator(incaberry.NewProcessL1SequenceForcedBatches(sync.state, sync), l2Blockchecker)) p.Register(incaberry.NewProcessorForkId(sync.state, sync)) p.Register(etrog.NewProcessorL1InfoTreeUpdate(sync.state)) - sequenceBatchesProcessor := etrog.NewProcessorL1SequenceBatches(sync.state, sync, common.DefaultTimeProvider{}, sync.halter) + sequenceBatchesProcessor := etrog.NewProcessorL1SequenceBatches(sync.state, sync, common.DefaultTimeProvider{}, sync.halter, etrog.ProcessorConfig{NoCounters: cfg.ExecuteBatchNoCountersFlag}) p.Register(actions.NewCheckL2BlockDecorator(sequenceBatchesProcessor, l2Blockchecker)) p.Register(incaberry.NewProcessorL1VerifyBatch(sync.state)) p.Register(etrog.NewProcessorL1UpdateEtrogSequence(sync.state, sync, common.DefaultTimeProvider{})) diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 75c3c3cefc..d6b26ab714 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -196,7 +196,7 @@ func NewSynchronizer( } } - res.l1EventProcessors = defaultsL1EventProcessors(res, l1checkerL2Blocks) + res.l1EventProcessors = defaultsL1EventProcessors(res, l1checkerL2Blocks, cfg) switch cfg.L1SynchronizationMode { case ParallelMode: log.Info("L1SynchronizationMode is parallel") diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ce0c27e280..7e494a7f3b 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -303,7 +303,7 @@ func TestForcedBatchEtrog(t *testing.T) { NewAccInputHash: common.Hash{}.Bytes(), NewLocalExitRoot: common.Hash{}.Bytes(), } - m.State.EXPECT().ExecuteBatchV2(ctx, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything). + m.State.EXPECT().ExecuteBatchV2(ctx, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, false, mock.Anything). Return(&executionResponse, nil). Times(1) From 7667202a5698838613244293b2bd5ddc9d4b0970 Mon Sep 17 00:00:00 2001 From: joanestebanr <129153821+joanestebanr@users.noreply.github.com> Date: Thu, 30 May 2024 22:32:42 +0200 Subject: [PATCH 2/2] fix documentation --- docs/config-file/node-config-doc.html | 2 +- docs/config-file/node-config-doc.md | 16 ++++++++++++++++ docs/config-file/node-config-schema.json | 5 +++++ synchronizer/config.go | 2 +- 4 files changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 35d4551dce..10222b5a46 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -28,7 +28,7 @@
"300ms"
RollupInfoRetriesSpacing is the minimum time between retries to request rollup info (it will sleep for fulfill this time) to avoid spamming L1
"1m"
"300ms"
-
FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized
If enabled then the L2 sync process is permitted (only for permissionless)
AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.
ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again
CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash
DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool
DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting
"1m"
+
FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized
If enabled then the L2 sync process is permitted (only for permissionless)
AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.
ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again
CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash
ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set
this a very dangerous option, if you are not sure set to FALSE
DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool
DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting
"1m"
"300ms"
TxLifetimeCheckInterval is the time the sequencer waits to check txs lifetime
"1m"
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index cff56349af..99401d4487 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -1362,6 +1362,7 @@ because depending of this values is going to ask to a trusted node for trusted t
| - [L1SynchronizationMode](#Synchronizer_L1SynchronizationMode ) | No | enum (of string) | No | - | L1SynchronizationMode define how to synchronize with L1:
- parallel: Request data to L1 in parallel, and process sequentially. The advantage is that executor is not blocked waiting for L1 data
- sequential: Request data to L1 and execute |
| - [L1ParallelSynchronization](#Synchronizer_L1ParallelSynchronization ) | No | object | No | - | L1ParallelSynchronization Configuration for parallel mode (if L1SynchronizationMode equal to 'parallel') |
| - [L2Synchronization](#Synchronizer_L2Synchronization ) | No | object | No | - | L2Synchronization Configuration for L2 synchronization |
+| - [ExecuteBatchNoCountersFlag](#Synchronizer_ExecuteBatchNoCountersFlag ) | No | boolean | No | - | ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set
this a very dangerous option, if you are not sure set to FALSE |
### 9.1. `Synchronizer.SyncInterval`
@@ -1938,6 +1939,21 @@ ReprocessFullBatchOnClose=false
CheckLastL2BlockHashOnCloseBatch=true
```
+### 9.11. `Synchronizer.ExecuteBatchNoCountersFlag`
+
+**Type:** : `boolean`
+
+**Default:** `false`
+
+**Description:** ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set
+this a very dangerous option, if you are not sure set to FALSE
+
+**Example setting the default value** (false):
+```
+[Synchronizer]
+ExecuteBatchNoCountersFlag=false
+```
+
## 10. `[Sequencer]`
**Type:** : `object`
diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json
index d70f2d44fc..25c108454f 100644
--- a/docs/config-file/node-config-schema.json
+++ b/docs/config-file/node-config-schema.json
@@ -722,6 +722,11 @@
"additionalProperties": false,
"type": "object",
"description": "L2Synchronization Configuration for L2 synchronization"
+ },
+ "ExecuteBatchNoCountersFlag": {
+ "type": "boolean",
+ "description": "ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set\nthis a very dangerous option, if you are not sure set to FALSE",
+ "default": false
}
},
"additionalProperties": false,
diff --git a/synchronizer/config.go b/synchronizer/config.go
index e69312e70a..ffe15e36e7 100644
--- a/synchronizer/config.go
+++ b/synchronizer/config.go
@@ -36,7 +36,7 @@ type Config struct {
// ExecuteBatchNoCountersFlag if is true then the executor will execute the batch with the flag NoCounters set
// this a very dangerous option, if you are not sure set to FALSE
- ExecuteBatchNoCountersFlag bool `mapstructure:"ExecuteBatchWithNoCounters"`
+ ExecuteBatchNoCountersFlag bool `mapstructure:"ExecuteBatchNoCountersFlag"`
}
// L1BlockCheckConfig Configuration for L1 Block Checker