From 3ce6ff878b60e6c119925cd78ddb62dd49b09080 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Tue, 30 Apr 2024 17:22:13 +0200 Subject: [PATCH 01/22] Update DS Format (#3608) * protobuf datastream --- Makefile | 1 + .../src/proto/datastream/v1/datastream.proto | 64 ++ sequencer/batch.go | 6 + sequencer/datastreamer.go | 27 +- sequencer/finalizer_test.go | 1 + sequencer/forcedbatch.go | 2 +- sequencer/l2block.go | 5 +- sequencer/sequencer.go | 134 ++- state/datastream.go | 482 +++++------ state/datastream/datastream.pb.go | 773 ++++++++++++++++++ state/pgstatestorage/datastream.go | 13 +- state/test/datastream_test.go | 82 -- test/docker-compose.yml | 6 + tools/datastreamer/Makefile | 22 +- tools/datastreamer/config/tool.config.toml | 10 +- tools/datastreamer/main.go | 733 +++++++---------- 16 files changed, 1541 insertions(+), 820 deletions(-) create mode 100644 proto/src/proto/datastream/v1/datastream.proto create mode 100644 state/datastream/datastream.pb.go delete mode 100644 state/test/datastream_test.go diff --git a/Makefile b/Makefile index 83d4fed5fe..ba55fb3429 100644 --- a/Makefile +++ b/Makefile @@ -164,6 +164,7 @@ generate-code-from-proto: ## Generates code from proto files cd proto/src/proto/hashdb/v1 && protoc --proto_path=. --proto_path=../../../../include --go_out=../../../../../merkletree/hashdb --go-grpc_out=../../../../../merkletree/hashdb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative hashdb.proto cd proto/src/proto/executor/v1 && protoc --proto_path=. --go_out=../../../../../state/runtime/executor --go-grpc_out=../../../../../state/runtime/executor --go-grpc_opt=paths=source_relative --go_opt=paths=source_relative executor.proto cd proto/src/proto/aggregator/v1 && protoc --proto_path=. --proto_path=../../../../include --go_out=../../../../../aggregator/prover --go-grpc_out=../../../../../aggregator/prover --go-grpc_opt=paths=source_relative --go_opt=paths=source_relative aggregator.proto + cd proto/src/proto/datastream/v1 && protoc --proto_path=. --proto_path=../../../../include --go_out=../../../../../state/datastream --go-grpc_out=../../../../../state/datastream --go-grpc_opt=paths=source_relative --go_opt=paths=source_relative datastream.proto ## Help display. ## Pulls comments from beside commands and prints a nicely formatted diff --git a/proto/src/proto/datastream/v1/datastream.proto b/proto/src/proto/datastream/v1/datastream.proto new file mode 100644 index 0000000000..6b8a89adc6 --- /dev/null +++ b/proto/src/proto/datastream/v1/datastream.proto @@ -0,0 +1,64 @@ +syntax = "proto3"; + +package datastream.v1; + +option go_package = "github.com/0xPolygonHermez/zkevm-node/state/datastream"; + +message Batch { + uint64 number = 1; + bytes local_exit_root = 2; + bytes state_root = 3; + uint64 fork_id = 4; + uint64 chain_id = 5; +} + +message L2Block { + uint64 number = 1; + uint64 batch_number = 2; + uint64 timestamp = 3; + uint32 delta_timestamp = 4; + uint64 min_timestamp = 5; + bytes l1_blockhash = 6; + uint32 l1_infotree_index = 7; + bytes hash = 8; + bytes state_root = 9; + bytes global_exit_root = 10; + bytes coinbase = 11; +} + +message Transaction { + uint64 l2block_number = 1; + bool is_valid = 2; + bytes encoded = 3; + uint32 effective_gas_price_percentage = 4; + bytes im_state_root = 5; +} + +message UpdateGER { + uint64 batch_number = 1; + uint64 timestamp = 2; + bytes global_exit_root = 3; + bytes coinbase = 4; + uint64 fork_id = 5; + uint64 chain_id = 6; + bytes state_root = 7; +} + +message BookMark { + BookmarkType type = 1; + uint64 value = 2; +} + +enum BookmarkType { + BOOKMARK_TYPE_UNSPECIFIED = 0; + BOOKMARK_TYPE_BATCH = 1; + BOOKMARK_TYPE_L2_BLOCK = 2; +} + +enum EntryType { + ENTRY_TYPE_UNSPECIFIED = 0; + ENTRY_TYPE_BATCH = 1; + ENTRY_TYPE_L2_BLOCK = 2; + ENTRY_TYPE_TRANSACTION = 3; + ENTRY_TYPE_UPDATE_GER = 4; +} diff --git a/sequencer/batch.go b/sequencer/batch.go index ecaa93aca7..8db3320a4d 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -29,6 +29,7 @@ type Batch struct { finalRemainingResources state.BatchResources // remaining batch resources when a L2 block is processed finalHighReservedZKCounters state.ZKCounters closingReason state.ClosingReason + finalLocalExitRoot common.Hash } func (b *Batch) isEmpty() bool { @@ -99,6 +100,7 @@ func (f *finalizer) setWIPBatch(ctx context.Context, wipStateBatch *state.Batch) finalRemainingResources: remainingResources, imHighReservedZKCounters: wipStateBatch.HighReservedZKCounters, finalHighReservedZKCounters: wipStateBatch.HighReservedZKCounters, + finalLocalExitRoot: wipStateBatch.LocalExitRoot, } return wipBatch, nil @@ -312,6 +314,7 @@ func (f *finalizer) openNewWIPBatch(batchNumber uint64, stateRoot common.Hash) * imRemainingResources: maxRemainingResources, finalRemainingResources: maxRemainingResources, closingReason: state.EmptyClosingReason, + finalLocalExitRoot: state.ZeroHash, } } @@ -400,6 +403,9 @@ func (f *finalizer) closeSIPBatch(ctx context.Context, dbTx pgx.Tx) error { }() } + // Sent batch to DS + f.DSSendBatch(f.wipBatch.batchNumber, f.wipBatch.finalStateRoot, f.wipBatch.finalLocalExitRoot) + log.Infof("sip batch %d closed in statedb, closing reason: %s", f.sipBatch.batchNumber, f.sipBatch.closingReason) f.sipBatch = nil diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 7f5e7e763a..81ed1083b0 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -3,9 +3,11 @@ package sequencer import ( "github.com/0xPolygonHermez/zkevm-node/log" "github.com/0xPolygonHermez/zkevm-node/state" + "github.com/0xPolygonHermez/zkevm-node/state/datastream" + "github.com/ethereum/go-ethereum/common" ) -func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32) error { +func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32, minTimestamp uint64) error { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) // Send data to streamer @@ -13,12 +15,13 @@ func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.Proce l2Block := state.DSL2Block{ BatchNumber: batchNumber, L2BlockNumber: blockResponse.BlockNumber, - Timestamp: int64(blockResponse.Timestamp), + Timestamp: blockResponse.Timestamp, + Min_timestamp: minTimestamp, L1InfoTreeIndex: l1InfoTreeIndex, L1BlockHash: blockResponse.BlockHashL1, GlobalExitRoot: blockResponse.GlobalExitRoot, Coinbase: f.sequencerAddress, - ForkID: uint16(forkID), + ForkID: forkID, BlockHash: blockResponse.BlockHash, StateRoot: blockResponse.BlockHash, //From etrog, the blockhash is the block root } @@ -57,9 +60,23 @@ func (f *finalizer) DSSendBatchBookmark(batchNumber uint64) { // Check if stream server enabled if f.streamServer != nil { // Send batch bookmark to the streamer - f.dataToStream <- state.DSBookMark{ - Type: state.BookMarkTypeBatch, + f.dataToStream <- datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, Value: batchNumber, } } } + +func (f *finalizer) DSSendBatch(batchNumber uint64, stateRoot common.Hash, localExitRoot common.Hash) { + forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) + + if f.streamServer != nil { + // Send batch to the streamer + f.dataToStream <- datastream.Batch{ + Number: batchNumber, + ForkId: forkID, + StateRoot: stateRoot.Bytes(), + LocalExitRoot: localExitRoot.Bytes(), + } + } +} diff --git a/sequencer/finalizer_test.go b/sequencer/finalizer_test.go index 6e54c342f8..84a2d6e8cf 100644 --- a/sequencer/finalizer_test.go +++ b/sequencer/finalizer_test.go @@ -980,6 +980,7 @@ func TestFinalizer_finalizeSIPBatch(t *testing.T) { // arrange stateMock.On("BeginStateTransaction", ctx).Return(dbTxMock, nilErr).Once() + stateMock.On("GetForkIDByBatchNumber", mock.Anything).Return(uint64(state.FORKID_BLUEBERRY)) stateMock.On("CloseWIPBatch", ctx, receipt, mock.Anything).Return(tc.managerErr).Once() if tc.managerErr == nil { diff --git a/sequencer/forcedbatch.go b/sequencer/forcedbatch.go index 85f74abee1..21a0c92504 100644 --- a/sequencer/forcedbatch.go +++ b/sequencer/forcedbatch.go @@ -198,7 +198,7 @@ func (f *finalizer) handleProcessForcedBatchResponse(ctx context.Context, newBat } // Send L2 block to data streamer - err = f.DSSendL2Block(newBatchNumber, forcedL2BlockResponse, 0) + err = f.DSSendL2Block(newBatchNumber, forcedL2BlockResponse, 0, forcedL2BlockResponse.Timestamp) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d to data streamer, error: %v", forcedL2BlockResponse.BlockNumber, err) diff --git a/sequencer/l2block.go b/sequencer/l2block.go index b64a6db902..f045ef2764 100644 --- a/sequencer/l2block.go +++ b/sequencer/l2block.go @@ -259,8 +259,9 @@ func (f *finalizer) processL2Block(ctx context.Context, l2Block *L2Block) error return fmt.Errorf(overflowLog) } - // Update finalStateRoot of the batch to the newStateRoot for the L2 block + // Update finalStateRoot/finalLocalExitRoot of the batch to the newStateRoot/newLocalExitRoot for the L2 block l2Block.batch.finalStateRoot = l2Block.batchResponse.NewStateRoot + l2Block.batch.finalLocalExitRoot = l2Block.batchResponse.NewLocalExitRoot f.updateFlushIDs(batchResponse.FlushID, batchResponse.StoredFlushID) @@ -491,7 +492,7 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { log.Infof("[ds-debug] l2 block %d [%d] transactions updated as selected in the pooldb", blockResponse.BlockNumber, l2Block.trackingNum) // Send L2 block to data streamer - err = f.DSSendL2Block(f.wipBatch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex()) + err = f.DSSendL2Block(f.wipBatch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d [%d] to data streamer, error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err) diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index b79ad26c17..d1945234d5 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -11,7 +11,9 @@ import ( "github.com/0xPolygonHermez/zkevm-node/log" "github.com/0xPolygonHermez/zkevm-node/pool" "github.com/0xPolygonHermez/zkevm-node/state" + "github.com/0xPolygonHermez/zkevm-node/state/datastream" "github.com/ethereum/go-ethereum/common" + "google.golang.org/protobuf/proto" ) const ( @@ -137,7 +139,7 @@ func (s *Sequencer) checkStateInconsistency(ctx context.Context) { } func (s *Sequencer) updateDataStreamerFile(ctx context.Context, chainID uint64) { - err := state.GenerateDataStreamerFile(ctx, s.streamServer, s.stateIntf, true, nil, chainID, s.cfg.StreamServer.UpgradeEtrogBatchNumber) + err := state.GenerateDataStreamFile(ctx, s.streamServer, s.stateIntf, true, nil, chainID, s.cfg.StreamServer.UpgradeEtrogBatchNumber) if err != nil { log.Fatalf("failed to generate data streamer file, error: %v", err) } @@ -278,54 +280,77 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { continue } - bookMark := state.DSBookMark{ - Type: state.BookMarkTypeL2Block, + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, Value: l2Block.L2BlockNumber, } //TODO: remove this log log.Infof("[ds-debug] add stream bookmark for l2block %d", l2Block.L2BlockNumber) - _, err = s.streamServer.AddStreamBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + log.Errorf("failed to marshal bookmark for l2block %d, error: %v", l2Block.L2BlockNumber, err) + continue + } + + _, err = s.streamServer.AddStreamBookmark(marshalledBookMark) if err != nil { log.Errorf("failed to add stream bookmark for l2block %d, error: %v", l2Block.L2BlockNumber, err) continue } // Get previous block timestamp to calculate delta timestamp - previousL2Block := state.DSL2BlockStart{} + previousL2Block := datastream.L2Block{} if l2Block.L2BlockNumber > 0 { - bookMark = state.DSBookMark{ - Type: state.BookMarkTypeL2Block, + bookMark = &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, Value: l2Block.L2BlockNumber - 1, } //TODO: remove this log log.Infof("[ds-debug] get previous l2block %d", l2Block.L2BlockNumber-1) - previousL2BlockEntry, err := s.streamServer.GetFirstEventAfterBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + log.Errorf("failed to marshal bookmark for l2block %d, error: %v", l2Block.L2BlockNumber, err) + continue + } + + previousL2BlockEntry, err := s.streamServer.GetFirstEventAfterBookmark(marshalledBookMark) if err != nil { log.Errorf("failed to get previous l2block %d, error: %v", l2Block.L2BlockNumber-1, err) continue } - previousL2Block = state.DSL2BlockStart{}.Decode(previousL2BlockEntry.Data) + err = proto.Unmarshal(previousL2BlockEntry.Data, &previousL2Block) + if err != nil { + log.Errorf("failed to unmarshal previous l2block %d, error: %v", l2Block.L2BlockNumber-1, err) + continue + } } - blockStart := state.DSL2BlockStart{ + streamL2Block := &datastream.L2Block{ + Number: l2Block.L2BlockNumber, BatchNumber: l2Block.BatchNumber, - L2BlockNumber: l2Block.L2BlockNumber, Timestamp: l2Block.Timestamp, DeltaTimestamp: uint32(l2Block.Timestamp - previousL2Block.Timestamp), - L1InfoTreeIndex: l2Block.L1InfoTreeIndex, - L1BlockHash: l2Block.L1BlockHash, - GlobalExitRoot: l2Block.GlobalExitRoot, - Coinbase: l2Block.Coinbase, - ForkID: l2Block.ForkID, - ChainID: uint32(chainID), + MinTimestamp: l2Block.Min_timestamp, + L1Blockhash: l2Block.L1BlockHash.Bytes(), + L1InfotreeIndex: l2Block.L1InfoTreeIndex, + Hash: l2Block.BlockHash.Bytes(), + StateRoot: l2Block.StateRoot.Bytes(), + GlobalExitRoot: l2Block.GlobalExitRoot.Bytes(), + Coinbase: l2Block.Coinbase.Bytes(), + } + + marshalledL2Block, err := proto.Marshal(streamL2Block) + if err != nil { + log.Errorf("failed to marshal l2block %d, error: %v", l2Block.L2BlockNumber, err) + continue } //TODO: remove this log log.Infof("[ds-debug] add l2blockStart stream entry for l2block %d", l2Block.L2BlockNumber) - _, err = s.streamServer.AddStreamEntry(state.EntryTypeL2BlockStart, blockStart.Encode()) + _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK), marshalledL2Block) if err != nil { log.Errorf("failed to add stream entry for l2block %d, error: %v", l2Block.L2BlockNumber, err) continue @@ -334,25 +359,25 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { //TODO: remove this log log.Infof("[ds-debug] adding l2tx stream entries for l2block %d", l2Block.L2BlockNumber) for _, l2Transaction := range l2Block.Txs { - _, err = s.streamServer.AddStreamEntry(state.EntryTypeL2Tx, l2Transaction.Encode()) + streamL2Transaction := &datastream.Transaction{ + L2BlockNumber: l2Transaction.L2BlockNumber, + IsValid: l2Transaction.IsValid != 0, + Encoded: l2Transaction.Encoded, + EffectiveGasPricePercentage: uint32(l2Transaction.EffectiveGasPricePercentage), + ImStateRoot: l2Transaction.ImStateRoot.Bytes(), + } + + marshalledL2Transaction, err := proto.Marshal(streamL2Transaction) if err != nil { - log.Errorf("failed to add l2tx stream entry for l2block %d, error: %v", l2Block.L2BlockNumber, err) + log.Errorf("failed to marshal l2tx for l2block %d, error: %v", l2Block.L2BlockNumber, err) continue } - } - blockEnd := state.DSL2BlockEnd{ - L2BlockNumber: l2Block.L2BlockNumber, - BlockHash: l2Block.BlockHash, - StateRoot: l2Block.StateRoot, - } - - //TODO: remove this log - log.Infof("[ds-debug] add l2blockEnd stream entry for l2block %d", l2Block.L2BlockNumber) - _, err = s.streamServer.AddStreamEntry(state.EntryTypeL2BlockEnd, blockEnd.Encode()) - if err != nil { - log.Errorf("failed to add stream entry for l2block %d, error: %v", l2Block.L2BlockNumber, err) - continue + _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION), marshalledL2Transaction) + if err != nil { + log.Errorf("failed to add l2tx stream entry for l2block %d, error: %v", l2Block.L2BlockNumber, err) + continue + } } //TODO: remove this log @@ -367,24 +392,55 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { log.Infof("[ds-debug] l2block %d sent to datastream", l2Block.L2BlockNumber) // Stream a bookmark - case state.DSBookMark: - bookmark := data + case datastream.BookMark: + err = s.streamServer.StartAtomicOp() + if err != nil { + log.Errorf("failed to start atomic op for bookmark type %d, value %d, error: %v", data.Type, data.Value, err) + continue + } + marshalledBookMark, err := proto.Marshal(&data) + if err != nil { + log.Errorf("failed to marshal bookmark type %d, value %d, error: %v", data.Type, data.Value, err) + continue + } + + _, err = s.streamServer.AddStreamBookmark(marshalledBookMark) + if err != nil { + log.Errorf("failed to add stream bookmark for bookmark type %d, value %d, error: %v", data.Type, data.Value, err) + continue + } + + err = s.streamServer.CommitAtomicOp() + if err != nil { + log.Errorf("failed to commit atomic op for bookmark type %d, value %d, error: %v", data.Type, data.Value, err) + continue + } + case datastream.Batch: err = s.streamServer.StartAtomicOp() if err != nil { - log.Errorf("failed to start atomic op for bookmark type %d, value %d, error: %v", bookmark.Type, bookmark.Value, err) + log.Errorf("failed to start atomic op for batch, error: %v", err) continue } - _, err = s.streamServer.AddStreamBookmark(bookmark.Encode()) + data.ChainId = chainID + + marshalledBatch, err := proto.Marshal(&data) + if err != nil { + log.Errorf("failed to marshal batch, error: %v", err) + continue + } + + _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledBatch) if err != nil { - log.Errorf("failed to add stream bookmark type %d, value %d, error: %v", bookmark.Type, bookmark.Value, err) + log.Errorf("failed to add stream entry for batch, error: %v", err) continue } err = s.streamServer.CommitAtomicOp() if err != nil { - log.Errorf("failed to commit atomic op for bookmark type %d, value %d, error: %v", bookmark.Type, bookmark.Value, err) + log.Errorf("failed to commit atomic op for batch, error: %v", err) + continue } // Invalid stream message type diff --git a/state/datastream.go b/state/datastream.go index d50c7adecf..eb3670e6f9 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -2,14 +2,16 @@ package state import ( "context" - "encoding/binary" "math/big" + "time" "github.com/0xPolygonHermez/zkevm-data-streamer/datastreamer" "github.com/0xPolygonHermez/zkevm-node/log" + "github.com/0xPolygonHermez/zkevm-node/state/datastream" "github.com/ethereum/go-ethereum/common" "github.com/iden3/go-iden3-crypto/keccak256" "github.com/jackc/pgx/v4" + "google.golang.org/protobuf/proto" ) const ( @@ -17,18 +19,6 @@ const ( StreamTypeSequencer datastreamer.StreamType = 1 // EntryTypeBookMark represents a bookmark entry EntryTypeBookMark datastreamer.EntryType = datastreamer.EtBookmark - // EntryTypeL2BlockStart represents a L2 block start - EntryTypeL2BlockStart datastreamer.EntryType = 1 - // EntryTypeL2Tx represents a L2 transaction - EntryTypeL2Tx datastreamer.EntryType = 2 - // EntryTypeL2BlockEnd represents a L2 block end - EntryTypeL2BlockEnd datastreamer.EntryType = 3 - // EntryTypeUpdateGER represents a GER update - EntryTypeUpdateGER datastreamer.EntryType = 4 - // BookMarkTypeL2Block represents a L2 block bookmark - BookMarkTypeL2Block byte = 0 - // BookMarkTypeBatch represents a batch - BookMarkTypeBatch byte = 1 // SystemSC is the system smart contract address SystemSC = "0x000000000000000000000000000000005ca1ab1e" // posConstant is the constant used to compute the position of the intermediate state root @@ -38,7 +28,8 @@ const ( // DSBatch represents a data stream batch type DSBatch struct { Batch - ForkID uint16 + ForkID uint64 + EtrogTimestamp *time.Time } // DSFullBatch represents a data stream batch ant its L2 blocks @@ -55,64 +46,18 @@ type DSL2FullBlock struct { // DSL2Block is a full l2 block type DSL2Block struct { - BatchNumber uint64 // 8 bytes - L2BlockNumber uint64 // 8 bytes - Timestamp int64 // 8 bytes - L1InfoTreeIndex uint32 // 4 bytes - L1BlockHash common.Hash // 32 bytes - GlobalExitRoot common.Hash // 32 bytes - Coinbase common.Address // 20 bytes - ForkID uint16 // 2 bytes - ChainID uint32 // 4 bytes - BlockHash common.Hash // 32 bytes - StateRoot common.Hash // 32 bytes -} - -// DSL2BlockStart represents a data stream L2 block start -type DSL2BlockStart struct { - BatchNumber uint64 // 8 bytes - L2BlockNumber uint64 // 8 bytes - Timestamp int64 // 8 bytes - DeltaTimestamp uint32 // 4 bytes - L1InfoTreeIndex uint32 // 4 bytes - L1BlockHash common.Hash // 32 bytes - GlobalExitRoot common.Hash // 32 bytes - Coinbase common.Address // 20 bytes - ForkID uint16 // 2 bytes - ChainID uint32 // 4 bytes - -} - -// Encode returns the encoded DSL2BlockStart as a byte slice -func (b DSL2BlockStart) Encode() []byte { - bytes := make([]byte, 0) - bytes = binary.BigEndian.AppendUint64(bytes, b.BatchNumber) - bytes = binary.BigEndian.AppendUint64(bytes, b.L2BlockNumber) - bytes = binary.BigEndian.AppendUint64(bytes, uint64(b.Timestamp)) - bytes = binary.BigEndian.AppendUint32(bytes, b.DeltaTimestamp) - bytes = binary.BigEndian.AppendUint32(bytes, b.L1InfoTreeIndex) - bytes = append(bytes, b.L1BlockHash.Bytes()...) - bytes = append(bytes, b.GlobalExitRoot.Bytes()...) - bytes = append(bytes, b.Coinbase.Bytes()...) - bytes = binary.BigEndian.AppendUint16(bytes, b.ForkID) - bytes = binary.BigEndian.AppendUint32(bytes, b.ChainID) - return bytes -} - -// Decode decodes the DSL2BlockStart from a byte slice -func (b DSL2BlockStart) Decode(data []byte) DSL2BlockStart { - b.BatchNumber = binary.BigEndian.Uint64(data[0:8]) - b.L2BlockNumber = binary.BigEndian.Uint64(data[8:16]) - b.Timestamp = int64(binary.BigEndian.Uint64(data[16:24])) - b.DeltaTimestamp = binary.BigEndian.Uint32(data[24:28]) - b.L1InfoTreeIndex = binary.BigEndian.Uint32(data[28:32]) - b.L1BlockHash = common.BytesToHash(data[32:64]) - b.GlobalExitRoot = common.BytesToHash(data[64:96]) - b.Coinbase = common.BytesToAddress(data[96:116]) - b.ForkID = binary.BigEndian.Uint16(data[116:118]) - b.ChainID = binary.BigEndian.Uint32(data[118:122]) - - return b + BatchNumber uint64 + L2BlockNumber uint64 + Timestamp uint64 + Min_timestamp uint64 + L1InfoTreeIndex uint32 + L1BlockHash common.Hash + GlobalExitRoot common.Hash + Coinbase common.Address + ForkID uint64 + ChainID uint64 + BlockHash common.Hash + StateRoot common.Hash } // DSL2Transaction represents a data stream L2 transaction @@ -126,108 +71,6 @@ type DSL2Transaction struct { Encoded []byte } -// Encode returns the encoded DSL2Transaction as a byte slice -func (l DSL2Transaction) Encode() []byte { - bytes := make([]byte, 0) - bytes = append(bytes, l.EffectiveGasPricePercentage) - bytes = append(bytes, l.IsValid) - bytes = append(bytes, l.StateRoot[:]...) - bytes = binary.BigEndian.AppendUint32(bytes, l.EncodedLength) - bytes = append(bytes, l.Encoded...) - return bytes -} - -// Decode decodes the DSL2Transaction from a byte slice -func (l DSL2Transaction) Decode(data []byte) DSL2Transaction { - l.EffectiveGasPricePercentage = data[0] - l.IsValid = data[1] - l.StateRoot = common.BytesToHash(data[2:34]) - l.EncodedLength = binary.BigEndian.Uint32(data[34:38]) - l.Encoded = data[38:] - return l -} - -// DSL2BlockEnd represents a L2 block end -type DSL2BlockEnd struct { - L2BlockNumber uint64 // 8 bytes - BlockHash common.Hash // 32 bytes - StateRoot common.Hash // 32 bytes -} - -// Encode returns the encoded DSL2BlockEnd as a byte slice -func (b DSL2BlockEnd) Encode() []byte { - bytes := make([]byte, 0) - bytes = binary.BigEndian.AppendUint64(bytes, b.L2BlockNumber) - bytes = append(bytes, b.BlockHash[:]...) - bytes = append(bytes, b.StateRoot[:]...) - return bytes -} - -// Decode decodes the DSL2BlockEnd from a byte slice -func (b DSL2BlockEnd) Decode(data []byte) DSL2BlockEnd { - b.L2BlockNumber = binary.BigEndian.Uint64(data[0:8]) - b.BlockHash = common.BytesToHash(data[8:40]) - b.StateRoot = common.BytesToHash(data[40:72]) - return b -} - -// DSBookMark represents a data stream bookmark -type DSBookMark struct { - Type byte // 1 byte - Value uint64 // 8 bytes -} - -// Encode returns the encoded DSBookMark as a byte slice -func (b DSBookMark) Encode() []byte { - bytes := make([]byte, 0) - bytes = append(bytes, b.Type) - bytes = binary.BigEndian.AppendUint64(bytes, b.Value) - return bytes -} - -// Decode decodes the DSBookMark from a byte slice -func (b DSBookMark) Decode(data []byte) DSBookMark { - b.Type = data[0] - b.Value = binary.BigEndian.Uint64(data[1:9]) - return b -} - -// DSUpdateGER represents a data stream GER update -type DSUpdateGER struct { - BatchNumber uint64 // 8 bytes - Timestamp int64 // 8 bytes - GlobalExitRoot common.Hash // 32 bytes - Coinbase common.Address // 20 bytes - ForkID uint16 // 2 bytes - ChainID uint32 // 4 bytes - StateRoot common.Hash // 32 bytes -} - -// Encode returns the encoded DSUpdateGER as a byte slice -func (g DSUpdateGER) Encode() []byte { - bytes := make([]byte, 0) - bytes = binary.BigEndian.AppendUint64(bytes, g.BatchNumber) - bytes = binary.BigEndian.AppendUint64(bytes, uint64(g.Timestamp)) - bytes = append(bytes, g.GlobalExitRoot[:]...) - bytes = append(bytes, g.Coinbase[:]...) - bytes = binary.BigEndian.AppendUint16(bytes, g.ForkID) - bytes = binary.BigEndian.AppendUint32(bytes, g.ChainID) - bytes = append(bytes, g.StateRoot[:]...) - return bytes -} - -// Decode decodes the DSUpdateGER from a byte slice -func (g DSUpdateGER) Decode(data []byte) DSUpdateGER { - g.BatchNumber = binary.BigEndian.Uint64(data[0:8]) - g.Timestamp = int64(binary.BigEndian.Uint64(data[8:16])) - g.GlobalExitRoot = common.BytesToHash(data[16:48]) - g.Coinbase = common.BytesToAddress(data[48:68]) - g.ForkID = binary.BigEndian.Uint16(data[68:70]) - g.ChainID = binary.BigEndian.Uint32(data[70:74]) - g.StateRoot = common.BytesToHash(data[74:106]) - return g -} - // DSState gathers the methods required to interact with the data stream state. type DSState interface { GetDSGenesisBlock(ctx context.Context, dbTx pgx.Tx) (*DSL2Block, error) @@ -240,14 +83,14 @@ type DSState interface { GetL1InfoRootLeafByIndex(ctx context.Context, l1InfoTreeIndex uint32, dbTx pgx.Tx) (L1InfoTreeExitRootStorageEntry, error) } -// GenerateDataStreamerFile generates or resumes a data stream file -func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.StreamServer, stateDB DSState, readWIPBatch bool, imStateRoots *map[uint64][]byte, chainID uint64, upgradeEtrogBatchNumber uint64) error { +// GenerateDataStreamFile generates or resumes a data stream file +func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.StreamServer, stateDB DSState, readWIPBatch bool, imStateRoots *map[uint64][]byte, chainID uint64, upgradeEtrogBatchNumber uint64) error { header := streamServer.GetHeader() var currentBatchNumber uint64 = 0 var lastAddedL2BlockNumber uint64 = 0 var lastAddedBatchNumber uint64 = 0 - var previousTimestamp int64 = 0 + var previousTimestamp uint64 = 0 if header.TotalEntries == 0 { // Get Genesis block @@ -261,52 +104,73 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St return err } - bookMark := DSBookMark{ - Type: BookMarkTypeBatch, + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, Value: genesisL2Block.BatchNumber, } - _, err = streamServer.AddStreamBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) if err != nil { return err } - bookMark = DSBookMark{ - Type: BookMarkTypeL2Block, + _, err = streamServer.AddStreamBookmark(marshalledBookMark) + if err != nil { + return err + } + + bookMark = &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, Value: genesisL2Block.L2BlockNumber, } - _, err = streamServer.AddStreamBookmark(bookMark.Encode()) + marshalledBookMark, err = proto.Marshal(bookMark) + if err != nil { + return err + } + + _, err = streamServer.AddStreamBookmark(marshalledBookMark) if err != nil { return err } - genesisBlock := DSL2BlockStart{ - BatchNumber: genesisL2Block.BatchNumber, - L2BlockNumber: genesisL2Block.L2BlockNumber, - Timestamp: genesisL2Block.Timestamp, + genesisBlock := &datastream.L2Block{ + Number: genesisL2Block.L2BlockNumber, DeltaTimestamp: 0, - L1InfoTreeIndex: 0, - GlobalExitRoot: genesisL2Block.GlobalExitRoot, - Coinbase: genesisL2Block.Coinbase, - ForkID: genesisL2Block.ForkID, - ChainID: uint32(chainID), + MinTimestamp: 0, + L1InfotreeIndex: 0, + Hash: genesisL2Block.BlockHash.Bytes(), + StateRoot: genesisL2Block.StateRoot.Bytes(), + GlobalExitRoot: genesisL2Block.GlobalExitRoot.Bytes(), + Coinbase: genesisL2Block.Coinbase.Bytes(), } - log.Infof("Genesis block: %+v", genesisBlock) + log.Debugf("Genesis block: %+v", genesisBlock) + + marshalledGenesisBlock, err := proto.Marshal(genesisBlock) + if err != nil { + return err + } - _, err = streamServer.AddStreamEntry(1, genesisBlock.Encode()) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK), marshalledGenesisBlock) if err != nil { return err } - genesisBlockEnd := DSL2BlockEnd{ - L2BlockNumber: genesisL2Block.L2BlockNumber, - BlockHash: genesisL2Block.BlockHash, - StateRoot: genesisL2Block.StateRoot, + genesisBatch := &datastream.Batch{ + Number: genesisL2Block.BatchNumber, + LocalExitRoot: common.Hash{}.Bytes(), + StateRoot: genesisL2Block.StateRoot.Bytes(), + ForkId: genesisL2Block.ForkID, + ChainId: chainID, + } + + marshalledGenesisBatch, err := proto.Marshal(genesisBatch) + if err != nil { + return err } - _, err = streamServer.AddStreamEntry(EntryTypeL2BlockEnd, genesisBlockEnd.Encode()) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledGenesisBatch) if err != nil { return err } @@ -325,35 +189,88 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St log.Infof("Latest entry: %+v", latestEntry) switch latestEntry.Type { - case EntryTypeUpdateGER: + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH): + log.Info("Latest entry type is Batch") + + batch := &datastream.Batch{} + if err := proto.Unmarshal(latestEntry.Data, batch); err != nil { + return err + } + + currentBatchNumber = batch.Number + currentBatchNumber++ + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_UPDATE_GER): log.Info("Latest entry type is UpdateGER") - currentBatchNumber = binary.BigEndian.Uint64(latestEntry.Data[0:8]) + + updateGer := &datastream.UpdateGER{} + if err := proto.Unmarshal(latestEntry.Data, updateGer); err != nil { + return err + } + + currentBatchNumber = updateGer.BatchNumber currentBatchNumber++ - case EntryTypeL2BlockEnd: - log.Info("Latest entry type is L2BlockEnd") - blockEnd := DSL2BlockEnd{}.Decode(latestEntry.Data) - currentL2BlockNumber := blockEnd.L2BlockNumber - - bookMark := DSBookMark{ - Type: BookMarkTypeL2Block, - Value: currentL2BlockNumber, + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK): + log.Info("Latest entry type is L2Block") + + l2Block := &datastream.L2Block{} + + if err := proto.Unmarshal(latestEntry.Data, l2Block); err != nil { + return err + } + + currentL2BlockNumber := l2Block.Number + currentBatchNumber = l2Block.BatchNumber + previousTimestamp = l2Block.Timestamp + lastAddedL2BlockNumber = currentL2BlockNumber + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION): + log.Info("Latest entry type is Transaction") + + transaction := &datastream.Transaction{} + if err := proto.Unmarshal(latestEntry.Data, transaction); err != nil { + return err + } + + currentL2BlockNumber := transaction.L2BlockNumber + currentBatchNumber = transaction.L2BlockNumber + lastAddedL2BlockNumber = currentL2BlockNumber + + // Get Previous l2block timestamp + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, + Value: currentL2BlockNumber - 1, } - firstEntry, err := streamServer.GetFirstEventAfterBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) if err != nil { return err } - blockStart := DSL2BlockStart{}.Decode(firstEntry.Data) + prevL2BlockEntryNumber, err := streamServer.GetBookmark(marshalledBookMark) + if err != nil { + return err + } + + prevL2BlockEntry, err := streamServer.GetEntry(prevL2BlockEntryNumber) + if err != nil { + return err + } + + prevL2Block := &datastream.L2Block{} + if err := proto.Unmarshal(prevL2BlockEntry.Data, prevL2Block); err != nil { + return err + } + + previousTimestamp = prevL2Block.Timestamp - currentBatchNumber = blockStart.BatchNumber - previousTimestamp = blockStart.Timestamp - lastAddedL2BlockNumber = currentL2BlockNumber case EntryTypeBookMark: log.Info("Latest entry type is BookMark") - bookMark := DSBookMark{} - bookMark = bookMark.Decode(latestEntry.Data) - if bookMark.Type == BookMarkTypeBatch { + + bookMark := &datastream.BookMark{} + if err := proto.Unmarshal(latestEntry.Data, bookMark); err != nil { + return err + } + + if bookMark.Type == datastream.BookmarkType_BOOKMARK_TYPE_BATCH { currentBatchNumber = bookMark.Value } else { log.Fatalf("Latest entry type is an unexpected bookmark type: %v", bookMark.Type) @@ -424,21 +341,26 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St return err } - bookMark := DSBookMark{ - Type: BookMarkTypeBatch, + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, Value: batch.BatchNumber, } + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err + } + missingBatchBookMark := true if b == 0 { - _, err = streamServer.GetBookmark(bookMark.Encode()) + _, err = streamServer.GetBookmark(marshalledBookMark) if err == nil { missingBatchBookMark = false } } if missingBatchBookMark { - _, err = streamServer.AddStreamBookmark(bookMark.Encode()) + _, err = streamServer.AddStreamBookmark(marshalledBookMark) if err != nil { return err } @@ -448,17 +370,22 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St // Empty batch // Check if there is a GER update if batch.GlobalExitRoot != currentGER && batch.GlobalExitRoot != (common.Hash{}) { - updateGer := DSUpdateGER{ + updateGER := &datastream.UpdateGER{ BatchNumber: batch.BatchNumber, - Timestamp: batch.Timestamp.Unix(), - GlobalExitRoot: batch.GlobalExitRoot, - Coinbase: batch.Coinbase, - ForkID: batch.ForkID, - ChainID: uint32(chainID), - StateRoot: batch.StateRoot, + Timestamp: uint64(batch.Timestamp.Unix()), + GlobalExitRoot: batch.GlobalExitRoot.Bytes(), + Coinbase: batch.Coinbase.Bytes(), + ForkId: batch.ForkID, + ChainId: chainID, + StateRoot: batch.StateRoot.Bytes(), + } + + marshalledUpdateGER, err := proto.Marshal(updateGER) + if err != nil { + return err } - _, err = streamServer.AddStreamEntry(EntryTypeUpdateGER, updateGer.Encode()) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_UPDATE_GER), marshalledUpdateGER) if err != nil { return err } @@ -517,38 +444,57 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St } } - blockStart := DSL2BlockStart{ + streamL2Block := &datastream.L2Block{ + Number: l2Block.L2BlockNumber, BatchNumber: l2Block.BatchNumber, - L2BlockNumber: l2Block.L2BlockNumber, Timestamp: l2Block.Timestamp, DeltaTimestamp: uint32(l2Block.Timestamp - previousTimestamp), - L1InfoTreeIndex: l1InfoTreeIndex, - L1BlockHash: l1BlockHash, - GlobalExitRoot: l2Block.GlobalExitRoot, - Coinbase: l2Block.Coinbase, - ForkID: l2Block.ForkID, - ChainID: uint32(chainID), + MinTimestamp: uint64(batch.Timestamp.Unix()), + L1Blockhash: l1BlockHash.Bytes(), + L1InfotreeIndex: l1InfoTreeIndex, + Hash: l2Block.BlockHash.Bytes(), + StateRoot: l2Block.StateRoot.Bytes(), + GlobalExitRoot: l2Block.GlobalExitRoot.Bytes(), + Coinbase: l2Block.Coinbase.Bytes(), + } + + if l2Block.ForkID >= FORKID_ETROG { + streamL2Block.Hash = l2Block.StateRoot.Bytes() + } + + if l2Block.ForkID == FORKID_ETROG && batch.EtrogTimestamp != nil { + streamL2Block.MinTimestamp = uint64(batch.EtrogTimestamp.Unix()) } previousTimestamp = l2Block.Timestamp - bookMark := DSBookMark{ - Type: BookMarkTypeL2Block, - Value: blockStart.L2BlockNumber, + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, + Value: streamL2Block.Number, + } + + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err } // Check if l2 block was already added - _, err = streamServer.GetBookmark(bookMark.Encode()) + _, err = streamServer.GetBookmark(marshalledBookMark) if err == nil { continue } - _, err = streamServer.AddStreamBookmark(bookMark.Encode()) + _, err = streamServer.AddStreamBookmark(marshalledBookMark) + if err != nil { + return err + } + + marshalledL2Block, err := proto.Marshal(streamL2Block) if err != nil { return err } - _, err = streamServer.AddStreamEntry(EntryTypeL2BlockStart, blockStart.Encode()) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK), marshalledL2Block) if err != nil { return err } @@ -559,7 +505,7 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St // > ETROG => IM State root is retrieved from the receipt.im_state_root if l2Block.ForkID < FORKID_ETROG { // Populate intermediate state root with information from the system SC (or cache if available) - if imStateRoots == nil || (*imStateRoots)[blockStart.L2BlockNumber] == nil { + if imStateRoots == nil || (*imStateRoots)[streamL2Block.Number] == nil { position := GetSystemSCPosition(l2Block.L2BlockNumber) imStateRoot, err := stateDB.GetStorageAt(ctx, common.HexToAddress(SystemSC), big.NewInt(0).SetBytes(position), l2Block.StateRoot) if err != nil { @@ -567,35 +513,53 @@ func GenerateDataStreamerFile(ctx context.Context, streamServer *datastreamer.St } tx.StateRoot = common.BigToHash(imStateRoot) } else { - tx.StateRoot = common.BytesToHash((*imStateRoots)[blockStart.L2BlockNumber]) + tx.StateRoot = common.BytesToHash((*imStateRoots)[streamL2Block.Number]) } } else if l2Block.ForkID > FORKID_ETROG { tx.StateRoot = tx.ImStateRoot } - _, err = streamServer.AddStreamEntry(EntryTypeL2Tx, tx.Encode()) + transaction := &datastream.Transaction{ + L2BlockNumber: tx.L2BlockNumber, + IsValid: tx.IsValid != 0, + Encoded: tx.Encoded, + EffectiveGasPricePercentage: uint32(tx.EffectiveGasPricePercentage), + ImStateRoot: tx.StateRoot.Bytes(), + } + + marshalledTransaction, err := proto.Marshal(transaction) if err != nil { return err } - } - - blockEnd := DSL2BlockEnd{ - L2BlockNumber: l2Block.L2BlockNumber, - BlockHash: l2Block.BlockHash, - StateRoot: l2Block.StateRoot, - } - if l2Block.ForkID >= FORKID_ETROG { - blockEnd.BlockHash = l2Block.StateRoot + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION), marshalledTransaction) + if err != nil { + return err + } } - _, err = streamServer.AddStreamEntry(EntryTypeL2BlockEnd, blockEnd.Encode()) - if err != nil { - return err - } currentGER = l2Block.GlobalExitRoot } } + + batch := &datastream.Batch{ + Number: batch.BatchNumber, + LocalExitRoot: batch.LocalExitRoot.Bytes(), + StateRoot: batch.StateRoot.Bytes(), + ForkId: batch.ForkID, + ChainId: chainID, + } + + marshalledBatch, err := proto.Marshal(batch) + if err != nil { + return err + } + + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledBatch) + if err != nil { + return err + } + // Commit at the end of each batch group err = streamServer.CommitAtomicOp() if err != nil { diff --git a/state/datastream/datastream.pb.go b/state/datastream/datastream.pb.go new file mode 100644 index 0000000000..1f0ede74fc --- /dev/null +++ b/state/datastream/datastream.pb.go @@ -0,0 +1,773 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.32.0 +// protoc v4.25.3 +// source: datastream.proto + +package datastream + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type BookmarkType int32 + +const ( + BookmarkType_BOOKMARK_TYPE_UNSPECIFIED BookmarkType = 0 + BookmarkType_BOOKMARK_TYPE_BATCH BookmarkType = 1 + BookmarkType_BOOKMARK_TYPE_L2_BLOCK BookmarkType = 2 +) + +// Enum value maps for BookmarkType. +var ( + BookmarkType_name = map[int32]string{ + 0: "BOOKMARK_TYPE_UNSPECIFIED", + 1: "BOOKMARK_TYPE_BATCH", + 2: "BOOKMARK_TYPE_L2_BLOCK", + } + BookmarkType_value = map[string]int32{ + "BOOKMARK_TYPE_UNSPECIFIED": 0, + "BOOKMARK_TYPE_BATCH": 1, + "BOOKMARK_TYPE_L2_BLOCK": 2, + } +) + +func (x BookmarkType) Enum() *BookmarkType { + p := new(BookmarkType) + *p = x + return p +} + +func (x BookmarkType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BookmarkType) Descriptor() protoreflect.EnumDescriptor { + return file_datastream_proto_enumTypes[0].Descriptor() +} + +func (BookmarkType) Type() protoreflect.EnumType { + return &file_datastream_proto_enumTypes[0] +} + +func (x BookmarkType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BookmarkType.Descriptor instead. +func (BookmarkType) EnumDescriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{0} +} + +type EntryType int32 + +const ( + EntryType_ENTRY_TYPE_UNSPECIFIED EntryType = 0 + EntryType_ENTRY_TYPE_BATCH EntryType = 1 + EntryType_ENTRY_TYPE_L2_BLOCK EntryType = 2 + EntryType_ENTRY_TYPE_TRANSACTION EntryType = 3 + EntryType_ENTRY_TYPE_UPDATE_GER EntryType = 4 +) + +// Enum value maps for EntryType. +var ( + EntryType_name = map[int32]string{ + 0: "ENTRY_TYPE_UNSPECIFIED", + 1: "ENTRY_TYPE_BATCH", + 2: "ENTRY_TYPE_L2_BLOCK", + 3: "ENTRY_TYPE_TRANSACTION", + 4: "ENTRY_TYPE_UPDATE_GER", + } + EntryType_value = map[string]int32{ + "ENTRY_TYPE_UNSPECIFIED": 0, + "ENTRY_TYPE_BATCH": 1, + "ENTRY_TYPE_L2_BLOCK": 2, + "ENTRY_TYPE_TRANSACTION": 3, + "ENTRY_TYPE_UPDATE_GER": 4, + } +) + +func (x EntryType) Enum() *EntryType { + p := new(EntryType) + *p = x + return p +} + +func (x EntryType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (EntryType) Descriptor() protoreflect.EnumDescriptor { + return file_datastream_proto_enumTypes[1].Descriptor() +} + +func (EntryType) Type() protoreflect.EnumType { + return &file_datastream_proto_enumTypes[1] +} + +func (x EntryType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use EntryType.Descriptor instead. +func (EntryType) EnumDescriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{1} +} + +type Batch struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` + LocalExitRoot []byte `protobuf:"bytes,2,opt,name=local_exit_root,json=localExitRoot,proto3" json:"local_exit_root,omitempty"` + StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` + ForkId uint64 `protobuf:"varint,4,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` + ChainId uint64 `protobuf:"varint,5,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` +} + +func (x *Batch) Reset() { + *x = Batch{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Batch) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Batch) ProtoMessage() {} + +func (x *Batch) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Batch.ProtoReflect.Descriptor instead. +func (*Batch) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{0} +} + +func (x *Batch) GetNumber() uint64 { + if x != nil { + return x.Number + } + return 0 +} + +func (x *Batch) GetLocalExitRoot() []byte { + if x != nil { + return x.LocalExitRoot + } + return nil +} + +func (x *Batch) GetStateRoot() []byte { + if x != nil { + return x.StateRoot + } + return nil +} + +func (x *Batch) GetForkId() uint64 { + if x != nil { + return x.ForkId + } + return 0 +} + +func (x *Batch) GetChainId() uint64 { + if x != nil { + return x.ChainId + } + return 0 +} + +type L2Block struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` + BatchNumber uint64 `protobuf:"varint,2,opt,name=batch_number,json=batchNumber,proto3" json:"batch_number,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + DeltaTimestamp uint32 `protobuf:"varint,4,opt,name=delta_timestamp,json=deltaTimestamp,proto3" json:"delta_timestamp,omitempty"` + MinTimestamp uint64 `protobuf:"varint,5,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp,omitempty"` + L1Blockhash []byte `protobuf:"bytes,6,opt,name=l1_blockhash,json=l1Blockhash,proto3" json:"l1_blockhash,omitempty"` + L1InfotreeIndex uint32 `protobuf:"varint,7,opt,name=l1_infotree_index,json=l1InfotreeIndex,proto3" json:"l1_infotree_index,omitempty"` + Hash []byte `protobuf:"bytes,8,opt,name=hash,proto3" json:"hash,omitempty"` + StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` + GlobalExitRoot []byte `protobuf:"bytes,10,opt,name=global_exit_root,json=globalExitRoot,proto3" json:"global_exit_root,omitempty"` + Coinbase []byte `protobuf:"bytes,11,opt,name=coinbase,proto3" json:"coinbase,omitempty"` +} + +func (x *L2Block) Reset() { + *x = L2Block{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *L2Block) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*L2Block) ProtoMessage() {} + +func (x *L2Block) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use L2Block.ProtoReflect.Descriptor instead. +func (*L2Block) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{1} +} + +func (x *L2Block) GetNumber() uint64 { + if x != nil { + return x.Number + } + return 0 +} + +func (x *L2Block) GetBatchNumber() uint64 { + if x != nil { + return x.BatchNumber + } + return 0 +} + +func (x *L2Block) GetTimestamp() uint64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +func (x *L2Block) GetDeltaTimestamp() uint32 { + if x != nil { + return x.DeltaTimestamp + } + return 0 +} + +func (x *L2Block) GetMinTimestamp() uint64 { + if x != nil { + return x.MinTimestamp + } + return 0 +} + +func (x *L2Block) GetL1Blockhash() []byte { + if x != nil { + return x.L1Blockhash + } + return nil +} + +func (x *L2Block) GetL1InfotreeIndex() uint32 { + if x != nil { + return x.L1InfotreeIndex + } + return 0 +} + +func (x *L2Block) GetHash() []byte { + if x != nil { + return x.Hash + } + return nil +} + +func (x *L2Block) GetStateRoot() []byte { + if x != nil { + return x.StateRoot + } + return nil +} + +func (x *L2Block) GetGlobalExitRoot() []byte { + if x != nil { + return x.GlobalExitRoot + } + return nil +} + +func (x *L2Block) GetCoinbase() []byte { + if x != nil { + return x.Coinbase + } + return nil +} + +type Transaction struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + L2BlockNumber uint64 `protobuf:"varint,1,opt,name=l2block_number,json=l2blockNumber,proto3" json:"l2block_number,omitempty"` + IsValid bool `protobuf:"varint,2,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"` + Encoded []byte `protobuf:"bytes,3,opt,name=encoded,proto3" json:"encoded,omitempty"` + EffectiveGasPricePercentage uint32 `protobuf:"varint,4,opt,name=effective_gas_price_percentage,json=effectiveGasPricePercentage,proto3" json:"effective_gas_price_percentage,omitempty"` + ImStateRoot []byte `protobuf:"bytes,5,opt,name=im_state_root,json=imStateRoot,proto3" json:"im_state_root,omitempty"` +} + +func (x *Transaction) Reset() { + *x = Transaction{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Transaction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Transaction) ProtoMessage() {} + +func (x *Transaction) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Transaction.ProtoReflect.Descriptor instead. +func (*Transaction) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{2} +} + +func (x *Transaction) GetL2BlockNumber() uint64 { + if x != nil { + return x.L2BlockNumber + } + return 0 +} + +func (x *Transaction) GetIsValid() bool { + if x != nil { + return x.IsValid + } + return false +} + +func (x *Transaction) GetEncoded() []byte { + if x != nil { + return x.Encoded + } + return nil +} + +func (x *Transaction) GetEffectiveGasPricePercentage() uint32 { + if x != nil { + return x.EffectiveGasPricePercentage + } + return 0 +} + +func (x *Transaction) GetImStateRoot() []byte { + if x != nil { + return x.ImStateRoot + } + return nil +} + +type UpdateGER struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BatchNumber uint64 `protobuf:"varint,1,opt,name=batch_number,json=batchNumber,proto3" json:"batch_number,omitempty"` + Timestamp uint64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + GlobalExitRoot []byte `protobuf:"bytes,3,opt,name=global_exit_root,json=globalExitRoot,proto3" json:"global_exit_root,omitempty"` + Coinbase []byte `protobuf:"bytes,4,opt,name=coinbase,proto3" json:"coinbase,omitempty"` + ForkId uint64 `protobuf:"varint,5,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` + ChainId uint64 `protobuf:"varint,6,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` + StateRoot []byte `protobuf:"bytes,7,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` +} + +func (x *UpdateGER) Reset() { + *x = UpdateGER{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateGER) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateGER) ProtoMessage() {} + +func (x *UpdateGER) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateGER.ProtoReflect.Descriptor instead. +func (*UpdateGER) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{3} +} + +func (x *UpdateGER) GetBatchNumber() uint64 { + if x != nil { + return x.BatchNumber + } + return 0 +} + +func (x *UpdateGER) GetTimestamp() uint64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +func (x *UpdateGER) GetGlobalExitRoot() []byte { + if x != nil { + return x.GlobalExitRoot + } + return nil +} + +func (x *UpdateGER) GetCoinbase() []byte { + if x != nil { + return x.Coinbase + } + return nil +} + +func (x *UpdateGER) GetForkId() uint64 { + if x != nil { + return x.ForkId + } + return 0 +} + +func (x *UpdateGER) GetChainId() uint64 { + if x != nil { + return x.ChainId + } + return 0 +} + +func (x *UpdateGER) GetStateRoot() []byte { + if x != nil { + return x.StateRoot + } + return nil +} + +type BookMark struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Type BookmarkType `protobuf:"varint,1,opt,name=type,proto3,enum=datastream.v1.BookmarkType" json:"type,omitempty"` + Value uint64 `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *BookMark) Reset() { + *x = BookMark{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BookMark) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BookMark) ProtoMessage() {} + +func (x *BookMark) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BookMark.ProtoReflect.Descriptor instead. +func (*BookMark) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{4} +} + +func (x *BookMark) GetType() BookmarkType { + if x != nil { + return x.Type + } + return BookmarkType_BOOKMARK_TYPE_UNSPECIFIED +} + +func (x *BookMark) GetValue() uint64 { + if x != nil { + return x.Value + } + return 0 +} + +var File_datastream_proto protoreflect.FileDescriptor + +var file_datastream_proto_rawDesc = []byte{ + 0x0a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, + 0x31, 0x22, 0x9a, 0x01, 0x0a, 0x05, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x6e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, + 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6c, 0x6f, + 0x63, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, + 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, + 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x22, 0xf8, + 0x02, 0x0a, 0x07, 0x4c, 0x32, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x64, 0x65, + 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, + 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x31, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, + 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6c, 0x31, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x68, 0x61, 0x73, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x6c, 0x31, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x74, + 0x72, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0f, 0x6c, 0x31, 0x49, 0x6e, 0x66, 0x6f, 0x74, 0x72, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x68, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, + 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x6f, 0x6f, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, + 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, + 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x32, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x0d, 0x6c, 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, + 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6e, + 0x63, 0x6f, 0x64, 0x65, 0x64, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, + 0x76, 0x65, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x65, + 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x47, 0x61, 0x73, 0x50, 0x72, 0x69, 0x63, 0x65, + 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x6d, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x0b, 0x69, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xe5, + 0x01, 0x0a, 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x47, 0x45, 0x52, 0x12, 0x21, 0x0a, 0x0c, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a, + 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, + 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, + 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, + 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, + 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x51, 0x0a, 0x08, 0x42, 0x6f, 0x6f, 0x6b, 0x4d, 0x61, + 0x72, 0x6b, 0x12, 0x2f, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1b, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, + 0x2e, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x2a, 0x62, 0x0a, 0x0c, 0x42, 0x6f, 0x6f, + 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x4f, 0x4f, + 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x42, 0x4f, 0x4f, 0x4b, + 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, + 0x01, 0x12, 0x1a, 0x0a, 0x16, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0x8d, 0x01, + 0x0a, 0x09, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x45, + 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, + 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, 0x54, 0x52, 0x59, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, 0x01, 0x12, 0x17, 0x0a, + 0x13, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, + 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, + 0x10, 0x03, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x45, 0x52, 0x10, 0x04, 0x42, 0x38, 0x5a, + 0x36, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x30, 0x78, 0x50, 0x6f, + 0x6c, 0x79, 0x67, 0x6f, 0x6e, 0x48, 0x65, 0x72, 0x6d, 0x65, 0x7a, 0x2f, 0x7a, 0x6b, 0x65, 0x76, + 0x6d, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x2f, 0x64, 0x61, 0x74, + 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_datastream_proto_rawDescOnce sync.Once + file_datastream_proto_rawDescData = file_datastream_proto_rawDesc +) + +func file_datastream_proto_rawDescGZIP() []byte { + file_datastream_proto_rawDescOnce.Do(func() { + file_datastream_proto_rawDescData = protoimpl.X.CompressGZIP(file_datastream_proto_rawDescData) + }) + return file_datastream_proto_rawDescData +} + +var file_datastream_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_datastream_proto_msgTypes = make([]protoimpl.MessageInfo, 5) +var file_datastream_proto_goTypes = []interface{}{ + (BookmarkType)(0), // 0: datastream.v1.BookmarkType + (EntryType)(0), // 1: datastream.v1.EntryType + (*Batch)(nil), // 2: datastream.v1.Batch + (*L2Block)(nil), // 3: datastream.v1.L2Block + (*Transaction)(nil), // 4: datastream.v1.Transaction + (*UpdateGER)(nil), // 5: datastream.v1.UpdateGER + (*BookMark)(nil), // 6: datastream.v1.BookMark +} +var file_datastream_proto_depIdxs = []int32{ + 0, // 0: datastream.v1.BookMark.type:type_name -> datastream.v1.BookmarkType + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_datastream_proto_init() } +func file_datastream_proto_init() { + if File_datastream_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_datastream_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Batch); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_datastream_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*L2Block); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_datastream_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Transaction); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_datastream_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateGER); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_datastream_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BookMark); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_datastream_proto_rawDesc, + NumEnums: 2, + NumMessages: 5, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_datastream_proto_goTypes, + DependencyIndexes: file_datastream_proto_depIdxs, + EnumInfos: file_datastream_proto_enumTypes, + MessageInfos: file_datastream_proto_msgTypes, + }.Build() + File_datastream_proto = out.File + file_datastream_proto_rawDesc = nil + file_datastream_proto_goTypes = nil + file_datastream_proto_depIdxs = nil +} diff --git a/state/pgstatestorage/datastream.go b/state/pgstatestorage/datastream.go index 4b15000aeb..31c9814c0a 100644 --- a/state/pgstatestorage/datastream.go +++ b/state/pgstatestorage/datastream.go @@ -78,7 +78,7 @@ func scanL2Block(row pgx.Row) (*state.DSL2Block, error) { } l2Block.GlobalExitRoot = common.HexToHash(gerStr) l2Block.Coinbase = common.HexToAddress(coinbaseStr) - l2Block.Timestamp = timestamp.Unix() + l2Block.Timestamp = uint64(timestamp.Unix()) l2Block.BlockHash = common.HexToHash(blockHashStr) l2Block.StateRoot = common.HexToHash(stateRootStr) @@ -151,9 +151,13 @@ func scanDSL2Transaction(row pgx.Row) (*state.DSL2Transaction, error) { // GetDSBatches returns the DS batches func (p *PostgresStorage) GetDSBatches(ctx context.Context, firstBatchNumber, lastBatchNumber uint64, readWIPBatch bool, dbTx pgx.Tx) ([]*state.DSBatch, error) { var getBatchByNumberSQL = ` - SELECT b.batch_num, b.global_exit_root, b.local_exit_root, b.acc_input_hash, b.state_root, b.timestamp, b.coinbase, b.raw_txs_data, b.forced_batch_num, b.wip, f.fork_id - FROM state.batch b, state.fork_id f - WHERE b.batch_num >= $1 AND b.batch_num <= $2 AND batch_num between f.from_batch_num AND f.to_batch_num` + SELECT b.batch_num, b.global_exit_root, b.local_exit_root, b.acc_input_hash, b.state_root, b.timestamp, b.coinbase, b.raw_txs_data, b.forced_batch_num, b.wip, f.fork_id, vb.timestamp_batch_etrog + FROM state.batch b + LEFT JOIN + state.fork_id f ON b.batch_num BETWEEN f.from_batch_num AND f.to_batch_num + LEFT JOIN + state.virtual_batch vb ON b.batch_num = vb.batch_num + WHERE b.batch_num >= $1 AND b.batch_num <= $2` if !readWIPBatch { getBatchByNumberSQL += " AND b.wip is false" @@ -205,6 +209,7 @@ func scanDSBatch(row pgx.Row) (state.DSBatch, error) { &batch.ForcedBatchNum, &batch.WIP, &batch.ForkID, + &batch.EtrogTimestamp, ) if err != nil { return batch, err diff --git a/state/test/datastream_test.go b/state/test/datastream_test.go deleted file mode 100644 index 8860cf9740..0000000000 --- a/state/test/datastream_test.go +++ /dev/null @@ -1,82 +0,0 @@ -package test - -import ( - "fmt" - "testing" - "time" - - "github.com/0xPolygonHermez/zkevm-node/state" - "github.com/ethereum/go-ethereum/common" - "github.com/stretchr/testify/assert" -) - -func TestL2BlockStartEncode(t *testing.T) { - l2BlockStart := state.DSL2BlockStart{ - BatchNumber: 1, - L2BlockNumber: 2, - Timestamp: 3, - DeltaTimestamp: 4, - L1InfoTreeIndex: 5, - L1BlockHash: common.HexToHash("0x06"), - GlobalExitRoot: common.HexToHash("0x07"), - Coinbase: common.HexToAddress("0x08"), - ForkID: 9, - ChainID: 10, - } - - encoded := l2BlockStart.Encode() - expected := []byte{ - 0, 0, 0, 0, 0, 0, 0, 1, - 0, 0, 0, 0, 0, 0, 0, 2, - 0, 0, 0, 0, 0, 0, 0, 3, - 0, 0, 0, 4, - 0, 0, 0, 5, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 6, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 7, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 8, - 0, 9, - 0, 0, 0, 10} - - assert.Equal(t, expected, encoded) -} - -func TestL2TransactionEncode(t *testing.T) { - l2Transaction := state.DSL2Transaction{ - EffectiveGasPricePercentage: 128, // 1 byte - IsValid: 1, // 1 byte - StateRoot: common.HexToHash("0x010203"), // 32 bytes - EncodedLength: 5, // 4 bytes - Encoded: []byte{1, 2, 3, 4, 5}, // 5 bytes - } - - encoded := l2Transaction.Encode() - expected := []byte{128, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 3, 0, 0, 0, 5, 1, 2, 3, 4, 5} - assert.Equal(t, expected, encoded) -} - -func TestL2BlockEndEncode(t *testing.T) { - l2BlockEnd := state.DSL2BlockEnd{ - L2BlockNumber: 1, // 8 bytes - BlockHash: common.HexToHash("0x02"), // 32 bytes - StateRoot: common.HexToHash("0x03"), // 32 bytes - } - - encoded := l2BlockEnd.Encode() - expected := []byte{0, 0, 0, 0, 0, 0, 0, 1, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3} - - assert.Equal(t, expected, encoded) -} - -func TestCalculateSCPosition(t *testing.T) { - a := time.Now() - blockNumber := uint64(2934867) - expected := common.HexToHash("0xaa93c484856be45716623765b429a967296594ca362e61e91d671fb422e0f744") - position := state.GetSystemSCPosition(blockNumber) - assert.Equal(t, expected, common.BytesToHash(position)) - b := time.Now() - - c := b.Sub(a) - fmt.Println(c) -} diff --git a/test/docker-compose.yml b/test/docker-compose.yml index feead65a40..fd82ce7b5f 100644 --- a/test/docker-compose.yml +++ b/test/docker-compose.yml @@ -513,9 +513,12 @@ services: zkevm-prover: container_name: zkevm-prover image: hermeznetwork/zkevm-prover:v6.0.0 + platform: linux/amd64 ports: - 50061:50061 # MT - 50071:50071 # Executor + environment: + - EXPERIMENTAL_DOCKER_DESKTOP_FORCE_QEMU=1 volumes: - ./config/test.prover.config.json:/usr/src/app/config.json command: > @@ -601,7 +604,10 @@ services: zkevm-permissionless-prover: container_name: zkevm-permissionless-prover + platform: linux/amd64 image: hermeznetwork/zkevm-prover:v6.0.0 + environment: + - EXPERIMENTAL_DOCKER_DESKTOP_FORCE_QEMU=1 ports: # - 50058:50058 # Prover - 50059:50052 # Mock prover diff --git a/tools/datastreamer/Makefile b/tools/datastreamer/Makefile index 5698417c34..e79b0cb1a8 100644 --- a/tools/datastreamer/Makefile +++ b/tools/datastreamer/Makefile @@ -6,12 +6,14 @@ check-go: # Targets that require the checks generate-file: check-go -reprocess: check-go decode-entry-offline: check-go decode-l2block-offline: check-go decode-entry: check-go decode-l2block: check-go +decode-batch: check-go +decode-batch-offline: check-go truncate: check-go +dump-batch: check-go arguments := $(wordlist 2,$(words $(MAKECMDGOALS)),$(MAKECMDGOALS)) @@ -27,6 +29,10 @@ decode-entry: ## Runs the tool to decode a given entry number decode-l2block: ## Runs the tool to decode a given L2 block go run main.go decode-l2block -cfg config/tool.config.toml -l2block $(arguments) +.PHONY: decode-batch +decode-batch: ## Runs the tool to decode a given batch + go run main.go decode-batch -cfg config/tool.config.toml -batch $(arguments) + .PHONY: decode-entry-offline decode-entry-offline: ## Runs the offline tool to decode a given entry number go run main.go decode-entry-offline -cfg config/tool.config.toml -entry $(arguments) @@ -35,13 +41,21 @@ decode-entry-offline: ## Runs the offline tool to decode a given entry number decode-l2block-offline: ## Runs the offline tool to decode a given L2 block go run main.go decode-l2block-offline -cfg config/tool.config.toml -l2block $(arguments) +.PHONY: decode-batch-offline +decode-batch-offline: ## Runs the offline tool to decode a given batch + go run main.go decode-batch-offline -cfg config/tool.config.toml -batch $(arguments) + .PHONY: truncate truncate: ## Runs the offline tool to truncate the stream file go run main.go truncate -cfg config/tool.config.toml -entry $(arguments) -# .PHONY: reprocess -reprocess: ## Runs the tool to reprocess the information in the stream since a given l2 block - go run main.go reprocess -cfg config/tool.config.toml -genesis ../test/config/test.genesis.config.json -l2block $(arguments) +.PHONY: dump-batch +dump-batch: ## Runs the tool to dump a given batch to file + go run main.go dump-batch -cfg config/tool.config.toml -d -batch $(arguments) + +.PHONY: dump-batch-offline +dump-batch-offline: ## Runs the tool to dump a given batch to file offline + go run main.go dump-batch-offline -cfg config/tool.config.toml -d -batch $(arguments) ## Help display. ## Pulls comments from beside commands and prints a nicely formatted diff --git a/tools/datastreamer/config/tool.config.toml b/tools/datastreamer/config/tool.config.toml index c497f3362f..0e8fc09fc9 100644 --- a/tools/datastreamer/config/tool.config.toml +++ b/tools/datastreamer/config/tool.config.toml @@ -1,11 +1,11 @@ [Online] -URI = "zkevm-sequencer:6900" +URI = "localhost:6900" StreamType = 1 [Offline] Port = 6901 Filename = "datastream.bin" -Version = 1 +Version = 3 ChainID = 1440 UpgradeEtrogBatchNumber = 0 @@ -18,12 +18,8 @@ Port = "5432" EnableLog = false MaxConns = 200 -[Executor] -URI = "zkevm-prover:50071" -MaxGRPCMessageSize = 100000000 - [MerkleTree] -URI = "zkevm-prover:50061" +URI = "localhost:50061" MaxThreads = 20 CacheFile = "merkle_tree_cache.json" diff --git a/tools/datastreamer/main.go b/tools/datastreamer/main.go index 975e4c7ecd..bb84aeebf3 100644 --- a/tools/datastreamer/main.go +++ b/tools/datastreamer/main.go @@ -2,7 +2,6 @@ package main import ( "context" - "encoding/binary" "encoding/json" "fmt" "math/big" @@ -12,19 +11,16 @@ import ( "github.com/0xPolygonHermez/zkevm-data-streamer/datastreamer" "github.com/0xPolygonHermez/zkevm-data-streamer/log" - nodeConfig "github.com/0xPolygonHermez/zkevm-node/config" "github.com/0xPolygonHermez/zkevm-node/db" - "github.com/0xPolygonHermez/zkevm-node/encoding" - "github.com/0xPolygonHermez/zkevm-node/hex" "github.com/0xPolygonHermez/zkevm-node/merkletree" "github.com/0xPolygonHermez/zkevm-node/state" + "github.com/0xPolygonHermez/zkevm-node/state/datastream" "github.com/0xPolygonHermez/zkevm-node/state/pgstatestorage" - "github.com/0xPolygonHermez/zkevm-node/state/runtime/executor" "github.com/0xPolygonHermez/zkevm-node/tools/datastreamer/config" "github.com/ethereum/go-ethereum/common" "github.com/fatih/color" - "github.com/google/uuid" "github.com/urfave/cli/v2" + "google.golang.org/protobuf/proto" ) const ( @@ -41,14 +37,6 @@ var ( Required: true, } - genesisFileFlag = cli.StringFlag{ - Name: config.FlagGenesis, - Aliases: []string{"g"}, - Usage: "Genesis `FILE`", - DefaultText: "./config/genesis.json", - Required: true, - } - entryFlag = cli.Uint64Flag{ Name: "entry", Aliases: []string{"e"}, @@ -63,10 +51,17 @@ var ( Required: true, } - updateFileFlag = cli.BoolFlag{ - Name: "update", - Aliases: []string{"u"}, - Usage: "Update `FILE`", + batchFlag = cli.Uint64Flag{ + Name: "batch", + Aliases: []string{"bn"}, + Usage: "Batch `NUMBER`", + Required: true, + } + + dumpFlag = cli.BoolFlag{ + Name: "dump", + Aliases: []string{"d"}, + Usage: "Dump batch to file", Required: false, } ) @@ -86,18 +81,6 @@ func main() { &configFileFlag, }, }, - { - Name: "reprocess", - Aliases: []string{}, - Usage: "Reprocess l2block since a given l2block number", - Action: reprocess, - Flags: []cli.Flag{ - &configFileFlag, - &genesisFileFlag, - &l2blockFlag, - &updateFileFlag, - }, - }, { Name: "decode-entry-offline", Aliases: []string{}, @@ -118,6 +101,16 @@ func main() { &l2blockFlag, }, }, + { + Name: "decode-batch-offline", + Aliases: []string{}, + Usage: "Decodes a batch offline", + Action: decodeBatchOffline, + Flags: []cli.Flag{ + &configFileFlag, + &batchFlag, + }, + }, { Name: "decode-entry", Aliases: []string{}, @@ -138,6 +131,16 @@ func main() { &l2blockFlag, }, }, + { + Name: "decode-batch", + Aliases: []string{}, + Usage: "Decodes a batch", + Action: decodeBatch, + Flags: []cli.Flag{ + &configFileFlag, + &batchFlag, + }, + }, { Name: "truncate", Aliases: []string{}, @@ -148,6 +151,28 @@ func main() { &entryFlag, }, }, + { + Name: "dump-batch", + Aliases: []string{}, + Usage: "Dumps a batch to file", + Action: decodeBatch, + Flags: []cli.Flag{ + &configFileFlag, + &batchFlag, + &dumpFlag, + }, + }, + { + Name: "dump-batch-offline", + Aliases: []string{}, + Usage: "Dumps a batch to file offline", + Action: decodeBatchOffline, + Flags: []cli.Flag{ + &configFileFlag, + &batchFlag, + &dumpFlag, + }, + }, } err := app.Run(os.Args) @@ -272,7 +297,7 @@ func generate(cliCtx *cli.Context) error { } } - err = state.GenerateDataStreamerFile(cliCtx.Context, streamServer, stateDB, false, &imStateRoots, c.Offline.ChainID, c.Offline.UpgradeEtrogBatchNumber) // nolint:gomnd + err = state.GenerateDataStreamFile(cliCtx.Context, streamServer, stateDB, false, &imStateRoots, c.Offline.ChainID, c.Offline.UpgradeEtrogBatchNumber) if err != nil { log.Error(err) os.Exit(1) @@ -299,13 +324,18 @@ func getImStateRoots(ctx context.Context, start, end uint64, isStateRoots *map[u log.Errorf("Error: %v\n", err) os.Exit(1) } + + if common.BytesToHash(imStateRoot.Bytes()) == state.ZeroHash && x != 0 { + break + } + imStateRootMux.Lock() (*isStateRoots)[x] = imStateRoot.Bytes() imStateRootMux.Unlock() } } -func reprocess(cliCtx *cli.Context) error { +func decodeEntry(cliCtx *cli.Context) error { c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -314,230 +344,90 @@ func reprocess(cliCtx *cli.Context) error { log.Init(c.Log) - ctx := cliCtx.Context - - genesisFileAsStr, err := nodeConfig.LoadGenesisFileAsString(cliCtx.String(config.FlagGenesis)) + client, err := datastreamer.NewClient(c.Online.URI, c.Online.StreamType) if err != nil { - fmt.Printf("failed to load genesis file. Error: %v", err) + log.Error(err) os.Exit(1) } - networkConfig, err := nodeConfig.LoadGenesisFromJSONString(genesisFileAsStr) + err = client.Start() if err != nil { - fmt.Printf("failed to load genesis configuration from file. Error: %v", err) + log.Error(err) os.Exit(1) } - currentL2BlockNumber := cliCtx.Uint64("l2block") - var stateRoot []byte - - streamServer, err := initializeStreamServer(c) + entry, err := client.ExecCommandGetEntry(cliCtx.Uint64("entry")) if err != nil { log.Error(err) os.Exit(1) } - if currentL2BlockNumber == 0 { - printColored(color.FgHiYellow, "\n\nSetting Genesis block\n\n") - - mtDBServerConfig := merkletree.Config{URI: c.MerkleTree.URI} - var mtDBCancel context.CancelFunc - mtDBServiceClient, mtDBClientConn, mtDBCancel := merkletree.NewMTDBServiceClient(ctx, mtDBServerConfig) - defer func() { - mtDBCancel() - mtDBClientConn.Close() - }() - - stateTree := merkletree.NewStateTree(mtDBServiceClient) - - stateRoot, err = setGenesis(ctx, stateTree, networkConfig.Genesis) - if err != nil { - log.Error(err) - os.Exit(1) - } - - // Get Genesis block from the file and validate the state root - bookMark := state.DSBookMark{ - Type: state.BookMarkTypeL2Block, - Value: 0, - } - - firstEntry, err := streamServer.GetFirstEventAfterBookmark(bookMark.Encode()) - if err != nil { - log.Error(err) - os.Exit(1) - } - printEntry(firstEntry) - - secondEntry, err := streamServer.GetEntry(firstEntry.Number + 1) - if err != nil { - log.Error(err) - os.Exit(1) - } - printEntry(secondEntry) + printEntry(entry) + return nil +} - if common.Bytes2Hex(stateRoot) != common.Bytes2Hex(secondEntry.Data[40:72]) { - printColored(color.FgRed, "\nError: Genesis state root does not match\n\n") - os.Exit(1) - } else { - printColored(color.FgGreen, "\nGenesis state root matches\n\n") - } - currentL2BlockNumber++ +func decodeL2Block(cliCtx *cli.Context) error { + c, err := config.Load(cliCtx) + if err != nil { + log.Error(err) + os.Exit(1) } - // Connect to the executor - executorClient, executorClientConn, executorCancel := executor.NewExecutorClient(ctx, c.Executor) - defer func() { - executorCancel() - executorClientConn.Close() - }() + log.Init(c.Log) - bookMark := state.DSBookMark{ - Type: state.BookMarkTypeL2Block, - Value: currentL2BlockNumber, + client, err := datastreamer.NewClient(c.Online.URI, c.Online.StreamType) + if err != nil { + log.Error(err) + os.Exit(1) } - startEntry, err := streamServer.GetFirstEventAfterBookmark(bookMark.Encode()) + err = client.Start() if err != nil { log.Error(err) os.Exit(1) } - var previousStateRoot = stateRoot - var maxEntry = streamServer.GetHeader().TotalEntries - - for x := startEntry.Number; x < maxEntry; x++ { - printColored(color.FgHiYellow, fmt.Sprintf("\nProcessing entity: %d\n", x)) - - currentEntry, err := streamServer.GetEntry(x) - if err != nil { - log.Error(err) - os.Exit(1) - } - - var processBatchRequest *executor.ProcessBatchRequest - var expectedNewRoot []byte - var entryToUpdate *datastreamer.FileEntry - - switch currentEntry.Type { - case state.EntryTypeBookMark: - printEntry(currentEntry) - entryToUpdate = nil - continue - case state.EntryTypeUpdateGER: - printEntry(currentEntry) - processBatchRequest = &executor.ProcessBatchRequest{ - OldBatchNum: binary.BigEndian.Uint64(currentEntry.Data[0:8]) - 1, - Coinbase: common.Bytes2Hex(currentEntry.Data[48:68]), - BatchL2Data: nil, - OldStateRoot: previousStateRoot, - GlobalExitRoot: currentEntry.Data[16:48], - OldAccInputHash: []byte{}, - EthTimestamp: binary.BigEndian.Uint64(currentEntry.Data[8:16]), - UpdateMerkleTree: uint32(1), - ChainId: c.Offline.ChainID, - ForkId: uint64(binary.BigEndian.Uint16(currentEntry.Data[68:70])), - } - - expectedNewRoot = currentEntry.Data[70:102] - entryToUpdate = nil - case state.EntryTypeL2BlockStart: - startEntry = currentEntry - printEntry(startEntry) - - txEntry, err := streamServer.GetEntry(startEntry.Number + 1) - if err != nil { - log.Error(err) - os.Exit(1) - } - printEntry(txEntry) - - endEntry, err := streamServer.GetEntry(startEntry.Number + 2) //nolint:gomnd - if err != nil { - log.Error(err) - os.Exit(1) - } - printEntry(endEntry) - - forkID := uint64(binary.BigEndian.Uint16(startEntry.Data[76:78])) - - tx, err := state.DecodeTx(common.Bytes2Hex((txEntry.Data[6:]))) - if err != nil { - log.Error(err) - os.Exit(1) - } + l2BlockNumber := cliCtx.Uint64("l2block") - // Get the old state root - oldStateRoot := getOldStateRoot(startEntry.Number, streamServer) + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, + Value: l2BlockNumber, + } - // RLP encode the transaction using the proper fork id - batchL2Data, err := state.EncodeTransaction(*tx, txEntry.Data[0], forkID) //nolint:gomnd - if err != nil { - log.Error(err) - os.Exit(1) - } + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err + } - processBatchRequest = &executor.ProcessBatchRequest{ - OldBatchNum: binary.BigEndian.Uint64(startEntry.Data[0:8]) - 1, - Coinbase: common.Bytes2Hex(startEntry.Data[56:76]), - BatchL2Data: batchL2Data, - OldStateRoot: oldStateRoot, - GlobalExitRoot: startEntry.Data[24:56], - OldAccInputHash: []byte{}, - EthTimestamp: binary.BigEndian.Uint64(startEntry.Data[16:24]), - UpdateMerkleTree: uint32(1), - ChainId: c.Offline.ChainID, - ForkId: uint64(binary.BigEndian.Uint16(startEntry.Data[76:78])), - } + firstEntry, err := client.ExecCommandGetBookmark(marshalledBookMark) + if err != nil { + log.Error(err) + os.Exit(1) + } + printEntry(firstEntry) - expectedNewRoot = endEntry.Data[40:72] - entryToUpdate = &endEntry - x += 2 //nolint:gomnd - } + secondEntry, err := client.ExecCommandGetEntry(firstEntry.Number + 1) + if err != nil { + log.Error(err) + os.Exit(1) + } - // Process batch - processBatchResponse, err := executorClient.ProcessBatch(ctx, processBatchRequest) + i := uint64(2) //nolint:gomnd + for secondEntry.Type == datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION) { + printEntry(secondEntry) + entry, err := client.ExecCommandGetEntry(firstEntry.Number + i) if err != nil { log.Error(err) os.Exit(1) } - - if processBatchResponse.Error != executor.ExecutorError_EXECUTOR_ERROR_NO_ERROR { - fmt.Printf("Error: %v\n", processBatchResponse.Error) - os.Exit(1) - } - - if common.Bytes2Hex(processBatchResponse.NewStateRoot) != common.Bytes2Hex(expectedNewRoot) { - printColored(color.FgRed, "\nNew state root does not match\n\n") - printColored(color.FgRed, fmt.Sprintf("Old State Root.........: %s\n", "0x"+common.Bytes2Hex(processBatchRequest.GetOldStateRoot()))) - printColored(color.FgRed, fmt.Sprintf("New State Root.........: %s\n", "0x"+common.Bytes2Hex(processBatchResponse.NewStateRoot))) - printColored(color.FgRed, fmt.Sprintf("Expected New State Root: %s\n", "0x"+common.Bytes2Hex(expectedNewRoot))) - // Check if we must update the file with the new state root - if cliCtx.Bool("update") { - if entryToUpdate.Type != state.EntryTypeL2BlockEnd { - printColored(color.FgRed, "Error: Entry to update is not a L2BlockEnd\n") - os.Exit(1) - } - blockEnd := state.DSL2BlockEnd{}.Decode(entryToUpdate.Data) - blockEnd.StateRoot = common.BytesToHash(processBatchResponse.NewStateRoot) - err = streamServer.UpdateEntryData(entryToUpdate.Number, state.EntryTypeL2BlockEnd, blockEnd.Encode()) - if err != nil { - printColored(color.FgRed, fmt.Sprintf("Error: %v\n", err)) - os.Exit(1) - } - } else { - break - } - } else { - printColored(color.FgGreen, "New state root matches\n") - previousStateRoot = processBatchResponse.NewStateRoot - } + secondEntry = entry + i++ } return nil } -func decodeEntry(cliCtx *cli.Context) error { +func decodeEntryOffline(cliCtx *cli.Context) error { c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -546,29 +436,24 @@ func decodeEntry(cliCtx *cli.Context) error { log.Init(c.Log) - client, err := datastreamer.NewClient(c.Online.URI, c.Online.StreamType) - if err != nil { - log.Error(err) - os.Exit(1) - } - - err = client.Start() + streamServer, err := initializeStreamServer(c) if err != nil { log.Error(err) os.Exit(1) } - entry, err := client.ExecCommandGetEntry(cliCtx.Uint64("entry")) + entry, err := streamServer.GetEntry(cliCtx.Uint64("entry")) if err != nil { log.Error(err) os.Exit(1) } printEntry(entry) + return nil } -func decodeL2Block(cliCtx *cli.Context) error { +func decodeL2BlockOffline(cliCtx *cli.Context) error { c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -577,13 +462,7 @@ func decodeL2Block(cliCtx *cli.Context) error { log.Init(c.Log) - client, err := datastreamer.NewClient(c.Online.URI, c.Online.StreamType) - if err != nil { - log.Error(err) - os.Exit(1) - } - - err = client.Start() + streamServer, err := initializeStreamServer(c) if err != nil { log.Error(err) os.Exit(1) @@ -591,41 +470,45 @@ func decodeL2Block(cliCtx *cli.Context) error { l2BlockNumber := cliCtx.Uint64("l2block") - bookMark := state.DSBookMark{ - Type: state.BookMarkTypeL2Block, + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, Value: l2BlockNumber, } - firstEntry, err := client.ExecCommandGetBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err + } + + firstEntry, err := streamServer.GetFirstEventAfterBookmark(marshalledBookMark) if err != nil { log.Error(err) os.Exit(1) } printEntry(firstEntry) - secondEntry, err := client.ExecCommandGetEntry(firstEntry.Number + 1) + secondEntry, err := streamServer.GetEntry(firstEntry.Number + 1) if err != nil { log.Error(err) os.Exit(1) } - printEntry(secondEntry) i := uint64(2) //nolint:gomnd - for secondEntry.Type == state.EntryTypeL2Tx { - entry, err := client.ExecCommandGetEntry(firstEntry.Number + i) + + for secondEntry.Type == datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION) { + printEntry(secondEntry) + secondEntry, err = streamServer.GetEntry(firstEntry.Number + i) if err != nil { log.Error(err) os.Exit(1) } - secondEntry = entry - printEntry(secondEntry) i++ } return nil } -func decodeEntryOffline(cliCtx *cli.Context) error { +func truncate(cliCtx *cli.Context) error { c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -640,18 +523,19 @@ func decodeEntryOffline(cliCtx *cli.Context) error { os.Exit(1) } - entry, err := streamServer.GetEntry(cliCtx.Uint64("entry")) + err = streamServer.TruncateFile(cliCtx.Uint64("entry")) if err != nil { log.Error(err) os.Exit(1) } - printEntry(entry) + printColored(color.FgGreen, "File truncated\n") return nil } -func decodeL2BlockOffline(cliCtx *cli.Context) error { +func decodeBatch(cliCtx *cli.Context) error { + var batchData = []byte{} c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -660,48 +544,85 @@ func decodeL2BlockOffline(cliCtx *cli.Context) error { log.Init(c.Log) - streamServer, err := initializeStreamServer(c) + client, err := datastreamer.NewClient(c.Online.URI, c.Online.StreamType) if err != nil { log.Error(err) os.Exit(1) } - l2BlockNumber := cliCtx.Uint64("l2block") + err = client.Start() + if err != nil { + log.Error(err) + os.Exit(1) + } - bookMark := state.DSBookMark{ - Type: state.BookMarkTypeL2Block, - Value: l2BlockNumber, + batchNumber := cliCtx.Uint64("batch") + + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, + Value: batchNumber, } - firstEntry, err := streamServer.GetFirstEventAfterBookmark(bookMark.Encode()) + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err + } + + firstEntry, err := client.ExecCommandGetBookmark(marshalledBookMark) if err != nil { log.Error(err) os.Exit(1) } printEntry(firstEntry) - secondEntry, err := streamServer.GetEntry(firstEntry.Number + 1) + batchData = append(batchData, firstEntry.Encode()...) + + secondEntry, err := client.ExecCommandGetEntry(firstEntry.Number + 1) if err != nil { log.Error(err) os.Exit(1) } + printEntry(secondEntry) + + batchData = append(batchData, secondEntry.Encode()...) i := uint64(2) //nolint:gomnd - printEntry(secondEntry) - for secondEntry.Type == state.EntryTypeL2Tx { - secondEntry, err = streamServer.GetEntry(firstEntry.Number + i) + for { + entry, err := client.ExecCommandGetEntry(firstEntry.Number + i) if err != nil { log.Error(err) os.Exit(1) } + + if entry.Type == state.EntryTypeBookMark { + if err := proto.Unmarshal(entry.Data, bookMark); err != nil { + return err + } + if bookMark.Type == datastream.BookmarkType_BOOKMARK_TYPE_BATCH { + break + } + } + + secondEntry = entry printEntry(secondEntry) + batchData = append(batchData, secondEntry.Encode()...) i++ } + // Dump batchdata to a file + if cliCtx.Bool("dump") { + err = os.WriteFile(fmt.Sprintf("batch_%d.bin", batchNumber), batchData, 0644) // nolint:gosec, gomnd + if err != nil { + log.Error(err) + os.Exit(1) + } + } + return nil } -func truncate(cliCtx *cli.Context) error { +func decodeBatchOffline(cliCtx *cli.Context) error { + var batchData = []byte{} c, err := config.Load(cliCtx) if err != nil { log.Error(err) @@ -716,26 +637,84 @@ func truncate(cliCtx *cli.Context) error { os.Exit(1) } - err = streamServer.TruncateFile(cliCtx.Uint64("entry")) + batchNumber := cliCtx.Uint64("batch") + + bookMark := &datastream.BookMark{ + Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, + Value: batchNumber, + } + + marshalledBookMark, err := proto.Marshal(bookMark) + if err != nil { + return err + } + + firstEntry, err := streamServer.GetFirstEventAfterBookmark(marshalledBookMark) if err != nil { log.Error(err) os.Exit(1) } + printEntry(firstEntry) + batchData = append(batchData, firstEntry.Encode()...) - printColored(color.FgGreen, "File truncated\n") + secondEntry, err := streamServer.GetEntry(firstEntry.Number + 1) + if err != nil { + log.Error(err) + os.Exit(1) + } + + i := uint64(2) //nolint:gomnd + printEntry(secondEntry) + batchData = append(batchData, secondEntry.Encode()...) + for { + secondEntry, err = streamServer.GetEntry(firstEntry.Number + i) + if err != nil { + log.Error(err) + os.Exit(1) + } + + if secondEntry.Type == state.EntryTypeBookMark { + if err := proto.Unmarshal(secondEntry.Data, bookMark); err != nil { + return err + } + if bookMark.Type == datastream.BookmarkType_BOOKMARK_TYPE_BATCH { + break + } + } + + printEntry(secondEntry) + batchData = append(batchData, secondEntry.Encode()...) + i++ + } + + // Dump batchdata to a file + if cliCtx.Bool("dump") { + err = os.WriteFile(fmt.Sprintf("offline_batch_%d.bin", batchNumber), batchData, 0644) // nolint:gosec, gomnd + if err != nil { + log.Error(err) + os.Exit(1) + } + } return nil } func printEntry(entry datastreamer.FileEntry) { - var bookmarkTypeDesc = map[byte]string{ - state.BookMarkTypeL2Block: "L2 Block Number", - state.BookMarkTypeBatch: "Batch Number", + var bookmarkTypeDesc = map[datastream.BookmarkType]string{ + datastream.BookmarkType_BOOKMARK_TYPE_UNSPECIFIED: "Unspecified", + datastream.BookmarkType_BOOKMARK_TYPE_BATCH: "Batch Number", + datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK: "L2 Block Number", } switch entry.Type { case state.EntryTypeBookMark: - bookmark := state.DSBookMark{}.Decode(entry.Data) + bookmark := &datastream.BookMark{} + err := proto.Unmarshal(entry.Data, bookmark) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") printColored(color.FgHiYellow, "BookMark\n") printColored(color.FgGreen, "Entry Number....: ") @@ -744,48 +723,83 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%d (%s)\n", bookmark.Type, bookmarkTypeDesc[bookmark.Type])) printColored(color.FgGreen, "Value...........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", bookmark.Value)) - case state.EntryTypeL2BlockStart: - blockStart := state.DSL2BlockStart{}.Decode(entry.Data) + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK): + l2Block := &datastream.L2Block{} + err := proto.Unmarshal(entry.Data, l2Block) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") - printColored(color.FgHiYellow, "L2 Block Start\n") + printColored(color.FgHiYellow, "L2 Block\n") printColored(color.FgGreen, "Entry Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) - printColored(color.FgGreen, "Batch Number....: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.BatchNumber)) printColored(color.FgGreen, "L2 Block Number.: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.L2BlockNumber)) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.Number)) + printColored(color.FgGreen, "Batch Number....: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.BatchNumber)) printColored(color.FgGreen, "Timestamp.......: ") - printColored(color.FgHiWhite, fmt.Sprintf("%v (%d)\n", time.Unix(blockStart.Timestamp, 0), blockStart.Timestamp)) + printColored(color.FgHiWhite, fmt.Sprintf("%d (%v)\n", l2Block.Timestamp, time.Unix(int64(l2Block.Timestamp), 0))) printColored(color.FgGreen, "Delta Timestamp.: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.DeltaTimestamp)) - printColored(color.FgGreen, "L1 InfoTree Idx.: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.L1InfoTreeIndex)) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.DeltaTimestamp)) + printColored(color.FgGreen, "Min. Timestamp..: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.MinTimestamp)) printColored(color.FgGreen, "L1 Block Hash...: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", blockStart.L1BlockHash)) + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.L1Blockhash))) + printColored(color.FgGreen, "L1 InfoTree Idx.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.L1InfotreeIndex)) + printColored(color.FgGreen, "Block Hash......: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.Hash))) + printColored(color.FgGreen, "State Root......: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.StateRoot))) printColored(color.FgGreen, "Global Exit Root: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", blockStart.GlobalExitRoot)) + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.GlobalExitRoot))) printColored(color.FgGreen, "Coinbase........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", blockStart.Coinbase)) + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToAddress(l2Block.Coinbase))) + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH): + batch := &datastream.Batch{} + err := proto.Unmarshal(entry.Data, batch) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") + printColored(color.FgHiYellow, "Batch\n") + printColored(color.FgGreen, "Entry Number....: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) + printColored(color.FgGreen, "Batch Number....: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.Number)) + printColored(color.FgGreen, "State Root......: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.StateRoot))) + printColored(color.FgGreen, "Local Exit Root.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.LocalExitRoot))) printColored(color.FgGreen, "Fork ID.........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.ForkID)) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ForkId)) printColored(color.FgGreen, "Chain ID........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockStart.ChainID)) - case state.EntryTypeL2Tx: - dsTx := state.DSL2Transaction{}.Decode(entry.Data) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ChainId)) + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION): + dsTx := &datastream.Transaction{} + err := proto.Unmarshal(entry.Data, dsTx) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") printColored(color.FgHiYellow, "L2 Transaction\n") printColored(color.FgGreen, "Entry Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) + printColored(color.FgGreen, "L2 Block Number.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", dsTx.L2BlockNumber)) + printColored(color.FgGreen, "Is Valid........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%t\n", dsTx.IsValid)) + printColored(color.FgGreen, "Data............: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(dsTx.Encoded))) printColored(color.FgGreen, "Effec. Gas Price: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", dsTx.EffectiveGasPricePercentage)) - printColored(color.FgGreen, "Is Valid........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%t\n", dsTx.IsValid == 1)) printColored(color.FgGreen, "IM State Root...: ") - printColored(color.FgHiWhite, fmt.Sprint(dsTx.StateRoot.Hex()+"\n")) - printColored(color.FgGreen, "Encoded Length..: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", dsTx.EncodedLength)) - printColored(color.FgGreen, "Encoded.........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(dsTx.Encoded))) + printColored(color.FgHiWhite, fmt.Sprint("0x"+common.Bytes2Hex(dsTx.ImStateRoot)+"\n")) tx, err := state.DecodeTx(common.Bytes2Hex(dsTx.Encoded)) if err != nil { @@ -804,20 +818,14 @@ func printEntry(entry datastreamer.FileEntry) { nonce := tx.Nonce() printColored(color.FgGreen, "Nonce...........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", nonce)) - case state.EntryTypeL2BlockEnd: - blockEnd := state.DSL2BlockEnd{}.Decode(entry.Data) - printColored(color.FgGreen, "Entry Type......: ") - printColored(color.FgHiYellow, "L2 Block End\n") - printColored(color.FgGreen, "Entry Number....: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) - printColored(color.FgGreen, "L2 Block Number.: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", blockEnd.L2BlockNumber)) - printColored(color.FgGreen, "L2 Block Hash...: ") - printColored(color.FgHiWhite, fmt.Sprint(blockEnd.BlockHash.Hex()+"\n")) - printColored(color.FgGreen, "State Root......: ") - printColored(color.FgHiWhite, fmt.Sprint(blockEnd.StateRoot.Hex()+"\n")) - case state.EntryTypeUpdateGER: - updateGer := state.DSUpdateGER{}.Decode(entry.Data) + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_UPDATE_GER): + updateGer := &datastream.UpdateGER{} + err := proto.Unmarshal(entry.Data, updateGer) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") printColored(color.FgHiYellow, "Update GER\n") printColored(color.FgGreen, "Entry Number....: ") @@ -825,17 +833,17 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgGreen, "Batch Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.BatchNumber)) printColored(color.FgGreen, "Timestamp.......: ") - printColored(color.FgHiWhite, fmt.Sprintf("%v (%d)\n", time.Unix(updateGer.Timestamp, 0), updateGer.Timestamp)) + printColored(color.FgHiWhite, fmt.Sprintf("%v (%d)\n", time.Unix(int64(updateGer.Timestamp), 0), updateGer.Timestamp)) printColored(color.FgGreen, "Global Exit Root: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", updateGer.GlobalExitRoot)) + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.Bytes2Hex(updateGer.GlobalExitRoot))) printColored(color.FgGreen, "Coinbase........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", updateGer.Coinbase)) + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToAddress(updateGer.Coinbase))) printColored(color.FgGreen, "Fork ID.........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.ForkID)) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.ForkId)) printColored(color.FgGreen, "Chain ID........: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.ChainID)) + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.ChainId)) printColored(color.FgGreen, "State Root......: ") - printColored(color.FgHiWhite, fmt.Sprint(updateGer.StateRoot.Hex()+"\n")) + printColored(color.FgHiWhite, fmt.Sprint(common.Bytes2Hex(updateGer.StateRoot)+"\n")) } } @@ -843,112 +851,3 @@ func printColored(color color.Attribute, text string) { colored := fmt.Sprintf("\x1b[%dm%s\x1b[0m", color, text) fmt.Print(colored) } - -// setGenesis populates state with genesis information -func setGenesis(ctx context.Context, tree *merkletree.StateTree, genesis state.Genesis) ([]byte, error) { - var ( - root common.Hash - newRoot []byte - err error - ) - - if tree == nil { - return newRoot, fmt.Errorf("state tree is nil") - } - - uuid := uuid.New().String() - - for _, action := range genesis.Actions { - address := common.HexToAddress(action.Address) - switch action.Type { - case int(merkletree.LeafTypeBalance): - balance, err := encoding.DecodeBigIntHexOrDecimal(action.Value) - if err != nil { - return newRoot, err - } - newRoot, _, err = tree.SetBalance(ctx, address, balance, newRoot, uuid) - if err != nil { - return newRoot, err - } - case int(merkletree.LeafTypeNonce): - nonce, err := encoding.DecodeBigIntHexOrDecimal(action.Value) - if err != nil { - return newRoot, err - } - newRoot, _, err = tree.SetNonce(ctx, address, nonce, newRoot, uuid) - if err != nil { - return newRoot, err - } - case int(merkletree.LeafTypeCode): - code, err := hex.DecodeHex(action.Bytecode) - if err != nil { - return newRoot, fmt.Errorf("could not decode SC bytecode for address %q: %v", address, err) - } - newRoot, _, err = tree.SetCode(ctx, address, code, newRoot, uuid) - if err != nil { - return newRoot, err - } - case int(merkletree.LeafTypeStorage): - // Parse position and value - positionBI, err := encoding.DecodeBigIntHexOrDecimal(action.StoragePosition) - if err != nil { - return newRoot, err - } - valueBI, err := encoding.DecodeBigIntHexOrDecimal(action.Value) - if err != nil { - return newRoot, err - } - // Store - newRoot, _, err = tree.SetStorageAt(ctx, address, positionBI, valueBI, newRoot, uuid) - if err != nil { - return newRoot, err - } - default: - return newRoot, fmt.Errorf("unknown genesis action type %q", action.Type) - } - } - - root.SetBytes(newRoot) - - // flush state db - err = tree.Flush(ctx, root, uuid) - if err != nil { - fmt.Printf("error flushing state tree after genesis: %v", err) - return newRoot, err - } - - return newRoot, nil -} - -func getOldStateRoot(entityNumber uint64, streamServer *datastreamer.StreamServer) []byte { - var found = false - var entry datastreamer.FileEntry - var err error - - for !found && entityNumber > 1 { - entityNumber-- - entry, err = streamServer.GetEntry(entityNumber) - if err != nil { - log.Error(err) - os.Exit(1) - } - - if entry.Type == state.EntryTypeL2BlockEnd || entry.Type == state.EntryTypeUpdateGER { - found = true - } - } - - if !found { - fmt.Printf("Error: Could not find old state root") - os.Exit(1) - } - - printColored(color.FgHiYellow, "Getting Old State Root from\n") - printEntry(entry) - - if entry.Type == state.EntryTypeUpdateGER { - return entry.Data[70:102] - } - - return entry.Data[40:72] -} From 3aac36be481a9ec5db8fd1f77c58d9eff0757858 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Thu, 2 May 2024 10:56:09 +0200 Subject: [PATCH 02/22] proto batch end (#3612) --- .../src/proto/datastream/v1/datastream.proto | 15 +- sequencer/batch.go | 4 +- sequencer/datastreamer.go | 17 +- sequencer/sequencer.go | 39 +- state/datastream.go | 67 +++- state/datastream/datastream.pb.go | 348 +++++++++++------- tools/datastreamer/main.go | 27 +- 7 files changed, 338 insertions(+), 179 deletions(-) diff --git a/proto/src/proto/datastream/v1/datastream.proto b/proto/src/proto/datastream/v1/datastream.proto index 6b8a89adc6..555bc17733 100644 --- a/proto/src/proto/datastream/v1/datastream.proto +++ b/proto/src/proto/datastream/v1/datastream.proto @@ -4,14 +4,18 @@ package datastream.v1; option go_package = "github.com/0xPolygonHermez/zkevm-node/state/datastream"; -message Batch { +message BatchStart { uint64 number = 1; - bytes local_exit_root = 2; - bytes state_root = 3; uint64 fork_id = 4; uint64 chain_id = 5; } +message BatchEnd { + uint64 number = 1; + bytes local_exit_root = 2; + bytes state_root = 3; +} + message L2Block { uint64 number = 1; uint64 batch_number = 2; @@ -57,8 +61,9 @@ enum BookmarkType { enum EntryType { ENTRY_TYPE_UNSPECIFIED = 0; - ENTRY_TYPE_BATCH = 1; + ENTRY_TYPE_BATCH_START = 1; ENTRY_TYPE_L2_BLOCK = 2; ENTRY_TYPE_TRANSACTION = 3; - ENTRY_TYPE_UPDATE_GER = 4; + ENTRY_TYPE_BATCH_END = 4; + ENTRY_TYPE_UPDATE_GER = 5; } diff --git a/sequencer/batch.go b/sequencer/batch.go index 8db3320a4d..cc2bea283a 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -339,6 +339,8 @@ func (f *finalizer) insertSIPBatch(ctx context.Context, batchNumber uint64, stat // Send batch bookmark to the datastream f.DSSendBatchBookmark(batchNumber) + // Send batch start to the datastream + f.DSSendBatchStart(batchNumber) // Check if synchronizer is up-to-date //TODO: review if this is needed @@ -404,7 +406,7 @@ func (f *finalizer) closeSIPBatch(ctx context.Context, dbTx pgx.Tx) error { } // Sent batch to DS - f.DSSendBatch(f.wipBatch.batchNumber, f.wipBatch.finalStateRoot, f.wipBatch.finalLocalExitRoot) + f.DSSendBatchEnd(f.wipBatch.batchNumber, f.wipBatch.finalStateRoot, f.wipBatch.finalLocalExitRoot) log.Infof("sip batch %d closed in statedb, closing reason: %s", f.sipBatch.batchNumber, f.sipBatch.closingReason) diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 81ed1083b0..8178593978 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -67,14 +67,23 @@ func (f *finalizer) DSSendBatchBookmark(batchNumber uint64) { } } -func (f *finalizer) DSSendBatch(batchNumber uint64, stateRoot common.Hash, localExitRoot common.Hash) { +func (f *finalizer) DSSendBatchStart(batchNumber uint64) { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) if f.streamServer != nil { - // Send batch to the streamer - f.dataToStream <- datastream.Batch{ + // Send batch start to the streamer + f.dataToStream <- datastream.BatchStart{ + Number: batchNumber, + ForkId: forkID, + } + } +} + +func (f *finalizer) DSSendBatchEnd(batchNumber uint64, stateRoot common.Hash, localExitRoot common.Hash) { + if f.streamServer != nil { + // Send batch end to the streamer + f.dataToStream <- datastream.BatchEnd{ Number: batchNumber, - ForkId: forkID, StateRoot: stateRoot.Bytes(), LocalExitRoot: localExitRoot.Bytes(), } diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index d1945234d5..a86d955d79 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -416,30 +416,55 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { log.Errorf("failed to commit atomic op for bookmark type %d, value %d, error: %v", data.Type, data.Value, err) continue } - case datastream.Batch: + case datastream.BatchStart: err = s.streamServer.StartAtomicOp() if err != nil { - log.Errorf("failed to start atomic op for batch, error: %v", err) + log.Errorf("failed to start atomic op for batch start, error: %v", err) continue } data.ChainId = chainID - marshalledBatch, err := proto.Marshal(&data) + marshalledBatchStart, err := proto.Marshal(&data) if err != nil { - log.Errorf("failed to marshal batch, error: %v", err) + log.Errorf("failed to marshal batch start error: %v", err) continue } - _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledBatch) + _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START), marshalledBatchStart) if err != nil { - log.Errorf("failed to add stream entry for batch, error: %v", err) + log.Errorf("failed to add stream entry for batch start, error: %v", err) continue } err = s.streamServer.CommitAtomicOp() if err != nil { - log.Errorf("failed to commit atomic op for batch, error: %v", err) + log.Errorf("failed to commit atomic op for batch start, error: %v", err) + continue + } + + case datastream.BatchEnd: + err = s.streamServer.StartAtomicOp() + if err != nil { + log.Errorf("failed to start atomic op for batch end, error: %v", err) + continue + } + + marshalledBatchEnd, err := proto.Marshal(&data) + if err != nil { + log.Errorf("failed to marshal batch end, error: %v", err) + continue + } + + _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END), marshalledBatchEnd) + if err != nil { + log.Errorf("failed to add stream entry for batch end, error: %v", err) + continue + } + + err = s.streamServer.CommitAtomicOp() + if err != nil { + log.Errorf("failed to commit atomic op for batch end, error: %v", err) continue } diff --git a/state/datastream.go b/state/datastream.go index eb3670e6f9..3ad11f3f75 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -157,20 +157,34 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre return err } - genesisBatch := &datastream.Batch{ + genesisBatchStart := &datastream.BatchStart{ + Number: genesisL2Block.BatchNumber, + ForkId: genesisL2Block.ForkID, + ChainId: chainID, + } + + marshalledGenesisBatchStart, err := proto.Marshal(genesisBatchStart) + if err != nil { + return err + } + + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START), marshalledGenesisBatchStart) + if err != nil { + return err + } + + genesisBatchEnd := &datastream.BatchEnd{ Number: genesisL2Block.BatchNumber, LocalExitRoot: common.Hash{}.Bytes(), StateRoot: genesisL2Block.StateRoot.Bytes(), - ForkId: genesisL2Block.ForkID, - ChainId: chainID, } - marshalledGenesisBatch, err := proto.Marshal(genesisBatch) + marshalledGenesisBatchEnd, err := proto.Marshal(genesisBatchEnd) if err != nil { return err } - _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledGenesisBatch) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END), marshalledGenesisBatchEnd) if err != nil { return err } @@ -189,15 +203,24 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre log.Infof("Latest entry: %+v", latestEntry) switch latestEntry.Type { - case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH): - log.Info("Latest entry type is Batch") + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START): + log.Info("Latest entry type is Batch Start") + + batchStart := &datastream.BatchStart{} + if err := proto.Unmarshal(latestEntry.Data, batchStart); err != nil { + return err + } + + currentBatchNumber = batchStart.Number + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END): + log.Info("Latest entry type is Batch End") - batch := &datastream.Batch{} - if err := proto.Unmarshal(latestEntry.Data, batch); err != nil { + batchEnd := &datastream.BatchStart{} + if err := proto.Unmarshal(latestEntry.Data, batchEnd); err != nil { return err } - currentBatchNumber = batch.Number + currentBatchNumber = batchEnd.Number currentBatchNumber++ case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_UPDATE_GER): log.Info("Latest entry type is UpdateGER") @@ -364,6 +387,22 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre if err != nil { return err } + + batchStart := &datastream.BatchStart{ + Number: batch.BatchNumber, + ForkId: batch.ForkID, + ChainId: chainID, + } + + marshalledBatchStart, err := proto.Marshal(batchStart) + if err != nil { + return err + } + + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START), marshalledBatchStart) + if err != nil { + return err + } } if len(batch.L2Blocks) == 0 { @@ -542,20 +581,18 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre } } - batch := &datastream.Batch{ + batchEnd := &datastream.BatchEnd{ Number: batch.BatchNumber, LocalExitRoot: batch.LocalExitRoot.Bytes(), StateRoot: batch.StateRoot.Bytes(), - ForkId: batch.ForkID, - ChainId: chainID, } - marshalledBatch, err := proto.Marshal(batch) + marshalledBatch, err := proto.Marshal(batchEnd) if err != nil { return err } - _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH), marshalledBatch) + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END), marshalledBatch) if err != nil { return err } diff --git a/state/datastream/datastream.pb.go b/state/datastream/datastream.pb.go index 1f0ede74fc..07a8ff7adf 100644 --- a/state/datastream/datastream.pb.go +++ b/state/datastream/datastream.pb.go @@ -73,27 +73,30 @@ type EntryType int32 const ( EntryType_ENTRY_TYPE_UNSPECIFIED EntryType = 0 - EntryType_ENTRY_TYPE_BATCH EntryType = 1 + EntryType_ENTRY_TYPE_BATCH_START EntryType = 1 EntryType_ENTRY_TYPE_L2_BLOCK EntryType = 2 EntryType_ENTRY_TYPE_TRANSACTION EntryType = 3 - EntryType_ENTRY_TYPE_UPDATE_GER EntryType = 4 + EntryType_ENTRY_TYPE_BATCH_END EntryType = 4 + EntryType_ENTRY_TYPE_UPDATE_GER EntryType = 5 ) // Enum value maps for EntryType. var ( EntryType_name = map[int32]string{ 0: "ENTRY_TYPE_UNSPECIFIED", - 1: "ENTRY_TYPE_BATCH", + 1: "ENTRY_TYPE_BATCH_START", 2: "ENTRY_TYPE_L2_BLOCK", 3: "ENTRY_TYPE_TRANSACTION", - 4: "ENTRY_TYPE_UPDATE_GER", + 4: "ENTRY_TYPE_BATCH_END", + 5: "ENTRY_TYPE_UPDATE_GER", } EntryType_value = map[string]int32{ "ENTRY_TYPE_UNSPECIFIED": 0, - "ENTRY_TYPE_BATCH": 1, + "ENTRY_TYPE_BATCH_START": 1, "ENTRY_TYPE_L2_BLOCK": 2, "ENTRY_TYPE_TRANSACTION": 3, - "ENTRY_TYPE_UPDATE_GER": 4, + "ENTRY_TYPE_BATCH_END": 4, + "ENTRY_TYPE_UPDATE_GER": 5, } ) @@ -124,20 +127,18 @@ func (EntryType) EnumDescriptor() ([]byte, []int) { return file_datastream_proto_rawDescGZIP(), []int{1} } -type Batch struct { +type BatchStart struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` - LocalExitRoot []byte `protobuf:"bytes,2,opt,name=local_exit_root,json=localExitRoot,proto3" json:"local_exit_root,omitempty"` - StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` - ForkId uint64 `protobuf:"varint,4,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` - ChainId uint64 `protobuf:"varint,5,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` + Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` + ForkId uint64 `protobuf:"varint,4,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` + ChainId uint64 `protobuf:"varint,5,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` } -func (x *Batch) Reset() { - *x = Batch{} +func (x *BatchStart) Reset() { + *x = BatchStart{} if protoimpl.UnsafeEnabled { mi := &file_datastream_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -145,13 +146,13 @@ func (x *Batch) Reset() { } } -func (x *Batch) String() string { +func (x *BatchStart) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Batch) ProtoMessage() {} +func (*BatchStart) ProtoMessage() {} -func (x *Batch) ProtoReflect() protoreflect.Message { +func (x *BatchStart) ProtoReflect() protoreflect.Message { mi := &file_datastream_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -163,44 +164,93 @@ func (x *Batch) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Batch.ProtoReflect.Descriptor instead. -func (*Batch) Descriptor() ([]byte, []int) { +// Deprecated: Use BatchStart.ProtoReflect.Descriptor instead. +func (*BatchStart) Descriptor() ([]byte, []int) { return file_datastream_proto_rawDescGZIP(), []int{0} } -func (x *Batch) GetNumber() uint64 { +func (x *BatchStart) GetNumber() uint64 { if x != nil { return x.Number } return 0 } -func (x *Batch) GetLocalExitRoot() []byte { +func (x *BatchStart) GetForkId() uint64 { if x != nil { - return x.LocalExitRoot + return x.ForkId } - return nil + return 0 } -func (x *Batch) GetStateRoot() []byte { +func (x *BatchStart) GetChainId() uint64 { if x != nil { - return x.StateRoot + return x.ChainId } - return nil + return 0 +} + +type BatchEnd struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` + LocalExitRoot []byte `protobuf:"bytes,2,opt,name=local_exit_root,json=localExitRoot,proto3" json:"local_exit_root,omitempty"` + StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` } -func (x *Batch) GetForkId() uint64 { +func (x *BatchEnd) Reset() { + *x = BatchEnd{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchEnd) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchEnd) ProtoMessage() {} + +func (x *BatchEnd) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchEnd.ProtoReflect.Descriptor instead. +func (*BatchEnd) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{1} +} + +func (x *BatchEnd) GetNumber() uint64 { if x != nil { - return x.ForkId + return x.Number } return 0 } -func (x *Batch) GetChainId() uint64 { +func (x *BatchEnd) GetLocalExitRoot() []byte { if x != nil { - return x.ChainId + return x.LocalExitRoot } - return 0 + return nil +} + +func (x *BatchEnd) GetStateRoot() []byte { + if x != nil { + return x.StateRoot + } + return nil } type L2Block struct { @@ -224,7 +274,7 @@ type L2Block struct { func (x *L2Block) Reset() { *x = L2Block{} if protoimpl.UnsafeEnabled { - mi := &file_datastream_proto_msgTypes[1] + mi := &file_datastream_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -237,7 +287,7 @@ func (x *L2Block) String() string { func (*L2Block) ProtoMessage() {} func (x *L2Block) ProtoReflect() protoreflect.Message { - mi := &file_datastream_proto_msgTypes[1] + mi := &file_datastream_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -250,7 +300,7 @@ func (x *L2Block) ProtoReflect() protoreflect.Message { // Deprecated: Use L2Block.ProtoReflect.Descriptor instead. func (*L2Block) Descriptor() ([]byte, []int) { - return file_datastream_proto_rawDescGZIP(), []int{1} + return file_datastream_proto_rawDescGZIP(), []int{2} } func (x *L2Block) GetNumber() uint64 { @@ -345,7 +395,7 @@ type Transaction struct { func (x *Transaction) Reset() { *x = Transaction{} if protoimpl.UnsafeEnabled { - mi := &file_datastream_proto_msgTypes[2] + mi := &file_datastream_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -358,7 +408,7 @@ func (x *Transaction) String() string { func (*Transaction) ProtoMessage() {} func (x *Transaction) ProtoReflect() protoreflect.Message { - mi := &file_datastream_proto_msgTypes[2] + mi := &file_datastream_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -371,7 +421,7 @@ func (x *Transaction) ProtoReflect() protoreflect.Message { // Deprecated: Use Transaction.ProtoReflect.Descriptor instead. func (*Transaction) Descriptor() ([]byte, []int) { - return file_datastream_proto_rawDescGZIP(), []int{2} + return file_datastream_proto_rawDescGZIP(), []int{3} } func (x *Transaction) GetL2BlockNumber() uint64 { @@ -426,7 +476,7 @@ type UpdateGER struct { func (x *UpdateGER) Reset() { *x = UpdateGER{} if protoimpl.UnsafeEnabled { - mi := &file_datastream_proto_msgTypes[3] + mi := &file_datastream_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -439,7 +489,7 @@ func (x *UpdateGER) String() string { func (*UpdateGER) ProtoMessage() {} func (x *UpdateGER) ProtoReflect() protoreflect.Message { - mi := &file_datastream_proto_msgTypes[3] + mi := &file_datastream_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -452,7 +502,7 @@ func (x *UpdateGER) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateGER.ProtoReflect.Descriptor instead. func (*UpdateGER) Descriptor() ([]byte, []int) { - return file_datastream_proto_rawDescGZIP(), []int{3} + return file_datastream_proto_rawDescGZIP(), []int{4} } func (x *UpdateGER) GetBatchNumber() uint64 { @@ -516,7 +566,7 @@ type BookMark struct { func (x *BookMark) Reset() { *x = BookMark{} if protoimpl.UnsafeEnabled { - mi := &file_datastream_proto_msgTypes[4] + mi := &file_datastream_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -529,7 +579,7 @@ func (x *BookMark) String() string { func (*BookMark) ProtoMessage() {} func (x *BookMark) ProtoReflect() protoreflect.Message { - mi := &file_datastream_proto_msgTypes[4] + mi := &file_datastream_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -542,7 +592,7 @@ func (x *BookMark) ProtoReflect() protoreflect.Message { // Deprecated: Use BookMark.ProtoReflect.Descriptor instead. func (*BookMark) Descriptor() ([]byte, []int) { - return file_datastream_proto_rawDescGZIP(), []int{4} + return file_datastream_proto_rawDescGZIP(), []int{5} } func (x *BookMark) GetType() BookmarkType { @@ -564,92 +614,97 @@ var File_datastream_proto protoreflect.FileDescriptor var file_datastream_proto_rawDesc = []byte{ 0x0a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, - 0x31, 0x22, 0x9a, 0x01, 0x0a, 0x05, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x6e, - 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, - 0x62, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, - 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6c, 0x6f, - 0x63, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, - 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, - 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x22, 0xf8, - 0x02, 0x0a, 0x07, 0x4c, 0x32, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, - 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, - 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, - 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x64, 0x65, - 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, - 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x31, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, - 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6c, 0x31, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x68, 0x61, 0x73, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x6c, 0x31, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x74, - 0x72, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x0f, 0x6c, 0x31, 0x49, 0x6e, 0x66, 0x6f, 0x74, 0x72, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, - 0x68, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, - 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x6f, 0x6f, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, - 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, - 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, - 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x54, 0x72, - 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x32, 0x62, - 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x0d, 0x6c, 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, - 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6e, - 0x63, 0x6f, 0x64, 0x65, 0x64, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, - 0x76, 0x65, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x72, - 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x65, - 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x47, 0x61, 0x73, 0x50, 0x72, 0x69, 0x63, 0x65, - 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x6d, - 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x0b, 0x69, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xe5, - 0x01, 0x0a, 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x47, 0x45, 0x52, 0x12, 0x21, 0x0a, 0x0c, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, - 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a, - 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, - 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, - 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, - 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, - 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, - 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x51, 0x0a, 0x08, 0x42, 0x6f, 0x6f, 0x6b, 0x4d, 0x61, - 0x72, 0x6b, 0x12, 0x2f, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x1b, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, - 0x2e, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x2a, 0x62, 0x0a, 0x0c, 0x42, 0x6f, 0x6f, - 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x4f, 0x4f, - 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, - 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x42, 0x4f, 0x4f, 0x4b, - 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, - 0x01, 0x12, 0x1a, 0x0a, 0x16, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, - 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0x8d, 0x01, - 0x0a, 0x09, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x45, - 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, - 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, 0x54, 0x52, 0x59, - 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, 0x01, 0x12, 0x17, 0x0a, - 0x13, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, - 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, - 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, - 0x10, 0x03, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x45, 0x52, 0x10, 0x04, 0x42, 0x38, 0x5a, - 0x36, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x30, 0x78, 0x50, 0x6f, - 0x6c, 0x79, 0x67, 0x6f, 0x6e, 0x48, 0x65, 0x72, 0x6d, 0x65, 0x7a, 0x2f, 0x7a, 0x6b, 0x65, 0x76, - 0x6d, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x2f, 0x64, 0x61, 0x74, - 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x31, 0x22, 0x58, 0x0a, 0x0a, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, + 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, + 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, + 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x22, 0x69, 0x0a, 0x08, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x45, 0x6e, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, + 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x45, + 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xf8, 0x02, 0x0a, 0x07, 0x4c, 0x32, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, + 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, + 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x64, + 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x69, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x31, 0x5f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x0b, 0x6c, 0x31, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, 0x68, 0x12, 0x2a, 0x0a, 0x11, + 0x6c, 0x31, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x6c, 0x31, 0x49, 0x6e, 0x66, 0x6f, 0x74, + 0x72, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x67, + 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, + 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, + 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, + 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6c, 0x32, 0x62, 0x6c, 0x6f, + 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x76, + 0x61, 0x6c, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x12, 0x43, 0x0a, + 0x1e, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x70, + 0x72, 0x69, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, + 0x47, 0x61, 0x73, 0x50, 0x72, 0x69, 0x63, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, + 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x6d, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x69, 0x6d, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xe5, 0x01, 0x0a, 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x47, 0x45, 0x52, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, + 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, + 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, + 0x72, 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x51, + 0x0a, 0x08, 0x42, 0x6f, 0x6f, 0x6b, 0x4d, 0x61, 0x72, 0x6b, 0x12, 0x2f, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, + 0x6b, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x2a, 0x62, 0x0a, 0x0c, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x17, 0x0a, 0x13, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, 0x01, 0x12, 0x1a, 0x0a, 0x16, 0x42, 0x4f, 0x4f, + 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, + 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0xad, 0x01, 0x0a, 0x09, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, + 0x54, 0x43, 0x48, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x45, + 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, 0x4f, + 0x43, 0x4b, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, + 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, + 0x41, 0x54, 0x43, 0x48, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x04, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x4e, + 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, + 0x47, 0x45, 0x52, 0x10, 0x05, 0x42, 0x38, 0x5a, 0x36, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x30, 0x78, 0x50, 0x6f, 0x6c, 0x79, 0x67, 0x6f, 0x6e, 0x48, 0x65, 0x72, + 0x6d, 0x65, 0x7a, 0x2f, 0x7a, 0x6b, 0x65, 0x76, 0x6d, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -665,15 +720,16 @@ func file_datastream_proto_rawDescGZIP() []byte { } var file_datastream_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_datastream_proto_msgTypes = make([]protoimpl.MessageInfo, 5) +var file_datastream_proto_msgTypes = make([]protoimpl.MessageInfo, 6) var file_datastream_proto_goTypes = []interface{}{ (BookmarkType)(0), // 0: datastream.v1.BookmarkType (EntryType)(0), // 1: datastream.v1.EntryType - (*Batch)(nil), // 2: datastream.v1.Batch - (*L2Block)(nil), // 3: datastream.v1.L2Block - (*Transaction)(nil), // 4: datastream.v1.Transaction - (*UpdateGER)(nil), // 5: datastream.v1.UpdateGER - (*BookMark)(nil), // 6: datastream.v1.BookMark + (*BatchStart)(nil), // 2: datastream.v1.BatchStart + (*BatchEnd)(nil), // 3: datastream.v1.BatchEnd + (*L2Block)(nil), // 4: datastream.v1.L2Block + (*Transaction)(nil), // 5: datastream.v1.Transaction + (*UpdateGER)(nil), // 6: datastream.v1.UpdateGER + (*BookMark)(nil), // 7: datastream.v1.BookMark } var file_datastream_proto_depIdxs = []int32{ 0, // 0: datastream.v1.BookMark.type:type_name -> datastream.v1.BookmarkType @@ -691,7 +747,7 @@ func file_datastream_proto_init() { } if !protoimpl.UnsafeEnabled { file_datastream_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Batch); i { + switch v := v.(*BatchStart); i { case 0: return &v.state case 1: @@ -703,7 +759,7 @@ func file_datastream_proto_init() { } } file_datastream_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*L2Block); i { + switch v := v.(*BatchEnd); i { case 0: return &v.state case 1: @@ -715,7 +771,7 @@ func file_datastream_proto_init() { } } file_datastream_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Transaction); i { + switch v := v.(*L2Block); i { case 0: return &v.state case 1: @@ -727,7 +783,7 @@ func file_datastream_proto_init() { } } file_datastream_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateGER); i { + switch v := v.(*Transaction); i { case 0: return &v.state case 1: @@ -739,6 +795,18 @@ func file_datastream_proto_init() { } } file_datastream_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateGER); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_datastream_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BookMark); i { case 0: return &v.state @@ -757,7 +825,7 @@ func file_datastream_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_datastream_proto_rawDesc, NumEnums: 2, - NumMessages: 5, + NumMessages: 6, NumExtensions: 0, NumServices: 0, }, diff --git a/tools/datastreamer/main.go b/tools/datastreamer/main.go index bb84aeebf3..97a5eab879 100644 --- a/tools/datastreamer/main.go +++ b/tools/datastreamer/main.go @@ -757,27 +757,40 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.GlobalExitRoot))) printColored(color.FgGreen, "Coinbase........: ") printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToAddress(l2Block.Coinbase))) - case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH): - batch := &datastream.Batch{} + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START): + batch := &datastream.BatchStart{} err := proto.Unmarshal(entry.Data, batch) if err != nil { log.Error(err) os.Exit(1) } printColored(color.FgGreen, "Entry Type......: ") - printColored(color.FgHiYellow, "Batch\n") + printColored(color.FgHiYellow, "Batch Start\n") printColored(color.FgGreen, "Entry Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) printColored(color.FgGreen, "Batch Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.Number)) - printColored(color.FgGreen, "State Root......: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.StateRoot))) - printColored(color.FgGreen, "Local Exit Root.: ") - printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.LocalExitRoot))) printColored(color.FgGreen, "Fork ID.........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ForkId)) printColored(color.FgGreen, "Chain ID........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ChainId)) + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END): + batch := &datastream.BatchEnd{} + err := proto.Unmarshal(entry.Data, batch) + if err != nil { + log.Error(err) + os.Exit(1) + } + printColored(color.FgGreen, "Entry Type......: ") + printColored(color.FgHiYellow, "Batch End\n") + printColored(color.FgGreen, "Entry Number....: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) + printColored(color.FgGreen, "Batch Number....: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.Number)) + printColored(color.FgGreen, "State Root......: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.StateRoot))) + printColored(color.FgGreen, "Local Exit Root.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.LocalExitRoot))) case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION): dsTx := &datastream.Transaction{} err := proto.Unmarshal(entry.Data, dsTx) From 02ba1fc3b263299de8f81293be44fc19eeb16a93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Fri, 3 May 2024 13:02:11 +0200 Subject: [PATCH 03/22] fix genesis DS (#3615) --- state/datastream.go | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/state/datastream.go b/state/datastream.go index 3ad11f3f75..08c071feba 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -119,6 +119,22 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre return err } + genesisBatchStart := &datastream.BatchStart{ + Number: genesisL2Block.BatchNumber, + ForkId: genesisL2Block.ForkID, + ChainId: chainID, + } + + marshalledGenesisBatchStart, err := proto.Marshal(genesisBatchStart) + if err != nil { + return err + } + + _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START), marshalledGenesisBatchStart) + if err != nil { + return err + } + bookMark = &datastream.BookMark{ Type: datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK, Value: genesisL2Block.L2BlockNumber, @@ -157,22 +173,6 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre return err } - genesisBatchStart := &datastream.BatchStart{ - Number: genesisL2Block.BatchNumber, - ForkId: genesisL2Block.ForkID, - ChainId: chainID, - } - - marshalledGenesisBatchStart, err := proto.Marshal(genesisBatchStart) - if err != nil { - return err - } - - _, err = streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START), marshalledGenesisBatchStart) - if err != nil { - return err - } - genesisBatchEnd := &datastream.BatchEnd{ Number: genesisL2Block.BatchNumber, LocalExitRoot: common.Hash{}.Bytes(), From de387c7ae329b48cec879ff5f87d23fa1b794731 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Fri, 3 May 2024 15:55:30 +0200 Subject: [PATCH 04/22] Fix DSSendL2Block batch number (#3617) * Fix DSSendL2Block batch number --- sequencer/l2block.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sequencer/l2block.go b/sequencer/l2block.go index f045ef2764..b20be1e67a 100644 --- a/sequencer/l2block.go +++ b/sequencer/l2block.go @@ -492,7 +492,7 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { log.Infof("[ds-debug] l2 block %d [%d] transactions updated as selected in the pooldb", blockResponse.BlockNumber, l2Block.trackingNum) // Send L2 block to data streamer - err = f.DSSendL2Block(f.wipBatch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) + err = f.DSSendL2Block(l2Block.batch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d [%d] to data streamer, error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err) From f7b11f68c1fbd99825729d7294df8a8065f682f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Mon, 6 May 2024 17:53:30 +0200 Subject: [PATCH 05/22] latest proto (#3620) --- .../src/proto/datastream/v1/datastream.proto | 32 +- sequencer/batch.go | 2 +- sequencer/datastreamer.go | 26 +- state/datastream.go | 28 +- state/datastream/datastream.pb.go | 445 +++++++++++++----- state/pgstatestorage/datastream.go | 11 +- tools/datastreamer/main.go | 45 +- 7 files changed, 449 insertions(+), 140 deletions(-) diff --git a/proto/src/proto/datastream/v1/datastream.proto b/proto/src/proto/datastream/v1/datastream.proto index 555bc17733..9cdd0266c1 100644 --- a/proto/src/proto/datastream/v1/datastream.proto +++ b/proto/src/proto/datastream/v1/datastream.proto @@ -6,14 +6,17 @@ option go_package = "github.com/0xPolygonHermez/zkevm-node/state/datastream"; message BatchStart { uint64 number = 1; - uint64 fork_id = 4; - uint64 chain_id = 5; + BatchType type = 2; + uint64 fork_id = 3; + uint64 chain_id = 4; + Debug debug = 5; } message BatchEnd { uint64 number = 1; bytes local_exit_root = 2; bytes state_root = 3; + Debug debug = 4; } message L2Block { @@ -28,14 +31,19 @@ message L2Block { bytes state_root = 9; bytes global_exit_root = 10; bytes coinbase = 11; + uint64 block_gas_limit = 12; + bytes block_info_root = 13; + Debug debug = 14; } message Transaction { uint64 l2block_number = 1; - bool is_valid = 2; - bytes encoded = 3; - uint32 effective_gas_price_percentage = 4; - bytes im_state_root = 5; + uint64 index = 2; + bool is_valid = 3; + bytes encoded = 4; + uint32 effective_gas_price_percentage = 5; + bytes im_state_root = 6; + Debug debug = 7; } message UpdateGER { @@ -46,6 +54,7 @@ message UpdateGER { uint64 fork_id = 5; uint64 chain_id = 6; bytes state_root = 7; + Debug debug = 8; } message BookMark { @@ -53,6 +62,10 @@ message BookMark { uint64 value = 2; } +message Debug { + string message = 1; +} + enum BookmarkType { BOOKMARK_TYPE_UNSPECIFIED = 0; BOOKMARK_TYPE_BATCH = 1; @@ -67,3 +80,10 @@ enum EntryType { ENTRY_TYPE_BATCH_END = 4; ENTRY_TYPE_UPDATE_GER = 5; } + +enum BatchType { + BATCH_TYPE_UNSPECIFIED = 0; + BATCH_TYPE_REGULAR = 1; + BATCH_TYPE_FORCED = 2; + BATCH_TYPE_INJECTED = 3; +} diff --git a/sequencer/batch.go b/sequencer/batch.go index cc2bea283a..79d84026dd 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -340,7 +340,7 @@ func (f *finalizer) insertSIPBatch(ctx context.Context, batchNumber uint64, stat // Send batch bookmark to the datastream f.DSSendBatchBookmark(batchNumber) // Send batch start to the datastream - f.DSSendBatchStart(batchNumber) + f.DSSendBatchStart(batchNumber, false) // Check if synchronizer is up-to-date //TODO: review if this is needed diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 8178593978..4c08d6b3e4 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -24,11 +24,12 @@ func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.Proce ForkID: forkID, BlockHash: blockResponse.BlockHash, StateRoot: blockResponse.BlockHash, //From etrog, the blockhash is the block root + BlockInfoRoot: blockResponse.BlockInfoRoot, } l2Transactions := []state.DSL2Transaction{} - for _, txResponse := range blockResponse.TransactionResponses { + for i, txResponse := range blockResponse.TransactionResponses { binaryTxData, err := txResponse.Tx.MarshalBinary() if err != nil { return err @@ -37,12 +38,17 @@ func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.Proce l2Transaction := state.DSL2Transaction{ L2BlockNumber: blockResponse.BlockNumber, EffectiveGasPricePercentage: uint8(txResponse.EffectivePercentage), + Index: uint64(i), IsValid: 1, EncodedLength: uint32(len(binaryTxData)), Encoded: binaryTxData, StateRoot: txResponse.StateRoot, } + if txResponse.Logs != nil && len(txResponse.Logs) > 0 { + l2Transaction.Index = uint64(txResponse.Logs[0].TxIndex) + } + l2Transactions = append(l2Transactions, l2Transaction) } @@ -67,15 +73,23 @@ func (f *finalizer) DSSendBatchBookmark(batchNumber uint64) { } } -func (f *finalizer) DSSendBatchStart(batchNumber uint64) { +func (f *finalizer) DSSendBatchStart(batchNumber uint64, isForced bool) { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) + batchStart := datastream.BatchStart{ + Number: batchNumber, + ForkId: forkID, + } + + if isForced { + batchStart.Type = datastream.BatchType_BATCH_TYPE_FORCED + } else { + batchStart.Type = datastream.BatchType_BATCH_TYPE_REGULAR + } + if f.streamServer != nil { // Send batch start to the streamer - f.dataToStream <- datastream.BatchStart{ - Number: batchNumber, - ForkId: forkID, - } + f.dataToStream <- batchStart } } diff --git a/state/datastream.go b/state/datastream.go index 08c071feba..90c7ff3211 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -58,16 +58,19 @@ type DSL2Block struct { ChainID uint64 BlockHash common.Hash StateRoot common.Hash + BlockGasLimit uint64 + BlockInfoRoot common.Hash } // DSL2Transaction represents a data stream L2 transaction type DSL2Transaction struct { - L2BlockNumber uint64 // Not included in the encoded data - ImStateRoot common.Hash // Not included in the encoded data - EffectiveGasPricePercentage uint8 // 1 byte - IsValid uint8 // 1 byte - StateRoot common.Hash // 32 bytes - EncodedLength uint32 // 4 bytes + L2BlockNumber uint64 + ImStateRoot common.Hash + EffectiveGasPricePercentage uint8 + IsValid uint8 + Index uint64 + StateRoot common.Hash + EncodedLength uint32 Encoded []byte } @@ -121,6 +124,7 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre genesisBatchStart := &datastream.BatchStart{ Number: genesisL2Block.BatchNumber, + Type: datastream.BatchType_BATCH_TYPE_UNSPECIFIED, ForkId: genesisL2Block.ForkID, ChainId: chainID, } @@ -390,10 +394,19 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre batchStart := &datastream.BatchStart{ Number: batch.BatchNumber, + Type: datastream.BatchType_BATCH_TYPE_REGULAR, ForkId: batch.ForkID, ChainId: chainID, } + if batch.ForkID >= FORKID_ETROG && (batch.BatchNumber == 1 || (upgradeEtrogBatchNumber != 0 && batch.BatchNumber == upgradeEtrogBatchNumber)) { + batchStart.Type = datastream.BatchType_BATCH_TYPE_INJECTED + } + + if batch.ForcedBatchNum != nil { + batchStart.Type = datastream.BatchType_BATCH_TYPE_FORCED + } + marshalledBatchStart, err := proto.Marshal(batchStart) if err != nil { return err @@ -495,6 +508,8 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre StateRoot: l2Block.StateRoot.Bytes(), GlobalExitRoot: l2Block.GlobalExitRoot.Bytes(), Coinbase: l2Block.Coinbase.Bytes(), + BlockInfoRoot: l2Block.BlockInfoRoot.Bytes(), + BlockGasLimit: l2Block.BlockGasLimit, } if l2Block.ForkID >= FORKID_ETROG { @@ -560,6 +575,7 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre transaction := &datastream.Transaction{ L2BlockNumber: tx.L2BlockNumber, + Index: tx.Index, IsValid: tx.IsValid != 0, Encoded: tx.Encoded, EffectiveGasPricePercentage: uint32(tx.EffectiveGasPricePercentage), diff --git a/state/datastream/datastream.pb.go b/state/datastream/datastream.pb.go index 07a8ff7adf..02a0e2267f 100644 --- a/state/datastream/datastream.pb.go +++ b/state/datastream/datastream.pb.go @@ -127,14 +127,68 @@ func (EntryType) EnumDescriptor() ([]byte, []int) { return file_datastream_proto_rawDescGZIP(), []int{1} } +type BatchType int32 + +const ( + BatchType_BATCH_TYPE_UNSPECIFIED BatchType = 0 + BatchType_BATCH_TYPE_REGULAR BatchType = 1 + BatchType_BATCH_TYPE_FORCED BatchType = 2 + BatchType_BATCH_TYPE_INJECTED BatchType = 3 +) + +// Enum value maps for BatchType. +var ( + BatchType_name = map[int32]string{ + 0: "BATCH_TYPE_UNSPECIFIED", + 1: "BATCH_TYPE_REGULAR", + 2: "BATCH_TYPE_FORCED", + 3: "BATCH_TYPE_INJECTED", + } + BatchType_value = map[string]int32{ + "BATCH_TYPE_UNSPECIFIED": 0, + "BATCH_TYPE_REGULAR": 1, + "BATCH_TYPE_FORCED": 2, + "BATCH_TYPE_INJECTED": 3, + } +) + +func (x BatchType) Enum() *BatchType { + p := new(BatchType) + *p = x + return p +} + +func (x BatchType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BatchType) Descriptor() protoreflect.EnumDescriptor { + return file_datastream_proto_enumTypes[2].Descriptor() +} + +func (BatchType) Type() protoreflect.EnumType { + return &file_datastream_proto_enumTypes[2] +} + +func (x BatchType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BatchType.Descriptor instead. +func (BatchType) EnumDescriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{2} +} + type BatchStart struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` - ForkId uint64 `protobuf:"varint,4,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` - ChainId uint64 `protobuf:"varint,5,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` + Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` + Type BatchType `protobuf:"varint,2,opt,name=type,proto3,enum=datastream.v1.BatchType" json:"type,omitempty"` + ForkId uint64 `protobuf:"varint,3,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` + ChainId uint64 `protobuf:"varint,4,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` + Debug *Debug `protobuf:"bytes,5,opt,name=debug,proto3" json:"debug,omitempty"` } func (x *BatchStart) Reset() { @@ -176,6 +230,13 @@ func (x *BatchStart) GetNumber() uint64 { return 0 } +func (x *BatchStart) GetType() BatchType { + if x != nil { + return x.Type + } + return BatchType_BATCH_TYPE_UNSPECIFIED +} + func (x *BatchStart) GetForkId() uint64 { if x != nil { return x.ForkId @@ -190,6 +251,13 @@ func (x *BatchStart) GetChainId() uint64 { return 0 } +func (x *BatchStart) GetDebug() *Debug { + if x != nil { + return x.Debug + } + return nil +} + type BatchEnd struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -198,6 +266,7 @@ type BatchEnd struct { Number uint64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"` LocalExitRoot []byte `protobuf:"bytes,2,opt,name=local_exit_root,json=localExitRoot,proto3" json:"local_exit_root,omitempty"` StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` + Debug *Debug `protobuf:"bytes,4,opt,name=debug,proto3" json:"debug,omitempty"` } func (x *BatchEnd) Reset() { @@ -253,6 +322,13 @@ func (x *BatchEnd) GetStateRoot() []byte { return nil } +func (x *BatchEnd) GetDebug() *Debug { + if x != nil { + return x.Debug + } + return nil +} + type L2Block struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -269,6 +345,9 @@ type L2Block struct { StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` GlobalExitRoot []byte `protobuf:"bytes,10,opt,name=global_exit_root,json=globalExitRoot,proto3" json:"global_exit_root,omitempty"` Coinbase []byte `protobuf:"bytes,11,opt,name=coinbase,proto3" json:"coinbase,omitempty"` + BlockGasLimit uint64 `protobuf:"varint,12,opt,name=block_gas_limit,json=blockGasLimit,proto3" json:"block_gas_limit,omitempty"` + BlockInfoRoot []byte `protobuf:"bytes,13,opt,name=block_info_root,json=blockInfoRoot,proto3" json:"block_info_root,omitempty"` + Debug *Debug `protobuf:"bytes,14,opt,name=debug,proto3" json:"debug,omitempty"` } func (x *L2Block) Reset() { @@ -380,16 +459,39 @@ func (x *L2Block) GetCoinbase() []byte { return nil } +func (x *L2Block) GetBlockGasLimit() uint64 { + if x != nil { + return x.BlockGasLimit + } + return 0 +} + +func (x *L2Block) GetBlockInfoRoot() []byte { + if x != nil { + return x.BlockInfoRoot + } + return nil +} + +func (x *L2Block) GetDebug() *Debug { + if x != nil { + return x.Debug + } + return nil +} + type Transaction struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields L2BlockNumber uint64 `protobuf:"varint,1,opt,name=l2block_number,json=l2blockNumber,proto3" json:"l2block_number,omitempty"` - IsValid bool `protobuf:"varint,2,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"` - Encoded []byte `protobuf:"bytes,3,opt,name=encoded,proto3" json:"encoded,omitempty"` - EffectiveGasPricePercentage uint32 `protobuf:"varint,4,opt,name=effective_gas_price_percentage,json=effectiveGasPricePercentage,proto3" json:"effective_gas_price_percentage,omitempty"` - ImStateRoot []byte `protobuf:"bytes,5,opt,name=im_state_root,json=imStateRoot,proto3" json:"im_state_root,omitempty"` + Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"` + IsValid bool `protobuf:"varint,3,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"` + Encoded []byte `protobuf:"bytes,4,opt,name=encoded,proto3" json:"encoded,omitempty"` + EffectiveGasPricePercentage uint32 `protobuf:"varint,5,opt,name=effective_gas_price_percentage,json=effectiveGasPricePercentage,proto3" json:"effective_gas_price_percentage,omitempty"` + ImStateRoot []byte `protobuf:"bytes,6,opt,name=im_state_root,json=imStateRoot,proto3" json:"im_state_root,omitempty"` + Debug *Debug `protobuf:"bytes,7,opt,name=debug,proto3" json:"debug,omitempty"` } func (x *Transaction) Reset() { @@ -431,6 +533,13 @@ func (x *Transaction) GetL2BlockNumber() uint64 { return 0 } +func (x *Transaction) GetIndex() uint64 { + if x != nil { + return x.Index + } + return 0 +} + func (x *Transaction) GetIsValid() bool { if x != nil { return x.IsValid @@ -459,6 +568,13 @@ func (x *Transaction) GetImStateRoot() []byte { return nil } +func (x *Transaction) GetDebug() *Debug { + if x != nil { + return x.Debug + } + return nil +} + type UpdateGER struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -471,6 +587,7 @@ type UpdateGER struct { ForkId uint64 `protobuf:"varint,5,opt,name=fork_id,json=forkId,proto3" json:"fork_id,omitempty"` ChainId uint64 `protobuf:"varint,6,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` StateRoot []byte `protobuf:"bytes,7,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` + Debug *Debug `protobuf:"bytes,8,opt,name=debug,proto3" json:"debug,omitempty"` } func (x *UpdateGER) Reset() { @@ -554,6 +671,13 @@ func (x *UpdateGER) GetStateRoot() []byte { return nil } +func (x *UpdateGER) GetDebug() *Debug { + if x != nil { + return x.Debug + } + return nil +} + type BookMark struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -609,102 +733,181 @@ func (x *BookMark) GetValue() uint64 { return 0 } +type Debug struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` +} + +func (x *Debug) Reset() { + *x = Debug{} + if protoimpl.UnsafeEnabled { + mi := &file_datastream_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Debug) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Debug) ProtoMessage() {} + +func (x *Debug) ProtoReflect() protoreflect.Message { + mi := &file_datastream_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Debug.ProtoReflect.Descriptor instead. +func (*Debug) Descriptor() ([]byte, []int) { + return file_datastream_proto_rawDescGZIP(), []int{6} +} + +func (x *Debug) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + var File_datastream_proto protoreflect.FileDescriptor var file_datastream_proto_rawDesc = []byte{ 0x0a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, - 0x31, 0x22, 0x58, 0x0a, 0x0a, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, - 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, - 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, - 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x22, 0x69, 0x0a, 0x08, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x45, 0x6e, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, - 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, - 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, - 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x45, - 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xf8, 0x02, 0x0a, 0x07, 0x4c, 0x32, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, - 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, + 0x31, 0x22, 0xb2, 0x01, 0x0a, 0x0a, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x2c, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, + 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x2a, 0x0a, 0x05, 0x64, 0x65, + 0x62, 0x75, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x52, + 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x22, 0x95, 0x01, 0x0a, 0x08, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x45, 0x6e, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, + 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, + 0x6f, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, + 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x52, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x22, 0xf4, + 0x03, 0x0a, 0x07, 0x4c, 0x32, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x64, 0x65, + 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, + 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x31, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, + 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6c, 0x31, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x68, 0x61, 0x73, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x6c, 0x31, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x74, + 0x72, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0f, 0x6c, 0x31, 0x49, 0x6e, 0x66, 0x6f, 0x74, 0x72, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x68, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, + 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x6f, 0x6f, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, + 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, + 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x62, 0x6c, 0x6f, + 0x63, 0x6b, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0d, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x47, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, + 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, + 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x62, 0x6c, 0x6f, 0x63, + 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x64, 0x65, 0x62, + 0x75, 0x67, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x52, 0x05, + 0x64, 0x65, 0x62, 0x75, 0x67, 0x22, 0x94, 0x02, 0x0a, 0x0b, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6c, + 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x12, 0x18, 0x0a, + 0x07, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, + 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x66, 0x66, 0x65, 0x63, + 0x74, 0x69, 0x76, 0x65, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x5f, 0x70, + 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x1b, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x47, 0x61, 0x73, 0x50, 0x72, 0x69, + 0x63, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, + 0x69, 0x6d, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x69, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, + 0x12, 0x2a, 0x0a, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x14, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x62, 0x75, 0x67, 0x52, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x22, 0x91, 0x02, 0x0a, + 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x47, 0x45, 0x52, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, + 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, - 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x64, - 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x69, 0x6e, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x31, 0x5f, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x0b, 0x6c, 0x31, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x68, 0x61, 0x73, 0x68, 0x12, 0x2a, 0x0a, 0x11, - 0x6c, 0x31, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x6c, 0x31, 0x49, 0x6e, 0x66, 0x6f, 0x74, - 0x72, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, - 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, - 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x67, + 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, - 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, + 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, - 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, - 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x32, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, - 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6c, 0x32, 0x62, 0x6c, 0x6f, - 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x76, - 0x61, 0x6c, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x12, 0x43, 0x0a, - 0x1e, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x70, - 0x72, 0x69, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, - 0x47, 0x61, 0x73, 0x50, 0x72, 0x69, 0x63, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, - 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x6d, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, - 0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x69, 0x6d, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xe5, 0x01, 0x0a, 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x47, 0x45, 0x52, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, - 0x65, 0x78, 0x69, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x0e, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x45, 0x78, 0x69, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, - 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, - 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, - 0x72, 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x51, - 0x0a, 0x08, 0x42, 0x6f, 0x6f, 0x6b, 0x4d, 0x61, 0x72, 0x6b, 0x12, 0x2f, 0x0a, 0x04, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, - 0x6b, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x2a, 0x62, 0x0a, 0x0c, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, - 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, - 0x12, 0x17, 0x0a, 0x13, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, - 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, 0x01, 0x12, 0x1a, 0x0a, 0x16, 0x42, 0x4f, 0x4f, - 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, - 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0xad, 0x01, 0x0a, 0x09, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, - 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, - 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, - 0x54, 0x43, 0x48, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x45, - 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x32, 0x5f, 0x42, 0x4c, 0x4f, - 0x43, 0x4b, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, - 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, - 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, - 0x41, 0x54, 0x43, 0x48, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x04, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x4e, - 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, - 0x47, 0x45, 0x52, 0x10, 0x05, 0x42, 0x38, 0x5a, 0x36, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x30, 0x78, 0x50, 0x6f, 0x6c, 0x79, 0x67, 0x6f, 0x6e, 0x48, 0x65, 0x72, - 0x6d, 0x65, 0x7a, 0x2f, 0x7a, 0x6b, 0x65, 0x76, 0x6d, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x63, 0x6f, 0x69, 0x6e, 0x62, 0x61, 0x73, + 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, + 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68, + 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x52, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, + 0x22, 0x51, 0x0a, 0x08, 0x42, 0x6f, 0x6f, 0x6b, 0x4d, 0x61, 0x72, 0x6b, 0x12, 0x2f, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, + 0x61, 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x22, 0x21, 0x0a, 0x05, 0x44, 0x65, 0x62, 0x75, 0x67, 0x12, 0x18, 0x0a, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2a, 0x62, 0x0a, 0x0c, 0x42, 0x6f, 0x6f, 0x6b, 0x6d, 0x61, + 0x72, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, + 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, + 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, + 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x10, 0x01, 0x12, 0x1a, + 0x0a, 0x16, 0x42, 0x4f, 0x4f, 0x4b, 0x4d, 0x41, 0x52, 0x4b, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x4c, 0x32, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0xad, 0x01, 0x0a, 0x09, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, + 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, + 0x45, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x01, + 0x12, 0x17, 0x0a, 0x13, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, + 0x32, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x4e, 0x54, + 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x04, 0x12, + 0x19, 0x0a, 0x15, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, + 0x44, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x45, 0x52, 0x10, 0x05, 0x2a, 0x6f, 0x0a, 0x09, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x42, 0x41, 0x54, 0x43, 0x48, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x52, 0x45, 0x47, 0x55, 0x4c, 0x41, 0x52, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x42, + 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x43, 0x45, 0x44, + 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x49, 0x4e, 0x4a, 0x45, 0x43, 0x54, 0x45, 0x44, 0x10, 0x03, 0x42, 0x38, 0x5a, 0x36, 0x67, + 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x30, 0x78, 0x50, 0x6f, 0x6c, 0x79, + 0x67, 0x6f, 0x6e, 0x48, 0x65, 0x72, 0x6d, 0x65, 0x7a, 0x2f, 0x7a, 0x6b, 0x65, 0x76, 0x6d, 0x2d, + 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -719,25 +922,33 @@ func file_datastream_proto_rawDescGZIP() []byte { return file_datastream_proto_rawDescData } -var file_datastream_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_datastream_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_datastream_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_datastream_proto_msgTypes = make([]protoimpl.MessageInfo, 7) var file_datastream_proto_goTypes = []interface{}{ (BookmarkType)(0), // 0: datastream.v1.BookmarkType (EntryType)(0), // 1: datastream.v1.EntryType - (*BatchStart)(nil), // 2: datastream.v1.BatchStart - (*BatchEnd)(nil), // 3: datastream.v1.BatchEnd - (*L2Block)(nil), // 4: datastream.v1.L2Block - (*Transaction)(nil), // 5: datastream.v1.Transaction - (*UpdateGER)(nil), // 6: datastream.v1.UpdateGER - (*BookMark)(nil), // 7: datastream.v1.BookMark + (BatchType)(0), // 2: datastream.v1.BatchType + (*BatchStart)(nil), // 3: datastream.v1.BatchStart + (*BatchEnd)(nil), // 4: datastream.v1.BatchEnd + (*L2Block)(nil), // 5: datastream.v1.L2Block + (*Transaction)(nil), // 6: datastream.v1.Transaction + (*UpdateGER)(nil), // 7: datastream.v1.UpdateGER + (*BookMark)(nil), // 8: datastream.v1.BookMark + (*Debug)(nil), // 9: datastream.v1.Debug } var file_datastream_proto_depIdxs = []int32{ - 0, // 0: datastream.v1.BookMark.type:type_name -> datastream.v1.BookmarkType - 1, // [1:1] is the sub-list for method output_type - 1, // [1:1] is the sub-list for method input_type - 1, // [1:1] is the sub-list for extension type_name - 1, // [1:1] is the sub-list for extension extendee - 0, // [0:1] is the sub-list for field type_name + 2, // 0: datastream.v1.BatchStart.type:type_name -> datastream.v1.BatchType + 9, // 1: datastream.v1.BatchStart.debug:type_name -> datastream.v1.Debug + 9, // 2: datastream.v1.BatchEnd.debug:type_name -> datastream.v1.Debug + 9, // 3: datastream.v1.L2Block.debug:type_name -> datastream.v1.Debug + 9, // 4: datastream.v1.Transaction.debug:type_name -> datastream.v1.Debug + 9, // 5: datastream.v1.UpdateGER.debug:type_name -> datastream.v1.Debug + 0, // 6: datastream.v1.BookMark.type:type_name -> datastream.v1.BookmarkType + 7, // [7:7] is the sub-list for method output_type + 7, // [7:7] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name } func init() { file_datastream_proto_init() } @@ -818,14 +1029,26 @@ func file_datastream_proto_init() { return nil } } + file_datastream_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Debug); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_datastream_proto_rawDesc, - NumEnums: 2, - NumMessages: 6, + NumEnums: 3, + NumMessages: 7, NumExtensions: 0, NumServices: 0, }, diff --git a/state/pgstatestorage/datastream.go b/state/pgstatestorage/datastream.go index 31c9814c0a..a7e99e6528 100644 --- a/state/pgstatestorage/datastream.go +++ b/state/pgstatestorage/datastream.go @@ -11,7 +11,7 @@ import ( // GetDSGenesisBlock returns the genesis block func (p *PostgresStorage) GetDSGenesisBlock(ctx context.Context, dbTx pgx.Tx) (*state.DSL2Block, error) { - const genesisL2BlockSQL = `SELECT 0 as batch_num, l2b.block_num, l2b.received_at, '0x0000000000000000000000000000000000000000' as global_exit_root, '0x0000000000000000000000000000000000000000' as block_global_exit_root, l2b.header->>'miner' AS coinbase, 0 as fork_id, l2b.block_hash, l2b.state_root + const genesisL2BlockSQL = `SELECT 0 as batch_num, l2b.block_num, l2b.received_at, '0x0000000000000000000000000000000000000000' as global_exit_root, '0x0000000000000000000000000000000000000000' as block_global_exit_root, l2b.header->>'miner' AS coinbase, 0 as fork_id, l2b.block_hash, l2b.state_root, '0x0000000000000000000000000000000000000000' as block_info_root FROM state.l2block l2b WHERE l2b.block_num = 0` @@ -29,7 +29,7 @@ func (p *PostgresStorage) GetDSGenesisBlock(ctx context.Context, dbTx pgx.Tx) (* // GetDSL2Blocks returns the L2 blocks func (p *PostgresStorage) GetDSL2Blocks(ctx context.Context, firstBatchNumber, lastBatchNumber uint64, dbTx pgx.Tx) ([]*state.DSL2Block, error) { - const l2BlockSQL = `SELECT l2b.batch_num, l2b.block_num, l2b.received_at, b.global_exit_root, COALESCE(l2b.header->>'globalExitRoot', '') AS block_global_exit_root, l2b.header->>'miner' AS coinbase, f.fork_id, l2b.block_hash, l2b.state_root + const l2BlockSQL = `SELECT l2b.batch_num, l2b.block_num, l2b.received_at, b.global_exit_root, COALESCE(l2b.header->>'globalExitRoot', '') AS block_global_exit_root, l2b.header->>'miner' AS coinbase, f.fork_id, l2b.block_hash, l2b.state_root, COALESCE(l2b.header->>'blockInfoRoot', '') AS block_info_root FROM state.l2block l2b, state.batch b, state.fork_id f WHERE l2b.batch_num BETWEEN $1 AND $2 AND l2b.batch_num = b.batch_num AND l2b.batch_num between f.from_batch_num AND f.to_batch_num ORDER BY l2b.block_num ASC` @@ -62,6 +62,7 @@ func scanL2Block(row pgx.Row) (*state.DSL2Block, error) { timestamp time.Time blockHashStr string stateRootStr string + blockInfoStr string ) if err := row.Scan( &l2Block.BatchNumber, @@ -73,6 +74,7 @@ func scanL2Block(row pgx.Row) (*state.DSL2Block, error) { &l2Block.ForkID, &blockHashStr, &stateRootStr, + &blockInfoStr, ); err != nil { return &l2Block, err } @@ -81,6 +83,7 @@ func scanL2Block(row pgx.Row) (*state.DSL2Block, error) { l2Block.Timestamp = uint64(timestamp.Unix()) l2Block.BlockHash = common.HexToHash(blockHashStr) l2Block.StateRoot = common.HexToHash(stateRootStr) + l2Block.BlockInfoRoot = common.HexToHash(blockInfoStr) if l2Block.ForkID >= state.FORKID_ETROG { l2Block.GlobalExitRoot = common.HexToHash(blockGERStr) @@ -91,7 +94,7 @@ func scanL2Block(row pgx.Row) (*state.DSL2Block, error) { // GetDSL2Transactions returns the L2 transactions func (p *PostgresStorage) GetDSL2Transactions(ctx context.Context, firstL2Block, lastL2Block uint64, dbTx pgx.Tx) ([]*state.DSL2Transaction, error) { - const l2TxSQL = `SELECT l2_block_num, t.effective_percentage, t.encoded, r.post_state, r.im_state_root + const l2TxSQL = `SELECT l2_block_num, t.effective_percentage, t.encoded, r.post_state, r.im_state_root, r.tx_index FROM state.transaction t, state.receipt r WHERE l2_block_num BETWEEN $1 AND $2 AND r.tx_hash = t.hash ORDER BY t.l2_block_num ASC, r.tx_index ASC` @@ -121,12 +124,14 @@ func scanDSL2Transaction(row pgx.Row) (*state.DSL2Transaction, error) { encoded := []byte{} postState := []byte{} imStateRoot := []byte{} + if err := row.Scan( &l2Transaction.L2BlockNumber, &l2Transaction.EffectiveGasPricePercentage, &encoded, &postState, &imStateRoot, + &l2Transaction.Index, ); err != nil { return nil, err } diff --git a/tools/datastreamer/main.go b/tools/datastreamer/main.go index 97a5eab879..a2e3d19003 100644 --- a/tools/datastreamer/main.go +++ b/tools/datastreamer/main.go @@ -700,12 +700,6 @@ func decodeBatchOffline(cliCtx *cli.Context) error { } func printEntry(entry datastreamer.FileEntry) { - var bookmarkTypeDesc = map[datastream.BookmarkType]string{ - datastream.BookmarkType_BOOKMARK_TYPE_UNSPECIFIED: "Unspecified", - datastream.BookmarkType_BOOKMARK_TYPE_BATCH: "Batch Number", - datastream.BookmarkType_BOOKMARK_TYPE_L2_BLOCK: "L2 Block Number", - } - switch entry.Type { case state.EntryTypeBookMark: bookmark := &datastream.BookMark{} @@ -720,7 +714,7 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgGreen, "Entry Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) printColored(color.FgGreen, "Type............: ") - printColored(color.FgHiWhite, fmt.Sprintf("%d (%s)\n", bookmark.Type, bookmarkTypeDesc[bookmark.Type])) + printColored(color.FgHiWhite, fmt.Sprintf("%d (%s)\n", bookmark.Type, datastream.BookmarkType_name[int32(bookmark.Type)])) printColored(color.FgGreen, "Value...........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", bookmark.Value)) case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK): @@ -757,6 +751,16 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.GlobalExitRoot))) printColored(color.FgGreen, "Coinbase........: ") printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToAddress(l2Block.Coinbase))) + printColored(color.FgGreen, "Block Gas Limit.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", l2Block.BlockGasLimit)) + printColored(color.FgGreen, "Block Info Root.: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", common.BytesToHash(l2Block.BlockInfoRoot))) + + if l2Block.Debug != nil && l2Block.Debug.Message != "" { + printColored(color.FgGreen, "Debug...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", l2Block.Debug)) + } + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_START): batch := &datastream.BatchStart{} err := proto.Unmarshal(entry.Data, batch) @@ -770,10 +774,18 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) printColored(color.FgGreen, "Batch Number....: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.Number)) + printColored(color.FgGreen, "Batch Type......: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", datastream.BatchType_name[int32(batch.Type)])) printColored(color.FgGreen, "Fork ID.........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ForkId)) printColored(color.FgGreen, "Chain ID........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", batch.ChainId)) + + if batch.Debug != nil && batch.Debug.Message != "" { + printColored(color.FgGreen, "Debug...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", batch.Debug)) + } + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_BATCH_END): batch := &datastream.BatchEnd{} err := proto.Unmarshal(entry.Data, batch) @@ -791,6 +803,12 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.StateRoot))) printColored(color.FgGreen, "Local Exit Root.: ") printColored(color.FgHiWhite, fmt.Sprintf("%s\n", "0x"+common.Bytes2Hex(batch.LocalExitRoot))) + + if batch.Debug != nil && batch.Debug.Message != "" { + printColored(color.FgGreen, "Debug...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", batch.Debug)) + } + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_TRANSACTION): dsTx := &datastream.Transaction{} err := proto.Unmarshal(entry.Data, dsTx) @@ -805,6 +823,8 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%d\n", entry.Number)) printColored(color.FgGreen, "L2 Block Number.: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", dsTx.L2BlockNumber)) + printColored(color.FgGreen, "Index...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%d\n", dsTx.Index)) printColored(color.FgGreen, "Is Valid........: ") printColored(color.FgHiWhite, fmt.Sprintf("%t\n", dsTx.IsValid)) printColored(color.FgGreen, "Data............: ") @@ -831,6 +851,12 @@ func printEntry(entry datastreamer.FileEntry) { nonce := tx.Nonce() printColored(color.FgGreen, "Nonce...........: ") printColored(color.FgHiWhite, fmt.Sprintf("%d\n", nonce)) + + if dsTx.Debug != nil && dsTx.Debug.Message != "" { + printColored(color.FgGreen, "Debug...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", dsTx.Debug)) + } + case datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_UPDATE_GER): updateGer := &datastream.UpdateGER{} err := proto.Unmarshal(entry.Data, updateGer) @@ -857,6 +883,11 @@ func printEntry(entry datastreamer.FileEntry) { printColored(color.FgHiWhite, fmt.Sprintf("%d\n", updateGer.ChainId)) printColored(color.FgGreen, "State Root......: ") printColored(color.FgHiWhite, fmt.Sprint(common.Bytes2Hex(updateGer.StateRoot)+"\n")) + + if updateGer.Debug != nil && updateGer.Debug.Message != "" { + printColored(color.FgGreen, "Debug...........: ") + printColored(color.FgHiWhite, fmt.Sprintf("%s\n", updateGer.Debug)) + } } } From 3ee3b6e7369d90838c45589eed7d5754621381f3 Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Mon, 6 May 2024 18:45:34 +0200 Subject: [PATCH 06/22] Fix DSSendBatchEnd (#3621) --- sequencer/batch.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sequencer/batch.go b/sequencer/batch.go index 79d84026dd..c6e2700fa8 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -406,7 +406,7 @@ func (f *finalizer) closeSIPBatch(ctx context.Context, dbTx pgx.Tx) error { } // Sent batch to DS - f.DSSendBatchEnd(f.wipBatch.batchNumber, f.wipBatch.finalStateRoot, f.wipBatch.finalLocalExitRoot) + f.DSSendBatchEnd(f.sipBatch.batchNumber, f.sipBatch.finalStateRoot, f.sipBatch.finalLocalExitRoot) log.Infof("sip batch %d closed in statedb, closing reason: %s", f.sipBatch.batchNumber, f.sipBatch.closingReason) From ff71691c5349ccc5ef2b5ebd8ac42d60f16a4265 Mon Sep 17 00:00:00 2001 From: Thiago Coimbra Lemos Date: Thu, 9 May 2024 14:18:10 -0300 Subject: [PATCH 07/22] add TxFeeCap config (#3611) --- config/config_test.go | 4 + config/default.go | 1 + docs/config-file/node-config-doc.html | 2 +- docs/config-file/node-config-doc.md | 44 ++++++--- docs/config-file/node-config-schema.json | 5 + pool/config.go | 4 + pool/pool.go | 22 +++++ pool/pool_test.go | 114 ++++++++++++++++++++++- 8 files changed, 180 insertions(+), 16 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index f93735eeb9..77ad9611d5 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -298,6 +298,10 @@ func Test_Defaults(t *testing.T) { path: "Pool.GlobalQueue", expectedValue: uint64(1024), }, + { + path: "Pool.TxFeeCap", + expectedValue: float64(1), + }, { path: "Pool.EffectiveGasPrice.Enabled", expectedValue: false, diff --git a/config/default.go b/config/default.go index f55feed513..17bd2baec2 100644 --- a/config/default.go +++ b/config/default.go @@ -44,6 +44,7 @@ MinAllowedGasPriceInterval = "5m" PollMinAllowedGasPriceInterval = "15s" AccountQueue = 64 GlobalQueue = 1024 +TxFeeCap = 1.0 [Pool.EffectiveGasPrice] Enabled = false L1GasPriceFactor = 0.25 diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 9a82ded83c..c5c9909036 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -10,7 +10,7 @@
"300ms"
 

Default: "15s"Type: string

PollMinAllowedGasPriceInterval is the interval to poll the suggested min gas price for a tx


Examples:

"1m"
 
"300ms"
-

Default: 64Type: integer

AccountQueue represents the maximum number of non-executable transaction slots permitted per account


Default: 1024Type: integer

GlobalQueue represents the maximum number of non-executable transaction slots for all accounts


EffectiveGasPrice is the config for the effective gas price calculation
Default: falseType: boolean

Enabled is a flag to enable/disable the effective gas price


Default: 0.25Type: number

L1GasPriceFactor is the percentage of the L1 gas price that will be used as the L2 min gas price


Default: 16Type: integer

ByteGasCost is the gas cost per byte that is not 0


Default: 4Type: integer

ZeroByteGasCost is the gas cost per byte that is 0


Default: 1Type: number

NetProfit is the profit margin to apply to the calculated breakEvenGasPrice


Default: 1.1Type: number

BreakEvenFactor is the factor to apply to the calculated breakevenGasPrice when comparing it with the gasPriceSigned of a tx


Default: 10Type: integer

FinalDeviationPct is the max allowed deviation percentage BreakEvenGasPrice on re-calculation


Default: 0Type: integer

EthTransferGasPrice is the fixed gas price returned as effective gas price for txs tha are ETH transfers (0 means disabled)
Only one of EthTransferGasPrice or EthTransferL1GasPriceFactor params can be different than 0. If both params are set to 0, the sequencer will halt and log an error


Default: 0Type: number

EthTransferL1GasPriceFactor is the percentage of L1 gas price returned as effective gas price for txs tha are ETH transfers (0 means disabled)
Only one of EthTransferGasPrice or EthTransferL1GasPriceFactor params can be different than 0. If both params are set to 0, the sequencer will halt and log an error


Default: 0.5Type: number

L2GasPriceSuggesterFactor is the factor to apply to L1 gas price to get the suggested L2 gas price used in the
calculations when the effective gas price is disabled (testing/metrics purposes)


Default: 0Type: integer

ForkID is the current fork ID of the chain


Configuration for RPC service. THis one offers a extended Ethereum JSON-RPC API interface to interact with the node
Default: "0.0.0.0"Type: string

Host defines the network adapter that will be used to serve the HTTP requests


Default: 8545Type: integer

Port defines the port to serve the endpoints via HTTP


Default: "1m0s"Type: string

ReadTimeout is the HTTP server read timeout
check net/http.server.ReadTimeout and net/http.server.ReadHeaderTimeout


Examples:

"1m"
+

Default: 64Type: integer

AccountQueue represents the maximum number of non-executable transaction slots permitted per account


Default: 1024Type: integer

GlobalQueue represents the maximum number of non-executable transaction slots for all accounts


EffectiveGasPrice is the config for the effective gas price calculation
Default: falseType: boolean

Enabled is a flag to enable/disable the effective gas price


Default: 0.25Type: number

L1GasPriceFactor is the percentage of the L1 gas price that will be used as the L2 min gas price


Default: 16Type: integer

ByteGasCost is the gas cost per byte that is not 0


Default: 4Type: integer

ZeroByteGasCost is the gas cost per byte that is 0


Default: 1Type: number

NetProfit is the profit margin to apply to the calculated breakEvenGasPrice


Default: 1.1Type: number

BreakEvenFactor is the factor to apply to the calculated breakevenGasPrice when comparing it with the gasPriceSigned of a tx


Default: 10Type: integer

FinalDeviationPct is the max allowed deviation percentage BreakEvenGasPrice on re-calculation


Default: 0Type: integer

EthTransferGasPrice is the fixed gas price returned as effective gas price for txs tha are ETH transfers (0 means disabled)
Only one of EthTransferGasPrice or EthTransferL1GasPriceFactor params can be different than 0. If both params are set to 0, the sequencer will halt and log an error


Default: 0Type: number

EthTransferL1GasPriceFactor is the percentage of L1 gas price returned as effective gas price for txs tha are ETH transfers (0 means disabled)
Only one of EthTransferGasPrice or EthTransferL1GasPriceFactor params can be different than 0. If both params are set to 0, the sequencer will halt and log an error


Default: 0.5Type: number

L2GasPriceSuggesterFactor is the factor to apply to L1 gas price to get the suggested L2 gas price used in the
calculations when the effective gas price is disabled (testing/metrics purposes)


Default: 0Type: integer

ForkID is the current fork ID of the chain


Default: 1Type: number

TxFeeCap is the global transaction fee(price * gaslimit) cap for
send-transaction variants. The unit is ether. 0 means no cap.


Configuration for RPC service. THis one offers a extended Ethereum JSON-RPC API interface to interact with the node
Default: "0.0.0.0"Type: string

Host defines the network adapter that will be used to serve the HTTP requests


Default: 8545Type: integer

Port defines the port to serve the endpoints via HTTP


Default: "1m0s"Type: string

ReadTimeout is the HTTP server read timeout
check net/http.server.ReadTimeout and net/http.server.ReadHeaderTimeout


Examples:

"1m"
 
"300ms"
 

Default: "1m0s"Type: string

WriteTimeout is the HTTP server write timeout
check net/http.server.WriteTimeout


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index ea312a6929..c23751bd20 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -412,20 +412,21 @@ MaxGasPriceLimit=0
 **Type:** : `object`
 **Description:** Pool service configuration
 
-| Property                                                                        | Pattern | Type    | Deprecated | Definition | Title/Description                                                                                    |
-| ------------------------------------------------------------------------------- | ------- | ------- | ---------- | ---------- | ---------------------------------------------------------------------------------------------------- |
-| - [IntervalToRefreshBlockedAddresses](#Pool_IntervalToRefreshBlockedAddresses ) | No      | string  | No         | -          | Duration                                                                                             |
-| - [IntervalToRefreshGasPrices](#Pool_IntervalToRefreshGasPrices )               | No      | string  | No         | -          | Duration                                                                                             |
-| - [MaxTxBytesSize](#Pool_MaxTxBytesSize )                                       | No      | integer | No         | -          | MaxTxBytesSize is the max size of a transaction in bytes                                             |
-| - [MaxTxDataBytesSize](#Pool_MaxTxDataBytesSize )                               | No      | integer | No         | -          | MaxTxDataBytesSize is the max size of the data field of a transaction in bytes                       |
-| - [DB](#Pool_DB )                                                               | No      | object  | No         | -          | DB is the database configuration                                                                     |
-| - [DefaultMinGasPriceAllowed](#Pool_DefaultMinGasPriceAllowed )                 | No      | integer | No         | -          | DefaultMinGasPriceAllowed is the default min gas price to suggest                                    |
-| - [MinAllowedGasPriceInterval](#Pool_MinAllowedGasPriceInterval )               | No      | string  | No         | -          | Duration                                                                                             |
-| - [PollMinAllowedGasPriceInterval](#Pool_PollMinAllowedGasPriceInterval )       | No      | string  | No         | -          | Duration                                                                                             |
-| - [AccountQueue](#Pool_AccountQueue )                                           | No      | integer | No         | -          | AccountQueue represents the maximum number of non-executable transaction slots permitted per account |
-| - [GlobalQueue](#Pool_GlobalQueue )                                             | No      | integer | No         | -          | GlobalQueue represents the maximum number of non-executable transaction slots for all accounts       |
-| - [EffectiveGasPrice](#Pool_EffectiveGasPrice )                                 | No      | object  | No         | -          | EffectiveGasPrice is the config for the effective gas price calculation                              |
-| - [ForkID](#Pool_ForkID )                                                       | No      | integer | No         | -          | ForkID is the current fork ID of the chain                                                           |
+| Property                                                                        | Pattern | Type    | Deprecated | Definition | Title/Description                                                                                                                   |
+| ------------------------------------------------------------------------------- | ------- | ------- | ---------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------- |
+| - [IntervalToRefreshBlockedAddresses](#Pool_IntervalToRefreshBlockedAddresses ) | No      | string  | No         | -          | Duration                                                                                                                            |
+| - [IntervalToRefreshGasPrices](#Pool_IntervalToRefreshGasPrices )               | No      | string  | No         | -          | Duration                                                                                                                            |
+| - [MaxTxBytesSize](#Pool_MaxTxBytesSize )                                       | No      | integer | No         | -          | MaxTxBytesSize is the max size of a transaction in bytes                                                                            |
+| - [MaxTxDataBytesSize](#Pool_MaxTxDataBytesSize )                               | No      | integer | No         | -          | MaxTxDataBytesSize is the max size of the data field of a transaction in bytes                                                      |
+| - [DB](#Pool_DB )                                                               | No      | object  | No         | -          | DB is the database configuration                                                                                                    |
+| - [DefaultMinGasPriceAllowed](#Pool_DefaultMinGasPriceAllowed )                 | No      | integer | No         | -          | DefaultMinGasPriceAllowed is the default min gas price to suggest                                                                   |
+| - [MinAllowedGasPriceInterval](#Pool_MinAllowedGasPriceInterval )               | No      | string  | No         | -          | Duration                                                                                                                            |
+| - [PollMinAllowedGasPriceInterval](#Pool_PollMinAllowedGasPriceInterval )       | No      | string  | No         | -          | Duration                                                                                                                            |
+| - [AccountQueue](#Pool_AccountQueue )                                           | No      | integer | No         | -          | AccountQueue represents the maximum number of non-executable transaction slots permitted per account                                |
+| - [GlobalQueue](#Pool_GlobalQueue )                                             | No      | integer | No         | -          | GlobalQueue represents the maximum number of non-executable transaction slots for all accounts                                      |
+| - [EffectiveGasPrice](#Pool_EffectiveGasPrice )                                 | No      | object  | No         | -          | EffectiveGasPrice is the config for the effective gas price calculation                                                             |
+| - [ForkID](#Pool_ForkID )                                                       | No      | integer | No         | -          | ForkID is the current fork ID of the chain                                                                                          |
+| - [TxFeeCap](#Pool_TxFeeCap )                                                   | No      | number  | No         | -          | TxFeeCap is the global transaction fee(price * gaslimit) cap for
send-transaction variants. The unit is ether. 0 means no cap. | ### 7.1. `Pool.IntervalToRefreshBlockedAddresses` @@ -890,6 +891,21 @@ L2GasPriceSuggesterFactor=0.5 ForkID=0 ``` +### 7.13. `Pool.TxFeeCap` + +**Type:** : `number` + +**Default:** `1` + +**Description:** TxFeeCap is the global transaction fee(price * gaslimit) cap for +send-transaction variants. The unit is ether. 0 means no cap. + +**Example setting the default value** (1): +``` +[Pool] +TxFeeCap=1 +``` + ## 8. `[RPC]` **Type:** : `object` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index dffb74f672..fb834ab093 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -327,6 +327,11 @@ "type": "integer", "description": "ForkID is the current fork ID of the chain", "default": 0 + }, + "TxFeeCap": { + "type": "number", + "description": "TxFeeCap is the global transaction fee(price * gaslimit) cap for\nsend-transaction variants. The unit is ether. 0 means no cap.", + "default": 1 } }, "additionalProperties": false, diff --git a/pool/config.go b/pool/config.go index 869c76a86c..62bfcc8512 100644 --- a/pool/config.go +++ b/pool/config.go @@ -43,6 +43,10 @@ type Config struct { // ForkID is the current fork ID of the chain ForkID uint64 `mapstructure:"ForkID"` + + // TxFeeCap is the global transaction fee(price * gaslimit) cap for + // send-transaction variants. The unit is ether. 0 means no cap. + TxFeeCap float64 `mapstructure:"TxFeeCap"` } // EffectiveGasPriceCfg contains the configuration properties for the effective gas price diff --git a/pool/pool.go b/pool/pool.go index 1677318a9f..62f7611ea4 100644 --- a/pool/pool.go +++ b/pool/pool.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "math/big" + "strconv" "sync" "time" @@ -16,6 +17,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/math" "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/params" ) var ( @@ -456,6 +458,10 @@ func (p *Pool) validateTx(ctx context.Context, poolTx Transaction) error { return ErrNegativeValue } + if err := checkTxFee(poolTx.GasPrice(), poolTx.Gas(), p.cfg.TxFeeCap); err != nil { + return err + } + // check if sender is blocked _, blocked := p.blockedAddresses.Load(from.String()) if blocked { @@ -728,3 +734,19 @@ func IntrinsicGas(tx types.Transaction) (uint64, error) { } return gas, nil } + +// checkTxFee is an internal function used to check whether the fee of +// the given transaction is _reasonable_(under the cap). +func checkTxFee(gasPrice *big.Int, gas uint64, cap float64) error { + // Short circuit if there is no cap for transaction fee at all. + if cap == 0 { + return nil + } + feeEth := new(big.Float).Quo(new(big.Float).SetInt(new(big.Int).Mul(gasPrice, new(big.Int).SetUint64(gas))), new(big.Float).SetInt(big.NewInt(params.Ether))) + feeFloat, _ := feeEth.Float64() + if feeFloat > cap { + feeFloatTruncated := strconv.FormatFloat(feeFloat, 'f', -1, 64) + return fmt.Errorf("tx fee (%s ether) exceeds the configured cap (%.2f ether)", feeFloatTruncated, cap) + } + return nil +} diff --git a/pool/pool_test.go b/pool/pool_test.go index a68b3bdebe..2bfe1d7b1a 100644 --- a/pool/pool_test.go +++ b/pool/pool_test.go @@ -67,6 +67,7 @@ var ( IntervalToRefreshGasPrices: cfgTypes.NewDuration(5 * time.Second), AccountQueue: 15, GlobalQueue: 20, + TxFeeCap: 1, EffectiveGasPrice: pool.EffectiveGasPriceCfg{ Enabled: true, L1GasPriceFactor: 0.25, @@ -1086,10 +1087,12 @@ func Test_TryAddIncompatibleTxs(t *testing.T) { expectedError: fmt.Errorf("chain id higher than allowed, max allowed is %v", uint64(math.MaxUint64)), }, } + c := cfg + c.TxFeeCap = 0 for _, testCase := range testCases { t.Run(testCase.name, func(t *testing.T) { incompatibleTx := testCase.createIncompatibleTx() - p := setupPool(t, cfg, bc, s, st, incompatibleTx.ChainId().Uint64(), ctx, eventLog) + p := setupPool(t, c, bc, s, st, incompatibleTx.ChainId().Uint64(), ctx, eventLog) err = p.AddTx(ctx, incompatibleTx, ip) assert.Equal(t, testCase.expectedError, err) }) @@ -1965,6 +1968,115 @@ func Test_AddTx_IPValidation(t *testing.T) { } } +func Test_AddTx_TxFeeCap(t *testing.T) { + eventStorage, err := nileventstorage.NewNilEventStorage() + if err != nil { + log.Fatal(err) + } + eventLog := event.NewEventLog(event.Config{}, eventStorage) + + initOrResetDB(t) + + stateSqlDB, err := db.NewSQLDB(stateDBCfg) + if err != nil { + panic(err) + } + defer stateSqlDB.Close() //nolint:gosec,errcheck + + poolSqlDB, err := db.NewSQLDB(poolDBCfg) + require.NoError(t, err) + defer poolSqlDB.Close() //nolint:gosec,errcheck + + st := newState(stateSqlDB, eventLog) + + genesisBlock := state.Block{ + BlockNumber: 0, + BlockHash: state.ZeroHash, + ParentHash: state.ZeroHash, + ReceivedAt: time.Now(), + } + genesis := state.Genesis{ + Actions: []*state.GenesisAction{ + { + Address: senderAddress, + Type: int(merkletree.LeafTypeBalance), + Value: "1000000000000000000000", + }, + }, + } + ctx := context.Background() + dbTx, err := st.BeginStateTransaction(ctx) + require.NoError(t, err) + _, err = st.SetGenesis(ctx, genesisBlock, genesis, metrics.SynchronizerCallerLabel, dbTx) + require.NoError(t, err) + require.NoError(t, dbTx.Commit(ctx)) + + s, err := pgpoolstorage.NewPostgresPoolStorage(poolDBCfg) + require.NoError(t, err) + + p := setupPool(t, cfg, bc, s, st, chainID.Uint64(), ctx, eventLog) + + privateKey, err := crypto.HexToECDSA(strings.TrimPrefix(senderPrivateKey, "0x")) + require.NoError(t, err) + + auth, err := bind.NewKeyedTransactorWithChainID(privateKey, chainID) + require.NoError(t, err) + + type testCase struct { + name string + nonce uint64 + gas uint64 + gasPrice string + expectedError error + } + + testCases := []testCase{ + { + name: "add tx with fee under cap", + nonce: 0, + gas: uint64(100000), + gasPrice: "9999999999999", + expectedError: nil, + }, + { + name: "add tx with fee exactly as cap", + nonce: 0, + gas: uint64(100000), + gasPrice: "10000000000000", + expectedError: nil, + }, + { + name: "add tx with fee over the cap", + nonce: 0, + gas: uint64(100000), + gasPrice: "10000000000001", + expectedError: fmt.Errorf("tx fee (1.0000000000001 ether) exceeds the configured cap (1.00 ether)"), + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + gasPrice, ok := big.NewInt(0).SetString(tc.gasPrice, encoding.Base10) + require.True(t, ok) + tx := ethTypes.NewTx(ðTypes.LegacyTx{ + Nonce: tc.nonce, + Gas: tc.gas, + GasPrice: gasPrice, + }) + + signedTx, err := auth.Signer(auth.From, tx) + require.NoError(t, err) + + err = p.AddTx(ctx, *signedTx, ip) + if tc.expectedError != nil { + require.Equal(t, err.Error(), tc.expectedError.Error()) + } else { + require.Nil(t, err) + } + }) + } +} + func setupPool(t *testing.T, cfg pool.Config, constraintsCfg state.BatchConstraintsCfg, s *pgpoolstorage.PostgresPoolStorage, st *state.State, chainID uint64, ctx context.Context, eventLog *event.EventLog) *pool.Pool { err := s.SetGasPrices(ctx, gasPrice.Uint64(), l1GasPrice.Uint64()) require.NoError(t, err) From 60d6c74103730e822eb32d6a7baa3f5c3f40e58e Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Wed, 22 May 2024 10:49:22 +0200 Subject: [PATCH 08/22] Add delay when getting storedFlushId from the executor/hashdb (#3625) * Add sleep delay when getting storedFlushId from the executor/hashdb * update config doc --- config/config_test.go | 4 ++ config/default.go | 1 + .../environments/local/local.node.config.toml | 1 + docs/config-file/node-config-doc.html | 2 + docs/config-file/node-config-doc.md | 39 ++++++++++++++++--- docs/config-file/node-config-schema.json | 10 +++++ sequencer/config.go | 3 ++ sequencer/finalizer.go | 7 ++++ sequencer/l2block.go | 26 +++++++------ test/config/debug.node.config.toml | 1 + test/config/test.node.config.toml | 1 + 11 files changed, 77 insertions(+), 18 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 77ad9611d5..0fd606a038 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -133,6 +133,10 @@ func Test_Defaults(t *testing.T) { path: "Sequencer.Finalizer.BatchMaxDeltaTimestamp", expectedValue: types.NewDuration(1800 * time.Second), }, + { + path: "Sequencer.Finalizer.FlushIdCheckInterval", + expectedValue: types.NewDuration(50 * time.Millisecond), + }, { path: "Sequencer.Finalizer.Metrics.Interval", expectedValue: types.NewDuration(60 * time.Minute), diff --git a/config/default.go b/config/default.go index 17bd2baec2..5834e90793 100644 --- a/config/default.go +++ b/config/default.go @@ -151,6 +151,7 @@ StateConsistencyCheckInterval = "5s" L2BlockMaxDeltaTimestamp = "3s" ResourceExhaustedMarginPct = 10 StateRootSyncInterval = "3600s" + FlushIdCheckInterval = "50ms" HaltOnBatchNumber = 0 SequentialBatchSanityCheck = false SequentialProcessL2Block = false diff --git a/config/environments/local/local.node.config.toml b/config/environments/local/local.node.config.toml index 436a0d84fa..376465b3c8 100644 --- a/config/environments/local/local.node.config.toml +++ b/config/environments/local/local.node.config.toml @@ -102,6 +102,7 @@ StateConsistencyCheckInterval = "5s" L2BlockMaxDeltaTimestamp = "3s" ResourceExhaustedMarginPct = 10 StateRootSyncInterval = "360s" + FlushIdCheckInterval = "50ms" HaltOnBatchNumber = 0 SequentialBatchSanityCheck = false SequentialProcessL2Block = false diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index c5c9909036..105b99ff15 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -52,6 +52,8 @@
"300ms"
 

Default: "1h0m0s"Type: string

StateRootSyncInterval indicates how often the stateroot generated by the L2 block process will be synchronized with
the stateroot used in the tx-by-tx execution


Examples:

"1m"
 
"300ms"
+

Default: "50ms"Type: string

FlushIdCheckInterval is the time interval to get storedFlushID value from the executor/hashdb


Examples:

"1m"
+
"300ms"
 

Default: 0Type: integer

HaltOnBatchNumber specifies the batch number where the Sequencer will stop to process more transactions and generate new batches.
The Sequencer will halt after it closes the batch equal to this number


Default: falseType: boolean

SequentialBatchSanityCheck indicates if the reprocess of a closed batch (sanity check) must be done in a
sequential way (instead than in parallel)


Default: falseType: boolean

SequentialProcessL2Block indicates if the processing of a L2 Block must be done in the same finalizer go func instead
in the processPendingL2Blocks go func


Metrics is the config for the sequencer metrics
Default: "1h0m0s"Type: string

Interval is the interval of time to calculate sequencer metrics


Examples:

"1m"
 
"300ms"
 

Default: trueType: boolean

EnableLog is a flag to enable/disable metrics logs


StreamServerCfg is the config for the stream server
Default: 0Type: integer

Port to listen on


Default: ""Type: string

Filename of the binary data file


Default: 0Type: integer

Version of the binary data file


Default: 0Type: integer

ChainID is the chain ID


Default: falseType: boolean

Enabled is a flag to enable/disable the data streamer


Log is the log configuration
Default: ""Type: enum (of string)

Must be one of:

  • "production"
  • "development"

Default: ""Type: enum (of string)

Must be one of:

  • "debug"
  • "info"
  • "warn"
  • "error"
  • "dpanic"
  • "panic"
  • "fatal"

Type: array of string

Each item of this array must be:


Default: 0Type: integer

UpgradeEtrogBatchNumber is the batch number of the upgrade etrog


Configuration of the sequence sender service
Default: "5s"Type: string

WaitPeriodSendSequence is the time the sequencer waits until
trying to send a sequence to L1


Examples:

"1m"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index c23751bd20..440cd5dac6 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -2100,6 +2100,7 @@ StateConsistencyCheckInterval="5s"
 | - [BatchMaxDeltaTimestamp](#Sequencer_Finalizer_BatchMaxDeltaTimestamp )                       | No      | string  | No         | -          | Duration                                                                                                                                                                                                      |
 | - [L2BlockMaxDeltaTimestamp](#Sequencer_Finalizer_L2BlockMaxDeltaTimestamp )                   | No      | string  | No         | -          | Duration                                                                                                                                                                                                      |
 | - [StateRootSyncInterval](#Sequencer_Finalizer_StateRootSyncInterval )                         | No      | string  | No         | -          | Duration                                                                                                                                                                                                      |
+| - [FlushIdCheckInterval](#Sequencer_Finalizer_FlushIdCheckInterval )                           | No      | string  | No         | -          | Duration                                                                                                                                                                                                      |
 | - [HaltOnBatchNumber](#Sequencer_Finalizer_HaltOnBatchNumber )                                 | No      | integer | No         | -          | HaltOnBatchNumber specifies the batch number where the Sequencer will stop to process more transactions and generate new batches.
The Sequencer will halt after it closes the batch equal to this number | | - [SequentialBatchSanityCheck](#Sequencer_Finalizer_SequentialBatchSanityCheck ) | No | boolean | No | - | SequentialBatchSanityCheck indicates if the reprocess of a closed batch (sanity check) must be done in a
sequential way (instead than in parallel) | | - [SequentialProcessL2Block](#Sequencer_Finalizer_SequentialProcessL2Block ) | No | boolean | No | - | SequentialProcessL2Block indicates if the processing of a L2 Block must be done in the same finalizer go func instead
in the processPendingL2Blocks go func | @@ -2330,7 +2331,33 @@ the stateroot used in the tx-by-tx execution StateRootSyncInterval="1h0m0s" ``` -#### 10.7.11. `Sequencer.Finalizer.HaltOnBatchNumber` +#### 10.7.11. `Sequencer.Finalizer.FlushIdCheckInterval` + +**Title:** Duration + +**Type:** : `string` + +**Default:** `"50ms"` + +**Description:** FlushIdCheckInterval is the time interval to get storedFlushID value from the executor/hashdb + +**Examples:** + +```json +"1m" +``` + +```json +"300ms" +``` + +**Example setting the default value** ("50ms"): +``` +[Sequencer.Finalizer] +FlushIdCheckInterval="50ms" +``` + +#### 10.7.12. `Sequencer.Finalizer.HaltOnBatchNumber` **Type:** : `integer` @@ -2345,7 +2372,7 @@ The Sequencer will halt after it closes the batch equal to this number HaltOnBatchNumber=0 ``` -#### 10.7.12. `Sequencer.Finalizer.SequentialBatchSanityCheck` +#### 10.7.13. `Sequencer.Finalizer.SequentialBatchSanityCheck` **Type:** : `boolean` @@ -2360,7 +2387,7 @@ sequential way (instead than in parallel) SequentialBatchSanityCheck=false ``` -#### 10.7.13. `Sequencer.Finalizer.SequentialProcessL2Block` +#### 10.7.14. `Sequencer.Finalizer.SequentialProcessL2Block` **Type:** : `boolean` @@ -2375,7 +2402,7 @@ in the processPendingL2Blocks go func SequentialProcessL2Block=false ``` -#### 10.7.14. `[Sequencer.Finalizer.Metrics]` +#### 10.7.15. `[Sequencer.Finalizer.Metrics]` **Type:** : `object` **Description:** Metrics is the config for the sequencer metrics @@ -2385,7 +2412,7 @@ SequentialProcessL2Block=false | - [Interval](#Sequencer_Finalizer_Metrics_Interval ) | No | string | No | - | Duration | | - [EnableLog](#Sequencer_Finalizer_Metrics_EnableLog ) | No | boolean | No | - | EnableLog is a flag to enable/disable metrics logs | -##### 10.7.14.1. `Sequencer.Finalizer.Metrics.Interval` +##### 10.7.15.1. `Sequencer.Finalizer.Metrics.Interval` **Title:** Duration @@ -2411,7 +2438,7 @@ SequentialProcessL2Block=false Interval="1h0m0s" ``` -##### 10.7.14.2. `Sequencer.Finalizer.Metrics.EnableLog` +##### 10.7.15.2. `Sequencer.Finalizer.Metrics.EnableLog` **Type:** : `boolean` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index fb834ab093..bf7314dc1f 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -867,6 +867,16 @@ "300ms" ] }, + "FlushIdCheckInterval": { + "type": "string", + "title": "Duration", + "description": "FlushIdCheckInterval is the time interval to get storedFlushID value from the executor/hashdb", + "default": "50ms", + "examples": [ + "1m", + "300ms" + ] + }, "HaltOnBatchNumber": { "type": "integer", "description": "HaltOnBatchNumber specifies the batch number where the Sequencer will stop to process more transactions and generate new batches.\nThe Sequencer will halt after it closes the batch equal to this number", diff --git a/sequencer/config.go b/sequencer/config.go index 8b813c52db..c0bfb8968d 100644 --- a/sequencer/config.go +++ b/sequencer/config.go @@ -83,6 +83,9 @@ type FinalizerCfg struct { // the stateroot used in the tx-by-tx execution StateRootSyncInterval types.Duration `mapstructure:"StateRootSyncInterval"` + // FlushIdCheckInterval is the time interval to get storedFlushID value from the executor/hashdb + FlushIdCheckInterval types.Duration `mapstructure:"FlushIdCheckInterval"` + // HaltOnBatchNumber specifies the batch number where the Sequencer will stop to process more transactions and generate new batches. // The Sequencer will halt after it closes the batch equal to this number HaltOnBatchNumber uint64 `mapstructure:"HaltOnBatchNumber"` diff --git a/sequencer/finalizer.go b/sequencer/finalizer.go index a06f2341fa..a7e884ac07 100644 --- a/sequencer/finalizer.go +++ b/sequencer/finalizer.go @@ -208,8 +208,15 @@ func (f *finalizer) updateProverIdAndFlushId(ctx context.Context) { f.storedFlushID = storedFlushID f.storedFlushIDCond.Broadcast() f.storedFlushIDCond.L.Unlock() + + // Exit the for loop o the storedFlushId is greater or equal that the lastPendingFlushID + if f.storedFlushID >= f.lastPendingFlushID { + break + } } } + + time.Sleep(f.cfg.FlushIdCheckInterval.Duration) } } } diff --git a/sequencer/l2block.go b/sequencer/l2block.go index b20be1e67a..fed7a631f9 100644 --- a/sequencer/l2block.go +++ b/sequencer/l2block.go @@ -265,10 +265,12 @@ func (f *finalizer) processL2Block(ctx context.Context, l2Block *L2Block) error f.updateFlushIDs(batchResponse.FlushID, batchResponse.StoredFlushID) + var waitStoreL2Block time.Duration if f.pendingL2BlocksToStoreWG.Count() > 0 { startWait := time.Now() f.pendingL2BlocksToStoreWG.Wait() - log.Debugf("waiting for previous L2 block to be stored took: %v", time.Since(startWait)) + waitStoreL2Block = time.Since(startWait) + log.Debugf("waiting for previous L2 block to be stored took: %v", waitStoreL2Block) } f.addPendingL2BlockToStore(ctx, l2Block) @@ -279,9 +281,9 @@ func (f *finalizer) processL2Block(ctx context.Context, l2Block *L2Block) error } f.metrics.addL2BlockMetrics(l2Block.metrics) - log.Infof("processed L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, initialStateRoot: %s, newStateRoot: %s, txs: %d/%d, blockHash: %s, infoRoot: %s, counters: {used: %s, reserved: %s, needed: %s, high: %s}, contextId: %s", + log.Infof("processed L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, initialStateRoot: %s, newStateRoot: %s, txs: %d/%d, blockHash: %s, infoRoot: %s, waitStoreL2Block: %v, counters: {used: %s, reserved: %s, needed: %s, high: %s}, contextId: %s", blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, l2Block.deltaTimestamp, l2Block.timestamp, l2Block.l1InfoTreeExitRoot.L1InfoTreeIndex, l2Block.l1InfoTreeExitRootChanged, initialStateRoot, l2Block.batchResponse.NewStateRoot, - len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot, + len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot, waitStoreL2Block, f.logZKCounters(batchResponse.UsedZkCounters), f.logZKCounters(batchResponse.ReservedZkCounters), f.logZKCounters(neededZKCounters), f.logZKCounters(l2Block.batch.finalHighReservedZKCounters), contextId) if f.cfg.Metrics.EnableLog { @@ -367,19 +369,21 @@ func (f *finalizer) executeL2Block(ctx context.Context, initialStateRoot common. func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { startStoring := time.Now() + blockResponse := l2Block.batchResponse.BlockResponses[0] + log.Infof("storing L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, txs: %d/%d, blockHash: %s, infoRoot: %s", + blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, l2Block.deltaTimestamp, l2Block.timestamp, l2Block.l1InfoTreeExitRoot.L1InfoTreeIndex, + l2Block.l1InfoTreeExitRootChanged, len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot.String()) + // Wait until L2 block has been flushed/stored by the executor + startWaitFlushId := time.Now() f.storedFlushIDCond.L.Lock() for f.storedFlushID < l2Block.batchResponse.FlushID { f.storedFlushIDCond.Wait() } f.storedFlushIDCond.L.Unlock() + waitFlushId := time.Since(startWaitFlushId) // If the L2 block has txs now f.storedFlushID >= l2BlockToStore.flushId, we can store tx - blockResponse := l2Block.batchResponse.BlockResponses[0] - log.Infof("storing L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, txs: %d/%d, blockHash: %s, infoRoot: %s", - blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, l2Block.deltaTimestamp, l2Block.timestamp, l2Block.l1InfoTreeExitRoot.L1InfoTreeIndex, - l2Block.l1InfoTreeExitRootChanged, len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot.String()) - dbTx, err := f.stateIntf.BeginStateTransaction(ctx) if err != nil { return fmt.Errorf("error creating db transaction to store L2 block %d [%d], error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err) @@ -506,11 +510,9 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { f.workerIntf.DeleteTxPendingToStore(tx.Hash, tx.From) } - endStoring := time.Now() - - log.Infof("stored L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, txs: %d/%d, blockHash: %s, infoRoot: %s, time: %v", + log.Infof("stored L2 block %d [%d], batch: %d, deltaTimestamp: %d, timestamp: %d, l1InfoTreeIndex: %d, l1InfoTreeIndexChanged: %v, txs: %d/%d, blockHash: %s, infoRoot: %s, time: %v, waitFlushId: %v", blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, l2Block.deltaTimestamp, l2Block.timestamp, l2Block.l1InfoTreeExitRoot.L1InfoTreeIndex, - l2Block.l1InfoTreeExitRootChanged, len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot.String(), endStoring.Sub(startStoring)) + l2Block.l1InfoTreeExitRootChanged, len(l2Block.transactions), len(blockResponse.TransactionResponses), blockResponse.BlockHash, blockResponse.BlockInfoRoot.String(), time.Since(startStoring), waitFlushId) return nil } diff --git a/test/config/debug.node.config.toml b/test/config/debug.node.config.toml index ccd423daee..698164c866 100644 --- a/test/config/debug.node.config.toml +++ b/test/config/debug.node.config.toml @@ -101,6 +101,7 @@ StateConsistencyCheckInterval = "5s" L2BlockMaxDeltaTimestamp = "3s" ResourceExhaustedMarginPct = 10 StateRootSyncInterval = "120s" + FlushIdCheckInterval = "50ms" HaltOnBatchNumber = 0 SequentialBatchSanityCheck = false SequentialProcessL2Block = false diff --git a/test/config/test.node.config.toml b/test/config/test.node.config.toml index 506dde2466..4f743a7b0a 100644 --- a/test/config/test.node.config.toml +++ b/test/config/test.node.config.toml @@ -116,6 +116,7 @@ StateConsistencyCheckInterval = "5s" L2BlockMaxDeltaTimestamp = "4s" ResourceExhaustedMarginPct = 10 StateRootSyncInterval = "60s" + FlushIdCheckInterval = "50ms" HaltOnBatchNumber = 0 SequentialBatchSanityCheck = false SequentialProcessL2Block = false From fb2c4e4dbe5d107a151797573718a56ac32b365d Mon Sep 17 00:00:00 2001 From: Joan Esteban <129153821+joanestebanr@users.noreply.github.com> Date: Wed, 22 May 2024 12:09:55 +0200 Subject: [PATCH 09/22] cherry-pick: fix #3613 timestamp needs to be greater or equal (#3614) (#3639) syncrhonizer update the tstamp from table state.batch when the batch is sequenced --- state/interfaces.go | 1 + state/mocks/mock_storage.go | 49 +++++++++++++++++++ state/pgstatestorage/batch.go | 8 +++ .../etrog/processor_l1_sequence_batches.go | 14 +++++- .../processor_l1_sequence_batches_test.go | 10 +++- .../mocks/state_full_interface.go | 49 +++++++++++++++++++ synchronizer/common/syncinterfaces/state.go | 1 + synchronizer/synchronizer_test.go | 2 + 8 files changed, 132 insertions(+), 2 deletions(-) diff --git a/state/interfaces.go b/state/interfaces.go index cc1f0127a9..33e8bc01be 100644 --- a/state/interfaces.go +++ b/state/interfaces.go @@ -163,4 +163,5 @@ type storage interface { GetNotCheckedBatches(ctx context.Context, dbTx pgx.Tx) ([]*Batch, error) GetLastL2BlockByBatchNumber(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*L2Block, error) GetPreviousBlockToBlockNumber(ctx context.Context, blockNumber uint64, dbTx pgx.Tx) (*Block, error) + UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error } diff --git a/state/mocks/mock_storage.go b/state/mocks/mock_storage.go index 27964e0247..57b72a61f6 100644 --- a/state/mocks/mock_storage.go +++ b/state/mocks/mock_storage.go @@ -8333,6 +8333,55 @@ func (_c *StorageMock_UpdateBatchL2Data_Call) RunAndReturn(run func(context.Cont return _c } +// UpdateBatchTimestamp provides a mock function with given fields: ctx, batchNumber, timestamp, dbTx +func (_m *StorageMock) UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error { + ret := _m.Called(ctx, batchNumber, timestamp, dbTx) + + if len(ret) == 0 { + panic("no return value specified for UpdateBatchTimestamp") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, uint64, time.Time, pgx.Tx) error); ok { + r0 = rf(ctx, batchNumber, timestamp, dbTx) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// StorageMock_UpdateBatchTimestamp_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateBatchTimestamp' +type StorageMock_UpdateBatchTimestamp_Call struct { + *mock.Call +} + +// UpdateBatchTimestamp is a helper method to define mock.On call +// - ctx context.Context +// - batchNumber uint64 +// - timestamp time.Time +// - dbTx pgx.Tx +func (_e *StorageMock_Expecter) UpdateBatchTimestamp(ctx interface{}, batchNumber interface{}, timestamp interface{}, dbTx interface{}) *StorageMock_UpdateBatchTimestamp_Call { + return &StorageMock_UpdateBatchTimestamp_Call{Call: _e.mock.On("UpdateBatchTimestamp", ctx, batchNumber, timestamp, dbTx)} +} + +func (_c *StorageMock_UpdateBatchTimestamp_Call) Run(run func(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx)) *StorageMock_UpdateBatchTimestamp_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(uint64), args[2].(time.Time), args[3].(pgx.Tx)) + }) + return _c +} + +func (_c *StorageMock_UpdateBatchTimestamp_Call) Return(_a0 error) *StorageMock_UpdateBatchTimestamp_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *StorageMock_UpdateBatchTimestamp_Call) RunAndReturn(run func(context.Context, uint64, time.Time, pgx.Tx) error) *StorageMock_UpdateBatchTimestamp_Call { + _c.Call.Return(run) + return _c +} + // UpdateCheckedBlockByNumber provides a mock function with given fields: ctx, blockNumber, newCheckedStatus, dbTx func (_m *StorageMock) UpdateCheckedBlockByNumber(ctx context.Context, blockNumber uint64, newCheckedStatus bool, dbTx pgx.Tx) error { ret := _m.Called(ctx, blockNumber, newCheckedStatus, dbTx) diff --git a/state/pgstatestorage/batch.go b/state/pgstatestorage/batch.go index 843c725b12..b0b1aa6389 100644 --- a/state/pgstatestorage/batch.go +++ b/state/pgstatestorage/batch.go @@ -1092,3 +1092,11 @@ func (p *PostgresStorage) GetNotCheckedBatches(ctx context.Context, dbTx pgx.Tx) return batches, nil } + +// UpdateBatchTimestamp updates the timestamp of the state.batch with the given number. +func (p *PostgresStorage) UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error { + const sql = "UPDATE state.batch SET timestamp = $1 WHERE batch_num = $2" + e := p.getExecQuerier(dbTx) + _, err := e.Exec(ctx, sql, timestamp.UTC(), batchNumber) + return err +} diff --git a/synchronizer/actions/etrog/processor_l1_sequence_batches.go b/synchronizer/actions/etrog/processor_l1_sequence_batches.go index 65e713137e..2f91d8a9d9 100644 --- a/synchronizer/actions/etrog/processor_l1_sequence_batches.go +++ b/synchronizer/actions/etrog/processor_l1_sequence_batches.go @@ -32,6 +32,7 @@ type stateProcessSequenceBatches interface { AddVirtualBatch(ctx context.Context, virtualBatch *state.VirtualBatch, dbTx pgx.Tx) error AddTrustedReorg(ctx context.Context, trustedReorg *state.TrustedReorg, dbTx pgx.Tx) error GetL1InfoTreeDataFromBatchL2Data(ctx context.Context, batchL2Data []byte, dbTx pgx.Tx) (map[uint32]state.L1DataV2, common.Hash, common.Hash, error) + UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error } type syncProcessSequenceBatchesInterface interface { @@ -158,7 +159,7 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con SkipVerifyL1InfoRoot: 1, ClosingReason: state.SyncL1EventSequencedForcedBatchClosingReason, } - } else if sbatch.PolygonRollupBaseEtrogBatchData.ForcedTimestamp > 0 && sbatch.BatchNumber == 1 { + } else if sbatch.PolygonRollupBaseEtrogBatchData.ForcedTimestamp > 0 && sbatch.BatchNumber == 1 { // This is the initial batch (injected) log.Debug("Processing initial batch") batch.GlobalExitRoot = sbatch.PolygonRollupBaseEtrogBatchData.ForcedGlobalExitRoot var fBHL1 common.Hash = sbatch.PolygonRollupBaseEtrogBatchData.ForcedBlockHashL1 @@ -251,6 +252,17 @@ func (p *ProcessorL1SequenceBatchesEtrog) ProcessSequenceBatches(ctx context.Con return err } } else { + // Batch already exists + // We update the timestamp of the batch to match the timestamp + err := p.state.UpdateBatchTimestamp(ctx, batch.BatchNumber, *processCtx.Timestamp, dbTx) + if err != nil { + log.Errorf("error updating batch timestamp %s. BatchNumber: %d, BlockNumber: %d, error: %v", processCtx.Timestamp, batch.BatchNumber, blockNumber, err) + rollbackErr := dbTx.Rollback(ctx) + if rollbackErr != nil { + log.Errorf("error rolling back state because error updating batch timestamp. BatchNumber: %d, BlockNumber: %d, rollbackErr: %s, error : %v", batch.BatchNumber, blockNumber, rollbackErr.Error(), err) + return rollbackErr + } + } // 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) if err != nil { diff --git a/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go b/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go index 4d6a47e95f..3b30db1c50 100644 --- a/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go +++ b/synchronizer/actions/etrog/processor_l1_sequence_batches_test.go @@ -101,9 +101,12 @@ func TestL1SequenceBatchesTrustedBatchSequencedThatAlreadyExistsHappyPath(t *tes expectationsPreExecution(t, mocks, ctx, batch, nil) executionResponse := newProcessBatchResponseV2(batch) expectationsForExecution(t, mocks, ctx, l1Block.SequencedBatches[1][0], l1Block.ReceivedAt, executionResponse) + mocks.State.EXPECT().UpdateBatchTimestamp(ctx, batch.BatchNumber, l1Block.ReceivedAt, mocks.DbTx).Return(nil) mocks.State.EXPECT().AddAccumulatedInputHash(ctx, executionResponse.NewBatchNum, common.BytesToHash(executionResponse.NewAccInputHash), mocks.DbTx).Return(nil) expectationsAddSequencedBatch(t, mocks, ctx, executionResponse) + err := sut.Process(ctx, etherman.Order{Pos: 1}, l1Block, mocks.DbTx) + require.NoError(t, err) } @@ -117,9 +120,12 @@ func TestL1SequenceBatchesPermissionlessBatchSequencedThatAlreadyExistsHappyPath expectationsPreExecution(t, mocks, ctx, batch, nil) executionResponse := newProcessBatchResponseV2(batch) expectationsForExecution(t, mocks, ctx, l1Block.SequencedBatches[1][0], l1Block.ReceivedAt, executionResponse) + mocks.State.EXPECT().UpdateBatchTimestamp(ctx, batch.BatchNumber, l1Block.ReceivedAt, mocks.DbTx).Return(nil) mocks.State.EXPECT().AddAccumulatedInputHash(ctx, executionResponse.NewBatchNum, common.BytesToHash(executionResponse.NewAccInputHash), mocks.DbTx).Return(nil) expectationsAddSequencedBatch(t, mocks, ctx, executionResponse) + err := sut.Process(ctx, etherman.Order{Pos: 1}, l1Block, mocks.DbTx) + require.NoError(t, err) } @@ -139,6 +145,7 @@ func TestL1SequenceBatchesPermissionlessBatchSequencedThatAlreadyExistsMismatch( executionResponse := newProcessBatchResponseV2(batch) executionResponse.NewStateRoot = common.HexToHash(hashExamplesValues[2]).Bytes() expectationsForExecution(t, mocks, ctx, l1Block.SequencedBatches[1][0], l1Block.ReceivedAt, executionResponse) + mocks.State.EXPECT().UpdateBatchTimestamp(ctx, batch.BatchNumber, l1Block.ReceivedAt, mocks.DbTx).Return(nil) mocks.State.EXPECT().AddAccumulatedInputHash(ctx, executionResponse.NewBatchNum, common.BytesToHash(executionResponse.NewAccInputHash), mocks.DbTx).Return(nil) mocks.Synchronizer.EXPECT().IsTrustedSequencer().Return(false) mocks.State.EXPECT().AddTrustedReorg(ctx, mock.Anything, mocks.DbTx).Return(nil) @@ -177,6 +184,7 @@ func TestL1SequenceBatchesTrustedBatchSequencedThatAlreadyExistsMismatch(t *test executionResponse := newProcessBatchResponseV2(batch) executionResponse.NewStateRoot = common.HexToHash(hashExamplesValues[2]).Bytes() expectationsForExecution(t, mocks, ctx, l1Block.SequencedBatches[1][0], l1Block.ReceivedAt, executionResponse) + mocks.State.EXPECT().UpdateBatchTimestamp(ctx, batch.BatchNumber, l1Block.ReceivedAt, mocks.DbTx).Return(nil) mocks.State.EXPECT().AddAccumulatedInputHash(ctx, executionResponse.NewBatchNum, common.BytesToHash(executionResponse.NewAccInputHash), mocks.DbTx).Return(nil) mocks.Synchronizer.EXPECT().IsTrustedSequencer().Return(true) @@ -295,7 +303,7 @@ func newL1Block(mocks *mocksEtrogProcessorL1, batch *state.Batch, l1InfoRoot com func newComposedL1Block(mocks *mocksEtrogProcessorL1, forcedBatch *etherman.SequencedBatch, l1InfoRoot common.Hash) *etherman.Block { l1Block := etherman.Block{ BlockNumber: 123, - ReceivedAt: mocks.TimeProvider.Now(), + ReceivedAt: time.Date(2024, 1, 1, 1, 0, 0, 0, time.UTC), SequencedBatches: [][]etherman.SequencedBatch{}, } l1Block.SequencedBatches = append(l1Block.SequencedBatches, []etherman.SequencedBatch{}) diff --git a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go index 81fe9a430e..ec779c4854 100644 --- a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go +++ b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go @@ -2963,6 +2963,55 @@ func (_c *StateFullInterface_UpdateBatchL2Data_Call) RunAndReturn(run func(conte return _c } +// UpdateBatchTimestamp provides a mock function with given fields: ctx, batchNumber, timestamp, dbTx +func (_m *StateFullInterface) UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error { + ret := _m.Called(ctx, batchNumber, timestamp, dbTx) + + if len(ret) == 0 { + panic("no return value specified for UpdateBatchTimestamp") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, uint64, time.Time, pgx.Tx) error); ok { + r0 = rf(ctx, batchNumber, timestamp, dbTx) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// StateFullInterface_UpdateBatchTimestamp_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateBatchTimestamp' +type StateFullInterface_UpdateBatchTimestamp_Call struct { + *mock.Call +} + +// UpdateBatchTimestamp is a helper method to define mock.On call +// - ctx context.Context +// - batchNumber uint64 +// - timestamp time.Time +// - dbTx pgx.Tx +func (_e *StateFullInterface_Expecter) UpdateBatchTimestamp(ctx interface{}, batchNumber interface{}, timestamp interface{}, dbTx interface{}) *StateFullInterface_UpdateBatchTimestamp_Call { + return &StateFullInterface_UpdateBatchTimestamp_Call{Call: _e.mock.On("UpdateBatchTimestamp", ctx, batchNumber, timestamp, dbTx)} +} + +func (_c *StateFullInterface_UpdateBatchTimestamp_Call) Run(run func(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx)) *StateFullInterface_UpdateBatchTimestamp_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(uint64), args[2].(time.Time), args[3].(pgx.Tx)) + }) + return _c +} + +func (_c *StateFullInterface_UpdateBatchTimestamp_Call) Return(_a0 error) *StateFullInterface_UpdateBatchTimestamp_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *StateFullInterface_UpdateBatchTimestamp_Call) RunAndReturn(run func(context.Context, uint64, time.Time, pgx.Tx) error) *StateFullInterface_UpdateBatchTimestamp_Call { + _c.Call.Return(run) + return _c +} + // UpdateCheckedBlockByNumber provides a mock function with given fields: ctx, blockNumber, newCheckedStatus, dbTx func (_m *StateFullInterface) UpdateCheckedBlockByNumber(ctx context.Context, blockNumber uint64, newCheckedStatus bool, dbTx pgx.Tx) error { ret := _m.Called(ctx, blockNumber, newCheckedStatus, dbTx) diff --git a/synchronizer/common/syncinterfaces/state.go b/synchronizer/common/syncinterfaces/state.go index 0b3e248cbf..2fb9ea2f16 100644 --- a/synchronizer/common/syncinterfaces/state.go +++ b/synchronizer/common/syncinterfaces/state.go @@ -78,4 +78,5 @@ type StateFullInterface interface { GetL2BlockByNumber(ctx context.Context, blockNumber uint64, dbTx pgx.Tx) (*state.L2Block, error) GetUncheckedBlocks(ctx context.Context, fromBlockNumber uint64, toBlockNumber uint64, dbTx pgx.Tx) ([]*state.Block, error) GetPreviousBlockToBlockNumber(ctx context.Context, blockNumber uint64, dbTx pgx.Tx) (*state.Block, error) + UpdateBatchTimestamp(ctx context.Context, batchNumber uint64, timestamp time.Time, dbTx pgx.Tx) error } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index 73bebe5a12..ed063cac1e 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -368,6 +368,8 @@ func TestForcedBatchEtrog(t *testing.T) { Return(nil). Once() + m.State.EXPECT().UpdateBatchTimestamp(ctx, sequencedBatch.BatchNumber, fb[0].ForcedAt, m.DbTx).Return(nil) + m.State. On("AddAccumulatedInputHash", ctx, sequencedBatch.BatchNumber, common.Hash{}, m.DbTx). Return(nil). From de0329dbd80c781501d929b6137cc751165c5109 Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Wed, 22 May 2024 17:31:50 +0200 Subject: [PATCH 10/22] Fix use of L2coinbase configured in sequencesender.L2Coinbase parameter (#3642) * fix use l2coinbase configured in sequencesender.L2Coinbase parameter * update doc --- cmd/run.go | 2 + docs/config-file/node-config-doc.html | 2 +- docs/config-file/node-config-doc.md | 84 +++++++++++++----------- docs/config-file/node-config-schema.json | 9 +++ sequencer/batch.go | 4 +- sequencer/config.go | 4 ++ sequencer/datastreamer.go | 2 +- sequencer/finalizer.go | 6 +- sequencer/finalizer_test.go | 18 ++--- sequencer/forcedbatch.go | 4 +- sequencer/sequencer.go | 11 +--- 11 files changed, 79 insertions(+), 67 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index cbc3f835e2..6da71dbd2d 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -416,6 +416,8 @@ func runJSONRPCServer(c config.Config, etherman *etherman.Client, chainID uint64 } func createSequencer(cfg config.Config, pool *pool.Pool, st *state.State, etherman *etherman.Client, eventLog *event.EventLog) *sequencer.Sequencer { + cfg.Sequencer.L2Coinbase = cfg.SequenceSender.L2Coinbase + seq, err := sequencer.New(cfg.Sequencer, cfg.State.Batch, cfg.Pool, pool, st, etherman, eventLog) if err != nil { log.Fatal(err) diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 105b99ff15..49dd80f15a 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -38,7 +38,7 @@
"300ms"
 

Default: "5s"Type: string

StateConsistencyCheckInterval is the time the sequencer waits to check if a state inconsistency has happened


Examples:

"1m"
 
"300ms"
-

Finalizer's specific config properties
Default: "1m0s"Type: string

ForcedBatchesTimeout is the time the finalizer waits after receiving closing signal to process Forced Batches


Examples:

"1m"
+

Type: array of integer

L2Coinbase defines which address is going to receive the fees. It gets the config value from SequenceSender.L2Coinbase

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


Finalizer's specific config properties
Default: "1m0s"Type: string

ForcedBatchesTimeout is the time the finalizer waits after receiving closing signal to process Forced Batches


Examples:

"1m"
 
"300ms"
 

Default: "100ms"Type: string

NewTxsWaitInterval is the time the finalizer sleeps between each iteration, if there are no transactions to be processed


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index 440cd5dac6..c5ba48e53b 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -1928,16 +1928,17 @@ CheckLastL2BlockHashOnCloseBatch=true
 **Type:** : `object`
 **Description:** Configuration of the sequencer service
 
-| Property                                                                             | Pattern | Type    | Deprecated | Definition | Title/Description                                                                                |
-| ------------------------------------------------------------------------------------ | ------- | ------- | ---------- | ---------- | ------------------------------------------------------------------------------------------------ |
-| - [DeletePoolTxsL1BlockConfirmations](#Sequencer_DeletePoolTxsL1BlockConfirmations ) | No      | integer | No         | -          | DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool |
-| - [DeletePoolTxsCheckInterval](#Sequencer_DeletePoolTxsCheckInterval )               | No      | string  | No         | -          | Duration                                                                                         |
-| - [TxLifetimeCheckInterval](#Sequencer_TxLifetimeCheckInterval )                     | No      | string  | No         | -          | Duration                                                                                         |
-| - [TxLifetimeMax](#Sequencer_TxLifetimeMax )                                         | No      | string  | No         | -          | Duration                                                                                         |
-| - [LoadPoolTxsCheckInterval](#Sequencer_LoadPoolTxsCheckInterval )                   | No      | string  | No         | -          | Duration                                                                                         |
-| - [StateConsistencyCheckInterval](#Sequencer_StateConsistencyCheckInterval )         | No      | string  | No         | -          | Duration                                                                                         |
-| - [Finalizer](#Sequencer_Finalizer )                                                 | No      | object  | No         | -          | Finalizer's specific config properties                                                           |
-| - [StreamServer](#Sequencer_StreamServer )                                           | No      | object  | No         | -          | StreamServerCfg is the config for the stream server                                              |
+| Property                                                                             | Pattern | Type             | Deprecated | Definition | Title/Description                                                                                                      |
+| ------------------------------------------------------------------------------------ | ------- | ---------------- | ---------- | ---------- | ---------------------------------------------------------------------------------------------------------------------- |
+| - [DeletePoolTxsL1BlockConfirmations](#Sequencer_DeletePoolTxsL1BlockConfirmations ) | No      | integer          | No         | -          | DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool                       |
+| - [DeletePoolTxsCheckInterval](#Sequencer_DeletePoolTxsCheckInterval )               | No      | string           | No         | -          | Duration                                                                                                               |
+| - [TxLifetimeCheckInterval](#Sequencer_TxLifetimeCheckInterval )                     | No      | string           | No         | -          | Duration                                                                                                               |
+| - [TxLifetimeMax](#Sequencer_TxLifetimeMax )                                         | No      | string           | No         | -          | Duration                                                                                                               |
+| - [LoadPoolTxsCheckInterval](#Sequencer_LoadPoolTxsCheckInterval )                   | No      | string           | No         | -          | Duration                                                                                                               |
+| - [StateConsistencyCheckInterval](#Sequencer_StateConsistencyCheckInterval )         | No      | string           | No         | -          | Duration                                                                                                               |
+| - [L2Coinbase](#Sequencer_L2Coinbase )                                               | No      | array of integer | No         | -          | L2Coinbase defines which address is going to receive the fees. It gets the config value from SequenceSender.L2Coinbase |
+| - [Finalizer](#Sequencer_Finalizer )                                                 | No      | object           | No         | -          | Finalizer's specific config properties                                                                                 |
+| - [StreamServer](#Sequencer_StreamServer )                                           | No      | object           | No         | -          | StreamServerCfg is the config for the stream server                                                                    |
 
 ### 10.1. `Sequencer.DeletePoolTxsL1BlockConfirmations`
 
@@ -2083,7 +2084,12 @@ LoadPoolTxsCheckInterval="500ms"
 StateConsistencyCheckInterval="5s"
 ```
 
-### 10.7. `[Sequencer.Finalizer]`
+### 10.7. `Sequencer.L2Coinbase`
+
+**Type:** : `array of integer`
+**Description:** L2Coinbase defines which address is going to receive the fees. It gets the config value from SequenceSender.L2Coinbase
+
+### 10.8. `[Sequencer.Finalizer]`
 
 **Type:** : `object`
 **Description:** Finalizer's specific config properties
@@ -2106,7 +2112,7 @@ StateConsistencyCheckInterval="5s"
 | - [SequentialProcessL2Block](#Sequencer_Finalizer_SequentialProcessL2Block )                   | No      | boolean | No         | -          | SequentialProcessL2Block indicates if the processing of a L2 Block must be done in the same finalizer go func instead
in the processPendingL2Blocks go func | | - [Metrics](#Sequencer_Finalizer_Metrics ) | No | object | No | - | Metrics is the config for the sequencer metrics | -#### 10.7.1. `Sequencer.Finalizer.ForcedBatchesTimeout` +#### 10.8.1. `Sequencer.Finalizer.ForcedBatchesTimeout` **Title:** Duration @@ -2132,7 +2138,7 @@ StateConsistencyCheckInterval="5s" ForcedBatchesTimeout="1m0s" ``` -#### 10.7.2. `Sequencer.Finalizer.NewTxsWaitInterval` +#### 10.8.2. `Sequencer.Finalizer.NewTxsWaitInterval` **Title:** Duration @@ -2158,7 +2164,7 @@ ForcedBatchesTimeout="1m0s" NewTxsWaitInterval="100ms" ``` -#### 10.7.3. `Sequencer.Finalizer.ResourceExhaustedMarginPct` +#### 10.8.3. `Sequencer.Finalizer.ResourceExhaustedMarginPct` **Type:** : `integer` @@ -2172,7 +2178,7 @@ NewTxsWaitInterval="100ms" ResourceExhaustedMarginPct=10 ``` -#### 10.7.4. `Sequencer.Finalizer.ForcedBatchesL1BlockConfirmations` +#### 10.8.4. `Sequencer.Finalizer.ForcedBatchesL1BlockConfirmations` **Type:** : `integer` @@ -2186,7 +2192,7 @@ ResourceExhaustedMarginPct=10 ForcedBatchesL1BlockConfirmations=64 ``` -#### 10.7.5. `Sequencer.Finalizer.L1InfoTreeL1BlockConfirmations` +#### 10.8.5. `Sequencer.Finalizer.L1InfoTreeL1BlockConfirmations` **Type:** : `integer` @@ -2200,7 +2206,7 @@ ForcedBatchesL1BlockConfirmations=64 L1InfoTreeL1BlockConfirmations=64 ``` -#### 10.7.6. `Sequencer.Finalizer.ForcedBatchesCheckInterval` +#### 10.8.6. `Sequencer.Finalizer.ForcedBatchesCheckInterval` **Title:** Duration @@ -2226,7 +2232,7 @@ L1InfoTreeL1BlockConfirmations=64 ForcedBatchesCheckInterval="10s" ``` -#### 10.7.7. `Sequencer.Finalizer.L1InfoTreeCheckInterval` +#### 10.8.7. `Sequencer.Finalizer.L1InfoTreeCheckInterval` **Title:** Duration @@ -2252,7 +2258,7 @@ ForcedBatchesCheckInterval="10s" L1InfoTreeCheckInterval="10s" ``` -#### 10.7.8. `Sequencer.Finalizer.BatchMaxDeltaTimestamp` +#### 10.8.8. `Sequencer.Finalizer.BatchMaxDeltaTimestamp` **Title:** Duration @@ -2278,7 +2284,7 @@ L1InfoTreeCheckInterval="10s" BatchMaxDeltaTimestamp="30m0s" ``` -#### 10.7.9. `Sequencer.Finalizer.L2BlockMaxDeltaTimestamp` +#### 10.8.9. `Sequencer.Finalizer.L2BlockMaxDeltaTimestamp` **Title:** Duration @@ -2304,7 +2310,7 @@ BatchMaxDeltaTimestamp="30m0s" L2BlockMaxDeltaTimestamp="3s" ``` -#### 10.7.10. `Sequencer.Finalizer.StateRootSyncInterval` +#### 10.8.10. `Sequencer.Finalizer.StateRootSyncInterval` **Title:** Duration @@ -2331,7 +2337,7 @@ the stateroot used in the tx-by-tx execution StateRootSyncInterval="1h0m0s" ``` -#### 10.7.11. `Sequencer.Finalizer.FlushIdCheckInterval` +#### 10.8.11. `Sequencer.Finalizer.FlushIdCheckInterval` **Title:** Duration @@ -2357,7 +2363,7 @@ StateRootSyncInterval="1h0m0s" FlushIdCheckInterval="50ms" ``` -#### 10.7.12. `Sequencer.Finalizer.HaltOnBatchNumber` +#### 10.8.12. `Sequencer.Finalizer.HaltOnBatchNumber` **Type:** : `integer` @@ -2372,7 +2378,7 @@ The Sequencer will halt after it closes the batch equal to this number HaltOnBatchNumber=0 ``` -#### 10.7.13. `Sequencer.Finalizer.SequentialBatchSanityCheck` +#### 10.8.13. `Sequencer.Finalizer.SequentialBatchSanityCheck` **Type:** : `boolean` @@ -2387,7 +2393,7 @@ sequential way (instead than in parallel) SequentialBatchSanityCheck=false ``` -#### 10.7.14. `Sequencer.Finalizer.SequentialProcessL2Block` +#### 10.8.14. `Sequencer.Finalizer.SequentialProcessL2Block` **Type:** : `boolean` @@ -2402,7 +2408,7 @@ in the processPendingL2Blocks go func SequentialProcessL2Block=false ``` -#### 10.7.15. `[Sequencer.Finalizer.Metrics]` +#### 10.8.15. `[Sequencer.Finalizer.Metrics]` **Type:** : `object` **Description:** Metrics is the config for the sequencer metrics @@ -2412,7 +2418,7 @@ SequentialProcessL2Block=false | - [Interval](#Sequencer_Finalizer_Metrics_Interval ) | No | string | No | - | Duration | | - [EnableLog](#Sequencer_Finalizer_Metrics_EnableLog ) | No | boolean | No | - | EnableLog is a flag to enable/disable metrics logs | -##### 10.7.15.1. `Sequencer.Finalizer.Metrics.Interval` +##### 10.8.15.1. `Sequencer.Finalizer.Metrics.Interval` **Title:** Duration @@ -2438,7 +2444,7 @@ SequentialProcessL2Block=false Interval="1h0m0s" ``` -##### 10.7.15.2. `Sequencer.Finalizer.Metrics.EnableLog` +##### 10.8.15.2. `Sequencer.Finalizer.Metrics.EnableLog` **Type:** : `boolean` @@ -2452,7 +2458,7 @@ Interval="1h0m0s" EnableLog=true ``` -### 10.8. `[Sequencer.StreamServer]` +### 10.9. `[Sequencer.StreamServer]` **Type:** : `object` **Description:** StreamServerCfg is the config for the stream server @@ -2467,7 +2473,7 @@ EnableLog=true | - [Log](#Sequencer_StreamServer_Log ) | No | object | No | - | Log is the log configuration | | - [UpgradeEtrogBatchNumber](#Sequencer_StreamServer_UpgradeEtrogBatchNumber ) | No | integer | No | - | UpgradeEtrogBatchNumber is the batch number of the upgrade etrog | -#### 10.8.1. `Sequencer.StreamServer.Port` +#### 10.9.1. `Sequencer.StreamServer.Port` **Type:** : `integer` @@ -2481,7 +2487,7 @@ EnableLog=true Port=0 ``` -#### 10.8.2. `Sequencer.StreamServer.Filename` +#### 10.9.2. `Sequencer.StreamServer.Filename` **Type:** : `string` @@ -2495,7 +2501,7 @@ Port=0 Filename="" ``` -#### 10.8.3. `Sequencer.StreamServer.Version` +#### 10.9.3. `Sequencer.StreamServer.Version` **Type:** : `integer` @@ -2509,7 +2515,7 @@ Filename="" Version=0 ``` -#### 10.8.4. `Sequencer.StreamServer.ChainID` +#### 10.9.4. `Sequencer.StreamServer.ChainID` **Type:** : `integer` @@ -2523,7 +2529,7 @@ Version=0 ChainID=0 ``` -#### 10.8.5. `Sequencer.StreamServer.Enabled` +#### 10.9.5. `Sequencer.StreamServer.Enabled` **Type:** : `boolean` @@ -2537,7 +2543,7 @@ ChainID=0 Enabled=false ``` -#### 10.8.6. `[Sequencer.StreamServer.Log]` +#### 10.9.6. `[Sequencer.StreamServer.Log]` **Type:** : `object` **Description:** Log is the log configuration @@ -2548,7 +2554,7 @@ Enabled=false | - [Level](#Sequencer_StreamServer_Log_Level ) | No | enum (of string) | No | - | - | | - [Outputs](#Sequencer_StreamServer_Log_Outputs ) | No | array of string | No | - | - | -##### 10.8.6.1. `Sequencer.StreamServer.Log.Environment` +##### 10.9.6.1. `Sequencer.StreamServer.Log.Environment` **Type:** : `enum (of string)` @@ -2564,7 +2570,7 @@ Must be one of: * "production" * "development" -##### 10.8.6.2. `Sequencer.StreamServer.Log.Level` +##### 10.9.6.2. `Sequencer.StreamServer.Log.Level` **Type:** : `enum (of string)` @@ -2585,11 +2591,11 @@ Must be one of: * "panic" * "fatal" -##### 10.8.6.3. `Sequencer.StreamServer.Log.Outputs` +##### 10.9.6.3. `Sequencer.StreamServer.Log.Outputs` **Type:** : `array of string` -#### 10.8.7. `Sequencer.StreamServer.UpgradeEtrogBatchNumber` +#### 10.9.7. `Sequencer.StreamServer.UpgradeEtrogBatchNumber` **Type:** : `integer` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index bf7314dc1f..4597acaff4 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -780,6 +780,15 @@ "300ms" ] }, + "L2Coinbase": { + "items": { + "type": "integer" + }, + "type": "array", + "maxItems": 20, + "minItems": 20, + "description": "L2Coinbase defines which address is going to receive the fees. It gets the config value from SequenceSender.L2Coinbase" + }, "Finalizer": { "properties": { "ForcedBatchesTimeout": { diff --git a/sequencer/batch.go b/sequencer/batch.go index c6e2700fa8..a5ada2b61a 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -306,7 +306,7 @@ func (f *finalizer) openNewWIPBatch(batchNumber uint64, stateRoot common.Hash) * return &Batch{ batchNumber: batchNumber, - coinbase: f.sequencerAddress, + coinbase: f.l2Coinbase, initialStateRoot: stateRoot, imStateRoot: stateRoot, finalStateRoot: stateRoot, @@ -323,7 +323,7 @@ func (f *finalizer) insertSIPBatch(ctx context.Context, batchNumber uint64, stat // open next batch newStateBatch := state.Batch{ BatchNumber: batchNumber, - Coinbase: f.sequencerAddress, + Coinbase: f.l2Coinbase, Timestamp: now(), StateRoot: stateRoot, GlobalExitRoot: state.ZeroHash, diff --git a/sequencer/config.go b/sequencer/config.go index c0bfb8968d..03aeeb740b 100644 --- a/sequencer/config.go +++ b/sequencer/config.go @@ -3,6 +3,7 @@ package sequencer import ( "github.com/0xPolygonHermez/zkevm-data-streamer/log" "github.com/0xPolygonHermez/zkevm-node/config/types" + "github.com/ethereum/go-ethereum/common" ) // Config represents the configuration of a sequencer @@ -25,6 +26,9 @@ type Config struct { // StateConsistencyCheckInterval is the time the sequencer waits to check if a state inconsistency has happened StateConsistencyCheckInterval types.Duration `mapstructure:"StateConsistencyCheckInterval"` + // L2Coinbase defines which address is going to receive the fees. It gets the config value from SequenceSender.L2Coinbase + L2Coinbase common.Address `mapstructure:"L2Coinbase"` + // Finalizer's specific config properties Finalizer FinalizerCfg `mapstructure:"Finalizer"` diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 4c08d6b3e4..50321c6c55 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -20,7 +20,7 @@ func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.Proce L1InfoTreeIndex: l1InfoTreeIndex, L1BlockHash: blockResponse.BlockHashL1, GlobalExitRoot: blockResponse.GlobalExitRoot, - Coinbase: f.sequencerAddress, + Coinbase: f.l2Coinbase, ForkID: forkID, BlockHash: blockResponse.BlockHash, StateRoot: blockResponse.BlockHash, //From etrog, the blockhash is the block root diff --git a/sequencer/finalizer.go b/sequencer/finalizer.go index a7e884ac07..9da6d691db 100644 --- a/sequencer/finalizer.go +++ b/sequencer/finalizer.go @@ -35,7 +35,7 @@ var ( type finalizer struct { cfg FinalizerCfg isSynced func(ctx context.Context) bool - sequencerAddress common.Address + l2Coinbase common.Address workerIntf workerInterface poolIntf txPool stateIntf stateInterface @@ -95,7 +95,7 @@ func newFinalizer( poolIntf txPool, stateIntf stateInterface, etherman ethermanInterface, - sequencerAddr common.Address, + l2Coinbase common.Address, isSynced func(ctx context.Context) bool, batchConstraints state.BatchConstraintsCfg, eventLog *event.EventLog, @@ -106,7 +106,7 @@ func newFinalizer( f := finalizer{ cfg: cfg, isSynced: isSynced, - sequencerAddress: sequencerAddr, + l2Coinbase: l2Coinbase, workerIntf: workerIntf, poolIntf: poolIntf, stateIntf: stateIntf, diff --git a/sequencer/finalizer_test.go b/sequencer/finalizer_test.go index 84a2d6e8cf..35a03ef4ea 100644 --- a/sequencer/finalizer_test.go +++ b/sequencer/finalizer_test.go @@ -79,11 +79,11 @@ var ( } // chainID = new(big.Int).SetInt64(400) // pvtKey = "0x28b2b0318721be8c8339199172cd7cc8f5e273800a35616ec893083a4b32c02e" - nonce1 = uint64(1) - nonce2 = uint64(2) - seqAddr = common.Address{} - oldHash = common.HexToHash("0x01") - newHash = common.HexToHash("0x02") + nonce1 = uint64(1) + nonce2 = uint64(2) + l2Coinbase = common.Address{} + oldHash = common.HexToHash("0x01") + newHash = common.HexToHash("0x02") // newHash2 = common.HexToHash("0x03") // stateRootHashes = []common.Hash{oldHash, newHash, newHash2} // txHash = common.HexToHash("0xf9e4fe4bd2256f782c66cffd76acdb455a76111842bb7e999af2f1b7f4d8d092") @@ -117,7 +117,7 @@ func TestNewFinalizer(t *testing.T) { poolMock.On("GetLastSentFlushID", context.Background()).Return(uint64(0), nil) // arrange and act - f = newFinalizer(cfg, poolCfg, workerMock, poolMock, stateMock, ethermanMock, seqAddr, isSynced, bc, eventLog, nil, newTimeoutCond(&sync.Mutex{}), nil) + f = newFinalizer(cfg, poolCfg, workerMock, poolMock, stateMock, ethermanMock, l2Coinbase, isSynced, bc, eventLog, nil, newTimeoutCond(&sync.Mutex{}), nil) // assert assert.NotNil(t, f) @@ -125,7 +125,7 @@ func TestNewFinalizer(t *testing.T) { assert.Equal(t, f.workerIntf, workerMock) assert.Equal(t, poolMock, poolMock) assert.Equal(t, f.stateIntf, stateMock) - assert.Equal(t, f.sequencerAddress, seqAddr) + assert.Equal(t, f.l2Coinbase, l2Coinbase) assert.Equal(t, f.batchConstraints, bc) } @@ -2197,7 +2197,7 @@ func setupFinalizer(withWipBatch bool) *finalizer { } wipBatch = &Batch{ batchNumber: 1, - coinbase: seqAddr, + coinbase: l2Coinbase, initialStateRoot: oldHash, imStateRoot: newHash, timestamp: now(), @@ -2213,7 +2213,7 @@ func setupFinalizer(withWipBatch bool) *finalizer { return &finalizer{ cfg: cfg, isSynced: isSynced, - sequencerAddress: seqAddr, + l2Coinbase: l2Coinbase, workerIntf: workerMock, poolIntf: poolMock, stateIntf: stateMock, diff --git a/sequencer/forcedbatch.go b/sequencer/forcedbatch.go index 21a0c92504..abe15acc77 100644 --- a/sequencer/forcedbatch.go +++ b/sequencer/forcedbatch.go @@ -85,7 +85,7 @@ func (f *finalizer) processForcedBatch(ctx context.Context, forcedBatch state.Fo // Open new batch on state for the forced batch processingCtx := state.ProcessingContext{ BatchNumber: newBatchNumber, - Coinbase: f.sequencerAddress, + Coinbase: f.l2Coinbase, Timestamp: time.Now(), GlobalExitRoot: forcedBatch.GlobalExitRoot, ForcedBatchNum: &forcedBatch.ForcedBatchNumber, @@ -101,7 +101,7 @@ func (f *finalizer) processForcedBatch(ctx context.Context, forcedBatch state.Fo ForcedBlockHashL1: fbL1Block.ParentHash, OldStateRoot: stateRoot, Transactions: forcedBatch.RawTxsData, - Coinbase: f.sequencerAddress, + Coinbase: f.l2Coinbase, TimestampLimit_V2: uint64(forcedBatch.ForcedAt.Unix()), ForkID: f.stateIntf.GetForkIDByBatchNumber(lastBatchNumber), SkipVerifyL1InfoRoot_V2: true, diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index a86d955d79..72a79cabcb 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -12,7 +12,6 @@ import ( "github.com/0xPolygonHermez/zkevm-node/pool" "github.com/0xPolygonHermez/zkevm-node/state" "github.com/0xPolygonHermez/zkevm-node/state/datastream" - "github.com/ethereum/go-ethereum/common" "google.golang.org/protobuf/proto" ) @@ -38,18 +37,11 @@ type Sequencer struct { streamServer *datastreamer.StreamServer dataToStream chan interface{} - address common.Address - numberOfStateInconsistencies uint64 } // New init sequencer func New(cfg Config, batchCfg state.BatchConfig, poolCfg pool.Config, txPool txPool, stateIntf stateInterface, etherman ethermanInterface, eventLog *event.EventLog) (*Sequencer, error) { - addr, err := etherman.TrustedSequencer() - if err != nil { - return nil, fmt.Errorf("failed to get trusted sequencer address, error: %v", err) - } - sequencer := &Sequencer{ cfg: cfg, batchCfg: batchCfg, @@ -57,7 +49,6 @@ func New(cfg Config, batchCfg state.BatchConfig, poolCfg pool.Config, txPool txP pool: txPool, stateIntf: stateIntf, etherman: etherman, - address: addr, eventLog: eventLog, } @@ -101,7 +92,7 @@ func (s *Sequencer) Start(ctx context.Context) { s.workerReadyTxsCond = newTimeoutCond(&sync.Mutex{}) s.worker = NewWorker(s.stateIntf, s.batchCfg.Constraints, s.workerReadyTxsCond) - s.finalizer = newFinalizer(s.cfg.Finalizer, s.poolCfg, s.worker, s.pool, s.stateIntf, s.etherman, s.address, s.isSynced, s.batchCfg.Constraints, s.eventLog, s.streamServer, s.workerReadyTxsCond, s.dataToStream) + s.finalizer = newFinalizer(s.cfg.Finalizer, s.poolCfg, s.worker, s.pool, s.stateIntf, s.etherman, s.cfg.L2Coinbase, s.isSynced, s.batchCfg.Constraints, s.eventLog, s.streamServer, s.workerReadyTxsCond, s.dataToStream) go s.finalizer.Start(ctx) go s.loadFromPool(ctx) From 0ab583a2a0aeac088b476b02b563e4a321136e49 Mon Sep 17 00:00:00 2001 From: Joan Esteban <129153821+joanestebanr@users.noreply.github.com> Date: Thu, 23 May 2024 11:41:04 +0200 Subject: [PATCH 11/22] Feature/3640 synchronizer choose to sync from l2 (#3641) * #3640. New Param Synchronizer.L2Synchronization.Enable to choose if sync from L2 --- config/config_test.go | 4 ++++ config/default.go | 1 + docs/config-file/node-config-doc.html | 2 +- docs/config-file/node-config-doc.md | 21 ++++++++++++++++++--- docs/config-file/node-config-schema.json | 5 +++++ synchronizer/l2_sync/config.go | 2 ++ synchronizer/synchronizer.go | 9 ++++++--- synchronizer/synchronizer_test.go | 22 ++++++++++++++++++++++ 8 files changed, 59 insertions(+), 7 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 0fd606a038..f73e34e584 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -64,6 +64,10 @@ func Test_Defaults(t *testing.T) { path: "Synchronizer.L2Synchronization.CheckLastL2BlockHashOnCloseBatch", expectedValue: true, }, + { + path: "Synchronizer.L2Synchronization.Enable", + expectedValue: true, + }, { path: "Sequencer.DeletePoolTxsL1BlockConfirmations", diff --git a/config/default.go b/config/default.go index 5834e90793..c50416b4a8 100644 --- a/config/default.go +++ b/config/default.go @@ -129,6 +129,7 @@ L1SyncCheckL2BlockNumberhModulus = 600 AceptableInacctivityTime = "5s" ApplyAfterNumRollupReceived = 10 [Synchronizer.L2Synchronization] + Enable = true AcceptEmptyClosedBatches = false ReprocessFullBatchOnClose = false CheckLastL2BlockHashOnCloseBatch = true diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 49dd80f15a..36f8c134f5 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -28,7 +28,7 @@
"300ms"
 

Default: "5s"Type: string

RollupInfoRetriesSpacing is the minimum time between retries to request rollup info (it will sleep for fulfill this time) to avoid spamming L1


Examples:

"1m"
 
"300ms"
-

Default: falseType: boolean

FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized


L2Synchronization Configuration for L2 synchronization
Default: falseType: boolean

AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.


Default: falseType: boolean

ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again


Default: trueType: boolean

CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash


Configuration of the sequencer service
Default: 100Type: integer

DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool


Default: "12h0m0s"Type: string

DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting


Examples:

"1m"
+

Default: falseType: boolean

FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized


L2Synchronization Configuration for L2 synchronization
Default: trueType: boolean

Enable if is true then the L2 sync process is permitted (only for permissionless)


Default: falseType: boolean

AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.


Default: falseType: boolean

ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again


Default: trueType: boolean

CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash


Configuration of the sequencer service
Default: 100Type: integer

DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool


Default: "12h0m0s"Type: string

DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting


Examples:

"1m"
 
"300ms"
 

Default: "10m0s"Type: string

TxLifetimeCheckInterval is the time the sequencer waits to check txs lifetime


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index c5ba48e53b..774c4870aa 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -1876,11 +1876,26 @@ FallbackToSequentialModeOnSynchronized=false
 
 | Property                                                                                                | Pattern | Type    | Deprecated | Definition | Title/Description                                                                                                                                                   |
 | ------------------------------------------------------------------------------------------------------- | ------- | ------- | ---------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
+| - [Enable](#Synchronizer_L2Synchronization_Enable )                                                     | No      | boolean | No         | -          | Enable if is true then the L2 sync process is permitted (only for permissionless)                                                                                   |
 | - [AcceptEmptyClosedBatches](#Synchronizer_L2Synchronization_AcceptEmptyClosedBatches )                 | No      | boolean | No         | -          | 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](#Synchronizer_L2Synchronization_ReprocessFullBatchOnClose ) | No | boolean | No | - | ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again | | - [CheckLastL2BlockHashOnCloseBatch](#Synchronizer_L2Synchronization_CheckLastL2BlockHashOnCloseBatch ) | No | boolean | No | - | CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash | -#### 9.10.1. `Synchronizer.L2Synchronization.AcceptEmptyClosedBatches` +#### 9.10.1. `Synchronizer.L2Synchronization.Enable` + +**Type:** : `boolean` + +**Default:** `true` + +**Description:** Enable if is true then the L2 sync process is permitted (only for permissionless) + +**Example setting the default value** (true): +``` +[Synchronizer.L2Synchronization] +Enable=true +``` + +#### 9.10.2. `Synchronizer.L2Synchronization.AcceptEmptyClosedBatches` **Type:** : `boolean` @@ -1895,7 +1910,7 @@ if true, the synchronizer will accept empty batches and process them. AcceptEmptyClosedBatches=false ``` -#### 9.10.2. `Synchronizer.L2Synchronization.ReprocessFullBatchOnClose` +#### 9.10.3. `Synchronizer.L2Synchronization.ReprocessFullBatchOnClose` **Type:** : `boolean` @@ -1909,7 +1924,7 @@ AcceptEmptyClosedBatches=false ReprocessFullBatchOnClose=false ``` -#### 9.10.3. `Synchronizer.L2Synchronization.CheckLastL2BlockHashOnCloseBatch` +#### 9.10.4. `Synchronizer.L2Synchronization.CheckLastL2BlockHashOnCloseBatch` **Type:** : `boolean` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index 4597acaff4..d6b0594885 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -698,6 +698,11 @@ }, "L2Synchronization": { "properties": { + "Enable": { + "type": "boolean", + "description": "Enable if is true then the L2 sync process is permitted (only for permissionless)", + "default": true + }, "AcceptEmptyClosedBatches": { "type": "boolean", "description": "AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.\nif true, the synchronizer will accept empty batches and process them.", diff --git a/synchronizer/l2_sync/config.go b/synchronizer/l2_sync/config.go index 7781c7bd6c..8d5219bea9 100644 --- a/synchronizer/l2_sync/config.go +++ b/synchronizer/l2_sync/config.go @@ -2,6 +2,8 @@ package l2_sync // Config configuration of L2 sync process type Config struct { + // Enable if is true then the L2 sync process is permitted (only for permissionless) + Enable bool `mapstructure:"Enable"` // AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches. // if true, the synchronizer will accept empty batches and process them. AcceptEmptyClosedBatches bool `mapstructure:"AcceptEmptyClosedBatches"` diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index ef9bbf08a3..a9d2d3c8a0 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -151,7 +151,7 @@ func NewSynchronizer( time.Second) } - if !isTrustedSequencer { + if !isTrustedSequencer && cfg.L2Synchronization.Enable { log.Info("Permissionless: creating and Initializing L2 synchronization components") L1SyncChecker := l2_sync_etrog.NewCheckSyncStatusToProcessBatch(res.zkEVMClient, res.state) sync := &res @@ -170,7 +170,10 @@ func NewSynchronizer( uint64(state.FORKID_ELDERBERRY): syncTrustedStateEtrog, uint64(state.FORKID_9): syncTrustedStateEtrog, }, res.state) + } else { + log.Info("L2 synchronization disabled or running in trusted sequencer mode") } + var l1checkerL2Blocks *actions.CheckL2BlockHash if cfg.L1SyncCheckL2BlockHash { if !isTrustedSequencer { @@ -413,7 +416,7 @@ func (s *ClientSynchronizer) Sync() error { // latestSequencedBatchNumber -> last batch on SMC if latestSyncedBatch >= latestSequencedBatchNumber { startTrusted := time.Now() - if s.syncTrustedStateExecutor != nil && !s.isTrustedSequencer { + if s.syncTrustedStateExecutor != nil { log.Info("Syncing trusted state (permissionless)") //Sync Trusted State log.Debug("Doing reorg check before L2 sync") @@ -771,7 +774,7 @@ func (s *ClientSynchronizer) ProcessBlockRange(blocks []etherman.Block, order ma } func (s *ClientSynchronizer) syncTrustedState(latestSyncedBatch uint64) error { - if s.syncTrustedStateExecutor == nil || s.isTrustedSequencer { + if s.syncTrustedStateExecutor == nil { return nil } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ed063cac1e..172868275b 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -16,6 +16,7 @@ import ( "github.com/0xPolygonHermez/zkevm-node/state/runtime/executor" "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" syncMocks "github.com/0xPolygonHermez/zkevm-node/synchronizer/mocks" "github.com/ethereum/go-ethereum/common" ethTypes "github.com/ethereum/go-ethereum/core/types" @@ -132,6 +133,9 @@ func TestForcedBatchEtrog(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -677,6 +681,9 @@ func setupGenericTest(t *testing.T) (*state.Genesis, *Config, *mocks) { RollupInfoRetriesSpacing: cfgTypes.Duration{Duration: 1 * time.Second}, FallbackToSequentialModeOnSynchronized: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -933,6 +940,9 @@ func TestReorg(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -1253,6 +1263,9 @@ func TestLatestSyncedBlockEmpty(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -1467,6 +1480,9 @@ func TestRegularReorg(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -1749,6 +1765,9 @@ func TestLatestSyncedBlockEmptyWithExtraReorg(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ @@ -2025,6 +2044,9 @@ func TestCallFromEmptyBlockAndReorg(t *testing.T) { L1BlockCheck: L1BlockCheckConfig{ Enable: false, }, + L2Synchronization: l2_sync.Config{ + Enable: true, + }, } m := mocks{ From 6023dca6cfc8dadf7bb5b691336553327bc16cd4 Mon Sep 17 00:00:00 2001 From: Joan Esteban <129153821+joanestebanr@users.noreply.github.com> Date: Mon, 27 May 2024 13:34:15 +0200 Subject: [PATCH 12/22] change synchronization config params 'Enable' to 'Enabled' (#3656) * change 'Enable' to 'Enabled' on sync config and changed comments --- config/config_test.go | 10 ++++++++- config/default.go | 6 +++--- docs/config-file/node-config-doc.html | 4 ++-- docs/config-file/node-config-doc.md | 24 +++++++++++----------- docs/config-file/node-config-schema.json | 12 +++++------ synchronizer/config.go | 10 ++++----- synchronizer/l2_sync/config.go | 4 ++-- synchronizer/synchronizer.go | 6 +++--- synchronizer/synchronizer_test.go | 26 ++++++++++++------------ 9 files changed, 55 insertions(+), 47 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index f73e34e584..0d6c93ea33 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -65,7 +65,15 @@ func Test_Defaults(t *testing.T) { expectedValue: true, }, { - path: "Synchronizer.L2Synchronization.Enable", + path: "Synchronizer.L1BlockCheck.Enabled", + expectedValue: true, + }, + { + path: "Synchronizer.L1BlockCheck.PreCheckEnabled", + expectedValue: true, + }, + { + path: "Synchronizer.L2Synchronization.Enabled", expectedValue: true, }, diff --git a/config/default.go b/config/default.go index c50416b4a8..970ba020ba 100644 --- a/config/default.go +++ b/config/default.go @@ -108,11 +108,11 @@ L1SynchronizationMode = "sequential" L1SyncCheckL2BlockHash = true L1SyncCheckL2BlockNumberhModulus = 600 [Synchronizer.L1BlockCheck] - Enable = true + Enabled = true L1SafeBlockPoint = "finalized" L1SafeBlockOffset = 0 ForceCheckBeforeStart = true - PreCheckEnable = true + PreCheckEnabled = true L1PreSafeBlockPoint = "safe" L1PreSafeBlockOffset = 0 [Synchronizer.L1ParallelSynchronization] @@ -129,7 +129,7 @@ L1SyncCheckL2BlockNumberhModulus = 600 AceptableInacctivityTime = "5s" ApplyAfterNumRollupReceived = 10 [Synchronizer.L2Synchronization] - Enable = true + Enabled = true AcceptEmptyClosedBatches = false ReprocessFullBatchOnClose = false CheckLastL2BlockHashOnCloseBatch = true diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 36f8c134f5..35d4551dce 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -16,7 +16,7 @@
"300ms"
 

Default: 500Type: number

MaxRequestsPerIPAndSecond defines how much requests a single IP can
send within a single second


Default: ""Type: string

SequencerNodeURI is used allow Non-Sequencer nodes
to relay transactions to the Sequencer node


Default: 0Type: integer

MaxCumulativeGasUsed is the max gas allowed per batch


WebSockets configuration
Default: trueType: boolean

Enabled defines if the WebSocket requests are enabled or disabled


Default: "0.0.0.0"Type: string

Host defines the network adapter that will be used to serve the WS requests


Default: 8546Type: integer

Port defines the port to serve the endpoints via WS


Default: 104857600Type: integer

ReadLimit defines the maximum size of a message read from the client (in bytes)


Default: trueType: boolean

EnableL2SuggestedGasPricePolling enables polling of the L2 gas price to block tx in the RPC with lower gas price.


Default: falseType: boolean

BatchRequestsEnabled defines if the Batch requests are enabled or disabled


Default: 20Type: integer

BatchRequestsLimit defines the limit of requests that can be incorporated into each batch request


Type: array of integer

L2Coinbase defines which address is going to receive the fees

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


Default: 10000Type: integer

MaxLogsCount is a configuration to set the max number of logs that can be returned
in a single call to the state, if zero it means no limit


Default: 10000Type: integer

MaxLogsBlockRange is a configuration to set the max range for block number when querying TXs
logs in a single call to the state, if zero it means no limit


Default: 60000Type: integer

MaxNativeBlockHashBlockRange is a configuration to set the max range for block number when querying
native block hashes in a single call to the state, if zero it means no limit


Default: trueType: boolean

EnableHttpLog allows the user to enable or disable the logs related to the HTTP
requests to be captured by the server.


ZKCountersLimits defines the ZK Counter limits
Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Default: 0Type: integer

Configuration of service `Syncrhonizer`. For this service is also really important the value of `IsTrustedSequencer` because depending of this values is going to ask to a trusted node for trusted transactions or not
Default: "1s"Type: string

SyncInterval is the delay interval between reading new rollup information


Examples:

"1m"
 
"300ms"
-

Default: 100Type: integer

SyncChunkSize is the number of blocks to sync on each chunk


Default: ""Type: string

TrustedSequencerURL is the rpc url to connect and sync the trusted state


Default: "safe"Type: string

SyncBlockProtection specify the state to sync (lastest, finalized or safe)


Default: trueType: boolean

L1SyncCheckL2BlockHash if is true when a batch is closed is force to check L2Block hash against trustedNode (only apply for permissionless)


Default: 600Type: integer

L1SyncCheckL2BlockNumberhModulus is the modulus used to choose the l2block to check
a modules 5, for instance, means check all l2block multiples of 5 (10,15,20,...)


Default: trueType: boolean

Enable if is true then the check l1 Block Hash is active


Default: "finalized"Type: enum (of string)

L1SafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest

Must be one of:

  • "finalized"
  • "safe"
  • "latest"

Default: 0Type: integer

L1SafeBlockOffset is the offset to add to L1SafeBlockPoint as a safe point
it can be positive or negative
Example: L1SafeBlockPoint= finalized, L1SafeBlockOffset= -10, then the safe block ten blocks before the finalized block


Default: trueType: boolean

ForceCheckBeforeStart if is true then the first time the system is started it will force to check all pending blocks


Default: trueType: boolean

PreCheckEnable if is true then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock


Default: "safe"Type: enum (of string)

L1PreSafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest

Must be one of:

  • "finalized"
  • "safe"
  • "latest"

Default: 0Type: integer

L1PreSafeBlockOffset is the offset to add to L1PreSafeBlockPoint as a safe point
it can be positive or negative
Example: L1PreSafeBlockPoint= finalized, L1PreSafeBlockOffset= -10, then the safe block ten blocks before the finalized block


Default: "sequential"Type: enum (of string)

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

Must be one of:

  • "sequential"
  • "parallel"

L1ParallelSynchronization Configuration for parallel mode (if L1SynchronizationMode equal to 'parallel')
Default: 10Type: integer

MaxClients Number of clients used to synchronize with L1


Default: 25Type: integer

MaxPendingNoProcessedBlocks Size of the buffer used to store rollup information from L1, must be >= to NumberOfEthereumClientsToSync
sugested twice of NumberOfParallelOfEthereumClients


Default: "5s"Type: string

RequestLastBlockPeriod is the time to wait to request the
last block to L1 to known if we need to retrieve more data.
This value only apply when the system is synchronized


Examples:

"1m"
+

Default: 100Type: integer

SyncChunkSize is the number of blocks to sync on each chunk


Default: ""Type: string

TrustedSequencerURL is the rpc url to connect and sync the trusted state


Default: "safe"Type: string

SyncBlockProtection specify the state to sync (lastest, finalized or safe)


Default: trueType: boolean

L1SyncCheckL2BlockHash if is true when a batch is closed is force to check L2Block hash against trustedNode (only apply for permissionless)


Default: 600Type: integer

L1SyncCheckL2BlockNumberhModulus is the modulus used to choose the l2block to check
a modules 5, for instance, means check all l2block multiples of 5 (10,15,20,...)


Default: trueType: boolean

If enabled then the check l1 Block Hash is active


Default: "finalized"Type: enum (of string)

L1SafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest

Must be one of:

  • "finalized"
  • "safe"
  • "latest"

Default: 0Type: integer

L1SafeBlockOffset is the offset to add to L1SafeBlockPoint as a safe point
it can be positive or negative
Example: L1SafeBlockPoint= finalized, L1SafeBlockOffset= -10, then the safe block ten blocks before the finalized block


Default: trueType: boolean

ForceCheckBeforeStart if is true then the first time the system is started it will force to check all pending blocks


Default: trueType: boolean

If enabled then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock


Default: "safe"Type: enum (of string)

L1PreSafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest

Must be one of:

  • "finalized"
  • "safe"
  • "latest"

Default: 0Type: integer

L1PreSafeBlockOffset is the offset to add to L1PreSafeBlockPoint as a safe point
it can be positive or negative
Example: L1PreSafeBlockPoint= finalized, L1PreSafeBlockOffset= -10, then the safe block ten blocks before the finalized block


Default: "sequential"Type: enum (of string)

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

Must be one of:

  • "sequential"
  • "parallel"

L1ParallelSynchronization Configuration for parallel mode (if L1SynchronizationMode equal to 'parallel')
Default: 10Type: integer

MaxClients Number of clients used to synchronize with L1


Default: 25Type: integer

MaxPendingNoProcessedBlocks Size of the buffer used to store rollup information from L1, must be >= to NumberOfEthereumClientsToSync
sugested twice of NumberOfParallelOfEthereumClients


Default: "5s"Type: string

RequestLastBlockPeriod is the time to wait to request the
last block to L1 to known if we need to retrieve more data.
This value only apply when the system is synchronized


Examples:

"1m"
 
"300ms"
 

Consumer Configuration for the consumer of rollup information from L1
Default: "5s"Type: string

AceptableInacctivityTime is the expected maximum time that the consumer
could wait until new data is produced. If the time is greater it emmit a log to warn about
that. The idea is keep working the consumer as much as possible, so if the producer is not
fast enought then you could increse the number of parallel clients to sync with L1


Examples:

"1m"
 
"300ms"
@@ -28,7 +28,7 @@
 
"300ms"
 

Default: "5s"Type: string

RollupInfoRetriesSpacing is the minimum time between retries to request rollup info (it will sleep for fulfill this time) to avoid spamming L1


Examples:

"1m"
 
"300ms"
-

Default: falseType: boolean

FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized


L2Synchronization Configuration for L2 synchronization
Default: trueType: boolean

Enable if is true then the L2 sync process is permitted (only for permissionless)


Default: falseType: boolean

AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.


Default: falseType: boolean

ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again


Default: trueType: boolean

CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash


Configuration of the sequencer service
Default: 100Type: integer

DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool


Default: "12h0m0s"Type: string

DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting


Examples:

"1m"
+

Default: falseType: boolean

FallbackToSequentialModeOnSynchronized if true switch to sequential mode if the system is synchronized


L2Synchronization Configuration for L2 synchronization
Default: trueType: boolean

If enabled then the L2 sync process is permitted (only for permissionless)


Default: falseType: boolean

AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches.
if true, the synchronizer will accept empty batches and process them.


Default: falseType: boolean

ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again


Default: trueType: boolean

CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash


Configuration of the sequencer service
Default: 100Type: integer

DeletePoolTxsL1BlockConfirmations is blocks amount after which txs will be deleted from the pool


Default: "12h0m0s"Type: string

DeletePoolTxsCheckInterval is frequency with which txs will be checked for deleting


Examples:

"1m"
 
"300ms"
 

Default: "10m0s"Type: string

TxLifetimeCheckInterval is the time the sequencer waits to check txs lifetime


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index 774c4870aa..cff56349af 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -1466,26 +1466,26 @@ L1SyncCheckL2BlockNumberhModulus=600
 
 | Property                                                                     | Pattern | Type             | Deprecated | Definition | Title/Description                                                                                                                                                                                                                                       |
 | ---------------------------------------------------------------------------- | ------- | ---------------- | ---------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
-| - [Enable](#Synchronizer_L1BlockCheck_Enable )                               | No      | boolean          | No         | -          | Enable if is true then the check l1 Block Hash is active                                                                                                                                                                                                |
+| - [Enabled](#Synchronizer_L1BlockCheck_Enabled )                             | No      | boolean          | No         | -          | If enabled then the check l1 Block Hash is active                                                                                                                                                                                                       |
 | - [L1SafeBlockPoint](#Synchronizer_L1BlockCheck_L1SafeBlockPoint )           | No      | enum (of string) | No         | -          | L1SafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest | | - [L1SafeBlockOffset](#Synchronizer_L1BlockCheck_L1SafeBlockOffset ) | No | integer | No | - | L1SafeBlockOffset is the offset to add to L1SafeBlockPoint as a safe point
it can be positive or negative
Example: L1SafeBlockPoint= finalized, L1SafeBlockOffset= -10, then the safe block ten blocks before the finalized block | | - [ForceCheckBeforeStart](#Synchronizer_L1BlockCheck_ForceCheckBeforeStart ) | No | boolean | No | - | ForceCheckBeforeStart if is true then the first time the system is started it will force to check all pending blocks | -| - [PreCheckEnable](#Synchronizer_L1BlockCheck_PreCheckEnable ) | No | boolean | No | - | PreCheckEnable if is true then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock | +| - [PreCheckEnabled](#Synchronizer_L1BlockCheck_PreCheckEnabled ) | No | boolean | No | - | If enabled then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock | | - [L1PreSafeBlockPoint](#Synchronizer_L1BlockCheck_L1PreSafeBlockPoint ) | No | enum (of string) | No | - | L1PreSafeBlockPoint is the point that a block is considered safe enough to be checked
it can be: finalized, safe,pending or latest | | - [L1PreSafeBlockOffset](#Synchronizer_L1BlockCheck_L1PreSafeBlockOffset ) | No | integer | No | - | L1PreSafeBlockOffset is the offset to add to L1PreSafeBlockPoint as a safe point
it can be positive or negative
Example: L1PreSafeBlockPoint= finalized, L1PreSafeBlockOffset= -10, then the safe block ten blocks before the finalized block | -#### 9.7.1. `Synchronizer.L1BlockCheck.Enable` +#### 9.7.1. `Synchronizer.L1BlockCheck.Enabled` **Type:** : `boolean` **Default:** `true` -**Description:** Enable if is true then the check l1 Block Hash is active +**Description:** If enabled then the check l1 Block Hash is active **Example setting the default value** (true): ``` [Synchronizer.L1BlockCheck] -Enable=true +Enabled=true ``` #### 9.7.2. `Synchronizer.L1BlockCheck.L1SafeBlockPoint` @@ -1538,18 +1538,18 @@ L1SafeBlockOffset=0 ForceCheckBeforeStart=true ``` -#### 9.7.5. `Synchronizer.L1BlockCheck.PreCheckEnable` +#### 9.7.5. `Synchronizer.L1BlockCheck.PreCheckEnabled` **Type:** : `boolean` **Default:** `true` -**Description:** PreCheckEnable if is true then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock +**Description:** If enabled then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock **Example setting the default value** (true): ``` [Synchronizer.L1BlockCheck] -PreCheckEnable=true +PreCheckEnabled=true ``` #### 9.7.6. `Synchronizer.L1BlockCheck.L1PreSafeBlockPoint` @@ -1876,23 +1876,23 @@ FallbackToSequentialModeOnSynchronized=false | Property | Pattern | Type | Deprecated | Definition | Title/Description | | ------------------------------------------------------------------------------------------------------- | ------- | ------- | ---------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| - [Enable](#Synchronizer_L2Synchronization_Enable ) | No | boolean | No | - | Enable if is true then the L2 sync process is permitted (only for permissionless) | +| - [Enabled](#Synchronizer_L2Synchronization_Enabled ) | No | boolean | No | - | If enabled then the L2 sync process is permitted (only for permissionless) | | - [AcceptEmptyClosedBatches](#Synchronizer_L2Synchronization_AcceptEmptyClosedBatches ) | No | boolean | No | - | 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](#Synchronizer_L2Synchronization_ReprocessFullBatchOnClose ) | No | boolean | No | - | ReprocessFullBatchOnClose if is true when a batch is closed is force to reprocess again | | - [CheckLastL2BlockHashOnCloseBatch](#Synchronizer_L2Synchronization_CheckLastL2BlockHashOnCloseBatch ) | No | boolean | No | - | CheckLastL2BlockHashOnCloseBatch if is true when a batch is closed is force to check the last L2Block hash | -#### 9.10.1. `Synchronizer.L2Synchronization.Enable` +#### 9.10.1. `Synchronizer.L2Synchronization.Enabled` **Type:** : `boolean` **Default:** `true` -**Description:** Enable if is true then the L2 sync process is permitted (only for permissionless) +**Description:** If enabled then the L2 sync process is permitted (only for permissionless) **Example setting the default value** (true): ``` [Synchronizer.L2Synchronization] -Enable=true +Enabled=true ``` #### 9.10.2. `Synchronizer.L2Synchronization.AcceptEmptyClosedBatches` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index d6b0594885..d70f2d44fc 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -539,9 +539,9 @@ }, "L1BlockCheck": { "properties": { - "Enable": { + "Enabled": { "type": "boolean", - "description": "Enable if is true then the check l1 Block Hash is active", + "description": "If enabled then the check l1 Block Hash is active", "default": true }, "L1SafeBlockPoint": { @@ -564,9 +564,9 @@ "description": "ForceCheckBeforeStart if is true then the first time the system is started it will force to check all pending blocks", "default": true }, - "PreCheckEnable": { + "PreCheckEnabled": { "type": "boolean", - "description": "PreCheckEnable if is true then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock", + "description": "If enabled then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock", "default": true }, "L1PreSafeBlockPoint": { @@ -698,9 +698,9 @@ }, "L2Synchronization": { "properties": { - "Enable": { + "Enabled": { "type": "boolean", - "description": "Enable if is true then the L2 sync process is permitted (only for permissionless)", + "description": "If enabled then the L2 sync process is permitted (only for permissionless)", "default": true }, "AcceptEmptyClosedBatches": { diff --git a/synchronizer/config.go b/synchronizer/config.go index ef51d41308..4ca64414f7 100644 --- a/synchronizer/config.go +++ b/synchronizer/config.go @@ -37,8 +37,8 @@ type Config struct { // L1BlockCheckConfig Configuration for L1 Block Checker type L1BlockCheckConfig struct { - // Enable if is true then the check l1 Block Hash is active - Enable bool `mapstructure:"Enable"` + // If enabled then the check l1 Block Hash is active + Enabled bool `mapstructure:"Enabled"` // L1SafeBlockPoint is the point that a block is considered safe enough to be checked // it can be: finalized, safe,pending or latest L1SafeBlockPoint string `mapstructure:"L1SafeBlockPoint" jsonschema:"enum=finalized,enum=safe, enum=pending,enum=latest"` @@ -49,8 +49,8 @@ type L1BlockCheckConfig struct { // ForceCheckBeforeStart if is true then the first time the system is started it will force to check all pending blocks ForceCheckBeforeStart bool `mapstructure:"ForceCheckBeforeStart"` - // PreCheckEnable if is true then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock - PreCheckEnable bool `mapstructure:"PreCheckEnable"` + // If enabled then the pre-check is active, will check blocks between L1SafeBlock and L1PreSafeBlock + PreCheckEnabled bool `mapstructure:"PreCheckEnabled"` // L1PreSafeBlockPoint is the point that a block is considered safe enough to be checked // it can be: finalized, safe,pending or latest L1PreSafeBlockPoint string `mapstructure:"L1PreSafeBlockPoint" jsonschema:"enum=finalized,enum=safe, enum=pending,enum=latest"` @@ -61,7 +61,7 @@ type L1BlockCheckConfig struct { } func (c *L1BlockCheckConfig) String() string { - return fmt.Sprintf("Enable: %v, L1SafeBlockPoint: %s, L1SafeBlockOffset: %d, ForceCheckBeforeStart: %v", c.Enable, c.L1SafeBlockPoint, c.L1SafeBlockOffset, c.ForceCheckBeforeStart) + return fmt.Sprintf("Enable: %v, L1SafeBlockPoint: %s, L1SafeBlockOffset: %d, ForceCheckBeforeStart: %v", c.Enabled, c.L1SafeBlockPoint, c.L1SafeBlockOffset, c.ForceCheckBeforeStart) } // L1ParallelSynchronizationConfig Configuration for parallel mode (if UL1SynchronizationMode equal to 'parallel') diff --git a/synchronizer/l2_sync/config.go b/synchronizer/l2_sync/config.go index 8d5219bea9..c3cf1faed9 100644 --- a/synchronizer/l2_sync/config.go +++ b/synchronizer/l2_sync/config.go @@ -2,8 +2,8 @@ package l2_sync // Config configuration of L2 sync process type Config struct { - // Enable if is true then the L2 sync process is permitted (only for permissionless) - Enable bool `mapstructure:"Enable"` + // If enabled then the L2 sync process is permitted (only for permissionless) + Enabled bool `mapstructure:"Enabled"` // AcceptEmptyClosedBatches is a flag to enable or disable the acceptance of empty batches. // if true, the synchronizer will accept empty batches and process them. AcceptEmptyClosedBatches bool `mapstructure:"AcceptEmptyClosedBatches"` diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index a9d2d3c8a0..75c3c3cefc 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -125,13 +125,13 @@ func NewSynchronizer( syncBlockProtection: syncBlockProtection, halter: syncCommon.NewCriticalErrorHalt(eventLog, 5*time.Second), //nolint:gomnd } - if cfg.L1BlockCheck.Enable { + if cfg.L1BlockCheck.Enabled { log.Infof("L1BlockChecker enabled: %s", cfg.L1BlockCheck.String()) l1BlockChecker := l1_check_block.NewCheckL1BlockHash(ethMan, res.state, l1_check_block.NewSafeL1BlockNumberFetch(l1_check_block.StringToL1BlockPoint(cfg.L1BlockCheck.L1SafeBlockPoint), cfg.L1BlockCheck.L1SafeBlockOffset)) var preCheckAsync syncinterfaces.AsyncL1BlockChecker - if cfg.L1BlockCheck.PreCheckEnable { + if cfg.L1BlockCheck.PreCheckEnabled { log.Infof("L1BlockChecker enabled precheck from: %s/%d to: %s/%d", cfg.L1BlockCheck.L1SafeBlockPoint, cfg.L1BlockCheck.L1SafeBlockOffset, cfg.L1BlockCheck.L1PreSafeBlockPoint, cfg.L1BlockCheck.L1PreSafeBlockOffset) @@ -151,7 +151,7 @@ func NewSynchronizer( time.Second) } - if !isTrustedSequencer && cfg.L2Synchronization.Enable { + if !isTrustedSequencer && cfg.L2Synchronization.Enabled { log.Info("Permissionless: creating and Initializing L2 synchronization components") L1SyncChecker := l2_sync_etrog.NewCheckSyncStatusToProcessBatch(res.zkEVMClient, res.state) sync := &res diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index 172868275b..da99ca4e18 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -131,10 +131,10 @@ func TestForcedBatchEtrog(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -682,7 +682,7 @@ func setupGenericTest(t *testing.T) (*state.Genesis, *Config, *mocks) { FallbackToSequentialModeOnSynchronized: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -938,10 +938,10 @@ func TestReorg(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -1261,10 +1261,10 @@ func TestLatestSyncedBlockEmpty(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -1478,10 +1478,10 @@ func TestRegularReorg(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -1763,10 +1763,10 @@ func TestLatestSyncedBlockEmptyWithExtraReorg(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } @@ -2042,10 +2042,10 @@ func TestCallFromEmptyBlockAndReorg(t *testing.T) { L1SynchronizationMode: SequentialMode, SyncBlockProtection: "latest", L1BlockCheck: L1BlockCheckConfig{ - Enable: false, + Enabled: false, }, L2Synchronization: l2_sync.Config{ - Enable: true, + Enabled: true, }, } From 0233b40936b4c585a8e09b8217b3c9cc39d433ec Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Mon, 27 May 2024 16:50:06 +0200 Subject: [PATCH 13/22] Cherry-pick #3650: Do fatal when datastream channel is full (workaround to fix datastream blocking issue) (#3654) * Do fatal when datastream channel is full (workaround to fix datastream blocking issue) (#3650) * Do fatal when datastream channel is full (this will restart sequencer automatically) * update datastream library (more ds-debug logs) * fix decrease DataToStreamChannelCount --- go.mod | 2 +- go.sum | 4 ++-- sequencer/batch.go | 6 +++--- sequencer/datastreamer.go | 35 +++++++++++++++++++++++++++++------ sequencer/finalizer.go | 10 ++++++++-- sequencer/forcedbatch.go | 2 +- sequencer/l2block.go | 11 +---------- sequencer/sequencer.go | 24 ++++-------------------- 8 files changed, 49 insertions(+), 45 deletions(-) diff --git a/go.mod b/go.mod index db2f719558..c94597b1ab 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/0xPolygonHermez/zkevm-node go 1.21 require ( - github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240426122934-6f47d2485fc1 + github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b github.com/didip/tollbooth/v6 v6.1.2 github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 github.com/ethereum/go-ethereum v1.13.11 diff --git a/go.sum b/go.sum index a9de27ba91..bff27c6313 100644 --- a/go.sum +++ b/go.sum @@ -39,8 +39,8 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9 dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240426122934-6f47d2485fc1 h1:4wbCJOGcZ8BTuOfNFrcZ1cAVfTWaX1W9EYHaDx3imLc= -github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240426122934-6f47d2485fc1/go.mod h1:0QkAXcFa92mFJrCbN3UPUJGJYes851yEgYHLONnaosE= +github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b h1:BzQRXbSnW7BsFvJrnZbCgnxD5+nCGyrYUgqH+3vsnrM= +github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b/go.mod h1:0QkAXcFa92mFJrCbN3UPUJGJYes851yEgYHLONnaosE= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 h1:w+iIsaOQNcT7OZ575w+acHgRric5iCyQh+xv+KJ4HB8= github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8= diff --git a/sequencer/batch.go b/sequencer/batch.go index a5ada2b61a..b7cb731fe5 100644 --- a/sequencer/batch.go +++ b/sequencer/batch.go @@ -338,9 +338,9 @@ func (f *finalizer) insertSIPBatch(ctx context.Context, batchNumber uint64, stat } // Send batch bookmark to the datastream - f.DSSendBatchBookmark(batchNumber) + f.DSSendBatchBookmark(ctx, batchNumber) // Send batch start to the datastream - f.DSSendBatchStart(batchNumber, false) + f.DSSendBatchStart(ctx, batchNumber, false) // Check if synchronizer is up-to-date //TODO: review if this is needed @@ -406,7 +406,7 @@ func (f *finalizer) closeSIPBatch(ctx context.Context, dbTx pgx.Tx) error { } // Sent batch to DS - f.DSSendBatchEnd(f.sipBatch.batchNumber, f.sipBatch.finalStateRoot, f.sipBatch.finalLocalExitRoot) + f.DSSendBatchEnd(ctx, f.sipBatch.batchNumber, f.sipBatch.finalStateRoot, f.sipBatch.finalLocalExitRoot) log.Infof("sip batch %d closed in statedb, closing reason: %s", f.sipBatch.batchNumber, f.sipBatch.closingReason) diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 50321c6c55..8330c8b11c 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -1,13 +1,15 @@ package sequencer import ( - "github.com/0xPolygonHermez/zkevm-node/log" + "context" + "fmt" + "github.com/0xPolygonHermez/zkevm-node/state" "github.com/0xPolygonHermez/zkevm-node/state/datastream" "github.com/ethereum/go-ethereum/common" ) -func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32, minTimestamp uint64) error { +func (f *finalizer) DSSendL2Block(ctx context.Context, batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32, minTimestamp uint64) error { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) // Send data to streamer @@ -52,28 +54,35 @@ func (f *finalizer) DSSendL2Block(batchNumber uint64, blockResponse *state.Proce l2Transactions = append(l2Transactions, l2Transaction) } - log.Infof("[ds-debug] sending l2block %d to datastream channel", blockResponse.BlockNumber) + f.checkDSBufferIsFull(ctx) + f.dataToStream <- state.DSL2FullBlock{ DSL2Block: l2Block, Txs: l2Transactions, } + + f.dataToStreamCount.Add(1) } return nil } -func (f *finalizer) DSSendBatchBookmark(batchNumber uint64) { +func (f *finalizer) DSSendBatchBookmark(ctx context.Context, batchNumber uint64) { // Check if stream server enabled if f.streamServer != nil { + f.checkDSBufferIsFull(ctx) + // Send batch bookmark to the streamer f.dataToStream <- datastream.BookMark{ Type: datastream.BookmarkType_BOOKMARK_TYPE_BATCH, Value: batchNumber, } + + f.dataToStreamCount.Add(1) } } -func (f *finalizer) DSSendBatchStart(batchNumber uint64, isForced bool) { +func (f *finalizer) DSSendBatchStart(ctx context.Context, batchNumber uint64, isForced bool) { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) batchStart := datastream.BatchStart{ @@ -88,18 +97,32 @@ func (f *finalizer) DSSendBatchStart(batchNumber uint64, isForced bool) { } if f.streamServer != nil { + f.checkDSBufferIsFull(ctx) + // Send batch start to the streamer f.dataToStream <- batchStart + + f.dataToStreamCount.Add(1) } } -func (f *finalizer) DSSendBatchEnd(batchNumber uint64, stateRoot common.Hash, localExitRoot common.Hash) { +func (f *finalizer) DSSendBatchEnd(ctx context.Context, batchNumber uint64, stateRoot common.Hash, localExitRoot common.Hash) { if f.streamServer != nil { + f.checkDSBufferIsFull(ctx) + // Send batch end to the streamer f.dataToStream <- datastream.BatchEnd{ Number: batchNumber, StateRoot: stateRoot.Bytes(), LocalExitRoot: localExitRoot.Bytes(), } + + f.dataToStreamCount.Add(1) + } +} + +func (f *finalizer) checkDSBufferIsFull(ctx context.Context) { + if f.dataToStreamCount.Load() == datastreamChannelBufferSize { + f.Halt(ctx, fmt.Errorf("datastream channel buffer full"), true) } } diff --git a/sequencer/finalizer.go b/sequencer/finalizer.go index 9da6d691db..4522a2e50a 100644 --- a/sequencer/finalizer.go +++ b/sequencer/finalizer.go @@ -83,8 +83,9 @@ type finalizer struct { // interval metrics metrics *intervalMetrics // stream server - streamServer *datastreamer.StreamServer - dataToStream chan interface{} + streamServer *datastreamer.StreamServer + dataToStream chan interface{} + dataToStreamCount atomic.Int32 } // newFinalizer returns a new instance of Finalizer. @@ -885,6 +886,11 @@ func (f *finalizer) logZKCounters(counters state.ZKCounters) string { counters.Binaries, counters.Sha256Hashes_V2, counters.Steps) } +// Decrease datastreamChannelCount variable +func (f *finalizer) DataToStreamChannelCountAdd(ct int32) { + f.dataToStreamCount.Add(ct) +} + // Halt halts the finalizer func (f *finalizer) Halt(ctx context.Context, err error, isFatal bool) { f.haltFinalizer.Store(true) diff --git a/sequencer/forcedbatch.go b/sequencer/forcedbatch.go index abe15acc77..a9707fd8db 100644 --- a/sequencer/forcedbatch.go +++ b/sequencer/forcedbatch.go @@ -198,7 +198,7 @@ func (f *finalizer) handleProcessForcedBatchResponse(ctx context.Context, newBat } // Send L2 block to data streamer - err = f.DSSendL2Block(newBatchNumber, forcedL2BlockResponse, 0, forcedL2BlockResponse.Timestamp) + err = f.DSSendL2Block(ctx, newBatchNumber, forcedL2BlockResponse, 0, forcedL2BlockResponse.Timestamp) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d to data streamer, error: %v", forcedL2BlockResponse.BlockNumber, err) diff --git a/sequencer/l2block.go b/sequencer/l2block.go index fed7a631f9..c260867166 100644 --- a/sequencer/l2block.go +++ b/sequencer/l2block.go @@ -480,9 +480,6 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { return err } - //TODO: remove this Log - log.Infof("[ds-debug] l2 block %d [%d] stored in statedb", blockResponse.BlockNumber, l2Block.trackingNum) - // Update txs status in the pool for _, txResponse := range blockResponse.TransactionResponses { // Change Tx status to selected @@ -492,19 +489,13 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { } } - //TODO: remove this log - log.Infof("[ds-debug] l2 block %d [%d] transactions updated as selected in the pooldb", blockResponse.BlockNumber, l2Block.trackingNum) - // Send L2 block to data streamer - err = f.DSSendL2Block(l2Block.batch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) + err = f.DSSendL2Block(ctx, l2Block.batch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d [%d] to data streamer, error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err) } - //TODO: remove this log - log.Infof("[ds-debug] l2 block %d [%d] sent to datastream", blockResponse.BlockNumber, l2Block.trackingNum) - for _, tx := range l2Block.transactions { // Delete the tx from the pending list in the worker (addrQueue) f.workerIntf.DeleteTxPendingToStore(tx.Hash, tx.From) diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index 72a79cabcb..9eda30cee3 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -16,7 +16,7 @@ import ( ) const ( - datastreamChannelMultiplier = 2 + datastreamChannelBufferSize = 20 ) // Sequencer represents a sequencer @@ -52,9 +52,7 @@ func New(cfg Config, batchCfg state.BatchConfig, poolCfg pool.Config, txPool txP eventLog: eventLog, } - // TODO: Make configurable - channelBufferSize := 200 * datastreamChannelMultiplier // nolint:gomnd - sequencer.dataToStream = make(chan interface{}, channelBufferSize) + sequencer.dataToStream = make(chan interface{}, datastreamChannelBufferSize) return sequencer, nil } @@ -257,14 +255,14 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { // Read data from channel dataStream := <-s.dataToStream + s.finalizer.DataToStreamChannelCountAdd(-1) + if s.streamServer != nil { switch data := dataStream.(type) { // Stream a complete L2 block with its transactions case state.DSL2FullBlock: l2Block := data - //TODO: remove this log - log.Infof("[ds-debug] start atomic op for l2block %d", l2Block.L2BlockNumber) err = s.streamServer.StartAtomicOp() if err != nil { log.Errorf("failed to start atomic op for l2block %d, error: %v ", l2Block.L2BlockNumber, err) @@ -276,8 +274,6 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { Value: l2Block.L2BlockNumber, } - //TODO: remove this log - log.Infof("[ds-debug] add stream bookmark for l2block %d", l2Block.L2BlockNumber) marshalledBookMark, err := proto.Marshal(bookMark) if err != nil { log.Errorf("failed to marshal bookmark for l2block %d, error: %v", l2Block.L2BlockNumber, err) @@ -298,8 +294,6 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { Value: l2Block.L2BlockNumber - 1, } - //TODO: remove this log - log.Infof("[ds-debug] get previous l2block %d", l2Block.L2BlockNumber-1) marshalledBookMark, err := proto.Marshal(bookMark) if err != nil { log.Errorf("failed to marshal bookmark for l2block %d, error: %v", l2Block.L2BlockNumber, err) @@ -339,16 +333,12 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { continue } - //TODO: remove this log - log.Infof("[ds-debug] add l2blockStart stream entry for l2block %d", l2Block.L2BlockNumber) _, err = s.streamServer.AddStreamEntry(datastreamer.EntryType(datastream.EntryType_ENTRY_TYPE_L2_BLOCK), marshalledL2Block) if err != nil { log.Errorf("failed to add stream entry for l2block %d, error: %v", l2Block.L2BlockNumber, err) continue } - //TODO: remove this log - log.Infof("[ds-debug] adding l2tx stream entries for l2block %d", l2Block.L2BlockNumber) for _, l2Transaction := range l2Block.Txs { streamL2Transaction := &datastream.Transaction{ L2BlockNumber: l2Transaction.L2BlockNumber, @@ -371,17 +361,11 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { } } - //TODO: remove this log - log.Infof("[ds-debug] commit atomic op for l2block %d", l2Block.L2BlockNumber) err = s.streamServer.CommitAtomicOp() if err != nil { log.Errorf("failed to commit atomic op for l2block %d, error: %v ", l2Block.L2BlockNumber, err) continue } - - //TODO: remove this log - log.Infof("[ds-debug] l2block %d sent to datastream", l2Block.L2BlockNumber) - // Stream a bookmark case datastream.BookMark: err = s.streamServer.StartAtomicOp() From d9a1a1097f128a2102627422cb1325e795e886a7 Mon Sep 17 00:00:00 2001 From: Thiago Coimbra Lemos Date: Tue, 28 May 2024 18:59:28 -0300 Subject: [PATCH 14/22] add optimistic gas computation while estimating gas (#3653) --- state/test/forkid_common/common.go | 2 +- state/transaction.go | 142 +++++++++++++++++++---------- test/e2e/jsonrpc1_test.go | 61 +++++++++++++ 3 files changed, 158 insertions(+), 47 deletions(-) diff --git a/state/test/forkid_common/common.go b/state/test/forkid_common/common.go index 478b7e0222..2275276cd0 100644 --- a/state/test/forkid_common/common.go +++ b/state/test/forkid_common/common.go @@ -53,7 +53,7 @@ func InitTestState(stateCfg state.Config) *state.State { panic(err) } - zkProverURI := testutils.GetEnv("ZKPROVER_URI", "zkevm-prover") + zkProverURI := testutils.GetEnv("ZKPROVER_URI", "localhost") executorServerConfig := executor.Config{URI: fmt.Sprintf("%s:50071", zkProverURI), MaxGRPCMessageSize: 100000000} ExecutorClient, executorClientConn, executorCancel = executor.NewExecutorClient(ctx, executorServerConfig) diff --git a/state/transaction.go b/state/transaction.go index 56e64b83e1..1c2e0e11da 100644 --- a/state/transaction.go +++ b/state/transaction.go @@ -15,6 +15,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/trie" "github.com/google/uuid" "github.com/jackc/pgx/v4" @@ -22,6 +23,13 @@ import ( "google.golang.org/grpc/status" ) +type testGasEstimationResult struct { + failed, reverted, ooc bool + gasUsed, gasRefund uint64 + returnValue []byte + executionError error +} + // GetSender gets the sender from the transaction's signature func GetSender(tx types.Transaction) (common.Address, error) { signer := types.NewEIP155Signer(tx.ChainId()) @@ -806,18 +814,23 @@ func (s *State) EstimateGas(transaction *types.Transaction, senderAddress common // Check if the highEnd is a good value to make the transaction pass, if it fails we // can return immediately. log.Debugf("Estimate gas. Trying to execute TX with %v gas", highEnd) - var failed, reverted bool - var gasUsed uint64 - var returnValue []byte + var estimationResult *testGasEstimationResult if forkID < FORKID_ETROG { - failed, reverted, gasUsed, returnValue, err = s.internalTestGasEstimationTransactionV1(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, highEnd, nonce, false) + estimationResult, err = s.internalTestGasEstimationTransactionV1(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, highEnd, nonce, false) } else { - failed, reverted, gasUsed, returnValue, err = s.internalTestGasEstimationTransactionV2(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, highEnd, nonce, false) + estimationResult, err = s.internalTestGasEstimationTransactionV2(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, highEnd, nonce, false) + } + if err != nil { + return 0, nil, err } - if failed { - if reverted { - return 0, returnValue, err + if estimationResult.failed { + if estimationResult.reverted { + return 0, estimationResult.returnValue, estimationResult.executionError + } + + if estimationResult.ooc { + return 0, nil, estimationResult.executionError } // The transaction shouldn't fail, for whatever reason, at highEnd @@ -828,8 +841,28 @@ func (s *State) EstimateGas(transaction *types.Transaction, senderAddress common } // sets - if lowEnd < gasUsed { - lowEnd = gasUsed + if lowEnd < estimationResult.gasUsed { + lowEnd = estimationResult.gasUsed + } + + optimisticGasLimit := (estimationResult.gasUsed + estimationResult.gasRefund + params.CallStipend) * 64 / 63 // nolint:gomnd + if optimisticGasLimit < highEnd { + if forkID < FORKID_ETROG { + estimationResult, err = s.internalTestGasEstimationTransactionV1(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, optimisticGasLimit, nonce, false) + } else { + estimationResult, err = s.internalTestGasEstimationTransactionV2(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, optimisticGasLimit, nonce, false) + } + if err != nil { + // This should not happen under normal conditions since if we make it this far the + // transaction had run without error at least once before. + log.Error("Execution error in estimate gas", "err", err) + return 0, nil, err + } + if estimationResult.failed { + lowEnd = optimisticGasLimit + } else { + highEnd = optimisticGasLimit + } } // Start the binary search for the lowest possible gas price @@ -845,20 +878,20 @@ func (s *State) EstimateGas(transaction *types.Transaction, senderAddress common log.Debugf("Estimate gas. Trying to execute TX with %v gas", mid) if forkID < FORKID_ETROG { - failed, reverted, _, _, err = s.internalTestGasEstimationTransactionV1(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, mid, nonce, true) + estimationResult, err = s.internalTestGasEstimationTransactionV1(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, mid, nonce, true) } else { - failed, reverted, _, _, err = s.internalTestGasEstimationTransactionV2(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, mid, nonce, true) + estimationResult, err = s.internalTestGasEstimationTransactionV2(ctx, batch, l2Block, latestL2BlockNumber, transaction, forkID, senderAddress, mid, nonce, true) } executionTime := time.Since(txExecutionStart) totalExecutionTime += executionTime txExecutions = append(txExecutions, executionTime) - if err != nil && !reverted { + if err != nil && !estimationResult.reverted { // Reverts are ignored in the binary search, but are checked later on // during the execution for the optimal gas limit found return 0, nil, err } - if failed { + if estimationResult.failed { // If the transaction failed => increase the gas lowEnd = mid + 1 } else { @@ -881,7 +914,7 @@ func (s *State) EstimateGas(transaction *types.Transaction, senderAddress common // before ETROG func (s *State) internalTestGasEstimationTransactionV1(ctx context.Context, batch *Batch, l2Block *L2Block, latestL2BlockNumber uint64, transaction *types.Transaction, forkID uint64, senderAddress common.Address, - gas uint64, nonce uint64, shouldOmitErr bool) (failed, reverted bool, gasUsed uint64, returnValue []byte, err error) { + gas uint64, nonce uint64, shouldOmitErr bool) (*testGasEstimationResult, error) { timestamp := l2Block.Time() if l2Block.NumberU64() == latestL2BlockNumber { timestamp = uint64(time.Now().Unix()) @@ -899,7 +932,7 @@ func (s *State) internalTestGasEstimationTransactionV1(ctx context.Context, batc batchL2Data, err := EncodeUnsignedTransaction(*tx, s.cfg.ChainID, &nonce, forkID) if err != nil { log.Errorf("error encoding unsigned transaction ", err) - return false, false, gasUsed, nil, err + return nil, err } // Create a batch to be sent to the executor @@ -938,38 +971,44 @@ func (s *State) internalTestGasEstimationTransactionV1(ctx context.Context, batc log.Debugf("executor time: %vms", time.Since(txExecutionOnExecutorTime).Milliseconds()) if err != nil { log.Errorf("error estimating gas: %v", err) - return false, false, gasUsed, nil, err + return nil, err } if processBatchResponse.Error != executor.ExecutorError_EXECUTOR_ERROR_NO_ERROR { err = executor.ExecutorErr(processBatchResponse.Error) s.eventLog.LogExecutorError(ctx, processBatchResponse.Error, processBatchRequestV1) - return false, false, gasUsed, nil, err + return nil, err } - gasUsed = processBatchResponse.Responses[0].GasUsed txResponse := processBatchResponse.Responses[0] + result := &testGasEstimationResult{} + result.gasUsed = txResponse.GasUsed + result.gasRefund = txResponse.GasRefunded // Check if an out of gas error happened during EVM execution if txResponse.Error != executor.RomError_ROM_ERROR_NO_ERROR { - err := executor.RomErr(txResponse.Error) + result.failed = true + result.executionError = executor.RomErr(txResponse.Error) - if (isGasEVMError(err) || isGasApplyError(err)) && shouldOmitErr { + if (isGasEVMError(result.executionError) || isGasApplyError(result.executionError)) && shouldOmitErr { // Specifying the transaction failed, but not providing an error // is an indication that a valid error occurred due to low gas, // which will increase the lower bound for the search - return true, false, gasUsed, nil, nil - } - - if isEVMRevertError(err) { + return result, nil + } else if isEVMRevertError(result.executionError) { // The EVM reverted during execution, attempt to extract the // error message and return it - returnValue := txResponse.ReturnValue - return true, true, gasUsed, returnValue, ConstructErrorFromRevert(err, returnValue) + result.reverted = true + result.returnValue = txResponse.ReturnValue + result.executionError = ConstructErrorFromRevert(err, txResponse.ReturnValue) + } else if isOOCError(result.executionError) { + // The EVM got into an OOC error + result.ooc = true + return result, nil } - return true, false, gasUsed, nil, err + return result, nil } - return false, false, gasUsed, nil, nil + return result, nil } // internalTestGasEstimationTransactionV2 is used by the EstimateGas to test the tx execution @@ -977,7 +1016,7 @@ func (s *State) internalTestGasEstimationTransactionV1(ctx context.Context, batc // after ETROG func (s *State) internalTestGasEstimationTransactionV2(ctx context.Context, batch *Batch, l2Block *L2Block, latestL2BlockNumber uint64, transaction *types.Transaction, forkID uint64, senderAddress common.Address, - gas uint64, nonce uint64, shouldOmitErr bool) (failed, reverted bool, gasUsed uint64, returnValue []byte, err error) { + gas uint64, nonce uint64, shouldOmitErr bool) (*testGasEstimationResult, error) { deltaTimestamp := uint32(uint64(time.Now().Unix()) - l2Block.Time()) transactions := s.BuildChangeL2Block(deltaTimestamp, uint32(0)) @@ -993,7 +1032,7 @@ func (s *State) internalTestGasEstimationTransactionV2(ctx context.Context, batc batchL2Data, err := EncodeUnsignedTransaction(*tx, s.cfg.ChainID, &nonce, forkID) if err != nil { log.Errorf("error encoding unsigned transaction ", err) - return false, false, gasUsed, nil, err + return nil, err } transactions = append(transactions, batchL2Data...) @@ -1038,43 +1077,48 @@ func (s *State) internalTestGasEstimationTransactionV2(ctx context.Context, batc log.Debugf("executor time: %vms", time.Since(txExecutionOnExecutorTime).Milliseconds()) if err != nil { log.Errorf("error estimating gas: %v", err) - return false, false, gasUsed, nil, err + return nil, err } if processBatchResponseV2.Error != executor.ExecutorError_EXECUTOR_ERROR_NO_ERROR { err = executor.ExecutorErr(processBatchResponseV2.Error) s.eventLog.LogExecutorErrorV2(ctx, processBatchResponseV2.Error, processBatchRequestV2) - return false, false, gasUsed, nil, err + return nil, err } if processBatchResponseV2.ErrorRom != executor.RomError_ROM_ERROR_NO_ERROR { err = executor.RomErr(processBatchResponseV2.ErrorRom) - return false, false, gasUsed, nil, err + return nil, err } - gasUsed = processBatchResponseV2.BlockResponses[0].GasUsed - txResponse := processBatchResponseV2.BlockResponses[0].Responses[0] + result := &testGasEstimationResult{} + result.gasUsed = txResponse.GasUsed + result.gasRefund = txResponse.GasRefunded // Check if an out of gas error happened during EVM execution if txResponse.Error != executor.RomError_ROM_ERROR_NO_ERROR { - err := executor.RomErr(txResponse.Error) + result.failed = true + result.executionError = executor.RomErr(txResponse.Error) - if (isGasEVMError(err) || isGasApplyError(err)) && shouldOmitErr { + if (isGasEVMError(result.executionError) || isGasApplyError(result.executionError)) && shouldOmitErr { // Specifying the transaction failed, but not providing an error // is an indication that a valid error occurred due to low gas, // which will increase the lower bound for the search - return true, false, gasUsed, nil, nil - } - - if isEVMRevertError(err) { + return result, nil + } else if isEVMRevertError(result.executionError) { // The EVM reverted during execution, attempt to extract the // error message and return it - returnValue := txResponse.ReturnValue - return true, true, gasUsed, returnValue, ConstructErrorFromRevert(err, returnValue) + result.reverted = true + result.returnValue = txResponse.ReturnValue + result.executionError = ConstructErrorFromRevert(result.executionError, txResponse.ReturnValue) + } else if isOOCError(result.executionError) { + // The EVM got into an OOC error + result.ooc = true + return result, nil } - return true, false, gasUsed, nil, err + return result, nil } - return false, false, gasUsed, nil, nil + return result, nil } // Checks if executor level valid gas errors occurred @@ -1091,3 +1135,9 @@ func isGasEVMError(err error) bool { func isEVMRevertError(err error) bool { return errors.Is(err, runtime.ErrExecutionReverted) } + +// Checks if the EVM stopped tx execution due to OOC error +func isOOCError(err error) bool { + romErr := executor.RomErrorCode(err) + return executor.IsROMOutOfCountersError(romErr) +} diff --git a/test/e2e/jsonrpc1_test.go b/test/e2e/jsonrpc1_test.go index c0c867bb50..5267d32208 100644 --- a/test/e2e/jsonrpc1_test.go +++ b/test/e2e/jsonrpc1_test.go @@ -6,6 +6,7 @@ import ( "math/big" "reflect" "testing" + "time" "github.com/0xPolygonHermez/zkevm-node/hex" "github.com/0xPolygonHermez/zkevm-node/jsonrpc/client" @@ -791,3 +792,63 @@ func Test_EstimateCounters(t *testing.T) { }) } } + +func Test_Gas_Bench2(t *testing.T) { + if testing.Short() { + t.Skip() + } + ctx := context.Background() + setup() + defer teardown() + ethClient, err := ethclient.Dial(operations.DefaultL2NetworkURL) + require.NoError(t, err) + auth, err := operations.GetAuth(operations.DefaultSequencerPrivateKey, operations.DefaultL2ChainID) + require.NoError(t, err) + + type testCase struct { + name string + execute func(*testing.T, context.Context, *triggerErrors.TriggerErrors, *ethclient.Client, bind.TransactOpts) string + expectedError string + } + + testCases := []testCase{ + { + name: "estimate gas with given gas limit", + execute: func(t *testing.T, ctx context.Context, sc *triggerErrors.TriggerErrors, c *ethclient.Client, a bind.TransactOpts) string { + a.GasLimit = 30000000 + a.NoSend = true + tx, err := sc.OutOfCountersPoseidon(&a) + require.NoError(t, err) + + t0 := time.Now() + _, err = c.EstimateGas(ctx, ethereum.CallMsg{ + From: a.From, + To: tx.To(), + Gas: tx.Gas(), + GasPrice: tx.GasPrice(), + Value: tx.Value(), + Data: tx.Data(), + }) + log.Infof("EstimateGas time: %v", time.Since(t0)) + if err != nil { + return err.Error() + } + return "" + }, + expectedError: "", + }, + } + + // deploy triggerErrors SC + _, tx, sc, err := triggerErrors.DeployTriggerErrors(auth, ethClient) + require.NoError(t, err) + + err = operations.WaitTxToBeMined(ctx, ethClient, tx, operations.DefaultTimeoutTxToBeMined) + require.NoError(t, err) + + for _, testCase := range testCases { + t.Run(testCase.name, func(t *testing.T) { + testCase.execute(t, context.Background(), sc, ethClient, *auth) + }) + } +} From cea67d181aec10d458e5310e0b1117ef544bbbe1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Thu, 30 May 2024 11:49:55 +0200 Subject: [PATCH 15/22] Use Eth block hash for l2 blocks in data stream (#3661) * Use Eth block hash for l2 blocks in data stream * handle minTimestamp --- sequencer/datastreamer.go | 10 +++++-- sequencer/forcedbatch.go | 4 +-- sequencer/interfaces.go | 2 +- sequencer/l2block.go | 4 +-- sequencer/mock_state.go | 22 +++++++++++---- sequencer/sequencer.go | 2 +- state/batchV2.go | 2 +- state/datastream.go | 16 ++++++++--- state/transaction.go | 10 +++---- .../mocks/state_full_interface.go | 28 +++++++++++++------ synchronizer/common/syncinterfaces/state.go | 2 +- .../executor_trusted_batch_sync.go | 4 +-- .../l2_sync_etrog/mocks/state_interface.go | 28 +++++++++++++------ synchronizer/synchronizer_test.go | 2 +- 14 files changed, 92 insertions(+), 44 deletions(-) diff --git a/sequencer/datastreamer.go b/sequencer/datastreamer.go index 8330c8b11c..0766ed348e 100644 --- a/sequencer/datastreamer.go +++ b/sequencer/datastreamer.go @@ -9,7 +9,7 @@ import ( "github.com/ethereum/go-ethereum/common" ) -func (f *finalizer) DSSendL2Block(ctx context.Context, batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32, minTimestamp uint64) error { +func (f *finalizer) DSSendL2Block(ctx context.Context, batchNumber uint64, blockResponse *state.ProcessBlockResponse, l1InfoTreeIndex uint32, minTimestamp uint64, blockHash common.Hash) error { forkID := f.stateIntf.GetForkIDByBatchNumber(batchNumber) // Send data to streamer @@ -18,17 +18,21 @@ func (f *finalizer) DSSendL2Block(ctx context.Context, batchNumber uint64, block BatchNumber: batchNumber, L2BlockNumber: blockResponse.BlockNumber, Timestamp: blockResponse.Timestamp, - Min_timestamp: minTimestamp, + MinTimestamp: minTimestamp, L1InfoTreeIndex: l1InfoTreeIndex, L1BlockHash: blockResponse.BlockHashL1, GlobalExitRoot: blockResponse.GlobalExitRoot, Coinbase: f.l2Coinbase, ForkID: forkID, - BlockHash: blockResponse.BlockHash, + BlockHash: blockHash, StateRoot: blockResponse.BlockHash, //From etrog, the blockhash is the block root BlockInfoRoot: blockResponse.BlockInfoRoot, } + if l2Block.ForkID >= state.FORKID_ETROG && l2Block.L1InfoTreeIndex == 0 { + l2Block.MinTimestamp = 0 + } + l2Transactions := []state.DSL2Transaction{} for i, txResponse := range blockResponse.TransactionResponses { diff --git a/sequencer/forcedbatch.go b/sequencer/forcedbatch.go index a9707fd8db..de1082c347 100644 --- a/sequencer/forcedbatch.go +++ b/sequencer/forcedbatch.go @@ -180,7 +180,7 @@ func (f *finalizer) handleProcessForcedBatchResponse(ctx context.Context, newBat // process L2 blocks responses for the forced batch for _, forcedL2BlockResponse := range batchResponse.BlockResponses { // Store forced L2 blocks in the state - err := f.stateIntf.StoreL2Block(ctx, newBatchNumber, forcedL2BlockResponse, nil, dbTx) + blockHash, err := f.stateIntf.StoreL2Block(ctx, newBatchNumber, forcedL2BlockResponse, nil, dbTx) if err != nil { return fmt.Errorf("database error on storing L2 block %d, error: %v", forcedL2BlockResponse.BlockNumber, err) } @@ -198,7 +198,7 @@ func (f *finalizer) handleProcessForcedBatchResponse(ctx context.Context, newBat } // Send L2 block to data streamer - err = f.DSSendL2Block(ctx, newBatchNumber, forcedL2BlockResponse, 0, forcedL2BlockResponse.Timestamp) + err = f.DSSendL2Block(ctx, newBatchNumber, forcedL2BlockResponse, 0, forcedL2BlockResponse.Timestamp, blockHash) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d to data streamer, error: %v", forcedL2BlockResponse.BlockNumber, err) diff --git a/sequencer/interfaces.go b/sequencer/interfaces.go index c92f502e10..72beb71578 100644 --- a/sequencer/interfaces.go +++ b/sequencer/interfaces.go @@ -72,7 +72,7 @@ type stateInterface interface { GetDSL2Blocks(ctx context.Context, firstBatchNumber, lastBatchNumber uint64, dbTx pgx.Tx) ([]*state.DSL2Block, error) GetDSL2Transactions(ctx context.Context, firstL2Block, lastL2Block uint64, dbTx pgx.Tx) ([]*state.DSL2Transaction, error) GetStorageAt(ctx context.Context, address common.Address, position *big.Int, root common.Hash) (*big.Int, error) - StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error + StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) BuildChangeL2Block(deltaTimestamp uint32, l1InfoTreeIndex uint32) []byte GetL1InfoTreeDataFromBatchL2Data(ctx context.Context, batchL2Data []byte, dbTx pgx.Tx) (map[uint32]state.L1DataV2, common.Hash, common.Hash, error) GetBlockByNumber(ctx context.Context, blockNumber uint64, dbTx pgx.Tx) (*state.Block, error) diff --git a/sequencer/l2block.go b/sequencer/l2block.go index c260867166..7cc21fc928 100644 --- a/sequencer/l2block.go +++ b/sequencer/l2block.go @@ -423,7 +423,7 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { } // Store L2 block in the state - err = f.stateIntf.StoreL2Block(ctx, l2Block.batch.batchNumber, blockResponse, txsEGPLog, dbTx) + blockHash, err := f.stateIntf.StoreL2Block(ctx, l2Block.batch.batchNumber, blockResponse, txsEGPLog, dbTx) if err != nil { return rollbackOnError(fmt.Errorf("database error on storing L2 block %d [%d], error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err)) } @@ -490,7 +490,7 @@ func (f *finalizer) storeL2Block(ctx context.Context, l2Block *L2Block) error { } // Send L2 block to data streamer - err = f.DSSendL2Block(ctx, l2Block.batch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp) + err = f.DSSendL2Block(ctx, l2Block.batch.batchNumber, blockResponse, l2Block.getL1InfoTreeIndex(), l2Block.timestamp, blockHash) if err != nil { //TODO: we need to halt/rollback the L2 block if we had an error sending to the data streamer? log.Errorf("error sending L2 block %d [%d] to data streamer, error: %v", blockResponse.BlockNumber, l2Block.trackingNum, err) diff --git a/sequencer/mock_state.go b/sequencer/mock_state.go index f7f3861d4b..0c1edc59e5 100644 --- a/sequencer/mock_state.go +++ b/sequencer/mock_state.go @@ -1019,21 +1019,33 @@ func (_m *StateMock) ProcessBatchV2(ctx context.Context, request state.ProcessRe } // StoreL2Block provides a mock function with given fields: ctx, batchNumber, l2Block, txsEGPLog, dbTx -func (_m *StateMock) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error { +func (_m *StateMock) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) { ret := _m.Called(ctx, batchNumber, l2Block, txsEGPLog, dbTx) if len(ret) == 0 { panic("no return value specified for StoreL2Block") } - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + var r0 common.Hash + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) (common.Hash, error)); ok { + return rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) common.Hash); ok { r0 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) } else { - r0 = ret.Error(0) + if ret.Get(0) != nil { + r0 = ret.Get(0).(common.Hash) + } } - return r0 + if rf, ok := ret.Get(1).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + r1 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // UpdateBatchAsChecked provides a mock function with given fields: ctx, batchNumber, dbTx diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index 9eda30cee3..c573c09924 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -318,7 +318,7 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { BatchNumber: l2Block.BatchNumber, Timestamp: l2Block.Timestamp, DeltaTimestamp: uint32(l2Block.Timestamp - previousL2Block.Timestamp), - MinTimestamp: l2Block.Min_timestamp, + MinTimestamp: l2Block.MinTimestamp, L1Blockhash: l2Block.L1BlockHash.Bytes(), L1InfotreeIndex: l2Block.L1InfoTreeIndex, Hash: l2Block.BlockHash.Bytes(), diff --git a/state/batchV2.go b/state/batchV2.go index 32f6fd856b..7674347e6e 100644 --- a/state/batchV2.go +++ b/state/batchV2.go @@ -401,7 +401,7 @@ func (s *State) ProcessAndStoreClosedBatchV2(ctx context.Context, processingCtx if len(processedBatch.BlockResponses) > 0 && !processedBatch.IsRomOOCError && processedBatch.RomError_V2 == nil { for _, blockResponse := range processedBatch.BlockResponses { - err = s.StoreL2Block(ctx, processingCtx.BatchNumber, blockResponse, nil, dbTx) + _, err = s.StoreL2Block(ctx, processingCtx.BatchNumber, blockResponse, nil, dbTx) if err != nil { log.Errorf("%s error StoreL2Block: %v", debugPrefix, err) return common.Hash{}, noFlushID, noProverID, err diff --git a/state/datastream.go b/state/datastream.go index 90c7ff3211..09fa7184b1 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -49,7 +49,7 @@ type DSL2Block struct { BatchNumber uint64 L2BlockNumber uint64 Timestamp uint64 - Min_timestamp uint64 + MinTimestamp uint64 L1InfoTreeIndex uint32 L1BlockHash common.Hash GlobalExitRoot common.Hash @@ -512,14 +512,22 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre BlockGasLimit: l2Block.BlockGasLimit, } - if l2Block.ForkID >= FORKID_ETROG { - streamL2Block.Hash = l2Block.StateRoot.Bytes() - } + // Keep the l2 block hash as it is, as the state root can be found in the StateRoot field + // So disable this + /* + if l2Block.ForkID >= FORKID_ETROG { + streamL2Block.Hash = l2Block.StateRoot.Bytes() + } + */ if l2Block.ForkID == FORKID_ETROG && batch.EtrogTimestamp != nil { streamL2Block.MinTimestamp = uint64(batch.EtrogTimestamp.Unix()) } + if l2Block.ForkID >= FORKID_ETROG && l2Block.L1InfoTreeIndex == 0 { + streamL2Block.MinTimestamp = 0 + } + previousTimestamp = l2Block.Timestamp bookMark := &datastream.BookMark{ diff --git a/state/transaction.go b/state/transaction.go index 1c2e0e11da..d9c044b33d 100644 --- a/state/transaction.go +++ b/state/transaction.go @@ -208,9 +208,9 @@ func (s *State) StoreTransactions(ctx context.Context, batchNumber uint64, proce } // StoreL2Block stores a l2 block into the state -func (s *State) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *ProcessBlockResponse, txsEGPLog []*EffectiveGasPriceLog, dbTx pgx.Tx) error { +func (s *State) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *ProcessBlockResponse, txsEGPLog []*EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) { if dbTx == nil { - return ErrDBTxNil + return common.Hash{}, ErrDBTxNil } log.Debugf("storing l2 block %d, txs %d, hash %s", l2Block.BlockNumber, len(l2Block.TransactionResponses), l2Block.BlockHash.String()) @@ -218,7 +218,7 @@ func (s *State) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *P prevL2BlockHash, err := s.GetL2BlockHashByNumber(ctx, l2Block.BlockNumber-1, dbTx) if err != nil { - return err + return common.Hash{}, err } forkID := s.GetForkIDByBatchNumber(batchNumber) @@ -289,12 +289,12 @@ func (s *State) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *P // Store L2 block and its transactions if err := s.AddL2Block(ctx, batchNumber, block, receipts, txsL2Hash, storeTxsEGPData, imStateRoots, dbTx); err != nil { - return err + return common.Hash{}, err } log.Debugf("stored L2 block %d for batch %d, storing time %v", header.Number, batchNumber, time.Since(start)) - return nil + return block.Hash(), nil } // PreProcessUnsignedTransaction processes the unsigned transaction in order to calculate its zkCounters diff --git a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go index ec779c4854..248c817fa6 100644 --- a/synchronizer/common/syncinterfaces/mocks/state_full_interface.go +++ b/synchronizer/common/syncinterfaces/mocks/state_full_interface.go @@ -2799,21 +2799,33 @@ func (_c *StateFullInterface_SetLastBatchInfoSeenOnEthereum_Call) RunAndReturn(r } // StoreL2Block provides a mock function with given fields: ctx, batchNumber, l2Block, txsEGPLog, dbTx -func (_m *StateFullInterface) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error { +func (_m *StateFullInterface) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) { ret := _m.Called(ctx, batchNumber, l2Block, txsEGPLog, dbTx) if len(ret) == 0 { panic("no return value specified for StoreL2Block") } - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + var r0 common.Hash + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) (common.Hash, error)); ok { + return rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) common.Hash); ok { r0 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) } else { - r0 = ret.Error(0) + if ret.Get(0) != nil { + r0 = ret.Get(0).(common.Hash) + } } - return r0 + if rf, ok := ret.Get(1).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + r1 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // StateFullInterface_StoreL2Block_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'StoreL2Block' @@ -2838,12 +2850,12 @@ func (_c *StateFullInterface_StoreL2Block_Call) Run(run func(ctx context.Context return _c } -func (_c *StateFullInterface_StoreL2Block_Call) Return(_a0 error) *StateFullInterface_StoreL2Block_Call { - _c.Call.Return(_a0) +func (_c *StateFullInterface_StoreL2Block_Call) Return(_a0 common.Hash, _a1 error) *StateFullInterface_StoreL2Block_Call { + _c.Call.Return(_a0, _a1) return _c } -func (_c *StateFullInterface_StoreL2Block_Call) RunAndReturn(run func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error) *StateFullInterface_StoreL2Block_Call { +func (_c *StateFullInterface_StoreL2Block_Call) RunAndReturn(run func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) (common.Hash, error)) *StateFullInterface_StoreL2Block_Call { _c.Call.Return(run) return _c } diff --git a/synchronizer/common/syncinterfaces/state.go b/synchronizer/common/syncinterfaces/state.go index 2fb9ea2f16..41d7acbedc 100644 --- a/synchronizer/common/syncinterfaces/state.go +++ b/synchronizer/common/syncinterfaces/state.go @@ -66,7 +66,7 @@ type StateFullInterface interface { GetForkIDByBlockNumber(blockNumber uint64) uint64 GetStoredFlushID(ctx context.Context) (uint64, string, error) AddL1InfoTreeLeaf(ctx context.Context, L1InfoTreeLeaf *state.L1InfoTreeLeaf, dbTx pgx.Tx) (*state.L1InfoTreeExitRootStorageEntry, error) - StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error + StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) GetL1InfoRootLeafByL1InfoRoot(ctx context.Context, l1InfoRoot common.Hash, dbTx pgx.Tx) (state.L1InfoTreeExitRootStorageEntry, error) UpdateWIPBatch(ctx context.Context, receipt state.ProcessingReceipt, dbTx pgx.Tx) error GetL1InfoTreeDataFromBatchL2Data(ctx context.Context, batchL2Data []byte, dbTx pgx.Tx) (map[uint32]state.L1DataV2, common.Hash, common.Hash, error) diff --git a/synchronizer/l2_sync/l2_sync_etrog/executor_trusted_batch_sync.go b/synchronizer/l2_sync/l2_sync_etrog/executor_trusted_batch_sync.go index 75aebc942c..b7e7e34122 100644 --- a/synchronizer/l2_sync/l2_sync_etrog/executor_trusted_batch_sync.go +++ b/synchronizer/l2_sync/l2_sync_etrog/executor_trusted_batch_sync.go @@ -37,7 +37,7 @@ type StateInterface interface { ResetTrustedState(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) error OpenBatch(ctx context.Context, processingContext state.ProcessingContext, dbTx pgx.Tx) error ProcessBatchV2(ctx context.Context, request state.ProcessRequest, updateMerkleTree bool) (*state.ProcessBatchResponse, string, error) - StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error + StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) GetL1InfoTreeDataFromBatchL2Data(ctx context.Context, batchL2Data []byte, dbTx pgx.Tx) (map[uint32]state.L1DataV2, common.Hash, common.Hash, error) GetLastVirtualBatchNum(ctx context.Context, dbTx pgx.Tx) (uint64, error) } @@ -392,7 +392,7 @@ func (b *SyncTrustedBatchExecutorForEtrog) processAndStoreTxs(ctx context.Contex } for _, block := range processBatchResp.BlockResponses { log.Debugf("%s Storing trusted tx %d", debugPrefix, block.BlockNumber) - if err = b.state.StoreL2Block(ctx, request.BatchNumber, block, nil, dbTx); err != nil { + if _, err = b.state.StoreL2Block(ctx, request.BatchNumber, block, nil, dbTx); err != nil { newErr := fmt.Errorf("%s failed to store l2block: %v err:%w", debugPrefix, block.BlockNumber, err) log.Error(newErr.Error()) return nil, newErr diff --git a/synchronizer/l2_sync/l2_sync_etrog/mocks/state_interface.go b/synchronizer/l2_sync/l2_sync_etrog/mocks/state_interface.go index 43e84ffba5..0b4d85bce7 100644 --- a/synchronizer/l2_sync/l2_sync_etrog/mocks/state_interface.go +++ b/synchronizer/l2_sync/l2_sync_etrog/mocks/state_interface.go @@ -538,21 +538,33 @@ func (_c *StateInterface_ResetTrustedState_Call) RunAndReturn(run func(context.C } // StoreL2Block provides a mock function with given fields: ctx, batchNumber, l2Block, txsEGPLog, dbTx -func (_m *StateInterface) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) error { +func (_m *StateInterface) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *state.ProcessBlockResponse, txsEGPLog []*state.EffectiveGasPriceLog, dbTx pgx.Tx) (common.Hash, error) { ret := _m.Called(ctx, batchNumber, l2Block, txsEGPLog, dbTx) if len(ret) == 0 { panic("no return value specified for StoreL2Block") } - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + var r0 common.Hash + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) (common.Hash, error)); ok { + return rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } + if rf, ok := ret.Get(0).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) common.Hash); ok { r0 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) } else { - r0 = ret.Error(0) + if ret.Get(0) != nil { + r0 = ret.Get(0).(common.Hash) + } } - return r0 + if rf, ok := ret.Get(1).(func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error); ok { + r1 = rf(ctx, batchNumber, l2Block, txsEGPLog, dbTx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // StateInterface_StoreL2Block_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'StoreL2Block' @@ -577,12 +589,12 @@ func (_c *StateInterface_StoreL2Block_Call) Run(run func(ctx context.Context, ba return _c } -func (_c *StateInterface_StoreL2Block_Call) Return(_a0 error) *StateInterface_StoreL2Block_Call { - _c.Call.Return(_a0) +func (_c *StateInterface_StoreL2Block_Call) Return(_a0 common.Hash, _a1 error) *StateInterface_StoreL2Block_Call { + _c.Call.Return(_a0, _a1) return _c } -func (_c *StateInterface_StoreL2Block_Call) RunAndReturn(run func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) error) *StateInterface_StoreL2Block_Call { +func (_c *StateInterface_StoreL2Block_Call) RunAndReturn(run func(context.Context, uint64, *state.ProcessBlockResponse, []*state.EffectiveGasPriceLog, pgx.Tx) (common.Hash, error)) *StateInterface_StoreL2Block_Call { _c.Call.Return(run) return _c } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index da99ca4e18..ce0c27e280 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -901,7 +901,7 @@ func expectedCallsForsyncTrustedState(t *testing.T, m *mocks, sync *ClientSynchr m.State.EXPECT().ProcessBatchV2(mock.Anything, mock.Anything, mock.Anything). Return(&processedBatch, "", nil).Times(1) m.State.EXPECT().StoreL2Block(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything). - Return(nil).Times(1) + Return(common.Hash{}, nil).Times(1) m.State.EXPECT().UpdateWIPBatch(mock.Anything, mock.Anything, mock.Anything). Return(nil).Times(1) m.State.EXPECT().GetBatchByNumber(mock.Anything, mock.Anything, mock.Anything). From d9a240f7dc511c104141c8fd8a15fd36aec7185d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Toni=20Ram=C3=ADrez?= <58293609+ToniRamirezM@users.noreply.github.com> Date: Thu, 30 May 2024 15:12:10 +0200 Subject: [PATCH 16/22] empty imStateRoot (#3663) * empty imStateRoot * fix comment --- sequencer/sequencer.go | 8 ++++++++ state/datastream.go | 5 +++++ 2 files changed, 13 insertions(+) diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go index c573c09924..f191973767 100644 --- a/sequencer/sequencer.go +++ b/sequencer/sequencer.go @@ -12,6 +12,7 @@ import ( "github.com/0xPolygonHermez/zkevm-node/pool" "github.com/0xPolygonHermez/zkevm-node/state" "github.com/0xPolygonHermez/zkevm-node/state/datastream" + "github.com/ethereum/go-ethereum/common" "google.golang.org/protobuf/proto" ) @@ -348,6 +349,13 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) { ImStateRoot: l2Transaction.ImStateRoot.Bytes(), } + // Clear the state root if the ForkID is >= ETROG + // currently this is redundant as the current implementation of the sequencer + // leaves the ImStateRoot empty + if l2Block.ForkID >= state.FORKID_ETROG { + streamL2Transaction.ImStateRoot = common.Hash{}.Bytes() + } + marshalledL2Transaction, err := proto.Marshal(streamL2Transaction) if err != nil { log.Errorf("failed to marshal l2tx for l2block %d, error: %v", l2Block.L2BlockNumber, err) diff --git a/state/datastream.go b/state/datastream.go index 09fa7184b1..ecb211d6c2 100644 --- a/state/datastream.go +++ b/state/datastream.go @@ -590,6 +590,11 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre ImStateRoot: tx.StateRoot.Bytes(), } + // Clear the state root if the ForkID is >= ETROG + if l2Block.ForkID >= FORKID_ETROG { + transaction.ImStateRoot = common.Hash{}.Bytes() + } + marshalledTransaction, err := proto.Marshal(transaction) if err != nil { return err From 92add9ae58c0ac7dd4e3413da41165f00ff24fc5 Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Fri, 31 May 2024 16:18:01 +0200 Subject: [PATCH 17/22] Cherry-pick #3659,#3662: Remove sync with virtual state (synchronizer). Add L1 block confirmations (#3666) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Remove sync with virtual state (synchronizer). Add L1 block confirmat… (#3659) * Remove sync with virtual state (synchronizer). Add L1 block confirmations to consider sequence final * fix get monitored tx receipt * update doc * Skip wait L1 block confirmations after restart (#3662) * skip wait L1 block confirmations after restart * skip checking last batch sequenced in SC after restart * set default value of SequenceL1BlockConfirmations to 32 * set default value of SequenceL1BlockConfirmations to 2 for debug/test * fix config_test * fix doc --- config/config_test.go | 4 + config/default.go | 1 + .../environments/local/local.node.config.toml | 1 + docs/config-file/node-config-doc.html | 2 +- docs/config-file/node-config-doc.md | 15 +++ docs/config-file/node-config-schema.json | 5 + sequencesender/config.go | 2 + sequencesender/sequencesender.go | 112 +++++++++++++----- sequencesender/sequencesender_test.go | 12 +- test/config/debug.node.config.toml | 1 + test/config/test.node.config.toml | 1 + 11 files changed, 118 insertions(+), 38 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 0d6c93ea33..6ba1131a3b 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -193,6 +193,10 @@ func Test_Defaults(t *testing.T) { path: "SequenceSender.GasOffset", expectedValue: uint64(80000), }, + { + path: "SequenceSender.SequenceL1BlockConfirmations", + expectedValue: uint64(32), + }, { path: "Etherman.URL", expectedValue: "http://localhost:8545", diff --git a/config/default.go b/config/default.go index 970ba020ba..0b73699125 100644 --- a/config/default.go +++ b/config/default.go @@ -170,6 +170,7 @@ WaitPeriodSendSequence = "5s" LastBatchVirtualizationTimeMaxWaitPeriod = "5s" L1BlockTimestampMargin = "30s" MaxTxSizeForL1 = 131072 +SequenceL1BlockConfirmations = 32 L2Coinbase = "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266" PrivateKey = {Path = "/pk/sequencer.keystore", Password = "testonly"} GasOffset = 80000 diff --git a/config/environments/local/local.node.config.toml b/config/environments/local/local.node.config.toml index 376465b3c8..11077845ca 100644 --- a/config/environments/local/local.node.config.toml +++ b/config/environments/local/local.node.config.toml @@ -119,6 +119,7 @@ WaitPeriodSendSequence = "5s" LastBatchVirtualizationTimeMaxWaitPeriod = "5s" L1BlockTimestampMargin = "30s" MaxTxSizeForL1 = 131072 +SequenceL1BlockConfirmations = 32 L2Coinbase = "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266" PrivateKey = {Path = "/pk/sequencer.keystore", Password = "testonly"} diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 35d4551dce..79d14baf44 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -62,7 +62,7 @@
"300ms"
 

Default: "30s"Type: string

L1BlockTimestampMargin is the time difference (margin) that must exists between last L1 block and last L2 block in the sequence before
to send the sequence to L1. If the difference is lower than this value then sequencesender will wait until the difference is equal or greater


Examples:

"1m"
 
"300ms"
-

Default: 131072Type: integer

MaxTxSizeForL1 is the maximum size a single transaction can have. This field has
non-trivial consequences: larger transactions than 128KB are significantly harder and
more expensive to propagate; larger transactions also take more resources
to validate whether they fit into the pool or not.


Type: array of integer

SenderAddress defines which private key the eth tx manager needs to use
to sign the L1 txs

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


Default: "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266"Type: array of integer

L2Coinbase defines which address is going to receive the fees

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


PrivateKey defines all the key store files that are going to be read in order to provide the private keys to sign the L1 txs
Default: "/pk/sequencer.keystore"Type: string

Path is the file path for the key store file


Default: "testonly"Type: string

Password is the password to decrypt the key store file


Default: 0Type: integer

Batch number where there is a forkid change (fork upgrade)


Default: 80000Type: integer

GasOffset is the amount of gas to be added to the gas estimation in order
to provide an amount that is higher than the estimated one. This is used
to avoid the TX getting reverted in case something has changed in the network
state after the estimation which can cause the TX to require more gas to be
executed.

ex:
gas estimation: 1000
gas offset: 100
final gas: 1100


Configuration of the aggregator service
Default: "0.0.0.0"Type: string

Host for the grpc server


Default: 50081Type: integer

Port for the grpc server


Default: "5s"Type: string

RetryTime is the time the aggregator main loop sleeps if there are no proofs to aggregate
or batches to generate proofs. It is also used in the isSynced loop


Examples:

"1m"
+

Default: 131072Type: integer

MaxTxSizeForL1 is the maximum size a single transaction can have. This field has
non-trivial consequences: larger transactions than 128KB are significantly harder and
more expensive to propagate; larger transactions also take more resources
to validate whether they fit into the pool or not.


Type: array of integer

SenderAddress defines which private key the eth tx manager needs to use
to sign the L1 txs

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


Default: "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266"Type: array of integer

L2Coinbase defines which address is going to receive the fees

Must contain a minimum of 20 items

Must contain a maximum of 20 items

Each item of this array must be:


PrivateKey defines all the key store files that are going to be read in order to provide the private keys to sign the L1 txs
Default: "/pk/sequencer.keystore"Type: string

Path is the file path for the key store file


Default: "testonly"Type: string

Password is the password to decrypt the key store file


Default: 0Type: integer

Batch number where there is a forkid change (fork upgrade)


Default: 80000Type: integer

GasOffset is the amount of gas to be added to the gas estimation in order
to provide an amount that is higher than the estimated one. This is used
to avoid the TX getting reverted in case something has changed in the network
state after the estimation which can cause the TX to require more gas to be
executed.

ex:
gas estimation: 1000
gas offset: 100
final gas: 1100


Default: 32Type: integer

SequenceL1BlockConfirmations is number of blocks to consider a sequence sent to L1 as final


Configuration of the aggregator service
Default: "0.0.0.0"Type: string

Host for the grpc server


Default: 50081Type: integer

Port for the grpc server


Default: "5s"Type: string

RetryTime is the time the aggregator main loop sleeps if there are no proofs to aggregate
or batches to generate proofs. It is also used in the isSynced loop


Examples:

"1m"
 
"300ms"
 

Default: "1m30s"Type: string

VerifyProofInterval is the interval of time to verify/send an proof in L1


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index cff56349af..afe68fdc2f 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -2640,6 +2640,7 @@ UpgradeEtrogBatchNumber=0
 | - [PrivateKey](#SequenceSender_PrivateKey )                                                             | No      | object           | No         | -          | PrivateKey defines all the key store files that are going
to be read in order to provide the private keys to sign the L1 txs | | - [ForkUpgradeBatchNumber](#SequenceSender_ForkUpgradeBatchNumber ) | No | integer | No | - | Batch number where there is a forkid change (fork upgrade) | | - [GasOffset](#SequenceSender_GasOffset ) | No | integer | No | - | GasOffset is the amount of gas to be added to the gas estimation in order
to provide an amount that is higher than the estimated one. This is used
to avoid the TX getting reverted in case something has changed in the network
state after the estimation which can cause the TX to require more gas to be
executed.

ex:
gas estimation: 1000
gas offset: 100
final gas: 1100 | +| - [SequenceL1BlockConfirmations](#SequenceSender_SequenceL1BlockConfirmations ) | No | integer | No | - | SequenceL1BlockConfirmations is number of blocks to consider a sequence sent to L1 as final | ### 11.1. `SequenceSender.WaitPeriodSendSequence` @@ -2834,6 +2835,20 @@ final gas: 1100 GasOffset=80000 ``` +### 11.10. `SequenceSender.SequenceL1BlockConfirmations` + +**Type:** : `integer` + +**Default:** `32` + +**Description:** SequenceL1BlockConfirmations is number of blocks to consider a sequence sent to L1 as final + +**Example setting the default value** (32): +``` +[SequenceSender] +SequenceL1BlockConfirmations=32 +``` + ## 12. `[Aggregator]` **Type:** : `object` diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json index d70f2d44fc..5096caac0e 100644 --- a/docs/config-file/node-config-schema.json +++ b/docs/config-file/node-config-schema.json @@ -1091,6 +1091,11 @@ "type": "integer", "description": "GasOffset is the amount of gas to be added to the gas estimation in order\nto provide an amount that is higher than the estimated one. This is used\nto avoid the TX getting reverted in case something has changed in the network\nstate after the estimation which can cause the TX to require more gas to be\nexecuted.\n\nex:\ngas estimation: 1000\ngas offset: 100\nfinal gas: 1100", "default": 80000 + }, + "SequenceL1BlockConfirmations": { + "type": "integer", + "description": "SequenceL1BlockConfirmations is number of blocks to consider a sequence sent to L1 as final", + "default": 32 } }, "additionalProperties": false, diff --git a/sequencesender/config.go b/sequencesender/config.go index 56da9f14ab..ca6881fab8 100644 --- a/sequencesender/config.go +++ b/sequencesender/config.go @@ -41,4 +41,6 @@ type Config struct { // gas offset: 100 // final gas: 1100 GasOffset uint64 `mapstructure:"GasOffset"` + // SequenceL1BlockConfirmations is number of blocks to consider a sequence sent to L1 as final + SequenceL1BlockConfirmations uint64 `mapstructure:"SequenceL1BlockConfirmations"` } diff --git a/sequencesender/sequencesender.go b/sequencesender/sequencesender.go index e07c5e379c..81b2c68fc6 100644 --- a/sequencesender/sequencesender.go +++ b/sequencesender/sequencesender.go @@ -12,6 +12,7 @@ import ( "github.com/0xPolygonHermez/zkevm-node/event" "github.com/0xPolygonHermez/zkevm-node/log" "github.com/0xPolygonHermez/zkevm-node/state" + "github.com/ethereum/go-ethereum/common" ethTypes "github.com/ethereum/go-ethereum/core/types" "github.com/jackc/pgx/v4" ) @@ -21,6 +22,7 @@ const ( monitoredIDFormat = "sequence-from-%v-to-%v" retriesSanityCheck = 8 waitRetrySanityCheck = 15 * time.Second + waitRetryGetL1Block = 2 * time.Second ) var ( @@ -36,11 +38,13 @@ var ( // SequenceSender represents a sequence sender type SequenceSender struct { - cfg Config - state stateInterface - ethTxManager ethTxManager - etherman etherman - eventLog *event.EventLog + cfg Config + state stateInterface + ethTxManager ethTxManager + etherman etherman + eventLog *event.EventLog + lastSequenceInitialBatch uint64 + lastSequenceEndBatch uint64 } // New inits sequence sender @@ -63,7 +67,7 @@ func (s *SequenceSender) Start(ctx context.Context) { // marginTimeElapsed checks if the time between currentTime and l2BlockTimestamp is greater than timeMargin. // If it's greater returns true, otherwise it returns false and the waitTime needed to achieve this timeMargin -func (s *SequenceSender) marginTimeElapsed(ctx context.Context, l2BlockTimestamp uint64, currentTime uint64, timeMargin int64) (bool, int64) { +func (s *SequenceSender) marginTimeElapsed(l2BlockTimestamp uint64, currentTime uint64, timeMargin int64) (bool, int64) { // Check the time difference between L2 block and currentTime var timeDiff int64 if l2BlockTimestamp >= currentTime { @@ -91,7 +95,58 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { retry := false // process monitored sequences before starting a next cycle s.ethTxManager.ProcessPendingMonitoredTxs(ctx, ethTxManagerOwner, func(result ethtxmanager.MonitoredTxResult, dbTx pgx.Tx) { - if result.Status == ethtxmanager.MonitoredTxStatusFailed { + if result.Status == ethtxmanager.MonitoredTxStatusConfirmed { + if len(result.Txs) > 0 { + var txL1BlockNumber uint64 + var txHash common.Hash + receiptFound := false + for _, tx := range result.Txs { + if tx.Receipt != nil { + txL1BlockNumber = tx.Receipt.BlockNumber.Uint64() + txHash = tx.Tx.Hash() + receiptFound = true + break + } + } + + if !receiptFound { + s.halt(ctx, fmt.Errorf("monitored tx %s for sequence [%d-%d] is confirmed but doesn't have a receipt", result.ID, s.lastSequenceInitialBatch, s.lastSequenceEndBatch)) + } + + // wait L1 confirmation blocks + log.Infof("waiting %d L1 block confirmations for sequence [%d-%d], L1 block: %d, tx: %s", + s.cfg.SequenceL1BlockConfirmations, s.lastSequenceInitialBatch, s.lastSequenceEndBatch, txL1BlockNumber, txHash) + for { + lastL1BlockHeader, err := s.etherman.GetLatestBlockHeader(ctx) + if err != nil { + log.Errorf("failed to get last L1 block number, err: %v", err) + } else { + lastL1BlockNumber := lastL1BlockHeader.Number.Uint64() + + if lastL1BlockNumber >= txL1BlockNumber+s.cfg.SequenceL1BlockConfirmations { + log.Infof("continuing, last L1 block: %d", lastL1BlockNumber) + break + } + } + time.Sleep(waitRetryGetL1Block) + } + + lastSCBatchNum, err := s.etherman.GetLatestBatchNumber() + if err != nil { + log.Warnf("failed to get from the SC last sequenced batch number, err: %v", err) + return + } + + // If it's the first time we call that function after the restart of the sequence-sender (lastSequenceBatch is 0) and we are having the + // confirmation of a pending L1 tx sent before the sequence-sender was restarted, we don't know which batch was the last sequenced. + // Therefore we cannot compare the last sequenced batch in the SC with the last sequenced from sequence-sender. We skip this check + if s.lastSequenceEndBatch != 0 && (lastSCBatchNum != s.lastSequenceEndBatch) { + s.halt(ctx, fmt.Errorf("last sequenced batch from SC %d doesn't match last sequenced batch sent %d", lastSCBatchNum, s.lastSequenceEndBatch)) + } + } else { + s.halt(ctx, fmt.Errorf("monitored tx %s for sequence [%d-%d] doesn't have transactions to be checked", result.ID, s.lastSequenceInitialBatch, s.lastSequenceEndBatch)) + } + } else { // Monitored tx is failed retry = true mTxResultLogger := ethtxmanager.CreateMonitoredTxResultLogger(ethTxManagerOwner, result) mTxResultLogger.Error("failed to send sequence, TODO: review this fatal and define what to do in this case") @@ -102,13 +157,12 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { return } - // Check if synchronizer is up to date - synced, err := s.isSynced(ctx, retriesSanityCheck, waitRetrySanityCheck) + sanityCheckOk, err := s.sanityCheck(ctx, retriesSanityCheck, waitRetrySanityCheck) if err != nil { s.halt(ctx, err) } - if !synced { - log.Info("wait virtual state to be synced...") + if !sanityCheckOk { + log.Info("sanity check failed, retrying...") time.Sleep(5 * time.Second) // nolint:gomnd return } @@ -126,7 +180,7 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { return } - lastVirtualBatchNum, err := s.state.GetLastVirtualBatchNum(ctx, nil) + lastVirtualBatchNum, err := s.etherman.GetLatestBatchNumber() if err != nil { log.Errorf("failed to get last virtual batch num, err: %v", err) return @@ -153,7 +207,7 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { return } - elapsed, waitTime := s.marginTimeElapsed(ctx, lastL2BlockTimestamp, lastL1BlockHeader.Time, timeMargin) + elapsed, waitTime := s.marginTimeElapsed(lastL2BlockTimestamp, lastL1BlockHeader.Time, timeMargin) if !elapsed { log.Infof("waiting at least %d seconds to send sequences, time difference between last L1 block %d (ts: %d) and last L2 block %d (ts: %d) in the sequence is lower than %d seconds", @@ -170,7 +224,7 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { for { currentTime := uint64(time.Now().Unix()) - elapsed, waitTime := s.marginTimeElapsed(ctx, lastL2BlockTimestamp, currentTime, timeMargin) + elapsed, waitTime := s.marginTimeElapsed(lastL2BlockTimestamp, currentTime, timeMargin) // Wait if the time difference is less than timeMargin (L1BlockTimestampMargin) if !elapsed { @@ -200,13 +254,16 @@ func (s *SequenceSender) tryToSendSequence(ctx context.Context) { mTxLogger.Errorf("error to add sequences tx to eth tx manager: ", err) return } + + s.lastSequenceInitialBatch = sequences[0].BatchNumber + s.lastSequenceEndBatch = lastSequence.BatchNumber } // getSequencesToSend generates an array of sequences to be send to L1. // If the array is empty, it doesn't necessarily mean that there are no sequences to be sent, // it could be that it's not worth it to do so yet. func (s *SequenceSender) getSequencesToSend(ctx context.Context) ([]types.Sequence, error) { - lastVirtualBatchNum, err := s.state.GetLastVirtualBatchNum(ctx, nil) + lastVirtualBatchNum, err := s.etherman.GetLatestBatchNumber() if err != nil { return nil, fmt.Errorf("failed to get last virtual batch num, err: %v", err) } @@ -297,7 +354,7 @@ func (s *SequenceSender) getSequencesToSend(ctx context.Context) ([]types.Sequen } if err != nil { log.Infof("Handling estimage gas send sequence error: %v", err) - sequences, err = s.handleEstimateGasSendSequenceErr(ctx, sequences, currentBatchNumToSequence, err) + sequences, err = s.handleEstimateGasSendSequenceErr(sequences, currentBatchNumToSequence, err) if sequences != nil { if len(sequences) > 0 { // Handling the error gracefully, re-processing the sequence as a sanity check @@ -347,12 +404,7 @@ func (s *SequenceSender) getSequencesToSend(ctx context.Context) ([]types.Sequen // nil, error: impossible to handle gracefully // sequence, nil: handled gracefully. Potentially manipulating the sequences // nil, nil: a situation that requires waiting -func (s *SequenceSender) handleEstimateGasSendSequenceErr( - ctx context.Context, - sequences []types.Sequence, - currentBatchNumToSequence uint64, - err error, -) ([]types.Sequence, error) { +func (s *SequenceSender) handleEstimateGasSendSequenceErr(sequences []types.Sequence, currentBatchNumToSequence uint64, err error) ([]types.Sequence, error) { // Insufficient allowance if errors.Is(err, ethman.ErrInsufficientAllowance) { return nil, err @@ -415,7 +467,7 @@ func isDataForEthTxTooBig(err error) bool { errors.Is(err, ethman.ErrContentLengthTooLarge) } -func (s *SequenceSender) isSynced(ctx context.Context, retries int, waitRetry time.Duration) (bool, error) { +func (s *SequenceSender) sanityCheck(ctx context.Context, retries int, waitRetry time.Duration) (bool, error) { lastVirtualBatchNum, err := s.state.GetLastVirtualBatchNum(ctx, nil) if err != nil && err != state.ErrNotFound { log.Warnf("failed to get last virtual batch number, err: %v", err) @@ -434,10 +486,8 @@ func (s *SequenceSender) isSynced(ctx context.Context, retries int, waitRetry ti return false, nil } - if lastVirtualBatchNum < lastSCBatchNum { - log.Infof("waiting for the state to be synced, last virtual batch: %d, last SC sequenced batch: %d", lastVirtualBatchNum, lastSCBatchNum) - return false, nil - } else if lastVirtualBatchNum > lastSCBatchNum { // Sanity check: virtual batch number cannot be greater than last batch sequenced in the SC + // Sanity check: virtual batch number cannot be greater than last batch sequenced in the SC + if lastVirtualBatchNum > lastSCBatchNum { // we will retry some times to check that really the last sequenced batch in the SC is lower that the las virtual batch log.Warnf("last virtual batch %d is greater than last SC sequenced batch %d, retrying...", lastVirtualBatchNum, lastSCBatchNum) for i := 0; i < retries; i++ { @@ -457,13 +507,13 @@ func (s *SequenceSender) isSynced(ctx context.Context, retries int, waitRetry ti log.Infof("last virtual batch %d is equal to last SC sequenced batch %d, continuing...", lastVirtualBatchNum, lastSCBatchNum) } - // At this point lastVirtualBatchNum = lastEthBatchNum. Check trusted batches - if lastTrustedBatchClosed.BatchNumber >= lastVirtualBatchNum { - return true, nil - } else { // Sanity check: virtual batch number cannot be greater than last trusted batch closed + // Sanity check: virtual batch number cannot be greater than last trusted batch closed + if lastTrustedBatchClosed.BatchNumber < lastVirtualBatchNum { log.Errorf("last virtual batch %d is greater than last trusted batch closed %d", lastVirtualBatchNum, lastTrustedBatchClosed.BatchNumber) return false, ErrSyncVirtualGreaterTrusted } + + return true, nil } // halt halts the SequenceSender diff --git a/sequencesender/sequencesender_test.go b/sequencesender/sequencesender_test.go index 71554eddd3..fbe2b3e2f0 100644 --- a/sequencesender/sequencesender_test.go +++ b/sequencesender/sequencesender_test.go @@ -35,7 +35,7 @@ func TestIsSynced(t *testing.T) { testCases := []IsSyncedTestCase{ { - name: "is synced", + name: "sanity check ok", lastVirtualBatchNum: 10, lastTrustedBatchClosed: 12, lastSCBatchNum: []uint64{10}, @@ -43,11 +43,11 @@ func TestIsSynced(t *testing.T) { err: nil, }, { - name: "not synced", + name: "sanity check ok", lastVirtualBatchNum: 9, lastTrustedBatchClosed: 12, lastSCBatchNum: []uint64{10}, - expectedResult: false, + expectedResult: true, err: nil, }, { @@ -67,7 +67,7 @@ func TestIsSynced(t *testing.T) { err: ErrSyncVirtualGreaterSequenced, }, { - name: "is synced, sc sequenced retries", + name: "sanity check ok: sc sequenced retries", lastVirtualBatchNum: 11, lastTrustedBatchClosed: 12, lastSCBatchNum: []uint64{10, 10, 11}, @@ -75,7 +75,7 @@ func TestIsSynced(t *testing.T) { err: nil, }, { - name: "is synced, sc sequenced retries (last)", + name: "sanity check ok: sc sequenced retries (last)", lastVirtualBatchNum: 11, lastTrustedBatchClosed: 12, lastSCBatchNum: []uint64{10, 10, 10, 11}, @@ -134,7 +134,7 @@ func TestIsSynced(t *testing.T) { } } - synced, err := ssender.isSynced(context.Background(), retries, waitRetry) + synced, err := ssender.sanityCheck(context.Background(), retries, waitRetry) assert.EqualValues(t, tc.expectedResult, synced) assert.EqualValues(t, tc.err, err) diff --git a/test/config/debug.node.config.toml b/test/config/debug.node.config.toml index 698164c866..68f7dd17ce 100644 --- a/test/config/debug.node.config.toml +++ b/test/config/debug.node.config.toml @@ -119,6 +119,7 @@ WaitPeriodSendSequence = "15s" LastBatchVirtualizationTimeMaxWaitPeriod = "10s" L1BlockTimestampMargin = "5s" MaxTxSizeForL1 = 131072 +SequenceL1BlockConfirmations = 2 L2Coinbase = "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266" PrivateKey = {Path = "./test/sequencer.keystore", Password = "testonly"} diff --git a/test/config/test.node.config.toml b/test/config/test.node.config.toml index 4f743a7b0a..1df2fc1882 100644 --- a/test/config/test.node.config.toml +++ b/test/config/test.node.config.toml @@ -135,6 +135,7 @@ WaitPeriodSendSequence = "15s" LastBatchVirtualizationTimeMaxWaitPeriod = "10s" L1BlockTimestampMargin = "5s" MaxTxSizeForL1 = 131072 +SequenceL1BlockConfirmations = 2 L2Coinbase = "0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266" PrivateKey = {Path = "/pk/sequencer.keystore", Password = "testonly"} [SequenceSender.StreamClient] From 8361140cb50e7164481c03a955579f47dbf303a3 Mon Sep 17 00:00:00 2001 From: Thiago Coimbra Lemos Date: Wed, 5 Jun 2024 10:17:03 -0300 Subject: [PATCH 18/22] remove db tx from RPC (#3648) (#3683) --- jsonrpc/dbtxmanager.go | 41 -- jsonrpc/dbtxmanager_test.go | 99 --- jsonrpc/endpoints_debug.go | 249 ++++--- jsonrpc/endpoints_eth.go | 818 +++++++++++------------ jsonrpc/endpoints_eth_test.go | 1106 ++++--------------------------- jsonrpc/endpoints_zkevm.go | 713 ++++++++++---------- jsonrpc/endpoints_zkevm_test.go | 707 +++----------------- jsonrpc/mocks/mock_dbtx.go | 350 ---------- jsonrpc/server_test.go | 31 +- jsonrpc/types/codec_test.go | 66 +- test/Makefile | 1 - 11 files changed, 1133 insertions(+), 3048 deletions(-) delete mode 100644 jsonrpc/dbtxmanager.go delete mode 100644 jsonrpc/dbtxmanager_test.go delete mode 100644 jsonrpc/mocks/mock_dbtx.go diff --git a/jsonrpc/dbtxmanager.go b/jsonrpc/dbtxmanager.go deleted file mode 100644 index bb073d0369..0000000000 --- a/jsonrpc/dbtxmanager.go +++ /dev/null @@ -1,41 +0,0 @@ -package jsonrpc - -import ( - "context" - - "github.com/0xPolygonHermez/zkevm-node/jsonrpc/types" - "github.com/jackc/pgx/v4" -) - -// DBTxManager allows to do scopped DB txs -type DBTxManager struct{} - -// DBTxScopedFn function to do scopped DB txs -type DBTxScopedFn func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) - -// DBTxer interface to begin DB txs -type DBTxer interface { - BeginStateTransaction(ctx context.Context) (pgx.Tx, error) -} - -// NewDbTxScope function to initiate DB scopped txs -func (f *DBTxManager) NewDbTxScope(db DBTxer, scopedFn DBTxScopedFn) (interface{}, types.Error) { - ctx := context.Background() - dbTx, err := db.BeginStateTransaction(ctx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to connect to the state", err, true) - } - - v, rpcErr := scopedFn(ctx, dbTx) - if rpcErr != nil { - if txErr := dbTx.Rollback(context.Background()); txErr != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to rollback db transaction", txErr, true) - } - return v, rpcErr - } - - if txErr := dbTx.Commit(context.Background()); txErr != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to commit db transaction", txErr, true) - } - return v, rpcErr -} diff --git a/jsonrpc/dbtxmanager_test.go b/jsonrpc/dbtxmanager_test.go deleted file mode 100644 index b3dba72625..0000000000 --- a/jsonrpc/dbtxmanager_test.go +++ /dev/null @@ -1,99 +0,0 @@ -package jsonrpc - -import ( - "context" - "errors" - "testing" - - "github.com/0xPolygonHermez/zkevm-node/jsonrpc/mocks" - "github.com/0xPolygonHermez/zkevm-node/jsonrpc/types" - "github.com/jackc/pgx/v4" - "github.com/stretchr/testify/assert" -) - -func TestNewDbTxScope(t *testing.T) { - type testCase struct { - Name string - Fn DBTxScopedFn - ExpectedResult interface{} - ExpectedError types.Error - SetupMocks func(s *mocks.StateMock, d *mocks.DBTxMock) - } - - testCases := []testCase{ - { - Name: "Run scoped func commits DB tx", - Fn: func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return 1, nil - }, - ExpectedResult: 1, - ExpectedError: nil, - SetupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock) { - d.On("Commit", context.Background()).Return(nil).Once() - s.On("BeginStateTransaction", context.Background()).Return(d, nil).Once() - }, - }, - { - Name: "Run scoped func rollbacks DB tx", - Fn: func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return nil, types.NewRPCError(types.DefaultErrorCode, "func returned an error") - }, - ExpectedResult: nil, - ExpectedError: types.NewRPCError(types.DefaultErrorCode, "func returned an error"), - SetupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock) { - d.On("Rollback", context.Background()).Return(nil).Once() - s.On("BeginStateTransaction", context.Background()).Return(d, nil).Once() - }, - }, - { - Name: "Run scoped func but fails create a db tx", - Fn: func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return nil, nil - }, - ExpectedResult: nil, - ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to connect to the state"), - SetupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock) { - s.On("BeginStateTransaction", context.Background()).Return(nil, errors.New("failed to create db tx")).Once() - }, - }, - { - Name: "Run scoped func but fails to commit DB tx", - Fn: func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return 1, nil - }, - ExpectedResult: nil, - ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to commit db transaction"), - SetupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock) { - d.On("Commit", context.Background()).Return(errors.New("failed to commit db tx")).Once() - s.On("BeginStateTransaction", context.Background()).Return(d, nil).Once() - }, - }, - { - Name: "Run scoped func but fails to rollbacks DB tx", - Fn: func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return nil, types.NewRPCError(types.DefaultErrorCode, "func returned an error") - }, - ExpectedResult: nil, - ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to rollback db transaction"), - SetupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock) { - d.On("Rollback", context.Background()).Return(errors.New("failed to rollback db tx")).Once() - s.On("BeginStateTransaction", context.Background()).Return(d, nil).Once() - }, - }, - } - - dbTxManager := DBTxManager{} - s := mocks.NewStateMock(t) - d := mocks.NewDBTxMock(t) - - for _, testCase := range testCases { - t.Run(testCase.Name, func(t *testing.T) { - tc := testCase - tc.SetupMocks(s, d) - - result, err := dbTxManager.NewDbTxScope(s, tc.Fn) - assert.Equal(t, tc.ExpectedResult, result) - assert.Equal(t, tc.ExpectedError, err) - }) - } -} diff --git a/jsonrpc/endpoints_debug.go b/jsonrpc/endpoints_debug.go index a91cd924da..2309db7c5c 100644 --- a/jsonrpc/endpoints_debug.go +++ b/jsonrpc/endpoints_debug.go @@ -31,7 +31,6 @@ type DebugEndpoints struct { cfg Config state types.StateInterface etherman types.EthermanInterface - txMan DBTxManager } // NewDebugEndpoints returns DebugEndpoints @@ -64,54 +63,51 @@ type traceBatchTransactionResponse struct { // TraceTransaction creates a response for debug_traceTransaction request. // See https://geth.ethereum.org/docs/interacting-with-geth/rpc/ns-debug#debugtracetransaction func (d *DebugEndpoints) TraceTransaction(hash types.ArgHash, cfg *traceConfig) (interface{}, types.Error) { - return d.txMan.NewDbTxScope(d.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return d.buildTraceTransaction(ctx, hash.Hash(), cfg, dbTx) - }) + ctx := context.Background() + return d.buildTraceTransaction(ctx, hash.Hash(), cfg, nil) } // TraceBlockByNumber creates a response for debug_traceBlockByNumber request. // See https://geth.ethereum.org/docs/interacting-with-geth/rpc/ns-debug#debugtraceblockbynumber func (d *DebugEndpoints) TraceBlockByNumber(number types.BlockNumber, cfg *traceConfig) (interface{}, types.Error) { - return d.txMan.NewDbTxScope(d.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, d.state, d.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, d.state, d.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - block, err := d.state.GetL2BlockByNumber(ctx, blockNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("block #%d not found", blockNumber)) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by number", err, true) - } + block, err := d.state.GetL2BlockByNumber(ctx, blockNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("block #%d not found", blockNumber)) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by number", err, true) + } - traces, rpcErr := d.buildTraceBlock(ctx, block.Transactions(), cfg, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + traces, rpcErr := d.buildTraceBlock(ctx, block.Transactions(), cfg, nil) + if rpcErr != nil { + return nil, rpcErr + } - return traces, nil - }) + return traces, nil } // TraceBlockByHash creates a response for debug_traceBlockByHash request. // See https://geth.ethereum.org/docs/interacting-with-geth/rpc/ns-debug#debugtraceblockbyhash func (d *DebugEndpoints) TraceBlockByHash(hash types.ArgHash, cfg *traceConfig) (interface{}, types.Error) { - return d.txMan.NewDbTxScope(d.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - block, err := d.state.GetL2BlockByHash(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("block %s not found", hash.Hash().String())) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash", err, true) - } + ctx := context.Background() + block, err := d.state.GetL2BlockByHash(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("block %s not found", hash.Hash().String())) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash", err, true) + } - traces, rpcErr := d.buildTraceBlock(ctx, block.Transactions(), cfg, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + traces, rpcErr := d.buildTraceBlock(ctx, block.Transactions(), cfg, nil) + if rpcErr != nil { + return nil, rpcErr + } - return traces, nil - }) + return traces, nil } // TraceBatchByNumber creates a response for debug_traceBatchByNumber request. @@ -144,113 +140,112 @@ func (d *DebugEndpoints) TraceBatchByNumber(httpRequest *http.Request, number ty // how many txs it will process in parallel. const bufferSize = 10 - return d.txMan.NewDbTxScope(d.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - batchNumber, rpcErr := number.GetNumericBatchNumber(ctx, d.state, d.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + batchNumber, rpcErr := number.GetNumericBatchNumber(ctx, d.state, d.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - batch, err := d.state.GetBatchByNumber(ctx, batchNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("batch #%d not found", batchNumber)) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get batch by number", err, true) - } + batch, err := d.state.GetBatchByNumber(ctx, batchNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, types.NewRPCError(types.DefaultErrorCode, fmt.Sprintf("batch #%d not found", batchNumber)) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get batch by number", err, true) + } - txs, _, err := d.state.GetTransactionsByBatchNumber(ctx, batch.BatchNumber, dbTx) - if !errors.Is(err, state.ErrNotFound) && err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch txs from state by number %v to create the traces", batchNumber), err, true) - } + txs, _, err := d.state.GetTransactionsByBatchNumber(ctx, batch.BatchNumber, nil) + if !errors.Is(err, state.ErrNotFound) && err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch txs from state by number %v to create the traces", batchNumber), err, true) + } - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := d.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v to get trace", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := d.state.GetTransactionReceipt(ctx, tx.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v to get trace", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - requests := make(chan (ethTypes.Receipt), bufferSize) - - mu := &sync.Mutex{} - wg := sync.WaitGroup{} - wg.Add(len(receipts)) - responses := make([]traceResponse, 0, len(receipts)) - - // gets the trace from the jRPC and adds it to the responses - loadTraceByTxHash := func(d *DebugEndpoints, receipt ethTypes.Receipt, cfg *traceConfig) { - response := traceResponse{ - blockNumber: receipt.BlockNumber.Uint64(), - txIndex: uint64(receipt.TransactionIndex), - txHash: receipt.TxHash, - } - - defer wg.Done() - trace, err := d.TraceTransaction(types.ArgHash(receipt.TxHash), cfg) - if err != nil { - err := fmt.Errorf("failed to get tx trace for tx %v, err: %w", receipt.TxHash.String(), err) - log.Errorf(err.Error()) - response.err = err - } else { - response.trace = trace - } - - // add to the responses - mu.Lock() - defer mu.Unlock() - responses = append(responses, response) + requests := make(chan (ethTypes.Receipt), bufferSize) + + mu := &sync.Mutex{} + wg := sync.WaitGroup{} + wg.Add(len(receipts)) + responses := make([]traceResponse, 0, len(receipts)) + + // gets the trace from the jRPC and adds it to the responses + loadTraceByTxHash := func(d *DebugEndpoints, receipt ethTypes.Receipt, cfg *traceConfig) { + response := traceResponse{ + blockNumber: receipt.BlockNumber.Uint64(), + txIndex: uint64(receipt.TransactionIndex), + txHash: receipt.TxHash, } - // goes through the buffer and loads the trace - // by all the transactions added in the buffer - // then add the results to the responses map - go func() { - index := uint(0) - for req := range requests { - go loadTraceByTxHash(d, req, cfg) - index++ - } - }() - - // add receipts to the buffer - for _, receipt := range receipts { - requests <- receipt + defer wg.Done() + trace, err := d.TraceTransaction(types.ArgHash(receipt.TxHash), cfg) + if err != nil { + err := fmt.Errorf("failed to get tx trace for tx %v, err: %w", receipt.TxHash.String(), err) + log.Errorf(err.Error()) + response.err = err + } else { + response.trace = trace } - // wait the traces to be loaded - if waitTimeout(&wg, d.cfg.ReadTimeout.Duration) { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("failed to get traces for batch %v: timeout reached", batchNumber), nil, true) + // add to the responses + mu.Lock() + defer mu.Unlock() + responses = append(responses, response) + } + + // goes through the buffer and loads the trace + // by all the transactions added in the buffer + // then add the results to the responses map + go func() { + index := uint(0) + for req := range requests { + go loadTraceByTxHash(d, req, cfg) + index++ } + }() - close(requests) - - // since the txs are attached to a L2 Block and the L2 Block is - // the struct attached to the Batch, in order to always respond - // the traces in the same order, we need to order the transactions - // first by block number and then by tx index, so we can have something - // close to the txs being sorted by a tx index related to the batch - sort.Slice(responses, func(i, j int) bool { - if responses[i].txIndex != responses[j].txIndex { - return responses[i].txIndex < responses[j].txIndex - } - return responses[i].blockNumber < responses[j].blockNumber - }) + // add receipts to the buffer + for _, receipt := range receipts { + requests <- receipt + } + + // wait the traces to be loaded + if waitTimeout(&wg, d.cfg.ReadTimeout.Duration) { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("failed to get traces for batch %v: timeout reached", batchNumber), nil, true) + } + + close(requests) - // build the batch trace response array - traces := make([]traceBatchTransactionResponse, 0, len(receipts)) - for _, response := range responses { - if response.err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("failed to get traces for batch %v: failed to get trace for tx: %v, err: %v", batchNumber, response.txHash.String(), response.err.Error()), nil, true) - } - - traces = append(traces, traceBatchTransactionResponse{ - TxHash: response.txHash, - Result: response.trace, - }) + // since the txs are attached to a L2 Block and the L2 Block is + // the struct attached to the Batch, in order to always respond + // the traces in the same order, we need to order the transactions + // first by block number and then by tx index, so we can have something + // close to the txs being sorted by a tx index related to the batch + sort.Slice(responses, func(i, j int) bool { + if responses[i].txIndex != responses[j].txIndex { + return responses[i].txIndex < responses[j].txIndex } - return traces, nil + return responses[i].blockNumber < responses[j].blockNumber }) + + // build the batch trace response array + traces := make([]traceBatchTransactionResponse, 0, len(receipts)) + for _, response := range responses { + if response.err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("failed to get traces for batch %v: failed to get trace for tx: %v, err: %v", batchNumber, response.txHash.String(), response.err.Error()), nil, true) + } + + traces = append(traces, traceBatchTransactionResponse{ + TxHash: response.txHash, + Result: response.trace, + }) + } + return traces, nil } func (d *DebugEndpoints) buildTraceBlock(ctx context.Context, txs []*ethTypes.Transaction, cfg *traceConfig, dbTx pgx.Tx) (interface{}, types.Error) { diff --git a/jsonrpc/endpoints_eth.go b/jsonrpc/endpoints_eth.go index 8231cab78b..bcfb21e770 100644 --- a/jsonrpc/endpoints_eth.go +++ b/jsonrpc/endpoints_eth.go @@ -37,7 +37,6 @@ type EthEndpoints struct { state types.StateInterface etherman types.EthermanInterface storage storageInterface - txMan DBTxManager } // NewEthEndpoints creates an new instance of Eth @@ -50,14 +49,13 @@ func NewEthEndpoints(cfg Config, chainID uint64, p types.PoolInterface, s types. // BlockNumber returns current block number func (e *EthEndpoints) BlockNumber() (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - lastBlockNumber, err := e.state.GetLastL2BlockNumber(ctx, dbTx) - if err != nil { - return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state") - } + ctx := context.Background() + lastBlockNumber, err := e.state.GetLastL2BlockNumber(ctx, nil) + if err != nil { + return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state") + } - return hex.EncodeUint64(lastBlockNumber), nil - }) + return hex.EncodeUint64(lastBlockNumber), nil } // Call executes a new message call immediately and returns the value of @@ -65,62 +63,61 @@ func (e *EthEndpoints) BlockNumber() (interface{}, types.Error) { // Note, this function doesn't make any changes in the state/blockchain and is // useful to execute view/pure methods and retrieve values. func (e *EthEndpoints) Call(arg *types.TxArgs, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if arg == nil { - return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) - } - block, respErr := e.getBlockByArg(ctx, blockArg, dbTx) - if respErr != nil { - return nil, respErr - } - var blockToProcess *uint64 - if blockArg != nil { - blockNumArg := blockArg.Number() - if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { - blockToProcess = nil - } else { - n := block.NumberU64() - blockToProcess = &n - } - } - - // If the caller didn't supply the gas limit in the message, then we set it to maximum possible => block gas limit - if arg.Gas == nil || uint64(*arg.Gas) <= 0 { - header, err := e.state.GetL2BlockHeaderByNumber(ctx, block.NumberU64(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block header", err, true) - } - - gas := types.ArgUint64(header.GasLimit) - arg.Gas = &gas + ctx := context.Background() + if arg == nil { + return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) + } + block, respErr := e.getBlockByArg(ctx, blockArg, nil) + if respErr != nil { + return nil, respErr + } + var blockToProcess *uint64 + if blockArg != nil { + blockNumArg := blockArg.Number() + if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { + blockToProcess = nil + } else { + n := block.NumberU64() + blockToProcess = &n } + } - defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) - sender, tx, err := arg.ToTransaction(ctx, e.state, state.MaxTxGasLimit, block.Root(), defaultSenderAddress, dbTx) + // If the caller didn't supply the gas limit in the message, then we set it to maximum possible => block gas limit + if arg.Gas == nil || uint64(*arg.Gas) <= 0 { + header, err := e.state.GetL2BlockHeaderByNumber(ctx, block.NumberU64(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block header", err, true) } - result, err := e.state.ProcessUnsignedTransaction(ctx, tx, sender, blockToProcess, true, dbTx) - if err != nil { - errMsg := fmt.Sprintf("failed to execute the unsigned transaction: %v", err.Error()) - logError := !executor.IsROMOutOfCountersError(executor.RomErrorCode(err)) && !errors.Is(err, runtime.ErrOutOfGas) - return RPCErrorResponse(types.DefaultErrorCode, errMsg, nil, logError) - } + gas := types.ArgUint64(header.GasLimit) + arg.Gas = &gas + } - if result.Reverted() { - data := make([]byte, len(result.ReturnValue)) - copy(data, result.ReturnValue) - if len(data) == 0 { - return nil, types.NewRPCError(types.DefaultErrorCode, result.Err.Error()) - } - return nil, types.NewRPCErrorWithData(types.RevertedErrorCode, result.Err.Error(), data) - } else if result.Failed() { + defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) + sender, tx, err := arg.ToTransaction(ctx, e.state, state.MaxTxGasLimit, block.Root(), defaultSenderAddress, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) + } + + result, err := e.state.ProcessUnsignedTransaction(ctx, tx, sender, blockToProcess, true, nil) + if err != nil { + errMsg := fmt.Sprintf("failed to execute the unsigned transaction: %v", err.Error()) + logError := !executor.IsROMOutOfCountersError(executor.RomErrorCode(err)) && !errors.Is(err, runtime.ErrOutOfGas) + return RPCErrorResponse(types.DefaultErrorCode, errMsg, nil, logError) + } + + if result.Reverted() { + data := make([]byte, len(result.ReturnValue)) + copy(data, result.ReturnValue) + if len(data) == 0 { return nil, types.NewRPCError(types.DefaultErrorCode, result.Err.Error()) } + return nil, types.NewRPCErrorWithData(types.RevertedErrorCode, result.Err.Error(), data) + } else if result.Failed() { + return nil, types.NewRPCError(types.DefaultErrorCode, result.Err.Error()) + } - return types.ArgBytesPtr(result.ReturnValue), nil - }) + return types.ArgBytesPtr(result.ReturnValue), nil } // ChainId returns the chain id of the client @@ -161,46 +158,45 @@ func (e *EthEndpoints) getCoinbaseFromSequencerNode() (interface{}, types.Error) // used by the transaction, for a variety of reasons including EVM mechanics and // node performance. func (e *EthEndpoints) EstimateGas(arg *types.TxArgs, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if arg == nil { - return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) - } + ctx := context.Background() + if arg == nil { + return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) + } - block, respErr := e.getBlockByArg(ctx, blockArg, dbTx) - if respErr != nil { - return nil, respErr - } + block, respErr := e.getBlockByArg(ctx, blockArg, nil) + if respErr != nil { + return nil, respErr + } - var blockToProcess *uint64 - if blockArg != nil { - blockNumArg := blockArg.Number() - if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { - blockToProcess = nil - } else { - n := block.NumberU64() - blockToProcess = &n - } + var blockToProcess *uint64 + if blockArg != nil { + blockNumArg := blockArg.Number() + if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { + blockToProcess = nil + } else { + n := block.NumberU64() + blockToProcess = &n } + } - defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) - sender, tx, err := arg.ToTransaction(ctx, e.state, state.MaxTxGasLimit, block.Root(), defaultSenderAddress, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) - } + defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) + sender, tx, err := arg.ToTransaction(ctx, e.state, state.MaxTxGasLimit, block.Root(), defaultSenderAddress, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) + } - gasEstimation, returnValue, err := e.state.EstimateGas(tx, sender, blockToProcess, dbTx) - if errors.Is(err, runtime.ErrExecutionReverted) { - data := make([]byte, len(returnValue)) - copy(data, returnValue) - if len(data) == 0 { - return nil, types.NewRPCError(types.DefaultErrorCode, err.Error()) - } - return nil, types.NewRPCErrorWithData(types.RevertedErrorCode, err.Error(), data) - } else if err != nil { + gasEstimation, returnValue, err := e.state.EstimateGas(tx, sender, blockToProcess, nil) + if errors.Is(err, runtime.ErrExecutionReverted) { + data := make([]byte, len(returnValue)) + copy(data, returnValue) + if len(data) == 0 { return nil, types.NewRPCError(types.DefaultErrorCode, err.Error()) } - return hex.EncodeUint64(gasEstimation), nil - }) + return nil, types.NewRPCErrorWithData(types.RevertedErrorCode, err.Error(), data) + } else if err != nil { + return nil, types.NewRPCError(types.DefaultErrorCode, err.Error()) + } + return hex.EncodeUint64(gasEstimation), nil } // GasPrice returns the average gas price based on the last x blocks @@ -253,21 +249,20 @@ func (e *EthEndpoints) getHighestL2BlockFromTrustedNode() (interface{}, types.Er // GetBalance returns the account's balance at the referenced block func (e *EthEndpoints) GetBalance(address types.ArgAddress, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - block, rpcErr := e.getBlockByArg(ctx, blockArg, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + block, rpcErr := e.getBlockByArg(ctx, blockArg, nil) + if rpcErr != nil { + return nil, rpcErr + } - balance, err := e.state.GetBalance(ctx, address.Address(), block.Root()) - if errors.Is(err, state.ErrNotFound) { - return hex.EncodeUint64(0), nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get balance from state", err, true) - } + balance, err := e.state.GetBalance(ctx, address.Address(), block.Root()) + if errors.Is(err, state.ErrNotFound) { + return hex.EncodeUint64(0), nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get balance from state", err, true) + } - return hex.EncodeBig(balance), nil - }) + return hex.EncodeBig(balance), nil } func (e *EthEndpoints) getBlockByArg(ctx context.Context, blockArg *types.BlockNumberOrHash, dbTx pgx.Tx) (*state.L2Block, types.Error) { @@ -308,111 +303,108 @@ func (e *EthEndpoints) getBlockByArg(ctx context.Context, blockArg *types.BlockN // GetBlockByHash returns information about a block by hash func (e *EthEndpoints) GetBlockByHash(hash types.ArgHash, fullTx bool, includeExtraInfo *bool) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - l2Block, err := e.state.GetL2BlockByHash(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) - } - - txs := l2Block.Transactions() - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) - } + ctx := context.Background() + l2Block, err := e.state.GetL2BlockByHash(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) + } - rpcBlock, err := types.NewBlock(ctx, e.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, false, includeExtraInfo, dbTx) + txs := l2Block.Transactions() + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by hash %v", hash.Hash()), err, true) + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - return rpcBlock, nil - }) + rpcBlock, err := types.NewBlock(ctx, e.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, false, includeExtraInfo, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by hash %v", hash.Hash()), err, true) + } + + return rpcBlock, nil } // GetBlockByNumber returns information about a block by block number func (e *EthEndpoints) GetBlockByNumber(number types.BlockNumber, fullTx bool, includeExtraInfo *bool) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if number == types.PendingBlockNumber { - lastBlock, err := e.state.GetLastL2Block(ctx, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block", err, true) - } - l2Header := state.NewL2Header(ðTypes.Header{ - ParentHash: lastBlock.Hash(), - Number: big.NewInt(0).SetUint64(lastBlock.Number().Uint64() + 1), - TxHash: ethTypes.EmptyRootHash, - UncleHash: ethTypes.EmptyUncleHash, - }) - l2Block := state.NewL2BlockWithHeader(l2Header) - rpcBlock, err := types.NewBlock(ctx, e.state, nil, l2Block, nil, fullTx, false, includeExtraInfo, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "couldn't build the pending block response", err, true) - } - - // clean fields that are not available for pending block - rpcBlock.Hash = nil - rpcBlock.Miner = nil - rpcBlock.Nonce = nil - rpcBlock.TotalDifficulty = nil - - return rpcBlock, nil + ctx := context.Background() + if number == types.PendingBlockNumber { + lastBlock, err := e.state.GetLastL2Block(ctx, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block", err, true) } - var err error - blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr + l2Header := state.NewL2Header(ðTypes.Header{ + ParentHash: lastBlock.Hash(), + Number: big.NewInt(0).SetUint64(lastBlock.Number().Uint64() + 1), + TxHash: ethTypes.EmptyRootHash, + UncleHash: ethTypes.EmptyUncleHash, + }) + l2Block := state.NewL2BlockWithHeader(l2Header) + rpcBlock, err := types.NewBlock(ctx, e.state, nil, l2Block, nil, fullTx, false, includeExtraInfo, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "couldn't build the pending block response", err, true) } - l2Block, err := e.state.GetL2BlockByNumber(ctx, blockNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load block from state by number %v", blockNumber), err, true) - } + // clean fields that are not available for pending block + rpcBlock.Hash = nil + rpcBlock.Miner = nil + rpcBlock.Nonce = nil + rpcBlock.TotalDifficulty = nil - txs := l2Block.Transactions() - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) - } + return rpcBlock, nil + } + var err error + blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - rpcBlock, err := types.NewBlock(ctx, e.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, false, includeExtraInfo, dbTx) + l2Block, err := e.state.GetL2BlockByNumber(ctx, blockNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load block from state by number %v", blockNumber), err, true) + } + + txs := l2Block.Transactions() + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by number %v", blockNumber), err, true) + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - return rpcBlock, nil - }) + rpcBlock, err := types.NewBlock(ctx, e.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, false, includeExtraInfo, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by number %v", blockNumber), err, true) + } + + return rpcBlock, nil } // GetCode returns account code at given block number func (e *EthEndpoints) GetCode(address types.ArgAddress, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var err error - block, rpcErr := e.getBlockByArg(ctx, blockArg, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + var err error + block, rpcErr := e.getBlockByArg(ctx, blockArg, nil) + if rpcErr != nil { + return nil, rpcErr + } - code, err := e.state.GetCode(ctx, address.Address(), block.Root()) - if errors.Is(err, state.ErrNotFound) { - return "0x", nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get code", err, true) - } + code, err := e.state.GetCode(ctx, address.Address(), block.Root()) + if errors.Is(err, state.ErrNotFound) { + return "0x", nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get code", err, true) + } - return types.ArgBytes(code), nil - }) + return types.ArgBytes(code), nil } // GetCompilers eth_getCompilers @@ -511,9 +503,8 @@ func (e *EthEndpoints) GetFilterLogs(filterID string) (interface{}, types.Error) // GetLogs returns a list of logs accordingly to the provided filter func (e *EthEndpoints) GetLogs(filter LogFilter) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return e.internalGetLogs(ctx, dbTx, filter) - }) + ctx := context.Background() + return e.internalGetLogs(ctx, nil, filter) } func (e *EthEndpoints) internalGetLogs(ctx context.Context, dbTx pgx.Tx, filter LogFilter) (interface{}, types.Error) { @@ -549,92 +540,123 @@ func (e *EthEndpoints) internalGetLogs(ctx context.Context, dbTx pgx.Tx, filter // GetStorageAt gets the value stored for an specific address and position func (e *EthEndpoints) GetStorageAt(address types.ArgAddress, storageKeyStr string, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { + ctx := context.Background() storageKey := types.ArgHash{} err := storageKey.UnmarshalText([]byte(storageKeyStr)) if err != nil { return RPCErrorResponse(types.DefaultErrorCode, "unable to decode storage key: hex string invalid", nil, false) } - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - block, respErr := e.getBlockByArg(ctx, blockArg, dbTx) - if respErr != nil { - return nil, respErr - } + block, respErr := e.getBlockByArg(ctx, blockArg, nil) + if respErr != nil { + return nil, respErr + } - value, err := e.state.GetStorageAt(ctx, address.Address(), storageKey.Hash().Big(), block.Root()) - if errors.Is(err, state.ErrNotFound) { - return types.ArgBytesPtr(common.Hash{}.Bytes()), nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get storage value from state", err, true) - } + value, err := e.state.GetStorageAt(ctx, address.Address(), storageKey.Hash().Big(), block.Root()) + if errors.Is(err, state.ErrNotFound) { + return types.ArgBytesPtr(common.Hash{}.Bytes()), nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get storage value from state", err, true) + } - return types.ArgBytesPtr(common.BigToHash(value).Bytes()), nil - }) + return types.ArgBytesPtr(common.BigToHash(value).Bytes()), nil } // GetTransactionByBlockHashAndIndex returns information about a transaction by // block hash and transaction index position. func (e *EthEndpoints) GetTransactionByBlockHashAndIndex(hash types.ArgHash, index types.Index, includeExtraInfo *bool) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - tx, err := e.state.GetTransactionByL2BlockHashAndIndex(ctx, hash.Hash(), uint64(index), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction", err, true) - } - - receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction receipt", err, true) - } + ctx := context.Background() + tx, err := e.state.GetTransactionByL2BlockHashAndIndex(ctx, hash.Hash(), uint64(index), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction", err, true) + } - var l2Hash *common.Hash - if includeExtraInfo != nil && *includeExtraInfo { - l2h, err := e.state.GetL2TxHashByTxHash(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) - } - l2Hash = l2h - } + receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction receipt", err, true) + } - res, err := types.NewTransaction(*tx, receipt, false, l2Hash) + var l2Hash *common.Hash + if includeExtraInfo != nil && *includeExtraInfo { + l2h, err := e.state.GetL2TxHashByTxHash(ctx, tx.Hash(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) + return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) } + l2Hash = l2h + } - return res, nil - }) + res, err := types.NewTransaction(*tx, receipt, false, l2Hash) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) + } + + return res, nil } // GetTransactionByBlockNumberAndIndex returns information about a transaction by // block number and transaction index position. func (e *EthEndpoints) GetTransactionByBlockNumberAndIndex(number *types.BlockNumber, index types.Index, includeExtraInfo *bool) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var err error - blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + var err error + blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - tx, err := e.state.GetTransactionByL2BlockNumberAndIndex(ctx, blockNumber, uint64(index), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction", err, true) + tx, err := e.state.GetTransactionByL2BlockNumberAndIndex(ctx, blockNumber, uint64(index), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction", err, true) + } + + receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction receipt", err, true) + } + + var l2Hash *common.Hash + if includeExtraInfo != nil && *includeExtraInfo { + l2h, err := e.state.GetL2TxHashByTxHash(ctx, tx.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) } + l2Hash = l2h + } + + res, err := types.NewTransaction(*tx, receipt, false, l2Hash) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) + } + + return res, nil +} - receipt, err := e.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) +// GetTransactionByHash returns a transaction by his hash +func (e *EthEndpoints) GetTransactionByHash(hash types.ArgHash, includeExtraInfo *bool) (interface{}, types.Error) { + ctx := context.Background() + // try to get tx from state + tx, err := e.state.GetTransactionByHash(ctx, hash.Hash(), nil) + if err != nil && !errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by hash from state", err, true) + } + if tx != nil { + receipt, err := e.state.GetTransactionReceipt(ctx, hash.Hash(), nil) if errors.Is(err, state.ErrNotFound) { - return nil, nil + return RPCErrorResponse(types.DefaultErrorCode, "transaction receipt not found", err, false) } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get transaction receipt", err, true) + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction receipt from state", err, true) } var l2Hash *common.Hash if includeExtraInfo != nil && *includeExtraInfo { - l2h, err := e.state.GetL2TxHashByTxHash(ctx, tx.Hash(), dbTx) + l2h, err := e.state.GetL2TxHashByTxHash(ctx, hash.Hash(), nil) if err != nil { return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) } @@ -647,62 +669,27 @@ func (e *EthEndpoints) GetTransactionByBlockNumberAndIndex(number *types.BlockNu } return res, nil - }) -} - -// GetTransactionByHash returns a transaction by his hash -func (e *EthEndpoints) GetTransactionByHash(hash types.ArgHash, includeExtraInfo *bool) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - // try to get tx from state - tx, err := e.state.GetTransactionByHash(ctx, hash.Hash(), dbTx) - if err != nil && !errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by hash from state", err, true) - } - if tx != nil { - receipt, err := e.state.GetTransactionReceipt(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, "transaction receipt not found", err, false) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction receipt from state", err, true) - } - - var l2Hash *common.Hash - if includeExtraInfo != nil && *includeExtraInfo { - l2h, err := e.state.GetL2TxHashByTxHash(ctx, hash.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) - } - l2Hash = l2h - } - - res, err := types.NewTransaction(*tx, receipt, false, l2Hash) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) - } - - return res, nil - } + } - // if the tx does not exist in the state, look for it in the pool - if e.cfg.SequencerNodeURI != "" { - return e.getTransactionByHashFromSequencerNode(hash.Hash(), includeExtraInfo) - } - poolTx, err := e.pool.GetTransactionByHash(ctx, hash.Hash()) - if errors.Is(err, pool.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by hash from pool", err, true) - } - if poolTx.Status == pool.TxStatusPending { - tx = &poolTx.Transaction - res, err := types.NewTransaction(*tx, nil, false, nil) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) - } - return res, nil - } + // if the tx does not exist in the state, look for it in the pool + if e.cfg.SequencerNodeURI != "" { + return e.getTransactionByHashFromSequencerNode(hash.Hash(), includeExtraInfo) + } + poolTx, err := e.pool.GetTransactionByHash(ctx, hash.Hash()) + if errors.Is(err, pool.ErrNotFound) { return nil, nil - }) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by hash from pool", err, true) + } + if poolTx.Status == pool.TxStatusPending { + tx = &poolTx.Transaction + res, err := types.NewTransaction(*tx, nil, false, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) + } + return res, nil + } + return nil, nil } func (e *EthEndpoints) getTransactionByHashFromSequencerNode(hash common.Hash, includeExtraInfo *bool) (interface{}, types.Error) { @@ -729,45 +716,44 @@ func (e *EthEndpoints) getTransactionByHashFromSequencerNode(hash common.Hash, i // GetTransactionCount returns account nonce func (e *EthEndpoints) GetTransactionCount(address types.ArgAddress, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var ( - pendingNonce uint64 - nonce uint64 - err error - ) - - block, respErr := e.getBlockByArg(ctx, blockArg, dbTx) - if respErr != nil { - return nil, respErr - } + ctx := context.Background() + var ( + pendingNonce uint64 + nonce uint64 + err error + ) + + block, respErr := e.getBlockByArg(ctx, blockArg, nil) + if respErr != nil { + return nil, respErr + } - if blockArg != nil { - blockNumArg := blockArg.Number() - if blockNumArg != nil && *blockNumArg == types.PendingBlockNumber { - if e.cfg.SequencerNodeURI != "" { - return e.getTransactionCountFromSequencerNode(address.Address(), blockArg.Number()) - } - pendingNonce, err = e.pool.GetNonce(ctx, address.Address()) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to count pending transactions", err, true) - } + if blockArg != nil { + blockNumArg := blockArg.Number() + if blockNumArg != nil && *blockNumArg == types.PendingBlockNumber { + if e.cfg.SequencerNodeURI != "" { + return e.getTransactionCountFromSequencerNode(address.Address(), blockArg.Number()) + } + pendingNonce, err = e.pool.GetNonce(ctx, address.Address()) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to count pending transactions", err, true) } } + } - nonce, err = e.state.GetNonce(ctx, address.Address(), block.Root()) + nonce, err = e.state.GetNonce(ctx, address.Address(), block.Root()) - if errors.Is(err, state.ErrNotFound) { - return hex.EncodeUint64(0), nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) - } + if errors.Is(err, state.ErrNotFound) { + return hex.EncodeUint64(0), nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) + } - if pendingNonce > nonce { - nonce = pendingNonce - } + if pendingNonce > nonce { + nonce = pendingNonce + } - return hex.EncodeUint64(nonce), nil - }) + return hex.EncodeUint64(nonce), nil } func (e *EthEndpoints) getTransactionCountFromSequencerNode(address common.Address, number *types.BlockNumber) (interface{}, types.Error) { @@ -791,44 +777,42 @@ func (e *EthEndpoints) getTransactionCountFromSequencerNode(address common.Addre // GetBlockTransactionCountByHash returns the number of transactions in a // block from a block matching the given block hash. func (e *EthEndpoints) GetBlockTransactionCountByHash(hash types.ArgHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - c, err := e.state.GetL2BlockTransactionCountByHash(ctx, hash.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) - } + ctx := context.Background() + c, err := e.state.GetL2BlockTransactionCountByHash(ctx, hash.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) + } - return types.ArgUint64(c), nil - }) + return types.ArgUint64(c), nil } // GetBlockTransactionCountByNumber returns the number of transactions in a // block from a block matching the given block number. func (e *EthEndpoints) GetBlockTransactionCountByNumber(number *types.BlockNumber) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if number != nil && *number == types.PendingBlockNumber { - if e.cfg.SequencerNodeURI != "" { - return e.getBlockTransactionCountByNumberFromSequencerNode(number) - } - c, err := e.pool.CountPendingTransactions(ctx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to count pending transactions", err, true) - } - return types.ArgUint64(c), nil - } - - var err error - blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr + ctx := context.Background() + if number != nil && *number == types.PendingBlockNumber { + if e.cfg.SequencerNodeURI != "" { + return e.getBlockTransactionCountByNumberFromSequencerNode(number) } - - c, err := e.state.GetL2BlockTransactionCountByNumber(ctx, blockNumber, dbTx) + c, err := e.pool.CountPendingTransactions(ctx) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) + return RPCErrorResponse(types.DefaultErrorCode, "failed to count pending transactions", err, true) } - return types.ArgUint64(c), nil - }) + } + + var err error + blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, e.state, e.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } + + c, err := e.state.GetL2BlockTransactionCountByNumber(ctx, blockNumber, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to count transactions", err, true) + } + + return types.ArgUint64(c), nil } func (e *EthEndpoints) getBlockTransactionCountByNumberFromSequencerNode(number *types.BlockNumber) (interface{}, types.Error) { @@ -851,28 +835,27 @@ func (e *EthEndpoints) getBlockTransactionCountByNumberFromSequencerNode(number // GetTransactionReceipt returns a transaction receipt by his hash func (e *EthEndpoints) GetTransactionReceipt(hash types.ArgHash) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - tx, err := e.state.GetTransactionByHash(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx from state", err, true) - } + ctx := context.Background() + tx, err := e.state.GetTransactionByHash(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx from state", err, true) + } - r, err := e.state.GetTransactionReceipt(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx receipt from state", err, true) - } + r, err := e.state.GetTransactionReceipt(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx receipt from state", err, true) + } - receipt, err := types.NewReceipt(*tx, r, nil) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build the receipt response", err, true) - } + receipt, err := types.NewReceipt(*tx, r, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build the receipt response", err, true) + } - return receipt, nil - }) + return receipt, nil } // NewBlockFilter creates a filter in the node, to notify when @@ -896,9 +879,8 @@ func (e *EthEndpoints) newBlockFilter(wsConn *concurrentWsConn) (interface{}, ty // to notify when the state changes (logs). To check if the state // has changed, call eth_getFilterChanges. func (e *EthEndpoints) NewFilter(filter LogFilter) (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - return e.newFilter(ctx, nil, filter, dbTx) - }) + ctx := context.Background() + return e.newFilter(ctx, nil, filter, nil) } // internal @@ -1006,46 +988,45 @@ func (e *EthEndpoints) UninstallFilter(filterID string) (interface{}, types.Erro // Syncing returns an object with data about the sync status or false. // https://eth.wiki/json-rpc/API#eth_syncing func (e *EthEndpoints) Syncing() (interface{}, types.Error) { - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - _, err := e.state.GetLastL2BlockNumber(ctx, dbTx) - if errors.Is(err, state.ErrStateNotSynchronized) { - return nil, types.NewRPCError(types.DefaultErrorCode, state.ErrStateNotSynchronized.Error()) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get last block number from state", err, true) - } + ctx := context.Background() + _, err := e.state.GetLastL2BlockNumber(ctx, nil) + if errors.Is(err, state.ErrStateNotSynchronized) { + return nil, types.NewRPCError(types.DefaultErrorCode, state.ErrStateNotSynchronized.Error()) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get last block number from state", err, true) + } - syncInfo, err := e.state.GetSyncingInfo(ctx, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get syncing info from state", err, true) - } + syncInfo, err := e.state.GetSyncingInfo(ctx, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get syncing info from state", err, true) + } - if !syncInfo.IsSynchronizing { - return false, nil - } - if e.cfg.SequencerNodeURI != "" { - // If we have a trusted node we ask it for the highest l2 block - res, err := e.getHighestL2BlockFromTrustedNode() - if err != nil { - log.Warnf("failed to get highest l2 block from trusted node: %v", err) + if !syncInfo.IsSynchronizing { + return false, nil + } + if e.cfg.SequencerNodeURI != "" { + // If we have a trusted node we ask it for the highest l2 block + res, err := e.getHighestL2BlockFromTrustedNode() + if err != nil { + log.Warnf("failed to get highest l2 block from trusted node: %v", err) + } else { + highestL2BlockInTrusted := res.(uint64) + if highestL2BlockInTrusted > syncInfo.CurrentBlockNumber { + syncInfo.EstimatedHighestBlock = highestL2BlockInTrusted } else { - highestL2BlockInTrusted := res.(uint64) - if highestL2BlockInTrusted > syncInfo.CurrentBlockNumber { - syncInfo.EstimatedHighestBlock = highestL2BlockInTrusted - } else { - log.Warnf("highest l2 block in trusted node (%d) is lower than the current block number in the state (%d)", highestL2BlockInTrusted, syncInfo.CurrentBlockNumber) - } + log.Warnf("highest l2 block in trusted node (%d) is lower than the current block number in the state (%d)", highestL2BlockInTrusted, syncInfo.CurrentBlockNumber) } } - return struct { - S types.ArgUint64 `json:"startingBlock"` - C types.ArgUint64 `json:"currentBlock"` - H types.ArgUint64 `json:"highestBlock"` - }{ - S: types.ArgUint64(syncInfo.InitialSyncingBlock), - C: types.ArgUint64(syncInfo.CurrentBlockNumber), - H: types.ArgUint64(syncInfo.EstimatedHighestBlock), - }, nil - }) + } + return struct { + S types.ArgUint64 `json:"startingBlock"` + C types.ArgUint64 `json:"currentBlock"` + H types.ArgUint64 `json:"highestBlock"` + }{ + S: types.ArgUint64(syncInfo.InitialSyncingBlock), + C: types.ArgUint64(syncInfo.CurrentBlockNumber), + H: types.ArgUint64(syncInfo.EstimatedHighestBlock), + }, nil } // GetUncleByBlockHashAndIndex returns information about a uncle of a @@ -1109,13 +1090,12 @@ func (e *EthEndpoints) Subscribe(wsConn *concurrentWsConn, name string, logFilte case "newHeads": return e.newBlockFilter(wsConn) case "logs": - return e.txMan.NewDbTxScope(e.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var lf LogFilter - if logFilter != nil { - lf = *logFilter - } - return e.newFilter(ctx, wsConn, lf, dbTx) - }) + ctx := context.Background() + var lf LogFilter + if logFilter != nil { + lf = *logFilter + } + return e.newFilter(ctx, wsConn, lf, nil) case "pendingTransactions", "newPendingTransactions": return e.newPendingTransactionFilter(wsConn) case "syncing": diff --git a/jsonrpc/endpoints_eth_test.go b/jsonrpc/endpoints_eth_test.go index 69291429d0..a60ce9418d 100644 --- a/jsonrpc/endpoints_eth_test.go +++ b/jsonrpc/endpoints_eth_test.go @@ -63,18 +63,8 @@ func TestBlockNumber(t *testing.T) { ExpectedError: nil, ExpectedResult: blockNumTen.Uint64(), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumTen.Uint64(), nil). Once() }, @@ -84,18 +74,8 @@ func TestBlockNumber(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), ExpectedResult: 0, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -155,8 +135,6 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -171,10 +149,10 @@ func TestCall(t *testing.T) { return match }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumOneUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumOneUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -198,11 +176,9 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { @@ -218,7 +194,7 @@ func TestCall(t *testing.T) { }) m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumOneUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumOneUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -240,9 +216,7 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -257,10 +231,10 @@ func TestCall(t *testing.T) { return match }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -282,11 +256,9 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { @@ -302,7 +274,7 @@ func TestCall(t *testing.T) { }) m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumTenUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumTenUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -324,8 +296,6 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -339,10 +309,10 @@ func TestCall(t *testing.T) { tx.Nonce() == nonce }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, nil).Return(block, nil).Once() m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumTenUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, &blockNumTenUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -362,10 +332,8 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { blockHeader := state.NewL2Header(ðTypes.Header{GasLimit: s.Config.MaxCumulativeGasUsed}) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() - m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(blockHeader, nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(blockHeader, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -379,9 +347,9 @@ func TestCall(t *testing.T) { return hasTx && gasMatch && toMatch && gasPriceMatch && valueMatch && dataMatch }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -401,10 +369,8 @@ func TestCall(t *testing.T) { expectedError: nil, setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { blockHeader := state.NewL2Header(ðTypes.Header{GasLimit: s.Config.MaxCumulativeGasUsed}) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() - m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(blockHeader, nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(blockHeader, nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -418,9 +384,9 @@ func TestCall(t *testing.T) { return hasTx && gasMatch && toMatch && gasPriceMatch && valueMatch && dataMatch }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, true, nil). Return(&runtime.ExecutionResult{ReturnValue: testCase.expectedResult}, nil). Once() }, @@ -439,12 +405,10 @@ func TestCall(t *testing.T) { expectedResult: nil, expectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get block header"), setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { - m.DbTx.On("Rollback", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() - m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(nil, errors.New("failed to get block header")).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() + m.State.On("GetL2BlockHeaderByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(nil, errors.New("failed to get block header")).Once() }, }, { @@ -464,9 +428,7 @@ func TestCall(t *testing.T) { expectedError: types.NewRPCError(types.DefaultErrorCode, "failed to process unsigned transaction"), setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Rollback", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -481,10 +443,10 @@ func TestCall(t *testing.T) { return hasTx && gasMatch && toMatch && gasPriceMatch && valueMatch && dataMatch && nonceMatch }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, nil). Return(&runtime.ExecutionResult{Err: errors.New("failed to process unsigned transaction")}, nil). Once() }, @@ -506,9 +468,7 @@ func TestCall(t *testing.T) { expectedError: types.NewRPCError(types.DefaultErrorCode, "execution reverted"), setupMocks: func(c Config, m *mocksWrapper, testCase *testCase) { nonce := uint64(7) - m.DbTx.On("Rollback", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(blockNumOne.Uint64(), nil).Once() + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(blockNumOne.Uint64(), nil).Once() txArgs := testCase.params[0].(types.TxArgs) txMatchBy := mock.MatchedBy(func(tx *ethTypes.Transaction) bool { gasPrice := big.NewInt(0).SetBytes(*txArgs.GasPrice) @@ -523,10 +483,10 @@ func TestCall(t *testing.T) { return hasTx && gasMatch && toMatch && gasPriceMatch && valueMatch && dataMatch && nonceMatch }) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOneUint64, nil).Return(block, nil).Once() m.State.On("GetNonce", context.Background(), *txArgs.From, blockRoot).Return(nonce, nil).Once() m.State. - On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, m.DbTx). + On("ProcessUnsignedTransaction", context.Background(), txMatchBy, *txArgs.From, nilUint64, true, nil). Return(&runtime.ExecutionResult{Err: runtime.ErrExecutionReverted}, nil). Once() }, @@ -677,18 +637,15 @@ func TestEstimateGas(t *testing.T) { return matchTo && matchGasPrice && matchValue && matchData && matchNonce }) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetLastL2Block", context.Background(), m.DbTx).Return(block, nil).Once() + m.State.On("GetLastL2Block", context.Background(), nil).Return(block, nil).Once() m.State. On("GetNonce", context.Background(), *txArgs.From, blockRoot). Return(nonce, nil). Once() m.State. - On("EstimateGas", txMatchBy, *txArgs.From, nilUint64, m.DbTx). + On("EstimateGas", txMatchBy, *txArgs.From, nilUint64, nil). Return(*testCase.expectedResult, nil, nil). Once() }, @@ -722,14 +679,11 @@ func TestEstimateGas(t *testing.T) { return matchTo && matchGasPrice && matchValue && matchData && matchNonce }) - m.DbTx.On("Commit", context.Background()).Return(nil).Once() - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetLastL2Block", context.Background(), m.DbTx).Return(block, nil).Once() + m.State.On("GetLastL2Block", context.Background(), nil).Return(block, nil).Once() m.State. - On("EstimateGas", txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, m.DbTx). + On("EstimateGas", txMatchBy, common.HexToAddress(state.DefaultSenderAddress), nilUint64, nil). Return(*testCase.expectedResult, nil, nil). Once() }, @@ -822,18 +776,8 @@ func TestGetBalance(t *testing.T) { expectedBalance: 0, expectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), setupMocks: func(m *mocksWrapper, t *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(nil, errors.New("failed to get last block number")).Once() }, }, @@ -846,19 +790,9 @@ func TestGetBalance(t *testing.T) { expectedBalance: 1000, expectedError: nil, setupMocks: func(m *mocksWrapper, t *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(block, nil).Once() m.State. @@ -879,19 +813,9 @@ func TestGetBalance(t *testing.T) { expectedBalance: 1000, expectedError: nil, setupMocks: func(m *mocksWrapper, t *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil). Once() @@ -910,18 +834,8 @@ func TestGetBalance(t *testing.T) { expectedBalance: 0, expectedError: nil, setupMocks: func(m *mocksWrapper, t *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetLastL2Block", context.Background(), m.DbTx).Return(block, nil).Once() + m.State.On("GetLastL2Block", context.Background(), nil).Return(block, nil).Once() m.State. On("GetBalance", context.Background(), addressArg, blockRoot). @@ -938,18 +852,8 @@ func TestGetBalance(t *testing.T) { expectedBalance: 0, expectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get balance from state"), setupMocks: func(m *mocksWrapper, t *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetLastL2Block", context.Background(), m.DbTx).Return(block, nil).Once() + m.State.On("GetLastL2Block", context.Background(), nil).Return(block, nil).Once() m.State. On("GetBalance", context.Background(), addressArg, blockRoot). @@ -1004,18 +908,8 @@ func TestGetL2BlockByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: ethereum.NotFound, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound) }, }, @@ -1025,18 +919,8 @@ func TestGetL2BlockByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get block by hash from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get block from state")). Once() }, @@ -1059,24 +943,14 @@ func TestGetL2BlockByHash(t *testing.T) { } block := state.NewL2Block(state.NewL2Header(tc.ExpectedResult.Header()), tc.ExpectedResult.Transactions(), uncles, []*ethTypes.Receipt{ethTypes.NewReceipt([]byte{}, false, uint64(0))}, st) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(block, nil). Once() for _, tx := range tc.ExpectedResult.Transactions() { m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil). Once() } @@ -1264,18 +1138,8 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetL2BlockByNumber", context.Background(), tc.Number.Uint64(), m.DbTx). + On("GetL2BlockByNumber", context.Background(), tc.Number.Uint64(), nil). Return(nil, state.ErrNotFound) }, }, @@ -1285,30 +1149,20 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: rpcBlock, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByNumber", context.Background(), tc.Number.Uint64(), m.DbTx). + On("GetL2BlockByNumber", context.Background(), tc.Number.Uint64(), nil). Return(l2Block, nil). Once() for _, receipt := range receipts { m.State. - On("GetTransactionReceipt", context.Background(), receipt.TxHash, m.DbTx). + On("GetTransactionReceipt", context.Background(), receipt.TxHash, nil). Return(receipt, nil). Once() } for _, signedTx := range signedTransactions { m.State. - On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), nil). Return(state.Ptr(signedTx.Hash()), nil). Once() } @@ -1320,35 +1174,25 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: rpcBlock, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(tc.ExpectedResult.Number), nil). Once() m.State. - On("GetL2BlockByNumber", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetL2BlockByNumber", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(l2Block, nil). Once() for _, receipt := range receipts { m.State. - On("GetTransactionReceipt", context.Background(), receipt.TxHash, m.DbTx). + On("GetTransactionReceipt", context.Background(), receipt.TxHash, nil). Return(receipt, nil). Once() } for _, signedTx := range signedTransactions { m.State. - On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), nil). Return(state.Ptr(signedTx.Hash()), nil). Once() } @@ -1360,18 +1204,8 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -1382,23 +1216,13 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load block from state by number 1"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(1), nil). Once() m.State. - On("GetL2BlockByNumber", context.Background(), uint64(1), m.DbTx). + On("GetL2BlockByNumber", context.Background(), uint64(1), nil). Return(nil, errors.New("failed to load block by number")). Once() }, @@ -1427,18 +1251,8 @@ func TestGetL2BlockByNumber(t *testing.T) { tc.ExpectedResult.Nonce = nil tc.ExpectedResult.TotalDifficulty = nil - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(lastBlock, nil). Once() }, @@ -1449,18 +1263,8 @@ func TestGetL2BlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(nil, errors.New("failed to load last block")). Once() }, @@ -1626,18 +1430,8 @@ func TestGetCode(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(nil, errors.New("failed to get last block number")). Once() }, @@ -1652,18 +1446,8 @@ func TestGetCode(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get code"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(block, nil).Once() m.State. On("GetCode", context.Background(), addressArg, blockRoot). @@ -1681,18 +1465,8 @@ func TestGetCode(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(block, nil).Once() m.State. On("GetCode", context.Background(), addressArg, blockRoot). @@ -1710,18 +1484,8 @@ func TestGetCode(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumOne, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumOne.Uint64(), nil).Return(block, nil).Once() m.State. On("GetCode", context.Background(), addressArg, blockRoot). @@ -1738,19 +1502,9 @@ func TestGetCode(t *testing.T) { ExpectedResult: []byte{1, 2, 3}, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil). Once() @@ -1819,18 +1573,8 @@ func TestGetStorageAt(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(nil, errors.New("failed to get last block number")). Once() }, @@ -1848,19 +1592,9 @@ func TestGetStorageAt(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get storage value from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - blockNumber := big.NewInt(1) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumber, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), nil).Return(block, nil).Once() m.State. On("GetStorageAt", context.Background(), addressArg, keyArg.Big(), blockRoot). @@ -1881,19 +1615,9 @@ func TestGetStorageAt(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - blockNumber := big.NewInt(1) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumber, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), nil).Return(block, nil).Once() m.State. On("GetStorageAt", context.Background(), addressArg, keyArg.Big(), blockRoot). @@ -1914,19 +1638,9 @@ func TestGetStorageAt(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - blockNumber := big.NewInt(1) block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumber, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumber.Uint64(), nil).Return(block, nil).Once() m.State. On("GetStorageAt", context.Background(), addressArg, keyArg.Big(), blockRoot). @@ -1947,19 +1661,9 @@ func TestGetStorageAt(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil). Once() @@ -2030,18 +1734,8 @@ func TestSyncing(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get last block number from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last l2 block number from state")). Once() }, @@ -2051,23 +1745,13 @@ func TestSyncing(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get syncing info from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(10), nil). Once() m.State. - On("GetSyncingInfo", context.Background(), m.DbTx). + On("GetSyncingInfo", context.Background(), nil). Return(state.SyncingInfo{}, errors.New("failed to get syncing info from state")). Once() }, @@ -2077,23 +1761,13 @@ func TestSyncing(t *testing.T) { ExpectedResult: ðereum.SyncProgress{StartingBlock: 1, CurrentBlock: 2, HighestBlock: 3}, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(10), nil). Once() m.State. - On("GetSyncingInfo", context.Background(), m.DbTx). + On("GetSyncingInfo", context.Background(), nil). Return(state.SyncingInfo{InitialSyncingBlock: 1, CurrentBlockNumber: 2, EstimatedHighestBlock: 3, IsSynchronizing: true}, nil). Once() }, @@ -2103,23 +1777,13 @@ func TestSyncing(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(10), nil). Once() m.State. - On("GetSyncingInfo", context.Background(), m.DbTx). + On("GetSyncingInfo", context.Background(), nil). Return(state.SyncingInfo{InitialSyncingBlock: 1, CurrentBlockNumber: 1, EstimatedHighestBlock: 3, IsSynchronizing: false}, nil). Once() }, @@ -2129,23 +1793,13 @@ func TestSyncing(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(10), nil). Once() m.State. - On("GetSyncingInfo", context.Background(), m.DbTx). + On("GetSyncingInfo", context.Background(), nil). Return(state.SyncingInfo{InitialSyncingBlock: 1, CurrentBlockNumber: 2, EstimatedHighestBlock: 3, IsSynchronizing: false}, nil). Once() }, @@ -2207,18 +1861,9 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { tx := tc.ExpectedResult - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() m.State. - On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), nil). Return(tx, nil). Once() @@ -2228,7 +1873,7 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { receipt.TransactionIndex = tc.Index m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(receipt, nil). Once() }, @@ -2240,18 +1885,8 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { ExpectedResult: nil, ExpectedError: ethereum.NotFound, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), nil). Return(nil, state.ErrNotFound). Once() }, @@ -2263,18 +1898,8 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get transaction"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), nil). Return(nil, errors.New("failed to get transaction by block and index from state")). Once() }, @@ -2287,23 +1912,14 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { ExpectedError: ethereum.NotFound, SetupMocks: func(m *mocksWrapper, tc testCase) { tx := ethTypes.NewTransaction(0, common.Address{}, big.NewInt(0), 0, big.NewInt(0), []byte{}) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(nil, state.ErrNotFound). Once() }, @@ -2316,23 +1932,14 @@ func TestGetTransactionL2onByBlockHashAndIndex(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get transaction receipt"), SetupMocks: func(m *mocksWrapper, tc testCase) { tx := ethTypes.NewTransaction(0, common.Address{}, big.NewInt(0), 0, big.NewInt(0), []byte{}) - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() m.State. - On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockHashAndIndex", context.Background(), tc.Hash, uint64(tc.Index), nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(nil, errors.New("failed to get transaction receipt from state")). Once() }, @@ -2395,18 +2002,9 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { SetupMocks: func(m *mocksWrapper, tc testCase) { tx := tc.ExpectedResult blockNumber, _ := encoding.DecodeUint64orHex(&tc.BlockNumber) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), nil). Return(tx, nil). Once() @@ -2415,7 +2013,7 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { receipt.BlockNumber = big.NewInt(1) receipt.TransactionIndex = tc.Index m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(receipt, nil). Once() }, @@ -2427,18 +2025,8 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -2451,18 +2039,8 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { blockNumber, _ := encoding.DecodeUint64orHex(&tc.BlockNumber) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), nil). Return(nil, state.ErrNotFound). Once() }, @@ -2475,18 +2053,8 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get transaction"), SetupMocks: func(m *mocksWrapper, tc testCase) { blockNumber, _ := encoding.DecodeUint64orHex(&tc.BlockNumber) - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), nil). Return(nil, errors.New("failed to get transaction by block and index from state")). Once() }, @@ -2501,23 +2069,13 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { tx := ethTypes.NewTransaction(0, common.Address{}, big.NewInt(0), 0, big.NewInt(0), []byte{}) blockNumber, _ := encoding.DecodeUint64orHex(&tc.BlockNumber) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(nil, state.ErrNotFound). Once() }, @@ -2532,23 +2090,13 @@ func TestGetTransactionByBlockNumberAndIndex(t *testing.T) { tx := ethTypes.NewTransaction(0, common.Address{}, big.NewInt(0), 0, big.NewInt(0), []byte{}) blockNumber, _ := encoding.DecodeUint64orHex(&tc.BlockNumber) - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), m.DbTx). + On("GetTransactionByL2BlockNumberAndIndex", context.Background(), blockNumber, uint64(tc.Index), nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(nil, errors.New("failed to get transaction receipt from state")). Once() }, @@ -2615,18 +2163,8 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedResult: signedTx, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(tc.ExpectedResult, nil). Once() @@ -2635,7 +2173,7 @@ func TestGetTransactionByHash(t *testing.T) { receipt.BlockNumber = big.NewInt(1) m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(receipt, nil). Once() }, @@ -2647,18 +2185,8 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedResult: ethTypes.NewTransaction(1, common.Address{}, big.NewInt(1), 1, big.NewInt(1), []byte{}), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2675,18 +2203,8 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: ethereum.NotFound, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2703,18 +2221,8 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction by hash from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to load transaction by hash from state")). Once() }, @@ -2726,18 +2234,8 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction by hash from pool"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2755,23 +2253,14 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "transaction receipt not found"), SetupMocks: func(m *mocksWrapper, tc testCase) { tx := ðTypes.Transaction{} - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -2784,23 +2273,14 @@ func TestGetTransactionByHash(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction receipt from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { tx := ðTypes.Transaction{} - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to load transaction receipt from state")). Once() }, @@ -2851,18 +2331,8 @@ func TestGetBlockTransactionCountByHash(t *testing.T) { ExpectedResult: uint(10), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetL2BlockTransactionCountByHash", context.Background(), tc.BlockHash, m.DbTx). + On("GetL2BlockTransactionCountByHash", context.Background(), tc.BlockHash, nil). Return(uint64(10), nil). Once() }, @@ -2873,18 +2343,8 @@ func TestGetBlockTransactionCountByHash(t *testing.T) { ExpectedResult: 0, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to count transactions"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockTransactionCountByHash", context.Background(), tc.BlockHash, m.DbTx). + On("GetL2BlockTransactionCountByHash", context.Background(), tc.BlockHash, nil). Return(uint64(0), errors.New("failed to count txs")). Once() }, @@ -2932,23 +2392,14 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) { ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { blockNumber := uint64(10) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumber, nil). Once() m.State. - On("GetL2BlockTransactionCountByNumber", context.Background(), blockNumber, m.DbTx). + On("GetL2BlockTransactionCountByNumber", context.Background(), blockNumber, nil). Return(uint64(10), nil). Once() }, @@ -2959,16 +2410,6 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) { ExpectedResult: uint(10), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Pool. On("CountPendingTransactions", context.Background()). Return(uint64(10), nil). @@ -2981,18 +2422,8 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) { ExpectedResult: 0, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -3004,23 +2435,14 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to count transactions"), SetupMocks: func(m *mocksWrapper, tc testCase) { blockNumber := uint64(10) - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumber, nil). Once() m.State. - On("GetL2BlockTransactionCountByNumber", context.Background(), blockNumber, m.DbTx). + On("GetL2BlockTransactionCountByNumber", context.Background(), blockNumber, nil). Return(uint64(0), errors.New("failed to count")). Once() }, @@ -3031,16 +2453,6 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) { ExpectedResult: 0, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to count pending transactions"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Pool. On("CountPendingTransactions", context.Background()). Return(uint64(0), errors.New("failed to count")). @@ -3093,18 +2505,8 @@ func TestGetTransactionCount(t *testing.T) { ExpectedResult: uint(10), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetLastL2Block", context.Background(), m.DbTx).Return(block, nil).Once() + m.State.On("GetLastL2Block", context.Background(), nil).Return(block, nil).Once() m.State. On("GetNonce", context.Background(), addressArg, blockRoot). @@ -3121,19 +2523,9 @@ func TestGetTransactionCount(t *testing.T) { ExpectedResult: uint(10), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) m.State. - On("GetL2BlockByHash", context.Background(), blockHash, m.DbTx). + On("GetL2BlockByHash", context.Background(), blockHash, nil). Return(block, nil). Once() @@ -3152,23 +2544,13 @@ func TestGetTransactionCount(t *testing.T) { ExpectedResult: 0, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumTen.Uint64(), nil). Once() block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, nil).Return(block, nil).Once() m.State. On("GetNonce", context.Background(), addressArg, blockRoot). @@ -3185,18 +2567,8 @@ func TestGetTransactionCount(t *testing.T) { ExpectedResult: 0, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -3210,23 +2582,13 @@ func TestGetTransactionCount(t *testing.T) { ExpectedResult: 0, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to count transactions"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumTen.Uint64(), nil). Once() block := state.NewL2BlockWithHeader(state.NewL2Header(ðTypes.Header{Number: blockNumTen, Root: blockRoot})) - m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, m.DbTx).Return(block, nil).Once() + m.State.On("GetL2BlockByNumber", context.Background(), blockNumTenUint64, nil).Return(block, nil).Once() m.State. On("GetNonce", context.Background(), addressArg, blockRoot). @@ -3336,23 +2698,13 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: &rpcReceipt, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(receipt, nil). Once() }, @@ -3363,18 +2715,8 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -3385,18 +2727,8 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get tx from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get tx")). Once() }, @@ -3407,23 +2739,13 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -3434,23 +2756,13 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get tx receipt from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get tx receipt from state")). Once() }, @@ -3461,23 +2773,13 @@ func TestGetTransactionReceipt(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to build the receipt response"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByHash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByHash", context.Background(), tc.Hash, nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(receipt, nil). Once() }, @@ -3852,16 +3154,6 @@ func TestNewFilter(t *testing.T) { ExpectedResult: "1", ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Storage. On("NewLogFilter", mock.IsType(&concurrentWsConn{}), mock.IsType(LogFilter{})). Return("1", nil). @@ -3876,16 +3168,6 @@ func TestNewFilter(t *testing.T) { ExpectedResult: "1", ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Storage. On("NewLogFilter", mock.IsType(&concurrentWsConn{}), mock.IsType(LogFilter{})). Return("1", nil). @@ -3901,15 +3183,6 @@ func TestNewFilter(t *testing.T) { ExpectedResult: "", ExpectedError: types.NewRPCError(types.InvalidParamsErrorCode, "invalid block range"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, { @@ -3921,15 +3194,6 @@ func TestNewFilter(t *testing.T) { ExpectedResult: "", ExpectedError: types.NewRPCError(types.InvalidParamsErrorCode, "logs are limited to a 10000 block range"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, { @@ -3940,15 +3204,6 @@ func TestNewFilter(t *testing.T) { ExpectedResult: "", ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to create new log filter"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() m.Storage. On("NewLogFilter", mock.IsType(&concurrentWsConn{}), mock.IsType(LogFilter{})). Return("", errors.New("failed to add new filter")). @@ -4215,18 +3470,8 @@ func TestGetLogs(t *testing.T) { logs = append(logs, &l) } - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, m.DbTx). + On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, nil). Return(logs, nil). Once() }, @@ -4244,18 +3489,8 @@ func TestGetLogs(t *testing.T) { }, SetupMocks: func(m *mocksWrapper, tc testCase) { var since *time.Time - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, m.DbTx). + On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, nil). Return(nil, errors.New("failed to get logs from state")). Once() }, @@ -4272,18 +3507,8 @@ func TestGetLogs(t *testing.T) { tc.ExpectedError = types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state") }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number from state")). Once() }, @@ -4300,18 +3525,8 @@ func TestGetLogs(t *testing.T) { tc.ExpectedError = types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state") }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number from state")). Once() }, @@ -4328,15 +3543,6 @@ func TestGetLogs(t *testing.T) { tc.ExpectedError = types.NewRPCError(types.InvalidParamsErrorCode, "logs are limited to a 10000 block range") }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, { @@ -4352,18 +3558,9 @@ func TestGetLogs(t *testing.T) { }, SetupMocks: func(m *mocksWrapper, tc testCase) { var since *time.Time - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, m.DbTx). + On("GetLogs", context.Background(), tc.Filter.FromBlock.Uint64(), tc.Filter.ToBlock.Uint64(), tc.Filter.Addresses, tc.Filter.Topics, tc.Filter.BlockHash, since, nil). Return(nil, state.ErrMaxLogsCountLimitExceeded). Once() }, @@ -4444,23 +3641,13 @@ func TestGetFilterLogs(t *testing.T) { Parameters: logFilter, } - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Storage. On("GetFilter", tc.FilterID). Return(filter, nil). Once() m.State. - On("GetLogs", context.Background(), uint64(*logFilter.FromBlock), uint64(*logFilter.ToBlock), logFilter.Addresses, logFilter.Topics, logFilter.BlockHash, since, m.DbTx). + On("GetLogs", context.Background(), uint64(*logFilter.FromBlock), uint64(*logFilter.ToBlock), logFilter.Addresses, logFilter.Topics, logFilter.BlockHash, since, nil). Return(logs, nil). Once() }, @@ -5198,16 +4385,6 @@ func TestSubscribeNewLogs(t *testing.T) { } }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Storage. On("NewLogFilter", mock.IsType(&concurrentWsConn{}), mock.IsType(LogFilter{})). Return("0x1", nil). @@ -5223,16 +4400,6 @@ func TestSubscribeNewLogs(t *testing.T) { } }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.Storage. On("NewLogFilter", mock.IsType(&concurrentWsConn{}), mock.IsType(LogFilter{})). Return("", fmt.Errorf("failed to add filter to storage")). @@ -5248,15 +4415,6 @@ func TestSubscribeNewLogs(t *testing.T) { } }, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, } diff --git a/jsonrpc/endpoints_zkevm.go b/jsonrpc/endpoints_zkevm.go index cb56dade8c..f159885cb3 100644 --- a/jsonrpc/endpoints_zkevm.go +++ b/jsonrpc/endpoints_zkevm.go @@ -27,7 +27,6 @@ type ZKEVMEndpoints struct { pool types.PoolInterface state types.StateInterface etherman types.EthermanInterface - txMan DBTxManager } // NewZKEVMEndpoints returns ZKEVMEndpoints @@ -42,359 +41,346 @@ func NewZKEVMEndpoints(cfg Config, pool types.PoolInterface, state types.StateIn // ConsolidatedBlockNumber returns last block number related to the last verified batch func (z *ZKEVMEndpoints) ConsolidatedBlockNumber() (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - lastBlockNumber, err := z.state.GetLastConsolidatedL2BlockNumber(ctx, dbTx) - if err != nil { - const errorMessage = "failed to get last consolidated block number from state" - log.Errorf("%v:%v", errorMessage, err) - return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) - } + ctx := context.Background() + lastBlockNumber, err := z.state.GetLastConsolidatedL2BlockNumber(ctx, nil) + if err != nil { + const errorMessage = "failed to get last consolidated block number from state" + log.Errorf("%v:%v", errorMessage, err) + return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) + } - return hex.EncodeUint64(lastBlockNumber), nil - }) + return hex.EncodeUint64(lastBlockNumber), nil } // IsBlockConsolidated returns the consolidation status of a provided block number func (z *ZKEVMEndpoints) IsBlockConsolidated(blockNumber types.ArgUint64) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - IsL2BlockConsolidated, err := z.state.IsL2BlockConsolidated(ctx, uint64(blockNumber), dbTx) - if err != nil { - const errorMessage = "failed to check if the block is consolidated" - log.Errorf("%v: %v", errorMessage, err) - return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) - } + ctx := context.Background() + IsL2BlockConsolidated, err := z.state.IsL2BlockConsolidated(ctx, uint64(blockNumber), nil) + if err != nil { + const errorMessage = "failed to check if the block is consolidated" + log.Errorf("%v: %v", errorMessage, err) + return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) + } - return IsL2BlockConsolidated, nil - }) + return IsL2BlockConsolidated, nil } // IsBlockVirtualized returns the virtualization status of a provided block number func (z *ZKEVMEndpoints) IsBlockVirtualized(blockNumber types.ArgUint64) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - IsL2BlockVirtualized, err := z.state.IsL2BlockVirtualized(ctx, uint64(blockNumber), dbTx) - if err != nil { - const errorMessage = "failed to check if the block is virtualized" - log.Errorf("%v: %v", errorMessage, err) - return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) - } + ctx := context.Background() + IsL2BlockVirtualized, err := z.state.IsL2BlockVirtualized(ctx, uint64(blockNumber), nil) + if err != nil { + const errorMessage = "failed to check if the block is virtualized" + log.Errorf("%v: %v", errorMessage, err) + return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) + } - return IsL2BlockVirtualized, nil - }) + return IsL2BlockVirtualized, nil } // BatchNumberByBlockNumber returns the batch number from which the passed block number is created func (z *ZKEVMEndpoints) BatchNumberByBlockNumber(blockNumber types.ArgUint64) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - batchNum, err := z.state.BatchNumberByL2BlockNumber(ctx, uint64(blockNumber), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - const errorMessage = "failed to get batch number from block number" - log.Errorf("%v: %v", errorMessage, err.Error()) - return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) - } + ctx := context.Background() + batchNum, err := z.state.BatchNumberByL2BlockNumber(ctx, uint64(blockNumber), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + const errorMessage = "failed to get batch number from block number" + log.Errorf("%v: %v", errorMessage, err.Error()) + return nil, types.NewRPCError(types.DefaultErrorCode, errorMessage) + } - return hex.EncodeUint64(batchNum), nil - }) + return hex.EncodeUint64(batchNum), nil } // BatchNumber returns the latest trusted batch number func (z *ZKEVMEndpoints) BatchNumber() (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - lastBatchNumber, err := z.state.GetLastBatchNumber(ctx, dbTx) - if err != nil { - return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last batch number from state") - } + ctx := context.Background() + lastBatchNumber, err := z.state.GetLastBatchNumber(ctx, nil) + if err != nil { + return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last batch number from state") + } - return hex.EncodeUint64(lastBatchNumber), nil - }) + return hex.EncodeUint64(lastBatchNumber), nil } // VirtualBatchNumber returns the latest virtualized batch number func (z *ZKEVMEndpoints) VirtualBatchNumber() (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - lastBatchNumber, err := z.state.GetLastVirtualBatchNum(ctx, dbTx) - if err != nil { - return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last virtual batch number from state") - } + ctx := context.Background() + lastBatchNumber, err := z.state.GetLastVirtualBatchNum(ctx, nil) + if err != nil { + return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last virtual batch number from state") + } - return hex.EncodeUint64(lastBatchNumber), nil - }) + return hex.EncodeUint64(lastBatchNumber), nil } // VerifiedBatchNumber returns the latest verified batch number func (z *ZKEVMEndpoints) VerifiedBatchNumber() (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - lastBatch, err := z.state.GetLastVerifiedBatch(ctx, dbTx) - if err != nil { - return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last verified batch number from state") - } - return hex.EncodeUint64(lastBatch.BatchNumber), nil - }) + ctx := context.Background() + lastBatch, err := z.state.GetLastVerifiedBatch(ctx, nil) + if err != nil { + return "0x0", types.NewRPCError(types.DefaultErrorCode, "failed to get the last verified batch number from state") + } + return hex.EncodeUint64(lastBatch.BatchNumber), nil } // GetBatchByNumber returns information about a batch by batch number func (z *ZKEVMEndpoints) GetBatchByNumber(batchNumber types.BatchNumber, fullTx bool) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var err error - batchNumber, rpcErr := batchNumber.GetNumericBatchNumber(ctx, z.state, z.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + var err error + numericBatchNumber, rpcErr := batchNumber.GetNumericBatchNumber(ctx, z.state, z.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - batch, err := z.state.GetBatchByNumber(ctx, batchNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch from state by number %v", batchNumber), err, true) - } - batchTimestamp, err := z.state.GetBatchTimestamp(ctx, batchNumber, nil, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch timestamp from state by number %v", batchNumber), err, true) - } + batch, err := z.state.GetBatchByNumber(ctx, numericBatchNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch from state by number %v", numericBatchNumber), err, true) + } + batchTimestamp, err := z.state.GetBatchTimestamp(ctx, numericBatchNumber, nil, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch timestamp from state by number %v", numericBatchNumber), err, true) + } - if batchTimestamp == nil { - batch.Timestamp = time.Time{} - } else { - batch.Timestamp = *batchTimestamp - } + if batchTimestamp == nil { + batch.Timestamp = time.Time{} + } else { + batch.Timestamp = *batchTimestamp + } - txs, _, err := z.state.GetTransactionsByBatchNumber(ctx, batchNumber, dbTx) - if !errors.Is(err, state.ErrNotFound) && err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch txs from state by number %v", batchNumber), err, true) - } + txs, _, err := z.state.GetTransactionsByBatchNumber(ctx, numericBatchNumber, nil) + if !errors.Is(err, state.ErrNotFound) && err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch txs from state by number %v", numericBatchNumber), err, true) + } - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - virtualBatch, err := z.state.GetVirtualBatch(ctx, batchNumber, dbTx) - if err != nil && !errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load virtual batch from state by number %v", batchNumber), err, true) - } + virtualBatch, err := z.state.GetVirtualBatch(ctx, numericBatchNumber, nil) + if err != nil && !errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load virtual batch from state by number %v", numericBatchNumber), err, true) + } - verifiedBatch, err := z.state.GetVerifiedBatch(ctx, batchNumber, dbTx) - if err != nil && !errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load virtual batch from state by number %v", batchNumber), err, true) - } + verifiedBatch, err := z.state.GetVerifiedBatch(ctx, numericBatchNumber, nil) + if err != nil && !errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load virtual batch from state by number %v", numericBatchNumber), err, true) + } - ger, err := z.state.GetExitRootByGlobalExitRoot(ctx, batch.GlobalExitRoot, dbTx) - if err != nil && !errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load full GER from state by number %v", batchNumber), err, true) - } else if errors.Is(err, state.ErrNotFound) { - ger = &state.GlobalExitRoot{} - } + ger, err := z.state.GetExitRootByGlobalExitRoot(ctx, batch.GlobalExitRoot, nil) + if err != nil && !errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load full GER from state by number %v", numericBatchNumber), err, true) + } else if errors.Is(err, state.ErrNotFound) { + ger = &state.GlobalExitRoot{} + } - blocks, err := z.state.GetL2BlocksByBatchNumber(ctx, batchNumber, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load blocks associated to the batch %v", batchNumber), err, true) - } + blocks, err := z.state.GetL2BlocksByBatchNumber(ctx, numericBatchNumber, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load blocks associated to the batch %v", numericBatchNumber), err, true) + } - batch.Transactions = txs - rpcBatch, err := types.NewBatch(ctx, z.state, batch, virtualBatch, verifiedBatch, blocks, receipts, fullTx, true, ger, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build the batch %v response", batchNumber), err, true) - } - return rpcBatch, nil - }) + batch.Transactions = txs + rpcBatch, err := types.NewBatch(ctx, z.state, batch, virtualBatch, verifiedBatch, blocks, receipts, fullTx, true, ger, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build the batch %v response", numericBatchNumber), err, true) + } + return rpcBatch, nil } // GetFullBlockByNumber returns information about a block by block number func (z *ZKEVMEndpoints) GetFullBlockByNumber(number types.BlockNumber, fullTx bool) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if number == types.PendingBlockNumber { - lastBlock, err := z.state.GetLastL2Block(ctx, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block", err, true) - } - l2Header := state.NewL2Header(ðTypes.Header{ - ParentHash: lastBlock.Hash(), - Number: big.NewInt(0).SetUint64(lastBlock.Number().Uint64() + 1), - TxHash: ethTypes.EmptyRootHash, - UncleHash: ethTypes.EmptyUncleHash, - }) - l2Block := state.NewL2BlockWithHeader(l2Header) - rpcBlock, err := types.NewBlock(ctx, z.state, nil, l2Block, nil, fullTx, false, state.Ptr(true), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "couldn't build the pending block response", err, true) - } - - // clean fields that are not available for pending block - rpcBlock.Hash = nil - rpcBlock.Miner = nil - rpcBlock.Nonce = nil - rpcBlock.TotalDifficulty = nil - - return rpcBlock, nil - } - var err error - blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, z.state, z.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr + ctx := context.Background() + if number == types.PendingBlockNumber { + lastBlock, err := z.state.GetLastL2Block(ctx, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block", err, true) + } + l2Header := state.NewL2Header(ðTypes.Header{ + ParentHash: lastBlock.Hash(), + Number: big.NewInt(0).SetUint64(lastBlock.Number().Uint64() + 1), + TxHash: ethTypes.EmptyRootHash, + UncleHash: ethTypes.EmptyUncleHash, + }) + l2Block := state.NewL2BlockWithHeader(l2Header) + rpcBlock, err := types.NewBlock(ctx, z.state, nil, l2Block, nil, fullTx, false, state.Ptr(true), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "couldn't build the pending block response", err, true) } - l2Block, err := z.state.GetL2BlockByNumber(ctx, blockNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load block from state by number %v", blockNumber), err, true) - } + // clean fields that are not available for pending block + rpcBlock.Hash = nil + rpcBlock.Miner = nil + rpcBlock.Nonce = nil + rpcBlock.TotalDifficulty = nil - txs := l2Block.Transactions() - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) - } + return rpcBlock, nil + } + var err error + blockNumber, rpcErr := number.GetNumericBlockNumber(ctx, z.state, z.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } + + l2Block, err := z.state.GetL2BlockByNumber(ctx, blockNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load block from state by number %v", blockNumber), err, true) + } - rpcBlock, err := types.NewBlock(ctx, z.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, true, state.Ptr(true), dbTx) + txs := l2Block.Transactions() + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by number %v", blockNumber), err, true) + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - return rpcBlock, nil - }) + rpcBlock, err := types.NewBlock(ctx, z.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, true, state.Ptr(true), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by number %v", blockNumber), err, true) + } + + return rpcBlock, nil } // GetFullBlockByHash returns information about a block by hash func (z *ZKEVMEndpoints) GetFullBlockByHash(hash types.ArgHash, fullTx bool) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - l2Block, err := z.state.GetL2BlockByHash(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) - } - - txs := l2Block.Transactions() - receipts := make([]ethTypes.Receipt, 0, len(txs)) - for _, tx := range txs { - receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) - } - receipts = append(receipts, *receipt) - } + ctx := context.Background() + l2Block, err := z.state.GetL2BlockByHash(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) + } - rpcBlock, err := types.NewBlock(ctx, z.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, true, state.Ptr(true), dbTx) + txs := l2Block.Transactions() + receipts := make([]ethTypes.Receipt, 0, len(txs)) + for _, tx := range txs { + receipt, err := z.state.GetTransactionReceipt(ctx, tx.Hash(), nil) if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by hash %v", hash.Hash()), err, true) + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load receipt for tx %v", tx.Hash().String()), err, true) } + receipts = append(receipts, *receipt) + } - return rpcBlock, nil - }) + rpcBlock, err := types.NewBlock(ctx, z.state, state.Ptr(l2Block.Hash()), l2Block, receipts, fullTx, true, state.Ptr(true), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't build block response for block by hash %v", hash.Hash()), err, true) + } + + return rpcBlock, nil } // GetNativeBlockHashesInRange return the state root for the blocks in range func (z *ZKEVMEndpoints) GetNativeBlockHashesInRange(filter NativeBlockHashBlockRangeFilter) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - fromBlockNumber, toBlockNumber, rpcErr := filter.GetNumericBlockNumbers(ctx, z.cfg, z.state, z.etherman, dbTx) - if rpcErr != nil { - return nil, rpcErr - } + ctx := context.Background() + fromBlockNumber, toBlockNumber, rpcErr := filter.GetNumericBlockNumbers(ctx, z.cfg, z.state, z.etherman, nil) + if rpcErr != nil { + return nil, rpcErr + } - nativeBlockHashes, err := z.state.GetNativeBlockHashesInRange(ctx, fromBlockNumber, toBlockNumber, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if errors.Is(err, state.ErrMaxNativeBlockHashBlockRangeLimitExceeded) { - errMsg := fmt.Sprintf(state.ErrMaxNativeBlockHashBlockRangeLimitExceeded.Error(), z.cfg.MaxNativeBlockHashBlockRange) - return RPCErrorResponse(types.InvalidParamsErrorCode, errMsg, nil, false) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) - } + nativeBlockHashes, err := z.state.GetNativeBlockHashesInRange(ctx, fromBlockNumber, toBlockNumber, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if errors.Is(err, state.ErrMaxNativeBlockHashBlockRangeLimitExceeded) { + errMsg := fmt.Sprintf(state.ErrMaxNativeBlockHashBlockRangeLimitExceeded.Error(), z.cfg.MaxNativeBlockHashBlockRange) + return RPCErrorResponse(types.InvalidParamsErrorCode, errMsg, nil, false) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get block by hash from state", err, true) + } - return nativeBlockHashes, nil - }) + return nativeBlockHashes, nil } // GetTransactionByL2Hash returns a transaction by his l2 hash func (z *ZKEVMEndpoints) GetTransactionByL2Hash(hash types.ArgHash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - // try to get tx from state - tx, err := z.state.GetTransactionByL2Hash(ctx, hash.Hash(), dbTx) - if err != nil && !errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by l2 hash from state", err, true) + ctx := context.Background() + // try to get tx from state + tx, err := z.state.GetTransactionByL2Hash(ctx, hash.Hash(), nil) + if err != nil && !errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by l2 hash from state", err, true) + } + if tx != nil { + receipt, err := z.state.GetTransactionReceipt(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return RPCErrorResponse(types.DefaultErrorCode, "transaction receipt not found", err, false) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction receipt from state", err, true) } - if tx != nil { - receipt, err := z.state.GetTransactionReceipt(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return RPCErrorResponse(types.DefaultErrorCode, "transaction receipt not found", err, false) - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction receipt from state", err, true) - } - l2Hash, err := z.state.GetL2TxHashByTxHash(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) - } - - res, err := types.NewTransaction(*tx, receipt, false, l2Hash) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) - } - - return res, nil + l2Hash, err := z.state.GetL2TxHashByTxHash(ctx, tx.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) } - // if the tx does not exist in the state, look for it in the pool - if z.cfg.SequencerNodeURI != "" { - return z.getTransactionByL2HashFromSequencerNode(hash.Hash()) - } - poolTx, err := z.pool.GetTransactionByL2Hash(ctx, hash.Hash()) - if errors.Is(err, pool.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by l2 hash from pool", err, true) - } - if poolTx.Status == pool.TxStatusPending { - tx = &poolTx.Transaction - res, err := types.NewTransaction(*tx, nil, false, nil) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) - } - return res, nil + res, err := types.NewTransaction(*tx, receipt, false, l2Hash) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) } + + return res, nil + } + + // if the tx does not exist in the state, look for it in the pool + if z.cfg.SequencerNodeURI != "" { + return z.getTransactionByL2HashFromSequencerNode(hash.Hash()) + } + poolTx, err := z.pool.GetTransactionByL2Hash(ctx, hash.Hash()) + if errors.Is(err, pool.ErrNotFound) { return nil, nil - }) + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to load transaction by l2 hash from pool", err, true) + } + if poolTx.Status == pool.TxStatusPending { + tx = &poolTx.Transaction + res, err := types.NewTransaction(*tx, nil, false, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build transaction response", err, true) + } + return res, nil + } + return nil, nil } // GetTransactionReceiptByL2Hash returns a transaction receipt by his hash func (z *ZKEVMEndpoints) GetTransactionReceiptByL2Hash(hash types.ArgHash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - tx, err := z.state.GetTransactionByL2Hash(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx from state", err, true) - } + ctx := context.Background() + tx, err := z.state.GetTransactionByL2Hash(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx from state", err, true) + } - r, err := z.state.GetTransactionReceipt(ctx, hash.Hash(), dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx receipt from state", err, true) - } + r, err := z.state.GetTransactionReceipt(ctx, hash.Hash(), nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get tx receipt from state", err, true) + } - l2Hash, err := z.state.GetL2TxHashByTxHash(ctx, tx.Hash(), dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) - } + l2Hash, err := z.state.GetL2TxHashByTxHash(ctx, tx.Hash(), nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get l2 transaction hash", err, true) + } - receipt, err := types.NewReceipt(*tx, r, l2Hash) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to build the receipt response", err, true) - } + receipt, err := types.NewReceipt(*tx, r, l2Hash) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to build the receipt response", err, true) + } - return receipt, nil - }) + return receipt, nil } func (z *ZKEVMEndpoints) getTransactionByL2HashFromSequencerNode(hash common.Hash) (interface{}, types.Error) { @@ -417,43 +403,40 @@ func (z *ZKEVMEndpoints) getTransactionByL2HashFromSequencerNode(hash common.Has // GetExitRootsByGER returns the exit roots accordingly to the provided Global Exit Root func (z *ZKEVMEndpoints) GetExitRootsByGER(globalExitRoot common.Hash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - exitRoots, err := z.state.GetExitRootByGlobalExitRoot(ctx, globalExitRoot, dbTx) - if errors.Is(err, state.ErrNotFound) { - return nil, nil - } else if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to get exit roots by global exit root from state", err, true) - } + ctx := context.Background() + exitRoots, err := z.state.GetExitRootByGlobalExitRoot(ctx, globalExitRoot, nil) + if errors.Is(err, state.ErrNotFound) { + return nil, nil + } else if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to get exit roots by global exit root from state", err, true) + } - return types.ExitRoots{ - BlockNumber: types.ArgUint64(exitRoots.BlockNumber), - Timestamp: types.ArgUint64(exitRoots.Timestamp.Unix()), - MainnetExitRoot: exitRoots.MainnetExitRoot, - RollupExitRoot: exitRoots.RollupExitRoot, - }, nil - }) + return types.ExitRoots{ + BlockNumber: types.ArgUint64(exitRoots.BlockNumber), + Timestamp: types.ArgUint64(exitRoots.Timestamp.Unix()), + MainnetExitRoot: exitRoots.MainnetExitRoot, + RollupExitRoot: exitRoots.RollupExitRoot, + }, nil } // EstimateGasPrice returns an estimate gas price for the transaction. func (z *ZKEVMEndpoints) EstimateGasPrice(arg *types.TxArgs, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - gasPrice, _, err := z.internalEstimateGasPriceAndFee(ctx, arg, blockArg, dbTx) - if err != nil { - return nil, err - } - return hex.EncodeBig(gasPrice), nil - }) + ctx := context.Background() + gasPrice, _, err := z.internalEstimateGasPriceAndFee(ctx, arg, blockArg, nil) + if err != nil { + return nil, err + } + return hex.EncodeBig(gasPrice), nil } // EstimateFee returns an estimate fee for the transaction. func (z *ZKEVMEndpoints) EstimateFee(arg *types.TxArgs, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - _, fee, err := z.internalEstimateGasPriceAndFee(ctx, arg, blockArg, dbTx) - if err != nil { - return nil, err - } - return hex.EncodeBig(fee), nil - }) + ctx := context.Background() + _, fee, err := z.internalEstimateGasPriceAndFee(ctx, arg, blockArg, nil) + if err != nil { + return nil, err + } + return hex.EncodeBig(fee), nil } // internalEstimateGasPriceAndFee computes the estimated gas price and the estimated fee for the transaction @@ -462,7 +445,7 @@ func (z *ZKEVMEndpoints) internalEstimateGasPriceAndFee(ctx context.Context, arg return nil, nil, types.NewRPCError(types.InvalidParamsErrorCode, "missing value for required argument 0") } - block, respErr := z.getBlockByArg(ctx, blockArg, dbTx) + block, respErr := z.getBlockByArg(ctx, blockArg, nil) if respErr != nil { return nil, nil, respErr } @@ -479,12 +462,12 @@ func (z *ZKEVMEndpoints) internalEstimateGasPriceAndFee(ctx context.Context, arg } defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) - sender, tx, err := arg.ToTransaction(ctx, z.state, z.cfg.MaxCumulativeGasUsed, block.Root(), defaultSenderAddress, dbTx) + sender, tx, err := arg.ToTransaction(ctx, z.state, z.cfg.MaxCumulativeGasUsed, block.Root(), defaultSenderAddress, nil) if err != nil { return nil, nil, types.NewRPCError(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction") } - gasEstimation, returnValue, err := z.state.EstimateGas(tx, sender, blockToProcess, dbTx) + gasEstimation, returnValue, err := z.state.EstimateGas(tx, sender, blockToProcess, nil) if errors.Is(err, runtime.ErrExecutionReverted) { data := make([]byte, len(returnValue)) copy(data, returnValue) @@ -540,78 +523,77 @@ func (z *ZKEVMEndpoints) internalEstimateGasPriceAndFee(ctx context.Context, arg // EstimateCounters returns an estimation of the counters that are going to be used while executing // this transaction. func (z *ZKEVMEndpoints) EstimateCounters(arg *types.TxArgs, blockArg *types.BlockNumberOrHash) (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - if arg == nil { - return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) - } + ctx := context.Background() + if arg == nil { + return RPCErrorResponse(types.InvalidParamsErrorCode, "missing value for required argument 0", nil, false) + } - block, respErr := z.getBlockByArg(ctx, blockArg, dbTx) - if respErr != nil { - return nil, respErr - } + block, respErr := z.getBlockByArg(ctx, blockArg, nil) + if respErr != nil { + return nil, respErr + } - var blockToProcess *uint64 - if blockArg != nil { - blockNumArg := blockArg.Number() - if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { - blockToProcess = nil - } else { - n := block.NumberU64() - blockToProcess = &n - } + var blockToProcess *uint64 + if blockArg != nil { + blockNumArg := blockArg.Number() + if blockNumArg != nil && (*blockArg.Number() == types.LatestBlockNumber || *blockArg.Number() == types.PendingBlockNumber) { + blockToProcess = nil + } else { + n := block.NumberU64() + blockToProcess = &n } + } - defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) - sender, tx, err := arg.ToTransaction(ctx, z.state, z.cfg.MaxCumulativeGasUsed, block.Root(), defaultSenderAddress, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) - } + defaultSenderAddress := common.HexToAddress(state.DefaultSenderAddress) + sender, tx, err := arg.ToTransaction(ctx, z.state, z.cfg.MaxCumulativeGasUsed, block.Root(), defaultSenderAddress, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "failed to convert arguments into an unsigned transaction", err, false) + } - var oocErr error - processBatchResponse, err := z.state.PreProcessUnsignedTransaction(ctx, tx, sender, blockToProcess, dbTx) - if err != nil { - if executor.IsROMOutOfCountersError(executor.RomErrorCode(err)) { - oocErr = err - } else { - errMsg := fmt.Sprintf("failed to estimate counters: %v", err.Error()) - return nil, types.NewRPCError(types.DefaultErrorCode, errMsg) - } + var oocErr error + processBatchResponse, err := z.state.PreProcessUnsignedTransaction(ctx, tx, sender, blockToProcess, nil) + if err != nil { + if executor.IsROMOutOfCountersError(executor.RomErrorCode(err)) { + oocErr = err + } else { + errMsg := fmt.Sprintf("failed to estimate counters: %v", err.Error()) + return nil, types.NewRPCError(types.DefaultErrorCode, errMsg) } + } - var revert *types.RevertInfo - if len(processBatchResponse.BlockResponses) > 0 && len(processBatchResponse.BlockResponses[0].TransactionResponses) > 0 { - txResponse := processBatchResponse.BlockResponses[0].TransactionResponses[0] - err = txResponse.RomError - if errors.Is(err, runtime.ErrExecutionReverted) { - returnValue := make([]byte, len(txResponse.ReturnValue)) - copy(returnValue, txResponse.ReturnValue) - err := state.ConstructErrorFromRevert(err, returnValue) - revert = &types.RevertInfo{ - Message: err.Error(), - Data: state.Ptr(types.ArgBytes(returnValue)), - } + var revert *types.RevertInfo + if len(processBatchResponse.BlockResponses) > 0 && len(processBatchResponse.BlockResponses[0].TransactionResponses) > 0 { + txResponse := processBatchResponse.BlockResponses[0].TransactionResponses[0] + err = txResponse.RomError + if errors.Is(err, runtime.ErrExecutionReverted) { + returnValue := make([]byte, len(txResponse.ReturnValue)) + copy(returnValue, txResponse.ReturnValue) + err := state.ConstructErrorFromRevert(err, returnValue) + revert = &types.RevertInfo{ + Message: err.Error(), + Data: state.Ptr(types.ArgBytes(returnValue)), } } + } - limits := types.ZKCountersLimits{ - MaxGasUsed: types.ArgUint64(state.MaxTxGasLimit), - MaxKeccakHashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxKeccakHashes), - MaxPoseidonHashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxPoseidonHashes), - MaxPoseidonPaddings: types.ArgUint64(z.cfg.ZKCountersLimits.MaxPoseidonPaddings), - MaxMemAligns: types.ArgUint64(z.cfg.ZKCountersLimits.MaxMemAligns), - MaxArithmetics: types.ArgUint64(z.cfg.ZKCountersLimits.MaxArithmetics), - MaxBinaries: types.ArgUint64(z.cfg.ZKCountersLimits.MaxBinaries), - MaxSteps: types.ArgUint64(z.cfg.ZKCountersLimits.MaxSteps), - MaxSHA256Hashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxSHA256Hashes), - } - return types.NewZKCountersResponse(processBatchResponse.UsedZkCounters, limits, revert, oocErr), nil - }) + limits := types.ZKCountersLimits{ + MaxGasUsed: types.ArgUint64(state.MaxTxGasLimit), + MaxKeccakHashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxKeccakHashes), + MaxPoseidonHashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxPoseidonHashes), + MaxPoseidonPaddings: types.ArgUint64(z.cfg.ZKCountersLimits.MaxPoseidonPaddings), + MaxMemAligns: types.ArgUint64(z.cfg.ZKCountersLimits.MaxMemAligns), + MaxArithmetics: types.ArgUint64(z.cfg.ZKCountersLimits.MaxArithmetics), + MaxBinaries: types.ArgUint64(z.cfg.ZKCountersLimits.MaxBinaries), + MaxSteps: types.ArgUint64(z.cfg.ZKCountersLimits.MaxSteps), + MaxSHA256Hashes: types.ArgUint64(z.cfg.ZKCountersLimits.MaxSHA256Hashes), + } + return types.NewZKCountersResponse(processBatchResponse.UsedZkCounters, limits, revert, oocErr), nil } func (z *ZKEVMEndpoints) getBlockByArg(ctx context.Context, blockArg *types.BlockNumberOrHash, dbTx pgx.Tx) (*state.L2Block, types.Error) { // If no block argument is provided, return the latest block if blockArg == nil { - block, err := z.state.GetLastL2Block(ctx, dbTx) + block, err := z.state.GetLastL2Block(ctx, nil) if err != nil { return nil, types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state") } @@ -620,7 +602,7 @@ func (z *ZKEVMEndpoints) getBlockByArg(ctx context.Context, blockArg *types.Bloc // If we have a block hash, try to get the block by hash if blockArg.IsHash() { - block, err := z.state.GetL2BlockByHash(ctx, blockArg.Hash().Hash(), dbTx) + block, err := z.state.GetL2BlockByHash(ctx, blockArg.Hash().Hash(), nil) if errors.Is(err, state.ErrNotFound) { return nil, types.NewRPCError(types.DefaultErrorCode, "header for hash not found") } else if err != nil { @@ -630,11 +612,11 @@ func (z *ZKEVMEndpoints) getBlockByArg(ctx context.Context, blockArg *types.Bloc } // Otherwise, try to get the block by number - blockNum, rpcErr := blockArg.Number().GetNumericBlockNumber(ctx, z.state, z.etherman, dbTx) + blockNum, rpcErr := blockArg.Number().GetNumericBlockNumber(ctx, z.state, z.etherman, nil) if rpcErr != nil { return nil, rpcErr } - block, err := z.state.GetL2BlockByNumber(context.Background(), blockNum, dbTx) + block, err := z.state.GetL2BlockByNumber(context.Background(), blockNum, nil) if errors.Is(err, state.ErrNotFound) || block == nil { return nil, types.NewRPCError(types.DefaultErrorCode, "header not found") } else if err != nil { @@ -646,14 +628,13 @@ func (z *ZKEVMEndpoints) getBlockByArg(ctx context.Context, blockArg *types.Bloc // GetLatestGlobalExitRoot returns the last global exit root used by l2 func (z *ZKEVMEndpoints) GetLatestGlobalExitRoot() (interface{}, types.Error) { - return z.txMan.NewDbTxScope(z.state, func(ctx context.Context, dbTx pgx.Tx) (interface{}, types.Error) { - var err error + ctx := context.Background() + var err error - ger, err := z.state.GetLatestBatchGlobalExitRoot(ctx, dbTx) - if err != nil { - return RPCErrorResponse(types.DefaultErrorCode, "couldn't load the last global exit root", err, true) - } + ger, err := z.state.GetLatestBatchGlobalExitRoot(ctx, nil) + if err != nil { + return RPCErrorResponse(types.DefaultErrorCode, "couldn't load the last global exit root", err, true) + } - return ger.String(), nil - }) + return ger.String(), nil } diff --git a/jsonrpc/endpoints_zkevm_test.go b/jsonrpc/endpoints_zkevm_test.go index 8c0090d1e2..ff2761158f 100644 --- a/jsonrpc/endpoints_zkevm_test.go +++ b/jsonrpc/endpoints_zkevm_test.go @@ -46,18 +46,8 @@ func TestConsolidatedBlockNumber(t *testing.T) { Name: "Get consolidated block number successfully", ExpectedResult: state.Ptr(uint64(10)), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastConsolidatedL2BlockNumber", context.Background(), m.DbTx). + On("GetLastConsolidatedL2BlockNumber", context.Background(), nil). Return(uint64(10), nil). Once() }, @@ -67,18 +57,8 @@ func TestConsolidatedBlockNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get last consolidated block number from state"), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastConsolidatedL2BlockNumber", context.Background(), m.DbTx). + On("GetLastConsolidatedL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last consolidated block number")). Once() }, @@ -124,18 +104,8 @@ func TestIsBlockConsolidated(t *testing.T) { Name: "Query status of block number successfully", ExpectedResult: true, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("IsL2BlockConsolidated", context.Background(), uint64(1), m.DbTx). + On("IsL2BlockConsolidated", context.Background(), uint64(1), nil). Return(true, nil). Once() }, @@ -145,18 +115,8 @@ func TestIsBlockConsolidated(t *testing.T) { ExpectedResult: false, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to check if the block is consolidated"), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("IsL2BlockConsolidated", context.Background(), uint64(1), m.DbTx). + On("IsL2BlockConsolidated", context.Background(), uint64(1), nil). Return(false, errors.New("failed to check if the block is consolidated")). Once() }, @@ -202,18 +162,8 @@ func TestIsBlockVirtualized(t *testing.T) { Name: "Query status of block number successfully", ExpectedResult: true, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("IsL2BlockVirtualized", context.Background(), uint64(1), m.DbTx). + On("IsL2BlockVirtualized", context.Background(), uint64(1), nil). Return(true, nil). Once() }, @@ -223,18 +173,8 @@ func TestIsBlockVirtualized(t *testing.T) { ExpectedResult: false, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to check if the block is virtualized"), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("IsL2BlockVirtualized", context.Background(), uint64(1), m.DbTx). + On("IsL2BlockVirtualized", context.Background(), uint64(1), nil). Return(false, errors.New("failed to check if the block is virtualized")). Once() }, @@ -282,18 +222,8 @@ func TestBatchNumberByBlockNumber(t *testing.T) { Name: "get batch number by block number successfully", ExpectedResult: &batchNumber, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, m.DbTx). + On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, nil). Return(batchNumber, nil). Once() }, @@ -303,18 +233,8 @@ func TestBatchNumberByBlockNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get batch number from block number"), SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, m.DbTx). + On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, nil). Return(uint64(0), errors.New("failed to get batch number of l2 batchNum")). Once() }, @@ -324,18 +244,8 @@ func TestBatchNumberByBlockNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, m.DbTx). + On("BatchNumberByL2BlockNumber", context.Background(), blockNumber, nil). Return(uint64(0), state.ErrNotFound). Once() }, @@ -393,18 +303,8 @@ func TestBatchNumber(t *testing.T) { ExpectedError: nil, ExpectedResult: 10, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastBatchNumber", context.Background(), m.DbTx). + On("GetLastBatchNumber", context.Background(), nil). Return(uint64(10), nil). Once() }, @@ -414,18 +314,8 @@ func TestBatchNumber(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last batch number from state"), ExpectedResult: 0, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastBatchNumber", context.Background(), m.DbTx). + On("GetLastBatchNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last batch number")). Once() }, @@ -472,18 +362,8 @@ func TestVirtualBatchNumber(t *testing.T) { ExpectedError: nil, ExpectedResult: 10, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastVirtualBatchNum", context.Background(), m.DbTx). + On("GetLastVirtualBatchNum", context.Background(), nil). Return(uint64(10), nil). Once() }, @@ -493,18 +373,8 @@ func TestVirtualBatchNumber(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last virtual batch number from state"), ExpectedResult: 0, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastVirtualBatchNum", context.Background(), m.DbTx). + On("GetLastVirtualBatchNum", context.Background(), nil). Return(uint64(0), errors.New("failed to get last batch number")). Once() }, @@ -551,18 +421,8 @@ func TestVerifiedBatchNumber(t *testing.T) { ExpectedError: nil, ExpectedResult: 10, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastVerifiedBatch", context.Background(), m.DbTx). + On("GetLastVerifiedBatch", context.Background(), nil). Return(&state.VerifiedBatch{BatchNumber: uint64(10)}, nil). Once() }, @@ -572,18 +432,8 @@ func TestVerifiedBatchNumber(t *testing.T) { ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last verified batch number from state"), ExpectedResult: 0, SetupMocks: func(m *mocksWrapper) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastVerifiedBatch", context.Background(), m.DbTx). + On("GetLastVerifiedBatch", context.Background(), nil). Return(nil, errors.New("failed to get last batch number")). Once() }, @@ -630,18 +480,8 @@ func TestGetBatchByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(nil, state.ErrNotFound) }, }, @@ -661,16 +501,6 @@ func TestGetBatchByNumber(t *testing.T) { }, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - txs := []*ethTypes.Transaction{ signTx(ethTypes.NewTransaction(1001, common.HexToAddress("0x1000"), big.NewInt(1000), 1001, big.NewInt(1002), []byte("1003")), s.ChainID()), signTx(ethTypes.NewTransaction(1002, common.HexToAddress("0x1000"), big.NewInt(1000), 1001, big.NewInt(1002), []byte("1003")), s.ChainID()), @@ -733,12 +563,12 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(batch, nil). Once() m.State. - On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx). + On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), nil). Return(&batch.Timestamp, nil). Once() @@ -747,7 +577,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVirtualBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetVirtualBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(virtualBatch, nil). Once() @@ -756,7 +586,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVerifiedBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetVerifiedBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(verifiedBatch, nil). Once() @@ -766,27 +596,27 @@ func TestGetBatchByNumber(t *testing.T) { GlobalExitRoot: common.HexToHash("0x4"), } m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, nil). Return(&ger, nil). Once() for i, tx := range txs { m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(receipts[i], nil). Once() m.State. - On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), nil). Return(state.Ptr(tx.Hash()), nil). Once() } m.State. - On("GetTransactionsByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetTransactionsByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(batchTxs, effectivePercentages, nil). Once() m.State. - On("GetL2BlocksByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetL2BlocksByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(blocks, nil). Once() }, @@ -807,16 +637,6 @@ func TestGetBatchByNumber(t *testing.T) { }, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - txs := []*ethTypes.Transaction{ signTx(ethTypes.NewTransaction(1001, common.HexToAddress("0x1000"), big.NewInt(1000), 1001, big.NewInt(1002), []byte("1003")), s.ChainID()), signTx(ethTypes.NewTransaction(1002, common.HexToAddress("0x1000"), big.NewInt(1000), 1001, big.NewInt(1002), []byte("1003")), s.ChainID()), @@ -861,12 +681,12 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetBatchByNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(batch, nil). Once() m.State. - On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx). + On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), nil). Return(&batch.Timestamp, nil). Once() @@ -875,7 +695,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVirtualBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetVirtualBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(virtualBatch, nil). Once() @@ -884,7 +704,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVerifiedBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetVerifiedBatch", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(verifiedBatch, nil). Once() @@ -894,22 +714,22 @@ func TestGetBatchByNumber(t *testing.T) { GlobalExitRoot: common.HexToHash("0x4"), } m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, nil). Return(&ger, nil). Once() for i, tx := range txs { m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(receipts[i], nil). Once() } m.State. - On("GetTransactionsByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetTransactionsByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(batchTxs, effectivePercentages, nil). Once() m.State. - On("GetL2BlocksByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), m.DbTx). + On("GetL2BlocksByBatchNumber", context.Background(), hex.DecodeBig(tc.Number).Uint64(), nil). Return(blocks, nil). Once() @@ -933,18 +753,8 @@ func TestGetBatchByNumber(t *testing.T) { }, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastClosedBatchNumber", context.Background(), m.DbTx). + On("GetLastClosedBatchNumber", context.Background(), nil). Return(uint64(tc.ExpectedResult.Number), nil). Once() @@ -1018,12 +828,12 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetBatchByNumber", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetBatchByNumber", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(batch, nil). Once() m.State. - On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx). + On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), nil). Return(&batch.Timestamp, nil). Once() @@ -1032,7 +842,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVirtualBatch", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetVirtualBatch", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(virtualBatch, nil). Once() @@ -1041,7 +851,7 @@ func TestGetBatchByNumber(t *testing.T) { } m.State. - On("GetVerifiedBatch", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetVerifiedBatch", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(verifiedBatch, nil). Once() @@ -1051,28 +861,28 @@ func TestGetBatchByNumber(t *testing.T) { GlobalExitRoot: common.HexToHash("0x4"), } m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), batch.GlobalExitRoot, nil). Return(&ger, nil). Once() for i, tx := range txs { m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(receipts[i], nil). Once() m.State. - On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), nil). Return(state.Ptr(tx.Hash()), nil). Once() } m.State. - On("GetTransactionsByBatchNumber", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetTransactionsByBatchNumber", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(batchTxs, effectivePercentages, nil). Once() m.State. - On("GetL2BlocksByBatchNumber", context.Background(), uint64(tc.ExpectedResult.Number), m.DbTx). + On("GetL2BlocksByBatchNumber", context.Background(), uint64(tc.ExpectedResult.Number), nil). Return(blocks, nil). Once() tc.ExpectedResult.BatchL2Data = batchL2Data @@ -1084,18 +894,8 @@ func TestGetBatchByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last batch number from state"), SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastClosedBatchNumber", context.Background(), m.DbTx). + On("GetLastClosedBatchNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last batch number")). Once() }, @@ -1106,23 +906,13 @@ func TestGetBatchByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load batch from state by number 1"), SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastClosedBatchNumber", context.Background(), m.DbTx). + On("GetLastClosedBatchNumber", context.Background(), nil). Return(uint64(1), nil). Once() m.State. - On("GetBatchByNumber", context.Background(), uint64(1), m.DbTx). + On("GetBatchByNumber", context.Background(), uint64(1), nil). Return(nil, errors.New("failed to load batch by number")). Once() }, @@ -1218,18 +1008,8 @@ func TestGetL2FullBlockByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound) }, }, @@ -1239,18 +1019,8 @@ func TestGetL2FullBlockByHash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get block by hash from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get block from state")). Once() }, @@ -1274,24 +1044,14 @@ func TestGetL2FullBlockByHash(t *testing.T) { st := trie.NewStackTrie(nil) block := state.NewL2Block(state.NewL2Header(tc.ExpectedResult.Header()), tc.ExpectedResult.Transactions(), uncles, []*ethTypes.Receipt{ethTypes.NewReceipt([]byte{}, false, uint64(0))}, st) - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByHash", context.Background(), tc.Hash, m.DbTx). + On("GetL2BlockByHash", context.Background(), tc.Hash, nil). Return(block, nil). Once() for _, tx := range tc.ExpectedResult.Transactions() { m.State. - On("GetTransactionReceipt", context.Background(), tx.Hash(), m.DbTx). + On("GetTransactionReceipt", context.Background(), tx.Hash(), nil). Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil). Once() } @@ -1486,18 +1246,8 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetL2BlockByNumber", context.Background(), hex.DecodeUint64(tc.Number), m.DbTx). + On("GetL2BlockByNumber", context.Background(), hex.DecodeUint64(tc.Number), nil). Return(nil, state.ErrNotFound). Once() }, @@ -1508,24 +1258,14 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: rpcBlock, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetL2BlockByNumber", context.Background(), hex.DecodeUint64(tc.Number), m.DbTx). + On("GetL2BlockByNumber", context.Background(), hex.DecodeUint64(tc.Number), nil). Return(l2Block, nil). Once() for _, receipt := range receipts { m.State. - On("GetTransactionReceipt", context.Background(), receipt.TxHash, m.DbTx). + On("GetTransactionReceipt", context.Background(), receipt.TxHash, nil). Return(receipt, nil). Once() } @@ -1537,31 +1277,21 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: rpcBlock, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - blockNumber := uint64(1) m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(blockNumber, nil). Once() m.State. - On("GetL2BlockByNumber", context.Background(), blockNumber, m.DbTx). + On("GetL2BlockByNumber", context.Background(), blockNumber, nil). Return(l2Block, nil). Once() for _, receipt := range receipts { m.State. - On("GetTransactionReceipt", context.Background(), receipt.TxHash, m.DbTx). + On("GetTransactionReceipt", context.Background(), receipt.TxHash, nil). Return(receipt, nil). Once() } @@ -1573,18 +1303,8 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get the last block number from state"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(0), errors.New("failed to get last block number")). Once() }, @@ -1595,23 +1315,13 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load block from state by number 1"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2BlockNumber", context.Background(), m.DbTx). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(1), nil). Once() m.State. - On("GetL2BlockByNumber", context.Background(), uint64(1), m.DbTx). + On("GetL2BlockByNumber", context.Background(), uint64(1), nil). Return(nil, errors.New("failed to load block by number")). Once() }, @@ -1640,18 +1350,8 @@ func TestGetL2FullBlockByNumber(t *testing.T) { tc.ExpectedResult.Nonce = nil tc.ExpectedResult.TotalDifficulty = nil - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(lastBlock, nil). Once() }, @@ -1662,18 +1362,8 @@ func TestGetL2FullBlockByNumber(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load last block from state to compute the pending block"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLastL2Block", context.Background(), m.DbTx). + On("GetLastL2Block", context.Background(), nil). Return(nil, errors.New("failed to load last block")). Once() }, @@ -1769,21 +1459,11 @@ func TestGetNativeBlockHashesInRange(t *testing.T) { ExpectedResult: state.Ptr([]string{}), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - fromBlock, _ := tc.Filter.FromBlock.GetNumericBlockNumber(context.Background(), nil, nil, nil) toBlock, _ := tc.Filter.ToBlock.GetNumericBlockNumber(context.Background(), nil, nil, nil) m.State. - On("GetNativeBlockHashesInRange", context.Background(), fromBlock, toBlock, m.DbTx). + On("GetNativeBlockHashesInRange", context.Background(), fromBlock, toBlock, nil). Return([]common.Hash{}, nil). Once() }, @@ -1797,16 +1477,6 @@ func TestGetNativeBlockHashesInRange(t *testing.T) { ExpectedResult: state.Ptr([]string{}), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - fromBlock, _ := tc.Filter.FromBlock.GetNumericBlockNumber(context.Background(), nil, nil, nil) toBlock, _ := tc.Filter.ToBlock.GetNumericBlockNumber(context.Background(), nil, nil, nil) hashes := []common.Hash{} @@ -1820,7 +1490,7 @@ func TestGetNativeBlockHashesInRange(t *testing.T) { tc.ExpectedResult = &expectedResult m.State. - On("GetNativeBlockHashesInRange", context.Background(), fromBlock, toBlock, m.DbTx). + On("GetNativeBlockHashesInRange", context.Background(), fromBlock, toBlock, nil). Return(hashes, nil). Once() }, @@ -1834,15 +1504,7 @@ func TestGetNativeBlockHashesInRange(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.InvalidParamsErrorCode, "invalid block range"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, { @@ -1854,15 +1516,7 @@ func TestGetNativeBlockHashesInRange(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.InvalidParamsErrorCode, "native block hashes are limited to a 60000 block range"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() }, }, } @@ -1969,28 +1623,18 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: &rpcTransaction, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(receipt, nil). Once() m.State. - On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), nil). Return(&l2Hash, nil). Once() }, @@ -2007,18 +1651,8 @@ func TestGetTransactionByL2Hash(t *testing.T) { tc.ExpectedResult.TxIndex = nil tc.ExpectedResult.L2Hash = nil - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2035,18 +1669,8 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2063,18 +1687,8 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction by l2 hash from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to load transaction by l2 hash from state")). Once() }, @@ -2086,18 +1700,8 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction by l2 hash from pool"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() @@ -2114,23 +1718,13 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "transaction receipt not found"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -2142,23 +1736,13 @@ func TestGetTransactionByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to load transaction receipt from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to load transaction receipt from state")). Once() }, @@ -2285,28 +1869,18 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: &rpcReceipt, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(receipt, nil). Once() m.State. - On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), signedTx.Hash(), nil). Return(&l2Hash, nil). Once() }, @@ -2317,18 +1891,8 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -2339,18 +1903,8 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get tx from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get tx")). Once() }, @@ -2361,23 +1915,13 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, state.ErrNotFound). Once() }, @@ -2388,23 +1932,13 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to get tx receipt from state"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(signedTx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(nil, errors.New("failed to get tx receipt from state")). Once() }, @@ -2415,28 +1949,18 @@ func TestGetTransactionReceiptByL2Hash(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "failed to build the receipt response"), SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetTransactionByL2Hash", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionByL2Hash", context.Background(), tc.Hash, nil). Return(tx, nil). Once() m.State. - On("GetTransactionReceipt", context.Background(), tc.Hash, m.DbTx). + On("GetTransactionReceipt", context.Background(), tc.Hash, nil). Return(ethTypes.NewReceipt([]byte{}, false, 0), nil). Once() m.State. - On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), m.DbTx). + On("GetL2TxHashByTxHash", context.Background(), tx.Hash(), nil). Return(&l2Hash, nil). Once() }, @@ -2530,18 +2054,8 @@ func TestGetExitRootsByGER(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, nil). Return(nil, state.ErrNotFound) }, }, @@ -2551,18 +2065,8 @@ func TestGetExitRootsByGER(t *testing.T) { ExpectedResult: nil, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, nil). Return(nil, fmt.Errorf("failed to load exit roots from state")) }, }, @@ -2577,15 +2081,6 @@ func TestGetExitRootsByGER(t *testing.T) { }, ExpectedError: nil, SetupMocks: func(s *mockedServer, m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() er := &state.GlobalExitRoot{ BlockNumber: uint64(tc.ExpectedResult.BlockNumber), Timestamp: time.Unix(int64(tc.ExpectedResult.Timestamp), 0), @@ -2594,7 +2089,7 @@ func TestGetExitRootsByGER(t *testing.T) { } m.State. - On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, m.DbTx). + On("GetExitRootByGlobalExitRoot", context.Background(), tc.GER, nil). Return(er, nil) }, }, @@ -2642,18 +2137,8 @@ func TestGetLatestGlobalExitRoot(t *testing.T) { ExpectedResult: nil, ExpectedError: types.NewRPCError(types.DefaultErrorCode, "couldn't load the last global exit root"), SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Rollback", context.Background()). - Return(nil). - Once() - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - - m.State. - On("GetLatestBatchGlobalExitRoot", context.Background(), m.DbTx). + On("GetLatestBatchGlobalExitRoot", context.Background(), nil). Return(nil, fmt.Errorf("failed to load GER from state")). Once() }, @@ -2663,18 +2148,8 @@ func TestGetLatestGlobalExitRoot(t *testing.T) { ExpectedResult: state.Ptr(common.HexToHash("0x1")), ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc *testCase) { - m.DbTx. - On("Commit", context.Background()). - Return(nil). - Once() - - m.State. - On("BeginStateTransaction", context.Background()). - Return(m.DbTx, nil). - Once() - m.State. - On("GetLatestBatchGlobalExitRoot", context.Background(), m.DbTx). + On("GetLatestBatchGlobalExitRoot", context.Background(), nil). Return(common.HexToHash("0x1"), nil). Once() }, diff --git a/jsonrpc/mocks/mock_dbtx.go b/jsonrpc/mocks/mock_dbtx.go deleted file mode 100644 index c17e30a024..0000000000 --- a/jsonrpc/mocks/mock_dbtx.go +++ /dev/null @@ -1,350 +0,0 @@ -// Code generated by mockery v2.39.0. DO NOT EDIT. - -package mocks - -import ( - context "context" - - pgconn "github.com/jackc/pgconn" - mock "github.com/stretchr/testify/mock" - - pgx "github.com/jackc/pgx/v4" -) - -// DBTxMock is an autogenerated mock type for the Tx type -type DBTxMock struct { - mock.Mock -} - -// Begin provides a mock function with given fields: ctx -func (_m *DBTxMock) Begin(ctx context.Context) (pgx.Tx, error) { - ret := _m.Called(ctx) - - if len(ret) == 0 { - panic("no return value specified for Begin") - } - - var r0 pgx.Tx - var r1 error - if rf, ok := ret.Get(0).(func(context.Context) (pgx.Tx, error)); ok { - return rf(ctx) - } - if rf, ok := ret.Get(0).(func(context.Context) pgx.Tx); ok { - r0 = rf(ctx) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgx.Tx) - } - } - - if rf, ok := ret.Get(1).(func(context.Context) error); ok { - r1 = rf(ctx) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// BeginFunc provides a mock function with given fields: ctx, f -func (_m *DBTxMock) BeginFunc(ctx context.Context, f func(pgx.Tx) error) error { - ret := _m.Called(ctx, f) - - if len(ret) == 0 { - panic("no return value specified for BeginFunc") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, func(pgx.Tx) error) error); ok { - r0 = rf(ctx, f) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Commit provides a mock function with given fields: ctx -func (_m *DBTxMock) Commit(ctx context.Context) error { - ret := _m.Called(ctx) - - if len(ret) == 0 { - panic("no return value specified for Commit") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context) error); ok { - r0 = rf(ctx) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Conn provides a mock function with given fields: -func (_m *DBTxMock) Conn() *pgx.Conn { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for Conn") - } - - var r0 *pgx.Conn - if rf, ok := ret.Get(0).(func() *pgx.Conn); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*pgx.Conn) - } - } - - return r0 -} - -// CopyFrom provides a mock function with given fields: ctx, tableName, columnNames, rowSrc -func (_m *DBTxMock) CopyFrom(ctx context.Context, tableName pgx.Identifier, columnNames []string, rowSrc pgx.CopyFromSource) (int64, error) { - ret := _m.Called(ctx, tableName, columnNames, rowSrc) - - if len(ret) == 0 { - panic("no return value specified for CopyFrom") - } - - var r0 int64 - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, pgx.Identifier, []string, pgx.CopyFromSource) (int64, error)); ok { - return rf(ctx, tableName, columnNames, rowSrc) - } - if rf, ok := ret.Get(0).(func(context.Context, pgx.Identifier, []string, pgx.CopyFromSource) int64); ok { - r0 = rf(ctx, tableName, columnNames, rowSrc) - } else { - r0 = ret.Get(0).(int64) - } - - if rf, ok := ret.Get(1).(func(context.Context, pgx.Identifier, []string, pgx.CopyFromSource) error); ok { - r1 = rf(ctx, tableName, columnNames, rowSrc) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// Exec provides a mock function with given fields: ctx, sql, arguments -func (_m *DBTxMock) Exec(ctx context.Context, sql string, arguments ...interface{}) (pgconn.CommandTag, error) { - var _ca []interface{} - _ca = append(_ca, ctx, sql) - _ca = append(_ca, arguments...) - ret := _m.Called(_ca...) - - if len(ret) == 0 { - panic("no return value specified for Exec") - } - - var r0 pgconn.CommandTag - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, string, ...interface{}) (pgconn.CommandTag, error)); ok { - return rf(ctx, sql, arguments...) - } - if rf, ok := ret.Get(0).(func(context.Context, string, ...interface{}) pgconn.CommandTag); ok { - r0 = rf(ctx, sql, arguments...) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgconn.CommandTag) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, string, ...interface{}) error); ok { - r1 = rf(ctx, sql, arguments...) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// LargeObjects provides a mock function with given fields: -func (_m *DBTxMock) LargeObjects() pgx.LargeObjects { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for LargeObjects") - } - - var r0 pgx.LargeObjects - if rf, ok := ret.Get(0).(func() pgx.LargeObjects); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(pgx.LargeObjects) - } - - return r0 -} - -// Prepare provides a mock function with given fields: ctx, name, sql -func (_m *DBTxMock) Prepare(ctx context.Context, name string, sql string) (*pgconn.StatementDescription, error) { - ret := _m.Called(ctx, name, sql) - - if len(ret) == 0 { - panic("no return value specified for Prepare") - } - - var r0 *pgconn.StatementDescription - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, string, string) (*pgconn.StatementDescription, error)); ok { - return rf(ctx, name, sql) - } - if rf, ok := ret.Get(0).(func(context.Context, string, string) *pgconn.StatementDescription); ok { - r0 = rf(ctx, name, sql) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*pgconn.StatementDescription) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, string, string) error); ok { - r1 = rf(ctx, name, sql) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// Query provides a mock function with given fields: ctx, sql, args -func (_m *DBTxMock) Query(ctx context.Context, sql string, args ...interface{}) (pgx.Rows, error) { - var _ca []interface{} - _ca = append(_ca, ctx, sql) - _ca = append(_ca, args...) - ret := _m.Called(_ca...) - - if len(ret) == 0 { - panic("no return value specified for Query") - } - - var r0 pgx.Rows - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, string, ...interface{}) (pgx.Rows, error)); ok { - return rf(ctx, sql, args...) - } - if rf, ok := ret.Get(0).(func(context.Context, string, ...interface{}) pgx.Rows); ok { - r0 = rf(ctx, sql, args...) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgx.Rows) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, string, ...interface{}) error); ok { - r1 = rf(ctx, sql, args...) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// QueryFunc provides a mock function with given fields: ctx, sql, args, scans, f -func (_m *DBTxMock) QueryFunc(ctx context.Context, sql string, args []interface{}, scans []interface{}, f func(pgx.QueryFuncRow) error) (pgconn.CommandTag, error) { - ret := _m.Called(ctx, sql, args, scans, f) - - if len(ret) == 0 { - panic("no return value specified for QueryFunc") - } - - var r0 pgconn.CommandTag - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, string, []interface{}, []interface{}, func(pgx.QueryFuncRow) error) (pgconn.CommandTag, error)); ok { - return rf(ctx, sql, args, scans, f) - } - if rf, ok := ret.Get(0).(func(context.Context, string, []interface{}, []interface{}, func(pgx.QueryFuncRow) error) pgconn.CommandTag); ok { - r0 = rf(ctx, sql, args, scans, f) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgconn.CommandTag) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, string, []interface{}, []interface{}, func(pgx.QueryFuncRow) error) error); ok { - r1 = rf(ctx, sql, args, scans, f) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// QueryRow provides a mock function with given fields: ctx, sql, args -func (_m *DBTxMock) QueryRow(ctx context.Context, sql string, args ...interface{}) pgx.Row { - var _ca []interface{} - _ca = append(_ca, ctx, sql) - _ca = append(_ca, args...) - ret := _m.Called(_ca...) - - if len(ret) == 0 { - panic("no return value specified for QueryRow") - } - - var r0 pgx.Row - if rf, ok := ret.Get(0).(func(context.Context, string, ...interface{}) pgx.Row); ok { - r0 = rf(ctx, sql, args...) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgx.Row) - } - } - - return r0 -} - -// Rollback provides a mock function with given fields: ctx -func (_m *DBTxMock) Rollback(ctx context.Context) error { - ret := _m.Called(ctx) - - if len(ret) == 0 { - panic("no return value specified for Rollback") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context) error); ok { - r0 = rf(ctx) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// SendBatch provides a mock function with given fields: ctx, b -func (_m *DBTxMock) SendBatch(ctx context.Context, b *pgx.Batch) pgx.BatchResults { - ret := _m.Called(ctx, b) - - if len(ret) == 0 { - panic("no return value specified for SendBatch") - } - - var r0 pgx.BatchResults - if rf, ok := ret.Get(0).(func(context.Context, *pgx.Batch) pgx.BatchResults); ok { - r0 = rf(ctx, b) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(pgx.BatchResults) - } - } - - return r0 -} - -// NewDBTxMock creates a new instance of DBTxMock. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -// The first argument is typically a *testing.T value. -func NewDBTxMock(t interface { - mock.TestingT - Cleanup(func()) -}) *DBTxMock { - mock := &DBTxMock{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/jsonrpc/server_test.go b/jsonrpc/server_test.go index ffb60f8717..ae30947e13 100644 --- a/jsonrpc/server_test.go +++ b/jsonrpc/server_test.go @@ -42,7 +42,6 @@ type mocksWrapper struct { State *mocks.StateMock Etherman *mocks.EthermanMock Storage *storageMock - DbTx *mocks.DBTxMock } func newMockedServer(t *testing.T, cfg Config) (*mockedServer, *mocksWrapper, *ethclient.Client) { @@ -50,7 +49,6 @@ func newMockedServer(t *testing.T, cfg Config) (*mockedServer, *mocksWrapper, *e st := mocks.NewStateMock(t) etherman := mocks.NewEthermanMock(t) storage := newStorageMock(t) - dbTx := mocks.NewDBTxMock(t) apis := map[string]bool{ APIEth: true, APINet: true, @@ -143,7 +141,6 @@ func newMockedServer(t *testing.T, cfg Config) (*mockedServer, *mocksWrapper, *e State: st, Etherman: etherman, Storage: storage, - DbTx: dbTx, } return msv, mks, ethClient @@ -262,11 +259,9 @@ func TestBatchRequests(t *testing.T) { NumberOfRequests: 100, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx.On("Commit", context.Background()).Return(nil).Times(tc.NumberOfRequests) - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Times(tc.NumberOfRequests) - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) - m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), m.DbTx).Return(block, nil).Times(tc.NumberOfRequests) - m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, m.DbTx).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) + m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), nil).Return(block, nil).Times(tc.NumberOfRequests) + m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, nil).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) }, }, { @@ -276,11 +271,9 @@ func TestBatchRequests(t *testing.T) { NumberOfRequests: 5, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx.On("Commit", context.Background()).Return(nil).Times(tc.NumberOfRequests) - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Times(tc.NumberOfRequests) - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) - m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), m.DbTx).Return(block, nil).Times(tc.NumberOfRequests) - m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, m.DbTx).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) + m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), nil).Return(block, nil).Times(tc.NumberOfRequests) + m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, nil).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) }, }, { @@ -290,11 +283,9 @@ func TestBatchRequests(t *testing.T) { NumberOfRequests: 4, ExpectedError: nil, SetupMocks: func(m *mocksWrapper, tc testCase) { - m.DbTx.On("Commit", context.Background()).Return(nil).Times(tc.NumberOfRequests) - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Times(tc.NumberOfRequests) - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) - m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), m.DbTx).Return(block, nil).Times(tc.NumberOfRequests) - m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, m.DbTx).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(block.Number().Uint64(), nil).Times(tc.NumberOfRequests) + m.State.On("GetL2BlockByNumber", context.Background(), block.Number().Uint64(), nil).Return(block, nil).Times(tc.NumberOfRequests) + m.State.On("GetTransactionReceipt", context.Background(), mock.Anything, nil).Return(ethTypes.NewReceipt([]byte{}, false, uint64(0)), nil) }, }, } @@ -589,9 +580,7 @@ func TestMaxRequestPerIPPerSec(t *testing.T) { // this makes us sure the code is calling these methods only for // allowed requests times := int(cfg.MaxRequestsPerIPAndSecond) - m.DbTx.On("Commit", context.Background()).Return(nil).Times(times) - m.State.On("BeginStateTransaction", context.Background()).Return(m.DbTx, nil).Times(times) - m.State.On("GetLastL2BlockNumber", context.Background(), m.DbTx).Return(uint64(1), nil).Times(times) + m.State.On("GetLastL2BlockNumber", context.Background(), nil).Return(uint64(1), nil).Times(times) // prepare the workers to process the requests as long as a job is available requestsLimitedCount := uint64(0) diff --git a/jsonrpc/types/codec_test.go b/jsonrpc/types/codec_test.go index d08dbd8ed6..33da973651 100644 --- a/jsonrpc/types/codec_test.go +++ b/jsonrpc/types/codec_test.go @@ -54,7 +54,7 @@ func TestGetNumericBlockNumber(t *testing.T) { bn *BlockNumber expectedBlockNumber uint64 expectedError Error - setupMocks func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) + setupMocks func(s *mocks.StateMock, t *testCase) } testCases := []testCase{ @@ -63,9 +63,9 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: nil, expectedBlockNumber: 40, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastL2BlockNumber", context.Background(), d). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(40), nil). Once() }, @@ -75,9 +75,9 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: state.Ptr(LatestBlockNumber), expectedBlockNumber: 50, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastL2BlockNumber", context.Background(), d). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(50), nil). Once() }, @@ -87,9 +87,9 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: state.Ptr(PendingBlockNumber), expectedBlockNumber: 30, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastL2BlockNumber", context.Background(), d). + On("GetLastL2BlockNumber", context.Background(), nil). Return(uint64(30), nil). Once() }, @@ -99,14 +99,14 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: state.Ptr(EarliestBlockNumber), expectedBlockNumber: 0, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, { name: "BlockNumber SafeBlockNumber", bn: state.Ptr(SafeBlockNumber), expectedBlockNumber: 40, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { safeBlockNumber := uint64(30) e. On("GetSafeBlockNumber", context.Background()). @@ -114,7 +114,7 @@ func TestGetNumericBlockNumber(t *testing.T) { Once() s. - On("GetLastVerifiedL2BlockNumberUntilL1Block", context.Background(), safeBlockNumber, d). + On("GetLastVerifiedL2BlockNumberUntilL1Block", context.Background(), safeBlockNumber, nil). Return(uint64(40), nil). Once() }, @@ -124,7 +124,7 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: state.Ptr(FinalizedBlockNumber), expectedBlockNumber: 60, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { finalizedBlockNumber := uint64(50) e. On("GetFinalizedBlockNumber", context.Background()). @@ -132,7 +132,7 @@ func TestGetNumericBlockNumber(t *testing.T) { Once() s. - On("GetLastVerifiedL2BlockNumberUntilL1Block", context.Background(), finalizedBlockNumber, d). + On("GetLastVerifiedL2BlockNumberUntilL1Block", context.Background(), finalizedBlockNumber, nil). Return(uint64(60), nil). Once() }, @@ -142,23 +142,22 @@ func TestGetNumericBlockNumber(t *testing.T) { bn: state.Ptr(BlockNumber(int64(10))), expectedBlockNumber: 10, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, { name: "BlockNumber Negative Number <= -6", bn: state.Ptr(BlockNumber(int64(-6))), expectedBlockNumber: 0, expectedError: NewRPCError(InvalidParamsErrorCode, "invalid block number: -6"), - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, } for _, testCase := range testCases { t.Run(testCase.name, func(t *testing.T) { tc := testCase - dbTx := mocks.NewDBTxMock(t) - testCase.setupMocks(s, dbTx, &tc) - result, rpcErr := testCase.bn.GetNumericBlockNumber(context.Background(), s, e, dbTx) + testCase.setupMocks(s, &tc) + result, rpcErr := testCase.bn.GetNumericBlockNumber(context.Background(), s, e, nil) assert.Equal(t, testCase.expectedBlockNumber, result) if rpcErr != nil || testCase.expectedError != nil { assert.Equal(t, testCase.expectedError.ErrorCode(), rpcErr.ErrorCode()) @@ -177,7 +176,7 @@ func TestGetNumericBatchNumber(t *testing.T) { bn *BatchNumber expectedBatchNumber uint64 expectedError Error - setupMocks func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) + setupMocks func(s *mocks.StateMock, t *testCase) } testCases := []testCase{ @@ -186,9 +185,9 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: nil, expectedBatchNumber: 40, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastClosedBatchNumber", context.Background(), d). + On("GetLastClosedBatchNumber", context.Background(), nil). Return(uint64(40), nil). Once() }, @@ -198,9 +197,9 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: state.Ptr(LatestBatchNumber), expectedBatchNumber: 50, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastClosedBatchNumber", context.Background(), d). + On("GetLastClosedBatchNumber", context.Background(), nil). Return(uint64(50), nil). Once() }, @@ -210,9 +209,9 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: state.Ptr(PendingBatchNumber), expectedBatchNumber: 90, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { s. - On("GetLastBatchNumber", context.Background(), d). + On("GetLastBatchNumber", context.Background(), nil). Return(uint64(90), nil). Once() }, @@ -222,14 +221,14 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: state.Ptr(EarliestBatchNumber), expectedBatchNumber: 0, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, { name: "BatchNumber SafeBatchNumber", bn: state.Ptr(SafeBatchNumber), expectedBatchNumber: 40, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { safeBlockNumber := uint64(30) e. On("GetSafeBlockNumber", context.Background()). @@ -237,7 +236,7 @@ func TestGetNumericBatchNumber(t *testing.T) { Once() s. - On("GetLastVerifiedBatchNumberUntilL1Block", context.Background(), safeBlockNumber, d). + On("GetLastVerifiedBatchNumberUntilL1Block", context.Background(), safeBlockNumber, nil). Return(uint64(40), nil). Once() }, @@ -247,7 +246,7 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: state.Ptr(FinalizedBatchNumber), expectedBatchNumber: 60, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) { + setupMocks: func(s *mocks.StateMock, t *testCase) { finalizedBlockNumber := uint64(50) e. On("GetFinalizedBlockNumber", context.Background()). @@ -255,7 +254,7 @@ func TestGetNumericBatchNumber(t *testing.T) { Once() s. - On("GetLastVerifiedBatchNumberUntilL1Block", context.Background(), finalizedBlockNumber, d). + On("GetLastVerifiedBatchNumberUntilL1Block", context.Background(), finalizedBlockNumber, nil). Return(uint64(60), nil). Once() }, @@ -265,23 +264,22 @@ func TestGetNumericBatchNumber(t *testing.T) { bn: state.Ptr(BatchNumber(int64(10))), expectedBatchNumber: 10, expectedError: nil, - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, { name: "BatchNumber Negative Number <= -6", bn: state.Ptr(BatchNumber(int64(-6))), expectedBatchNumber: 0, expectedError: NewRPCError(InvalidParamsErrorCode, "invalid batch number: -6"), - setupMocks: func(s *mocks.StateMock, d *mocks.DBTxMock, t *testCase) {}, + setupMocks: func(s *mocks.StateMock, t *testCase) {}, }, } for _, testCase := range testCases { t.Run(testCase.name, func(t *testing.T) { tc := testCase - dbTx := mocks.NewDBTxMock(t) - testCase.setupMocks(s, dbTx, &tc) - result, rpcErr := testCase.bn.GetNumericBatchNumber(context.Background(), s, e, dbTx) + testCase.setupMocks(s, &tc) + result, rpcErr := testCase.bn.GetNumericBatchNumber(context.Background(), s, e, nil) assert.Equal(t, testCase.expectedBatchNumber, result) if rpcErr != nil || testCase.expectedError != nil { assert.Equal(t, testCase.expectedError.ErrorCode(), rpcErr.ErrorCode()) diff --git a/test/Makefile b/test/Makefile index 831b2cad10..ca7daa3b88 100644 --- a/test/Makefile +++ b/test/Makefile @@ -661,7 +661,6 @@ generate-mocks-jsonrpc: ## Generates mocks for jsonrpc , using mockery tool export "GOROOT=$$(go env GOROOT)" && $$(go env GOPATH)/bin/mockery --name=PoolInterface --dir=../jsonrpc/types --output=../jsonrpc/mocks --outpkg=mocks --structname=PoolMock --filename=mock_pool.go export "GOROOT=$$(go env GOROOT)" && $$(go env GOPATH)/bin/mockery --name=StateInterface --dir=../jsonrpc/types --output=../jsonrpc/mocks --outpkg=mocks --structname=StateMock --filename=mock_state.go export "GOROOT=$$(go env GOROOT)" && $$(go env GOPATH)/bin/mockery --name=EthermanInterface --dir=../jsonrpc/types --output=../jsonrpc/mocks --outpkg=mocks --structname=EthermanMock --filename=mock_etherman.go - export "GOROOT=$$(go env GOROOT)" && $$(go env GOPATH)/bin/mockery --name=Tx --srcpkg=github.com/jackc/pgx/v4 --output=../jsonrpc/mocks --outpkg=mocks --structname=DBTxMock --filename=mock_dbtx.go .PHONY: generate-mocks-sequencer generate-mocks-sequencer: ## Generates mocks for sequencer , using mockery tool From 68fb0484c2d4f22cee00456db51c1617c5294a72 Mon Sep 17 00:00:00 2001 From: Thiago Coimbra Lemos Date: Wed, 5 Jun 2024 10:17:20 -0300 Subject: [PATCH 19/22] remove synchronizer dependency from eth tx manager to confirm monitored tx (#3658) (#3677) --- ethtxmanager/ethtxmanager.go | 23 ++------------- ethtxmanager/ethtxmanager_test.go | 49 ++----------------------------- 2 files changed, 5 insertions(+), 67 deletions(-) diff --git a/ethtxmanager/ethtxmanager.go b/ethtxmanager/ethtxmanager.go index 1c48071cf0..79f9262962 100644 --- a/ethtxmanager/ethtxmanager.go +++ b/ethtxmanager/ethtxmanager.go @@ -13,7 +13,6 @@ import ( "time" "github.com/0xPolygonHermez/zkevm-node/log" - "github.com/0xPolygonHermez/zkevm-node/state" "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -457,25 +456,9 @@ func (c *Client) monitorTx(ctx context.Context, mTx monitoredTx, logger *log.Log // if mined, check receipt and mark as Failed or Confirmed if lastReceiptChecked.Status == types.ReceiptStatusSuccessful { - receiptBlockNum := lastReceiptChecked.BlockNumber.Uint64() - - // check if state is already synchronized until the block - // where the tx was mined - block, err := c.state.GetLastBlock(ctx, nil) - if errors.Is(err, state.ErrStateNotSynchronized) { - logger.Debugf("state not synchronized yet, waiting for L1 block %v to be synced", receiptBlockNum) - return - } else if err != nil { - logger.Errorf("failed to check if L1 block %v is already synced: %v", receiptBlockNum, err) - return - } else if block.BlockNumber < receiptBlockNum { - logger.Debugf("L1 block %v not synchronized yet, waiting for L1 block to be synced in order to confirm monitored tx", receiptBlockNum) - return - } else { - mTx.status = MonitoredTxStatusConfirmed - mTx.blockNumber = lastReceiptChecked.BlockNumber - logger.Info("confirmed") - } + mTx.status = MonitoredTxStatusConfirmed + mTx.blockNumber = lastReceiptChecked.BlockNumber + logger.Info("confirmed") } else { // if we should continue to monitor, we move to the next one and this will // be reviewed in the next monitoring cycle diff --git a/ethtxmanager/ethtxmanager_test.go b/ethtxmanager/ethtxmanager_test.go index 0ca53086fd..350332aae7 100644 --- a/ethtxmanager/ethtxmanager_test.go +++ b/ethtxmanager/ethtxmanager_test.go @@ -9,7 +9,6 @@ import ( "time" "github.com/0xPolygonHermez/zkevm-node/config/types" - "github.com/0xPolygonHermez/zkevm-node/state" "github.com/0xPolygonHermez/zkevm-node/test/dbutils" "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" @@ -118,14 +117,6 @@ func TestTxGetMined(t *testing.T) { Return("", nil). Once() - block := &state.Block{ - BlockNumber: blockNumber.Uint64(), - } - st. - On("GetLastBlock", ctx, nil). - Return(block, nil). - Once() - err = ethTxManagerClient.Add(ctx, owner, id, from, to, value, data, gasOffset, nil) require.NoError(t, err) @@ -263,14 +254,6 @@ func TestTxGetMinedAfterReviewed(t *testing.T) { Return(receipt, nil). Once() - block := &state.Block{ - BlockNumber: blockNumber.Uint64(), - } - st. - On("GetLastBlock", ctx, nil). - Return(block, nil). - Once() - // Build result etherman. On("GetTx", ctx, firstSignedTx.Hash()). @@ -383,16 +366,8 @@ func TestTxGetMinedAfterConfirmedAndReorged(t *testing.T) { } etherman. On("GetTxReceipt", ctx, signedTx.Hash()). - Return(receipt, nil). - Once() - - block := &state.Block{ - BlockNumber: blockNumber.Uint64(), - } - st. - On("GetLastBlock", ctx, nil). Run(func(args mock.Arguments) { ethTxManagerClient.Stop() }). // stops the management cycle to avoid problems with mocks - Return(block, nil). + Return(receipt, nil). Once() // Build Result 1 @@ -433,12 +408,8 @@ func TestTxGetMinedAfterConfirmedAndReorged(t *testing.T) { // Monitoring Cycle 3 etherman. On("CheckTxWasMined", ctx, signedTx.Hash()). - Return(true, receipt, nil). - Once() - st. - On("GetLastBlock", ctx, nil). Run(func(args mock.Arguments) { ethTxManagerClient.Stop() }). // stops the management cycle to avoid problems with mocks - Return(block, nil). + Return(true, receipt, nil). Once() // Build Result 3 @@ -643,14 +614,6 @@ func TestExecutionReverted(t *testing.T) { Return(receipt, nil). Once() - block := &state.Block{ - BlockNumber: blockNumber.Uint64(), - } - st. - On("GetLastBlock", ctx, nil). - Return(block, nil). - Once() - // Build result etherman. On("GetTx", ctx, firstSignedTx.Hash()). @@ -959,14 +922,6 @@ func TestFailedToEstimateTxWithForcedGasGetMined(t *testing.T) { Return("", nil). Once() - block := &state.Block{ - BlockNumber: blockNumber.Uint64(), - } - st. - On("GetLastBlock", ctx, nil). - Return(block, nil). - Once() - err = ethTxManagerClient.Add(ctx, owner, id, from, to, value, data, gasOffset, nil) require.NoError(t, err) From d555d6580f59dc4156244577554d91f00d25b93c Mon Sep 17 00:00:00 2001 From: agnusmor <100322135+agnusmor@users.noreply.github.com> Date: Wed, 5 Jun 2024 18:56:37 +0200 Subject: [PATCH 20/22] Cherry-pick #3669: Add WriteTimeout config parameter to StreamServer (#3690) * Add WriteTimeout config parameter to StreamServer (#3669) * Add WriteTimeout config parameter to StreamServer. Update DS library * update doc * update default value for StreamServer.WriteTimeout config parameter. Increase buffer for datastream channel * fix doc * fix config test * fix doc --- config/config_test.go | 4 +++ config/default.go | 1 + .../environments/local/local.node.config.toml | 1 + docs/config-file/node-config-doc.html | 4 ++- docs/config-file/node-config-doc.md | 27 +++++++++++++++++++ docs/config-file/node-config-schema.json | 10 +++++++ go.mod | 2 +- go.sum | 4 +-- sequencer/config.go | 2 ++ sequencer/l2block.go | 4 +-- sequencer/sequencer.go | 4 +-- test/config/debug.node.config.toml | 1 + test/config/test.node.config.toml | 1 + tools/datastreamer/config/config.go | 3 +++ tools/datastreamer/config/tool.config.toml | 1 + tools/datastreamer/main.go | 2 +- 16 files changed, 62 insertions(+), 9 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 6ba1131a3b..d6c40ed30a 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -169,6 +169,10 @@ func Test_Defaults(t *testing.T) { path: "Sequencer.StreamServer.Version", expectedValue: uint8(0), }, + { + path: "Sequencer.StreamServer.WriteTimeout", + expectedValue: types.NewDuration(5 * time.Second), + }, { path: "Sequencer.StreamServer.Enabled", expectedValue: false, diff --git a/config/default.go b/config/default.go index 0b73699125..bff3d90a70 100644 --- a/config/default.go +++ b/config/default.go @@ -163,6 +163,7 @@ StateConsistencyCheckInterval = "5s" Port = 0 Filename = "" Version = 0 + WriteTimeout = "5s" Enabled = false [SequenceSender] diff --git a/config/environments/local/local.node.config.toml b/config/environments/local/local.node.config.toml index 11077845ca..c403a0718a 100644 --- a/config/environments/local/local.node.config.toml +++ b/config/environments/local/local.node.config.toml @@ -112,6 +112,7 @@ StateConsistencyCheckInterval = "5s" [Sequencer.StreamServer] Port = 0 Filename = "" + WriteTimeout = "5s" Enabled = false [SequenceSender] diff --git a/docs/config-file/node-config-doc.html b/docs/config-file/node-config-doc.html index 79d14baf44..14800feab3 100644 --- a/docs/config-file/node-config-doc.html +++ b/docs/config-file/node-config-doc.html @@ -56,7 +56,9 @@
"300ms"
 

Default: 0Type: integer

HaltOnBatchNumber specifies the batch number where the Sequencer will stop to process more transactions and generate new batches.
The Sequencer will halt after it closes the batch equal to this number


Default: falseType: boolean

SequentialBatchSanityCheck indicates if the reprocess of a closed batch (sanity check) must be done in a
sequential way (instead than in parallel)


Default: falseType: boolean

SequentialProcessL2Block indicates if the processing of a L2 Block must be done in the same finalizer go func instead
in the processPendingL2Blocks go func


Metrics is the config for the sequencer metrics
Default: "1h0m0s"Type: string

Interval is the interval of time to calculate sequencer metrics


Examples:

"1m"
 
"300ms"
-

Default: trueType: boolean

EnableLog is a flag to enable/disable metrics logs


StreamServerCfg is the config for the stream server
Default: 0Type: integer

Port to listen on


Default: ""Type: string

Filename of the binary data file


Default: 0Type: integer

Version of the binary data file


Default: 0Type: integer

ChainID is the chain ID


Default: falseType: boolean

Enabled is a flag to enable/disable the data streamer


Log is the log configuration
Default: ""Type: enum (of string)

Must be one of:

  • "production"
  • "development"

Default: ""Type: enum (of string)

Must be one of:

  • "debug"
  • "info"
  • "warn"
  • "error"
  • "dpanic"
  • "panic"
  • "fatal"

Type: array of string

Each item of this array must be:


Default: 0Type: integer

UpgradeEtrogBatchNumber is the batch number of the upgrade etrog


Configuration of the sequence sender service
Default: "5s"Type: string

WaitPeriodSendSequence is the time the sequencer waits until
trying to send a sequence to L1


Examples:

"1m"
+

Default: trueType: boolean

EnableLog is a flag to enable/disable metrics logs


StreamServerCfg is the config for the stream server
Default: 0Type: integer

Port to listen on


Default: ""Type: string

Filename of the binary data file


Default: 0Type: integer

Version of the binary data file


Default: 0Type: integer

ChainID is the chain ID


Default: falseType: boolean

Enabled is a flag to enable/disable the data streamer


Log is the log configuration
Default: ""Type: enum (of string)

Must be one of:

  • "production"
  • "development"

Default: ""Type: enum (of string)

Must be one of:

  • "debug"
  • "info"
  • "warn"
  • "error"
  • "dpanic"
  • "panic"
  • "fatal"

Type: array of string

Each item of this array must be:


Default: 0Type: integer

UpgradeEtrogBatchNumber is the batch number of the upgrade etrog


Default: "5s"Type: string

WriteTimeout is the TCP write timeout when sending data to a datastream client


Examples:

"1m"
+
"300ms"
+

Configuration of the sequence sender service
Default: "5s"Type: string

WaitPeriodSendSequence is the time the sequencer waits until
trying to send a sequence to L1


Examples:

"1m"
 
"300ms"
 

Default: "5s"Type: string

LastBatchVirtualizationTimeMaxWaitPeriod is time since sequences should be sent


Examples:

"1m"
 
"300ms"
diff --git a/docs/config-file/node-config-doc.md b/docs/config-file/node-config-doc.md
index afe68fdc2f..3dab591984 100644
--- a/docs/config-file/node-config-doc.md
+++ b/docs/config-file/node-config-doc.md
@@ -2487,6 +2487,7 @@ EnableLog=true
 | - [Enabled](#Sequencer_StreamServer_Enabled )                                 | No      | boolean | No         | -          | Enabled is a flag to enable/disable the data streamer            |
 | - [Log](#Sequencer_StreamServer_Log )                                         | No      | object  | No         | -          | Log is the log configuration                                     |
 | - [UpgradeEtrogBatchNumber](#Sequencer_StreamServer_UpgradeEtrogBatchNumber ) | No      | integer | No         | -          | UpgradeEtrogBatchNumber is the batch number of the upgrade etrog |
+| - [WriteTimeout](#Sequencer_StreamServer_WriteTimeout )                       | No      | string  | No         | -          | Duration                                                         |
 
 #### 10.9.1. `Sequencer.StreamServer.Port`
 
@@ -2624,6 +2625,32 @@ Must be one of:
 UpgradeEtrogBatchNumber=0
 ```
 
+#### 10.9.8. `Sequencer.StreamServer.WriteTimeout`
+
+**Title:** Duration
+
+**Type:** : `string`
+
+**Default:** `"5s"`
+
+**Description:** WriteTimeout is the TCP write timeout when sending data to a datastream client
+
+**Examples:** 
+
+```json
+"1m"
+```
+
+```json
+"300ms"
+```
+
+**Example setting the default value** ("5s"):
+```
+[Sequencer.StreamServer]
+WriteTimeout="5s"
+```
+
 ## 11. `[SequenceSender]`
 
 **Type:** : `object`
diff --git a/docs/config-file/node-config-schema.json b/docs/config-file/node-config-schema.json
index 5096caac0e..489ee17274 100644
--- a/docs/config-file/node-config-schema.json
+++ b/docs/config-file/node-config-schema.json
@@ -998,6 +998,16 @@
 							"type": "integer",
 							"description": "UpgradeEtrogBatchNumber is the batch number of the upgrade etrog",
 							"default": 0
+						},
+						"WriteTimeout": {
+							"type": "string",
+							"title": "Duration",
+							"description": "WriteTimeout is the TCP write timeout when sending data to a datastream client",
+							"default": "5s",
+							"examples": [
+								"1m",
+								"300ms"
+							]
 						}
 					},
 					"additionalProperties": false,
diff --git a/go.mod b/go.mod
index c94597b1ab..64ced2a895 100644
--- a/go.mod
+++ b/go.mod
@@ -3,7 +3,7 @@ module github.com/0xPolygonHermez/zkevm-node
 go 1.21
 
 require (
-	github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b
+	github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-RC4
 	github.com/didip/tollbooth/v6 v6.1.2
 	github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127
 	github.com/ethereum/go-ethereum v1.13.11
diff --git a/go.sum b/go.sum
index bff27c6313..da7a1cbead 100644
--- a/go.sum
+++ b/go.sum
@@ -39,8 +39,8 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9
 dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk=
 dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk=
 dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
-github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b h1:BzQRXbSnW7BsFvJrnZbCgnxD5+nCGyrYUgqH+3vsnrM=
-github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-0.20240527085154-ca3561dd370b/go.mod h1:0QkAXcFa92mFJrCbN3UPUJGJYes851yEgYHLONnaosE=
+github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-RC4 h1:+4K+xSzv0ImbK30B/T9FauNTrTFUmWcNKYhIgwsE4C4=
+github.com/0xPolygonHermez/zkevm-data-streamer v0.2.3-RC4/go.mod h1:0QkAXcFa92mFJrCbN3UPUJGJYes851yEgYHLONnaosE=
 github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
 github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 h1:w+iIsaOQNcT7OZ575w+acHgRric5iCyQh+xv+KJ4HB8=
 github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8=
diff --git a/sequencer/config.go b/sequencer/config.go
index 03aeeb740b..5f34cad1d9 100644
--- a/sequencer/config.go
+++ b/sequencer/config.go
@@ -52,6 +52,8 @@ type StreamServerCfg struct {
 	Log log.Config `mapstructure:"Log"`
 	// UpgradeEtrogBatchNumber is the batch number of the upgrade etrog
 	UpgradeEtrogBatchNumber uint64 `mapstructure:"UpgradeEtrogBatchNumber"`
+	// WriteTimeout is the TCP write timeout when sending data to a datastream client
+	WriteTimeout types.Duration `mapstructure:"WriteTimeout"`
 }
 
 // FinalizerCfg contains the finalizer's configuration properties
diff --git a/sequencer/l2block.go b/sequencer/l2block.go
index 7cc21fc928..511d233988 100644
--- a/sequencer/l2block.go
+++ b/sequencer/l2block.go
@@ -244,7 +244,7 @@ func (f *finalizer) processL2Block(ctx context.Context, l2Block *L2Block) error
 		if subOverflow { // Sanity check, this cannot happen as reservedZKCounters should be >= that usedZKCounters
 			return fmt.Errorf("error subtracting L2 block %d [%d] needed resources from the batch %d, overflow resource: %s, batch bytes: %d, L2 block bytes: %d, counters: {batch: %s, used: %s, reserved: %s, needed: %s, high: %s}",
 				blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, overflowResource, l2Block.batch.finalRemainingResources.Bytes, batchL2DataSize,
-				f.logZKCounters(l2Block.batch.finalRemainingResources.ZKCounters), f.logZKCounters(batchResponse.UsedZkCounters), f.logZKCounters(batchResponse.ReservedZkCounters), f.logZKCounters(neededZKCounters), f.logZKCounters(l2Block.batch.imHighReservedZKCounters))
+				f.logZKCounters(l2Block.batch.finalRemainingResources.ZKCounters), f.logZKCounters(batchResponse.UsedZkCounters), f.logZKCounters(batchResponse.ReservedZkCounters), f.logZKCounters(neededZKCounters), f.logZKCounters(l2Block.batch.finalHighReservedZKCounters))
 		}
 
 		l2Block.batch.finalHighReservedZKCounters = newHighZKCounters
@@ -252,7 +252,7 @@ func (f *finalizer) processL2Block(ctx context.Context, l2Block *L2Block) error
 	} else {
 		overflowLog := fmt.Sprintf("L2 block %d [%d] needed resources exceeds the remaining batch %d resources, overflow resource: %s, batch bytes: %d, L2 block bytes: %d, counters: {batch: %s, used: %s, reserved: %s, needed: %s, high: %s}",
 			blockResponse.BlockNumber, l2Block.trackingNum, l2Block.batch.batchNumber, overflowResource, l2Block.batch.finalRemainingResources.Bytes, batchL2DataSize,
-			f.logZKCounters(l2Block.batch.finalRemainingResources.ZKCounters), f.logZKCounters(batchResponse.UsedZkCounters), f.logZKCounters(batchResponse.ReservedZkCounters), f.logZKCounters(neededZKCounters), f.logZKCounters(l2Block.batch.imHighReservedZKCounters))
+			f.logZKCounters(l2Block.batch.finalRemainingResources.ZKCounters), f.logZKCounters(batchResponse.UsedZkCounters), f.logZKCounters(batchResponse.ReservedZkCounters), f.logZKCounters(neededZKCounters), f.logZKCounters(l2Block.batch.finalHighReservedZKCounters))
 
 		f.LogEvent(ctx, event.Level_Warning, event.EventID_ReservedZKCountersOverflow, overflowLog, nil)
 
diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go
index f191973767..92909e2946 100644
--- a/sequencer/sequencer.go
+++ b/sequencer/sequencer.go
@@ -17,7 +17,7 @@ import (
 )
 
 const (
-	datastreamChannelBufferSize = 20
+	datastreamChannelBufferSize = 50
 )
 
 // Sequencer represents a sequencer
@@ -72,7 +72,7 @@ func (s *Sequencer) Start(ctx context.Context) {
 
 	// Start stream server if enabled
 	if s.cfg.StreamServer.Enabled {
-		s.streamServer, err = datastreamer.NewServer(s.cfg.StreamServer.Port, s.cfg.StreamServer.Version, s.cfg.StreamServer.ChainID, state.StreamTypeSequencer, s.cfg.StreamServer.Filename, &s.cfg.StreamServer.Log)
+		s.streamServer, err = datastreamer.NewServer(s.cfg.StreamServer.Port, s.cfg.StreamServer.Version, s.cfg.StreamServer.ChainID, state.StreamTypeSequencer, s.cfg.StreamServer.Filename, s.cfg.StreamServer.WriteTimeout.Duration, &s.cfg.StreamServer.Log)
 		if err != nil {
 			log.Fatalf("failed to create stream server, error: %v", err)
 		}
diff --git a/test/config/debug.node.config.toml b/test/config/debug.node.config.toml
index 68f7dd17ce..54ca62a789 100644
--- a/test/config/debug.node.config.toml
+++ b/test/config/debug.node.config.toml
@@ -112,6 +112,7 @@ StateConsistencyCheckInterval = "5s"
 		Port = 6900
 		Filename = "/datastreamer/datastream.bin"
 		Version = 1
+		WriteTimeout = "5s"
 		Enabled = true
 
 [SequenceSender]
diff --git a/test/config/test.node.config.toml b/test/config/test.node.config.toml
index 1df2fc1882..0358afa59e 100644
--- a/test/config/test.node.config.toml
+++ b/test/config/test.node.config.toml
@@ -128,6 +128,7 @@ StateConsistencyCheckInterval = "5s"
 		Filename = "/datastreamer/datastream.bin"
 		Version = 1
 		ChainID = 1337
+		WriteTimeout = "5s"
 		Enabled = true
 
 [SequenceSender]
diff --git a/tools/datastreamer/config/config.go b/tools/datastreamer/config/config.go
index 0acb225cf9..b6c841e591 100644
--- a/tools/datastreamer/config/config.go
+++ b/tools/datastreamer/config/config.go
@@ -7,6 +7,7 @@ import (
 
 	"github.com/0xPolygonHermez/zkevm-data-streamer/datastreamer"
 	"github.com/0xPolygonHermez/zkevm-data-streamer/log"
+	"github.com/0xPolygonHermez/zkevm-node/config/types"
 	"github.com/0xPolygonHermez/zkevm-node/db"
 	"github.com/0xPolygonHermez/zkevm-node/state/runtime/executor"
 	"github.com/mitchellh/mapstructure"
@@ -48,6 +49,8 @@ type StreamServerCfg struct {
 	Log log.Config `mapstructure:"Log"`
 	// UpgradeEtrogBatchNumber is the batch number of the upgrade etrog
 	UpgradeEtrogBatchNumber uint64 `mapstructure:"UpgradeEtrogBatchNumber"`
+	// WriteTimeout is the TCP write timeout when sending data to a datastream client
+	WriteTimeout types.Duration `mapstructure:"WriteTimeout"`
 }
 
 // Config is the configuration for the tool
diff --git a/tools/datastreamer/config/tool.config.toml b/tools/datastreamer/config/tool.config.toml
index 0e8fc09fc9..f5530b8271 100644
--- a/tools/datastreamer/config/tool.config.toml
+++ b/tools/datastreamer/config/tool.config.toml
@@ -7,6 +7,7 @@ Port = 6901
 Filename = "datastream.bin"
 Version = 3
 ChainID = 1440
+WriteTimeout = "5s"
 UpgradeEtrogBatchNumber = 0
 
 [StateDB]
diff --git a/tools/datastreamer/main.go b/tools/datastreamer/main.go
index a2e3d19003..fc069117af 100644
--- a/tools/datastreamer/main.go
+++ b/tools/datastreamer/main.go
@@ -184,7 +184,7 @@ func main() {
 
 func initializeStreamServer(c *config.Config) (*datastreamer.StreamServer, error) {
 	// Create a stream server
-	streamServer, err := datastreamer.NewServer(c.Offline.Port, c.Offline.Version, c.Offline.ChainID, state.StreamTypeSequencer, c.Offline.Filename, &c.Log)
+	streamServer, err := datastreamer.NewServer(c.Offline.Port, c.Offline.Version, c.Offline.ChainID, state.StreamTypeSequencer, c.Offline.Filename, c.Offline.WriteTimeout.Duration, &c.Log)
 	if err != nil {
 		return nil, err
 	}

From bdeddb321ebefc688116b261116f84be8f042b76 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Toni=20Ram=C3=ADrez?=
 <58293609+ToniRamirezM@users.noreply.github.com>
Date: Thu, 6 Jun 2024 18:27:27 +0200
Subject: [PATCH 21/22] change imstateroot handling in DS (#3698)

* change imstateroot handling in DS
---
 sequencer/sequencer.go | 4 ++--
 state/datastream.go    | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/sequencer/sequencer.go b/sequencer/sequencer.go
index 92909e2946..3f57d9cc85 100644
--- a/sequencer/sequencer.go
+++ b/sequencer/sequencer.go
@@ -349,10 +349,10 @@ func (s *Sequencer) sendDataToStreamer(chainID uint64) {
 						ImStateRoot:                 l2Transaction.ImStateRoot.Bytes(),
 					}
 
-					// Clear the state root if the ForkID is >= ETROG
+					// Clear the state root if the ForkID is > ETROG
 					// currently this is redundant as the current implementation of the sequencer
 					// leaves the ImStateRoot empty
-					if l2Block.ForkID >= state.FORKID_ETROG {
+					if l2Block.ForkID > state.FORKID_ETROG {
 						streamL2Transaction.ImStateRoot = common.Hash{}.Bytes()
 					}
 
diff --git a/state/datastream.go b/state/datastream.go
index ecb211d6c2..236a69df43 100644
--- a/state/datastream.go
+++ b/state/datastream.go
@@ -590,8 +590,8 @@ func GenerateDataStreamFile(ctx context.Context, streamServer *datastreamer.Stre
 							ImStateRoot:                 tx.StateRoot.Bytes(),
 						}
 
-						// Clear the state root if the ForkID is >= ETROG
-						if l2Block.ForkID >= FORKID_ETROG {
+						// Clear the state root if the ForkID is > ETROG
+						if l2Block.ForkID > FORKID_ETROG {
 							transaction.ImStateRoot = common.Hash{}.Bytes()
 						}
 

From fd233e3d88e95c4f15d03818f69e5a95a6411e8b Mon Sep 17 00:00:00 2001
From: JianGuo 
Date: Tue, 25 Jun 2024 15:30:05 +0800
Subject: [PATCH 22/22] Modify code

---
 jsonrpc/endpoints_zkevm.go        | 36 -----------------------------
 jsonrpc/endpoints_zkevm_xlayer.go | 38 +++++++++++++++++++++++++++++++
 state/helper.go                   |  8 ++-----
 state/transaction.go              |  2 +-
 4 files changed, 41 insertions(+), 43 deletions(-)

diff --git a/jsonrpc/endpoints_zkevm.go b/jsonrpc/endpoints_zkevm.go
index 252f092fdc..a819bc4896 100644
--- a/jsonrpc/endpoints_zkevm.go
+++ b/jsonrpc/endpoints_zkevm.go
@@ -195,42 +195,6 @@ func (z *ZKEVMEndpoints) GetBatchByNumber(batchNumber types.BatchNumber, fullTx
 	return rpcBatch, nil
 }
 
-// GetBatchDataByNumbers returns the batch data for batches by numbers, XLayer
-func (z *ZKEVMEndpoints) GetBatchDataByNumbers(filter types.BatchFilter) (interface{}, types.Error) {
-	ctx := context.Background()
-
-	var batchNumbers []uint64
-	for _, bn := range filter.Numbers {
-		n, rpcErr := bn.GetNumericBatchNumber(ctx, z.state, z.etherman, nil)
-		if rpcErr != nil {
-			return nil, rpcErr
-		}
-		batchNumbers = append(batchNumbers, n)
-	}
-
-	batchesData, err := z.state.GetBatchL2DataByNumbers(ctx, batchNumbers, nil)
-	if errors.Is(err, state.ErrNotFound) {
-		return nil, nil
-	} else if err != nil {
-		return RPCErrorResponse(types.DefaultErrorCode,
-			fmt.Sprintf("couldn't load batch data from state by numbers %v", filter.Numbers), err, true)
-	}
-
-	var ret []*types.BatchData
-	for _, n := range batchNumbers {
-		data := &types.BatchData{Number: types.ArgUint64(n)}
-		if b, ok := batchesData[n]; ok {
-			data.BatchL2Data = b
-			data.Empty = false
-		} else {
-			data.Empty = true
-		}
-		ret = append(ret, data)
-	}
-
-	return types.BatchDataResult{Data: ret}, nil
-}
-
 // GetFullBlockByNumber returns information about a block by block number
 func (z *ZKEVMEndpoints) GetFullBlockByNumber(number types.BlockNumber, fullTx bool) (interface{}, types.Error) {
 	ctx := context.Background()
diff --git a/jsonrpc/endpoints_zkevm_xlayer.go b/jsonrpc/endpoints_zkevm_xlayer.go
index 56643a7af6..9e304570b9 100644
--- a/jsonrpc/endpoints_zkevm_xlayer.go
+++ b/jsonrpc/endpoints_zkevm_xlayer.go
@@ -2,10 +2,12 @@ package jsonrpc
 
 import (
 	"context"
+	"errors"
 	"fmt"
 
 	"github.com/0xPolygonHermez/zkevm-node/hex"
 	"github.com/0xPolygonHermez/zkevm-node/jsonrpc/types"
+	"github.com/0xPolygonHermez/zkevm-node/state"
 )
 
 // GetBatchSealTime returns the seal time
@@ -24,3 +26,39 @@ func (z *ZKEVMEndpoints) GetBatchSealTime(batchNumber types.BatchNumber) (interf
 
 	return hex.EncodeUint64(sealTime), nil
 }
+
+// GetBatchDataByNumbers returns the batch data for batches by numbers, XLayer
+func (z *ZKEVMEndpoints) GetBatchDataByNumbers(filter types.BatchFilter) (interface{}, types.Error) {
+	ctx := context.Background()
+
+	var batchNumbers []uint64
+	for _, bn := range filter.Numbers {
+		n, rpcErr := bn.GetNumericBatchNumber(ctx, z.state, z.etherman, nil)
+		if rpcErr != nil {
+			return nil, rpcErr
+		}
+		batchNumbers = append(batchNumbers, n)
+	}
+
+	batchesData, err := z.state.GetBatchL2DataByNumbers(ctx, batchNumbers, nil)
+	if errors.Is(err, state.ErrNotFound) {
+		return nil, nil
+	} else if err != nil {
+		return RPCErrorResponse(types.DefaultErrorCode,
+			fmt.Sprintf("couldn't load batch data from state by numbers %v", filter.Numbers), err, true)
+	}
+
+	var ret []*types.BatchData
+	for _, n := range batchNumbers {
+		data := &types.BatchData{Number: types.ArgUint64(n)}
+		if b, ok := batchesData[n]; ok {
+			data.BatchL2Data = b
+			data.Empty = false
+		} else {
+			data.Empty = true
+		}
+		ret = append(ret, data)
+	}
+
+	return types.BatchDataResult{Data: ret}, nil
+}
diff --git a/state/helper.go b/state/helper.go
index 0d4a51c274..3b37d121ee 100644
--- a/state/helper.go
+++ b/state/helper.go
@@ -287,9 +287,7 @@ func GenerateReceipt(blockNumber *big.Int, processedTx *ProcessTransactionRespon
 		ContractAddress:  processedTx.CreateAddress,
 		Logs:             processedTx.Logs,
 	}
-
-	// for XLayer
-	if forkID <= FORKID_ELDERBERRY {
+	if forkID <= FORKID_ETROG {
 		receipt.PostState = processedTx.StateRoot.Bytes()
 		receipt.CumulativeGasUsed = processedTx.GasUsed
 	} else {
@@ -313,9 +311,7 @@ func GenerateReceipt(blockNumber *big.Int, processedTx *ProcessTransactionRespon
 	for i := 0; i < len(receipt.Logs); i++ {
 		receipt.Logs[i].TxHash = processedTx.Tx.Hash()
 	}
-
-	// for XLayer
-	if forkID <= FORKID_ELDERBERRY {
+	if forkID <= FORKID_ETROG {
 		if processedTx.RomError == nil {
 			receipt.Status = types.ReceiptStatusSuccessful
 		} else {
diff --git a/state/transaction.go b/state/transaction.go
index fee29cbe42..f1df72c4ca 100644
--- a/state/transaction.go
+++ b/state/transaction.go
@@ -225,7 +225,7 @@ func (s *State) StoreL2Block(ctx context.Context, batchNumber uint64, l2Block *P
 
 	gasLimit := l2Block.GasLimit
 	// We check/set the maximum value of gasLimit for batches <= to ETROG fork. For batches >= to ELDERBERRY fork we use always the value returned by the executor
-	if forkID <= FORKID_ELDERBERRY && gasLimit > MaxL2BlockGasLimit {
+	if forkID <= FORKID_ETROG && gasLimit > MaxL2BlockGasLimit {
 		gasLimit = MaxL2BlockGasLimit
 	}