From ee8f5e87aaf688324fc2f71dcb7126a7af4c07b9 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 7 Oct 2024 09:52:52 +0200 Subject: [PATCH 1/3] implement L1MsgStorage --- core/rawdb/accessors_l1_message.go | 19 ++ core/rawdb/schema.go | 2 + rollup/l1/msg_storage.go | 262 ++++++++++++++++++++ rollup/l1/reader.go | 17 +- rollup/rollup_sync_service/l1client_test.go | 6 + rollup/sync_service/types.go | 1 + 6 files changed, 294 insertions(+), 13 deletions(-) create mode 100644 rollup/l1/msg_storage.go diff --git a/core/rawdb/accessors_l1_message.go b/core/rawdb/accessors_l1_message.go index 4ae27c7b8ff0..7fbd7f7cc5c6 100644 --- a/core/rawdb/accessors_l1_message.go +++ b/core/rawdb/accessors_l1_message.go @@ -275,3 +275,22 @@ func ReadFirstQueueIndexNotInL2Block(db ethdb.Reader, l2BlockHash common.Hash) * queueIndex := binary.BigEndian.Uint64(data) return &queueIndex } + +// WriteL1MsgStorageState writes the L1MsgStorage state +func WriteL1MsgStorageState(db ethdb.KeyValueWriter, state []byte) { + if err := db.Put(l1MsgStorageStateKey, state); err != nil { + log.Crit("Failed to update L1MsgStorage state", "err", err) + } +} + +// ReadL1MsgStorageState retrieves the L1MsgStorage state +func ReadL1MsgStorageState(db ethdb.Reader) []byte { + data, err := db.Get(l1MsgStorageStateKey) + if err != nil && isNotFoundErr(err) { + return nil + } + if err != nil { + log.Crit("Failed to read highest synced L1 message queue index from database", "err", err) + } + return data +} diff --git a/core/rawdb/schema.go b/core/rawdb/schema.go index 19883114fd12..7573ec280112 100644 --- a/core/rawdb/schema.go +++ b/core/rawdb/schema.go @@ -147,6 +147,8 @@ var ( firstQueueIndexNotInL2BlockPrefix = []byte("q") // firstQueueIndexNotInL2BlockPrefix + L2 block hash -> enqueue index highestSyncedQueueIndexKey = []byte("HighestSyncedQueueIndex") + l1MsgStorageStateKey = []byte("L1MsgStorageState") + // Scroll rollup event store rollupEventSyncedL1BlockNumberKey = []byte("R-LastRollupEventSyncedL1BlockNumber") batchChunkRangesPrefix = []byte("R-bcr") diff --git a/rollup/l1/msg_storage.go b/rollup/l1/msg_storage.go new file mode 100644 index 000000000000..e3eb447804fe --- /dev/null +++ b/rollup/l1/msg_storage.go @@ -0,0 +1,262 @@ +package l1 + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/log" +) + +const ( + defaultFetchInterval = 5 * time.Second +) + +type MsgStorageState struct { + StartBlockHeader *types.Header + EndBlockHeader *types.Header +} + +type MsgStorage struct { + state MsgStorageState + + ctx context.Context + cancel context.CancelFunc + + msgs *common.ShrinkingMap[uint64, storedL1Message] + reader *Reader + unsubscribeTracker func() + newChainNotifications []newChainNotification + + msgsMu sync.RWMutex + notifsMu sync.Mutex +} + +func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader) (*MsgStorage, error) { + if tracker == nil || reader == nil { + return nil, fmt.Errorf("failed to create MsgStorage, reader or tracker is nil") + } + ctx, cancel := context.WithCancel(ctx) + msgStorage := &MsgStorage{ + ctx: ctx, + cancel: cancel, + msgs: common.NewShrinkingMap[uint64, storedL1Message](1000), + reader: reader, + } + msgStorage.unsubscribeTracker = tracker.Subscribe(LatestChainHead, func(old, new []*types.Header) { + msgStorage.notifsMu.Lock() + defer msgStorage.notifsMu.Unlock() + msgStorage.newChainNotifications = append(msgStorage.newChainNotifications, newChainNotification{old, new}) + }) + + msgStorage.Start() + return msgStorage, nil +} + +func (ms *MsgStorage) Start() { + log.Info("starting MsgStorage") + go func() { + fetchTicker := time.NewTicker(defaultFetchInterval) + defer fetchTicker.Stop() + + for { + select { + case <-ms.ctx.Done(): + return + default: + } + select { + case <-ms.ctx.Done(): + return + case <-fetchTicker.C: + if len(ms.newChainNotifications) != 0 { + err := ms.fetchMessages() + if err != nil { + log.Warn("MsgStorage: failed to fetch messages", "err", err) + } + } + } + + } + }() +} + +// ReadL1Message retrieves the L1 message corresponding to the enqueue index. +func (ms *MsgStorage) ReadL1Message(queueIndex uint64) *types.L1MessageTx { + ms.msgsMu.RLock() + defer ms.msgsMu.RUnlock() + msg, exists := ms.msgs.Get(queueIndex) + if !exists { + return nil + } + return msg.l1msg +} + +// IterateL1MessagesFrom creates an L1MessageIterator that iterates over +// all L1 message in the MsgStorage starting at the provided enqueue index. +func (ms *MsgStorage) IterateL1MessagesFrom(fromQueueIndex uint64) L1MessageIterator { + return L1MessageIterator{ + curIndex: fromQueueIndex, + msgStorage: ms, + } +} + +// ReadL1MessagesFrom retrieves up to `maxCount` L1 messages starting at `startIndex`. +func (ms *MsgStorage) ReadL1MessagesFrom(startIndex, maxCount uint64) []types.L1MessageTx { + msgs := make([]types.L1MessageTx, 0, maxCount) + + index := startIndex + count := maxCount + + storedL1Msg, exists := ms.msgs.Get(index) + for count > 0 && exists { + msg := storedL1Msg.l1msg + + // sanity check + if msg.QueueIndex != index { + log.Crit( + "Unexpected QueueIndex in ReadL1MessagesFrom", + "expected", index, + "got", msg.QueueIndex, + "startIndex", startIndex, + "maxCount", maxCount, + ) + } + + msgs = append(msgs, *msg) + index += 1 + count -= 1 + storedL1Msg, exists = ms.msgs.Get(index) + } + + return msgs +} + +func (ms *MsgStorage) fetchMessages() error { + ms.notifsMu.Lock() + notifs := ms.newChainNotifications + ms.newChainNotifications = nil + ms.notifsMu.Unlock() + + // go through all chain notifications and process + for _, newChainNotification := range notifs { + old, new := newChainNotification.old, newChainNotification.new + + // check if there is old chain to delete l1msgs from + if old != nil { + // find msgs that come for reorged chain + ms.msgsMu.RLock() + msgs := ms.msgs.Values() + ms.msgsMu.RUnlock() + var indexesToDelete []uint64 + for _, msg := range msgs { + contains := false + for _, header := range old { + if header.Hash() == msg.headerHash { + contains = true + break + } + } + if contains { + indexesToDelete = append(indexesToDelete, msg.l1msg.QueueIndex) + } + } + if len(indexesToDelete) > 0 { + ms.msgsMu.Lock() + for _, index := range indexesToDelete { + ms.msgs.Delete(index) + } + ms.msgsMu.Unlock() + } + } + + // load messages from new chain + start := new[len(new)-1].Number.Uint64() + end := new[0].Number.Uint64() + events, err := ms.reader.FetchL1MessageEventsInRange(start, end) + if err != nil { + return fmt.Errorf("failed to fetch l1 messages in range, start: %d, end: %d, err: %w", start, end, err) + } + msgsToStore := make([]storedL1Message, len(events)) + for _, event := range events { + msg := &types.L1MessageTx{ + QueueIndex: event.QueueIndex, + Gas: event.GasLimit.Uint64(), + To: &event.Target, + Value: event.Value, + Data: event.Data, + Sender: event.Sender, + } + msgsToStore = append(msgsToStore, storedL1Message{ + l1msg: msg, + headerHash: event.Raw.BlockHash, + }) + } + ms.msgsMu.Lock() + for _, msg := range msgsToStore { + ms.msgs.Set(msg.l1msg.QueueIndex, msg) + } + ms.msgsMu.Unlock() + // update storage state + ms.state.EndBlockHeader = new[0] + if ms.state.StartBlockHeader == nil { + ms.state.StartBlockHeader = new[len(new)-1] + } + } + return nil +} + +// PruneMessages deletes all messages that are older or equal to provided index +func (ms *MsgStorage) PruneMessages(lastIndex uint64) { + ms.msgsMu.Lock() + defer ms.msgsMu.Unlock() + + // todo: update state for graceful restart + deleted := ms.msgs.Delete(lastIndex) + for deleted { + lastIndex-- + deleted = ms.msgs.Delete(lastIndex) + } +} + +func (ms *MsgStorage) Stop() { + log.Info("stopping MsgStorage") + ms.cancel() + log.Info("MsgStorage stopped") +} + +type storedL1Message struct { + l1msg *types.L1MessageTx + headerHash common.Hash +} + +type newChainNotification struct { + old []*types.Header + new []*types.Header +} + +type L1MessageIterator struct { + curIndex uint64 + curMsg *types.L1MessageTx + msgStorage *MsgStorage +} + +// Next moves the iterator to the next key/value pair. +// It returns false when there is no next L1Msg +func (it *L1MessageIterator) Next() bool { + it.curMsg = it.msgStorage.ReadL1Message(it.curIndex) + it.curIndex++ + if it.curMsg == nil { + return false + } else { + return true + } +} + +// L1Message returns the current L1 message. +func (it *L1MessageIterator) L1Message() types.L1MessageTx { + return *it.curMsg +} diff --git a/rollup/l1/reader.go b/rollup/l1/reader.go index dde19b8622a0..2523049c0e3c 100644 --- a/rollup/l1/reader.go +++ b/rollup/l1/reader.go @@ -144,9 +144,8 @@ func (r *Reader) FetchRollupEventsInRange(from, to uint64) (RollupEvents, error) return r.processLogsToRollupEvents(logs) } -func (r *Reader) FetchL1MessagesInRange(fromBlock, toBlock uint64) ([]types.L1MessageTx, error) { - var msgs []types.L1MessageTx - +func (r *Reader) FetchL1MessageEventsInRange(fromBlock, toBlock uint64) ([]*L1MessageQueueQueueTransaction, error) { + var events []*L1MessageQueueQueueTransaction err := r.queryInBatches(fromBlock, toBlock, defaultL1MsgFetchBlockRange, func(from, to uint64) error { it, err := r.filterer.FilterQueueTransaction(&bind.FilterOpts{ Start: from, @@ -163,22 +162,14 @@ func (r *Reader) FetchL1MessagesInRange(fromBlock, toBlock uint64) ([]types.L1Me if !event.GasLimit.IsUint64() { return fmt.Errorf("invalid QueueTransaction event: QueueIndex = %v, GasLimit = %v", event.QueueIndex, event.GasLimit) } - - msgs = append(msgs, types.L1MessageTx{ - QueueIndex: event.QueueIndex, - Gas: event.GasLimit.Uint64(), - To: &event.Target, - Value: event.Value, - Data: event.Data, - Sender: event.Sender, - }) + events = append(events, event) } return it.Error() }) if err != nil { return nil, err } - return msgs, nil + return events, nil } func (r *Reader) processLogsToRollupEvents(logs []types.Log) (RollupEvents, error) { diff --git a/rollup/rollup_sync_service/l1client_test.go b/rollup/rollup_sync_service/l1client_test.go index 38719d220f62..c6a92792cd38 100644 --- a/rollup/rollup_sync_service/l1client_test.go +++ b/rollup/rollup_sync_service/l1client_test.go @@ -57,6 +57,12 @@ func (m *mockEthClient) HeaderByNumber(ctx context.Context, number *big.Int) (*t }, nil } +func (m *mockEthClient) HeaderByHash(ctx context.Context, hash common.Hash) (*types.Header, error) { + return &types.Header{ + Number: big.NewInt(100 - 64), + }, nil +} + func (m *mockEthClient) SubscribeFilterLogs(ctx context.Context, query ethereum.FilterQuery, ch chan<- types.Log) (ethereum.Subscription, error) { return nil, nil } diff --git a/rollup/sync_service/types.go b/rollup/sync_service/types.go index 3429ec1bb778..e838b039df85 100644 --- a/rollup/sync_service/types.go +++ b/rollup/sync_service/types.go @@ -16,6 +16,7 @@ type EthClient interface { ChainID(ctx context.Context) (*big.Int, error) FilterLogs(ctx context.Context, q ethereum.FilterQuery) ([]types.Log, error) HeaderByNumber(ctx context.Context, number *big.Int) (*types.Header, error) + HeaderByHash(ctx context.Context, hash common.Hash) (*types.Header, error) SubscribeFilterLogs(ctx context.Context, query ethereum.FilterQuery, ch chan<- types.Log) (ethereum.Subscription, error) TransactionByHash(ctx context.Context, txHash common.Hash) (tx *types.Transaction, isPending bool, err error) BlockByHash(ctx context.Context, hash common.Hash) (*types.Block, error) From f12698243da0d6c7397b50905a2762e688de9705 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 21 Oct 2024 08:56:46 +0200 Subject: [PATCH 2/3] msg storage usage, address comments --- rollup/da_syncer/da/calldata_blob_source.go | 17 +++--- rollup/da_syncer/da/commitV0.go | 15 +++-- rollup/da_syncer/da/commitV1.go | 9 ++- rollup/da_syncer/da/commitV2.go | 5 +- rollup/da_syncer/da/commitV4.go | 5 +- rollup/da_syncer/data_source.go | 8 +-- rollup/da_syncer/syncing_pipeline.go | 17 +++++- rollup/l1/msg_storage.go | 65 +++++++++++---------- rollup/l1/tracker.go | 4 +- rollup/l1/tracker_test.go | 2 +- rollup/l1/types.go | 4 +- 11 files changed, 82 insertions(+), 69 deletions(-) diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go index 451e06b7d2a2..bc4b47f79c33 100644 --- a/rollup/da_syncer/da/calldata_blob_source.go +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -6,7 +6,6 @@ import ( "fmt" "github.com/scroll-tech/go-ethereum/accounts/abi" - "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/serrors" "github.com/scroll-tech/go-ethereum/rollup/l1" @@ -34,12 +33,12 @@ type CalldataBlobSource struct { blobClient blob_client.BlobClient l1height uint64 scrollChainABI *abi.ABI - db ethdb.Database + msgStorage *l1.MsgStorage l1Finalized uint64 } -func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Reader *l1.Reader, blobClient blob_client.BlobClient, db ethdb.Database) (*CalldataBlobSource, error) { +func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Reader *l1.Reader, blobClient blob_client.BlobClient, msgStorage *l1.MsgStorage) (*CalldataBlobSource, error) { scrollChainABI, err := l1.ScrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) @@ -50,7 +49,7 @@ func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Reader *l1.Re blobClient: blobClient, l1height: l1height, scrollChainABI: scrollChainABI, - db: db, + msgStorage: msgStorage, }, nil } @@ -185,11 +184,11 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, commitEvent *l } switch args.Version { case 0: - return NewCommitBatchDAV0(ds.db, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap, commitEvent.BlockNumber()) + return NewCommitBatchDAV0(ds.msgStorage, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap, commitEvent.BlockNumber()) case 1: - return NewCommitBatchDAV1(ds.ctx, ds.db, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + return NewCommitBatchDAV1(ds.ctx, ds.msgStorage, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) case 2: - return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + return NewCommitBatchDAV2(ds.ctx, ds.msgStorage, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) default: return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) } @@ -201,9 +200,9 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, commitEvent *l switch args.Version { case 3: // we can use V2 for version 3, because it's same - return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + return NewCommitBatchDAV2(ds.ctx, ds.msgStorage, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) case 4: - return NewCommitBatchDAV4(ds.ctx, ds.db, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + return NewCommitBatchDAV4(ds.ctx, ds.msgStorage, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) default: return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) } diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 66a13786c9cb..bd8d30207350 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -7,10 +7,9 @@ import ( "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" - "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/core/types" - "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/serrors" + "github.com/scroll-tech/go-ethereum/rollup/l1" ) type CommitBatchDAV0 struct { @@ -24,7 +23,7 @@ type CommitBatchDAV0 struct { l1BlockNumber uint64 } -func NewCommitBatchDAV0(db ethdb.Database, +func NewCommitBatchDAV0(msgStorage *l1.MsgStorage, version uint8, batchIndex uint64, parentBatchHeader []byte, @@ -37,10 +36,10 @@ func NewCommitBatchDAV0(db ethdb.Database, return nil, fmt.Errorf("failed to unpack chunks: %d, err: %w", batchIndex, err) } - return NewCommitBatchDAV0WithChunks(db, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, l1BlockNumber) + return NewCommitBatchDAV0WithChunks(msgStorage, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, l1BlockNumber) } -func NewCommitBatchDAV0WithChunks(db ethdb.Database, +func NewCommitBatchDAV0WithChunks(msgStorage *l1.MsgStorage, version uint8, batchIndex uint64, parentBatchHeader []byte, @@ -49,7 +48,7 @@ func NewCommitBatchDAV0WithChunks(db ethdb.Database, l1BlockNumber uint64, ) (*CommitBatchDAV0, error) { parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(parentBatchHeader) - l1Txs, err := getL1Messages(db, parentTotalL1MessagePopped, skippedL1MessageBitmap, getTotalMessagesPoppedFromChunks(decodedChunks)) + l1Txs, err := getL1Messages(msgStorage, parentTotalL1MessagePopped, skippedL1MessageBitmap, getTotalMessagesPoppedFromChunks(decodedChunks)) if err != nil { return nil, fmt.Errorf("failed to get L1 messages for v0 batch %d: %w", batchIndex, err) } @@ -139,7 +138,7 @@ func getTotalMessagesPoppedFromChunks(decodedChunks []*codecv0.DAChunkRawTx) int return totalL1MessagePopped } -func getL1Messages(db ethdb.Database, parentTotalL1MessagePopped uint64, skippedBitmap []byte, totalL1MessagePopped int) ([]*types.L1MessageTx, error) { +func getL1Messages(msgStorage *l1.MsgStorage, parentTotalL1MessagePopped uint64, skippedBitmap []byte, totalL1MessagePopped int) ([]*types.L1MessageTx, error) { var txs []*types.L1MessageTx decodedSkippedBitmap, err := encoding.DecodeBitmap(skippedBitmap, totalL1MessagePopped) @@ -154,7 +153,7 @@ func getL1Messages(db ethdb.Database, parentTotalL1MessagePopped uint64, skipped currentIndex++ continue } - l1Tx := rawdb.ReadL1Message(db, currentIndex) + l1Tx := msgStorage.ReadL1Message(currentIndex) if l1Tx == nil { // message not yet available // we return serrors.EOFError as this will be handled in the syncing pipeline with a backoff and retry diff --git a/rollup/da_syncer/da/commitV1.go b/rollup/da_syncer/da/commitV1.go index 31ad2a668084..5109b310b503 100644 --- a/rollup/da_syncer/da/commitV1.go +++ b/rollup/da_syncer/da/commitV1.go @@ -13,14 +13,13 @@ import ( "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" - "github.com/scroll-tech/go-ethereum/ethdb" ) type CommitBatchDAV1 struct { *CommitBatchDAV0 } -func NewCommitBatchDAV1(ctx context.Context, db ethdb.Database, +func NewCommitBatchDAV1(ctx context.Context, msgStorage *l1.MsgStorage, l1Reader *l1.Reader, blobClient blob_client.BlobClient, commitEvent *l1.CommitBatchEvent, @@ -30,10 +29,10 @@ func NewCommitBatchDAV1(ctx context.Context, db ethdb.Database, chunks [][]byte, skippedL1MessageBitmap []byte, ) (*CommitBatchDAV1, error) { - return NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv1.DecodeTxsFromBlob) + return NewCommitBatchDAV1WithBlobDecodeFunc(ctx, msgStorage, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv1.DecodeTxsFromBlob) } -func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, db ethdb.Database, +func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, msgStorage *l1.MsgStorage, l1Reader *l1.Reader, blobClient blob_client.BlobClient, commitEvent *l1.CommitBatchEvent, @@ -82,7 +81,7 @@ func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, db ethdb.Database return nil, fmt.Errorf("failed to decode txs from blob: %w", err) } - v0, err := NewCommitBatchDAV0WithChunks(db, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, commitEvent.BlockNumber()) + v0, err := NewCommitBatchDAV0WithChunks(msgStorage, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, commitEvent.BlockNumber()) if err != nil { return nil, err } diff --git a/rollup/da_syncer/da/commitV2.go b/rollup/da_syncer/da/commitV2.go index 56110ff739bc..6231cf2e57f0 100644 --- a/rollup/da_syncer/da/commitV2.go +++ b/rollup/da_syncer/da/commitV2.go @@ -5,7 +5,6 @@ import ( "github.com/scroll-tech/da-codec/encoding/codecv2" - "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/l1" ) @@ -14,7 +13,7 @@ type CommitBatchDAV2 struct { *CommitBatchDAV1 } -func NewCommitBatchDAV2(ctx context.Context, db ethdb.Database, +func NewCommitBatchDAV2(ctx context.Context, msgStorage *l1.MsgStorage, l1Reader *l1.Reader, blobClient blob_client.BlobClient, commitEvent *l1.CommitBatchEvent, @@ -25,7 +24,7 @@ func NewCommitBatchDAV2(ctx context.Context, db ethdb.Database, skippedL1MessageBitmap []byte, ) (*CommitBatchDAV2, error) { - v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv2.DecodeTxsFromBlob) + v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, msgStorage, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv2.DecodeTxsFromBlob) if err != nil { return nil, err } diff --git a/rollup/da_syncer/da/commitV4.go b/rollup/da_syncer/da/commitV4.go index 3d3366386756..324c9eb4a87d 100644 --- a/rollup/da_syncer/da/commitV4.go +++ b/rollup/da_syncer/da/commitV4.go @@ -5,7 +5,6 @@ import ( "github.com/scroll-tech/da-codec/encoding/codecv4" - "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/l1" ) @@ -14,7 +13,7 @@ type CommitBatchDAV4 struct { *CommitBatchDAV1 } -func NewCommitBatchDAV4(ctx context.Context, db ethdb.Database, +func NewCommitBatchDAV4(ctx context.Context, msgStorage *l1.MsgStorage, l1Reader *l1.Reader, blobClient blob_client.BlobClient, commitEvent *l1.CommitBatchEvent, @@ -25,7 +24,7 @@ func NewCommitBatchDAV4(ctx context.Context, db ethdb.Database, skippedL1MessageBitmap []byte, ) (*CommitBatchDAV2, error) { - v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv4.DecodeTxsFromBlob) + v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, msgStorage, l1Reader, blobClient, commitEvent, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv4.DecodeTxsFromBlob) if err != nil { return nil, err } diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index 582b72037c32..f456d55d096c 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -5,7 +5,6 @@ import ( "errors" "github.com/scroll-tech/go-ethereum/core" - "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" @@ -22,22 +21,21 @@ type DataSourceFactory struct { genesisConfig *params.ChainConfig l1Reader *l1.Reader blobClient blob_client.BlobClient - db ethdb.Database + msgStorage *l1.MsgStorage } -func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Reader *l1.Reader, blobClient blob_client.BlobClient, db ethdb.Database) *DataSourceFactory { +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Reader *l1.Reader, msgStorage *l1.MsgStorage, blobClient blob_client.BlobClient) *DataSourceFactory { return &DataSourceFactory{ config: config, genesisConfig: genesisConfig, l1Reader: l1Reader, blobClient: blobClient, - db: db, } } func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { if ds.config.FetcherMode == L1RPC { - return da.NewCalldataBlobSource(ctx, l1height, ds.l1Reader, ds.blobClient, ds.db) + return da.NewCalldataBlobSource(ctx, l1height, ds.l1Reader, ds.blobClient, ds.msgStorage) } else { return nil, errors.New("snapshot_data_source: not implemented") } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index c27ee6733a9d..6fb25d6d80aa 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -36,6 +36,8 @@ type SyncingPipeline struct { expBackoff *backoff.Exponential l1DeploymentBlock uint64 + l1Tracker *l1.Tracker + msgStorage *l1.MsgStorage db ethdb.Database blockchain *core.BlockChain @@ -51,6 +53,12 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi if err != nil { return nil, fmt.Errorf("failed to initialize l1.Reader, err = %w", err) } + l1Tracker := l1.NewTracker(ctx, ethClient, blockchain.Genesis().Hash()) + + msgStorage, err := l1.NewMsgStorage(ctx, l1Tracker, l1Reader, l1.LatestChainHead) + if err != nil { + return nil, fmt.Errorf("failed to initialize msg storage, err = %w", err) + } blobClientList := blob_client.NewBlobClientList() if config.BeaconNodeAPIEndpoint != "" { @@ -71,7 +79,7 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi log.Crit("DA syncing is enabled but no blob client is configured. Please provide at least one blob client via command line flag.") } - dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Reader, blobClientList, db) + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Reader, msgStorage, blobClientList) syncedL1Height := l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) if from != nil { @@ -90,6 +98,8 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi expBackoff: backoff.NewExponential(100*time.Millisecond, 10*time.Second, 100*time.Millisecond), wg: sync.WaitGroup{}, l1DeploymentBlock: l1DeploymentBlock, + l1Tracker: l1Tracker, + msgStorage: msgStorage, db: db, blockchain: blockchain, blockQueue: blockQueue, @@ -109,6 +119,9 @@ func (s *SyncingPipeline) Step() error { func (s *SyncingPipeline) Start() { log.Info("sync from DA: starting pipeline") + s.msgStorage.Start() + s.l1Tracker.Start() + s.wg.Add(1) go func() { s.mainLoop() @@ -213,6 +226,8 @@ func (s *SyncingPipeline) mainLoop() { func (s *SyncingPipeline) Stop() { log.Info("sync from DA: stopping pipeline...") s.cancel() + s.msgStorage.Stop() + s.l1Tracker.Stop() s.wg.Wait() log.Info("sync from DA: stopping pipeline... done") } diff --git a/rollup/l1/msg_storage.go b/rollup/l1/msg_storage.go index e3eb447804fe..b8b267563eb9 100644 --- a/rollup/l1/msg_storage.go +++ b/rollup/l1/msg_storage.go @@ -28,36 +28,36 @@ type MsgStorage struct { msgs *common.ShrinkingMap[uint64, storedL1Message] reader *Reader + tracker *Tracker + confirmationRule ConfirmationRule unsubscribeTracker func() - newChainNotifications []newChainNotification + newChainNotifications chan newChainNotification - msgsMu sync.RWMutex - notifsMu sync.Mutex + msgsMu sync.RWMutex } -func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader) (*MsgStorage, error) { +func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader, confirmationRule ConfirmationRule) (*MsgStorage, error) { if tracker == nil || reader == nil { return nil, fmt.Errorf("failed to create MsgStorage, reader or tracker is nil") } ctx, cancel := context.WithCancel(ctx) msgStorage := &MsgStorage{ - ctx: ctx, - cancel: cancel, - msgs: common.NewShrinkingMap[uint64, storedL1Message](1000), - reader: reader, + ctx: ctx, + cancel: cancel, + msgs: common.NewShrinkingMap[uint64, storedL1Message](1000), + reader: reader, + tracker: tracker, + confirmationRule: confirmationRule, + newChainNotifications: make(chan newChainNotification, 10), } - msgStorage.unsubscribeTracker = tracker.Subscribe(LatestChainHead, func(old, new []*types.Header) { - msgStorage.notifsMu.Lock() - defer msgStorage.notifsMu.Unlock() - msgStorage.newChainNotifications = append(msgStorage.newChainNotifications, newChainNotification{old, new}) - }) - - msgStorage.Start() return msgStorage, nil } func (ms *MsgStorage) Start() { log.Info("starting MsgStorage") + ms.unsubscribeTracker = ms.tracker.Subscribe(ms.confirmationRule, func(old, new []*types.Header) { + ms.newChainNotifications <- newChainNotification{old, new} + }, 64) go func() { fetchTicker := time.NewTicker(defaultFetchInterval) defer fetchTicker.Stop() @@ -106,16 +106,18 @@ func (ms *MsgStorage) IterateL1MessagesFrom(fromQueueIndex uint64) L1MessageIter // ReadL1MessagesFrom retrieves up to `maxCount` L1 messages starting at `startIndex`. func (ms *MsgStorage) ReadL1MessagesFrom(startIndex, maxCount uint64) []types.L1MessageTx { + if maxCount == 0 { + return []types.L1MessageTx{} + } msgs := make([]types.L1MessageTx, 0, maxCount) - index := startIndex - count := maxCount - - storedL1Msg, exists := ms.msgs.Get(index) - for count > 0 && exists { + for index := startIndex; len(msgs) < int(maxCount); index++ { + storedL1Msg, exists := ms.msgs.Get(index) + if !exists { + break // No more messages to read + } msg := storedL1Msg.l1msg - - // sanity check + // Sanity check for QueueIndex if msg.QueueIndex != index { log.Crit( "Unexpected QueueIndex in ReadL1MessagesFrom", @@ -125,21 +127,22 @@ func (ms *MsgStorage) ReadL1MessagesFrom(startIndex, maxCount uint64) []types.L1 "maxCount", maxCount, ) } - msgs = append(msgs, *msg) - index += 1 - count -= 1 - storedL1Msg, exists = ms.msgs.Get(index) } - return msgs } func (ms *MsgStorage) fetchMessages() error { - ms.notifsMu.Lock() - notifs := ms.newChainNotifications - ms.newChainNotifications = nil - ms.notifsMu.Unlock() + var notifs []newChainNotification +out: + for { + select { + case msg := <-ms.newChainNotifications: + notifs = append(notifs, msg) + default: + break out + } + } // go through all chain notifications and process for _, newChainNotification := range notifs { diff --git a/rollup/l1/tracker.go b/rollup/l1/tracker.go index 73633f810cb7..b6b030d7e295 100644 --- a/rollup/l1/tracker.go +++ b/rollup/l1/tracker.go @@ -408,7 +408,7 @@ func (t *Tracker) chain(start, end *types.Header, includeStart bool) []*types.He return chain } -func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback SubscriptionCallback) (unsubscribe func()) { +func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback SubscriptionCallback, maxHeadersSent int) (unsubscribe func()) { t.mu.Lock() defer t.mu.Unlock() @@ -425,7 +425,7 @@ func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback Subscrip panic(fmt.Sprintf("invalid confirmation rule %d", confirmationRule)) } - sub := newSubscription(t.subscriptionCounter, confirmationRule, callback) + sub := newSubscription(t.subscriptionCounter, confirmationRule, callback, maxHeadersSent) subscriptionsByType := t.subscriptions[confirmationType] subscriptionsByType = append(subscriptionsByType, sub) diff --git a/rollup/l1/tracker_test.go b/rollup/l1/tracker_test.go index 954a625f4c2b..11e79bc2b1b2 100644 --- a/rollup/l1/tracker_test.go +++ b/rollup/l1/tracker_test.go @@ -171,7 +171,7 @@ func newSubscriptionCalls(tracker *Tracker, alias string, rule ConfirmationRule) unsubscribe := tracker.Subscribe(rule, func(old, new []*types.Header) { s.addActual(old, new) - }) + }, 0) return s, unsubscribe } diff --git a/rollup/l1/types.go b/rollup/l1/types.go index ed4535d42cdc..998078eb3d6a 100644 --- a/rollup/l1/types.go +++ b/rollup/l1/types.go @@ -47,13 +47,15 @@ type subscription struct { confirmationRule ConfirmationRule callback SubscriptionCallback lastSentHeader *types.Header + maxHeadersSent int // number of headers that could be sent at the time } -func newSubscription(id int, confirmationRule ConfirmationRule, callback SubscriptionCallback) *subscription { +func newSubscription(id int, confirmationRule ConfirmationRule, callback SubscriptionCallback, maxHeadersSent int) *subscription { return &subscription{ id: id, confirmationRule: confirmationRule, callback: callback, + maxHeadersSent: maxHeadersSent, } } From 48b784cdc57b0aa0978be6a4270764ff39291872 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Sun, 3 Nov 2024 20:19:09 +0100 Subject: [PATCH 3/3] tracker with oflline/online subscribers, msgstorage adjustments --- eth/backend.go | 10 +- rollup/da_syncer/da/calldata_blob_source.go | 12 +- rollup/da_syncer/da/commitV1.go | 6 +- rollup/da_syncer/data_source.go | 1 + rollup/da_syncer/syncing_pipeline.go | 28 ++- rollup/l1/abi.go | 73 ++----- rollup/l1/abi_test.go | 12 +- rollup/l1/msg_storage.go | 179 +++++++++++++---- rollup/l1/reader.go | 27 ++- rollup/l1/tracker.go | 210 +++++++++++--------- rollup/l1/tracker_test.go | 122 +++++++----- rollup/l1/types.go | 11 +- 12 files changed, 422 insertions(+), 269 deletions(-) diff --git a/eth/backend.go b/eth/backend.go index ee94f867d846..74dc893bc745 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -254,10 +254,12 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl eth.syncingPipeline.Start() } - // initialize and start L1 message sync service - eth.syncService, err = sync_service.NewSyncService(context.Background(), chainConfig, stack.Config(), eth.chainDb, l1Client) - if err != nil { - return nil, fmt.Errorf("cannot initialize L1 sync service: %w", err) + if !config.EnableDASyncing { + // initialize and start L1 message sync service + eth.syncService, err = sync_service.NewSyncService(context.Background(), chainConfig, stack.Config(), eth.chainDb, l1Client) + if err != nil { + return nil, fmt.Errorf("cannot initialize L1 sync service: %w", err) + } } eth.syncService.Start() if config.EnableRollupVerify { diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go index bc4b47f79c33..a475866aed08 100644 --- a/rollup/da_syncer/da/calldata_blob_source.go +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -102,15 +102,15 @@ func (ds *CalldataBlobSource) processRollupEventsToDA(rollupEvents l1.RollupEven if !ok { return nil, fmt.Errorf("unexpected type of rollup event: %T", rollupEvent) } - if entry, err = ds.getCommitBatchDA(rollupEvent.BatchIndex().Uint64(), commitEvent); err != nil { - return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", rollupEvent.BatchIndex().Uint64(), err) + if entry, err = ds.getCommitBatchDA(rollupEvent.GetBatchIndex().Uint64(), commitEvent); err != nil { + return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", rollupEvent.GetBatchIndex().Uint64(), err) } case l1.RevertEventType: - entry = NewRevertBatch(rollupEvent.BatchIndex().Uint64()) + entry = NewRevertBatch(rollupEvent.GetBatchIndex().Uint64()) case l1.FinalizeEventType: - entry = NewFinalizeBatch(rollupEvent.BatchIndex().Uint64()) + entry = NewFinalizeBatch(rollupEvent.GetBatchIndex().Uint64()) default: return nil, fmt.Errorf("unknown rollup event, type: %v", rollupEvent.Type()) @@ -160,7 +160,7 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, commitEvent *l return NewCommitBatchDAV0Empty(), nil } - txData, err := ds.l1Reader.FetchTxData(commitEvent.TxHash(), commitEvent.BlockHash()) + txData, err := ds.l1Reader.FetchTxData(commitEvent.TxHash, commitEvent.BlockHash) if err != nil { return nil, err } @@ -184,7 +184,7 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, commitEvent *l } switch args.Version { case 0: - return NewCommitBatchDAV0(ds.msgStorage, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap, commitEvent.BlockNumber()) + return NewCommitBatchDAV0(ds.msgStorage, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap, commitEvent.BlockNumber) case 1: return NewCommitBatchDAV1(ds.ctx, ds.msgStorage, ds.l1Reader, ds.blobClient, commitEvent, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) case 2: diff --git a/rollup/da_syncer/da/commitV1.go b/rollup/da_syncer/da/commitV1.go index 5109b310b503..0af18aebe01b 100644 --- a/rollup/da_syncer/da/commitV1.go +++ b/rollup/da_syncer/da/commitV1.go @@ -48,12 +48,12 @@ func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, msgStorage *l1.Ms return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - versionedHash, err := l1Reader.FetchTxBlobHash(commitEvent.TxHash(), commitEvent.BlockHash()) + versionedHash, err := l1Reader.FetchTxBlobHash(commitEvent.TxHash, commitEvent.BlockHash) if err != nil { return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) } - header, err := l1Reader.FetchBlockHeaderByNumber(commitEvent.BlockNumber()) + header, err := l1Reader.FetchBlockHeaderByNumber(commitEvent.BlockNumber) if err != nil { return nil, fmt.Errorf("failed to get header by number, err: %w", err) } @@ -81,7 +81,7 @@ func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, msgStorage *l1.Ms return nil, fmt.Errorf("failed to decode txs from blob: %w", err) } - v0, err := NewCommitBatchDAV0WithChunks(msgStorage, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, commitEvent.BlockNumber()) + v0, err := NewCommitBatchDAV0WithChunks(msgStorage, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, commitEvent.BlockNumber) if err != nil { return nil, err } diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index f456d55d096c..a5bfdbad3e17 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -30,6 +30,7 @@ func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.Cha genesisConfig: genesisConfig, l1Reader: l1Reader, blobClient: blobClient, + msgStorage: msgStorage, } } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 6fb25d6d80aa..c650ec6ee2f2 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -18,6 +18,10 @@ import ( "github.com/scroll-tech/go-ethereum/rollup/l1" ) +const ( + defaultPruneInterval = 30 * time.Second +) + // Config is the configuration parameters of data availability syncing. type Config struct { FetcherMode FetcherMode // mode of fetcher @@ -53,9 +57,13 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi if err != nil { return nil, fmt.Errorf("failed to initialize l1.Reader, err = %w", err) } + l1DeploymentBlockHeader, err := l1Reader.FetchBlockHeaderByNumber(l1DeploymentBlock) + if err != nil { + return nil, fmt.Errorf("reader failed to fetch block header by number, err = %w", err) + } l1Tracker := l1.NewTracker(ctx, ethClient, blockchain.Genesis().Hash()) - msgStorage, err := l1.NewMsgStorage(ctx, l1Tracker, l1Reader, l1.LatestChainHead) + msgStorage, err := l1.NewMsgStorage(ctx, l1Tracker, l1Reader, db, l1.LatestChainHead, l1DeploymentBlockHeader) if err != nil { return nil, fmt.Errorf("failed to initialize msg storage, err = %w", err) } @@ -135,6 +143,9 @@ func (s *SyncingPipeline) mainLoop() { var resetCounter int var tempErrorCounter int + l1msgPruneTicker := time.NewTicker(defaultPruneInterval) + defer l1msgPruneTicker.Stop() + // reqStep is a helper function to request a step to be executed. // If delay is true, it will request a delayed step with exponential backoff, otherwise it will request an immediate step. reqStep := func(delay bool) { @@ -163,6 +174,21 @@ func (s *SyncingPipeline) mainLoop() { return default: } + select { + case <-s.ctx.Done(): + return + case <-l1msgPruneTicker.C: + nextQueueIndex := rawdb.ReadFirstQueueIndexNotInL2Block(s.db, s.blockchain.CurrentBlock().Hash()) + if nextQueueIndex == nil { + // should not happen + log.Warn("FirstQueueIndexNotInL2Block is nil for block", "hash", s.blockchain.CurrentBlock().Hash()) + continue + } + if *nextQueueIndex > 0 { + s.msgStorage.PruneMessages(*nextQueueIndex - 1) + } + default: + } select { case <-s.ctx.Done(): diff --git a/rollup/l1/abi.go b/rollup/l1/abi.go index 0d8e14193903..49919ce37546 100644 --- a/rollup/l1/abi.go +++ b/rollup/l1/abi.go @@ -27,95 +27,62 @@ const ( // RollupEvent represents a single rollup event (commit, revert, finalize) type RollupEvent interface { Type() int - BatchIndex() *big.Int - BatchHash() common.Hash + GetBatchIndex() *big.Int } type RollupEvents []RollupEvent // CommitBatchEventUnpacked represents a CommitBatch event raised by the ScrollChain contract. type CommitBatchEventUnpacked struct { - batchIndex *big.Int - batchHash common.Hash + BatchIndex *big.Int + BatchHash common.Hash } // CommitBatchEvent represents a CommitBatch event raised by the ScrollChain contract with additional fields. type CommitBatchEvent struct { - batchIndex *big.Int - batchHash common.Hash - txHash common.Hash - blockHash common.Hash - blockNumber uint64 + BatchIndex *big.Int + BatchHash common.Hash + TxHash common.Hash + BlockHash common.Hash + BlockNumber uint64 } func (c *CommitBatchEvent) Type() int { return CommitEventType } -func (c *CommitBatchEvent) BatchIndex() *big.Int { - return c.batchIndex -} - -func (c *CommitBatchEvent) BatchHash() common.Hash { - return c.batchHash -} - -func (c *CommitBatchEvent) TxHash() common.Hash { - return c.txHash -} - -func (c *CommitBatchEvent) BlockHash() common.Hash { - return c.blockHash -} - -func (c *CommitBatchEvent) BlockNumber() uint64 { - return c.blockNumber +func (c *CommitBatchEvent) GetBatchIndex() *big.Int { + return c.BatchIndex } // RevertBatchEvent represents a RevertBatch event raised by the ScrollChain contract. type RevertBatchEvent struct { - batchIndex *big.Int - batchHash common.Hash + BatchIndex *big.Int + BatchHash common.Hash } func (r *RevertBatchEvent) Type() int { return RevertEventType } -func (r *RevertBatchEvent) BatchIndex() *big.Int { - return r.batchIndex -} - -func (r *RevertBatchEvent) BatchHash() common.Hash { - return r.batchHash +func (r *RevertBatchEvent) GetBatchIndex() *big.Int { + return r.BatchIndex } // FinalizeBatchEvent represents a FinalizeBatch event raised by the ScrollChain contract. type FinalizeBatchEvent struct { - batchIndex *big.Int - batchHash common.Hash - stateRoot common.Hash - withdrawRoot common.Hash + BatchIndex *big.Int + BatchHash common.Hash + StateRoot common.Hash + WithdrawRoot common.Hash } func (r *FinalizeBatchEvent) Type() int { return FinalizeEventType } -func (r *FinalizeBatchEvent) BatchIndex() *big.Int { - return r.batchIndex -} - -func (r *FinalizeBatchEvent) BatchHash() common.Hash { - return r.batchHash -} - -func (r *FinalizeBatchEvent) StateRoot() common.Hash { - return r.stateRoot -} - -func (r *FinalizeBatchEvent) WithdrawRoot() common.Hash { - return r.withdrawRoot +func (r *FinalizeBatchEvent) GetBatchIndex() *big.Int { + return r.BatchIndex } // UnpackLog unpacks a retrieved log into the provided output structure. diff --git a/rollup/l1/abi_test.go b/rollup/l1/abi_test.go index ab4c9d473a16..13e27276db8e 100644 --- a/rollup/l1/abi_test.go +++ b/rollup/l1/abi_test.go @@ -44,7 +44,7 @@ func TestUnpackLog(t *testing.T) { Data: []byte{}, Topics: []common.Hash{scrollChainABI.Events["CommitBatch"].ID, common.BigToHash(mockBatchIndex), mockBatchHash}, }, - &CommitBatchEvent{batchIndex: mockBatchIndex, batchHash: mockBatchHash}, + &CommitBatchEvent{BatchIndex: mockBatchIndex, BatchHash: mockBatchHash}, &CommitBatchEvent{}, }, { @@ -53,7 +53,7 @@ func TestUnpackLog(t *testing.T) { Data: []byte{}, Topics: []common.Hash{scrollChainABI.Events["RevertBatch"].ID, common.BigToHash(mockBatchIndex), mockBatchHash}, }, - &RevertBatchEvent{batchIndex: mockBatchIndex, batchHash: mockBatchHash}, + &RevertBatchEvent{BatchIndex: mockBatchIndex, BatchHash: mockBatchHash}, &RevertBatchEvent{}, }, { @@ -63,10 +63,10 @@ func TestUnpackLog(t *testing.T) { Topics: []common.Hash{scrollChainABI.Events["FinalizeBatch"].ID, common.BigToHash(mockBatchIndex), mockBatchHash}, }, &FinalizeBatchEvent{ - batchIndex: mockBatchIndex, - batchHash: mockBatchHash, - stateRoot: mockStateRoot, - withdrawRoot: mockWithdrawRoot, + BatchIndex: mockBatchIndex, + BatchHash: mockBatchHash, + StateRoot: mockStateRoot, + WithdrawRoot: mockWithdrawRoot, }, &FinalizeBatchEvent{}, }, diff --git a/rollup/l1/msg_storage.go b/rollup/l1/msg_storage.go index b8b267563eb9..62fd3508732e 100644 --- a/rollup/l1/msg_storage.go +++ b/rollup/l1/msg_storage.go @@ -1,18 +1,24 @@ package l1 import ( + "bytes" "context" "fmt" "sync" "time" "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rlp" ) const ( - defaultFetchInterval = 5 * time.Second + defaultFetchInterval = 5 * time.Second + maxMsgsStored = 2000 + defaultFetchBlockRange = uint64(500) ) type MsgStorageState struct { @@ -26,17 +32,19 @@ type MsgStorage struct { ctx context.Context cancel context.CancelFunc + db ethdb.Database msgs *common.ShrinkingMap[uint64, storedL1Message] reader *Reader tracker *Tracker confirmationRule ConfirmationRule unsubscribeTracker func() newChainNotifications chan newChainNotification + latestFinalized *types.Header msgsMu sync.RWMutex } -func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader, confirmationRule ConfirmationRule) (*MsgStorage, error) { +func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader, db ethdb.Database, confirmationRule ConfirmationRule, l1DeploymentBlockHeader *types.Header) (*MsgStorage, error) { if tracker == nil || reader == nil { return nil, fmt.Errorf("failed to create MsgStorage, reader or tracker is nil") } @@ -44,20 +52,43 @@ func NewMsgStorage(ctx context.Context, tracker *Tracker, reader *Reader, confir msgStorage := &MsgStorage{ ctx: ctx, cancel: cancel, - msgs: common.NewShrinkingMap[uint64, storedL1Message](1000), + db: db, + msgs: common.NewShrinkingMap[uint64, storedL1Message](maxMsgsStored), reader: reader, tracker: tracker, confirmationRule: confirmationRule, newChainNotifications: make(chan newChainNotification, 10), + state: MsgStorageState{ + StartBlockHeader: l1DeploymentBlockHeader, + EndBlockHeader: l1DeploymentBlockHeader, + }, + latestFinalized: l1DeploymentBlockHeader, } return msgStorage, nil } func (ms *MsgStorage) Start() { log.Info("starting MsgStorage") - ms.unsubscribeTracker = ms.tracker.Subscribe(ms.confirmationRule, func(old, new []*types.Header) { - ms.newChainNotifications <- newChainNotification{old, new} - }, 64) + + data := rawdb.ReadL1MsgStorageState(ms.db) + state := &MsgStorageState{} + if err := rlp.Decode(bytes.NewReader(data), state); err == nil { + ms.state = *state + ms.resyncToState() + } + ms.unsubscribeTracker = ms.tracker.Subscribe(ms.confirmationRule, func(isOnline bool, old, new []*types.Header) bool { + if isOnline { + ms.newChainNotifications <- newChainNotification{old, new} + return true + } else { + ms.latestFinalized = new[0] + // check if latest synced block is already finalized or later -> can become an online subscriber + if ms.state.EndBlockHeader.Number.Uint64() >= new[0].Number.Uint64() { + return true + } + } + return false + }) go func() { fetchTicker := time.NewTicker(defaultFetchInterval) defer fetchTicker.Stop() @@ -72,8 +103,13 @@ func (ms *MsgStorage) Start() { case <-ms.ctx.Done(): return case <-fetchTicker.C: - if len(ms.newChainNotifications) != 0 { - err := ms.fetchMessages() + if ms.state.EndBlockHeader.Number.Uint64() < ms.latestFinalized.Number.Uint64() { + err := ms.syncToTheFinalizedOrLimit() + if err != nil { + log.Warn("MsgStorage: failed to fetch messages", "err", err) + } + } else if len(ms.newChainNotifications) != 0 { + err := ms.syncRecentNotifs() if err != nil { log.Warn("MsgStorage: failed to fetch messages", "err", err) } @@ -132,7 +168,81 @@ func (ms *MsgStorage) ReadL1MessagesFrom(startIndex, maxCount uint64) []types.L1 return msgs } -func (ms *MsgStorage) fetchMessages() error { +func (ms *MsgStorage) syncRange(from, to uint64) error { + events, err := ms.reader.FetchL1MessageEventsInRange(from, to) + if err != nil { + return fmt.Errorf("failed to fetch l1 messages in range, start: %d, end: %d, err: %w", from, to, err) + } + msgsToStore := make([]storedL1Message, 0, len(events)) + for _, event := range events { + msg := &types.L1MessageTx{ + QueueIndex: event.QueueIndex, + Gas: event.GasLimit.Uint64(), + To: &event.Target, + Value: event.Value, + Data: event.Data, + Sender: event.Sender, + } + msgsToStore = append(msgsToStore, storedL1Message{ + l1msg: msg, + headerHash: event.Raw.BlockHash, + }) + } + ms.msgsMu.Lock() + for _, msg := range msgsToStore { + ms.msgs.Set(msg.l1msg.QueueIndex, msg) + } + ms.msgsMu.Unlock() + return nil +} + +func (ms *MsgStorage) resyncToState() error { + log.Info("MsgStorage: resync to state", "start block number", ms.state.StartBlockHeader.Number.Uint64(), "endBlockNumber", ms.state.EndBlockHeader.Number.Uint64()) + endBlockNumber := ms.state.EndBlockHeader.Number.Uint64() + for from := ms.state.StartBlockHeader.Number.Uint64(); from <= endBlockNumber; from += defaultFetchBlockRange { + to := from + defaultFetchBlockRange - 1 + if to > endBlockNumber { + to = endBlockNumber + } + if err := ms.syncRange(from, to); err != nil { + return err + } + } + log.Info("MsgStorage: resync to state finished", "number of messages", ms.msgs.Size()) + + // don't need to update state here, because we are syncing to match defined state + return nil +} + +func (ms *MsgStorage) syncToTheFinalizedOrLimit() error { + if ms.msgs.Size() >= maxMsgsStored { + return nil + } + for { + from := ms.state.EndBlockHeader.Number.Uint64() + 1 + if from > ms.latestFinalized.Number.Uint64() { + break + } + to := from + defaultFetchBlockRange + if err := ms.syncRange(from, to); err != nil { + return err + } + // update storage state + toHeader, err := ms.reader.FetchBlockHeaderByNumber(to) + if err != nil { + return fmt.Errorf("failed to fetch block header by number %d, err: %w", to, err) + } + ms.state.EndBlockHeader = toHeader + ms.updateStorageState() + // if the number of stored l1 msgs exceeded the limit stop syncing. It will remain when some of the old messages pruned + if ms.msgs.Size() >= maxMsgsStored { + break + } + } + return nil +} + +func (ms *MsgStorage) syncRecentNotifs() error { var notifs []newChainNotification out: for { @@ -179,45 +289,41 @@ out: // load messages from new chain start := new[len(new)-1].Number.Uint64() end := new[0].Number.Uint64() - events, err := ms.reader.FetchL1MessageEventsInRange(start, end) - if err != nil { - return fmt.Errorf("failed to fetch l1 messages in range, start: %d, end: %d, err: %w", start, end, err) - } - msgsToStore := make([]storedL1Message, len(events)) - for _, event := range events { - msg := &types.L1MessageTx{ - QueueIndex: event.QueueIndex, - Gas: event.GasLimit.Uint64(), - To: &event.Target, - Value: event.Value, - Data: event.Data, - Sender: event.Sender, - } - msgsToStore = append(msgsToStore, storedL1Message{ - l1msg: msg, - headerHash: event.Raw.BlockHash, - }) + if err := ms.syncRange(start, end); err != nil { + return err } - ms.msgsMu.Lock() - for _, msg := range msgsToStore { - ms.msgs.Set(msg.l1msg.QueueIndex, msg) - } - ms.msgsMu.Unlock() // update storage state ms.state.EndBlockHeader = new[0] - if ms.state.StartBlockHeader == nil { - ms.state.StartBlockHeader = new[len(new)-1] - } + ms.updateStorageState() } return nil } +func (ms *MsgStorage) updateStorageState() { + bytes, err := rlp.EncodeToBytes(ms.state) + if err != nil { + log.Crit("Failed to RLP encode message storage state", "err", err) + } + rawdb.WriteL1MsgStorageState(ms.db, bytes) +} + // PruneMessages deletes all messages that are older or equal to provided index func (ms *MsgStorage) PruneMessages(lastIndex uint64) { + log.Info("PruneMessages", "size", ms.msgs.Size(), "lastIndex", lastIndex) ms.msgsMu.Lock() defer ms.msgsMu.Unlock() - // todo: update state for graceful restart + // update storage state + lastBlock, exists := ms.msgs.Get(lastIndex) + fmt.Println(lastBlock, exists) + if exists { + header, err := ms.reader.FetchBlockHeaderByHash(lastBlock.headerHash) + if err == nil { + ms.state.StartBlockHeader = header + ms.updateStorageState() + } + } + deleted := ms.msgs.Delete(lastIndex) for deleted { lastIndex-- @@ -227,6 +333,7 @@ func (ms *MsgStorage) PruneMessages(lastIndex uint64) { func (ms *MsgStorage) Stop() { log.Info("stopping MsgStorage") + ms.unsubscribeTracker() ms.cancel() log.Info("MsgStorage stopped") } diff --git a/rollup/l1/reader.go b/rollup/l1/reader.go index 2523049c0e3c..c3b73f6de947 100644 --- a/rollup/l1/reader.go +++ b/rollup/l1/reader.go @@ -21,7 +21,7 @@ const ( finalizeBatchEventName = "FinalizeBatch" defaultL1MsgFetchBlockRange = 500 - defaultRollupEventsFetchBlockRange = 100 + defaultRollupEventsFetchBlockRange = 500 ) type Reader struct { @@ -90,6 +90,11 @@ func (r *Reader) FetchBlockHeaderByNumber(blockNumber uint64) (*types.Header, er return r.client.HeaderByNumber(r.ctx, big.NewInt(int64(blockNumber))) } +// FetchBlockHeaderByNumber fetches the block header by number +func (r *Reader) FetchBlockHeaderByHash(hash common.Hash) (*types.Header, error) { + return r.client.HeaderByHash(r.ctx, hash) +} + // FetchTxData fetches tx data corresponding to given event log func (r *Reader) FetchTxData(txHash, blockHash common.Hash) ([]byte, error) { tx, err := r.fetchTx(txHash, blockHash) @@ -184,13 +189,13 @@ func (r *Reader) processLogsToRollupEvents(logs []types.Log) (RollupEvents, erro if err = UnpackLog(r.scrollChainABI, event, commitBatchEventName, vLog); err != nil { return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) } - log.Trace("found new CommitBatch event", "batch index", event.batchIndex.Uint64()) + log.Trace("found new CommitBatch event", "batch index", event.BatchIndex.Uint64()) rollupEvent = &CommitBatchEvent{ - batchIndex: event.batchIndex, - batchHash: event.batchHash, - txHash: vLog.TxHash, - blockHash: vLog.BlockHash, - blockNumber: vLog.BlockNumber, + BatchIndex: event.BatchIndex, + BatchHash: event.BatchHash, + TxHash: vLog.TxHash, + BlockHash: vLog.BlockHash, + BlockNumber: vLog.BlockNumber, } case r.l1RevertBatchEventSignature: @@ -198,7 +203,7 @@ func (r *Reader) processLogsToRollupEvents(logs []types.Log) (RollupEvents, erro if err = UnpackLog(r.scrollChainABI, event, revertBatchEventName, vLog); err != nil { return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) } - log.Trace("found new RevertBatchType event", "batch index", event.batchIndex.Uint64()) + log.Trace("found new RevertBatchType event", "batch index", event.BatchIndex.Uint64()) rollupEvent = event case r.l1FinalizeBatchEventSignature: @@ -206,7 +211,7 @@ func (r *Reader) processLogsToRollupEvents(logs []types.Log) (RollupEvents, erro if err = UnpackLog(r.scrollChainABI, event, finalizeBatchEventName, vLog); err != nil { return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) } - log.Trace("found new FinalizeBatchType event", "batch index", event.batchIndex.Uint64()) + log.Trace("found new FinalizeBatchType event", "batch index", event.BatchIndex.Uint64()) rollupEvent = event default: @@ -218,9 +223,9 @@ func (r *Reader) processLogsToRollupEvents(logs []types.Log) (RollupEvents, erro return rollupEvents, nil } -func (r *Reader) queryInBatches(fromBlock, toBlock uint64, batchSize int, queryFunc func(from, to uint64) error) error { +func (r *Reader) queryInBatches(fromBlock, toBlock uint64, batchSize uint64, queryFunc func(from, to uint64) error) error { for from := fromBlock; from <= toBlock; from += uint64(batchSize) { - to := from + defaultL1MsgFetchBlockRange - 1 + to := from + batchSize - 1 if to > toBlock { to = toBlock } diff --git a/rollup/l1/tracker.go b/rollup/l1/tracker.go index b6b030d7e295..933f1ea52012 100644 --- a/rollup/l1/tracker.go +++ b/rollup/l1/tracker.go @@ -26,8 +26,8 @@ type Tracker struct { canonicalChain *common.ShrinkingMap[uint64, *types.Header] headers *common.ShrinkingMap[common.Hash, *types.Header] - lastSafeHeader *types.Header - lastFinalizedHeader *types.Header + // lastSafeHeader *types.Header + // lastFinalizedHeader *types.Header subscriptionCounter int // used to assign unique IDs to subscriptions subscriptions map[ConfirmationRule][]*subscription // sorted by confirmationRule ascending @@ -36,7 +36,7 @@ type Tracker struct { const ( // defaultSyncInterval is the frequency at which we query for new chain head - defaultSyncInterval = 12 * time.Second + defaultSyncInterval = 3 * time.Second defaultPruneInterval = 60 * time.Second ) @@ -145,6 +145,9 @@ func (t *Tracker) syncLatestHead() error { current := newHeader for { prevNumber := current.Number.Uint64() - 1 + if prevNumber+uint64(maxConfirmationRule) < newHeader.Number.Uint64() { // don't need to store headers older then last finalized + break + } prevHeader, exists := t.canonicalChain.Get(prevNumber) if prevNumber == 0 { @@ -222,12 +225,39 @@ func (t *Tracker) notifyLatest(newHeader *types.Header, reorged *reorgedHeaders) t.mu.RLock() defer t.mu.RUnlock() + var ( + latestFinalizedHeader *types.Header + exists bool + latestFinalizedNumber uint64 + ) + if newHeader.Number.Uint64() >= uint64(maxConfirmationRule) { + latestFinalizedNumber = newHeader.Number.Uint64() - uint64(maxConfirmationRule) + latestFinalizedHeader, exists = t.canonicalChain.Get(latestFinalizedNumber) + } + for _, sub := range t.subscriptions[LatestChainHead] { // Ignore subscriptions with deeper ConfirmationRule than the new block. if newHeader.Number.Uint64() < uint64(sub.confirmationRule) { continue } + // if subscription is offline, just return latest finalized block and skip + if !sub.isOnline { + if exists { + sub.isOnline = sub.callback(false, nil, []*types.Header{latestFinalizedHeader}) + sub.lastSentHeader = latestFinalizedHeader + continue + } else { // if not exists then probably chain just started and can convert subscriber into online + sub.isOnline = true + } + } + + if sub.lastSentHeader != nil && sub.lastSentHeader.Number.Uint64()+1 < latestFinalizedNumber { + sub.isOnline = sub.callback(false, nil, []*types.Header{latestFinalizedHeader}) + sub.lastSentHeader = latestFinalizedHeader + continue + } + // 1 confirmation == latest block // 2 confirmations == latest block - 1 // ... @@ -273,14 +303,14 @@ func (t *Tracker) notifyLatest(newHeader *types.Header, reorged *reorgedHeaders) // TODO: we should store both the reorged and new chain in a structure such as reorgedHeaders // maybe repurpose to headerChain or something similar -> have this for reorged and new chain - newChainMin, exists := t.canonicalChain.Get(minReorgedHeader.Number.Uint64() - 1) // new chain min -1 because t.chain() excludes start header + newChainMin, exists := t.canonicalChain.Get(minReorgedHeader.Number.Uint64()) if !exists { return errors.Errorf("failed to find header %d in canonical chain", minReorgedHeader.Number.Uint64()) } - newChain := t.chain(newChainMin, headerToNotify, false) - sub.callback(oldChain, newChain) + newChain := t.chain(newChainMin, headerToNotify, true) + sub.isOnline = sub.callback(true, oldChain, newChain) } else { - sub.callback(nil, t.chain(sub.lastSentHeader, headerToNotify, false)) + sub.isOnline = sub.callback(true, nil, t.chain(sub.lastSentHeader, headerToNotify, false)) } sub.lastSentHeader = headerToNotify } @@ -288,86 +318,86 @@ func (t *Tracker) notifyLatest(newHeader *types.Header, reorged *reorgedHeaders) return nil } -func (t *Tracker) syncSafeHead() error { - newHeader, err := t.headerByNumber(rpc.SafeBlockNumber) - if err != nil { - return errors.Wrapf(err, "failed to retrieve safe header") - } - - if t.lastSafeHeader != nil { - // We already saw this header, nothing to do. - if t.lastSafeHeader.Hash() == newHeader.Hash() { - return nil - } - - // This means there was a L1 reorg and the safe block changed. While this is possible, it should be very rare. - if t.lastSafeHeader.Number.Uint64() >= newHeader.Number.Uint64() { - t.notifySafeHead(newHeader, true) - return nil - } - } - - // Notify all subscribers to new SafeChainHead. - t.notifySafeHead(newHeader, false) - - return nil -} - -func (t *Tracker) notifySafeHead(newHeader *types.Header, reorg bool) { - t.lastSafeHeader = newHeader - - t.mu.RLock() - defer t.mu.RUnlock() - - for _, sub := range t.subscriptions[SafeChainHead] { - // TODO: implement handling of old chain -> this is concurrent to the canonical chain, so we might need to handle this differently - // but: think about the use cases of the safe block: usually it's just about marking the safe head, so there's no need for the old chain. - // this could mean that we should have a different type of callback for safe and finalized head. - sub.callback(nil, t.chain(sub.lastSentHeader, newHeader, false)) - sub.lastSentHeader = newHeader - } -} - -func (t *Tracker) syncFinalizedHead() error { - newHeader, err := t.headerByNumber(rpc.FinalizedBlockNumber) - if err != nil { - return errors.Wrapf(err, "failed to retrieve safe header") - } - - if t.lastFinalizedHeader != nil { - // We already saw this header, nothing to do. - if t.lastFinalizedHeader.Hash() == newHeader.Hash() { - return nil - } - - // This means the finalized block changed as read from L1. The Ethereum protocol guarantees that this can never - // happen. Must be some issue with the RPC node. - if t.lastFinalizedHeader.Number.Uint64() >= newHeader.Number.Uint64() { - return errors.Errorf("RPC node faulty: finalized block number decreased from %d to %d", t.lastFinalizedHeader.Number.Uint64(), newHeader.Number.Uint64()) - } - } - - t.notifyFinalizedHead(newHeader) - - // TODO: prune old headers from headers cache and canonical chain - - return nil -} - -func (t *Tracker) notifyFinalizedHead(newHeader *types.Header) { - t.lastFinalizedHeader = newHeader - - t.mu.RLock() - defer t.mu.RUnlock() - - // Notify all subscribers to new FinalizedChainHead. - for _, sub := range t.subscriptions[FinalizedChainHead] { - newChain := t.chain(sub.lastSentHeader, newHeader, false) - - sub.callback(nil, newChain) - sub.lastSentHeader = newHeader - } -} +// func (t *Tracker) syncSafeHead() error { +// newHeader, err := t.headerByNumber(rpc.SafeBlockNumber) +// if err != nil { +// return errors.Wrapf(err, "failed to retrieve safe header") +// } + +// if t.lastSafeHeader != nil { +// // We already saw this header, nothing to do. +// if t.lastSafeHeader.Hash() == newHeader.Hash() { +// return nil +// } + +// // This means there was a L1 reorg and the safe block changed. While this is possible, it should be very rare. +// if t.lastSafeHeader.Number.Uint64() >= newHeader.Number.Uint64() { +// t.notifySafeHead(newHeader, true) +// return nil +// } +// } + +// // Notify all subscribers to new SafeChainHead. +// t.notifySafeHead(newHeader, false) + +// return nil +// } + +// func (t *Tracker) notifySafeHead(newHeader *types.Header, reorg bool) { +// t.lastSafeHeader = newHeader + +// t.mu.RLock() +// defer t.mu.RUnlock() + +// for _, sub := range t.subscriptions[SafeChainHead] { +// // TODO: implement handling of old chain -> this is concurrent to the canonical chain, so we might need to handle this differently +// // but: think about the use cases of the safe block: usually it's just about marking the safe head, so there's no need for the old chain. +// // this could mean that we should have a different type of callback for safe and finalized head. +// sub.callback(nil, t.chain(sub.lastSentHeader, newHeader, false)) +// sub.lastSentHeader = newHeader +// } +// } + +// func (t *Tracker) syncFinalizedHead() error { +// newHeader, err := t.headerByNumber(rpc.FinalizedBlockNumber) +// if err != nil { +// return errors.Wrapf(err, "failed to retrieve safe header") +// } + +// if t.lastFinalizedHeader != nil { +// // We already saw this header, nothing to do. +// if t.lastFinalizedHeader.Hash() == newHeader.Hash() { +// return nil +// } + +// // This means the finalized block changed as read from L1. The Ethereum protocol guarantees that this can never +// // happen. Must be some issue with the RPC node. +// if t.lastFinalizedHeader.Number.Uint64() >= newHeader.Number.Uint64() { +// return errors.Errorf("RPC node faulty: finalized block number decreased from %d to %d", t.lastFinalizedHeader.Number.Uint64(), newHeader.Number.Uint64()) +// } +// } + +// t.notifyFinalizedHead(newHeader) + +// // TODO: prune old headers from headers cache and canonical chain + +// return nil +// } + +// func (t *Tracker) notifyFinalizedHead(newHeader *types.Header) { +// t.lastFinalizedHeader = newHeader + +// t.mu.RLock() +// defer t.mu.RUnlock() + +// // Notify all subscribers to new FinalizedChainHead. +// for _, sub := range t.subscriptions[FinalizedChainHead] { +// newChain := t.chain(sub.lastSentHeader, newHeader, false) + +// sub.callback(nil, newChain) +// sub.lastSentHeader = newHeader +// } +// } // generates the chain limited by start and end headers. Star may be included or not depending on includeStart func (t *Tracker) chain(start, end *types.Header, includeStart bool) []*types.Header { @@ -408,7 +438,7 @@ func (t *Tracker) chain(start, end *types.Header, includeStart bool) []*types.He return chain } -func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback SubscriptionCallback, maxHeadersSent int) (unsubscribe func()) { +func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback SubscriptionCallback) (unsubscribe func()) { t.mu.Lock() defer t.mu.Unlock() @@ -425,7 +455,7 @@ func (t *Tracker) Subscribe(confirmationRule ConfirmationRule, callback Subscrip panic(fmt.Sprintf("invalid confirmation rule %d", confirmationRule)) } - sub := newSubscription(t.subscriptionCounter, confirmationRule, callback, maxHeadersSent) + sub := newSubscription(t.subscriptionCounter, confirmationRule, callback) subscriptionsByType := t.subscriptions[confirmationType] subscriptionsByType = append(subscriptionsByType, sub) @@ -469,7 +499,7 @@ func (t *Tracker) pruneOldHeaders() { var minNumber *big.Int for _, confRule := range []ConfirmationRule{LatestChainHead, SafeChainHead, FinalizedChainHead} { for _, sub := range t.subscriptions[confRule] { - if sub.lastSentHeader == nil { // did not sent anything to this subscriber, so it's impossible to determine no, which headers could be pruned + if sub.isOnline && sub.lastSentHeader == nil { // did not sent anything to this subscriber, so it's impossible to determine now, which headers could be pruned return } if minNumber == nil { diff --git a/rollup/l1/tracker_test.go b/rollup/l1/tracker_test.go index 11e79bc2b1b2..e935f8742162 100644 --- a/rollup/l1/tracker_test.go +++ b/rollup/l1/tracker_test.go @@ -152,8 +152,9 @@ func (m *mockETHClient) BlockByHash(ctx context.Context, hash common.Hash) (*typ } type subscriptionCallTrace struct { - old []*types.Header - new []*types.Header + isOnline bool + old []*types.Header + new []*types.Header } type subscriptionCalls struct { @@ -169,19 +170,20 @@ func newSubscriptionCalls(tracker *Tracker, alias string, rule ConfirmationRule) expected: []subscriptionCallTrace{}, } - unsubscribe := tracker.Subscribe(rule, func(old, new []*types.Header) { - s.addActual(old, new) - }, 0) + unsubscribe := tracker.Subscribe(rule, func(isOnline bool, old, new []*types.Header) bool { + s.addActual(isOnline, old, new) + return true + },) return s, unsubscribe } -func (s *subscriptionCalls) addActual(last, new []*types.Header) { - s.actual = append(s.actual, subscriptionCallTrace{last, new}) +func (s *subscriptionCalls) addActual(isOnline bool, last, new []*types.Header) { + s.actual = append(s.actual, subscriptionCallTrace{isOnline, last, new}) } -func (s *subscriptionCalls) addExpected(last, new []*types.Header) { - s.expected = append(s.expected, subscriptionCallTrace{last, new}) +func (s *subscriptionCalls) addExpected(isOnline bool, last, new []*types.Header) { + s.expected = append(s.expected, subscriptionCallTrace{isOnline, last, new}) } func (s *subscriptionCalls) requireExpectedCalls(t *testing.T) { @@ -243,8 +245,8 @@ func TestTracker_HappyCases(t *testing.T) { { require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(1, 1)) - sub5.addExpected(nil, client.Headers(1, 1)) + sub1.addExpected(true, nil, client.Headers(1, 1)) + sub5.addExpected(true, nil, client.Headers(1, 1)) subs.requireAll(t) } @@ -254,8 +256,8 @@ func TestTracker_HappyCases(t *testing.T) { client.setLatestBlock(2) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(2, 2)) - sub5.addExpected(nil, client.Headers(2, 2)) + sub1.addExpected(true, nil, client.Headers(2, 2)) + sub5.addExpected(true, nil, client.Headers(2, 2)) subs.requireAll(t) } @@ -268,8 +270,8 @@ func TestTracker_HappyCases(t *testing.T) { client.setLatestBlock(3) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(3, 3)) - sub2.addExpected(nil, client.Headers(1, 1)) + sub1.addExpected(true, nil, client.Headers(3, 3)) + sub2.addExpected(true, nil, client.Headers(1, 1)) subs.requireAll(t) } @@ -289,11 +291,21 @@ func TestTracker_HappyCases(t *testing.T) { // Block 70 - we skip a bunch of blocks { + // range of skipped blocks is too much so tracker marks subscribers as offline and returns latest finalized block client.setLatestBlock(70) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(4, 70)) - sub2.addExpected(nil, client.Headers(2, 68)) + sub1.addExpected(false, nil, client.Headers(6, 6)) + sub2.addExpected(false, nil, client.Headers(6, 6)) + + subs.requireAll(t) + + // after subscribers caught up tracker continues to return ranges of headers + client.setLatestBlock(71) + require.NoError(t, tracker.syncLatestHead()) + + sub1.addExpected(true, nil, client.Headers(7, 71)) + sub2.addExpected(true, nil, client.Headers(7, 69)) subs.requireAll(t) } @@ -321,13 +333,13 @@ func TestTracker_HappyCases(t *testing.T) { // subs.requireAll(t) //} // - // Block 72 - we skip again 1 block + // Block 73 - we skip again 1 block { - client.setLatestBlock(72) + client.setLatestBlock(73) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(71, 72)) - sub2.addExpected(nil, client.Headers(69, 70)) + sub1.addExpected(true, nil, client.Headers(72, 73)) + sub2.addExpected(true, nil, client.Headers(70, 71)) subs.requireAll(t) } @@ -478,83 +490,83 @@ func TestTracker_LatestChainHead_Reorg(t *testing.T) { { require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(1, 1)) + sub1.addExpected(true, nil, client.Headers(1, 1)) subs.requireAll(t) } - // Block 90 - we skip a bunch of blocks + // Block 50 - we skip a bunch of blocks { - client.setLatestBlock(90) + client.setLatestBlock(50) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(2, 90)) - sub2.addExpected(nil, client.Headers(1, 88)) - sub3.addExpected(nil, client.Headers(1, 88)) - sub4.addExpected(nil, client.Headers(1, 86)) + sub1.addExpected(true, nil, client.Headers(2, 50)) + sub2.addExpected(true, nil, client.Headers(1, 48)) + sub3.addExpected(true, nil, client.Headers(1, 48)) + sub4.addExpected(true, nil, client.Headers(1, 46)) subs.requireAll(t) } - // Block 90 - reorg of depth 1 - only sub1 affected - beforeReorg90 := client.Headers(90, 90) + // Block 50 - reorg of depth 1 - only sub1 affected + beforeReorg50 := client.Headers(50, 50) //beforeReorg88 := client.Header(88) //beforeReorg86 := client.Header(86) { - client.createFork(89) - client.setLatestBlock(90) + client.createFork(49) + client.setLatestBlock(50) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(beforeReorg90, client.Headers(90, 90)) + sub1.addExpected(true, beforeReorg50, client.Headers(50, 50)) subs.requireAll(t) } - //// Block 98 - gap - since subs 2-4 were not affected by the reorg they should not be notified about the reorg (form their PoV it's just a gap) + //// Block 58 - gap - since subs 2-4 were not affected by the reorg they should not be notified about the reorg (form their PoV it's just a gap) { - client.setLatestBlock(98) + client.setLatestBlock(58) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(nil, client.Headers(91, 98)) - sub2.addExpected(nil, client.Headers(89, 96)) - sub3.addExpected(nil, client.Headers(89, 96)) - sub4.addExpected(nil, client.Headers(87, 94)) + sub1.addExpected(true, nil, client.Headers(51, 58)) + sub2.addExpected(true, nil, client.Headers(49, 56)) + sub3.addExpected(true, nil, client.Headers(49, 56)) + sub4.addExpected(true, nil, client.Headers(47, 54)) subs.requireAll(t) } // reorg of depth 1 + new block - beforeReorg98 := client.Headers(98, 98) + beforeReorg58 := client.Headers(58, 58) { - client.createFork(97) - client.setLatestBlock(99) + client.createFork(57) + client.setLatestBlock(59) require.NoError(t, tracker.syncLatestHead()) - sub1.addExpected(beforeReorg98, client.Headers(98, 99)) - sub2.addExpected(nil, client.Headers(97, 97)) - sub3.addExpected(nil, client.Headers(97, 97)) - sub4.addExpected(nil, client.Headers(95, 95)) + sub1.addExpected(true, beforeReorg58, client.Headers(58, 59)) + sub2.addExpected(true, nil, client.Headers(57, 57)) + sub3.addExpected(true, nil, client.Headers(57, 57)) + sub4.addExpected(true, nil, client.Headers(55, 55)) subs.requireAll(t) } - // Block 99 - reorg of depth 4, subs 1-3 affected + // Block 59 - reorg of depth 4, subs 1-3 affected // TODO: we need to make sure that we notify the subscribers correctly about the reorged headers - beforeReorg99 := client.Headers(96, 99) - beforeReorg97 := client.Headers(96, 97) + beforeReorg59 := client.Headers(56, 59) + beforeReorg57 := client.Headers(56, 57) { - client.createFork(95) - client.setLatestBlock(99) + client.createFork(55) + client.setLatestBlock(59) require.NoError(t, tracker.syncLatestHead()) - fmt.Println("len", len(beforeReorg99)) + fmt.Println("len", len(beforeReorg59)) - sub1.addExpected(beforeReorg99, client.Headers(96, 99)) - sub2.addExpected(beforeReorg97, client.Headers(96, 97)) - sub3.addExpected(beforeReorg97, client.Headers(96, 97)) + sub1.addExpected(true, beforeReorg59, client.Headers(56, 59)) + sub2.addExpected(true, beforeReorg57, client.Headers(56, 57)) + sub3.addExpected(true, beforeReorg57, client.Headers(56, 57)) subs.requireAll(t) } diff --git a/rollup/l1/types.go b/rollup/l1/types.go index 998078eb3d6a..086a495a4198 100644 --- a/rollup/l1/types.go +++ b/rollup/l1/types.go @@ -47,15 +47,15 @@ type subscription struct { confirmationRule ConfirmationRule callback SubscriptionCallback lastSentHeader *types.Header - maxHeadersSent int // number of headers that could be sent at the time + isOnline bool // isOnline marks whether subscriber now wants to receive most recent data or still sync to the top, false by default and may be changed by callback later } -func newSubscription(id int, confirmationRule ConfirmationRule, callback SubscriptionCallback, maxHeadersSent int) *subscription { +func newSubscription(id int, confirmationRule ConfirmationRule, callback SubscriptionCallback) *subscription { return &subscription{ id: id, confirmationRule: confirmationRule, callback: callback, - maxHeadersSent: maxHeadersSent, + isOnline: false, } } @@ -71,4 +71,7 @@ const ( LatestChainHead = ConfirmationRule(1) ) -type SubscriptionCallback func(old, new []*types.Header) +// SubscriptionCallback returns true if subscriber is synced to the top of chain now +// and ready to receive most recent data from chain +// otherwise, if subscriber still syncs from the some old block to the most recent, it returns false +type SubscriptionCallback func(isOnline bool, old, new []*types.Header) bool