From 4fe6c3a1b9f6e717109646d62fe4bdfd935af272 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 5 Feb 2024 20:12:25 +0100 Subject: [PATCH 01/59] implement l1rpcfetcher and prepare for implementing da_syncer --- rollup/da_syncer/abi.go | 55 +++++++ rollup/da_syncer/block_context.go | 36 +++++ rollup/da_syncer/chunk.go | 62 ++++++++ rollup/da_syncer/da.go | 48 ++++++ rollup/da_syncer/da_fetcher.go | 6 + rollup/da_syncer/da_syncer.go | 101 +++++++++++++ rollup/da_syncer/l1_client.go | 92 ++++++++++++ rollup/da_syncer/l1_tx.go | 6 + rollup/da_syncer/l1rpc_fetcher.go | 217 +++++++++++++++++++++++++++ rollup/da_syncer/snapshot_fetcher.go | 16 ++ 10 files changed, 639 insertions(+) create mode 100644 rollup/da_syncer/abi.go create mode 100644 rollup/da_syncer/block_context.go create mode 100644 rollup/da_syncer/chunk.go create mode 100644 rollup/da_syncer/da.go create mode 100644 rollup/da_syncer/da_fetcher.go create mode 100644 rollup/da_syncer/da_syncer.go create mode 100644 rollup/da_syncer/l1_client.go create mode 100644 rollup/da_syncer/l1_tx.go create mode 100644 rollup/da_syncer/l1rpc_fetcher.go create mode 100644 rollup/da_syncer/snapshot_fetcher.go diff --git a/rollup/da_syncer/abi.go b/rollup/da_syncer/abi.go new file mode 100644 index 000000000000..21603628a658 --- /dev/null +++ b/rollup/da_syncer/abi.go @@ -0,0 +1,55 @@ +package da_syncer + +import ( + "fmt" + "math/big" + + "github.com/scroll-tech/go-ethereum/accounts/abi" + "github.com/scroll-tech/go-ethereum/accounts/abi/bind" + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" +) + +// scrollChainMetaData contains ABI of the ScrollChain contract. +var scrollChainMetaData = &bind.MetaData{ + ABI: "[{\"inputs\":[{\"internalType\":\"uint64\",\"name\":\"_chainId\",\"type\":\"uint64\"}],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"}],\"name\":\"CommitBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"},{\"indexed\":false,\"internalType\":\"bytes32\",\"name\":\"stateRoot\",\"type\":\"bytes32\"},{\"indexed\":false,\"internalType\":\"bytes32\",\"name\":\"withdrawRoot\",\"type\":\"bytes32\"}],\"name\":\"FinalizeBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint8\",\"name\":\"version\",\"type\":\"uint8\"}],\"name\":\"Initialized\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"previousOwner\",\"type\":\"address\"},{\"indexed\":true,\"internalType\":\"address\",\"name\":\"newOwner\",\"type\":\"address\"}],\"name\":\"OwnershipTransferred\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"}],\"name\":\"Paused\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"}],\"name\":\"RevertBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"}],\"name\":\"Unpaused\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint256\",\"name\":\"oldMaxNumTxInChunk\",\"type\":\"uint256\"},{\"indexed\":false,\"internalType\":\"uint256\",\"name\":\"newMaxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"UpdateMaxNumTxInChunk\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"},{\"indexed\":false,\"internalType\":\"bool\",\"name\":\"status\",\"type\":\"bool\"}],\"name\":\"UpdateProver\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"},{\"indexed\":false,\"internalType\":\"bool\",\"name\":\"status\",\"type\":\"bool\"}],\"name\":\"UpdateSequencer\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"oldVerifier\",\"type\":\"address\"},{\"indexed\":true,\"internalType\":\"address\",\"name\":\"newVerifier\",\"type\":\"address\"}],\"name\":\"UpdateVerifier\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"addProver\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"addSequencer\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint8\",\"name\":\"_version\",\"type\":\"uint8\"},{\"internalType\":\"bytes\",\"name\":\"_parentBatchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes[]\",\"name\":\"_chunks\",\"type\":\"bytes[]\"},{\"internalType\":\"bytes\",\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\"}],\"name\":\"commitBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"committedBatches\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes32\",\"name\":\"_prevStateRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes32\",\"name\":\"_postStateRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes32\",\"name\":\"_withdrawRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes\",\"name\":\"_aggrProof\",\"type\":\"bytes\"}],\"name\":\"finalizeBatchWithProof\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"finalizedStateRoots\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes32\",\"name\":\"_stateRoot\",\"type\":\"bytes32\"}],\"name\":\"importGenesisBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_messageQueue\",\"type\":\"address\"},{\"internalType\":\"address\",\"name\":\"_verifier\",\"type\":\"address\"},{\"internalType\":\"uint256\",\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"initialize\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"_batchIndex\",\"type\":\"uint256\"}],\"name\":\"isBatchFinalized\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"name\":\"isProver\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"name\":\"isSequencer\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"lastFinalizedBatchIndex\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"layer2ChainId\",\"outputs\":[{\"internalType\":\"uint64\",\"name\":\"\",\"type\":\"uint64\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"maxNumTxInChunk\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"messageQueue\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"owner\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"paused\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"removeProver\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"removeSequencer\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"renounceOwnership\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"uint256\",\"name\":\"_count\",\"type\":\"uint256\"}],\"name\":\"revertBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bool\",\"name\":\"_status\",\"type\":\"bool\"}],\"name\":\"setPause\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"newOwner\",\"type\":\"address\"}],\"name\":\"transferOwnership\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"updateMaxNumTxInChunk\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_newVerifier\",\"type\":\"address\"}],\"name\":\"updateVerifier\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"verifier\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"withdrawRoots\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"}]", +} + +// L1CommitBatchEvent represents a CommitBatch event raised by the ScrollChain contract. +type L1CommitBatchEvent struct { + BatchIndex *big.Int + BatchHash common.Hash +} + +// L1RevertBatchEvent represents a RevertBatch event raised by the ScrollChain contract. +type L1RevertBatchEvent struct { + BatchIndex *big.Int + BatchHash common.Hash +} + +// L1FinalizeBatchEvent represents a FinalizeBatch event raised by the ScrollChain contract. +type L1FinalizeBatchEvent struct { + BatchIndex *big.Int + BatchHash common.Hash + StateRoot common.Hash + WithdrawRoot common.Hash +} + +// UnpackLog unpacks a retrieved log into the provided output structure. +func UnpackLog(c *abi.ABI, out interface{}, event string, log types.Log) error { + if log.Topics[0] != c.Events[event].ID { + return fmt.Errorf("event signature mismatch") + } + if len(log.Data) > 0 { + if err := c.UnpackIntoInterface(out, event, log.Data); err != nil { + return err + } + } + var indexed abi.Arguments + for _, arg := range c.Events[event].Inputs { + if arg.Indexed { + indexed = append(indexed, arg) + } + } + return abi.ParseTopics(out, indexed, log.Topics[1:]) +} diff --git a/rollup/da_syncer/block_context.go b/rollup/da_syncer/block_context.go new file mode 100644 index 000000000000..40cdd0ac0255 --- /dev/null +++ b/rollup/da_syncer/block_context.go @@ -0,0 +1,36 @@ +package da_syncer + +import ( + "encoding/binary" + "errors" + "math/big" +) + +// BlockContext represents the essential data of a block in the ScrollChain. +// It provides an overview of block attributes including hash values, block numbers, gas details, and transaction counts. +type BlockContext struct { + BlockNumber uint64 + Timestamp uint64 + BaseFee *big.Int + GasLimit uint64 + NumTransactions uint16 + NumL1Messages uint16 +} + +type BlockContexts []*BlockContext + +func decodeBlockContext(encodedBlockContext []byte) (*BlockContext, error) { + if len(encodedBlockContext) != blockContextByteSize { + return nil, errors.New("block encoding is not 60 bytes long") + } + baseFee := big.NewInt(0).SetBytes(encodedBlockContext[16:48]) + + return &BlockContext{ + BlockNumber: binary.BigEndian.Uint64(encodedBlockContext[0:8]), + Timestamp: binary.BigEndian.Uint64(encodedBlockContext[8:16]), + BaseFee: baseFee, + GasLimit: binary.BigEndian.Uint64(encodedBlockContext[48:56]), + NumTransactions: binary.BigEndian.Uint16(encodedBlockContext[56:58]), + NumL1Messages: binary.BigEndian.Uint16(encodedBlockContext[58:60]), + }, nil +} diff --git a/rollup/da_syncer/chunk.go b/rollup/da_syncer/chunk.go new file mode 100644 index 000000000000..c493330173e3 --- /dev/null +++ b/rollup/da_syncer/chunk.go @@ -0,0 +1,62 @@ +package da_syncer + +import ( + "encoding/binary" + "fmt" +) + +const blockContextByteSize = 60 + +type Chunk struct { + BlockContexts BlockContexts + L2Txs [][]byte +} + +type Chunks []*Chunk + +// DecodeChunks decodes the provided chunks into a list of chunks. +func DecodeChunks(chunksData [][]byte) (Chunks, error) { + var chunks Chunks + for _, chunk := range chunksData { + if len(chunk) < 1 { + return nil, fmt.Errorf("invalid chunk, length is less than 1") + } + + numBlocks := int(chunk[0]) + if len(chunk) < 1+numBlocks*blockContextByteSize { + return nil, fmt.Errorf("chunk size doesn't match with numBlocks, byte length of chunk: %v, expected length: %v", len(chunk), 1+numBlocks*blockContextByteSize) + } + + blockContexts := make(BlockContexts, numBlocks) + for i := 0; i < numBlocks; i++ { + startIdx := 1 + i*blockContextByteSize // add 1 to skip numBlocks byte + endIdx := startIdx + blockContextByteSize + blockContext, err := decodeBlockContext(chunk[startIdx:endIdx]) + if err != nil { + return nil, err + } + blockContexts[i] = blockContext + } + + var l2Txs [][]byte + var txLen int = 0 + + for currentIndex := 1 + numBlocks*blockContextByteSize; currentIndex < len(chunk); currentIndex += 4 + txLen { + if len(chunk) < currentIndex+4 { + return nil, fmt.Errorf("chunk size doesn't match, next tx size is less then 4, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4) + } + txLen = int(binary.BigEndian.Uint32(chunk[currentIndex : currentIndex+4])) + if len(chunk) < currentIndex+4+txLen { + return nil, fmt.Errorf("chunk size doesn't match with next tx length, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4+txLen) + } + txData := chunk[currentIndex+4 : currentIndex+4+txLen] + l2Txs = append(l2Txs, txData) + } + + chunks = append(chunks, &Chunk{ + BlockContexts: blockContexts, + L2Txs: l2Txs, + }) + } + return chunks, nil +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go new file mode 100644 index 000000000000..a90fae0bd7c9 --- /dev/null +++ b/rollup/da_syncer/da.go @@ -0,0 +1,48 @@ +package da_syncer + +type DAType int + +const ( + // CommitBatch contains data of event of CommitBatch + CommitBatch DAType = iota + // RevertBatch contains data of event of RevertBatch + RevertBatch + // FinalizeBatch contains data of event of FinalizeBatch + FinalizeBatch +) + +type DAEntry struct { + // DaType is a type of DA entry (CommitBatch, RevertBatch, FinalizeBatch) + DaType DAType + // BatchIndex index of batch + BatchIndex uint64 + // Chunks contains chunk of a batch + Chunks Chunks + // L1Txs contains l1txs of a batch + L1Txs L1Txs +} + +type DA []*DAEntry + +func NewCommitBatchDA(batchIndex uint64, chunks Chunks, l1txs L1Txs) *DAEntry { + return &DAEntry{ + DaType: CommitBatch, + BatchIndex: batchIndex, + Chunks: chunks, + L1Txs: l1txs, + } +} + +func NewRevertBatchDA(batchIndex uint64) *DAEntry { + return &DAEntry{ + DaType: RevertBatch, + BatchIndex: batchIndex, + } +} + +func NewFinalizeBatchDA(batchIndex uint64) *DAEntry { + return &DAEntry{ + DaType: FinalizeBatch, + BatchIndex: batchIndex, + } +} diff --git a/rollup/da_syncer/da_fetcher.go b/rollup/da_syncer/da_fetcher.go new file mode 100644 index 000000000000..86c49f8014e3 --- /dev/null +++ b/rollup/da_syncer/da_fetcher.go @@ -0,0 +1,6 @@ +package da_syncer + +// DaFetcher encapsulates functions required to fetch data from l1 +type DaFetcher interface { + FetchDA() (DA, error) +} diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go new file mode 100644 index 000000000000..5164d3b8d66b --- /dev/null +++ b/rollup/da_syncer/da_syncer.go @@ -0,0 +1,101 @@ +package da_syncer + +import ( + "context" + "time" + + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/log" +) + +// defaultSyncInterval is the frequency at which we query for new rollup event. +const defaultSyncInterval = 60 * time.Second + +type DaSyncer struct { + DaFetcher DaFetcher + ctx context.Context + cancel context.CancelFunc + // batches is map from batchIndex to batch blocks + batches map[uint64][]*types.Block +} + +func NewDaSyncer(ctx context.Context, daFetcher DaFetcher) (*DaSyncer, error) { + ctx, cancel := context.WithCancel(ctx) + daSyncer := DaSyncer{ + DaFetcher: daFetcher, + ctx: ctx, + cancel: cancel, + batches: make(map[uint64][]*types.Block), + } + return &daSyncer, nil +} + +func (s *DaSyncer) Start() { + if s == nil { + return + } + + log.Info("Starting DaSyncer") + + go func() { + syncTicker := time.NewTicker(defaultSyncInterval) + defer syncTicker.Stop() + + for { + select { + case <-s.ctx.Done(): + return + case <-syncTicker.C: + s.syncWithDa() + } + } + }() +} + +func (s *DaSyncer) Stop() { + if s == nil { + return + } + + log.Info("Stopping DaSyncer") + + if s.cancel != nil { + s.cancel() + } +} + +func (s *DaSyncer) syncWithDa() { + da, err := s.DaFetcher.FetchDA() + if err != nil { + log.Error("failed to fetch DA", "err", err) + return + } + for _, daEntry := range da { + switch daEntry.DaType { + case CommitBatch: + blocks, err := s.processDaToBlocks(daEntry) + if err != nil { + log.Warn("failed to process DA to blocks", "err", err) + return + } + s.batches[daEntry.BatchIndex] = blocks + case RevertBatch: + delete(s.batches, daEntry.BatchIndex) + case FinalizeBatch: + blocks, ok := s.batches[daEntry.BatchIndex] + if !ok { + log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) + return + } + s.insertBlocks(blocks) + } + } +} + +func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { + return nil, nil +} + +func (s *DaSyncer) insertBlocks([]*types.Block) { + +} diff --git a/rollup/da_syncer/l1_client.go b/rollup/da_syncer/l1_client.go new file mode 100644 index 000000000000..a7f4616ef46a --- /dev/null +++ b/rollup/da_syncer/l1_client.go @@ -0,0 +1,92 @@ +package da_syncer + +import ( + "context" + "errors" + "fmt" + "math/big" + + "github.com/scroll-tech/go-ethereum" + "github.com/scroll-tech/go-ethereum/accounts/abi" + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rpc" + + "github.com/scroll-tech/go-ethereum/rollup/sync_service" +) + +// L1Client is a wrapper around EthClient that adds +// methods for conveniently collecting rollup events of ScrollChain contract. +type L1Client struct { + ctx context.Context + client sync_service.EthClient + scrollChainAddress common.Address + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash +} + +// newL1Client initializes a new L1Client instance with the provided configuration. +// It checks for a valid scrollChainAddress and verifies the chain ID. +func newL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId uint64, scrollChainAddress common.Address, scrollChainABI *abi.ABI) (*L1Client, error) { + if scrollChainAddress == (common.Address{}) { + return nil, errors.New("must pass non-zero scrollChainAddress to L1Client") + } + + // sanity check: compare chain IDs + got, err := l1Client.ChainID(ctx) + if err != nil { + return nil, fmt.Errorf("failed to query L1 chain ID, err: %w", err) + } + if got.Cmp(big.NewInt(0).SetUint64(l1ChainId)) != 0 { + return nil, fmt.Errorf("unexpected chain ID, expected: %v, got: %v", l1ChainId, got) + } + + client := L1Client{ + ctx: ctx, + client: l1Client, + scrollChainAddress: scrollChainAddress, + l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, + l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + } + + return &client, nil +} + +// fetcRollupEventsInRange retrieves and parses commit/revert/finalize rollup events between block numbers: [from, to]. +func (c *L1Client) fetchRollupEventsInRange(ctx context.Context, from, to uint64) ([]types.Log, error) { + log.Trace("L1Client fetchRollupEventsInRange", "fromBlock", from, "toBlock", to) + + query := ethereum.FilterQuery{ + FromBlock: big.NewInt(int64(from)), // inclusive + ToBlock: big.NewInt(int64(to)), // inclusive + Addresses: []common.Address{ + c.scrollChainAddress, + }, + Topics: make([][]common.Hash, 1), + } + query.Topics[0] = make([]common.Hash, 3) + query.Topics[0][0] = c.l1CommitBatchEventSignature + query.Topics[0][1] = c.l1RevertBatchEventSignature + query.Topics[0][2] = c.l1FinalizeBatchEventSignature + + logs, err := c.client.FilterLogs(c.ctx, query) + if err != nil { + return nil, fmt.Errorf("failed to filter logs, err: %w", err) + } + return logs, nil +} + +// getLatestFinalizedBlockNumber fetches the block number of the latest finalized block from the L1 chain. +func (c *L1Client) getLatestFinalizedBlockNumber(ctx context.Context) (uint64, error) { + header, err := c.client.HeaderByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) + if err != nil { + return 0, err + } + if !header.Number.IsInt64() { + return 0, fmt.Errorf("received unexpected block number in L1Client: %v", header.Number) + } + return header.Number.Uint64(), nil +} diff --git a/rollup/da_syncer/l1_tx.go b/rollup/da_syncer/l1_tx.go new file mode 100644 index 000000000000..80de038df95d --- /dev/null +++ b/rollup/da_syncer/l1_tx.go @@ -0,0 +1,6 @@ +package da_syncer + +type L1Tx struct { +} + +type L1Txs []*L1Tx diff --git a/rollup/da_syncer/l1rpc_fetcher.go b/rollup/da_syncer/l1rpc_fetcher.go new file mode 100644 index 000000000000..97f3c7838a9b --- /dev/null +++ b/rollup/da_syncer/l1rpc_fetcher.go @@ -0,0 +1,217 @@ +package da_syncer + +import ( + "context" + "fmt" + "reflect" + + "github.com/scroll-tech/go-ethereum/accounts/abi" + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rollup/sync_service" +) + +type L1RPCFetcher struct { + fetchBlockRange uint64 + client *L1Client + ctx context.Context + latestProcessedBlock uint64 + scrollChainABI *abi.ABI + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash +} + +func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient, l1DeploymentBlock, fetchBlockRange uint64) (DaFetcher, error) { + // terminate if the caller does not provide an L1 client (e.g. in tests) + if l1Client == nil || (reflect.ValueOf(l1Client).Kind() == reflect.Ptr && reflect.ValueOf(l1Client).IsNil()) { + log.Warn("No L1 client provided, L1 rollup sync service will not run") + return nil, nil + } + + if genesisConfig.Scroll.L1Config == nil { + return nil, fmt.Errorf("missing L1 config in genesis") + } + + scrollChainABI, err := scrollChainMetaData.GetAbi() + if err != nil { + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + + client, err := newL1Client(ctx, l1Client, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) + if err != nil { + return nil, fmt.Errorf("failed to initialize l1 client: %w", err) + } + + // Initialize the latestProcessedBlock with the block just before the L1 deployment block. + // This serves as a default value when there's no L1 rollup events synced in the database. + var latestProcessedBlock uint64 + if l1DeploymentBlock > 0 { + latestProcessedBlock = l1DeploymentBlock - 1 + } + + // todo: read latest processed block from db + + daFetcher := L1RPCFetcher{ + fetchBlockRange: fetchBlockRange, + ctx: ctx, + client: client, + latestProcessedBlock: latestProcessedBlock, + scrollChainABI: scrollChainABI, + l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, + l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + } + return &daFetcher, nil +} + +// Fetch DA fetches all da events and converts it to DA format in some fetchBlockRange +func (f *L1RPCFetcher) FetchDA() (DA, error) { + latestConfirmed, err := f.client.getLatestFinalizedBlockNumber(f.ctx) + if err != nil { + log.Warn("failed to get latest confirmed block number", "err", err) + return nil, err + } + + log.Trace("Da fetcher fetch rollup events", "latest processed block", f.latestProcessedBlock, "latest confirmed", latestConfirmed) + + from := f.latestProcessedBlock + 1 + to := f.latestProcessedBlock + f.fetchBlockRange + if to > latestConfirmed { + to = latestConfirmed + } + + logs, err := f.client.fetchRollupEventsInRange(f.ctx, from, to) + if err != nil { + log.Error("failed to fetch rollup events in range", "from block", from, "to block", to, "err", err) + return nil, err + } + + da, err := f.processLogsToDA(logs) + if err != nil { + log.Error("failed to process rollup events in range", "from block", from, "to block", to, "err", err) + return nil, err + } + + f.latestProcessedBlock = to + return da, nil +} + +func (f *L1RPCFetcher) processLogsToDA(logs []types.Log) (DA, error) { + var da DA + for _, vLog := range logs { + switch vLog.Topics[0] { + case f.l1CommitBatchEventSignature: + event := &L1CommitBatchEvent{} + if err := UnpackLog(f.scrollChainABI, event, "CommitBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new CommitBatch event", "batch index", batchIndex) + + chunks, l1Txs, err := f.getBatch(batchIndex, &vLog) + if err != nil { + return nil, fmt.Errorf("failed to get chunks, batch index: %v, err: %w", batchIndex, err) + } + da = append(da, NewCommitBatchDA(batchIndex, chunks, l1Txs)) + + case f.l1RevertBatchEventSignature: + event := &L1RevertBatchEvent{} + if err := UnpackLog(f.scrollChainABI, event, "RevertBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new RevertBatch event", "batch index", batchIndex) + da = append(da, NewRevertBatchDA(batchIndex)) + + case f.l1FinalizeBatchEventSignature: + event := &L1FinalizeBatchEvent{} + if err := UnpackLog(f.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new FinalizeBatch event", "batch index", batchIndex) + + da = append(da, NewFinalizeBatchDA(batchIndex)) + + default: + return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) + } + } + + // note: the batch updates above are idempotent, if we crash + // before this line and reexecute the previous steps, we will + // get the same result. + // todo: store to db latest process block number + return da, nil +} + +func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, L1Txs, error) { + var chunks Chunks + var l1Txs L1Txs + if batchIndex == 0 { + return chunks, l1Txs, nil + } + + tx, _, err := f.client.client.TransactionByHash(f.ctx, vLog.TxHash) + if err != nil { + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := f.client.client.BlockByHash(f.ctx, vLog.BlockHash) + if err != nil { + return nil, nil, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return nil, nil, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } + } + + txData := tx.Data() + const methodIDLength = 4 + if len(txData) < methodIDLength { + return nil, nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) + } + + method, err := f.scrollChainABI.MethodById(txData[:methodIDLength]) + if err != nil { + return nil, nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) + } + + values, err := method.Inputs.Unpack(txData[methodIDLength:]) + if err != nil { + return nil, nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) + } + + type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte + } + var args commitBatchArgs + err = method.Inputs.Copy(&args, values) + if err != nil { + return nil, nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + + chunks, err = DecodeChunks(args.Chunks) + if err != nil { + return nil, nil, fmt.Errorf("failed to unpack decode chunks in batch number: %v, err: %w", batchIndex, err) + } + + // todo: l1txs can be loaded from db that filled by existing l1 msg sync service + l1Txs = nil + return chunks, l1Txs, nil + +} diff --git a/rollup/da_syncer/snapshot_fetcher.go b/rollup/da_syncer/snapshot_fetcher.go new file mode 100644 index 000000000000..a109086a2c0f --- /dev/null +++ b/rollup/da_syncer/snapshot_fetcher.go @@ -0,0 +1,16 @@ +package da_syncer + +type SnapshotFetcher struct { + fetchBlockRange uint64 +} + +func newSnapshotFetcher(fetchBlockRange uint64) (DaFetcher, error) { + daFetcher := SnapshotFetcher{ + fetchBlockRange: fetchBlockRange, + } + return &daFetcher, nil +} + +func (f *SnapshotFetcher) FetchDA() (DA, error) { + return nil, nil +} From 25bc860994433425a6c9ad16a7f3319142ba1ce8 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 13 Feb 2024 10:06:21 +0100 Subject: [PATCH 02/59] finish da_syncer, add db access, small fixes --- cmd/geth/main.go | 3 ++ cmd/utils/flags.go | 30 +++++++++++++ core/rawdb/accessors_da_syncer.go | 39 +++++++++++++++++ core/rawdb/schema.go | 3 ++ eth/backend.go | 13 ++++++ eth/ethconfig/config.go | 10 +++++ rollup/da_syncer/chunk.go | 16 +++++-- rollup/da_syncer/da.go | 6 ++- rollup/da_syncer/da_syncer.go | 65 ++++++++++++++++++++++++++- rollup/da_syncer/l1_tx.go | 6 --- rollup/da_syncer/l1rpc_fetcher.go | 73 +++++++++++++++++++++++++------ rollup/da_syncer/modes.go | 52 ++++++++++++++++++++++ 12 files changed, 289 insertions(+), 27 deletions(-) create mode 100644 core/rawdb/accessors_da_syncer.go delete mode 100644 rollup/da_syncer/l1_tx.go create mode 100644 rollup/da_syncer/modes.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index d4280144e49b..dea4b7fbec7e 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -166,6 +166,9 @@ var ( utils.L1DeploymentBlockFlag, utils.CircuitCapacityCheckEnabledFlag, utils.RollupVerifyEnabledFlag, + utils.DaSyncEnabledFlag, + utils.DAModeFlag, + utils.DASnapshotFileFlag, } rpcFlags = []cli.Flag{ diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 24f64f1c3df2..79f79fe353a8 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -73,6 +73,7 @@ import ( "github.com/scroll-tech/go-ethereum/p2p/nat" "github.com/scroll-tech/go-ethereum/p2p/netutil" "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer" "github.com/scroll-tech/go-ethereum/rollup/tracing" "github.com/scroll-tech/go-ethereum/rpc" ) @@ -848,6 +849,22 @@ var ( Name: "rpc.getlogs.maxrange", Usage: "Limit max fetched block range for `eth_getLogs` method", } + + // Da syncing settings + DaSyncEnabledFlag = cli.BoolFlag{ + Name: "da.sync", + Usage: "Enable node syncing from DA", + } + defaultDa = ethconfig.Defaults.DA.FetcherMode + DAModeFlag = TextMarshalerFlag{ + Name: "da.mode", + Usage: `Da sync sync mode ("l1rpc" or "snapshot")`, + Value: &defaultDa, + } + DASnapshotFileFlag = cli.StringFlag{ + Name: "da.snapshot.file", + Usage: "Snapshot file to sync from da", + } ) // MakeDataDir retrieves the currently requested data directory, terminating @@ -1559,6 +1576,18 @@ func setEnableRollupVerify(ctx *cli.Context, cfg *ethconfig.Config) { } } +func setDa(ctx *cli.Context, cfg *ethconfig.Config) { + if ctx.GlobalIsSet(DaSyncEnabledFlag.Name) { + cfg.EnableDASyncing = ctx.GlobalBool(DaSyncEnabledFlag.Name) + if ctx.GlobalIsSet(DAModeFlag.Name) { + cfg.DA.FetcherMode = *GlobalTextMarshaler(ctx, DAModeFlag.Name).(*da_syncer.FetcherMode) + } + if ctx.GlobalIsSet(DASnapshotFileFlag.Name) { + cfg.DA.SnapshotFilePath = ctx.GlobalString(DASnapshotFileFlag.Name) + } + } +} + func setMaxBlockRange(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.GlobalIsSet(MaxBlockRangeFlag.Name) { cfg.MaxBlockRange = ctx.GlobalInt64(MaxBlockRangeFlag.Name) @@ -1634,6 +1663,7 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { setLes(ctx, cfg) setCircuitCapacityCheck(ctx, cfg) setEnableRollupVerify(ctx, cfg) + setDa(ctx, cfg) setMaxBlockRange(ctx, cfg) // Cap the cache allowance and tune the garbage collector diff --git a/core/rawdb/accessors_da_syncer.go b/core/rawdb/accessors_da_syncer.go new file mode 100644 index 000000000000..c4097e59b8e3 --- /dev/null +++ b/core/rawdb/accessors_da_syncer.go @@ -0,0 +1,39 @@ +package rawdb + +import ( + "math/big" + + "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/log" +) + +// WriteDASyncedL1BlockNumber writes the highest synced L1 block number to the database. +func WriteDASyncedL1BlockNumber(db ethdb.KeyValueWriter, L1BlockNumber uint64) { + value := big.NewInt(0).SetUint64(L1BlockNumber).Bytes() + + if err := db.Put(daSyncedL1BlockNumberKey, value); err != nil { + log.Crit("Failed to update da synced L1 block number", "err", err) + } +} + +// ReadDASyncedL1BlockNumber retrieves the highest synced L1 block number. +func ReadDASyncedL1BlockNumber(db ethdb.Reader) *uint64 { + data, err := db.Get(daSyncedL1BlockNumberKey) + if err != nil && isNotFoundErr(err) { + return nil + } + if err != nil { + log.Crit("Failed to read da synced L1 block number from database", "err", err) + } + if len(data) == 0 { + return nil + } + + number := new(big.Int).SetBytes(data) + if !number.IsUint64() { + log.Crit("Unexpected da synced L1 block number in database", "number", number) + } + + value := number.Uint64() + return &value +} diff --git a/core/rawdb/schema.go b/core/rawdb/schema.go index f5f82cb2cafb..a864cdb403e1 100644 --- a/core/rawdb/schema.go +++ b/core/rawdb/schema.go @@ -116,6 +116,9 @@ var ( batchMetaPrefix = []byte("R-bm") finalizedL2BlockNumberKey = []byte("R-finalized") + // Scroll da syncer store + daSyncedL1BlockNumberKey = []byte("LastDASyncedL1BlockNumber") + // Row consumption rowConsumptionPrefix = []byte("rc") // rowConsumptionPrefix + hash -> row consumption by block diff --git a/eth/backend.go b/eth/backend.go index c25147e63522..865751108c1d 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -55,6 +55,7 @@ import ( "github.com/scroll-tech/go-ethereum/p2p/enode" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rlp" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer" "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" "github.com/scroll-tech/go-ethereum/rollup/sync_service" "github.com/scroll-tech/go-ethereum/rollup/tracing" @@ -73,6 +74,7 @@ type Ethereum struct { txPool *core.TxPool syncService *sync_service.SyncService rollupSyncService *rollup_sync_service.RollupSyncService + daSyncer *da_syncer.DaSyncer blockchain *core.BlockChain handler *handler ethDialCandidates enode.Iterator @@ -233,6 +235,14 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl eth.rollupSyncService.Start() } + if config.EnableDASyncing { + eth.daSyncer, err = da_syncer.NewDaSyncer(context.Background(), chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) + if err != nil { + return nil, fmt.Errorf("cannot initialize da syncer: %w", err) + } + eth.daSyncer.Start() + } + // Permit the downloader to use the trie cache allowance during fast sync cacheLimit := cacheConfig.TrieCleanLimit + cacheConfig.TrieDirtyLimit + cacheConfig.SnapshotLimit checkpoint := config.Checkpoint @@ -592,6 +602,9 @@ func (s *Ethereum) Stop() error { if s.config.EnableRollupVerify { s.rollupSyncService.Stop() } + if s.config.EnableDASyncing{ + s.daSyncer.Stop() + } s.miner.Close() s.blockchain.Stop() s.engine.Close() diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 42ba8739d871..4c53670c9537 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -37,6 +37,7 @@ import ( "github.com/scroll-tech/go-ethereum/miner" "github.com/scroll-tech/go-ethereum/node" "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer" ) // FullNodeGPO contains default gasprice oracle settings for full node. @@ -93,6 +94,9 @@ var Defaults = Config{ GPO: FullNodeGPO, RPCTxFeeCap: 1, // 1 ether MaxBlockRange: -1, // Default unconfigured value: no block range limit for backward compatibility + DA: da_syncer.Config{ + FetcherMode: da_syncer.L1RPC, + }, } func init() { @@ -217,6 +221,12 @@ type Config struct { // Max block range for eth_getLogs api method MaxBlockRange int64 + + // Enable syncing node from DA + EnableDASyncing bool + + // DA syncer config + DA da_syncer.Config } // CreateConsensusEngine creates a consensus engine for the given chain configuration. diff --git a/rollup/da_syncer/chunk.go b/rollup/da_syncer/chunk.go index c493330173e3..cac5290b2ab0 100644 --- a/rollup/da_syncer/chunk.go +++ b/rollup/da_syncer/chunk.go @@ -14,8 +14,8 @@ type Chunk struct { type Chunks []*Chunk -// DecodeChunks decodes the provided chunks into a list of chunks. -func DecodeChunks(chunksData [][]byte) (Chunks, error) { +// decodeChunks decodes the provided chunks into a list of chunks. +func decodeChunks(chunksData [][]byte) (Chunks, error) { var chunks Chunks for _, chunk := range chunksData { if len(chunk) < 1 { @@ -39,7 +39,7 @@ func DecodeChunks(chunksData [][]byte) (Chunks, error) { } var l2Txs [][]byte - var txLen int = 0 + txLen := 0 for currentIndex := 1 + numBlocks*blockContextByteSize; currentIndex < len(chunk); currentIndex += 4 + txLen { if len(chunk) < currentIndex+4 { @@ -60,3 +60,13 @@ func DecodeChunks(chunksData [][]byte) (Chunks, error) { } return chunks, nil } + +func countTotalL1MessagePopped(chunks Chunks) uint64 { + var total uint64 = 0 + for _, chunk := range chunks { + for _, block := range chunk.BlockContexts { + total += uint64(block.NumL1Messages) + } + } + return total +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index a90fae0bd7c9..b7c0b6824b7c 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -1,5 +1,7 @@ package da_syncer +import "github.com/scroll-tech/go-ethereum/core/types" + type DAType int const ( @@ -19,12 +21,12 @@ type DAEntry struct { // Chunks contains chunk of a batch Chunks Chunks // L1Txs contains l1txs of a batch - L1Txs L1Txs + L1Txs []*types.L1MessageTx } type DA []*DAEntry -func NewCommitBatchDA(batchIndex uint64, chunks Chunks, l1txs L1Txs) *DAEntry { +func NewCommitBatchDA(batchIndex uint64, chunks Chunks, l1txs []*types.L1MessageTx) *DAEntry { return &DAEntry{ DaType: CommitBatch, BatchIndex: batchIndex, diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 5164d3b8d66b..d877c9529a08 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -2,15 +2,28 @@ package da_syncer import ( "context" + "math/big" "time" "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/params" + "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) +// Config is the configuration parameters of da syncer. +type Config struct { + FetcherMode FetcherMode // mode of fetcher + SnapshotFilePath string // path to snapshot file +} + // defaultSyncInterval is the frequency at which we query for new rollup event. const defaultSyncInterval = 60 * time.Second +// defaultFetchBlockRange is number of L1 blocks that is loaded by fetcher in one request. +const defaultFetchBlockRange = 100 + type DaSyncer struct { DaFetcher DaFetcher ctx context.Context @@ -19,8 +32,23 @@ type DaSyncer struct { batches map[uint64][]*types.Block } -func NewDaSyncer(ctx context.Context, daFetcher DaFetcher) (*DaSyncer, error) { +func NewDaSyncer(ctx context.Context, genesisConfig *params.ChainConfig, db ethdb.Database, l1Client sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*DaSyncer, error) { ctx, cancel := context.WithCancel(ctx) + var daFetcher DaFetcher + var err error + if config.FetcherMode == L1RPC { + daFetcher, err = newL1RpcDaFetcher(ctx, genesisConfig, l1Client, db, l1DeploymentBlock, defaultFetchBlockRange) + if err != nil { + cancel() + return nil, err + } + } else { + daFetcher, err = newSnapshotFetcher(defaultFetchBlockRange) + if err != nil { + cancel() + return nil, err + } + } daSyncer := DaSyncer{ DaFetcher: daFetcher, ctx: ctx, @@ -90,10 +118,43 @@ func (s *DaSyncer) syncWithDa() { s.insertBlocks(blocks) } } + } func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { - return nil, nil + var blocks []*types.Block + l1TxIndex := 0 + for _, chunk := range daEntry.Chunks { + l2TxIndex := 0 + for _, blockContext := range chunk.BlockContexts { + // create header + header := types.Header{ + // todo: maybe need to get ParentHash here too + Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), + Time: blockContext.Timestamp, + BaseFee: blockContext.BaseFee, + GasLimit: blockContext.GasLimit, + } + // create txs + var txs types.Transactions + // insert l1 msgs + for id := 0; id < int(blockContext.NumL1Messages); id++ { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) + txs = append(txs, l1Tx) + l1TxIndex++ + } + // insert l2 txs + for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { + var l2Tx *types.Transaction + l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) + txs = append(txs, l2Tx) + l2TxIndex++ + } + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil } func (s *DaSyncer) insertBlocks([]*types.Block) { diff --git a/rollup/da_syncer/l1_tx.go b/rollup/da_syncer/l1_tx.go deleted file mode 100644 index 80de038df95d..000000000000 --- a/rollup/da_syncer/l1_tx.go +++ /dev/null @@ -1,6 +0,0 @@ -package da_syncer - -type L1Tx struct { -} - -type L1Txs []*L1Tx diff --git a/rollup/da_syncer/l1rpc_fetcher.go b/rollup/da_syncer/l1rpc_fetcher.go index 97f3c7838a9b..1c27d90b6c14 100644 --- a/rollup/da_syncer/l1rpc_fetcher.go +++ b/rollup/da_syncer/l1rpc_fetcher.go @@ -2,12 +2,16 @@ package da_syncer import ( "context" + "encoding/binary" "fmt" + "math/big" "reflect" "github.com/scroll-tech/go-ethereum/accounts/abi" "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/params" "github.com/scroll-tech/go-ethereum/rollup/sync_service" @@ -16,6 +20,7 @@ import ( type L1RPCFetcher struct { fetchBlockRange uint64 client *L1Client + db ethdb.Database ctx context.Context latestProcessedBlock uint64 scrollChainABI *abi.ABI @@ -24,7 +29,7 @@ type L1RPCFetcher struct { l1FinalizeBatchEventSignature common.Hash } -func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient, l1DeploymentBlock, fetchBlockRange uint64) (DaFetcher, error) { +func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient, db ethdb.Database, l1DeploymentBlock, fetchBlockRange uint64) (DaFetcher, error) { // terminate if the caller does not provide an L1 client (e.g. in tests) if l1Client == nil || (reflect.ValueOf(l1Client).Kind() == reflect.Ptr && reflect.ValueOf(l1Client).IsNil()) { log.Warn("No L1 client provided, L1 rollup sync service will not run") @@ -51,13 +56,17 @@ func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l if l1DeploymentBlock > 0 { latestProcessedBlock = l1DeploymentBlock - 1 } - - // todo: read latest processed block from db + block := rawdb.ReadDASyncedL1BlockNumber(db) + if block != nil { + // restart from latest synced block number + latestProcessedBlock = *block + } daFetcher := L1RPCFetcher{ fetchBlockRange: fetchBlockRange, ctx: ctx, client: client, + db: db, latestProcessedBlock: latestProcessedBlock, scrollChainABI: scrollChainABI, l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, @@ -96,6 +105,7 @@ func (f *L1RPCFetcher) FetchDA() (DA, error) { } f.latestProcessedBlock = to + rawdb.WriteDASyncedL1BlockNumber(f.db, to) return da, nil } @@ -140,17 +150,12 @@ func (f *L1RPCFetcher) processLogsToDA(logs []types.Log) (DA, error) { return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) } } - - // note: the batch updates above are idempotent, if we crash - // before this line and reexecute the previous steps, we will - // get the same result. - // todo: store to db latest process block number return da, nil } -func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, L1Txs, error) { +func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, []*types.L1MessageTx, error) { var chunks Chunks - var l1Txs L1Txs + var l1Txs []*types.L1MessageTx if batchIndex == 0 { return chunks, l1Txs, nil } @@ -205,13 +210,53 @@ func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, L1T return nil, nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } - chunks, err = DecodeChunks(args.Chunks) + chunks, err = decodeChunks(args.Chunks) if err != nil { - return nil, nil, fmt.Errorf("failed to unpack decode chunks in batch number: %v, err: %w", batchIndex, err) + return nil, nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - // todo: l1txs can be loaded from db that filled by existing l1 msg sync service - l1Txs = nil + parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) + totalL1MessagePopped := countTotalL1MessagePopped(chunks) + skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < int(totalL1MessagePopped); index++ { + for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Txs = append(l1Txs, rawdb.ReadL1Message(f.db, currentIndex)) + currentIndex++ + } return chunks, l1Txs, nil } + +func getBatchTotalL1MessagePopped(batchHeader []byte) uint64 { + return binary.BigEndian.Uint64(batchHeader[17:25]) +} + +func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped uint64) ([]*big.Int, error) { + length := len(skippedL1MessageBitmap) + if length%32 != 0 { + return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) + } + if length*8 < int(totalL1MessagePopped) { + return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) + } + var skippedBitmap []*big.Int + for index := 0; index < length/32; index++ { + bitmap := big.NewInt(0).SetBytes(skippedL1MessageBitmap[index*32 : index*32+32]) + skippedBitmap = append(skippedBitmap, bitmap) + } + return skippedBitmap, nil +} + +func isL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { + quo := index / 256 + rem := index % 256 + return skippedBitmap[quo].Bit(int(rem)) != 0 +} diff --git a/rollup/da_syncer/modes.go b/rollup/da_syncer/modes.go new file mode 100644 index 000000000000..14e004d04471 --- /dev/null +++ b/rollup/da_syncer/modes.go @@ -0,0 +1,52 @@ +package da_syncer + +import "fmt" + +// FetcherMode represents the mode of fetcher +type FetcherMode int + +const ( + // L1RPC mode fetches DA from L1RPC + L1RPC FetcherMode = iota + // Snapshot mode loads DA from snapshot file + Snapshot +) + +func (mode FetcherMode) IsValid() bool { + return mode >= L1RPC && mode <= Snapshot +} + +// String implements the stringer interface. +func (mode FetcherMode) String() string { + switch mode { + case L1RPC: + return "l1rpc" + case Snapshot: + return "snapshot" + default: + return "unknown" + } +} + +func (mode FetcherMode) MarshalText() ([]byte, error) { + switch mode { + case L1RPC: + return []byte("l1rpc"), nil + case Snapshot: + return []byte("snapshot"), nil + default: + return nil, fmt.Errorf("unknown sync mode %d", mode) + } +} + +func (mode *FetcherMode) UnmarshalText(text []byte) error { + switch string(text) { + case "l1rpc": + *mode = L1RPC + case "snapshot": + *mode = Snapshot + default: + return fmt.Errorf(`unknown sync mode %q, want "full", "fast" or "light"`, text) + } + return nil +} From 635f74474b0f155b855c942bcd51f39d33a76717 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 13 Feb 2024 10:09:41 +0100 Subject: [PATCH 03/59] fix lint --- eth/backend.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eth/backend.go b/eth/backend.go index 865751108c1d..1995659ef53f 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -602,7 +602,7 @@ func (s *Ethereum) Stop() error { if s.config.EnableRollupVerify { s.rollupSyncService.Stop() } - if s.config.EnableDASyncing{ + if s.config.EnableDASyncing { s.daSyncer.Stop() } s.miner.Close() From 69d6c392d506891795f461fbf57df3cd7c1b6cc2 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 19 Feb 2024 11:38:55 +0100 Subject: [PATCH 04/59] address comments --- cmd/geth/main.go | 2 +- cmd/utils/flags.go | 6 +++--- rollup/da_syncer/da_syncer.go | 3 ++- rollup/da_syncer/modes.go | 2 +- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/cmd/geth/main.go b/cmd/geth/main.go index dea4b7fbec7e..79d6b38a8f6d 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -166,7 +166,7 @@ var ( utils.L1DeploymentBlockFlag, utils.CircuitCapacityCheckEnabledFlag, utils.RollupVerifyEnabledFlag, - utils.DaSyncEnabledFlag, + utils.DASyncEnabledFlag, utils.DAModeFlag, utils.DASnapshotFileFlag, } diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 79f79fe353a8..6247ec0633a8 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -851,7 +851,7 @@ var ( } // Da syncing settings - DaSyncEnabledFlag = cli.BoolFlag{ + DASyncEnabledFlag = cli.BoolFlag{ Name: "da.sync", Usage: "Enable node syncing from DA", } @@ -1577,8 +1577,8 @@ func setEnableRollupVerify(ctx *cli.Context, cfg *ethconfig.Config) { } func setDa(ctx *cli.Context, cfg *ethconfig.Config) { - if ctx.GlobalIsSet(DaSyncEnabledFlag.Name) { - cfg.EnableDASyncing = ctx.GlobalBool(DaSyncEnabledFlag.Name) + if ctx.GlobalIsSet(DASyncEnabledFlag.Name) { + cfg.EnableDASyncing = ctx.GlobalBool(DASyncEnabledFlag.Name) if ctx.GlobalIsSet(DAModeFlag.Name) { cfg.DA.FetcherMode = *GlobalTextMarshaler(ctx, DAModeFlag.Name).(*da_syncer.FetcherMode) } diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index d877c9529a08..eeef0394ebef 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -136,7 +136,8 @@ func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { GasLimit: blockContext.GasLimit, } // create txs - var txs types.Transactions + // var txs types.Transactions + txs := make(types.Transactions, 0, blockContext.NumTransactions) // insert l1 msgs for id := 0; id < int(blockContext.NumL1Messages); id++ { l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) diff --git a/rollup/da_syncer/modes.go b/rollup/da_syncer/modes.go index 14e004d04471..bfcc1d1dfba0 100644 --- a/rollup/da_syncer/modes.go +++ b/rollup/da_syncer/modes.go @@ -46,7 +46,7 @@ func (mode *FetcherMode) UnmarshalText(text []byte) error { case "snapshot": *mode = Snapshot default: - return fmt.Errorf(`unknown sync mode %q, want "full", "fast" or "light"`, text) + return fmt.Errorf(`unknown sync mode %q, want "l1rpc" or "snapshot"`, text) } return nil } From 2bef690d89f3657f8a691f77f6861db90ad9ea7d Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 9 Apr 2024 14:36:23 +0200 Subject: [PATCH 05/59] poc --- eth/backend.go | 42 +++++++------- rollup/da_syncer/da_fetcher.go | 2 +- rollup/da_syncer/da_syncer.go | 82 +++++++++++++++++++++------- rollup/da_syncer/l1rpc_fetcher.go | 28 +++++++--- rollup/da_syncer/snapshot_fetcher.go | 4 +- rollup/sync_service/sync_service.go | 6 +- 6 files changed, 108 insertions(+), 56 deletions(-) diff --git a/eth/backend.go b/eth/backend.go index 7d870cece655..ab5d1faa739c 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -236,7 +236,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl } if config.EnableDASyncing { - eth.daSyncer, err = da_syncer.NewDaSyncer(context.Background(), chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) + eth.daSyncer, err = da_syncer.NewDaSyncer(context.Background(), eth.blockchain, chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) if err != nil { return nil, fmt.Errorf("cannot initialize da syncer: %w", err) } @@ -249,18 +249,20 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl if checkpoint == nil { checkpoint = params.TrustedCheckpoints[genesisHash] } - if eth.handler, err = newHandler(&handlerConfig{ - Database: chainDb, - Chain: eth.blockchain, - TxPool: eth.txPool, - Network: config.NetworkId, - Sync: config.SyncMode, - BloomCache: uint64(cacheLimit), - EventMux: eth.eventMux, - Checkpoint: checkpoint, - Whitelist: config.Whitelist, - }); err != nil { - return nil, err + if !config.EnableDASyncing { + if eth.handler, err = newHandler(&handlerConfig{ + Database: chainDb, + Chain: eth.blockchain, + TxPool: eth.txPool, + Network: config.NetworkId, + Sync: config.SyncMode, + BloomCache: uint64(cacheLimit), + EventMux: eth.eventMux, + Checkpoint: checkpoint, + Whitelist: config.Whitelist, + }); err != nil { + return nil, err + } } eth.miner = miner.New(eth, &config.Miner, chainConfig, eth.EventMux(), eth.engine, eth.isLocalBlock) @@ -292,7 +294,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl // Register the backend on the node stack.RegisterAPIs(eth.APIs()) - stack.RegisterProtocols(eth.Protocols()) + // stack.RegisterProtocols(eth.Protocols()) stack.RegisterLifecycle(eth) // Check for unclean shutdown if uncleanShutdowns, discards, err := rawdb.PushUncleanShutdownMarker(chainDb); err != nil { @@ -347,12 +349,12 @@ func (s *Ethereum) APIs() []rpc.API { Version: "1.0", Service: NewPublicMinerAPI(s), Public: true, - }, { + }, /*{ Namespace: "eth", Version: "1.0", Service: downloader.NewPublicDownloaderAPI(s.handler.downloader, s.eventMux), Public: true, - }, { + },*/{ Namespace: "miner", Version: "1.0", Service: NewPrivateMinerAPI(s), @@ -518,7 +520,7 @@ func (s *Ethereum) StartMining(threads int) error { } // If mining is started, we can disable the transaction rejection mechanism // introduced to speed sync times. - atomic.StoreUint32(&s.handler.acceptTxs, 1) + // atomic.StoreUint32(&s.handler.acceptTxs, 1) go s.miner.Start(eb) } @@ -574,7 +576,7 @@ func (s *Ethereum) Start() error { s.startBloomHandlers(params.BloomBitsBlocks) // Figure out a max peers count based on the server limits - maxPeers := s.p2pServer.MaxPeers + // maxPeers := s.p2pServer.MaxPeers //if s.config.LightServ > 0 { // if s.config.LightPeers >= s.p2pServer.MaxPeers { // return fmt.Errorf("invalid peer config: light peer count (%d) >= total peer count (%d)", s.config.LightPeers, s.p2pServer.MaxPeers) @@ -582,7 +584,7 @@ func (s *Ethereum) Start() error { // maxPeers -= s.config.LightPeers //} // Start the networking layer and the light server if requested - s.handler.Start(maxPeers) + // s.handler.Start(maxPeers) return nil } @@ -592,7 +594,7 @@ func (s *Ethereum) Stop() error { // Stop all the peer-related stuff first. s.ethDialCandidates.Close() s.snapDialCandidates.Close() - s.handler.Stop() + // s.handler.Stop() // Then stop everything else. s.bloomIndexer.Close() diff --git a/rollup/da_syncer/da_fetcher.go b/rollup/da_syncer/da_fetcher.go index 86c49f8014e3..72033ef7572f 100644 --- a/rollup/da_syncer/da_fetcher.go +++ b/rollup/da_syncer/da_fetcher.go @@ -2,5 +2,5 @@ package da_syncer // DaFetcher encapsulates functions required to fetch data from l1 type DaFetcher interface { - FetchDA() (DA, error) + FetchDA() (DA, uint64, error) } diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index eeef0394ebef..6184d5f04135 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -2,9 +2,13 @@ package da_syncer import ( "context" + "errors" + "fmt" "math/big" "time" + "github.com/scroll-tech/go-ethereum/core" + "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" @@ -18,21 +22,27 @@ type Config struct { SnapshotFilePath string // path to snapshot file } +var ( + errInvalidChain = errors.New("retrieved hash chain is invalid") +) + // defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 60 * time.Second +const defaultSyncInterval = 45 * time.Second // defaultFetchBlockRange is number of L1 blocks that is loaded by fetcher in one request. -const defaultFetchBlockRange = 100 +const defaultFetchBlockRange = 1000 type DaSyncer struct { - DaFetcher DaFetcher - ctx context.Context - cancel context.CancelFunc + DaFetcher DaFetcher + ctx context.Context + cancel context.CancelFunc + db ethdb.Database + blockchain *core.BlockChain // batches is map from batchIndex to batch blocks batches map[uint64][]*types.Block } -func NewDaSyncer(ctx context.Context, genesisConfig *params.ChainConfig, db ethdb.Database, l1Client sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*DaSyncer, error) { +func NewDaSyncer(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, l1Client sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*DaSyncer, error) { ctx, cancel := context.WithCancel(ctx) var daFetcher DaFetcher var err error @@ -50,10 +60,12 @@ func NewDaSyncer(ctx context.Context, genesisConfig *params.ChainConfig, db ethd } } daSyncer := DaSyncer{ - DaFetcher: daFetcher, - ctx: ctx, - cancel: cancel, - batches: make(map[uint64][]*types.Block), + DaFetcher: daFetcher, + ctx: ctx, + cancel: cancel, + db: db, + blockchain: blockchain, + batches: make(map[uint64][]*types.Block), } return &daSyncer, nil } @@ -70,11 +82,12 @@ func (s *DaSyncer) Start() { defer syncTicker.Stop() for { + s.syncWithDa() select { case <-s.ctx.Done(): return case <-syncTicker.C: - s.syncWithDa() + continue } } }() @@ -93,7 +106,8 @@ func (s *DaSyncer) Stop() { } func (s *DaSyncer) syncWithDa() { - da, err := s.DaFetcher.FetchDA() + log.Info("DaSyncer syncing") + da, to, err := s.DaFetcher.FetchDA() if err != nil { log.Error("failed to fetch DA", "err", err) return @@ -106,34 +120,39 @@ func (s *DaSyncer) syncWithDa() { log.Warn("failed to process DA to blocks", "err", err) return } + log.Info("commit batch", "batchindex", daEntry.BatchIndex) s.batches[daEntry.BatchIndex] = blocks case RevertBatch: + log.Info("revert batch", "batchindex", daEntry.BatchIndex) delete(s.batches, daEntry.BatchIndex) case FinalizeBatch: + log.Info("finalize batch", "batchindex", daEntry.BatchIndex) blocks, ok := s.batches[daEntry.BatchIndex] if !ok { - log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) + log.Info("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) return } s.insertBlocks(blocks) } } - + rawdb.WriteDASyncedL1BlockNumber(s.db, to) } func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { var blocks []*types.Block l1TxIndex := 0 + prevHash := s.blockchain.CurrentBlock().Hash() for _, chunk := range daEntry.Chunks { l2TxIndex := 0 for _, blockContext := range chunk.BlockContexts { // create header header := types.Header{ // todo: maybe need to get ParentHash here too - Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), - Time: blockContext.Timestamp, - BaseFee: blockContext.BaseFee, - GasLimit: blockContext.GasLimit, + ParentHash: prevHash, + Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), + Time: blockContext.Timestamp, + BaseFee: blockContext.BaseFee, + GasLimit: blockContext.GasLimit, } // create txs // var txs types.Transactions @@ -144,20 +163,41 @@ func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { txs = append(txs, l1Tx) l1TxIndex++ } + // log.Info("processing block", "block hash", blockContext.BlockNumber, "numl1messages", blockContext.NumL1Messages, "numtxs", blockContext.NumTransactions) // insert l2 txs for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { - var l2Tx *types.Transaction + l2Tx := &types.Transaction{} + // log.Info("processing l2 tx", "num", id, "l2tx", l2Tx) l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) txs = append(txs, l2Tx) l2TxIndex++ } block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + prevHash = block.Hash() blocks = append(blocks, block) } } return blocks, nil } -func (s *DaSyncer) insertBlocks([]*types.Block) { - +func (s *DaSyncer) insertBlocks(blocks []*types.Block) error { + for _, block := range blocks { + log.Info("block info", "number", block.Number(), "hash", block.Hash(), "parentHash", block.ParentHash()) + log.Info("block header", "block header", block.Header()) + } + if index, err := s.blockchain.InsertChain(blocks); err != nil { + log.Info("err != nil") + if index < len(blocks) { + log.Debug("Downloaded item processing failed", "number", blocks[index].Header().Number, "hash", blocks[index].Header().Hash(), "err", err) + } else { + // The InsertChain method in blockchain.go will sometimes return an out-of-bounds index, + // when it needs to preprocess blocks to import a sidechain. + // The importer will put together a new list of blocks to import, which is a superset + // of the blocks delivered from the downloader, and the indexing will be off. + log.Debug("Downloaded item processing failed on sidechain import", "index", index, "err", err) + } + return fmt.Errorf("%w: %v", errInvalidChain, err) + } + log.Info("insertblocks completed", "blockchain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + return nil } diff --git a/rollup/da_syncer/l1rpc_fetcher.go b/rollup/da_syncer/l1rpc_fetcher.go index 1c27d90b6c14..5dcf0c400c1d 100644 --- a/rollup/da_syncer/l1rpc_fetcher.go +++ b/rollup/da_syncer/l1rpc_fetcher.go @@ -77,11 +77,11 @@ func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l } // Fetch DA fetches all da events and converts it to DA format in some fetchBlockRange -func (f *L1RPCFetcher) FetchDA() (DA, error) { +func (f *L1RPCFetcher) FetchDA() (DA, uint64, error) { latestConfirmed, err := f.client.getLatestFinalizedBlockNumber(f.ctx) if err != nil { log.Warn("failed to get latest confirmed block number", "err", err) - return nil, err + return nil, 0, err } log.Trace("Da fetcher fetch rollup events", "latest processed block", f.latestProcessedBlock, "latest confirmed", latestConfirmed) @@ -91,22 +91,26 @@ func (f *L1RPCFetcher) FetchDA() (DA, error) { if to > latestConfirmed { to = latestConfirmed } + log.Info("L1RpcFetcher fetching...", "from", from, "to", to, "latest processed block", f.latestProcessedBlock, "latest confirmed", latestConfirmed) logs, err := f.client.fetchRollupEventsInRange(f.ctx, from, to) + for id, ll := range logs { + log.Info("log number", "id", id, "block number", ll.BlockNumber, "txhash", ll.TxHash) + } if err != nil { log.Error("failed to fetch rollup events in range", "from block", from, "to block", to, "err", err) - return nil, err + return nil, 0, err } - + log.Info("L1RPCFetcher fetched logs", "log size", len(logs)) da, err := f.processLogsToDA(logs) if err != nil { log.Error("failed to process rollup events in range", "from block", from, "to block", to, "err", err) - return nil, err + return nil, 0, err } - + log.Info("L1Rpcfetcher processed logs to da", "da len", len(da)) f.latestProcessedBlock = to - rawdb.WriteDASyncedL1BlockNumber(f.db, to) - return da, nil + // rawdb.WriteDASyncedL1BlockNumber(f.db, to) + return da, to, nil } func (f *L1RPCFetcher) processLogsToDA(logs []types.Log) (DA, error) { @@ -228,7 +232,11 @@ func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, []* for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ } - l1Txs = append(l1Txs, rawdb.ReadL1Message(f.db, currentIndex)) + l1Tx := rawdb.ReadL1Message(f.db, currentIndex) + if l1Tx == nil { + return nil, nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) currentIndex++ } return chunks, l1Txs, nil @@ -260,3 +268,5 @@ func isL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { rem := index % 256 return skippedBitmap[quo].Bit(int(rem)) != 0 } + +// l2geth --scroll --datadir "./l2geth-datadir" --gcmode archive --cache.noprefetch --http --http.addr "0.0.0.0" --http.port 8545 --http.api "eth,net,web3,debug,scroll" --l1.endpoint "$L2GETH_L1_ENDPOINT" --da.sync diff --git a/rollup/da_syncer/snapshot_fetcher.go b/rollup/da_syncer/snapshot_fetcher.go index a109086a2c0f..e82c04ffebc9 100644 --- a/rollup/da_syncer/snapshot_fetcher.go +++ b/rollup/da_syncer/snapshot_fetcher.go @@ -11,6 +11,6 @@ func newSnapshotFetcher(fetchBlockRange uint64) (DaFetcher, error) { return &daFetcher, nil } -func (f *SnapshotFetcher) FetchDA() (DA, error) { - return nil, nil +func (f *SnapshotFetcher) FetchDA() (DA, uint64, error) { + return nil, 0, nil } diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 2720aec76b1b..1a275173ecde 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -20,7 +20,7 @@ const ( DefaultFetchBlockRange = uint64(100) // DefaultPollInterval is the frequency at which we query for new L1 messages. - DefaultPollInterval = time.Second * 10 + DefaultPollInterval = time.Second * 60 // LogProgressInterval is the frequency at which we log progress. LogProgressInterval = time.Second * 10 @@ -97,7 +97,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - s.fetchMessages() + // s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -107,7 +107,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - s.fetchMessages() + // s.fetchMessages() select { case <-s.ctx.Done(): From b6fc6dd47a98af402e4c5369590c7f246849303e Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 29 Apr 2024 16:32:17 +0200 Subject: [PATCH 06/59] update poc, preprocess blocks before inserting to generate missing header fields --- consensus/clique/clique.go | 113 ++++++++++++++------------- core/blockchain.go | 21 +++++ rollup/da_syncer/block_context.go | 6 +- rollup/da_syncer/da_fetcher.go | 1 + rollup/da_syncer/da_syncer.go | 67 +++++++++------- rollup/da_syncer/l1rpc_fetcher.go | 13 ++- rollup/da_syncer/snapshot_fetcher.go | 4 + rollup/sync_service/sync_service.go | 6 +- 8 files changed, 135 insertions(+), 96 deletions(-) diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index 33ccb82a7a86..a0e033cb6a34 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -145,27 +145,28 @@ type SignerFn func(signer accounts.Account, mimeType string, message []byte) ([] // ecrecover extracts the Ethereum account address from a signed header. func ecrecover(header *types.Header, sigcache *lru.ARCCache) (common.Address, error) { - // If the signature's already cached, return that - hash := header.Hash() - if address, known := sigcache.Get(hash); known { - return address.(common.Address), nil - } - // Retrieve the signature from the header extra-data - if len(header.Extra) < extraSeal { - return common.Address{}, errMissingSignature - } - signature := header.Extra[len(header.Extra)-extraSeal:] - - // Recover the public key and the Ethereum address - pubkey, err := crypto.Ecrecover(SealHash(header).Bytes(), signature) - if err != nil { - return common.Address{}, err - } - var signer common.Address - copy(signer[:], crypto.Keccak256(pubkey[1:])[12:]) - - sigcache.Add(hash, signer) - return signer, nil + return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil + // // If the signature's already cached, return that + // hash := header.Hash() + // if address, known := sigcache.Get(hash); known { + // return address.(common.Address), nil + // } + // // Retrieve the signature from the header extra-data + // if len(header.Extra) < extraSeal { + // return common.Address{}, errMissingSignature + // } + // signature := header.Extra[len(header.Extra)-extraSeal:] + + // // Recover the public key and the Ethereum address + // pubkey, err := crypto.Ecrecover(SealHash(header).Bytes(), signature) + // if err != nil { + // return common.Address{}, err + // } + // var signer common.Address + // copy(signer[:], crypto.Keccak256(pubkey[1:])[12:]) + + // sigcache.Add(hash, signer) + // return signer, nil } // Clique is the proof-of-authority consensus engine proposed to support the @@ -211,7 +212,8 @@ func New(config *params.CliqueConfig, db ethdb.Database) *Clique { // Author implements consensus.Engine, returning the Ethereum address recovered // from the signature in the header's extra-data section. func (c *Clique) Author(header *types.Header) (common.Address, error) { - return ecrecover(header, c.signatures) + return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil + // return ecrecover(header, c.signatures) } // VerifyHeader checks whether a header conforms to the consensus rules. @@ -266,21 +268,21 @@ func (c *Clique) verifyHeader(chain consensus.ChainHeaderReader, header *types.H if checkpoint && !bytes.Equal(header.Nonce[:], nonceDropVote) { return errInvalidCheckpointVote } - // Check that the extra-data contains both the vanity and signature - if len(header.Extra) < extraVanity { - return errMissingVanity - } - if len(header.Extra) < extraVanity+extraSeal { - return errMissingSignature - } - // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise - signersBytes := len(header.Extra) - extraVanity - extraSeal - if !checkpoint && signersBytes != 0 { - return errExtraSigners - } - if checkpoint && signersBytes%common.AddressLength != 0 { - return errInvalidCheckpointSigners - } + // // Check that the extra-data contains both the vanity and signature + // if len(header.Extra) < extraVanity { + // return errMissingVanity + // } + // if len(header.Extra) < extraVanity+extraSeal { + // return errMissingSignature + // } + // // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise + // signersBytes := len(header.Extra) - extraVanity - extraSeal + // if !checkpoint && signersBytes != 0 { + // return errExtraSigners + // } + // if checkpoint && signersBytes%common.AddressLength != 0 { + // return errInvalidCheckpointSigners + // } // Ensure that the mix digest is zero as we don't have fork protection currently if header.MixDigest != (common.Hash{}) { return errInvalidMixDigest @@ -347,24 +349,25 @@ func (c *Clique) verifyCascadingFields(chain consensus.ChainHeaderReader, header // Verify the header's EIP-1559 attributes. return err } - // Retrieve the snapshot needed to verify this header and cache it - snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) - if err != nil { - return err - } - // If the block is a checkpoint block, verify the signer list - if number%c.config.Epoch == 0 { - signers := make([]byte, len(snap.Signers)*common.AddressLength) - for i, signer := range snap.signers() { - copy(signers[i*common.AddressLength:], signer[:]) - } - extraSuffix := len(header.Extra) - extraSeal - if !bytes.Equal(header.Extra[extraVanity:extraSuffix], signers) { - return errMismatchingCheckpointSigners - } - } - // All basic checks passed, verify the seal and return - return c.verifySeal(snap, header, parents) + // // Retrieve the snapshot needed to verify this header and cache it + // snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) + // if err != nil { + // return err + // } + // // If the block is a checkpoint block, verify the signer list + // if number%c.config.Epoch == 0 { + // signers := make([]byte, len(snap.Signers)*common.AddressLength) + // for i, signer := range snap.signers() { + // copy(signers[i*common.AddressLength:], signer[:]) + // } + // extraSuffix := len(header.Extra) - extraSeal + // if !bytes.Equal(header.Extra[extraVanity:extraSuffix], signers) { + // return errMismatchingCheckpointSigners + // } + // } + // // All basic checks passed, verify the seal and return + // return c.verifySeal(snap, header, parents) + return nil } // snapshot retrieves the authorization snapshot at a given point in time. diff --git a/core/blockchain.go b/core/blockchain.go index add957a9440e..463aae8909b2 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1799,6 +1799,27 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er return it.index, err } +// PreprocessBlock processes block on top of the chain to calculate receipts, bloom and state root +func (bc *BlockChain) PreprocessBlock(block *types.Block) (common.Hash, types.Bloom, common.Hash, uint64, error) { + // Retrieve the parent block and it's state to execute on top + parent := bc.CurrentBlock().Header() + if parent == nil { + parent = bc.GetHeader(block.ParentHash(), block.NumberU64()-1) + } + statedb, err := state.New(parent.Root, bc.stateCache, bc.snaps) + if err != nil { + return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err + } + receipts, _, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig) + if err != nil { + return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err + } + receiptSha := types.DeriveSha(receipts, trie.NewStackTrie(nil)) + bloom := types.CreateBloom(receipts) + stateRoot := statedb.GetRootHash() + return receiptSha, bloom, stateRoot, usedGas, nil +} + // insertSideChain is called when an import batch hits upon a pruned ancestor // error, which happens when a sidechain with a sufficiently old fork-block is // found. diff --git a/rollup/da_syncer/block_context.go b/rollup/da_syncer/block_context.go index 40cdd0ac0255..9f486b25033e 100644 --- a/rollup/da_syncer/block_context.go +++ b/rollup/da_syncer/block_context.go @@ -4,6 +4,8 @@ import ( "encoding/binary" "errors" "math/big" + + "github.com/scroll-tech/go-ethereum/common" ) // BlockContext represents the essential data of a block in the ScrollChain. @@ -24,7 +26,9 @@ func decodeBlockContext(encodedBlockContext []byte) (*BlockContext, error) { return nil, errors.New("block encoding is not 60 bytes long") } baseFee := big.NewInt(0).SetBytes(encodedBlockContext[16:48]) - + if baseFee.Cmp(common.Big0) == 0 { + baseFee = nil + } return &BlockContext{ BlockNumber: binary.BigEndian.Uint64(encodedBlockContext[0:8]), Timestamp: binary.BigEndian.Uint64(encodedBlockContext[8:16]), diff --git a/rollup/da_syncer/da_fetcher.go b/rollup/da_syncer/da_fetcher.go index 72033ef7572f..0b26890cd124 100644 --- a/rollup/da_syncer/da_fetcher.go +++ b/rollup/da_syncer/da_fetcher.go @@ -3,4 +3,5 @@ package da_syncer // DaFetcher encapsulates functions required to fetch data from l1 type DaFetcher interface { FetchDA() (DA, uint64, error) + SetLatestProcessedBlock(to uint64) } diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 6184d5f04135..5daff9b804a1 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -7,6 +7,7 @@ import ( "math/big" "time" + "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core" "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/core/types" @@ -14,6 +15,7 @@ import ( "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/sync_service" + "github.com/scroll-tech/go-ethereum/trie" ) // Config is the configuration parameters of da syncer. @@ -120,39 +122,37 @@ func (s *DaSyncer) syncWithDa() { log.Warn("failed to process DA to blocks", "err", err) return } - log.Info("commit batch", "batchindex", daEntry.BatchIndex) + log.Debug("commit batch", "batchindex", daEntry.BatchIndex) s.batches[daEntry.BatchIndex] = blocks case RevertBatch: - log.Info("revert batch", "batchindex", daEntry.BatchIndex) + log.Debug("revert batch", "batchindex", daEntry.BatchIndex) delete(s.batches, daEntry.BatchIndex) case FinalizeBatch: - log.Info("finalize batch", "batchindex", daEntry.BatchIndex) + log.Debug("finalize batch", "batchindex", daEntry.BatchIndex) blocks, ok := s.batches[daEntry.BatchIndex] if !ok { - log.Info("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) + log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) return } s.insertBlocks(blocks) } } rawdb.WriteDASyncedL1BlockNumber(s.db, to) + s.DaFetcher.SetLatestProcessedBlock(to) } func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { var blocks []*types.Block l1TxIndex := 0 - prevHash := s.blockchain.CurrentBlock().Hash() for _, chunk := range daEntry.Chunks { l2TxIndex := 0 for _, blockContext := range chunk.BlockContexts { // create header header := types.Header{ - // todo: maybe need to get ParentHash here too - ParentHash: prevHash, - Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), - Time: blockContext.Timestamp, - BaseFee: blockContext.BaseFee, - GasLimit: blockContext.GasLimit, + Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), + Time: blockContext.Timestamp, + BaseFee: blockContext.BaseFee, + GasLimit: blockContext.GasLimit, } // create txs // var txs types.Transactions @@ -163,17 +163,14 @@ func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { txs = append(txs, l1Tx) l1TxIndex++ } - // log.Info("processing block", "block hash", blockContext.BlockNumber, "numl1messages", blockContext.NumL1Messages, "numtxs", blockContext.NumTransactions) // insert l2 txs for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { l2Tx := &types.Transaction{} - // log.Info("processing l2 tx", "num", id, "l2tx", l2Tx) l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) txs = append(txs, l2Tx) l2TxIndex++ } block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) - prevHash = block.Hash() blocks = append(blocks, block) } } @@ -181,23 +178,35 @@ func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { } func (s *DaSyncer) insertBlocks(blocks []*types.Block) error { + prevHash := s.blockchain.CurrentBlock().Hash() for _, block := range blocks { - log.Info("block info", "number", block.Number(), "hash", block.Hash(), "parentHash", block.ParentHash()) - log.Info("block header", "block header", block.Header()) - } - if index, err := s.blockchain.InsertChain(blocks); err != nil { - log.Info("err != nil") - if index < len(blocks) { - log.Debug("Downloaded item processing failed", "number", blocks[index].Header().Number, "hash", blocks[index].Header().Hash(), "err", err) - } else { - // The InsertChain method in blockchain.go will sometimes return an out-of-bounds index, - // when it needs to preprocess blocks to import a sidechain. - // The importer will put together a new list of blocks to import, which is a superset - // of the blocks delivered from the downloader, and the indexing will be off. - log.Debug("Downloaded item processing failed on sidechain import", "index", index, "err", err) + header := block.Header() + txs := block.Transactions() + + // fill header with all necessary fields + var err error + header.ReceiptHash, header.Bloom, header.Root, header.GasUsed, err = s.blockchain.PreprocessBlock(block) + if err != nil { + return fmt.Errorf("block preprocessing failed, block number: %d, error: %v", block.Number(), err) } - return fmt.Errorf("%w: %v", errInvalidChain, err) + header.UncleHash = common.HexToHash("0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347") + header.Difficulty = common.Big1 + header.BaseFee = nil + header.TxHash = types.DeriveSha(txs, trie.NewStackTrie(nil)) + header.ParentHash = prevHash + + fullBlock := types.NewBlockWithHeader(header).WithBody(txs, make([]*types.Header, 0)) + + if index, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { + if index < len(blocks) { + log.Debug("Block insert failed", "number", blocks[index].Header().Number, "hash", blocks[index].Header().Hash(), "err", err) + } + return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) + } + prevHash = fullBlock.Hash() + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) } + log.Info("insertblocks completed", "blockchain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) return nil -} +} \ No newline at end of file diff --git a/rollup/da_syncer/l1rpc_fetcher.go b/rollup/da_syncer/l1rpc_fetcher.go index 5dcf0c400c1d..53aec0b8539f 100644 --- a/rollup/da_syncer/l1rpc_fetcher.go +++ b/rollup/da_syncer/l1rpc_fetcher.go @@ -76,6 +76,10 @@ func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l return &daFetcher, nil } +func (f *L1RPCFetcher) SetLatestProcessedBlock(to uint64) { + f.latestProcessedBlock = to +} + // Fetch DA fetches all da events and converts it to DA format in some fetchBlockRange func (f *L1RPCFetcher) FetchDA() (DA, uint64, error) { latestConfirmed, err := f.client.getLatestFinalizedBlockNumber(f.ctx) @@ -91,25 +95,18 @@ func (f *L1RPCFetcher) FetchDA() (DA, uint64, error) { if to > latestConfirmed { to = latestConfirmed } - log.Info("L1RpcFetcher fetching...", "from", from, "to", to, "latest processed block", f.latestProcessedBlock, "latest confirmed", latestConfirmed) logs, err := f.client.fetchRollupEventsInRange(f.ctx, from, to) - for id, ll := range logs { - log.Info("log number", "id", id, "block number", ll.BlockNumber, "txhash", ll.TxHash) - } if err != nil { log.Error("failed to fetch rollup events in range", "from block", from, "to block", to, "err", err) return nil, 0, err } - log.Info("L1RPCFetcher fetched logs", "log size", len(logs)) da, err := f.processLogsToDA(logs) if err != nil { log.Error("failed to process rollup events in range", "from block", from, "to block", to, "err", err) return nil, 0, err } - log.Info("L1Rpcfetcher processed logs to da", "da len", len(da)) - f.latestProcessedBlock = to - // rawdb.WriteDASyncedL1BlockNumber(f.db, to) + log.Trace("L1Rpcfetcher fetched and processed logs to da", "from", from, "to", to, "da len", len(da)) return da, to, nil } diff --git a/rollup/da_syncer/snapshot_fetcher.go b/rollup/da_syncer/snapshot_fetcher.go index e82c04ffebc9..b317c03edecb 100644 --- a/rollup/da_syncer/snapshot_fetcher.go +++ b/rollup/da_syncer/snapshot_fetcher.go @@ -14,3 +14,7 @@ func newSnapshotFetcher(fetchBlockRange uint64) (DaFetcher, error) { func (f *SnapshotFetcher) FetchDA() (DA, uint64, error) { return nil, 0, nil } + +func (f *SnapshotFetcher) SetLatestProcessedBlock(to uint64) { + return +} diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 1a275173ecde..2720aec76b1b 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -20,7 +20,7 @@ const ( DefaultFetchBlockRange = uint64(100) // DefaultPollInterval is the frequency at which we query for new L1 messages. - DefaultPollInterval = time.Second * 60 + DefaultPollInterval = time.Second * 10 // LogProgressInterval is the frequency at which we log progress. LogProgressInterval = time.Second * 10 @@ -97,7 +97,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - // s.fetchMessages() + s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -107,7 +107,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - // s.fetchMessages() + s.fetchMessages() select { case <-s.ctx.Done(): From 5f112871c8131abf1ee04780c25b55c8cb429134 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Fri, 17 May 2024 17:50:40 +0200 Subject: [PATCH 07/59] refactor code to new design with pipeline --- rollup/da_syncer/batch_queue.go | 43 +++++ rollup/da_syncer/block_queue.go | 85 +++++++++ rollup/da_syncer/calldata_source.go | 206 ++++++++++++++++++++ rollup/da_syncer/da.go | 72 ++++--- rollup/da_syncer/da_queue.go | 52 ++++++ rollup/da_syncer/da_syncer.go | 204 +++----------------- rollup/da_syncer/data_source.go | 46 +++++ rollup/da_syncer/l1_client.go | 51 +++-- rollup/da_syncer/l1rpc_fetcher.go | 269 --------------------------- rollup/da_syncer/syncing_pipeline.go | 139 ++++++++++++++ rollup/sync_service/sync_service.go | 4 +- 11 files changed, 679 insertions(+), 492 deletions(-) create mode 100644 rollup/da_syncer/batch_queue.go create mode 100644 rollup/da_syncer/block_queue.go create mode 100644 rollup/da_syncer/calldata_source.go create mode 100644 rollup/da_syncer/da_queue.go create mode 100644 rollup/da_syncer/data_source.go delete mode 100644 rollup/da_syncer/l1rpc_fetcher.go create mode 100644 rollup/da_syncer/syncing_pipeline.go diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go new file mode 100644 index 000000000000..c75b4b7ec3f4 --- /dev/null +++ b/rollup/da_syncer/batch_queue.go @@ -0,0 +1,43 @@ +package da_syncer + +import ( + "context" + "fmt" +) + +type BatchQueue struct { + // batches is map from batchIndex to batch blocks + batches map[uint64]DAEntry + daQueue *DaQueue +} + +func NewBatchQueue(daQueue *DaQueue) *BatchQueue { + return &BatchQueue{ + batches: make(map[uint64]DAEntry), + daQueue: daQueue, + } +} + +func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { + + for { + daEntry, err := bq.daQueue.NextDA(ctx) + if err != nil { + return nil, err + } + switch daEntry := daEntry.(type) { + case *CommitBatchDaV0: + bq.batches[daEntry.BatchIndex] = daEntry + case *RevertBatchDA: + delete(bq.batches, daEntry.BatchIndex) + case *FinalizeBatchDA: + ret, ok := bq.batches[daEntry.BatchIndex] + if !ok { + return nil, fmt.Errorf("failed to get batch data, batchIndex: %d", daEntry.BatchIndex) + } + return ret, nil + default: + return nil, fmt.Errorf("unexpected type of daEntry: %T", daEntry) + } + } +} diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go new file mode 100644 index 000000000000..4800601caafe --- /dev/null +++ b/rollup/da_syncer/block_queue.go @@ -0,0 +1,85 @@ +package da_syncer + +import ( + "context" + "fmt" + + "github.com/scroll-tech/go-ethereum/core/types" +) + +type BlockQueue struct { + batchQueue *BatchQueue + blocks []*types.Block +} + +func NewBlockQueue(batchQueue *BatchQueue) *BlockQueue { + return &BlockQueue{ + batchQueue: batchQueue, + blocks: []*types.Block{}, + } +} + +func (bq *BlockQueue) NextBlock(ctx context.Context) (*types.Block, error) { + for len(bq.blocks) == 0 { + err := bq.getBlocksFromBatch(ctx) + if err != nil { + return nil, err + } + } + block := bq.blocks[0] + bq.blocks = bq.blocks[1:] + return block, nil +} + +func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { + daEntry, err := bq.batchQueue.NextBatch(ctx) + if err != nil { + return err + } + switch daEntry := daEntry.(type) { + case *CommitBatchDaV0: + // to be implemented in codecv0 + // bq.blocks := codecv0.DecodeFromCalldata(daEntry) + default: + return fmt.Errorf("unexpected type of daEntry: %T", daEntry) + } + return nil +} +/* + +func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { + var blocks []*types.Block + l1TxIndex := 0 + for _, chunk := range daEntry.Chunks { + l2TxIndex := 0 + for _, blockContext := range chunk.BlockContexts { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), + Time: blockContext.Timestamp, + BaseFee: blockContext.BaseFee, + GasLimit: blockContext.GasLimit, + } + // create txs + // var txs types.Transactions + txs := make(types.Transactions, 0, blockContext.NumTransactions) + // insert l1 msgs + for id := 0; id < int(blockContext.NumL1Messages); id++ { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) + txs = append(txs, l1Tx) + l1TxIndex++ + } + // insert l2 txs + for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { + l2Tx := &types.Transaction{} + l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) + txs = append(txs, l2Tx) + l2TxIndex++ + } + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil +} +*/ \ No newline at end of file diff --git a/rollup/da_syncer/calldata_source.go b/rollup/da_syncer/calldata_source.go new file mode 100644 index 000000000000..a1569a4afa1f --- /dev/null +++ b/rollup/da_syncer/calldata_source.go @@ -0,0 +1,206 @@ +package da_syncer + +import ( + "context" + "encoding/binary" + "fmt" + "math/big" + + "github.com/scroll-tech/go-ethereum/accounts/abi" + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" +) + +var ( + callDataSourceFetchBlockRange uint64 = 100 +) + +type CalldataSource struct { + ctx context.Context + l1Client *L1Client + l1height uint64 + maxL1Height uint64 + scrollChainABI *abi.ABI + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash +} + +func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client) (DataSource, error) { + scrollChainABI, err := scrollChainMetaData.GetAbi() + if err != nil { + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + return &CalldataSource{ + ctx: ctx, + l1Client: l1Client, + l1height: l1height, + maxL1Height: maxL1Height, + scrollChainABI: scrollChainABI, + l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, + l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + }, nil +} + +func (ds *CalldataSource) NextData() (DA, error) { + to := ds.l1height + callDataSourceFetchBlockRange + if to > ds.maxL1Height { + to = ds.maxL1Height + } + if ds.l1height > to { + return nil, sourceExhaustedErr + } + logs, err := ds.l1Client.fetchRollupEventsInRange(ds.ctx, ds.l1height, to) + if err != nil { + return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) + } + return ds.processLogsToDA(logs) +} + +func (ds *CalldataSource) L1Height() uint64 { + return ds.l1height +} + +func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { + var da DA + for _, vLog := range logs { + switch vLog.Topics[0] { + case ds.l1CommitBatchEventSignature: + event := &L1CommitBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new CommitBatch event", "batch index", batchIndex) + + daEntry, err := ds.getCommitBatchDa(batchIndex, &vLog) + if err != nil { + return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) + } + da = append(da, daEntry) + + case ds.l1RevertBatchEventSignature: + event := &L1RevertBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new RevertBatch event", "batch index", batchIndex) + da = append(da, NewRevertBatchDA(batchIndex)) + + case ds.l1FinalizeBatchEventSignature: + event := &L1FinalizeBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new FinalizeBatch event", "batch index", batchIndex) + + da = append(da, NewFinalizeBatchDA(batchIndex)) + + default: + return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) + } + } + return da, nil +} + +func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { + var chunks Chunks + if batchIndex == 0 { + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks), nil + } + + txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) + if err != nil { + return nil, err + } + const methodIDLength = 4 + if len(txData) < methodIDLength { + return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) + } + + method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) + if err != nil { + return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) + } + + values, err := method.Inputs.Unpack(txData[methodIDLength:]) + if err != nil { + return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) + } + + type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte + } + var args commitBatchArgs + err = method.Inputs.Copy(&args, values) + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + + // todo: use codecv0 chunks + chunks, err = decodeChunks(args.Chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + parentBatchHeader, err := codecv0.NewDABatchFromBytes(args.ParentBatchHeader) + if err != nil { + return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) + } + da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks) + return da, nil + + // parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) + // totalL1MessagePopped := countTotalL1MessagePopped(chunks) + // skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + // if err != nil { + // return nil, nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + // } + // // get all necessary l1msgs without skipped + // currentIndex := parentTotalL1MessagePopped + // for index := 0; index < int(totalL1MessagePopped); index++ { + // for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + // currentIndex++ + // } + // l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + // if l1Tx == nil { + // return nil, nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + // } + // l1Txs = append(l1Txs, l1Tx) + // currentIndex++ + // } + // return chunks, l1Txs, nil +} + +func getBatchTotalL1MessagePopped(batchHeader []byte) uint64 { + return binary.BigEndian.Uint64(batchHeader[17:25]) +} + +func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped uint64) ([]*big.Int, error) { + length := len(skippedL1MessageBitmap) + if length%32 != 0 { + return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) + } + if length*8 < int(totalL1MessagePopped) { + return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) + } + var skippedBitmap []*big.Int + for index := 0; index < length/32; index++ { + bitmap := big.NewInt(0).SetBytes(skippedL1MessageBitmap[index*32 : index*32+32]) + skippedBitmap = append(skippedBitmap, bitmap) + } + return skippedBitmap, nil +} + +func isL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { + quo := index / 256 + rem := index % 256 + return skippedBitmap[quo].Bit(int(rem)) != 0 +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index b7c0b6824b7c..d2b389e5e2ce 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -1,50 +1,78 @@ package da_syncer -import "github.com/scroll-tech/go-ethereum/core/types" +import ( + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" +) type DAType int const ( // CommitBatch contains data of event of CommitBatch - CommitBatch DAType = iota + CommitBatchV0 DAType = iota // RevertBatch contains data of event of RevertBatch RevertBatch // FinalizeBatch contains data of event of FinalizeBatch FinalizeBatch ) -type DAEntry struct { - // DaType is a type of DA entry (CommitBatch, RevertBatch, FinalizeBatch) - DaType DAType - // BatchIndex index of batch - BatchIndex uint64 - // Chunks contains chunk of a batch - Chunks Chunks - // L1Txs contains l1txs of a batch - L1Txs []*types.L1MessageTx +type DAEntry interface { + DAType() DAType } -type DA []*DAEntry +type DA []DAEntry -func NewCommitBatchDA(batchIndex uint64, chunks Chunks, l1txs []*types.L1MessageTx) *DAEntry { - return &DAEntry{ - DaType: CommitBatch, - BatchIndex: batchIndex, - Chunks: chunks, - L1Txs: l1txs, +type CommitBatchDaV0 struct { + DaType DAType + Version uint8 + BatchIndex uint64 + ParentBatchHeader *codecv0.DABatch + SkippedL1MessageBitmap []byte + Chunks Chunks +} + +func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks Chunks) DAEntry { + return &CommitBatchDaV0{ + DaType: CommitBatchV0, + Version: version, + BatchIndex: batchIndex, + ParentBatchHeader: parentBatchHeader, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, } } -func NewRevertBatchDA(batchIndex uint64) *DAEntry { - return &DAEntry{ +func (f *CommitBatchDaV0) DAType() DAType { + return f.DaType +} + +type RevertBatchDA struct { + DaType DAType + BatchIndex uint64 +} + +func NewRevertBatchDA(batchIndex uint64) DAEntry { + return &FinalizeBatchDA{ DaType: RevertBatch, BatchIndex: batchIndex, } } -func NewFinalizeBatchDA(batchIndex uint64) *DAEntry { - return &DAEntry{ +func (f *RevertBatchDA) DAType() DAType { + return f.DaType +} + +type FinalizeBatchDA struct { + DaType DAType + BatchIndex uint64 +} + +func NewFinalizeBatchDA(batchIndex uint64) DAEntry { + return &FinalizeBatchDA{ DaType: FinalizeBatch, BatchIndex: batchIndex, } } + +func (f *FinalizeBatchDA) DAType() DAType { + return f.DaType +} diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go new file mode 100644 index 000000000000..bd143d3837f3 --- /dev/null +++ b/rollup/da_syncer/da_queue.go @@ -0,0 +1,52 @@ +package da_syncer + +import "context" + +type DaQueue struct { + l1height uint64 + dataSourceFactory *DataSourceFactory + dataSource DataSource + da DA +} + +func NewDaQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DaQueue { + return &DaQueue{ + l1height: l1height, + dataSourceFactory: dataSourceFactory, + dataSource: nil, + da: []DAEntry{}, + } +} + +func (dq *DaQueue) NextDA(ctx context.Context) (DAEntry, error) { + if len(dq.da) == 0 { + err := dq.getNextData(ctx) + if err != nil { + return nil, err + } + } + daEntry := dq.da[0] + dq.da = dq.da[1:] + return daEntry, nil +} + +func (dq *DaQueue) getNextData(ctx context.Context) error { + var err error + if dq.dataSource == nil { + dq.dataSource, err = dq.dataSourceFactory.OpenDataSource(ctx, dq.l1height) + if err != nil { + return err + } + } + dq.da, err = dq.dataSource.NextData() + // previous dataSource has been exhausted, create new + if err == sourceExhaustedErr { + dq.l1height = dq.dataSource.L1Height() + dq.dataSource = nil + return dq.getNextData(ctx) + } + if err != nil { + return err + } + return nil +} diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 5daff9b804a1..78f84063f295 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -1,212 +1,50 @@ package da_syncer import ( - "context" - "errors" "fmt" - "math/big" - "time" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core" - "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/params" - "github.com/scroll-tech/go-ethereum/rollup/sync_service" "github.com/scroll-tech/go-ethereum/trie" ) -// Config is the configuration parameters of da syncer. -type Config struct { - FetcherMode FetcherMode // mode of fetcher - SnapshotFilePath string // path to snapshot file -} - -var ( - errInvalidChain = errors.New("retrieved hash chain is invalid") -) - -// defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 45 * time.Second - -// defaultFetchBlockRange is number of L1 blocks that is loaded by fetcher in one request. -const defaultFetchBlockRange = 1000 - type DaSyncer struct { - DaFetcher DaFetcher - ctx context.Context - cancel context.CancelFunc - db ethdb.Database blockchain *core.BlockChain - // batches is map from batchIndex to batch blocks - batches map[uint64][]*types.Block } -func NewDaSyncer(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, l1Client sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*DaSyncer, error) { - ctx, cancel := context.WithCancel(ctx) - var daFetcher DaFetcher - var err error - if config.FetcherMode == L1RPC { - daFetcher, err = newL1RpcDaFetcher(ctx, genesisConfig, l1Client, db, l1DeploymentBlock, defaultFetchBlockRange) - if err != nil { - cancel() - return nil, err - } - } else { - daFetcher, err = newSnapshotFetcher(defaultFetchBlockRange) - if err != nil { - cancel() - return nil, err - } - } - daSyncer := DaSyncer{ - DaFetcher: daFetcher, - ctx: ctx, - cancel: cancel, - db: db, +func NewDaSyncer(blockchain *core.BlockChain) *DaSyncer { + return &DaSyncer{ blockchain: blockchain, - batches: make(map[uint64][]*types.Block), } - return &daSyncer, nil } -func (s *DaSyncer) Start() { - if s == nil { - return - } - - log.Info("Starting DaSyncer") - - go func() { - syncTicker := time.NewTicker(defaultSyncInterval) - defer syncTicker.Stop() - - for { - s.syncWithDa() - select { - case <-s.ctx.Done(): - return - case <-syncTicker.C: - continue - } - } - }() -} - -func (s *DaSyncer) Stop() { - if s == nil { - return - } - - log.Info("Stopping DaSyncer") +func (s *DaSyncer) SyncOneBlock(block *types.Block) error { + prevHash := s.blockchain.CurrentBlock().Hash() + log.Info("now", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) - if s.cancel != nil { - s.cancel() - } -} + header := block.Header() + txs := block.Transactions() -func (s *DaSyncer) syncWithDa() { - log.Info("DaSyncer syncing") - da, to, err := s.DaFetcher.FetchDA() + // fill header with all necessary fields + var err error + header.ReceiptHash, header.Bloom, header.Root, header.GasUsed, err = s.blockchain.PreprocessBlock(block) if err != nil { - log.Error("failed to fetch DA", "err", err) - return - } - for _, daEntry := range da { - switch daEntry.DaType { - case CommitBatch: - blocks, err := s.processDaToBlocks(daEntry) - if err != nil { - log.Warn("failed to process DA to blocks", "err", err) - return - } - log.Debug("commit batch", "batchindex", daEntry.BatchIndex) - s.batches[daEntry.BatchIndex] = blocks - case RevertBatch: - log.Debug("revert batch", "batchindex", daEntry.BatchIndex) - delete(s.batches, daEntry.BatchIndex) - case FinalizeBatch: - log.Debug("finalize batch", "batchindex", daEntry.BatchIndex) - blocks, ok := s.batches[daEntry.BatchIndex] - if !ok { - log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) - return - } - s.insertBlocks(blocks) - } - } - rawdb.WriteDASyncedL1BlockNumber(s.db, to) - s.DaFetcher.SetLatestProcessedBlock(to) -} - -func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { - var blocks []*types.Block - l1TxIndex := 0 - for _, chunk := range daEntry.Chunks { - l2TxIndex := 0 - for _, blockContext := range chunk.BlockContexts { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), - Time: blockContext.Timestamp, - BaseFee: blockContext.BaseFee, - GasLimit: blockContext.GasLimit, - } - // create txs - // var txs types.Transactions - txs := make(types.Transactions, 0, blockContext.NumTransactions) - // insert l1 msgs - for id := 0; id < int(blockContext.NumL1Messages); id++ { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) - txs = append(txs, l1Tx) - l1TxIndex++ - } - // insert l2 txs - for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { - l2Tx := &types.Transaction{} - l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) - txs = append(txs, l2Tx) - l2TxIndex++ - } - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) - blocks = append(blocks, block) - } + return fmt.Errorf("block preprocessing failed, block number: %d, error: %v", block.Number(), err) } - return blocks, nil -} + header.UncleHash = common.HexToHash("0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347") + header.Difficulty = common.Big1 + header.BaseFee = nil + header.TxHash = types.DeriveSha(txs, trie.NewStackTrie(nil)) + header.ParentHash = prevHash -func (s *DaSyncer) insertBlocks(blocks []*types.Block) error { - prevHash := s.blockchain.CurrentBlock().Hash() - for _, block := range blocks { - header := block.Header() - txs := block.Transactions() + fullBlock := types.NewBlockWithHeader(header).WithBody(txs, make([]*types.Header, 0)) - // fill header with all necessary fields - var err error - header.ReceiptHash, header.Bloom, header.Root, header.GasUsed, err = s.blockchain.PreprocessBlock(block) - if err != nil { - return fmt.Errorf("block preprocessing failed, block number: %d, error: %v", block.Number(), err) - } - header.UncleHash = common.HexToHash("0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347") - header.Difficulty = common.Big1 - header.BaseFee = nil - header.TxHash = types.DeriveSha(txs, trie.NewStackTrie(nil)) - header.ParentHash = prevHash - - fullBlock := types.NewBlockWithHeader(header).WithBody(txs, make([]*types.Header, 0)) - - if index, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { - if index < len(blocks) { - log.Debug("Block insert failed", "number", blocks[index].Header().Number, "hash", blocks[index].Header().Hash(), "err", err) - } - return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) - } - prevHash = fullBlock.Hash() - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { + return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) } + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) - log.Info("insertblocks completed", "blockchain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) return nil -} \ No newline at end of file +} diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go new file mode 100644 index 000000000000..f9c555573a85 --- /dev/null +++ b/rollup/da_syncer/data_source.go @@ -0,0 +1,46 @@ +package da_syncer + +import ( + "context" + "errors" + "math/big" + + "github.com/scroll-tech/go-ethereum/core" + "github.com/scroll-tech/go-ethereum/params" +) + +var ( + sourceExhaustedErr = errors.New("data source has been exhausted") +) + +type DataSource interface { + NextData() (DA, error) + L1Height() uint64 +} + +type DataSourceFactory struct { + config Config + genesisConfig *params.ChainConfig + l1Client *L1Client +} + +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client) *DataSourceFactory { + return &DataSourceFactory{ + config: config, + genesisConfig: genesisConfig, + l1Client: l1Client, + } +} + +func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { + if ds.config.FetcherMode == L1RPC { + if ds.genesisConfig.IsBernoulli(big.NewInt(0).SetUint64(l1height)) { + return nil, errors.New("blob_data_source: not implemented") + } else { + var maxL1Height uint64 = ds.genesisConfig.BernoulliBlock.Uint64() + return NewCalldataSource(ctx, l1height, maxL1Height, ds.l1Client) + } + } else { + return nil, errors.New("snapshot_data_source: not implemented") + } +} diff --git a/rollup/da_syncer/l1_client.go b/rollup/da_syncer/l1_client.go index a7f4616ef46a..3ed691be5323 100644 --- a/rollup/da_syncer/l1_client.go +++ b/rollup/da_syncer/l1_client.go @@ -7,11 +7,10 @@ import ( "math/big" "github.com/scroll-tech/go-ethereum" - "github.com/scroll-tech/go-ethereum/accounts/abi" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" - "github.com/scroll-tech/go-ethereum/rpc" + "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) @@ -19,7 +18,6 @@ import ( // L1Client is a wrapper around EthClient that adds // methods for conveniently collecting rollup events of ScrollChain contract. type L1Client struct { - ctx context.Context client sync_service.EthClient scrollChainAddress common.Address l1CommitBatchEventSignature common.Hash @@ -29,7 +27,14 @@ type L1Client struct { // newL1Client initializes a new L1Client instance with the provided configuration. // It checks for a valid scrollChainAddress and verifies the chain ID. -func newL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId uint64, scrollChainAddress common.Address, scrollChainABI *abi.ABI) (*L1Client, error) { +func newL1Client(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient) (*L1Client, error) { + + scrollChainABI, err := scrollChainMetaData.GetAbi() + if err != nil { + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + + scrollChainAddress := genesisConfig.Scroll.L1Config.ScrollChainAddress if scrollChainAddress == (common.Address{}) { return nil, errors.New("must pass non-zero scrollChainAddress to L1Client") } @@ -39,12 +44,11 @@ func newL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId if err != nil { return nil, fmt.Errorf("failed to query L1 chain ID, err: %w", err) } - if got.Cmp(big.NewInt(0).SetUint64(l1ChainId)) != 0 { - return nil, fmt.Errorf("unexpected chain ID, expected: %v, got: %v", l1ChainId, got) + if got.Cmp(big.NewInt(0).SetUint64(genesisConfig.Scroll.L1Config.L1ChainId)) != 0 { + return nil, fmt.Errorf("unexpected chain ID, expected: %v, got: %v", genesisConfig.Scroll.L1Config.L1ChainId, got) } client := L1Client{ - ctx: ctx, client: l1Client, scrollChainAddress: scrollChainAddress, l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, @@ -72,21 +76,36 @@ func (c *L1Client) fetchRollupEventsInRange(ctx context.Context, from, to uint64 query.Topics[0][1] = c.l1RevertBatchEventSignature query.Topics[0][2] = c.l1FinalizeBatchEventSignature - logs, err := c.client.FilterLogs(c.ctx, query) + logs, err := c.client.FilterLogs(ctx, query) if err != nil { return nil, fmt.Errorf("failed to filter logs, err: %w", err) } return logs, nil } -// getLatestFinalizedBlockNumber fetches the block number of the latest finalized block from the L1 chain. -func (c *L1Client) getLatestFinalizedBlockNumber(ctx context.Context) (uint64, error) { - header, err := c.client.HeaderByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) +// fetchTxData fetches tx data corresponding to given event log +func (c *L1Client) fetchTxData(ctx context.Context, vLog *types.Log) ([]byte, error) { + tx, _, err := c.client.TransactionByHash(ctx, vLog.TxHash) if err != nil { - return 0, err - } - if !header.Number.IsInt64() { - return 0, fmt.Errorf("received unexpected block number in L1Client: %v", header.Number) + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := c.client.BlockByHash(ctx, vLog.BlockHash) + if err != nil { + return nil, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return nil, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } } - return header.Number.Uint64(), nil + + return tx.Data(), nil } diff --git a/rollup/da_syncer/l1rpc_fetcher.go b/rollup/da_syncer/l1rpc_fetcher.go deleted file mode 100644 index 53aec0b8539f..000000000000 --- a/rollup/da_syncer/l1rpc_fetcher.go +++ /dev/null @@ -1,269 +0,0 @@ -package da_syncer - -import ( - "context" - "encoding/binary" - "fmt" - "math/big" - "reflect" - - "github.com/scroll-tech/go-ethereum/accounts/abi" - "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/params" - "github.com/scroll-tech/go-ethereum/rollup/sync_service" -) - -type L1RPCFetcher struct { - fetchBlockRange uint64 - client *L1Client - db ethdb.Database - ctx context.Context - latestProcessedBlock uint64 - scrollChainABI *abi.ABI - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash -} - -func newL1RpcDaFetcher(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient, db ethdb.Database, l1DeploymentBlock, fetchBlockRange uint64) (DaFetcher, error) { - // terminate if the caller does not provide an L1 client (e.g. in tests) - if l1Client == nil || (reflect.ValueOf(l1Client).Kind() == reflect.Ptr && reflect.ValueOf(l1Client).IsNil()) { - log.Warn("No L1 client provided, L1 rollup sync service will not run") - return nil, nil - } - - if genesisConfig.Scroll.L1Config == nil { - return nil, fmt.Errorf("missing L1 config in genesis") - } - - scrollChainABI, err := scrollChainMetaData.GetAbi() - if err != nil { - return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) - } - - client, err := newL1Client(ctx, l1Client, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) - if err != nil { - return nil, fmt.Errorf("failed to initialize l1 client: %w", err) - } - - // Initialize the latestProcessedBlock with the block just before the L1 deployment block. - // This serves as a default value when there's no L1 rollup events synced in the database. - var latestProcessedBlock uint64 - if l1DeploymentBlock > 0 { - latestProcessedBlock = l1DeploymentBlock - 1 - } - block := rawdb.ReadDASyncedL1BlockNumber(db) - if block != nil { - // restart from latest synced block number - latestProcessedBlock = *block - } - - daFetcher := L1RPCFetcher{ - fetchBlockRange: fetchBlockRange, - ctx: ctx, - client: client, - db: db, - latestProcessedBlock: latestProcessedBlock, - scrollChainABI: scrollChainABI, - l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, - l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, - } - return &daFetcher, nil -} - -func (f *L1RPCFetcher) SetLatestProcessedBlock(to uint64) { - f.latestProcessedBlock = to -} - -// Fetch DA fetches all da events and converts it to DA format in some fetchBlockRange -func (f *L1RPCFetcher) FetchDA() (DA, uint64, error) { - latestConfirmed, err := f.client.getLatestFinalizedBlockNumber(f.ctx) - if err != nil { - log.Warn("failed to get latest confirmed block number", "err", err) - return nil, 0, err - } - - log.Trace("Da fetcher fetch rollup events", "latest processed block", f.latestProcessedBlock, "latest confirmed", latestConfirmed) - - from := f.latestProcessedBlock + 1 - to := f.latestProcessedBlock + f.fetchBlockRange - if to > latestConfirmed { - to = latestConfirmed - } - - logs, err := f.client.fetchRollupEventsInRange(f.ctx, from, to) - if err != nil { - log.Error("failed to fetch rollup events in range", "from block", from, "to block", to, "err", err) - return nil, 0, err - } - da, err := f.processLogsToDA(logs) - if err != nil { - log.Error("failed to process rollup events in range", "from block", from, "to block", to, "err", err) - return nil, 0, err - } - log.Trace("L1Rpcfetcher fetched and processed logs to da", "from", from, "to", to, "da len", len(da)) - return da, to, nil -} - -func (f *L1RPCFetcher) processLogsToDA(logs []types.Log) (DA, error) { - var da DA - for _, vLog := range logs { - switch vLog.Topics[0] { - case f.l1CommitBatchEventSignature: - event := &L1CommitBatchEvent{} - if err := UnpackLog(f.scrollChainABI, event, "CommitBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new CommitBatch event", "batch index", batchIndex) - - chunks, l1Txs, err := f.getBatch(batchIndex, &vLog) - if err != nil { - return nil, fmt.Errorf("failed to get chunks, batch index: %v, err: %w", batchIndex, err) - } - da = append(da, NewCommitBatchDA(batchIndex, chunks, l1Txs)) - - case f.l1RevertBatchEventSignature: - event := &L1RevertBatchEvent{} - if err := UnpackLog(f.scrollChainABI, event, "RevertBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new RevertBatch event", "batch index", batchIndex) - da = append(da, NewRevertBatchDA(batchIndex)) - - case f.l1FinalizeBatchEventSignature: - event := &L1FinalizeBatchEvent{} - if err := UnpackLog(f.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new FinalizeBatch event", "batch index", batchIndex) - - da = append(da, NewFinalizeBatchDA(batchIndex)) - - default: - return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) - } - } - return da, nil -} - -func (f *L1RPCFetcher) getBatch(batchIndex uint64, vLog *types.Log) (Chunks, []*types.L1MessageTx, error) { - var chunks Chunks - var l1Txs []*types.L1MessageTx - if batchIndex == 0 { - return chunks, l1Txs, nil - } - - tx, _, err := f.client.client.TransactionByHash(f.ctx, vLog.TxHash) - if err != nil { - log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", - "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) - block, err := f.client.client.BlockByHash(f.ctx, vLog.BlockHash) - if err != nil { - return nil, nil, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) - } - - found := false - for _, txInBlock := range block.Transactions() { - if txInBlock.Hash() == vLog.TxHash { - tx = txInBlock - found = true - break - } - } - if !found { - return nil, nil, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) - } - } - - txData := tx.Data() - const methodIDLength = 4 - if len(txData) < methodIDLength { - return nil, nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) - } - - method, err := f.scrollChainABI.MethodById(txData[:methodIDLength]) - if err != nil { - return nil, nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) - } - - values, err := method.Inputs.Unpack(txData[methodIDLength:]) - if err != nil { - return nil, nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) - } - - type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - } - var args commitBatchArgs - err = method.Inputs.Copy(&args, values) - if err != nil { - return nil, nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - - chunks, err = decodeChunks(args.Chunks) - if err != nil { - return nil, nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - - parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) - totalL1MessagePopped := countTotalL1MessagePopped(chunks) - skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < int(totalL1MessagePopped); index++ { - for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - } - l1Tx := rawdb.ReadL1Message(f.db, currentIndex) - if l1Tx == nil { - return nil, nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - return chunks, l1Txs, nil - -} - -func getBatchTotalL1MessagePopped(batchHeader []byte) uint64 { - return binary.BigEndian.Uint64(batchHeader[17:25]) -} - -func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped uint64) ([]*big.Int, error) { - length := len(skippedL1MessageBitmap) - if length%32 != 0 { - return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) - } - if length*8 < int(totalL1MessagePopped) { - return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) - } - var skippedBitmap []*big.Int - for index := 0; index < length/32; index++ { - bitmap := big.NewInt(0).SetBytes(skippedL1MessageBitmap[index*32 : index*32+32]) - skippedBitmap = append(skippedBitmap, bitmap) - } - return skippedBitmap, nil -} - -func isL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { - quo := index / 256 - rem := index % 256 - return skippedBitmap[quo].Bit(int(rem)) != 0 -} - -// l2geth --scroll --datadir "./l2geth-datadir" --gcmode archive --cache.noprefetch --http --http.addr "0.0.0.0" --http.port 8545 --http.api "eth,net,web3,debug,scroll" --l1.endpoint "$L2GETH_L1_ENDPOINT" --da.sync diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go new file mode 100644 index 000000000000..2cb1b5c1cd06 --- /dev/null +++ b/rollup/da_syncer/syncing_pipeline.go @@ -0,0 +1,139 @@ +package da_syncer + +import ( + "context" + "errors" + "time" + + "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/sync_service" +) + +// Config is the configuration parameters of data availability syncing. +type Config struct { + FetcherMode FetcherMode // mode of fetcher + SnapshotFilePath string // path to snapshot file +} + +var ( + errInvalidChain = errors.New("retrieved hash chain is invalid") +) + +// defaultSyncInterval is the frequency at which we query for new rollup event. +const defaultSyncInterval = 45 * time.Second + +type SyncingPipeline struct { + db ethdb.Database + blockchain *core.BlockChain + blockQueue *BlockQueue + daSyncer *DaSyncer +} + +func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, ethClient sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*SyncingPipeline, error) { + var err error + + l1Client, err := newL1Client(ctx, genesisConfig, ethClient) + if err != nil { + return nil, err + } + + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client) + // todo: keep synced l1 height somewhere + var syncedL1Height uint64 = 0 + daQueue := NewDaQueue(syncedL1Height, dataSourceFactory) + batchQueue := NewBatchQueue(daQueue) + blockQueue := NewBlockQueue(batchQueue) + daSyncer := NewDaSyncer(blockchain) + + return &SyncingPipeline{ + db: db, + blockchain: blockchain, + blockQueue: blockQueue, + daSyncer: daSyncer, + }, nil +} + +func (sp *SyncingPipeline) Step(ctx context.Context) error { + block, err := sp.blockQueue.NextBlock(ctx) + if err != nil { + return err + } + err = sp.daSyncer.SyncOneBlock(block) + return err +} + +// func (s *DaSyncer) Start() { +// if s == nil { +// return +// } + +// log.Info("Starting DaSyncer") + +// go func() { +// syncTicker := time.NewTicker(defaultSyncInterval) +// defer syncTicker.Stop() + +// for { +// s.syncWithDa() +// select { +// case <-s.ctx.Done(): +// return +// case <-syncTicker.C: +// continue +// } +// } +// }() +// } + +// func (s *DaSyncer) Stop() { +// if s == nil { +// return +// } + +// log.Info("Stopping DaSyncer") + +// if s.cancel != nil { +// s.cancel() +// } +// } + +// func (s *DaSyncer) syncWithDa() { +// log.Info("DaSyncer syncing") +// da, to, err := s.DaFetcher.FetchDA() +// if err != nil { +// log.Error("failed to fetch DA", "err", err) +// return +// } +// for _, daEntry := range da { +// switch daEntry.DaType { +// case CommitBatch: +// blocks, err := s.processDaToBlocks(daEntry) +// if err != nil { +// log.Warn("failed to process DA to blocks", "err", err) +// return +// } +// log.Debug("commit batch", "batchindex", daEntry.BatchIndex) +// s.batches[daEntry.BatchIndex] = blocks +// case RevertBatch: +// log.Debug("revert batch", "batchindex", daEntry.BatchIndex) +// delete(s.batches, daEntry.BatchIndex) +// case FinalizeBatch: +// log.Debug("finalize batch", "batchindex", daEntry.BatchIndex) +// blocks, ok := s.batches[daEntry.BatchIndex] +// if !ok { +// log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) +// return +// } +// err := s.insertBlocks(blocks) +// if err != nil { +// log.Warn("cannot insert blocks for batch", "batch index", daEntry.BatchIndex, "err", err) +// return +// } +// } +// } +// rawdb.WriteDASyncedL1BlockNumber(s.db, to) +// s.DaFetcher.SetLatestProcessedBlock(to) +// log.Info("DaSyncer synced") +// } diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 091f2d19691f..8138abd5a1b8 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -102,7 +102,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - s.fetchMessages() + // s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -112,7 +112,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - s.fetchMessages() + // s.fetchMessages() select { case <-s.ctx.Done(): From 6b9e837e390d4d8c83bd8ac600caacc2e3b34222 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 20 May 2024 17:13:39 +0200 Subject: [PATCH 08/59] adapt for codecv0 types --- eth/backend.go | 8 +- rollup/da_syncer/block_context.go | 40 ------- rollup/da_syncer/block_queue.go | 33 +++--- rollup/da_syncer/calldata_source.go | 66 +++++++----- rollup/da_syncer/chunk.go | 72 ------------- rollup/da_syncer/da.go | 7 +- rollup/da_syncer/da_queue.go | 2 +- rollup/da_syncer/da_syncer.go | 4 + rollup/da_syncer/data_source.go | 17 ++- rollup/da_syncer/syncing_pipeline.go | 128 +++++++++-------------- rollup/types/encoding/codecv0/codecv0.go | 63 ++++++++++- 11 files changed, 192 insertions(+), 248 deletions(-) delete mode 100644 rollup/da_syncer/block_context.go delete mode 100644 rollup/da_syncer/chunk.go diff --git a/eth/backend.go b/eth/backend.go index ab5d1faa739c..660c67af0b77 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -74,7 +74,7 @@ type Ethereum struct { txPool *core.TxPool syncService *sync_service.SyncService rollupSyncService *rollup_sync_service.RollupSyncService - daSyncer *da_syncer.DaSyncer + syncingPipeline *da_syncer.SyncingPipeline blockchain *core.BlockChain handler *handler ethDialCandidates enode.Iterator @@ -236,11 +236,11 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl } if config.EnableDASyncing { - eth.daSyncer, err = da_syncer.NewDaSyncer(context.Background(), eth.blockchain, chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) + eth.syncingPipeline, err = da_syncer.NewSyncingPipeline(context.Background(), eth.blockchain, chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) if err != nil { return nil, fmt.Errorf("cannot initialize da syncer: %w", err) } - eth.daSyncer.Start() + eth.syncingPipeline.Start() } // Permit the downloader to use the trie cache allowance during fast sync @@ -605,7 +605,7 @@ func (s *Ethereum) Stop() error { s.rollupSyncService.Stop() } if s.config.EnableDASyncing { - s.daSyncer.Stop() + s.syncingPipeline.Stop() } s.miner.Close() s.blockchain.Stop() diff --git a/rollup/da_syncer/block_context.go b/rollup/da_syncer/block_context.go deleted file mode 100644 index 9f486b25033e..000000000000 --- a/rollup/da_syncer/block_context.go +++ /dev/null @@ -1,40 +0,0 @@ -package da_syncer - -import ( - "encoding/binary" - "errors" - "math/big" - - "github.com/scroll-tech/go-ethereum/common" -) - -// BlockContext represents the essential data of a block in the ScrollChain. -// It provides an overview of block attributes including hash values, block numbers, gas details, and transaction counts. -type BlockContext struct { - BlockNumber uint64 - Timestamp uint64 - BaseFee *big.Int - GasLimit uint64 - NumTransactions uint16 - NumL1Messages uint16 -} - -type BlockContexts []*BlockContext - -func decodeBlockContext(encodedBlockContext []byte) (*BlockContext, error) { - if len(encodedBlockContext) != blockContextByteSize { - return nil, errors.New("block encoding is not 60 bytes long") - } - baseFee := big.NewInt(0).SetBytes(encodedBlockContext[16:48]) - if baseFee.Cmp(common.Big0) == 0 { - baseFee = nil - } - return &BlockContext{ - BlockNumber: binary.BigEndian.Uint64(encodedBlockContext[0:8]), - Timestamp: binary.BigEndian.Uint64(encodedBlockContext[8:16]), - BaseFee: baseFee, - GasLimit: binary.BigEndian.Uint64(encodedBlockContext[48:56]), - NumTransactions: binary.BigEndian.Uint16(encodedBlockContext[56:58]), - NumL1Messages: binary.BigEndian.Uint16(encodedBlockContext[58:60]), - }, nil -} diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 4800601caafe..ca128446d71e 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -3,6 +3,7 @@ package da_syncer import ( "context" "fmt" + "math/big" "github.com/scroll-tech/go-ethereum/core/types" ) @@ -38,48 +39,42 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { } switch daEntry := daEntry.(type) { case *CommitBatchDaV0: - // to be implemented in codecv0 - // bq.blocks := codecv0.DecodeFromCalldata(daEntry) + bq.blocks, err = bq.processDaV0ToBlocks(daEntry) + if err != nil { + return err + } default: return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } return nil } -/* -func (s *DaSyncer) processDaToBlocks(daEntry *DAEntry) ([]*types.Block, error) { +func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Block, error) { var blocks []*types.Block l1TxIndex := 0 for _, chunk := range daEntry.Chunks { - l2TxIndex := 0 - for _, blockContext := range chunk.BlockContexts { + for blockId, daBlock := range chunk.Blocks { // create header header := types.Header{ - Number: big.NewInt(0).SetUint64(blockContext.BlockNumber), - Time: blockContext.Timestamp, - BaseFee: blockContext.BaseFee, - GasLimit: blockContext.GasLimit, + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, } // create txs // var txs types.Transactions - txs := make(types.Transactions, 0, blockContext.NumTransactions) + txs := make(types.Transactions, 0, daBlock.NumTransactions) // insert l1 msgs - for id := 0; id < int(blockContext.NumL1Messages); id++ { + for id := 0; id < int(daBlock.NumL1Messages); id++ { l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) txs = append(txs, l1Tx) l1TxIndex++ } // insert l2 txs - for id := int(blockContext.NumL1Messages); id < int(blockContext.NumTransactions); id++ { - l2Tx := &types.Transaction{} - l2Tx.UnmarshalBinary(chunk.L2Txs[l2TxIndex]) - txs = append(txs, l2Tx) - l2TxIndex++ - } + txs = append(txs, chunk.Transactions[blockId]...) block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) blocks = append(blocks, block) } } return blocks, nil } -*/ \ No newline at end of file diff --git a/rollup/da_syncer/calldata_source.go b/rollup/da_syncer/calldata_source.go index a1569a4afa1f..48b93021ead4 100644 --- a/rollup/da_syncer/calldata_source.go +++ b/rollup/da_syncer/calldata_source.go @@ -8,7 +8,9 @@ import ( "github.com/scroll-tech/go-ethereum/accounts/abi" "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/rollup/types/encoding/codecv0" ) @@ -26,9 +28,10 @@ type CalldataSource struct { l1CommitBatchEventSignature common.Hash l1RevertBatchEventSignature common.Hash l1FinalizeBatchEventSignature common.Hash + db ethdb.Database } -func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client) (DataSource, error) { +func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, db ethdb.Database) (DataSource, error) { scrollChainABI, err := scrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) @@ -42,6 +45,7 @@ func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Clie l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + db: db, }, nil } @@ -57,6 +61,7 @@ func (ds *CalldataSource) NextData() (DA, error) { if err != nil { return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) } + ds.l1height = to + 1 return ds.processLogsToDA(logs) } @@ -109,9 +114,10 @@ func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { } func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - var chunks Chunks + var chunks []*codecv0.DAChunkRawTx + var l1Txs []*types.L1MessageTx if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks), nil + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil } txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) @@ -146,7 +152,7 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( } // todo: use codecv0 chunks - chunks, err = decodeChunks(args.Chunks) + chunks, err = codecv0.DecodeDAChunksRawTx(args.Chunks) if err != nil { return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } @@ -154,41 +160,45 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( if err != nil { return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks) - return da, nil - // parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) - // totalL1MessagePopped := countTotalL1MessagePopped(chunks) - // skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - // if err != nil { - // return nil, nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - // } - // // get all necessary l1msgs without skipped - // currentIndex := parentTotalL1MessagePopped - // for index := 0; index < int(totalL1MessagePopped); index++ { - // for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - // currentIndex++ - // } - // l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - // if l1Tx == nil { - // return nil, nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - // } - // l1Txs = append(l1Txs, l1Tx) - // currentIndex++ - // } - // return chunks, l1Txs, nil + parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + totalL1MessagePopped := 0 + for _, chunk := range chunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < int(totalL1MessagePopped); index++ { + for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + if l1Tx == nil { + return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) + currentIndex++ + } + da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + return da, nil } func getBatchTotalL1MessagePopped(batchHeader []byte) uint64 { return binary.BigEndian.Uint64(batchHeader[17:25]) } -func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped uint64) ([]*big.Int, error) { +func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped int) ([]*big.Int, error) { length := len(skippedL1MessageBitmap) if length%32 != 0 { return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) } - if length*8 < int(totalL1MessagePopped) { + if length*8 < totalL1MessagePopped { return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) } var skippedBitmap []*big.Int diff --git a/rollup/da_syncer/chunk.go b/rollup/da_syncer/chunk.go deleted file mode 100644 index cac5290b2ab0..000000000000 --- a/rollup/da_syncer/chunk.go +++ /dev/null @@ -1,72 +0,0 @@ -package da_syncer - -import ( - "encoding/binary" - "fmt" -) - -const blockContextByteSize = 60 - -type Chunk struct { - BlockContexts BlockContexts - L2Txs [][]byte -} - -type Chunks []*Chunk - -// decodeChunks decodes the provided chunks into a list of chunks. -func decodeChunks(chunksData [][]byte) (Chunks, error) { - var chunks Chunks - for _, chunk := range chunksData { - if len(chunk) < 1 { - return nil, fmt.Errorf("invalid chunk, length is less than 1") - } - - numBlocks := int(chunk[0]) - if len(chunk) < 1+numBlocks*blockContextByteSize { - return nil, fmt.Errorf("chunk size doesn't match with numBlocks, byte length of chunk: %v, expected length: %v", len(chunk), 1+numBlocks*blockContextByteSize) - } - - blockContexts := make(BlockContexts, numBlocks) - for i := 0; i < numBlocks; i++ { - startIdx := 1 + i*blockContextByteSize // add 1 to skip numBlocks byte - endIdx := startIdx + blockContextByteSize - blockContext, err := decodeBlockContext(chunk[startIdx:endIdx]) - if err != nil { - return nil, err - } - blockContexts[i] = blockContext - } - - var l2Txs [][]byte - txLen := 0 - - for currentIndex := 1 + numBlocks*blockContextByteSize; currentIndex < len(chunk); currentIndex += 4 + txLen { - if len(chunk) < currentIndex+4 { - return nil, fmt.Errorf("chunk size doesn't match, next tx size is less then 4, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4) - } - txLen = int(binary.BigEndian.Uint32(chunk[currentIndex : currentIndex+4])) - if len(chunk) < currentIndex+4+txLen { - return nil, fmt.Errorf("chunk size doesn't match with next tx length, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4+txLen) - } - txData := chunk[currentIndex+4 : currentIndex+4+txLen] - l2Txs = append(l2Txs, txData) - } - - chunks = append(chunks, &Chunk{ - BlockContexts: blockContexts, - L2Txs: l2Txs, - }) - } - return chunks, nil -} - -func countTotalL1MessagePopped(chunks Chunks) uint64 { - var total uint64 = 0 - for _, chunk := range chunks { - for _, block := range chunk.BlockContexts { - total += uint64(block.NumL1Messages) - } - } - return total -} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index d2b389e5e2ce..27c9f5a9c7fc 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -1,6 +1,7 @@ package da_syncer import ( + "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" ) @@ -27,10 +28,11 @@ type CommitBatchDaV0 struct { BatchIndex uint64 ParentBatchHeader *codecv0.DABatch SkippedL1MessageBitmap []byte - Chunks Chunks + Chunks []*codecv0.DAChunkRawTx + L1Txs []*types.L1MessageTx } -func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks Chunks) DAEntry { +func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx) DAEntry { return &CommitBatchDaV0{ DaType: CommitBatchV0, Version: version, @@ -38,6 +40,7 @@ func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *cod ParentBatchHeader: parentBatchHeader, SkippedL1MessageBitmap: skippedL1MessageBitmap, Chunks: chunks, + L1Txs: l1Txs, } } diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index bd143d3837f3..bf22c45c0da5 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -19,7 +19,7 @@ func NewDaQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DaQueue } func (dq *DaQueue) NextDA(ctx context.Context) (DAEntry, error) { - if len(dq.da) == 0 { + for len(dq.da) == 0 { err := dq.getNextData(ctx) if err != nil { return nil, err diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 78f84063f295..fe0a693f2d69 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -2,6 +2,7 @@ package da_syncer import ( "fmt" + "math/big" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core" @@ -22,6 +23,9 @@ func NewDaSyncer(blockchain *core.BlockChain) *DaSyncer { func (s *DaSyncer) SyncOneBlock(block *types.Block) error { prevHash := s.blockchain.CurrentBlock().Hash() + if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { + return fmt.Errorf("not consecutive block, number: %d", block.Number()) + } log.Info("now", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) header := block.Header() diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index f9c555573a85..182cd2ab4bad 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -3,9 +3,9 @@ package da_syncer import ( "context" "errors" - "math/big" "github.com/scroll-tech/go-ethereum/core" + "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/params" ) @@ -22,25 +22,32 @@ type DataSourceFactory struct { config Config genesisConfig *params.ChainConfig l1Client *L1Client + db ethdb.Database } -func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client) *DataSourceFactory { +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, db ethdb.Database) *DataSourceFactory { return &DataSourceFactory{ config: config, genesisConfig: genesisConfig, l1Client: l1Client, + db: db, } } func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { if ds.config.FetcherMode == L1RPC { - if ds.genesisConfig.IsBernoulli(big.NewInt(0).SetUint64(l1height)) { + if isBernoulliByL1Height(l1height) { return nil, errors.New("blob_data_source: not implemented") } else { - var maxL1Height uint64 = ds.genesisConfig.BernoulliBlock.Uint64() - return NewCalldataSource(ctx, l1height, maxL1Height, ds.l1Client) + // todo: set l1 block where l2 changes to bernoulli + var maxL1Height uint64 = 1000000000000 + return NewCalldataSource(ctx, l1height, maxL1Height, ds.l1Client, ds.db) } } else { return nil, errors.New("snapshot_data_source: not implemented") } } + +func isBernoulliByL1Height(l1height uint64) bool { + return false +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 2cb1b5c1cd06..6a301d44f0be 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -6,7 +6,9 @@ import ( "time" "github.com/scroll-tech/go-ethereum/core" + "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) @@ -22,9 +24,11 @@ var ( ) // defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 45 * time.Second +const defaultSyncInterval = 1 * time.Second type SyncingPipeline struct { + ctx context.Context + cancel context.CancelFunc db ethdb.Database blockchain *core.BlockChain blockQueue *BlockQueue @@ -32,22 +36,30 @@ type SyncingPipeline struct { } func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, ethClient sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*SyncingPipeline, error) { + ctx, cancel := context.WithCancel(ctx) var err error l1Client, err := newL1Client(ctx, genesisConfig, ethClient) if err != nil { + cancel() return nil, err } - dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client) + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, db) // todo: keep synced l1 height somewhere - var syncedL1Height uint64 = 0 + var syncedL1Height uint64 = l1DeploymentBlock - 1 + from := rawdb.ReadDASyncedL1BlockNumber(db) + if from != nil { + syncedL1Height = *from + } daQueue := NewDaQueue(syncedL1Height, dataSourceFactory) batchQueue := NewBatchQueue(daQueue) blockQueue := NewBlockQueue(batchQueue) daSyncer := NewDaSyncer(blockchain) return &SyncingPipeline{ + ctx: ctx, + cancel: cancel, db: db, blockchain: blockchain, blockQueue: blockQueue, @@ -64,76 +76,40 @@ func (sp *SyncingPipeline) Step(ctx context.Context) error { return err } -// func (s *DaSyncer) Start() { -// if s == nil { -// return -// } - -// log.Info("Starting DaSyncer") - -// go func() { -// syncTicker := time.NewTicker(defaultSyncInterval) -// defer syncTicker.Stop() - -// for { -// s.syncWithDa() -// select { -// case <-s.ctx.Done(): -// return -// case <-syncTicker.C: -// continue -// } -// } -// }() -// } - -// func (s *DaSyncer) Stop() { -// if s == nil { -// return -// } - -// log.Info("Stopping DaSyncer") - -// if s.cancel != nil { -// s.cancel() -// } -// } - -// func (s *DaSyncer) syncWithDa() { -// log.Info("DaSyncer syncing") -// da, to, err := s.DaFetcher.FetchDA() -// if err != nil { -// log.Error("failed to fetch DA", "err", err) -// return -// } -// for _, daEntry := range da { -// switch daEntry.DaType { -// case CommitBatch: -// blocks, err := s.processDaToBlocks(daEntry) -// if err != nil { -// log.Warn("failed to process DA to blocks", "err", err) -// return -// } -// log.Debug("commit batch", "batchindex", daEntry.BatchIndex) -// s.batches[daEntry.BatchIndex] = blocks -// case RevertBatch: -// log.Debug("revert batch", "batchindex", daEntry.BatchIndex) -// delete(s.batches, daEntry.BatchIndex) -// case FinalizeBatch: -// log.Debug("finalize batch", "batchindex", daEntry.BatchIndex) -// blocks, ok := s.batches[daEntry.BatchIndex] -// if !ok { -// log.Warn("cannot find blocks for batch", "batch index", daEntry.BatchIndex, "err", err) -// return -// } -// err := s.insertBlocks(blocks) -// if err != nil { -// log.Warn("cannot insert blocks for batch", "batch index", daEntry.BatchIndex, "err", err) -// return -// } -// } -// } -// rawdb.WriteDASyncedL1BlockNumber(s.db, to) -// s.DaFetcher.SetLatestProcessedBlock(to) -// log.Info("DaSyncer synced") -// } +func (sp *SyncingPipeline) Start() { + if sp == nil { + return + } + + log.Info("Starting SyncingPipeline") + + go func() { + syncTicker := time.NewTicker(defaultSyncInterval) + defer syncTicker.Stop() + + for { + err := sp.Step(sp.ctx) + if err != nil { + log.Warn("syncing pipeline step failed", "err", err) + } + select { + case <-sp.ctx.Done(): + return + case <-syncTicker.C: + continue + } + } + }() +} + +func (sp *SyncingPipeline) Stop() { + if sp == nil { + return + } + + log.Info("Stopping DaSyncer") + + if sp.cancel != nil { + sp.cancel() + } +} diff --git a/rollup/types/encoding/codecv0/codecv0.go b/rollup/types/encoding/codecv0/codecv0.go index 3df5d7ba4381..ee979be58654 100644 --- a/rollup/types/encoding/codecv0/codecv0.go +++ b/rollup/types/encoding/codecv0/codecv0.go @@ -17,6 +17,7 @@ import ( // CodecV0Version denotes the version of the codec. const CodecV0Version = 0 +const blockContextByteSize = 60 // DABlock represents a Data Availability Block. type DABlock struct { @@ -34,6 +35,12 @@ type DAChunk struct { Transactions [][]*types.TransactionData } +// DAChunk groups consecutive DABlocks with their transactions. +type DAChunkRawTx struct { + Blocks []*DABlock + Transactions []types.Transactions +} + // DABatch contains metadata about a batch of DAChunks. type DABatch struct { Version uint8 @@ -92,7 +99,7 @@ func (b *DABlock) Encode() []byte { // DecodeDABlock takes a byte slice and decodes it into a DABlock. func DecodeDABlock(bytes []byte) (*DABlock, error) { - if len(bytes) != 60 { + if len(bytes) != blockContextByteSize { return nil, errors.New("block encoding is not 60 bytes long") } @@ -174,6 +181,60 @@ func (c *DAChunk) Encode() ([]byte, error) { return chunkBytes, nil } +// DecodeDAChunksRawTx takes a byte slice and decodes it into a []DAChunkRawTx. +func DecodeDAChunksRawTx(bytes [][]byte) ([]*DAChunkRawTx, error) { + var chunks []*DAChunkRawTx + for _, chunk := range bytes { + if len(chunk) < 1 { + return nil, fmt.Errorf("invalid chunk, length is less than 1") + } + + numBlocks := int(chunk[0]) + if len(chunk) < 1+numBlocks*blockContextByteSize { + return nil, fmt.Errorf("chunk size doesn't match with numBlocks, byte length of chunk: %v, expected length: %v", len(chunk), 1+numBlocks*blockContextByteSize) + } + + blocks := make([]*DABlock, numBlocks) + for i := 0; i < numBlocks; i++ { + startIdx := 1 + i*blockContextByteSize // add 1 to skip numBlocks byte + endIdx := startIdx + blockContextByteSize + block, err := DecodeDABlock(chunk[startIdx:endIdx]) + if err != nil { + return nil, err + } + blocks[i] = block + } + + var transactions []types.Transactions + currentIndex := 1 + numBlocks*blockContextByteSize + for _, block := range blocks { + var blockTransactions types.Transactions + var txNum int = int(block.NumTransactions - block.NumL1Messages) + for i := 0; i < txNum; i++ { + if len(chunk) < currentIndex+4 { + return nil, fmt.Errorf("chunk size doesn't match, next tx size is less then 4, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4) + } + txLen := int(binary.BigEndian.Uint32(chunk[currentIndex : currentIndex+4])) + if len(chunk) < currentIndex+4+txLen { + return nil, fmt.Errorf("chunk size doesn't match with next tx length, byte length of chunk: %v, expected length: %v", len(chunk), currentIndex+4+txLen) + } + txData := chunk[currentIndex+4 : currentIndex+4+txLen] + tx := &types.Transaction{} + tx.UnmarshalBinary(txData) + blockTransactions = append(blockTransactions, tx) + currentIndex += 4 + txLen + } + transactions = append(transactions, blockTransactions) + } + + chunks = append(chunks, &DAChunkRawTx{ + Blocks: blocks, + Transactions: transactions, + }) + } + return chunks, nil +} + // Hash computes the hash of the DAChunk data. func (c *DAChunk) Hash() (common.Hash, error) { chunkBytes, err := c.Encode() From c6201c3bd5216cb1caff1469d74f0f933613cd00 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 21 May 2024 10:16:38 +0200 Subject: [PATCH 09/59] prepare for blob dta asource --- rollup/da_syncer/blob_client.go | 34 +++++ rollup/da_syncer/blob_source.go | 190 +++++++++++++++++++++++++++ rollup/da_syncer/calldata_source.go | 33 +---- rollup/da_syncer/da_fetcher.go | 7 - rollup/da_syncer/snapshot_fetcher.go | 20 --- rollup/types/encoding/bitmap.go | 24 ++++ 6 files changed, 251 insertions(+), 57 deletions(-) create mode 100644 rollup/da_syncer/blob_client.go create mode 100644 rollup/da_syncer/blob_source.go delete mode 100644 rollup/da_syncer/da_fetcher.go delete mode 100644 rollup/da_syncer/snapshot_fetcher.go diff --git a/rollup/da_syncer/blob_client.go b/rollup/da_syncer/blob_client.go new file mode 100644 index 000000000000..caef4d887d22 --- /dev/null +++ b/rollup/da_syncer/blob_client.go @@ -0,0 +1,34 @@ +package da_syncer + +import ( + "context" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" + "github.com/scroll-tech/go-ethereum/params" + + "github.com/scroll-tech/go-ethereum/rollup/sync_service" +) + +// BlobClient is a wrapper around EthClient that adds +// methods for conveniently collecting rollup events of ScrollChain contract. +type BlobClient struct { + scrollChainAddress common.Address + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash +} + +// newL1Client initializes a new L1Client instance with the provided configuration. +// It checks for a valid scrollChainAddress and verifies the chain ID. +func newBlobClient(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient) (*BlobClient, error) { + client := BlobClient{} + + return &client, nil +} + +// fetchBlob fetches blob by it's commitment +func (c *BlobClient) fetchBlob(ctx context.Context, commitment []*kzg4844.Commitment) ([]*kzg4844.Blob, error) { + // todo: + return nil, nil +} diff --git a/rollup/da_syncer/blob_source.go b/rollup/da_syncer/blob_source.go new file mode 100644 index 000000000000..8aa876f6ff49 --- /dev/null +++ b/rollup/da_syncer/blob_source.go @@ -0,0 +1,190 @@ +package da_syncer + +import ( + "context" + "fmt" + + "github.com/scroll-tech/go-ethereum/accounts/abi" + "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/rollup/types/encoding" +) + +var ( + blobSourceFetchBlockRange uint64 = 100 +) + +type BlobDataSource struct { + ctx context.Context + l1Client *L1Client + blobClient *BlobClient + l1height uint64 + maxL1Height uint64 + scrollChainABI *abi.ABI + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash + db ethdb.Database +} + +func NewBlobDataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, blobClient *BlobClient, db ethdb.Database) (DataSource, error) { + scrollChainABI, err := scrollChainMetaData.GetAbi() + if err != nil { + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + return &BlobDataSource{ + ctx: ctx, + l1Client: l1Client, + blobClient: blobClient, + l1height: l1height, + maxL1Height: maxL1Height, + scrollChainABI: scrollChainABI, + l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, + l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + db: db, + }, nil +} + +func (ds *BlobDataSource) NextData() (DA, error) { + to := ds.l1height + callDataSourceFetchBlockRange + if to > ds.maxL1Height { + to = ds.maxL1Height + } + if ds.l1height > to { + return nil, sourceExhaustedErr + } + logs, err := ds.l1Client.fetchRollupEventsInRange(ds.ctx, ds.l1height, to) + if err != nil { + return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) + } + ds.l1height = to + 1 + return ds.processLogsToDA(logs) +} + +func (ds *BlobDataSource) L1Height() uint64 { + return ds.l1height +} + +func (ds *BlobDataSource) processLogsToDA(logs []types.Log) (DA, error) { + var da DA + for _, vLog := range logs { + switch vLog.Topics[0] { + case ds.l1CommitBatchEventSignature: + event := &L1CommitBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new CommitBatch event", "batch index", batchIndex) + + daEntry, err := ds.getCommitBatchDa(batchIndex, &vLog) + if err != nil { + return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) + } + da = append(da, daEntry) + + case ds.l1RevertBatchEventSignature: + event := &L1RevertBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new RevertBatch event", "batch index", batchIndex) + da = append(da, NewRevertBatchDA(batchIndex)) + + case ds.l1FinalizeBatchEventSignature: + event := &L1FinalizeBatchEvent{} + if err := UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { + return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) + } + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new FinalizeBatch event", "batch index", batchIndex) + + da = append(da, NewFinalizeBatchDA(batchIndex)) + + default: + return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) + } + } + return da, nil +} + +func (ds *BlobDataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { + var chunks []*codecv1.DAChunkRawTx + var l1Txs []*types.L1MessageTx + if batchIndex == 0 { + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil + } + + txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) + if err != nil { + return nil, err + } + const methodIDLength = 4 + if len(txData) < methodIDLength { + return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) + } + + method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) + if err != nil { + return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) + } + + values, err := method.Inputs.Unpack(txData[methodIDLength:]) + if err != nil { + return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) + } + + type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte + } + var args commitBatchArgs + err = method.Inputs.Copy(&args, values) + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + + // todo: use codecv1 chunks + chunks, err = codecv1.DecodeDAChunksRawTx(args.Chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) + if err != nil { + return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) + } + + parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + totalL1MessagePopped := 0 + for _, chunk := range chunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < int(totalL1MessagePopped); index++ { + for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + if l1Tx == nil { + return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) + currentIndex++ + } + da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + return da, nil +} diff --git a/rollup/da_syncer/calldata_source.go b/rollup/da_syncer/calldata_source.go index 48b93021ead4..9c8b9f291df4 100644 --- a/rollup/da_syncer/calldata_source.go +++ b/rollup/da_syncer/calldata_source.go @@ -2,9 +2,7 @@ package da_syncer import ( "context" - "encoding/binary" "fmt" - "math/big" "github.com/scroll-tech/go-ethereum/accounts/abi" "github.com/scroll-tech/go-ethereum/common" @@ -12,6 +10,7 @@ import ( "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/rollup/types/encoding" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" ) @@ -168,14 +167,14 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( totalL1MessagePopped += int(block.NumL1Messages) } } - skippedBitmap, err := decodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) if err != nil { return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) } // get all necessary l1msgs without skipped currentIndex := parentTotalL1MessagePopped for index := 0; index < int(totalL1MessagePopped); index++ { - for isL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ } l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) @@ -188,29 +187,3 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) return da, nil } - -func getBatchTotalL1MessagePopped(batchHeader []byte) uint64 { - return binary.BigEndian.Uint64(batchHeader[17:25]) -} - -func decodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped int) ([]*big.Int, error) { - length := len(skippedL1MessageBitmap) - if length%32 != 0 { - return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) - } - if length*8 < totalL1MessagePopped { - return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) - } - var skippedBitmap []*big.Int - for index := 0; index < length/32; index++ { - bitmap := big.NewInt(0).SetBytes(skippedL1MessageBitmap[index*32 : index*32+32]) - skippedBitmap = append(skippedBitmap, bitmap) - } - return skippedBitmap, nil -} - -func isL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { - quo := index / 256 - rem := index % 256 - return skippedBitmap[quo].Bit(int(rem)) != 0 -} diff --git a/rollup/da_syncer/da_fetcher.go b/rollup/da_syncer/da_fetcher.go deleted file mode 100644 index 0b26890cd124..000000000000 --- a/rollup/da_syncer/da_fetcher.go +++ /dev/null @@ -1,7 +0,0 @@ -package da_syncer - -// DaFetcher encapsulates functions required to fetch data from l1 -type DaFetcher interface { - FetchDA() (DA, uint64, error) - SetLatestProcessedBlock(to uint64) -} diff --git a/rollup/da_syncer/snapshot_fetcher.go b/rollup/da_syncer/snapshot_fetcher.go deleted file mode 100644 index b317c03edecb..000000000000 --- a/rollup/da_syncer/snapshot_fetcher.go +++ /dev/null @@ -1,20 +0,0 @@ -package da_syncer - -type SnapshotFetcher struct { - fetchBlockRange uint64 -} - -func newSnapshotFetcher(fetchBlockRange uint64) (DaFetcher, error) { - daFetcher := SnapshotFetcher{ - fetchBlockRange: fetchBlockRange, - } - return &daFetcher, nil -} - -func (f *SnapshotFetcher) FetchDA() (DA, uint64, error) { - return nil, 0, nil -} - -func (f *SnapshotFetcher) SetLatestProcessedBlock(to uint64) { - return -} diff --git a/rollup/types/encoding/bitmap.go b/rollup/types/encoding/bitmap.go index 87fc31c6456b..51afb3bb9c04 100644 --- a/rollup/types/encoding/bitmap.go +++ b/rollup/types/encoding/bitmap.go @@ -62,3 +62,27 @@ func ConstructSkippedBitmap(batchIndex uint64, chunks []*Chunk, totalL1MessagePo return bitmapBytes, nextIndex, nil } + +// DecodeBitmap decodes skipped L1 message bitmap of the batch from bytes to big.Int's +func DecodeBitmap(skippedL1MessageBitmap []byte, totalL1MessagePopped int) ([]*big.Int, error) { + length := len(skippedL1MessageBitmap) + if length%32 != 0 { + return nil, fmt.Errorf("skippedL1MessageBitmap length doesn't match, skippedL1MessageBitmap length should be equal 0 modulo 32, length of skippedL1MessageBitmap: %v", length) + } + if length*8 < totalL1MessagePopped { + return nil, fmt.Errorf("skippedL1MessageBitmap length is too small, skippedL1MessageBitmap length should be at least %v, length of skippedL1MessageBitmap: %v", (totalL1MessagePopped+7)/8, length) + } + var skippedBitmap []*big.Int + for index := 0; index < length/32; index++ { + bitmap := big.NewInt(0).SetBytes(skippedL1MessageBitmap[index*32 : index*32+32]) + skippedBitmap = append(skippedBitmap, bitmap) + } + return skippedBitmap, nil +} + +// IsL1MessageSkipped checks if index is skipped in bitmap +func IsL1MessageSkipped(skippedBitmap []*big.Int, index uint64) bool { + quo := index / 256 + rem := index % 256 + return skippedBitmap[quo].Bit(int(rem)) != 0 +} From 27a4e51b55345cb79b290c12868af8c0137ee57c Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Fri, 24 May 2024 20:00:31 +0200 Subject: [PATCH 10/59] implement blob fethcer --- rollup/da_syncer/blob_client.go | 26 +-- rollup/da_syncer/blob_scan_client.go | 94 +++++++++ rollup/da_syncer/blob_source.go | 190 ------------------ ...data_source.go => calldata_blob_source.go} | 124 +++++++++--- rollup/da_syncer/da.go | 31 ++- rollup/da_syncer/data_source.go | 12 +- rollup/da_syncer/l1_client.go | 40 +++- rollup/da_syncer/syncing_pipeline.go | 8 +- rollup/types/encoding/codecv0/codecv0.go | 2 +- rollup/types/encoding/codecv1/codecv1.go | 41 ++++ 10 files changed, 317 insertions(+), 251 deletions(-) create mode 100644 rollup/da_syncer/blob_scan_client.go delete mode 100644 rollup/da_syncer/blob_source.go rename rollup/da_syncer/{calldata_source.go => calldata_blob_source.go} (56%) diff --git a/rollup/da_syncer/blob_client.go b/rollup/da_syncer/blob_client.go index caef4d887d22..9e3043dd96d2 100644 --- a/rollup/da_syncer/blob_client.go +++ b/rollup/da_syncer/blob_client.go @@ -5,30 +5,8 @@ import ( "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" - "github.com/scroll-tech/go-ethereum/params" - - "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) -// BlobClient is a wrapper around EthClient that adds -// methods for conveniently collecting rollup events of ScrollChain contract. -type BlobClient struct { - scrollChainAddress common.Address - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash -} - -// newL1Client initializes a new L1Client instance with the provided configuration. -// It checks for a valid scrollChainAddress and verifies the chain ID. -func newBlobClient(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient) (*BlobClient, error) { - client := BlobClient{} - - return &client, nil -} - -// fetchBlob fetches blob by it's commitment -func (c *BlobClient) fetchBlob(ctx context.Context, commitment []*kzg4844.Commitment) ([]*kzg4844.Blob, error) { - // todo: - return nil, nil +type BlobClient interface { + GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) } diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_scan_client.go new file mode 100644 index 000000000000..dd7b9a657e72 --- /dev/null +++ b/rollup/da_syncer/blob_scan_client.go @@ -0,0 +1,94 @@ +package da_syncer + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "net/http" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" +) + +const ( + blobScanApiUrl string = "https://api.blobscan.com/blobs/" + okStatusCode int = 200 + lenBlobBytes int = 131072 +) + +type BlobScanClient struct { + client *http.Client +} + +func newBlobScanClient() (*BlobScanClient, error) { + return &BlobScanClient{ + client: http.DefaultClient, + }, nil +} + +func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { + // some api call + req, err := http.NewRequestWithContext(ctx, "GET", blobScanApiUrl+versionedHash.String(), nil) + if err != nil { + return nil, fmt.Errorf("cannot create request, err: %v", err) + } + req.Header.Set("accept", "application/json") + resp, err := c.client.Do(req) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %v", err) + } + defer resp.Body.Close() + if resp.StatusCode != okStatusCode { + return nil, fmt.Errorf("response code is not ok, code: %d", resp.StatusCode) + } + var result BlobResp + err = json.NewDecoder(resp.Body).Decode(&result) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %v", err) + } + blobBytes, err := hex.DecodeString(result.Data[2:]) + if err != nil { + return nil, fmt.Errorf("failed to decode data to bytes, err: %v", err) + } + if len(blobBytes) != lenBlobBytes { + return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) + } + blob := kzg4844.Blob(blobBytes) + return &blob, nil +} + +type BlobResp struct { + Commitment string `json:"commitment"` + Proof string `json:"proof"` + Size int `json:"size"` + VersionedHash string `json:"versionedHash"` + Data string `json:"data"` + DataStorageReferences []struct { + BlobStorage string `json:"blobStorage"` + DataReference string `json:"dataReference"` + } `json:"dataStorageReferences"` + Transactions []struct { + Hash string `json:"hash"` + Index int `json:"index"` + Block struct { + Number int `json:"number"` + BlobGasUsed string `json:"blobGasUsed"` + BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` + BlobGasPrice string `json:"blobGasPrice"` + ExcessBlobGas string `json:"excessBlobGas"` + Hash string `json:"hash"` + Timestamp string `json:"timestamp"` + Slot int `json:"slot"` + } `json:"block"` + From string `json:"from"` + To string `json:"to"` + MaxFeePerBlobGas string `json:"maxFeePerBlobGas"` + BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` + Rollup string `json:"rollup"` + BlobAsCalldataGasFee string `json:"blobAsCalldataGasFee"` + BlobGasBaseFee string `json:"blobGasBaseFee"` + BlobGasMaxFee string `json:"blobGasMaxFee"` + BlobGasUsed string `json:"blobGasUsed"` + } `json:"transactions"` +} diff --git a/rollup/da_syncer/blob_source.go b/rollup/da_syncer/blob_source.go deleted file mode 100644 index 8aa876f6ff49..000000000000 --- a/rollup/da_syncer/blob_source.go +++ /dev/null @@ -1,190 +0,0 @@ -package da_syncer - -import ( - "context" - "fmt" - - "github.com/scroll-tech/go-ethereum/accounts/abi" - "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/rollup/types/encoding" -) - -var ( - blobSourceFetchBlockRange uint64 = 100 -) - -type BlobDataSource struct { - ctx context.Context - l1Client *L1Client - blobClient *BlobClient - l1height uint64 - maxL1Height uint64 - scrollChainABI *abi.ABI - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash - db ethdb.Database -} - -func NewBlobDataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, blobClient *BlobClient, db ethdb.Database) (DataSource, error) { - scrollChainABI, err := scrollChainMetaData.GetAbi() - if err != nil { - return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) - } - return &BlobDataSource{ - ctx: ctx, - l1Client: l1Client, - blobClient: blobClient, - l1height: l1height, - maxL1Height: maxL1Height, - scrollChainABI: scrollChainABI, - l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, - l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, - db: db, - }, nil -} - -func (ds *BlobDataSource) NextData() (DA, error) { - to := ds.l1height + callDataSourceFetchBlockRange - if to > ds.maxL1Height { - to = ds.maxL1Height - } - if ds.l1height > to { - return nil, sourceExhaustedErr - } - logs, err := ds.l1Client.fetchRollupEventsInRange(ds.ctx, ds.l1height, to) - if err != nil { - return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) - } - ds.l1height = to + 1 - return ds.processLogsToDA(logs) -} - -func (ds *BlobDataSource) L1Height() uint64 { - return ds.l1height -} - -func (ds *BlobDataSource) processLogsToDA(logs []types.Log) (DA, error) { - var da DA - for _, vLog := range logs { - switch vLog.Topics[0] { - case ds.l1CommitBatchEventSignature: - event := &L1CommitBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new CommitBatch event", "batch index", batchIndex) - - daEntry, err := ds.getCommitBatchDa(batchIndex, &vLog) - if err != nil { - return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) - } - da = append(da, daEntry) - - case ds.l1RevertBatchEventSignature: - event := &L1RevertBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new RevertBatch event", "batch index", batchIndex) - da = append(da, NewRevertBatchDA(batchIndex)) - - case ds.l1FinalizeBatchEventSignature: - event := &L1FinalizeBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new FinalizeBatch event", "batch index", batchIndex) - - da = append(da, NewFinalizeBatchDA(batchIndex)) - - default: - return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) - } - } - return da, nil -} - -func (ds *BlobDataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - var chunks []*codecv1.DAChunkRawTx - var l1Txs []*types.L1MessageTx - if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil - } - - txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) - if err != nil { - return nil, err - } - const methodIDLength = 4 - if len(txData) < methodIDLength { - return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) - } - - method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) - if err != nil { - return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) - } - - values, err := method.Inputs.Unpack(txData[methodIDLength:]) - if err != nil { - return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) - } - - type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - } - var args commitBatchArgs - err = method.Inputs.Copy(&args, values) - if err != nil { - return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - - // todo: use codecv1 chunks - chunks, err = codecv1.DecodeDAChunksRawTx(args.Chunks) - if err != nil { - return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) - if err != nil { - return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) - } - - parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped - totalL1MessagePopped := 0 - for _, chunk := range chunks { - for _, block := range chunk.Blocks { - totalL1MessagePopped += int(block.NumL1Messages) - } - } - skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < int(totalL1MessagePopped); index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - } - l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - if l1Tx == nil { - return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) - return da, nil -} diff --git a/rollup/da_syncer/calldata_source.go b/rollup/da_syncer/calldata_blob_source.go similarity index 56% rename from rollup/da_syncer/calldata_source.go rename to rollup/da_syncer/calldata_blob_source.go index 9c8b9f291df4..033046d46de0 100644 --- a/rollup/da_syncer/calldata_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -2,27 +2,30 @@ package da_syncer import ( "context" + "crypto/sha256" "fmt" "github.com/scroll-tech/go-ethereum/accounts/abi" "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/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/rollup/types/encoding" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) var ( - callDataSourceFetchBlockRange uint64 = 100 + callDataBlobSourceFetchBlockRange uint64 = 100 ) -type CalldataSource struct { +type CalldataBlobSource struct { ctx context.Context l1Client *L1Client + blobClient BlobClient l1height uint64 - maxL1Height uint64 scrollChainABI *abi.ABI l1CommitBatchEventSignature common.Hash l1RevertBatchEventSignature common.Hash @@ -30,16 +33,16 @@ type CalldataSource struct { db ethdb.Database } -func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, db ethdb.Database) (DataSource, error) { +func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) (DataSource, error) { scrollChainABI, err := scrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) } - return &CalldataSource{ + return &CalldataBlobSource{ ctx: ctx, l1Client: l1Client, + blobClient: blobClient, l1height: l1height, - maxL1Height: maxL1Height, scrollChainABI: scrollChainABI, l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, @@ -48,10 +51,14 @@ func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Clie }, nil } -func (ds *CalldataSource) NextData() (DA, error) { - to := ds.l1height + callDataSourceFetchBlockRange - if to > ds.maxL1Height { - to = ds.maxL1Height +func (ds *CalldataBlobSource) NextData() (DA, error) { + to := ds.l1height + callDataBlobSourceFetchBlockRange + l1Finalized, err := ds.l1Client.getFinalizedBlockNumber(ds.ctx) + if err != nil { + return nil, fmt.Errorf("cannot get l1height, error: %v", err) + } + if to > l1Finalized.Uint64() { + to = l1Finalized.Uint64() } if ds.l1height > to { return nil, sourceExhaustedErr @@ -64,11 +71,11 @@ func (ds *CalldataSource) NextData() (DA, error) { return ds.processLogsToDA(logs) } -func (ds *CalldataSource) L1Height() uint64 { +func (ds *CalldataBlobSource) L1Height() uint64 { return ds.l1height } -func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { +func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { var da DA for _, vLog := range logs { switch vLog.Topics[0] { @@ -112,11 +119,16 @@ func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { return da, nil } -func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - var chunks []*codecv0.DAChunkRawTx - var l1Txs []*types.L1MessageTx +type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte +} + +func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}), nil } txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) @@ -138,20 +150,26 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) } - type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - } var args commitBatchArgs err = method.Inputs.Copy(&args, values) if err != nil { return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } + switch args.Version { + case codecv0.CodecV0Version: + return ds.decodeDAV0(batchIndex, vLog, &args) + case codecv1.CodecV1Version: + return ds.decodeDAV1(batchIndex, vLog, &args) + default: + return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) + } + +} - // todo: use codecv0 chunks - chunks, err = codecv0.DecodeDAChunksRawTx(args.Chunks) +func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { + var chunks []*codecv0.DAChunkRawTx + var l1Txs []*types.L1MessageTx + chunks, err := codecv0.DecodeDAChunksRawTx(args.Chunks) if err != nil { return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } @@ -187,3 +205,61 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) return da, nil } + +func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { + var chunks []*codecv1.DAChunkRawTx + var l1Txs []*types.L1MessageTx + chunks, err := codecv1.DecodeDAChunksRawTx(args.Chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) + if err != nil { + return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) + } + versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) + } + blob, err := ds.blobClient.GetBlobByVersionedHash(ds.ctx, versionedHash) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) + } + // compute blob versioned hash and compare with one from tx + c, err := kzg4844.BlobToCommitment(blob) + if err != nil { + return nil, fmt.Errorf("failed to create blob commitment") + } + blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) + } + // todo: decode blob into txs? txs are written in rlp encoded format one by one without any delimiter + + parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + totalL1MessagePopped := 0 + for _, chunk := range chunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < int(totalL1MessagePopped); index++ { + for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + if l1Tx == nil { + return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) + currentIndex++ + } + da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + return da, nil +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 27c9f5a9c7fc..3f098a8a0783 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -3,13 +3,16 @@ package da_syncer import ( "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) type DAType int const ( - // CommitBatch contains data of event of CommitBatch + // CommitBatchV0 contains data of event of CommitBatchV0 CommitBatchV0 DAType = iota + // CommitBatchV1 contains data of event of CommitBatchV1 + CommitBatchV1 // RevertBatch contains data of event of RevertBatch RevertBatch // FinalizeBatch contains data of event of FinalizeBatch @@ -48,6 +51,32 @@ func (f *CommitBatchDaV0) DAType() DAType { return f.DaType } +type CommitBatchDaV1 struct { + DaType DAType + Version uint8 + BatchIndex uint64 + ParentBatchHeader *codecv1.DABatch + SkippedL1MessageBitmap []byte + Chunks []*codecv1.DAChunkRawTx + L1Txs []*types.L1MessageTx +} + +func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *codecv1.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx) DAEntry { + return &CommitBatchDaV1{ + DaType: CommitBatchV1, + Version: version, + BatchIndex: batchIndex, + ParentBatchHeader: parentBatchHeader, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + } +} + +func (f *CommitBatchDaV1) DAType() DAType { + return f.DaType +} + type RevertBatchDA struct { DaType DAType BatchIndex uint64 diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index 182cd2ab4bad..3ce4d68cd92c 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -22,27 +22,23 @@ type DataSourceFactory struct { config Config genesisConfig *params.ChainConfig l1Client *L1Client + blobClient BlobClient db ethdb.Database } -func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, db ethdb.Database) *DataSourceFactory { +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) *DataSourceFactory { return &DataSourceFactory{ config: config, genesisConfig: genesisConfig, l1Client: l1Client, + blobClient: blobClient, db: db, } } func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { if ds.config.FetcherMode == L1RPC { - if isBernoulliByL1Height(l1height) { - return nil, errors.New("blob_data_source: not implemented") - } else { - // todo: set l1 block where l2 changes to bernoulli - var maxL1Height uint64 = 1000000000000 - return NewCalldataSource(ctx, l1height, maxL1Height, ds.l1Client, ds.db) - } + return NewCalldataBlobSource(ctx, l1height, ds.l1Client, ds.blobClient, ds.db) } else { return nil, errors.New("snapshot_data_source: not implemented") } diff --git a/rollup/da_syncer/l1_client.go b/rollup/da_syncer/l1_client.go index 3ed691be5323..8d234c68a9f3 100644 --- a/rollup/da_syncer/l1_client.go +++ b/rollup/da_syncer/l1_client.go @@ -11,6 +11,7 @@ import ( "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rpc" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) @@ -55,7 +56,6 @@ func newL1Client(ctx context.Context, genesisConfig *params.ChainConfig, l1Clien l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, } - return &client, nil } @@ -109,3 +109,41 @@ func (c *L1Client) fetchTxData(ctx context.Context, vLog *types.Log) ([]byte, er return tx.Data(), nil } + +// fetchTxBlobHash fetches tx blob hash corresponding to given event log +func (c *L1Client) fetchTxBlobHash(ctx context.Context, vLog *types.Log) (common.Hash, error) { + tx, _, err := c.client.TransactionByHash(ctx, vLog.TxHash) + if err != nil { + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := c.client.BlockByHash(ctx, vLog.BlockHash) + if err != nil { + return common.Hash{}, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return common.Hash{}, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } + } + blobHashes := tx.BlobHashes() + if len(blobHashes) == 0 { + return common.Hash{}, fmt.Errorf("transaction does not contain any blobs, tx hash: %v", vLog.TxHash.Hex()) + } + return blobHashes[0], nil +} + +func (c *L1Client) getFinalizedBlockNumber(ctx context.Context) (*big.Int, error) { + h, err := c.client.HeaderByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) + if err != nil { + return nil, err + } + return h.Number, nil +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 6a301d44f0be..06d84ed693e5 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -44,8 +44,12 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi cancel() return nil, err } - - dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, db) + blobClient, err := newBlobScanClient() + if err != nil { + cancel() + return nil, err + } + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) // todo: keep synced l1 height somewhere var syncedL1Height uint64 = l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) diff --git a/rollup/types/encoding/codecv0/codecv0.go b/rollup/types/encoding/codecv0/codecv0.go index ee979be58654..3016e4c04f9d 100644 --- a/rollup/types/encoding/codecv0/codecv0.go +++ b/rollup/types/encoding/codecv0/codecv0.go @@ -35,7 +35,7 @@ type DAChunk struct { Transactions [][]*types.TransactionData } -// DAChunk groups consecutive DABlocks with their transactions. +// DAChunkRawTx groups consecutive DABlocks with their transactions. type DAChunkRawTx struct { Blocks []*DABlock Transactions []types.Transactions diff --git a/rollup/types/encoding/codecv1/codecv1.go b/rollup/types/encoding/codecv1/codecv1.go index 7795b85b03f8..1176e3448c5f 100644 --- a/rollup/types/encoding/codecv1/codecv1.go +++ b/rollup/types/encoding/codecv1/codecv1.go @@ -55,6 +55,7 @@ func init() { // CodecV1Version denotes the version of the codec. const CodecV1Version = 1 +const blockContextByteSize = 60 // DABlock represents a Data Availability Block. type DABlock struct { @@ -72,6 +73,12 @@ type DAChunk struct { Transactions [][]*types.TransactionData } +// DAChunkRawTx groups consecutive DABlocks with their transactions. +type DAChunkRawTx struct { + Blocks []*DABlock + Transactions []types.Transactions +} + // DABatch contains metadata about a batch of DAChunks. type DABatch struct { // header @@ -188,6 +195,40 @@ func (c *DAChunk) Encode() []byte { return chunkBytes } +// DecodeDAChunksRawTx takes a byte slice and decodes it into a []DAChunkRawTx. +func DecodeDAChunksRawTx(bytes [][]byte) ([]*DAChunkRawTx, error) { + var chunks []*DAChunkRawTx + for _, chunk := range bytes { + if len(chunk) < 1 { + return nil, fmt.Errorf("invalid chunk, length is less than 1") + } + + numBlocks := int(chunk[0]) + if len(chunk) < 1+numBlocks*blockContextByteSize { + return nil, fmt.Errorf("chunk size doesn't match with numBlocks, byte length of chunk: %v, expected length: %v", len(chunk), 1+numBlocks*blockContextByteSize) + } + + blocks := make([]*DABlock, numBlocks) + for i := 0; i < numBlocks; i++ { + startIdx := 1 + i*blockContextByteSize // add 1 to skip numBlocks byte + endIdx := startIdx + blockContextByteSize + block, err := DecodeDABlock(chunk[startIdx:endIdx]) + if err != nil { + return nil, err + } + blocks[i] = block + } + + var transactions []types.Transactions + + chunks = append(chunks, &DAChunkRawTx{ + Blocks: blocks, + Transactions: transactions, + }) + } + return chunks, nil +} + // Hash computes the hash of the DAChunk data. func (c *DAChunk) Hash() (common.Hash, error) { var dataBytes []byte From 6991da071ed2d091c05e741914b4044fd3de9489 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Fri, 24 May 2024 20:00:31 +0200 Subject: [PATCH 11/59] implement blob fethcer --- rollup/da_syncer/blob_client.go | 26 +-- rollup/da_syncer/blob_scan_client.go | 94 +++++++++ rollup/da_syncer/blob_source.go | 190 ------------------ ...data_source.go => calldata_blob_source.go} | 127 +++++++++--- rollup/da_syncer/da.go | 31 ++- rollup/da_syncer/data_source.go | 12 +- rollup/da_syncer/l1_client.go | 40 +++- rollup/da_syncer/syncing_pipeline.go | 8 +- rollup/types/encoding/codecv0/codecv0.go | 2 +- rollup/types/encoding/codecv1/codecv1.go | 133 ++++++++++++ 10 files changed, 412 insertions(+), 251 deletions(-) create mode 100644 rollup/da_syncer/blob_scan_client.go delete mode 100644 rollup/da_syncer/blob_source.go rename rollup/da_syncer/{calldata_source.go => calldata_blob_source.go} (56%) diff --git a/rollup/da_syncer/blob_client.go b/rollup/da_syncer/blob_client.go index caef4d887d22..9e3043dd96d2 100644 --- a/rollup/da_syncer/blob_client.go +++ b/rollup/da_syncer/blob_client.go @@ -5,30 +5,8 @@ import ( "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" - "github.com/scroll-tech/go-ethereum/params" - - "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) -// BlobClient is a wrapper around EthClient that adds -// methods for conveniently collecting rollup events of ScrollChain contract. -type BlobClient struct { - scrollChainAddress common.Address - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash -} - -// newL1Client initializes a new L1Client instance with the provided configuration. -// It checks for a valid scrollChainAddress and verifies the chain ID. -func newBlobClient(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient) (*BlobClient, error) { - client := BlobClient{} - - return &client, nil -} - -// fetchBlob fetches blob by it's commitment -func (c *BlobClient) fetchBlob(ctx context.Context, commitment []*kzg4844.Commitment) ([]*kzg4844.Blob, error) { - // todo: - return nil, nil +type BlobClient interface { + GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) } diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_scan_client.go new file mode 100644 index 000000000000..dd7b9a657e72 --- /dev/null +++ b/rollup/da_syncer/blob_scan_client.go @@ -0,0 +1,94 @@ +package da_syncer + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "net/http" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" +) + +const ( + blobScanApiUrl string = "https://api.blobscan.com/blobs/" + okStatusCode int = 200 + lenBlobBytes int = 131072 +) + +type BlobScanClient struct { + client *http.Client +} + +func newBlobScanClient() (*BlobScanClient, error) { + return &BlobScanClient{ + client: http.DefaultClient, + }, nil +} + +func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { + // some api call + req, err := http.NewRequestWithContext(ctx, "GET", blobScanApiUrl+versionedHash.String(), nil) + if err != nil { + return nil, fmt.Errorf("cannot create request, err: %v", err) + } + req.Header.Set("accept", "application/json") + resp, err := c.client.Do(req) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %v", err) + } + defer resp.Body.Close() + if resp.StatusCode != okStatusCode { + return nil, fmt.Errorf("response code is not ok, code: %d", resp.StatusCode) + } + var result BlobResp + err = json.NewDecoder(resp.Body).Decode(&result) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %v", err) + } + blobBytes, err := hex.DecodeString(result.Data[2:]) + if err != nil { + return nil, fmt.Errorf("failed to decode data to bytes, err: %v", err) + } + if len(blobBytes) != lenBlobBytes { + return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) + } + blob := kzg4844.Blob(blobBytes) + return &blob, nil +} + +type BlobResp struct { + Commitment string `json:"commitment"` + Proof string `json:"proof"` + Size int `json:"size"` + VersionedHash string `json:"versionedHash"` + Data string `json:"data"` + DataStorageReferences []struct { + BlobStorage string `json:"blobStorage"` + DataReference string `json:"dataReference"` + } `json:"dataStorageReferences"` + Transactions []struct { + Hash string `json:"hash"` + Index int `json:"index"` + Block struct { + Number int `json:"number"` + BlobGasUsed string `json:"blobGasUsed"` + BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` + BlobGasPrice string `json:"blobGasPrice"` + ExcessBlobGas string `json:"excessBlobGas"` + Hash string `json:"hash"` + Timestamp string `json:"timestamp"` + Slot int `json:"slot"` + } `json:"block"` + From string `json:"from"` + To string `json:"to"` + MaxFeePerBlobGas string `json:"maxFeePerBlobGas"` + BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` + Rollup string `json:"rollup"` + BlobAsCalldataGasFee string `json:"blobAsCalldataGasFee"` + BlobGasBaseFee string `json:"blobGasBaseFee"` + BlobGasMaxFee string `json:"blobGasMaxFee"` + BlobGasUsed string `json:"blobGasUsed"` + } `json:"transactions"` +} diff --git a/rollup/da_syncer/blob_source.go b/rollup/da_syncer/blob_source.go deleted file mode 100644 index 8aa876f6ff49..000000000000 --- a/rollup/da_syncer/blob_source.go +++ /dev/null @@ -1,190 +0,0 @@ -package da_syncer - -import ( - "context" - "fmt" - - "github.com/scroll-tech/go-ethereum/accounts/abi" - "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/rollup/types/encoding" -) - -var ( - blobSourceFetchBlockRange uint64 = 100 -) - -type BlobDataSource struct { - ctx context.Context - l1Client *L1Client - blobClient *BlobClient - l1height uint64 - maxL1Height uint64 - scrollChainABI *abi.ABI - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash - db ethdb.Database -} - -func NewBlobDataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, blobClient *BlobClient, db ethdb.Database) (DataSource, error) { - scrollChainABI, err := scrollChainMetaData.GetAbi() - if err != nil { - return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) - } - return &BlobDataSource{ - ctx: ctx, - l1Client: l1Client, - blobClient: blobClient, - l1height: l1height, - maxL1Height: maxL1Height, - scrollChainABI: scrollChainABI, - l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, - l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, - db: db, - }, nil -} - -func (ds *BlobDataSource) NextData() (DA, error) { - to := ds.l1height + callDataSourceFetchBlockRange - if to > ds.maxL1Height { - to = ds.maxL1Height - } - if ds.l1height > to { - return nil, sourceExhaustedErr - } - logs, err := ds.l1Client.fetchRollupEventsInRange(ds.ctx, ds.l1height, to) - if err != nil { - return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) - } - ds.l1height = to + 1 - return ds.processLogsToDA(logs) -} - -func (ds *BlobDataSource) L1Height() uint64 { - return ds.l1height -} - -func (ds *BlobDataSource) processLogsToDA(logs []types.Log) (DA, error) { - var da DA - for _, vLog := range logs { - switch vLog.Topics[0] { - case ds.l1CommitBatchEventSignature: - event := &L1CommitBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new CommitBatch event", "batch index", batchIndex) - - daEntry, err := ds.getCommitBatchDa(batchIndex, &vLog) - if err != nil { - return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) - } - da = append(da, daEntry) - - case ds.l1RevertBatchEventSignature: - event := &L1RevertBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new RevertBatch event", "batch index", batchIndex) - da = append(da, NewRevertBatchDA(batchIndex)) - - case ds.l1FinalizeBatchEventSignature: - event := &L1FinalizeBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new FinalizeBatch event", "batch index", batchIndex) - - da = append(da, NewFinalizeBatchDA(batchIndex)) - - default: - return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) - } - } - return da, nil -} - -func (ds *BlobDataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - var chunks []*codecv1.DAChunkRawTx - var l1Txs []*types.L1MessageTx - if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil - } - - txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) - if err != nil { - return nil, err - } - const methodIDLength = 4 - if len(txData) < methodIDLength { - return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) - } - - method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) - if err != nil { - return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) - } - - values, err := method.Inputs.Unpack(txData[methodIDLength:]) - if err != nil { - return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) - } - - type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - } - var args commitBatchArgs - err = method.Inputs.Copy(&args, values) - if err != nil { - return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - - // todo: use codecv1 chunks - chunks, err = codecv1.DecodeDAChunksRawTx(args.Chunks) - if err != nil { - return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) - if err != nil { - return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) - } - - parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped - totalL1MessagePopped := 0 - for _, chunk := range chunks { - for _, block := range chunk.Blocks { - totalL1MessagePopped += int(block.NumL1Messages) - } - } - skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < int(totalL1MessagePopped); index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - } - l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - if l1Tx == nil { - return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) - return da, nil -} diff --git a/rollup/da_syncer/calldata_source.go b/rollup/da_syncer/calldata_blob_source.go similarity index 56% rename from rollup/da_syncer/calldata_source.go rename to rollup/da_syncer/calldata_blob_source.go index 9c8b9f291df4..ce6ae009e89a 100644 --- a/rollup/da_syncer/calldata_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -2,27 +2,30 @@ package da_syncer import ( "context" + "crypto/sha256" "fmt" "github.com/scroll-tech/go-ethereum/accounts/abi" "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/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/rollup/types/encoding" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) var ( - callDataSourceFetchBlockRange uint64 = 100 + callDataBlobSourceFetchBlockRange uint64 = 100 ) -type CalldataSource struct { +type CalldataBlobSource struct { ctx context.Context l1Client *L1Client + blobClient BlobClient l1height uint64 - maxL1Height uint64 scrollChainABI *abi.ABI l1CommitBatchEventSignature common.Hash l1RevertBatchEventSignature common.Hash @@ -30,16 +33,16 @@ type CalldataSource struct { db ethdb.Database } -func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Client *L1Client, db ethdb.Database) (DataSource, error) { +func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) (DataSource, error) { scrollChainABI, err := scrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) } - return &CalldataSource{ + return &CalldataBlobSource{ ctx: ctx, l1Client: l1Client, + blobClient: blobClient, l1height: l1height, - maxL1Height: maxL1Height, scrollChainABI: scrollChainABI, l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, @@ -48,10 +51,14 @@ func NewCalldataSource(ctx context.Context, l1height, maxL1Height uint64, l1Clie }, nil } -func (ds *CalldataSource) NextData() (DA, error) { - to := ds.l1height + callDataSourceFetchBlockRange - if to > ds.maxL1Height { - to = ds.maxL1Height +func (ds *CalldataBlobSource) NextData() (DA, error) { + to := ds.l1height + callDataBlobSourceFetchBlockRange + l1Finalized, err := ds.l1Client.getFinalizedBlockNumber(ds.ctx) + if err != nil { + return nil, fmt.Errorf("cannot get l1height, error: %v", err) + } + if to > l1Finalized.Uint64() { + to = l1Finalized.Uint64() } if ds.l1height > to { return nil, sourceExhaustedErr @@ -64,11 +71,11 @@ func (ds *CalldataSource) NextData() (DA, error) { return ds.processLogsToDA(logs) } -func (ds *CalldataSource) L1Height() uint64 { +func (ds *CalldataBlobSource) L1Height() uint64 { return ds.l1height } -func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { +func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { var da DA for _, vLog := range logs { switch vLog.Topics[0] { @@ -112,11 +119,16 @@ func (ds *CalldataSource) processLogsToDA(logs []types.Log) (DA, error) { return da, nil } -func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - var chunks []*codecv0.DAChunkRawTx - var l1Txs []*types.L1MessageTx +type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte +} + +func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, chunks, l1Txs), nil + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}), nil } txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) @@ -138,20 +150,26 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) } - type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - } var args commitBatchArgs err = method.Inputs.Copy(&args, values) if err != nil { return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } + switch args.Version { + case codecv0.CodecV0Version: + return ds.decodeDAV0(batchIndex, vLog, &args) + case codecv1.CodecV1Version: + return ds.decodeDAV1(batchIndex, vLog, &args) + default: + return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) + } + +} - // todo: use codecv0 chunks - chunks, err = codecv0.DecodeDAChunksRawTx(args.Chunks) +func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { + var chunks []*codecv0.DAChunkRawTx + var l1Txs []*types.L1MessageTx + chunks, err := codecv0.DecodeDAChunksRawTx(args.Chunks) if err != nil { return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } @@ -187,3 +205,64 @@ func (ds *CalldataSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) ( da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) return da, nil } + +func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { + var chunks []*codecv1.DAChunkRawTx + var l1Txs []*types.L1MessageTx + chunks, err := codecv1.DecodeDAChunksRawTx(args.Chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) + if err != nil { + return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) + } + versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) + } + blob, err := ds.blobClient.GetBlobByVersionedHash(ds.ctx, versionedHash) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) + } + // compute blob versioned hash and compare with one from tx + c, err := kzg4844.BlobToCommitment(blob) + if err != nil { + return nil, fmt.Errorf("failed to create blob commitment") + } + blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) + } + // decode txs from blob + err = codecv1.DecodeTxsFromBlob(blob, chunks) + if err != nil { + return nil, fmt.Errorf("failed to decode txs from blob: %w", err) + } + parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + totalL1MessagePopped := 0 + for _, chunk := range chunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < int(totalL1MessagePopped); index++ { + for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + if l1Tx == nil { + return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) + currentIndex++ + } + da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + return da, nil +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 27c9f5a9c7fc..3f098a8a0783 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -3,13 +3,16 @@ package da_syncer import ( "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" + "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) type DAType int const ( - // CommitBatch contains data of event of CommitBatch + // CommitBatchV0 contains data of event of CommitBatchV0 CommitBatchV0 DAType = iota + // CommitBatchV1 contains data of event of CommitBatchV1 + CommitBatchV1 // RevertBatch contains data of event of RevertBatch RevertBatch // FinalizeBatch contains data of event of FinalizeBatch @@ -48,6 +51,32 @@ func (f *CommitBatchDaV0) DAType() DAType { return f.DaType } +type CommitBatchDaV1 struct { + DaType DAType + Version uint8 + BatchIndex uint64 + ParentBatchHeader *codecv1.DABatch + SkippedL1MessageBitmap []byte + Chunks []*codecv1.DAChunkRawTx + L1Txs []*types.L1MessageTx +} + +func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *codecv1.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx) DAEntry { + return &CommitBatchDaV1{ + DaType: CommitBatchV1, + Version: version, + BatchIndex: batchIndex, + ParentBatchHeader: parentBatchHeader, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + } +} + +func (f *CommitBatchDaV1) DAType() DAType { + return f.DaType +} + type RevertBatchDA struct { DaType DAType BatchIndex uint64 diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index 182cd2ab4bad..3ce4d68cd92c 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -22,27 +22,23 @@ type DataSourceFactory struct { config Config genesisConfig *params.ChainConfig l1Client *L1Client + blobClient BlobClient db ethdb.Database } -func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, db ethdb.Database) *DataSourceFactory { +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) *DataSourceFactory { return &DataSourceFactory{ config: config, genesisConfig: genesisConfig, l1Client: l1Client, + blobClient: blobClient, db: db, } } func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { if ds.config.FetcherMode == L1RPC { - if isBernoulliByL1Height(l1height) { - return nil, errors.New("blob_data_source: not implemented") - } else { - // todo: set l1 block where l2 changes to bernoulli - var maxL1Height uint64 = 1000000000000 - return NewCalldataSource(ctx, l1height, maxL1Height, ds.l1Client, ds.db) - } + return NewCalldataBlobSource(ctx, l1height, ds.l1Client, ds.blobClient, ds.db) } else { return nil, errors.New("snapshot_data_source: not implemented") } diff --git a/rollup/da_syncer/l1_client.go b/rollup/da_syncer/l1_client.go index 3ed691be5323..8d234c68a9f3 100644 --- a/rollup/da_syncer/l1_client.go +++ b/rollup/da_syncer/l1_client.go @@ -11,6 +11,7 @@ import ( "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rpc" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) @@ -55,7 +56,6 @@ func newL1Client(ctx context.Context, genesisConfig *params.ChainConfig, l1Clien l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, } - return &client, nil } @@ -109,3 +109,41 @@ func (c *L1Client) fetchTxData(ctx context.Context, vLog *types.Log) ([]byte, er return tx.Data(), nil } + +// fetchTxBlobHash fetches tx blob hash corresponding to given event log +func (c *L1Client) fetchTxBlobHash(ctx context.Context, vLog *types.Log) (common.Hash, error) { + tx, _, err := c.client.TransactionByHash(ctx, vLog.TxHash) + if err != nil { + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := c.client.BlockByHash(ctx, vLog.BlockHash) + if err != nil { + return common.Hash{}, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return common.Hash{}, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } + } + blobHashes := tx.BlobHashes() + if len(blobHashes) == 0 { + return common.Hash{}, fmt.Errorf("transaction does not contain any blobs, tx hash: %v", vLog.TxHash.Hex()) + } + return blobHashes[0], nil +} + +func (c *L1Client) getFinalizedBlockNumber(ctx context.Context) (*big.Int, error) { + h, err := c.client.HeaderByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) + if err != nil { + return nil, err + } + return h.Number, nil +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 6a301d44f0be..06d84ed693e5 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -44,8 +44,12 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi cancel() return nil, err } - - dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, db) + blobClient, err := newBlobScanClient() + if err != nil { + cancel() + return nil, err + } + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) // todo: keep synced l1 height somewhere var syncedL1Height uint64 = l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) diff --git a/rollup/types/encoding/codecv0/codecv0.go b/rollup/types/encoding/codecv0/codecv0.go index ee979be58654..3016e4c04f9d 100644 --- a/rollup/types/encoding/codecv0/codecv0.go +++ b/rollup/types/encoding/codecv0/codecv0.go @@ -35,7 +35,7 @@ type DAChunk struct { Transactions [][]*types.TransactionData } -// DAChunk groups consecutive DABlocks with their transactions. +// DAChunkRawTx groups consecutive DABlocks with their transactions. type DAChunkRawTx struct { Blocks []*DABlock Transactions []types.Transactions diff --git a/rollup/types/encoding/codecv1/codecv1.go b/rollup/types/encoding/codecv1/codecv1.go index 7795b85b03f8..db3a0275532d 100644 --- a/rollup/types/encoding/codecv1/codecv1.go +++ b/rollup/types/encoding/codecv1/codecv1.go @@ -55,6 +55,7 @@ func init() { // CodecV1Version denotes the version of the codec. const CodecV1Version = 1 +const blockContextByteSize = 60 // DABlock represents a Data Availability Block. type DABlock struct { @@ -72,6 +73,12 @@ type DAChunk struct { Transactions [][]*types.TransactionData } +// DAChunkRawTx groups consecutive DABlocks with their transactions. +type DAChunkRawTx struct { + Blocks []*DABlock + Transactions []types.Transactions +} + // DABatch contains metadata about a batch of DAChunks. type DABatch struct { // header @@ -188,6 +195,40 @@ func (c *DAChunk) Encode() []byte { return chunkBytes } +// DecodeDAChunksRawTx takes a byte slice and decodes it into a []DAChunkRawTx. +func DecodeDAChunksRawTx(bytes [][]byte) ([]*DAChunkRawTx, error) { + var chunks []*DAChunkRawTx + for _, chunk := range bytes { + if len(chunk) < 1 { + return nil, fmt.Errorf("invalid chunk, length is less than 1") + } + + numBlocks := int(chunk[0]) + if len(chunk) < 1+numBlocks*blockContextByteSize { + return nil, fmt.Errorf("chunk size doesn't match with numBlocks, byte length of chunk: %v, expected length: %v", len(chunk), 1+numBlocks*blockContextByteSize) + } + + blocks := make([]*DABlock, numBlocks) + for i := 0; i < numBlocks; i++ { + startIdx := 1 + i*blockContextByteSize // add 1 to skip numBlocks byte + endIdx := startIdx + blockContextByteSize + block, err := DecodeDABlock(chunk[startIdx:endIdx]) + if err != nil { + return nil, err + } + blocks[i] = block + } + + var transactions []types.Transactions + + chunks = append(chunks, &DAChunkRawTx{ + Blocks: blocks, + Transactions: transactions, + }) + } + return chunks, nil +} + // Hash computes the hash of the DAChunk data. func (c *DAChunk) Hash() (common.Hash, error) { var dataBytes []byte @@ -377,6 +418,98 @@ func constructBlobPayload(chunks []*encoding.Chunk) (*kzg4844.Blob, common.Hash, return blob, blobVersionedHash, &z, nil } +func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { + blobBytes := bytesFromBlobCanonical(blob) + numChunks := int(binary.BigEndian.Uint16(blobBytes[0:2])) + if numChunks != len(chunks) { + return fmt.Errorf("blob chunk number is not same as calldata, blob num chunks: %d, calldata num chunks: %d", numChunks, len(chunks)) + } + index := 2 + for _, chunk := range chunks { + var transactions []types.Transactions + chunkSize := int(binary.BigEndian.Uint32(blobBytes[index : index+4])) + chunkBytes := blobBytes[index+4 : index+4+chunkSize] + curIndex := 0 + for _, block := range chunk.Blocks { + var blockTransactions types.Transactions + var txNum = int(block.NumTransactions - block.NumL1Messages) + for i := 0; i < txNum; i++ { + tx, nextIndex, err := getNextTx(chunkBytes, curIndex) + if err != nil { + return fmt.Errorf("couldn't decode next tx from blob bytes: %w", err) + } + curIndex = nextIndex + blockTransactions = append(blockTransactions, tx) + } + transactions = append(transactions, blockTransactions) + } + chunk.Transactions = transactions + index += 4 + chunkSize + } + return nil +} + +var errSmallLength error = fmt.Errorf("length of blob bytes is too small") + +// getNextTx parses blob bytes to find length of payload of next Tx and decode it +func getNextTx(bytes []byte, index int) (*types.Transaction, int, error) { + var nextIndex int + length := len(bytes) + if length < index+1 { + return nil, 0, errSmallLength + } + var txBytes []byte + if bytes[index] <= 0x7f { + // the first byte is transaction type, rlp encoding begins from next byte + txBytes = append(txBytes, bytes[index]) + index++ + + } + if length < index+1 { + return nil, 0, errSmallLength + } + if bytes[index] >= 0xc0 && bytes[index] <= 0xf7 { + // length of payload is simply bytes[index] - 0xc0 + payloadLen := int(bytes[index] - 0xc0) + if length < index+1+payloadLen { + return nil, 0, errSmallLength + } + txBytes = append(txBytes, bytes[index:index+1+payloadLen]...) + nextIndex = index + 1 + payloadLen + } else if bytes[index] > 0xf7 { + // the length of payload is encoded in next bytes[index] - 0xf7 bytes + // length of bytes representation of length of payload + lenPayloadLen := int(bytes[index] - 0xf7) + if length < index+1+lenPayloadLen { + return nil, 0, errSmallLength + } + lenBytes := bytes[index+1 : index+1+lenPayloadLen] + for len(lenBytes) < 8 { + lenBytes = append([]byte{0x0}, lenBytes...) + } + payloadLen := binary.BigEndian.Uint64(lenBytes) + + if length < index+1+lenPayloadLen+int(payloadLen) { + return nil, 0, errSmallLength + } + txBytes = append(txBytes, bytes[index:index+1+lenPayloadLen+int(payloadLen)]...) + nextIndex = index + 1 + lenPayloadLen + int(payloadLen) + } else { + return nil, 0, fmt.Errorf("incorrect format of rlp encoding") + } + tx := &types.Transaction{} + tx.UnmarshalBinary(txBytes) + return tx, nextIndex, nil +} + +func bytesFromBlobCanonical(blob *kzg4844.Blob) [126976]byte { + var blobBytes [126976]byte + for from := 0; from < len(blob); from += 32 { + copy(blobBytes[from*31:], blob[from+1:from+32]) + } + return blobBytes +} + // makeBlobCanonical converts the raw blob data into the canonical blob representation of 4096 BLSFieldElements. func makeBlobCanonical(blobBytes []byte) (*kzg4844.Blob, error) { // blob contains 131072 bytes but we can only utilize 31/32 of these From e4832191bb10f0d1b7a204686331de66817d19d3 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Fri, 7 Jun 2024 14:46:36 +0200 Subject: [PATCH 12/59] fixes after testing and add synced l1 height to db --- rollup/da_syncer/batch_queue.go | 39 ++++++++++++++++++++++-- rollup/da_syncer/calldata_blob_source.go | 7 +++-- rollup/da_syncer/da.go | 33 ++++++++++++++++++-- rollup/da_syncer/syncing_pipeline.go | 2 +- 4 files changed, 71 insertions(+), 10 deletions(-) diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index c75b4b7ec3f4..0e55e6d980dd 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -3,21 +3,28 @@ package da_syncer import ( "context" "fmt" + "math" + + "github.com/scroll-tech/go-ethereum/core/rawdb" + "github.com/scroll-tech/go-ethereum/ethdb" ) type BatchQueue struct { // batches is map from batchIndex to batch blocks batches map[uint64]DAEntry daQueue *DaQueue + db ethdb.Database } -func NewBatchQueue(daQueue *DaQueue) *BatchQueue { +func NewBatchQueue(daQueue *DaQueue, db ethdb.Database) *BatchQueue { return &BatchQueue{ batches: make(map[uint64]DAEntry), daQueue: daQueue, + db: db, } } +// NextBatch finds next finalized batch and returns data, that was committed in that batch func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { for { @@ -28,16 +35,42 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { switch daEntry := daEntry.(type) { case *CommitBatchDaV0: bq.batches[daEntry.BatchIndex] = daEntry + case *CommitBatchDaV1: + bq.batches[daEntry.BatchIndex] = daEntry case *RevertBatchDA: - delete(bq.batches, daEntry.BatchIndex) + bq.deleteBatch(daEntry.BatchIndex) case *FinalizeBatchDA: ret, ok := bq.batches[daEntry.BatchIndex] if !ok { - return nil, fmt.Errorf("failed to get batch data, batchIndex: %d", daEntry.BatchIndex) + // most probable, we met FinalizeBatch event for already committed batch after restart + continue } + bq.deleteBatch(daEntry.BatchIndex) return ret, nil default: return nil, fmt.Errorf("unexpected type of daEntry: %T", daEntry) } } } + +// deleteBatch deletes data committed in the batch, because this batch is reverted or finalized +// updates DASyncedL1BlockNumber +func (bq *BatchQueue) deleteBatch(batchIndex uint64) { + batch, ok := bq.batches[batchIndex] + if !ok { + return + } + curBatchL1Height := batch.GetL1BlockNumber() + delete(bq.batches, batchIndex) + if len(bq.batches) == 0 { + rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height) + return + } + var minBatchL1Height uint64 = math.MaxUint64 + for _, val := range bq.batches { + if val.GetL1BlockNumber() < minBatchL1Height { + minBatchL1Height = val.GetL1BlockNumber() + } + } + rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height-1) +} diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index ce6ae009e89a..8c5e8207841e 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -128,7 +128,7 @@ type commitBatchArgs struct { func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}), nil + return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil } txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) @@ -202,7 +202,7 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } @@ -213,6 +213,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg if err != nil { return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } + parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) if err != nil { return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) @@ -263,6 +264,6 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs) + da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 3f098a8a0783..9f134cb3da3f 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -21,6 +21,7 @@ const ( type DAEntry interface { DAType() DAType + GetL1BlockNumber() uint64 } type DA []DAEntry @@ -33,9 +34,11 @@ type CommitBatchDaV0 struct { SkippedL1MessageBitmap []byte Chunks []*codecv0.DAChunkRawTx L1Txs []*types.L1MessageTx + + L1BlockNumber uint64 } -func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx) DAEntry { +func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { return &CommitBatchDaV0{ DaType: CommitBatchV0, Version: version, @@ -44,6 +47,7 @@ func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *cod SkippedL1MessageBitmap: skippedL1MessageBitmap, Chunks: chunks, L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, } } @@ -51,6 +55,10 @@ func (f *CommitBatchDaV0) DAType() DAType { return f.DaType } +func (f *CommitBatchDaV0) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} + type CommitBatchDaV1 struct { DaType DAType Version uint8 @@ -59,9 +67,11 @@ type CommitBatchDaV1 struct { SkippedL1MessageBitmap []byte Chunks []*codecv1.DAChunkRawTx L1Txs []*types.L1MessageTx + + L1BlockNumber uint64 } -func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *codecv1.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx) DAEntry { +func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *codecv1.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { return &CommitBatchDaV1{ DaType: CommitBatchV1, Version: version, @@ -70,6 +80,7 @@ func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *cod SkippedL1MessageBitmap: skippedL1MessageBitmap, Chunks: chunks, L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, } } @@ -77,13 +88,19 @@ func (f *CommitBatchDaV1) DAType() DAType { return f.DaType } +func (f *CommitBatchDaV1) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} + type RevertBatchDA struct { DaType DAType BatchIndex uint64 + + L1BlockNumber uint64 } func NewRevertBatchDA(batchIndex uint64) DAEntry { - return &FinalizeBatchDA{ + return &RevertBatchDA{ DaType: RevertBatch, BatchIndex: batchIndex, } @@ -93,9 +110,15 @@ func (f *RevertBatchDA) DAType() DAType { return f.DaType } +func (f *RevertBatchDA) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} + type FinalizeBatchDA struct { DaType DAType BatchIndex uint64 + + L1BlockNumber uint64 } func NewFinalizeBatchDA(batchIndex uint64) DAEntry { @@ -108,3 +131,7 @@ func NewFinalizeBatchDA(batchIndex uint64) DAEntry { func (f *FinalizeBatchDA) DAType() DAType { return f.DaType } + +func (f *FinalizeBatchDA) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 06d84ed693e5..540bfc317bcc 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -57,7 +57,7 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi syncedL1Height = *from } daQueue := NewDaQueue(syncedL1Height, dataSourceFactory) - batchQueue := NewBatchQueue(daQueue) + batchQueue := NewBatchQueue(daQueue, db) blockQueue := NewBlockQueue(batchQueue) daSyncer := NewDaSyncer(blockchain) From eb377fea0ea444c9377ed1a1487bd1b678251992 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 11 Jun 2024 16:46:55 +0200 Subject: [PATCH 13/59] fixes and improvements after testing --- rollup/da_syncer/batch_queue.go | 4 +- rollup/da_syncer/blob_scan_client.go | 42 +++++++++++++------ rollup/da_syncer/block_queue.go | 53 ++++++++++++++++++++++-- rollup/da_syncer/calldata_blob_source.go | 12 ++++-- rollup/da_syncer/da_queue.go | 4 +- rollup/da_syncer/da_syncer.go | 5 ++- rollup/da_syncer/syncing_pipeline.go | 5 +-- rollup/types/encoding/codecv1/codecv1.go | 15 +++---- 8 files changed, 104 insertions(+), 36 deletions(-) diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index 0e55e6d980dd..8d103109ec3a 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -63,7 +63,7 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { curBatchL1Height := batch.GetL1BlockNumber() delete(bq.batches, batchIndex) if len(bq.batches) == 0 { - rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height) + rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) return } var minBatchL1Height uint64 = math.MaxUint64 @@ -72,5 +72,5 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { minBatchL1Height = val.GetL1BlockNumber() } } - rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height-1) + rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height-1) } diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_scan_client.go index dd7b9a657e72..66531329f4ea 100644 --- a/rollup/da_syncer/blob_scan_client.go +++ b/rollup/da_syncer/blob_scan_client.go @@ -12,44 +12,52 @@ import ( ) const ( - blobScanApiUrl string = "https://api.blobscan.com/blobs/" - okStatusCode int = 200 - lenBlobBytes int = 131072 + okStatusCode int = 200 + lenBlobBytes int = 131072 ) type BlobScanClient struct { - client *http.Client + client *http.Client + blobScanApiEndpoint string } -func newBlobScanClient() (*BlobScanClient, error) { +func newBlobScanClient(blobScanApiEndpoint string) (*BlobScanClient, error) { return &BlobScanClient{ - client: http.DefaultClient, + client: http.DefaultClient, + blobScanApiEndpoint: blobScanApiEndpoint, }, nil } func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { - // some api call - req, err := http.NewRequestWithContext(ctx, "GET", blobScanApiUrl+versionedHash.String(), nil) + req, err := http.NewRequestWithContext(ctx, "GET", c.blobScanApiEndpoint+versionedHash.String(), nil) if err != nil { - return nil, fmt.Errorf("cannot create request, err: %v", err) + return nil, fmt.Errorf("cannot create request, err: %w", err) } req.Header.Set("accept", "application/json") resp, err := c.client.Do(req) if err != nil { - return nil, fmt.Errorf("cannot do request, err: %v", err) + return nil, fmt.Errorf("cannot do request, err: %w", err) } defer resp.Body.Close() if resp.StatusCode != okStatusCode { - return nil, fmt.Errorf("response code is not ok, code: %d", resp.StatusCode) + if resp.StatusCode == 404 { + return nil, fmt.Errorf("no blob with versioned hash : %s", versionedHash.String()) + } + var res ErrorResp + err = json.NewDecoder(resp.Body).Decode(&res) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + return nil, fmt.Errorf("error while fetching blob, message: %s, code: %s, versioned hash: %s", res.Message, res.Code, versionedHash.String()) } var result BlobResp err = json.NewDecoder(resp.Body).Decode(&result) if err != nil { - return nil, fmt.Errorf("failed to decode result into struct, err: %v", err) + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) } blobBytes, err := hex.DecodeString(result.Data[2:]) if err != nil { - return nil, fmt.Errorf("failed to decode data to bytes, err: %v", err) + return nil, fmt.Errorf("failed to decode data to bytes, err: %w", err) } if len(blobBytes) != lenBlobBytes { return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) @@ -92,3 +100,11 @@ type BlobResp struct { BlobGasUsed string `json:"blobGasUsed"` } `json:"transactions"` } + +type ErrorResp struct { + Message string `json:"message"` + Code string `json:"code"` + Issues []struct { + Message string `json:"message"` + } `json:"issues"` +} diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index ca128446d71e..da32ee6700df 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -43,6 +43,11 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { if err != nil { return err } + case *CommitBatchDaV1: + bq.blocks, err = bq.processDaV1ToBlocks(daEntry) + if err != nil { + return err + } default: return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } @@ -51,7 +56,46 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Block, error) { var blocks []*types.Block - l1TxIndex := 0 + l1TxPointer := 0 + var curL1TxIndex uint64 = 0 + if daEntry.ParentBatchHeader != nil { + curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped + } + for _, chunk := range daEntry.Chunks { + for blockId, daBlock := range chunk.Blocks { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + } + // create txs + // var txs types.Transactions + txs := make(types.Transactions, 0, daBlock.NumTransactions) + // insert l1 msgs + for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) + txs = append(txs, l1Tx) + l1TxPointer++ + } + curL1TxIndex += uint64(daBlock.NumL1Messages) + // insert l2 txs + txs = append(txs, chunk.Transactions[blockId]...) + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil +} + +func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Block, error) { + var blocks []*types.Block + l1TxPointer := 0 + var curL1TxIndex uint64 = 0 + if daEntry.ParentBatchHeader != nil { + curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped + } for _, chunk := range daEntry.Chunks { for blockId, daBlock := range chunk.Blocks { // create header @@ -65,11 +109,12 @@ func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Bl // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) // insert l1 msgs - for id := 0; id < int(daBlock.NumL1Messages); id++ { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) + for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) txs = append(txs, l1Tx) - l1TxIndex++ + l1TxPointer++ } + curL1TxIndex += uint64(daBlock.NumL1Messages) // insert l2 txs txs = append(txs, chunk.Transactions[blockId]...) block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 8c5e8207841e..4d7c0b03db02 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -18,7 +18,7 @@ import ( ) var ( - callDataBlobSourceFetchBlockRange uint64 = 100 + callDataBlobSourceFetchBlockRange uint64 = 500 ) type CalldataBlobSource struct { @@ -67,8 +67,11 @@ func (ds *CalldataBlobSource) NextData() (DA, error) { if err != nil { return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) } - ds.l1height = to + 1 - return ds.processLogsToDA(logs) + da, err := ds.processLogsToDA(logs) + if err == nil { + ds.l1height = to + 1 + } + return da, err } func (ds *CalldataBlobSource) L1Height() uint64 { @@ -192,8 +195,9 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg // get all necessary l1msgs without skipped currentIndex := parentTotalL1MessagePopped for index := 0; index < int(totalL1MessagePopped); index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + if encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ + continue } l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) if l1Tx == nil { diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index bf22c45c0da5..925670eafa05 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -1,6 +1,8 @@ package da_syncer -import "context" +import ( + "context" +) type DaQueue struct { l1height uint64 diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index fe0a693f2d69..d8dee5e45642 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -26,7 +26,6 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) } - log.Info("now", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) header := block.Header() txs := block.Transactions() @@ -48,7 +47,9 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) } - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + } return nil } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 540bfc317bcc..eab1a7c43f0f 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -24,7 +24,7 @@ var ( ) // defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 1 * time.Second +const defaultSyncInterval = 1 * time.Millisecond type SyncingPipeline struct { ctx context.Context @@ -44,13 +44,12 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi cancel() return nil, err } - blobClient, err := newBlobScanClient() + blobClient, err := newBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanApiEndpoint) if err != nil { cancel() return nil, err } dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) - // todo: keep synced l1 height somewhere var syncedL1Height uint64 = l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) if from != nil { diff --git a/rollup/types/encoding/codecv1/codecv1.go b/rollup/types/encoding/codecv1/codecv1.go index db3a0275532d..d0da868b4534 100644 --- a/rollup/types/encoding/codecv1/codecv1.go +++ b/rollup/types/encoding/codecv1/codecv1.go @@ -424,11 +424,12 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { if numChunks != len(chunks) { return fmt.Errorf("blob chunk number is not same as calldata, blob num chunks: %d, calldata num chunks: %d", numChunks, len(chunks)) } - index := 2 - for _, chunk := range chunks { + index := 2 + MaxNumChunks*4 + for chunkID, chunk := range chunks { var transactions []types.Transactions - chunkSize := int(binary.BigEndian.Uint32(blobBytes[index : index+4])) - chunkBytes := blobBytes[index+4 : index+4+chunkSize] + chunkSize := int(binary.BigEndian.Uint32(blobBytes[2+4*chunkID : 2+4*chunkID+4])) + + chunkBytes := blobBytes[index : index+chunkSize] curIndex := 0 for _, block := range chunk.Blocks { var blockTransactions types.Transactions @@ -436,7 +437,7 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { for i := 0; i < txNum; i++ { tx, nextIndex, err := getNextTx(chunkBytes, curIndex) if err != nil { - return fmt.Errorf("couldn't decode next tx from blob bytes: %w", err) + return fmt.Errorf("couldn't decode next tx from blob bytes: %w, index: %d", err, index+curIndex+4) } curIndex = nextIndex blockTransactions = append(blockTransactions, tx) @@ -444,7 +445,7 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { transactions = append(transactions, blockTransactions) } chunk.Transactions = transactions - index += 4 + chunkSize + index += chunkSize } return nil } @@ -505,7 +506,7 @@ func getNextTx(bytes []byte, index int) (*types.Transaction, int, error) { func bytesFromBlobCanonical(blob *kzg4844.Blob) [126976]byte { var blobBytes [126976]byte for from := 0; from < len(blob); from += 32 { - copy(blobBytes[from*31:], blob[from+1:from+32]) + copy(blobBytes[from/32*31:], blob[from+1:from+32]) } return blobBytes } From 5a4800030528c488195060191a03a23f15e18c97 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 11 Jun 2024 16:46:55 +0200 Subject: [PATCH 14/59] fixes and improvements after testing --- params/config.go | 14 +++++++ rollup/da_syncer/batch_queue.go | 4 +- rollup/da_syncer/blob_scan_client.go | 42 +++++++++++++------ rollup/da_syncer/block_queue.go | 53 ++++++++++++++++++++++-- rollup/da_syncer/calldata_blob_source.go | 12 ++++-- rollup/da_syncer/da_queue.go | 4 +- rollup/da_syncer/da_syncer.go | 5 ++- rollup/da_syncer/syncing_pipeline.go | 5 +-- rollup/types/encoding/codecv1/codecv1.go | 15 +++---- 9 files changed, 118 insertions(+), 36 deletions(-) diff --git a/params/config.go b/params/config.go index e0e3b4147329..b7895e4b9957 100644 --- a/params/config.go +++ b/params/config.go @@ -336,6 +336,9 @@ var ( NumL1MessagesPerBlock: 10, ScrollChainAddress: common.HexToAddress("0x2D567EcE699Eabe5afCd141eDB7A4f2D0D6ce8a0"), }, + DAConfig: &DAConfig{ + BlobScanApiEndpoint: "https://api.sepolia.blobscan.com/blobs/", + }, }, } @@ -375,6 +378,9 @@ var ( NumL1MessagesPerBlock: 10, ScrollChainAddress: common.HexToAddress("0xa13BAF47339d63B743e7Da8741db5456DAc1E556"), }, + DAConfig: &DAConfig{ + BlobScanApiEndpoint: "https://api.blobscan.com/blobs/", + }, }, } @@ -535,6 +541,9 @@ type ScrollConfig struct { // L1 config L1Config *L1Config `json:"l1Config,omitempty"` + + // DA Config + DAConfig *DAConfig `json:"daConfig,omitempty"` } // L1Config contains the l1 parameters needed to sync l1 contract events (e.g., l1 messages, commit/revert/finalize batches) in the sequencer @@ -545,6 +554,11 @@ type L1Config struct { ScrollChainAddress common.Address `json:"scrollChainAddress,omitempty"` } +// DAConfig contains the parameters to sync from DA +type DAConfig struct { + BlobScanApiEndpoint string `json:"blobScanApiEndpoint,omitempty"` +} + func (c *L1Config) String() string { if c == nil { return "" diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index 0e55e6d980dd..8d103109ec3a 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -63,7 +63,7 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { curBatchL1Height := batch.GetL1BlockNumber() delete(bq.batches, batchIndex) if len(bq.batches) == 0 { - rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height) + rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) return } var minBatchL1Height uint64 = math.MaxUint64 @@ -72,5 +72,5 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { minBatchL1Height = val.GetL1BlockNumber() } } - rawdb.WriteSyncedL1BlockNumber(bq.db, curBatchL1Height-1) + rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height-1) } diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_scan_client.go index dd7b9a657e72..66531329f4ea 100644 --- a/rollup/da_syncer/blob_scan_client.go +++ b/rollup/da_syncer/blob_scan_client.go @@ -12,44 +12,52 @@ import ( ) const ( - blobScanApiUrl string = "https://api.blobscan.com/blobs/" - okStatusCode int = 200 - lenBlobBytes int = 131072 + okStatusCode int = 200 + lenBlobBytes int = 131072 ) type BlobScanClient struct { - client *http.Client + client *http.Client + blobScanApiEndpoint string } -func newBlobScanClient() (*BlobScanClient, error) { +func newBlobScanClient(blobScanApiEndpoint string) (*BlobScanClient, error) { return &BlobScanClient{ - client: http.DefaultClient, + client: http.DefaultClient, + blobScanApiEndpoint: blobScanApiEndpoint, }, nil } func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { - // some api call - req, err := http.NewRequestWithContext(ctx, "GET", blobScanApiUrl+versionedHash.String(), nil) + req, err := http.NewRequestWithContext(ctx, "GET", c.blobScanApiEndpoint+versionedHash.String(), nil) if err != nil { - return nil, fmt.Errorf("cannot create request, err: %v", err) + return nil, fmt.Errorf("cannot create request, err: %w", err) } req.Header.Set("accept", "application/json") resp, err := c.client.Do(req) if err != nil { - return nil, fmt.Errorf("cannot do request, err: %v", err) + return nil, fmt.Errorf("cannot do request, err: %w", err) } defer resp.Body.Close() if resp.StatusCode != okStatusCode { - return nil, fmt.Errorf("response code is not ok, code: %d", resp.StatusCode) + if resp.StatusCode == 404 { + return nil, fmt.Errorf("no blob with versioned hash : %s", versionedHash.String()) + } + var res ErrorResp + err = json.NewDecoder(resp.Body).Decode(&res) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + return nil, fmt.Errorf("error while fetching blob, message: %s, code: %s, versioned hash: %s", res.Message, res.Code, versionedHash.String()) } var result BlobResp err = json.NewDecoder(resp.Body).Decode(&result) if err != nil { - return nil, fmt.Errorf("failed to decode result into struct, err: %v", err) + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) } blobBytes, err := hex.DecodeString(result.Data[2:]) if err != nil { - return nil, fmt.Errorf("failed to decode data to bytes, err: %v", err) + return nil, fmt.Errorf("failed to decode data to bytes, err: %w", err) } if len(blobBytes) != lenBlobBytes { return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) @@ -92,3 +100,11 @@ type BlobResp struct { BlobGasUsed string `json:"blobGasUsed"` } `json:"transactions"` } + +type ErrorResp struct { + Message string `json:"message"` + Code string `json:"code"` + Issues []struct { + Message string `json:"message"` + } `json:"issues"` +} diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index ca128446d71e..da32ee6700df 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -43,6 +43,11 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { if err != nil { return err } + case *CommitBatchDaV1: + bq.blocks, err = bq.processDaV1ToBlocks(daEntry) + if err != nil { + return err + } default: return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } @@ -51,7 +56,46 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Block, error) { var blocks []*types.Block - l1TxIndex := 0 + l1TxPointer := 0 + var curL1TxIndex uint64 = 0 + if daEntry.ParentBatchHeader != nil { + curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped + } + for _, chunk := range daEntry.Chunks { + for blockId, daBlock := range chunk.Blocks { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + } + // create txs + // var txs types.Transactions + txs := make(types.Transactions, 0, daBlock.NumTransactions) + // insert l1 msgs + for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) + txs = append(txs, l1Tx) + l1TxPointer++ + } + curL1TxIndex += uint64(daBlock.NumL1Messages) + // insert l2 txs + txs = append(txs, chunk.Transactions[blockId]...) + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil +} + +func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Block, error) { + var blocks []*types.Block + l1TxPointer := 0 + var curL1TxIndex uint64 = 0 + if daEntry.ParentBatchHeader != nil { + curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped + } for _, chunk := range daEntry.Chunks { for blockId, daBlock := range chunk.Blocks { // create header @@ -65,11 +109,12 @@ func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Bl // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) // insert l1 msgs - for id := 0; id < int(daBlock.NumL1Messages); id++ { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxIndex]) + for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) txs = append(txs, l1Tx) - l1TxIndex++ + l1TxPointer++ } + curL1TxIndex += uint64(daBlock.NumL1Messages) // insert l2 txs txs = append(txs, chunk.Transactions[blockId]...) block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 8c5e8207841e..4d7c0b03db02 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -18,7 +18,7 @@ import ( ) var ( - callDataBlobSourceFetchBlockRange uint64 = 100 + callDataBlobSourceFetchBlockRange uint64 = 500 ) type CalldataBlobSource struct { @@ -67,8 +67,11 @@ func (ds *CalldataBlobSource) NextData() (DA, error) { if err != nil { return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) } - ds.l1height = to + 1 - return ds.processLogsToDA(logs) + da, err := ds.processLogsToDA(logs) + if err == nil { + ds.l1height = to + 1 + } + return da, err } func (ds *CalldataBlobSource) L1Height() uint64 { @@ -192,8 +195,9 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg // get all necessary l1msgs without skipped currentIndex := parentTotalL1MessagePopped for index := 0; index < int(totalL1MessagePopped); index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + if encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ + continue } l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) if l1Tx == nil { diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index bf22c45c0da5..925670eafa05 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -1,6 +1,8 @@ package da_syncer -import "context" +import ( + "context" +) type DaQueue struct { l1height uint64 diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index fe0a693f2d69..d8dee5e45642 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -26,7 +26,6 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) } - log.Info("now", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) header := block.Header() txs := block.Transactions() @@ -48,7 +47,9 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) } - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + } return nil } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 540bfc317bcc..eab1a7c43f0f 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -24,7 +24,7 @@ var ( ) // defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 1 * time.Second +const defaultSyncInterval = 1 * time.Millisecond type SyncingPipeline struct { ctx context.Context @@ -44,13 +44,12 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi cancel() return nil, err } - blobClient, err := newBlobScanClient() + blobClient, err := newBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanApiEndpoint) if err != nil { cancel() return nil, err } dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) - // todo: keep synced l1 height somewhere var syncedL1Height uint64 = l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) if from != nil { diff --git a/rollup/types/encoding/codecv1/codecv1.go b/rollup/types/encoding/codecv1/codecv1.go index db3a0275532d..d0da868b4534 100644 --- a/rollup/types/encoding/codecv1/codecv1.go +++ b/rollup/types/encoding/codecv1/codecv1.go @@ -424,11 +424,12 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { if numChunks != len(chunks) { return fmt.Errorf("blob chunk number is not same as calldata, blob num chunks: %d, calldata num chunks: %d", numChunks, len(chunks)) } - index := 2 - for _, chunk := range chunks { + index := 2 + MaxNumChunks*4 + for chunkID, chunk := range chunks { var transactions []types.Transactions - chunkSize := int(binary.BigEndian.Uint32(blobBytes[index : index+4])) - chunkBytes := blobBytes[index+4 : index+4+chunkSize] + chunkSize := int(binary.BigEndian.Uint32(blobBytes[2+4*chunkID : 2+4*chunkID+4])) + + chunkBytes := blobBytes[index : index+chunkSize] curIndex := 0 for _, block := range chunk.Blocks { var blockTransactions types.Transactions @@ -436,7 +437,7 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { for i := 0; i < txNum; i++ { tx, nextIndex, err := getNextTx(chunkBytes, curIndex) if err != nil { - return fmt.Errorf("couldn't decode next tx from blob bytes: %w", err) + return fmt.Errorf("couldn't decode next tx from blob bytes: %w, index: %d", err, index+curIndex+4) } curIndex = nextIndex blockTransactions = append(blockTransactions, tx) @@ -444,7 +445,7 @@ func DecodeTxsFromBlob(blob *kzg4844.Blob, chunks []*DAChunkRawTx) error { transactions = append(transactions, blockTransactions) } chunk.Transactions = transactions - index += 4 + chunkSize + index += chunkSize } return nil } @@ -505,7 +506,7 @@ func getNextTx(bytes []byte, index int) (*types.Transaction, int, error) { func bytesFromBlobCanonical(blob *kzg4844.Blob) [126976]byte { var blobBytes [126976]byte for from := 0; from < len(blob); from += 32 { - copy(blobBytes[from*31:], blob[from+1:from+32]) + copy(blobBytes[from/32*31:], blob[from+1:from+32]) } return blobBytes } From fd6d899ad1d3249dcec012d7124b7e0ec7a26d57 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 11 Jun 2024 23:40:28 +0200 Subject: [PATCH 15/59] update codec version --- go.mod | 2 +- go.sum | 2 ++ rollup/da_syncer/calldata_blob_source.go | 10 +++++----- rollup/da_syncer/da.go | 4 ++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index 92ffd5b23362..a8228e009918 100644 --- a/go.mod +++ b/go.mod @@ -50,7 +50,7 @@ require ( github.com/prometheus/tsdb v0.7.1 github.com/rjeczalik/notify v0.9.1 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.0.0-20240605080813-32bfc9fccde7 + github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23 github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 diff --git a/go.sum b/go.sum index a6255b577a02..05bed2eec62a 100644 --- a/go.sum +++ b/go.sum @@ -394,6 +394,8 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/scroll-tech/da-codec v0.0.0-20240605080813-32bfc9fccde7 h1:CDrPMqifvAVyYqu0x1J5qickVV0b51tApPnOwDYLESI= github.com/scroll-tech/da-codec v0.0.0-20240605080813-32bfc9fccde7/go.mod h1:1wWYii0OPwd5kw+xrz0PFgS420xNadrNF1x/ELJT+TM= +github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23 h1:T2ImyyPVdtc3S6pAVwSd/QER3g/pEo4luetgOFDnrNI= +github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23/go.mod h1:D6XEESeNVJkQJlv3eK+FyR+ufPkgVQbJzERylQi53Bs= github.com/scroll-tech/zktrie v0.8.4 h1:UagmnZ4Z3ITCk+aUq9NQZJNAwnWl4gSxsLb2Nl7IgRE= github.com/scroll-tech/zktrie v0.8.4/go.mod h1:XvNo7vAk8yxNyTjBDj5WIiFzYW4bx/gJ78+NK6Zn6Uk= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 4d7c0b03db02..c170e389610f 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -5,6 +5,9 @@ import ( "crypto/sha256" "fmt" + "github.com/scroll-tech/da-codec/encoding" + "github.com/scroll-tech/da-codec/encoding/codecv0" + "github.com/scroll-tech/da-codec/encoding/codecv1" "github.com/scroll-tech/go-ethereum/accounts/abi" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core/rawdb" @@ -12,9 +15,6 @@ import ( "github.com/scroll-tech/go-ethereum/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" - "github.com/scroll-tech/go-ethereum/rollup/types/encoding" - "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" - "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) var ( @@ -159,9 +159,9 @@ func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Lo return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } switch args.Version { - case codecv0.CodecV0Version: + case 0: return ds.decodeDAV0(batchIndex, vLog, &args) - case codecv1.CodecV1Version: + case 1: return ds.decodeDAV1(batchIndex, vLog, &args) 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.go b/rollup/da_syncer/da.go index 9f134cb3da3f..676b1c3fffd6 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -1,9 +1,9 @@ package da_syncer import ( + "github.com/scroll-tech/da-codec/encoding/codecv0" + "github.com/scroll-tech/da-codec/encoding/codecv1" "github.com/scroll-tech/go-ethereum/core/types" - "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv0" - "github.com/scroll-tech/go-ethereum/rollup/types/encoding/codecv1" ) type DAType int From 9b929ff9b525b5a6b96036713a447004df62c7be Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 11 Jun 2024 23:46:31 +0200 Subject: [PATCH 16/59] lint --- miner/worker.go | 6 +++--- params/config.go | 5 +++-- rollup/da_syncer/calldata_blob_source.go | 4 ++-- rollup/da_syncer/data_source.go | 4 ---- rollup/da_syncer/syncing_pipeline.go | 5 ----- 5 files changed, 8 insertions(+), 16 deletions(-) diff --git a/miner/worker.go b/miner/worker.go index 7cef7e3e5c02..152e74d8fab0 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -556,8 +556,8 @@ func (w *worker) mainLoop() { for { select { - case req := <-w.newWorkCh: - w.commitNewWork(req.interrupt, req.noempty, req.timestamp) + case _ = <-w.newWorkCh: + // w.commitNewWork(req.interrupt, req.noempty, req.timestamp) // new block created. case ev := <-w.chainSideCh: @@ -632,7 +632,7 @@ func (w *worker) mainLoop() { // submit mining work here since all empty submission will be rejected // by clique. Of course the advance sealing(empty submission) is disabled. if w.chainConfig.Clique != nil && w.chainConfig.Clique.Period == 0 { - w.commitNewWork(nil, true, time.Now().Unix()) + // w.commitNewWork(nil, true, time.Now().Unix()) } } atomic.AddInt32(&w.newTxs, int32(len(ev.Txs))) diff --git a/params/config.go b/params/config.go index b7895e4b9957..e6070d9a8d42 100644 --- a/params/config.go +++ b/params/config.go @@ -616,8 +616,9 @@ func (c *EthashConfig) String() string { // CliqueConfig is the consensus engine configs for proof-of-authority based sealing. type CliqueConfig struct { - Period uint64 `json:"period"` // Number of seconds between blocks to enforce - Epoch uint64 `json:"epoch"` // Epoch length to reset votes and checkpoint + Period uint64 `json:"period"` // Number of seconds between blocks to enforce + Epoch uint64 `json:"epoch"` // Epoch length to reset votes and checkpoint + RelaxedPeriod bool `json:"relaxed_period"` // Relaxes the period to be just an upper bound } // String implements the stringer interface, returning the consensus engine details. diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index c170e389610f..2e46e5d8e6be 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -194,7 +194,7 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg } // get all necessary l1msgs without skipped currentIndex := parentTotalL1MessagePopped - for index := 0; index < int(totalL1MessagePopped); index++ { + for index := 0; index < totalL1MessagePopped; index++ { if encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ continue @@ -257,7 +257,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg } // get all necessary l1msgs without skipped currentIndex := parentTotalL1MessagePopped - for index := 0; index < int(totalL1MessagePopped); index++ { + for index := 0; index < totalL1MessagePopped; index++ { for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { currentIndex++ } diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index 3ce4d68cd92c..d3ee9f26ab2d 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -43,7 +43,3 @@ func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64 return nil, errors.New("snapshot_data_source: not implemented") } } - -func isBernoulliByL1Height(l1height uint64) bool { - return false -} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index eab1a7c43f0f..0c11f463c379 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -2,7 +2,6 @@ package da_syncer import ( "context" - "errors" "time" "github.com/scroll-tech/go-ethereum/core" @@ -19,10 +18,6 @@ type Config struct { SnapshotFilePath string // path to snapshot file } -var ( - errInvalidChain = errors.New("retrieved hash chain is invalid") -) - // defaultSyncInterval is the frequency at which we query for new rollup event. const defaultSyncInterval = 1 * time.Millisecond From 9b1395921dc1d37b79faf8f74ebfdbf3fb6d725f Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 17 Jun 2024 10:44:07 +0200 Subject: [PATCH 17/59] update codec dependency --- go.mod | 3 +- go.sum | 8 +-- rollup/da_syncer/batch_queue.go | 2 + rollup/da_syncer/block_queue.go | 41 +++++++++++++++ rollup/da_syncer/calldata_blob_source.go | 65 ++++++++++++++++++++++++ rollup/da_syncer/da.go | 36 +++++++++++++ 6 files changed, 150 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index a8228e009918..170272ae4d93 100644 --- a/go.mod +++ b/go.mod @@ -50,7 +50,7 @@ require ( github.com/prometheus/tsdb v0.7.1 github.com/rjeczalik/notify v0.9.1 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23 + github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0 github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 @@ -84,6 +84,7 @@ require ( github.com/go-sourcemap/sourcemap v2.1.3+incompatible // indirect github.com/gotestyourself/gotestyourself v1.4.0 // indirect github.com/influxdata/line-protocol v0.0.0-20210311194329-9aa0e372d097 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/kylelemons/godebug v1.1.0 // indirect github.com/mattn/go-ieproxy v0.0.0-20190702010315-6dee0af9227d // indirect github.com/mattn/go-runewidth v0.0.15 // indirect diff --git a/go.sum b/go.sum index 05bed2eec62a..035c6a5d7bd0 100644 --- a/go.sum +++ b/go.sum @@ -278,6 +278,8 @@ github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQL github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= @@ -392,10 +394,8 @@ github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncj github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/scroll-tech/da-codec v0.0.0-20240605080813-32bfc9fccde7 h1:CDrPMqifvAVyYqu0x1J5qickVV0b51tApPnOwDYLESI= -github.com/scroll-tech/da-codec v0.0.0-20240605080813-32bfc9fccde7/go.mod h1:1wWYii0OPwd5kw+xrz0PFgS420xNadrNF1x/ELJT+TM= -github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23 h1:T2ImyyPVdtc3S6pAVwSd/QER3g/pEo4luetgOFDnrNI= -github.com/scroll-tech/da-codec v0.1.1-0.20240611213627-a523bcec1b23/go.mod h1:D6XEESeNVJkQJlv3eK+FyR+ufPkgVQbJzERylQi53Bs= +github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0 h1:1yE2Y/NQhEQZ42pTi+PCY8EzVuIcOy02IqUOAyHDcNQ= +github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= github.com/scroll-tech/zktrie v0.8.4 h1:UagmnZ4Z3ITCk+aUq9NQZJNAwnWl4gSxsLb2Nl7IgRE= github.com/scroll-tech/zktrie v0.8.4/go.mod h1:XvNo7vAk8yxNyTjBDj5WIiFzYW4bx/gJ78+NK6Zn6Uk= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index 8d103109ec3a..38b6e7870e4a 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -37,6 +37,8 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { bq.batches[daEntry.BatchIndex] = daEntry case *CommitBatchDaV1: bq.batches[daEntry.BatchIndex] = daEntry + case *CommitBatchDaV2: + bq.batches[daEntry.BatchIndex] = daEntry case *RevertBatchDA: bq.deleteBatch(daEntry.BatchIndex) case *FinalizeBatchDA: diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index da32ee6700df..0f14947afb76 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -48,6 +48,11 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { if err != nil { return err } + case *CommitBatchDaV2: + bq.blocks, err = bq.processDaV2ToBlocks(daEntry) + if err != nil { + return err + } default: return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } @@ -123,3 +128,39 @@ func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Bl } return blocks, nil } + +func (bq *BlockQueue) processDaV2ToBlocks(daEntry *CommitBatchDaV2) ([]*types.Block, error) { + var blocks []*types.Block + l1TxPointer := 0 + var curL1TxIndex uint64 = 0 + if daEntry.ParentBatchHeader != nil { + curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped + } + for _, chunk := range daEntry.Chunks { + for blockId, daBlock := range chunk.Blocks { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + } + // create txs + // var txs types.Transactions + txs := make(types.Transactions, 0, daBlock.NumTransactions) + // insert l1 msgs + for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) + txs = append(txs, l1Tx) + l1TxPointer++ + } + curL1TxIndex += uint64(daBlock.NumL1Messages) + // insert l2 txs + txs = append(txs, chunk.Transactions[blockId]...) + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil +} + diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 2e46e5d8e6be..5c554ca93ce3 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -8,6 +8,7 @@ import ( "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" "github.com/scroll-tech/da-codec/encoding/codecv1" + "github.com/scroll-tech/da-codec/encoding/codecv2" "github.com/scroll-tech/go-ethereum/accounts/abi" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core/rawdb" @@ -163,6 +164,8 @@ func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Lo return ds.decodeDAV0(batchIndex, vLog, &args) case 1: return ds.decodeDAV1(batchIndex, vLog, &args) + case 2: + return ds.decodeDAV2(batchIndex, vLog, &args) default: return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) } @@ -271,3 +274,65 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } + +func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { + var chunks []*codecv2.DAChunkRawTx + var l1Txs []*types.L1MessageTx + chunks, err := codecv2.DecodeDAChunksRawTx(args.Chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + + parentBatchHeader, err := codecv2.NewDABatchFromBytes(args.ParentBatchHeader) + if err != nil { + return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) + } + versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) + } + blob, err := ds.blobClient.GetBlobByVersionedHash(ds.ctx, versionedHash) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) + } + // compute blob versioned hash and compare with one from tx + c, err := kzg4844.BlobToCommitment(blob) + if err != nil { + return nil, fmt.Errorf("failed to create blob commitment") + } + blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) + } + // decode txs from blob + err = codecv2.DecodeTxsFromBlob(blob, chunks) + if err != nil { + return nil, fmt.Errorf("failed to decode txs from blob: %w", err) + } + parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + totalL1MessagePopped := 0 + for _, chunk := range chunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) + } + // get all necessary l1msgs without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < totalL1MessagePopped; index++ { + for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + } + l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) + if l1Tx == nil { + return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) + } + l1Txs = append(l1Txs, l1Tx) + currentIndex++ + } + da := NewCommitBatchDaV2(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + return da, nil +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 676b1c3fffd6..7dcdca42b9c8 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -3,6 +3,7 @@ package da_syncer import ( "github.com/scroll-tech/da-codec/encoding/codecv0" "github.com/scroll-tech/da-codec/encoding/codecv1" + "github.com/scroll-tech/da-codec/encoding/codecv2" "github.com/scroll-tech/go-ethereum/core/types" ) @@ -13,6 +14,8 @@ const ( CommitBatchV0 DAType = iota // CommitBatchV1 contains data of event of CommitBatchV1 CommitBatchV1 + // CommitBatchV2 contains data of event of CommitBatchV2 + CommitBatchV2 // RevertBatch contains data of event of RevertBatch RevertBatch // FinalizeBatch contains data of event of FinalizeBatch @@ -92,6 +95,39 @@ func (f *CommitBatchDaV1) GetL1BlockNumber() uint64 { return f.L1BlockNumber } +type CommitBatchDaV2 struct { + DaType DAType + Version uint8 + BatchIndex uint64 + ParentBatchHeader *codecv2.DABatch + SkippedL1MessageBitmap []byte + Chunks []*codecv2.DAChunkRawTx + L1Txs []*types.L1MessageTx + + L1BlockNumber uint64 +} + +func NewCommitBatchDaV2(version uint8, batchIndex uint64, parentBatchHeader *codecv2.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { + return &CommitBatchDaV2{ + DaType: CommitBatchV2, + Version: version, + BatchIndex: batchIndex, + ParentBatchHeader: parentBatchHeader, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, + } +} + +func (f *CommitBatchDaV2) DAType() DAType { + return f.DaType +} + +func (f *CommitBatchDaV2) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} + type RevertBatchDA struct { DaType DAType BatchIndex uint64 From bf1d02e819945d67bd89bd81a086cb472e5f7941 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 17 Jun 2024 10:48:06 +0200 Subject: [PATCH 18/59] goimports --- rollup/da_syncer/block_queue.go | 1 - rollup/da_syncer/calldata_blob_source.go | 1 + rollup/da_syncer/da.go | 1 + 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 0f14947afb76..62021f3cd67a 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -163,4 +163,3 @@ func (bq *BlockQueue) processDaV2ToBlocks(daEntry *CommitBatchDaV2) ([]*types.Bl } return blocks, nil } - diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 5c554ca93ce3..fcf1d345502d 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -9,6 +9,7 @@ import ( "github.com/scroll-tech/da-codec/encoding/codecv0" "github.com/scroll-tech/da-codec/encoding/codecv1" "github.com/scroll-tech/da-codec/encoding/codecv2" + "github.com/scroll-tech/go-ethereum/accounts/abi" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core/rawdb" diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 7dcdca42b9c8..0283e764ec4c 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -4,6 +4,7 @@ import ( "github.com/scroll-tech/da-codec/encoding/codecv0" "github.com/scroll-tech/da-codec/encoding/codecv1" "github.com/scroll-tech/da-codec/encoding/codecv2" + "github.com/scroll-tech/go-ethereum/core/types" ) From bc9b360ad5a7a5b8109850fb37731813b75862f4 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 18 Jun 2024 11:35:25 +0200 Subject: [PATCH 19/59] small fixes --- miner/worker.go | 6 +- params/config.go | 142 +++++++++++++++++++++++++--- rollup/sync_service/sync_service.go | 4 +- 3 files changed, 134 insertions(+), 18 deletions(-) diff --git a/miner/worker.go b/miner/worker.go index 152e74d8fab0..7cef7e3e5c02 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -556,8 +556,8 @@ func (w *worker) mainLoop() { for { select { - case _ = <-w.newWorkCh: - // w.commitNewWork(req.interrupt, req.noempty, req.timestamp) + case req := <-w.newWorkCh: + w.commitNewWork(req.interrupt, req.noempty, req.timestamp) // new block created. case ev := <-w.chainSideCh: @@ -632,7 +632,7 @@ func (w *worker) mainLoop() { // submit mining work here since all empty submission will be rejected // by clique. Of course the advance sealing(empty submission) is disabled. if w.chainConfig.Clique != nil && w.chainConfig.Clique.Period == 0 { - // w.commitNewWork(nil, true, time.Now().Unix()) + w.commitNewWork(nil, true, time.Now().Unix()) } } atomic.AddInt32(&w.newTxs, int32(len(ev.Txs))) diff --git a/params/config.go b/params/config.go index 6c51d81b3cd1..02ed43f82241 100644 --- a/params/config.go +++ b/params/config.go @@ -319,7 +319,7 @@ var ( ArchimedesBlock: big.NewInt(0), ShanghaiBlock: big.NewInt(0), BernoulliBlock: big.NewInt(3747132), - CurieBlock: nil, + CurieBlock: big.NewInt(4740239), DescartesBlock: nil, Clique: &CliqueConfig{ Period: 3, @@ -389,13 +389,42 @@ var ( // // This configuration is intentionally not using keyed fields to force anyone // adding flags to the config to also have to set these fields. - AllEthashProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, new(EthashConfig), nil, - ScrollConfig{ + AllEthashProtocolChanges = &ChainConfig{ + ChainID: big.NewInt(1337), + HomesteadBlock: big.NewInt(0), + DAOForkBlock: nil, + DAOForkSupport: false, + EIP150Block: big.NewInt(0), + EIP150Hash: common.Hash{}, + EIP155Block: big.NewInt(0), + EIP158Block: big.NewInt(0), + ByzantiumBlock: big.NewInt(0), + ConstantinopleBlock: big.NewInt(0), + PetersburgBlock: big.NewInt(0), + IstanbulBlock: big.NewInt(0), + MuirGlacierBlock: big.NewInt(0), + BerlinBlock: big.NewInt(0), + LondonBlock: big.NewInt(0), + ArrowGlacierBlock: big.NewInt(0), + ArchimedesBlock: big.NewInt(0), + ShanghaiBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DescartesBlock: big.NewInt(0), + TerminalTotalDifficulty: nil, + Ethash: new(EthashConfig), + Clique: nil, + Scroll: ScrollConfig{ UseZktrie: false, FeeVaultAddress: nil, MaxTxPerBlock: nil, MaxTxPayloadBytesPerBlock: nil, - L1Config: &L1Config{5, common.HexToAddress("0x0000000000000000000000000000000000000000"), 0, common.HexToAddress("0x0000000000000000000000000000000000000000")}, + L1Config: &L1Config{ + L1ChainId: 5, + L1MessageQueueAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + NumL1MessagesPerBlock: 0, + ScrollChainAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + }, }} // AllCliqueProtocolChanges contains every protocol change (EIPs) introduced @@ -403,32 +432,119 @@ var ( // // This configuration is intentionally not using keyed fields to force anyone // adding flags to the config to also have to set these fields. - AllCliqueProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, &CliqueConfig{Period: 0, Epoch: 30000}, - ScrollConfig{ + AllCliqueProtocolChanges = &ChainConfig{ + ChainID: big.NewInt(1337), + HomesteadBlock: big.NewInt(0), + DAOForkBlock: nil, + DAOForkSupport: false, + EIP150Block: big.NewInt(0), + EIP150Hash: common.Hash{}, + EIP155Block: big.NewInt(0), + EIP158Block: big.NewInt(0), + ByzantiumBlock: big.NewInt(0), + ConstantinopleBlock: big.NewInt(0), + PetersburgBlock: big.NewInt(0), + IstanbulBlock: big.NewInt(0), + MuirGlacierBlock: big.NewInt(0), + BerlinBlock: big.NewInt(0), + LondonBlock: big.NewInt(0), + ArrowGlacierBlock: big.NewInt(0), + ArchimedesBlock: big.NewInt(0), + ShanghaiBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DescartesBlock: big.NewInt(0), + TerminalTotalDifficulty: nil, + Ethash: nil, + Clique: &CliqueConfig{Period: 0, Epoch: 30000}, + Scroll: ScrollConfig{ UseZktrie: false, FeeVaultAddress: nil, MaxTxPerBlock: nil, MaxTxPayloadBytesPerBlock: nil, - L1Config: &L1Config{5, common.HexToAddress("0x0000000000000000000000000000000000000000"), 0, common.HexToAddress("0x0000000000000000000000000000000000000000")}, + L1Config: &L1Config{ + L1ChainId: 5, + L1MessageQueueAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + NumL1MessagesPerBlock: 0, + ScrollChainAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + }, }} - TestChainConfig = &ChainConfig{big.NewInt(1), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, new(EthashConfig), nil, - ScrollConfig{ + TestChainConfig = &ChainConfig{ + ChainID: big.NewInt(1), + HomesteadBlock: big.NewInt(0), + DAOForkBlock: nil, + DAOForkSupport: false, + EIP150Block: big.NewInt(0), + EIP150Hash: common.Hash{}, + EIP155Block: big.NewInt(0), + EIP158Block: big.NewInt(0), + ByzantiumBlock: big.NewInt(0), + ConstantinopleBlock: big.NewInt(0), + PetersburgBlock: big.NewInt(0), + IstanbulBlock: big.NewInt(0), + MuirGlacierBlock: big.NewInt(0), + BerlinBlock: big.NewInt(0), + LondonBlock: big.NewInt(0), + ArrowGlacierBlock: big.NewInt(0), + ArchimedesBlock: big.NewInt(0), + ShanghaiBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DescartesBlock: big.NewInt(0), + TerminalTotalDifficulty: nil, + Ethash: new(EthashConfig), + Clique: nil, + Scroll: ScrollConfig{ UseZktrie: false, FeeVaultAddress: &common.Address{123}, MaxTxPerBlock: nil, MaxTxPayloadBytesPerBlock: nil, - L1Config: &L1Config{5, common.HexToAddress("0x0000000000000000000000000000000000000000"), 0, common.HexToAddress("0x0000000000000000000000000000000000000000")}, + L1Config: &L1Config{ + L1ChainId: 5, + L1MessageQueueAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + NumL1MessagesPerBlock: 0, + ScrollChainAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + }, }} TestRules = TestChainConfig.Rules(new(big.Int)) - TestNoL1DataFeeChainConfig = &ChainConfig{big.NewInt(1), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, new(EthashConfig), nil, - ScrollConfig{ + TestNoL1DataFeeChainConfig = &ChainConfig{ + ChainID: big.NewInt(1), + HomesteadBlock: big.NewInt(0), + DAOForkBlock: nil, + DAOForkSupport: false, + EIP150Block: big.NewInt(0), + EIP150Hash: common.Hash{}, + EIP155Block: big.NewInt(0), + EIP158Block: big.NewInt(0), + ByzantiumBlock: big.NewInt(0), + ConstantinopleBlock: big.NewInt(0), + PetersburgBlock: big.NewInt(0), + IstanbulBlock: big.NewInt(0), + MuirGlacierBlock: big.NewInt(0), + BerlinBlock: big.NewInt(0), + LondonBlock: big.NewInt(0), + ArrowGlacierBlock: big.NewInt(0), + ArchimedesBlock: big.NewInt(0), + ShanghaiBlock: big.NewInt(0), + BernoulliBlock: big.NewInt(0), + CurieBlock: big.NewInt(0), + DescartesBlock: big.NewInt(0), + TerminalTotalDifficulty: nil, + Ethash: new(EthashConfig), + Clique: nil, + Scroll: ScrollConfig{ UseZktrie: false, FeeVaultAddress: nil, MaxTxPerBlock: nil, MaxTxPayloadBytesPerBlock: nil, - L1Config: &L1Config{5, common.HexToAddress("0x0000000000000000000000000000000000000000"), 0, common.HexToAddress("0x0000000000000000000000000000000000000000")}, + L1Config: &L1Config{ + L1ChainId: 5, + L1MessageQueueAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + NumL1MessagesPerBlock: 0, + ScrollChainAddress: common.HexToAddress("0x0000000000000000000000000000000000000000"), + }, }} ) diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 8138abd5a1b8..091f2d19691f 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -102,7 +102,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - // s.fetchMessages() + s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -112,7 +112,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - // s.fetchMessages() + s.fetchMessages() select { case <-s.ctx.Done(): From e55416ed203736c241f898fa7306de399feed8c9 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 18 Jun 2024 14:29:46 +0200 Subject: [PATCH 20/59] fix lint, turn off handler, worker, miner during da syncing --- consensus/clique/clique.go | 122 ++++++++++++++------------- eth/backend.go | 37 +++++--- miner/miner.go | 10 ++- miner/miner_test.go | 2 +- miner/scroll_worker.go | 6 +- miner/scroll_worker_test.go | 2 +- params/config.go | 7 +- rollup/da_syncer/da_syncer.go | 2 - rollup/da_syncer/syncing_pipeline.go | 7 +- rollup/sync_service/sync_service.go | 4 +- 10 files changed, 114 insertions(+), 85 deletions(-) diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index d71334a058a5..efe10a2e389f 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -145,28 +145,28 @@ type SignerFn func(signer accounts.Account, mimeType string, message []byte) ([] // ecrecover extracts the Ethereum account address from a signed header. func ecrecover(header *types.Header, sigcache *lru.ARCCache) (common.Address, error) { - return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil - // // If the signature's already cached, return that - // hash := header.Hash() - // if address, known := sigcache.Get(hash); known { - // return address.(common.Address), nil - // } - // // Retrieve the signature from the header extra-data - // if len(header.Extra) < extraSeal { - // return common.Address{}, errMissingSignature - // } - // signature := header.Extra[len(header.Extra)-extraSeal:] - - // // Recover the public key and the Ethereum address - // pubkey, err := crypto.Ecrecover(SealHash(header).Bytes(), signature) - // if err != nil { - // return common.Address{}, err - // } - // var signer common.Address - // copy(signer[:], crypto.Keccak256(pubkey[1:])[12:]) - - // sigcache.Add(hash, signer) - // return signer, nil + // return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil + // If the signature's already cached, return that + hash := header.Hash() + if address, known := sigcache.Get(hash); known { + return address.(common.Address), nil + } + // Retrieve the signature from the header extra-data + if len(header.Extra) < extraSeal { + return common.Address{}, errMissingSignature + } + signature := header.Extra[len(header.Extra)-extraSeal:] + + // Recover the public key and the Ethereum address + pubkey, err := crypto.Ecrecover(SealHash(header).Bytes(), signature) + if err != nil { + return common.Address{}, err + } + var signer common.Address + copy(signer[:], crypto.Keccak256(pubkey[1:])[12:]) + + sigcache.Add(hash, signer) + return signer, nil } // Clique is the proof-of-authority consensus engine proposed to support the @@ -212,8 +212,10 @@ func New(config *params.CliqueConfig, db ethdb.Database) *Clique { // Author implements consensus.Engine, returning the Ethereum address recovered // from the signature in the header's extra-data section. func (c *Clique) Author(header *types.Header) (common.Address, error) { - return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil - // return ecrecover(header, c.signatures) + if c.config.DaSyncingEnabled { + return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil + } + return ecrecover(header, c.signatures) } // VerifyHeader checks whether a header conforms to the consensus rules. @@ -268,21 +270,23 @@ func (c *Clique) verifyHeader(chain consensus.ChainHeaderReader, header *types.H if checkpoint && !bytes.Equal(header.Nonce[:], nonceDropVote) { return errInvalidCheckpointVote } - // // Check that the extra-data contains both the vanity and signature - // if len(header.Extra) < extraVanity { - // return errMissingVanity - // } - // if len(header.Extra) < extraVanity+extraSeal { - // return errMissingSignature - // } - // // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise - // signersBytes := len(header.Extra) - extraVanity - extraSeal - // if !checkpoint && signersBytes != 0 { - // return errExtraSigners - // } - // if checkpoint && signersBytes%common.AddressLength != 0 { - // return errInvalidCheckpointSigners - // } + if !c.config.DaSyncingEnabled { + // Check that the extra-data contains both the vanity and signature + if len(header.Extra) < extraVanity { + return errMissingVanity + } + if len(header.Extra) < extraVanity+extraSeal { + return errMissingSignature + } + // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise + signersBytes := len(header.Extra) - extraVanity - extraSeal + if !checkpoint && signersBytes != 0 { + return errExtraSigners + } + if checkpoint && signersBytes%common.AddressLength != 0 { + return errInvalidCheckpointSigners + } + } // Ensure that the mix digest is zero as we don't have fork protection currently if header.MixDigest != (common.Hash{}) { return errInvalidMixDigest @@ -349,25 +353,27 @@ func (c *Clique) verifyCascadingFields(chain consensus.ChainHeaderReader, header // Verify the header's EIP-1559 attributes. return err } - // // Retrieve the snapshot needed to verify this header and cache it - // snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) - // if err != nil { - // return err - // } - // // If the block is a checkpoint block, verify the signer list - // if number%c.config.Epoch == 0 { - // signers := make([]byte, len(snap.Signers)*common.AddressLength) - // for i, signer := range snap.signers() { - // copy(signers[i*common.AddressLength:], signer[:]) - // } - // extraSuffix := len(header.Extra) - extraSeal - // if !bytes.Equal(header.Extra[extraVanity:extraSuffix], signers) { - // return errMismatchingCheckpointSigners - // } - // } - // // All basic checks passed, verify the seal and return - // return c.verifySeal(snap, header, parents) - return nil + if c.config.DaSyncingEnabled { + return nil + } + // Retrieve the snapshot needed to verify this header and cache it + snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) + if err != nil { + return err + } + // If the block is a checkpoint block, verify the signer list + if number%c.config.Epoch == 0 { + signers := make([]byte, len(snap.Signers)*common.AddressLength) + for i, signer := range snap.signers() { + copy(signers[i*common.AddressLength:], signer[:]) + } + extraSuffix := len(header.Extra) - extraSeal + if !bytes.Equal(header.Extra[extraVanity:extraSuffix], signers) { + return errMismatchingCheckpointSigners + } + } + // All basic checks passed, verify the seal and return + return c.verifySeal(snap, header, parents) } // snapshot retrieves the authorization snapshot at a given point in time. diff --git a/eth/backend.go b/eth/backend.go index 3ac75a429a72..88f44a4df9a4 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -143,6 +143,9 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl if _, ok := genesisErr.(*params.ConfigCompatError); genesisErr != nil && !ok { return nil, genesisErr } + if chainConfig.Clique != nil && config.EnableDASyncing { + chainConfig.Clique.DaSyncingEnabled = true + } log.Info("Initialised chain configuration", "config", chainConfig) if err := pruner.RecoverPruning(stack.ResolvePath(""), chainDb, stack.ResolvePath(config.TrieCleanCacheJournal)); err != nil { @@ -264,7 +267,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl } } - eth.miner = miner.New(eth, &config.Miner, chainConfig, eth.EventMux(), eth.engine, eth.isLocalBlock) + eth.miner = miner.New(eth, &config.Miner, chainConfig, eth.EventMux(), eth.engine, eth.isLocalBlock, config.EnableDASyncing) eth.miner.SetExtra(makeExtraData(config.Miner.ExtraData)) eth.APIBackend = &EthAPIBackend{stack.Config().ExtRPCEnabled(), stack.Config().AllowUnprotectedTxs, eth, nil} @@ -294,7 +297,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl // Register the backend on the node stack.RegisterAPIs(eth.APIs()) - // stack.RegisterProtocols(eth.Protocols()) + stack.RegisterProtocols(eth.Protocols()) stack.RegisterLifecycle(eth) // Check for unclean shutdown if uncleanShutdowns, discards, err := rawdb.PushUncleanShutdownMarker(chainDb); err != nil { @@ -336,6 +339,14 @@ func (s *Ethereum) APIs() []rpc.API { // Append any APIs exposed explicitly by the consensus engine apis = append(apis, s.engine.APIs(s.BlockChain())...) + if s.handler != nil { + apis = append(apis, rpc.API{ + Namespace: "eth", + Version: "1.0", + Service: downloader.NewPublicDownloaderAPI(s.handler.downloader, s.eventMux), + Public: true, + }) + } // Append all the local APIs and return return append(apis, []rpc.API{ @@ -349,12 +360,7 @@ func (s *Ethereum) APIs() []rpc.API { Version: "1.0", Service: NewPublicMinerAPI(s), Public: true, - }, /*{ - Namespace: "eth", - Version: "1.0", - Service: downloader.NewPublicDownloaderAPI(s.handler.downloader, s.eventMux), - Public: true, - },*/{ + }, { Namespace: "miner", Version: "1.0", Service: NewPrivateMinerAPI(s), @@ -520,7 +526,7 @@ func (s *Ethereum) StartMining(threads int) error { } // If mining is started, we can disable the transaction rejection mechanism // introduced to speed sync times. - // atomic.StoreUint32(&s.handler.acceptTxs, 1) + atomic.StoreUint32(&s.handler.acceptTxs, 1) go s.miner.Start(eb) } @@ -560,6 +566,9 @@ func (s *Ethereum) SyncService() *sync_service.SyncService { return s.syncServic // Protocols returns all the currently configured // network protocols to start. func (s *Ethereum) Protocols() []p2p.Protocol { + if s.handler == nil { + return nil + } protos := eth.MakeProtocols((*ethHandler)(s.handler), s.networkID, s.ethDialCandidates) if !s.blockchain.Config().Scroll.ZktrieEnabled() && s.config.SnapshotCache > 0 { protos = append(protos, snap.MakeProtocols((*snapHandler)(s.handler), s.snapDialCandidates)...) @@ -576,7 +585,7 @@ func (s *Ethereum) Start() error { s.startBloomHandlers(params.BloomBitsBlocks) // Figure out a max peers count based on the server limits - // maxPeers := s.p2pServer.MaxPeers + maxPeers := s.p2pServer.MaxPeers //if s.config.LightServ > 0 { // if s.config.LightPeers >= s.p2pServer.MaxPeers { // return fmt.Errorf("invalid peer config: light peer count (%d) >= total peer count (%d)", s.config.LightPeers, s.p2pServer.MaxPeers) @@ -584,7 +593,9 @@ func (s *Ethereum) Start() error { // maxPeers -= s.config.LightPeers //} // Start the networking layer and the light server if requested - // s.handler.Start(maxPeers) + if s.handler != nil { + s.handler.Start(maxPeers) + } return nil } @@ -594,7 +605,9 @@ func (s *Ethereum) Stop() error { // Stop all the peer-related stuff first. s.ethDialCandidates.Close() s.snapDialCandidates.Close() - // s.handler.Stop() + if s.handler != nil { + s.handler.Stop() + } // Then stop everything else. s.bloomIndexer.Close() diff --git a/miner/miner.go b/miner/miner.go index 4c79ef4d182e..bc2eac138f3f 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -75,7 +75,7 @@ type Miner struct { wg sync.WaitGroup } -func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(block *types.Block) bool) *Miner { +func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(block *types.Block) bool, daSyncingEnabled bool) *Miner { miner := &Miner{ eth: eth, mux: mux, @@ -83,10 +83,12 @@ func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *even exitCh: make(chan struct{}), startCh: make(chan common.Address), stopCh: make(chan struct{}), - worker: newWorker(config, chainConfig, engine, eth, mux, isLocalBlock, true), + worker: newWorker(config, chainConfig, engine, eth, mux, isLocalBlock, true, daSyncingEnabled), + } + if !daSyncingEnabled { + miner.wg.Add(1) + go miner.update() } - miner.wg.Add(1) - go miner.update() return miner } diff --git a/miner/miner_test.go b/miner/miner_test.go index 56f6c1f2da88..3824c3625738 100644 --- a/miner/miner_test.go +++ b/miner/miner_test.go @@ -271,5 +271,5 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux) { // Create event Mux mux := new(event.TypeMux) // Create Miner - return New(backend, &config, chainConfig, mux, engine, nil), mux + return New(backend, &config, chainConfig, mux, engine, nil, false), mux } diff --git a/miner/scroll_worker.go b/miner/scroll_worker.go index 749f290fb9c5..0319c79bd31e 100644 --- a/miner/scroll_worker.go +++ b/miner/scroll_worker.go @@ -135,7 +135,7 @@ type worker struct { beforeTxHook func() // Method to call before processing a transaction. } -func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(*types.Block) bool, init bool) *worker { +func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(*types.Block) bool, init bool, daSyncingEnabled bool) *worker { worker := &worker{ config: config, chainConfig: chainConfig, @@ -151,6 +151,10 @@ func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus circuitCapacityChecker: circuitcapacitychecker.NewCircuitCapacityChecker(true), } log.Info("created new worker", "CircuitCapacityChecker ID", worker.circuitCapacityChecker.ID) + if daSyncingEnabled { + log.Info("worker will not start, because DA syncing is enabled") + return worker + } // Subscribe NewTxsEvent for tx pool worker.txsSub = eth.TxPool().SubscribeNewTxsEvent(worker.txsCh) diff --git a/miner/scroll_worker_test.go b/miner/scroll_worker_test.go index f5feed40ab2e..002a4dffd36d 100644 --- a/miner/scroll_worker_test.go +++ b/miner/scroll_worker_test.go @@ -207,7 +207,7 @@ func (b *testWorkerBackend) newRandomTx(creation bool) *types.Transaction { func newTestWorker(t *testing.T, chainConfig *params.ChainConfig, engine consensus.Engine, db ethdb.Database, blocks int) (*worker, *testWorkerBackend) { backend := newTestWorkerBackend(t, chainConfig, engine, db, blocks) backend.txPool.AddLocals(pendingTxs) - w := newWorker(testConfig, chainConfig, engine, backend, new(event.TypeMux), nil, false) + w := newWorker(testConfig, chainConfig, engine, backend, new(event.TypeMux), nil, false, false) w.setEtherbase(testBankAddress) return w, backend } diff --git a/params/config.go b/params/config.go index 02ed43f82241..7cf246715b34 100644 --- a/params/config.go +++ b/params/config.go @@ -732,9 +732,10 @@ func (c *EthashConfig) String() string { // CliqueConfig is the consensus engine configs for proof-of-authority based sealing. type CliqueConfig struct { - Period uint64 `json:"period"` // Number of seconds between blocks to enforce - Epoch uint64 `json:"epoch"` // Epoch length to reset votes and checkpoint - RelaxedPeriod bool `json:"relaxed_period"` // Relaxes the period to be just an upper bound + Period uint64 `json:"period"` // Number of seconds between blocks to enforce + Epoch uint64 `json:"epoch"` // Epoch length to reset votes and checkpoint + RelaxedPeriod bool `json:"relaxed_period"` // Relaxes the period to be just an upper bound + DaSyncingEnabled bool `json:"da_syncing_enabled"` // Is daSyncingEnabled } // String implements the stringer interface, returning the consensus engine details. diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index d8dee5e45642..5115dcd0c067 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -26,7 +26,6 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) } - header := block.Header() txs := block.Transactions() @@ -47,7 +46,6 @@ func (s *DaSyncer) SyncOneBlock(block *types.Block) error { if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) } - if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 0c11f463c379..53e57fab9265 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -2,6 +2,7 @@ package da_syncer import ( "context" + "strings" "time" "github.com/scroll-tech/go-ethereum/core" @@ -88,7 +89,11 @@ func (sp *SyncingPipeline) Start() { for { err := sp.Step(sp.ctx) if err != nil { - log.Warn("syncing pipeline step failed", "err", err) + if strings.HasPrefix(err.Error(), "not consecutive block") { + log.Warn("syncing pipeline step failed, probably because of restart", "err", err) + } else { + log.Crit("syncing pipeline step failed", "err", err) + } } select { case <-sp.ctx.Done(): diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 091f2d19691f..8138abd5a1b8 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -102,7 +102,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - s.fetchMessages() + // s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -112,7 +112,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - s.fetchMessages() + // s.fetchMessages() select { case <-s.ctx.Done(): From 1ff0973a15dd29370ae9ac784d99848344fb179e Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 2 Jul 2024 16:38:55 +0200 Subject: [PATCH 21/59] remove comments --- rollup/sync_service/sync_service.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rollup/sync_service/sync_service.go b/rollup/sync_service/sync_service.go index 8138abd5a1b8..091f2d19691f 100644 --- a/rollup/sync_service/sync_service.go +++ b/rollup/sync_service/sync_service.go @@ -102,7 +102,7 @@ func (s *SyncService) Start() { latestConfirmed, err := s.client.getLatestConfirmedBlockNumber(s.ctx) if err == nil && latestConfirmed > s.latestProcessedBlock+1000 { log.Warn("Running initial sync of L1 messages before starting l2geth, this might take a while...") - // s.fetchMessages() + s.fetchMessages() log.Info("L1 message initial sync completed", "latestProcessedBlock", s.latestProcessedBlock) } @@ -112,7 +112,7 @@ func (s *SyncService) Start() { for { // don't wait for ticker during startup - // s.fetchMessages() + s.fetchMessages() select { case <-s.ctx.Done(): From b3726ff1e25f721980c91845f184b12f2a40cb28 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 8 Jul 2024 10:51:28 +0200 Subject: [PATCH 22/59] support blocknative --- cmd/geth/main.go | 1 + cmd/utils/flags.go | 9 ++++ eth/ethconfig/config.go | 1 + params/config.go | 4 +- rollup/da_syncer/blob_client.go | 54 +++++++++++++++++++ rollup/da_syncer/blob_scan_client.go | 18 +++---- rollup/da_syncer/block_native_client.go | 70 +++++++++++++++++++++++++ rollup/da_syncer/syncing_pipeline.go | 14 +++-- 8 files changed, 156 insertions(+), 15 deletions(-) create mode 100644 rollup/da_syncer/block_native_client.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 8b9c47f5eede..24158daeb9ad 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -172,6 +172,7 @@ var ( utils.DASyncEnabledFlag, utils.DAModeFlag, utils.DASnapshotFileFlag, + utils.DABlobSourceFlag, } rpcFlags = []cli.Flag{ diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 81ccda786f85..7451fec99420 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -871,6 +871,12 @@ var ( Usage: `Da sync sync mode ("l1rpc" or "snapshot")`, Value: &defaultDa, } + defaultBlobSource = ethconfig.Defaults.DA.BLobSource + DABlobSourceFlag = TextMarshalerFlag{ + Name: "da.blob.source", + Usage: `Blob data source, currently supported "blobscan" or "blocknative"`, + Value: &defaultBlobSource, + } DASnapshotFileFlag = cli.StringFlag{ Name: "da.snapshot.file", Usage: "Snapshot file to sync from da", @@ -1601,6 +1607,9 @@ func setDa(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.GlobalIsSet(DASnapshotFileFlag.Name) { cfg.DA.SnapshotFilePath = ctx.GlobalString(DASnapshotFileFlag.Name) } + if ctx.GlobalIsSet(DABlobSourceFlag.Name) { + cfg.DA.BLobSource = *GlobalTextMarshaler(ctx, DABlobSourceFlag.Name).(*da_syncer.BLobSource) + } } } diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 8a4498398ec8..94a1f17d143b 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -96,6 +96,7 @@ var Defaults = Config{ MaxBlockRange: -1, // Default unconfigured value: no block range limit for backward compatibility DA: da_syncer.Config{ FetcherMode: da_syncer.L1RPC, + BLobSource: da_syncer.BlobScan, }, } diff --git a/params/config.go b/params/config.go index 7cf246715b34..84013ea017ae 100644 --- a/params/config.go +++ b/params/config.go @@ -380,6 +380,7 @@ var ( }, DAConfig: &DAConfig{ BlobScanApiEndpoint: "https://api.blobscan.com/blobs/", + BlockNativeApiEndpoint: "https://api.ethernow.xyz/v1/blob/", }, }, } @@ -672,7 +673,8 @@ type L1Config struct { // DAConfig contains the parameters to sync from DA type DAConfig struct { - BlobScanApiEndpoint string `json:"blobScanApiEndpoint,omitempty"` + BlobScanApiEndpoint string `json:"blobScanApiEndpoint,omitempty"` + BlockNativeApiEndpoint string `json:"blockNativeApiEndpoint,omitempty"` } func (c *L1Config) String() string { diff --git a/rollup/da_syncer/blob_client.go b/rollup/da_syncer/blob_client.go index 9e3043dd96d2..ac718d2e92a5 100644 --- a/rollup/da_syncer/blob_client.go +++ b/rollup/da_syncer/blob_client.go @@ -2,11 +2,65 @@ package da_syncer import ( "context" + "fmt" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" ) +const ( + okStatusCode int = 200 + lenBlobBytes int = 131072 +) + type BlobClient interface { GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) } + +type BLobSource int + +const ( + // BlobScan + BlobScan BLobSource = iota + // BlockNative + BlockNative +) + +func (src BLobSource) IsValid() bool { + return src >= BlobScan && src <= BlockNative +} + +// String implements the stringer interface. +func (src BLobSource) String() string { + switch src { + case BlobScan: + return "blobscan" + case BlockNative: + return "blocknative" + default: + return "unknown" + } +} + +func (src BLobSource) MarshalText() ([]byte, error) { + switch src { + case BlobScan: + return []byte("blobscan"), nil + case BlockNative: + return []byte("blocknative"), nil + default: + return nil, fmt.Errorf("unknown blob source %d", src) + } +} + +func (src *BLobSource) UnmarshalText(text []byte) error { + switch string(text) { + case "blobscan": + *src = BlobScan + case "blocknative": + *src = BlockNative + default: + return fmt.Errorf(`unknown blob source %q, want "blobscan" or "blocknative"`, text) + } + return nil +} diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_scan_client.go index 66531329f4ea..c65841aa7367 100644 --- a/rollup/da_syncer/blob_scan_client.go +++ b/rollup/da_syncer/blob_scan_client.go @@ -11,21 +11,16 @@ import ( "github.com/scroll-tech/go-ethereum/crypto/kzg4844" ) -const ( - okStatusCode int = 200 - lenBlobBytes int = 131072 -) - type BlobScanClient struct { client *http.Client blobScanApiEndpoint string } -func newBlobScanClient(blobScanApiEndpoint string) (*BlobScanClient, error) { +func newBlobScanClient(blobScanApiEndpoint string) *BlobScanClient { return &BlobScanClient{ client: http.DefaultClient, blobScanApiEndpoint: blobScanApiEndpoint, - }, nil + } } func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { @@ -43,14 +38,15 @@ func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHa if resp.StatusCode == 404 { return nil, fmt.Errorf("no blob with versioned hash : %s", versionedHash.String()) } - var res ErrorResp + var res ErrorRespBlobScan err = json.NewDecoder(resp.Body).Decode(&res) if err != nil { return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) } return nil, fmt.Errorf("error while fetching blob, message: %s, code: %s, versioned hash: %s", res.Message, res.Code, versionedHash.String()) } - var result BlobResp + var result BlobRespBlobScan + err = json.NewDecoder(resp.Body).Decode(&result) if err != nil { return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) @@ -66,7 +62,7 @@ func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHa return &blob, nil } -type BlobResp struct { +type BlobRespBlobScan struct { Commitment string `json:"commitment"` Proof string `json:"proof"` Size int `json:"size"` @@ -101,7 +97,7 @@ type BlobResp struct { } `json:"transactions"` } -type ErrorResp struct { +type ErrorRespBlobScan struct { Message string `json:"message"` Code string `json:"code"` Issues []struct { diff --git a/rollup/da_syncer/block_native_client.go b/rollup/da_syncer/block_native_client.go new file mode 100644 index 000000000000..84bf00e5413e --- /dev/null +++ b/rollup/da_syncer/block_native_client.go @@ -0,0 +1,70 @@ +package da_syncer + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "net/http" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" +) + +type BlockNativeClient struct { + blockNativeApiEndpoint string +} + +func newBlockNativeClient(blockNativeApiEndpoint string) *BlockNativeClient { + return &BlockNativeClient{ + blockNativeApiEndpoint: blockNativeApiEndpoint, + } +} + +func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { + resp, err := http.Get(c.blockNativeApiEndpoint+versionedHash.String()) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %w", err) + } + defer resp.Body.Close() + if resp.StatusCode != okStatusCode { + var res ErrorRespBlockNative + err = json.NewDecoder(resp.Body).Decode(&res) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + return nil, fmt.Errorf("error while fetching blob, message: %s, code: %d, versioned hash: %s", res.Error.Message, res.Error.Code, versionedHash.String()) + } + var result BlobRespBlockNative + err = json.NewDecoder(resp.Body).Decode(&result) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + blobBytes, err := hex.DecodeString(result.Blob.Data[2:]) + if err != nil { + return nil, fmt.Errorf("failed to decode data to bytes, err: %w", err) + } + if len(blobBytes) != lenBlobBytes { + return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) + } + blob := kzg4844.Blob(blobBytes) + return &blob, nil +} + +type BlobRespBlockNative struct { + Blob struct { + VersionedHash string `json:"versionedHash"` + Commitment string `json:"commitment"` + Proof string `json:"proof"` + ZeroBytes int `json:"zeroBytes"` + NonZeroBytes int `json:"nonZeroBytes"` + Data string `json:"data"` + } `json:"blob"` +} + +type ErrorRespBlockNative struct { + Error struct { + Code int `json:"code"` + Message string `json:"message"` + } `json:"error"` +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 53e57fab9265..18d47ef37dd6 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -2,6 +2,7 @@ package da_syncer import ( "context" + "fmt" "strings" "time" @@ -17,6 +18,7 @@ import ( type Config struct { FetcherMode FetcherMode // mode of fetcher SnapshotFilePath string // path to snapshot file + BLobSource BLobSource // blob source } // defaultSyncInterval is the frequency at which we query for new rollup event. @@ -40,11 +42,17 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi cancel() return nil, err } - blobClient, err := newBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanApiEndpoint) - if err != nil { + var blobClient BlobClient + switch config.BLobSource { + case BlobScan: + blobClient = newBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanApiEndpoint) + case BlockNative: + blobClient = newBlockNativeClient(genesisConfig.Scroll.DAConfig.BlockNativeApiEndpoint) + default: cancel() - return nil, err + return nil, fmt.Errorf("unknown blob scan client: %d", config.BLobSource) } + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) var syncedL1Height uint64 = l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) From 6e0ccdbe64d4c8daed76a7d882498df5f658cc0c Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 9 Jul 2024 13:59:52 +0200 Subject: [PATCH 23/59] fix --- go.mod | 2 +- go.sum | 2 ++ rollup/da_syncer/da.go | 24 ++++++++++++++++++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 170272ae4d93..aea7d762976c 100644 --- a/go.mod +++ b/go.mod @@ -50,7 +50,7 @@ require ( github.com/prometheus/tsdb v0.7.1 github.com/rjeczalik/notify v0.9.1 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0 + github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 diff --git a/go.sum b/go.sum index 035c6a5d7bd0..edcb6a0b5361 100644 --- a/go.sum +++ b/go.sum @@ -396,6 +396,8 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0 h1:1yE2Y/NQhEQZ42pTi+PCY8EzVuIcOy02IqUOAyHDcNQ= github.com/scroll-tech/da-codec v0.1.1-0.20240614180230-4d863639aab0/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= +github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f h1:ZKPhn674+2AgBdIn2ZLGePsUZdM2823m2tJp+JlQf/Y= +github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= github.com/scroll-tech/zktrie v0.8.4 h1:UagmnZ4Z3ITCk+aUq9NQZJNAwnWl4gSxsLb2Nl7IgRE= github.com/scroll-tech/zktrie v0.8.4/go.mod h1:XvNo7vAk8yxNyTjBDj5WIiFzYW4bx/gJ78+NK6Zn6Uk= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 0283e764ec4c..83b87a4980c7 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -21,6 +21,8 @@ const ( RevertBatch // FinalizeBatch contains data of event of FinalizeBatch FinalizeBatch + // FinalizeBatchV3 contains data of event of FinalizeBatch v3 + FinalizeBatchV3 ) type DAEntry interface { @@ -172,3 +174,25 @@ func (f *FinalizeBatchDA) DAType() DAType { func (f *FinalizeBatchDA) GetL1BlockNumber() uint64 { return f.L1BlockNumber } + +type FinalizeBatchDAV3 struct { + DaType DAType + BatchIndex uint64 + + L1BlockNumber uint64 +} + +func NewFinalizeBatchDAV3(batchIndex uint64) DAEntry { + return &FinalizeBatchDAV3{ + DaType: FinalizeBatchV3, + BatchIndex: batchIndex, + } +} + +func (f *FinalizeBatchDAV3) DAType() DAType { + return f.DaType +} + +func (f *FinalizeBatchDAV3) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} From 49c3c156301ecf74583a19e59d8d26addffd0a5c Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Tue, 9 Jul 2024 14:01:15 +0200 Subject: [PATCH 24/59] fix lint --- eth/ethconfig/config.go | 2 +- params/config.go | 2 +- rollup/da_syncer/block_native_client.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 94a1f17d143b..0a3ea43b7b75 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -96,7 +96,7 @@ var Defaults = Config{ MaxBlockRange: -1, // Default unconfigured value: no block range limit for backward compatibility DA: da_syncer.Config{ FetcherMode: da_syncer.L1RPC, - BLobSource: da_syncer.BlobScan, + BLobSource: da_syncer.BlobScan, }, } diff --git a/params/config.go b/params/config.go index a3df6868d9be..01e2cdd31dfe 100644 --- a/params/config.go +++ b/params/config.go @@ -379,7 +379,7 @@ var ( ScrollChainAddress: common.HexToAddress("0xa13BAF47339d63B743e7Da8741db5456DAc1E556"), }, DAConfig: &DAConfig{ - BlobScanApiEndpoint: "https://api.blobscan.com/blobs/", + BlobScanApiEndpoint: "https://api.blobscan.com/blobs/", BlockNativeApiEndpoint: "https://api.ethernow.xyz/v1/blob/", }, }, diff --git a/rollup/da_syncer/block_native_client.go b/rollup/da_syncer/block_native_client.go index 84bf00e5413e..f9cbb7819163 100644 --- a/rollup/da_syncer/block_native_client.go +++ b/rollup/da_syncer/block_native_client.go @@ -22,7 +22,7 @@ func newBlockNativeClient(blockNativeApiEndpoint string) *BlockNativeClient { } func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { - resp, err := http.Get(c.blockNativeApiEndpoint+versionedHash.String()) + resp, err := http.Get(c.blockNativeApiEndpoint + versionedHash.String()) if err != nil { return nil, fmt.Errorf("cannot do request, err: %w", err) } From e825a4243e679c1d63265deb36f04abe56a44219 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Sun, 14 Jul 2024 14:55:23 +0200 Subject: [PATCH 25/59] fix bug with decoding [parentbatch --- rollup/da_syncer/block_queue.go | 15 +--- rollup/da_syncer/calldata_blob_source.go | 31 +++----- rollup/da_syncer/da.go | 96 ++++++++++++------------ 3 files changed, 63 insertions(+), 79 deletions(-) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 62021f3cd67a..b4ea08e6e8e3 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -62,10 +62,7 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 - var curL1TxIndex uint64 = 0 - if daEntry.ParentBatchHeader != nil { - curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped - } + var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped for _, chunk := range daEntry.Chunks { for blockId, daBlock := range chunk.Blocks { // create header @@ -97,10 +94,7 @@ func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Bl func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 - var curL1TxIndex uint64 = 0 - if daEntry.ParentBatchHeader != nil { - curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped - } + var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped for _, chunk := range daEntry.Chunks { for blockId, daBlock := range chunk.Blocks { // create header @@ -132,10 +126,7 @@ func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Bl func (bq *BlockQueue) processDaV2ToBlocks(daEntry *CommitBatchDaV2) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 - var curL1TxIndex uint64 = 0 - if daEntry.ParentBatchHeader != nil { - curL1TxIndex = daEntry.ParentBatchHeader.TotalL1MessagePopped - } + var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped for _, chunk := range daEntry.Chunks { for blockId, daBlock := range chunk.Blocks { // create header diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index fcf1d345502d..901122147c65 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -3,6 +3,7 @@ package da_syncer import ( "context" "crypto/sha256" + "encoding/binary" "fmt" "github.com/scroll-tech/da-codec/encoding" @@ -133,7 +134,7 @@ type commitBatchArgs struct { func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, nil, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil + return NewCommitBatchDaV0(0, batchIndex, 0, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil } txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) @@ -180,12 +181,8 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg if err != nil { return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - parentBatchHeader, err := codecv0.NewDABatchFromBytes(args.ParentBatchHeader) - if err != nil { - return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) - } - parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) totalL1MessagePopped := 0 for _, chunk := range chunks { for _, block := range chunk.Blocks { @@ -210,7 +207,7 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDaV0(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } @@ -222,10 +219,6 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - parentBatchHeader, err := codecv1.NewDABatchFromBytes(args.ParentBatchHeader) - if err != nil { - return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) - } versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) if err != nil { return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) @@ -248,7 +241,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg if err != nil { return nil, fmt.Errorf("failed to decode txs from blob: %w", err) } - parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) totalL1MessagePopped := 0 for _, chunk := range chunks { for _, block := range chunk.Blocks { @@ -272,7 +265,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV1(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDaV1(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } @@ -284,10 +277,6 @@ func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, arg return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - parentBatchHeader, err := codecv2.NewDABatchFromBytes(args.ParentBatchHeader) - if err != nil { - return nil, fmt.Errorf("failed to decode batch bytes into batch, values: %v, err: %w", args.ParentBatchHeader, err) - } versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) if err != nil { return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) @@ -310,7 +299,7 @@ func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, arg if err != nil { return nil, fmt.Errorf("failed to decode txs from blob: %w", err) } - parentTotalL1MessagePopped := parentBatchHeader.TotalL1MessagePopped + parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) totalL1MessagePopped := 0 for _, chunk := range chunks { for _, block := range chunk.Blocks { @@ -334,6 +323,10 @@ func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV2(args.Version, batchIndex, parentBatchHeader, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDaV2(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } + +func getBatchTotalL1MessagePopped(data []byte) uint64 { + return binary.BigEndian.Uint64(data[17:25]) +} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index 83b87a4980c7..c8daa74f42f4 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -33,27 +33,27 @@ type DAEntry interface { type DA []DAEntry type CommitBatchDaV0 struct { - DaType DAType - Version uint8 - BatchIndex uint64 - ParentBatchHeader *codecv0.DABatch - SkippedL1MessageBitmap []byte - Chunks []*codecv0.DAChunkRawTx - L1Txs []*types.L1MessageTx + DaType DAType + Version uint8 + BatchIndex uint64 + ParentTotalL1MessagePopped uint64 + SkippedL1MessageBitmap []byte + Chunks []*codecv0.DAChunkRawTx + L1Txs []*types.L1MessageTx L1BlockNumber uint64 } -func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentBatchHeader *codecv0.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { +func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { return &CommitBatchDaV0{ - DaType: CommitBatchV0, - Version: version, - BatchIndex: batchIndex, - ParentBatchHeader: parentBatchHeader, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, + DaType: CommitBatchV0, + Version: version, + BatchIndex: batchIndex, + ParentTotalL1MessagePopped: parentTotalL1MessagePopped, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, } } @@ -66,27 +66,27 @@ func (f *CommitBatchDaV0) GetL1BlockNumber() uint64 { } type CommitBatchDaV1 struct { - DaType DAType - Version uint8 - BatchIndex uint64 - ParentBatchHeader *codecv1.DABatch - SkippedL1MessageBitmap []byte - Chunks []*codecv1.DAChunkRawTx - L1Txs []*types.L1MessageTx + DaType DAType + Version uint8 + BatchIndex uint64 + ParentTotalL1MessagePopped uint64 + SkippedL1MessageBitmap []byte + Chunks []*codecv1.DAChunkRawTx + L1Txs []*types.L1MessageTx L1BlockNumber uint64 } -func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentBatchHeader *codecv1.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { +func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { return &CommitBatchDaV1{ - DaType: CommitBatchV1, - Version: version, - BatchIndex: batchIndex, - ParentBatchHeader: parentBatchHeader, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, + DaType: CommitBatchV1, + Version: version, + BatchIndex: batchIndex, + ParentTotalL1MessagePopped: parentTotalL1MessagePopped, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, } } @@ -99,27 +99,27 @@ func (f *CommitBatchDaV1) GetL1BlockNumber() uint64 { } type CommitBatchDaV2 struct { - DaType DAType - Version uint8 - BatchIndex uint64 - ParentBatchHeader *codecv2.DABatch - SkippedL1MessageBitmap []byte - Chunks []*codecv2.DAChunkRawTx - L1Txs []*types.L1MessageTx + DaType DAType + Version uint8 + BatchIndex uint64 + ParentTotalL1MessagePopped uint64 + SkippedL1MessageBitmap []byte + Chunks []*codecv2.DAChunkRawTx + L1Txs []*types.L1MessageTx L1BlockNumber uint64 } -func NewCommitBatchDaV2(version uint8, batchIndex uint64, parentBatchHeader *codecv2.DABatch, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { +func NewCommitBatchDaV2(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { return &CommitBatchDaV2{ - DaType: CommitBatchV2, - Version: version, - BatchIndex: batchIndex, - ParentBatchHeader: parentBatchHeader, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, + DaType: CommitBatchV2, + Version: version, + BatchIndex: batchIndex, + ParentTotalL1MessagePopped: parentTotalL1MessagePopped, + SkippedL1MessageBitmap: skippedL1MessageBitmap, + Chunks: chunks, + L1Txs: l1Txs, + L1BlockNumber: l1BlockNumber, } } From 750c615bea3d2c69bcc12dffd2b3e422bbd5a111 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 15 Jul 2024 12:14:46 +0200 Subject: [PATCH 26/59] support codec v3 --- rollup/da_syncer/abi.go | 2 +- rollup/da_syncer/batch_queue.go | 53 ++++++++++++++++++------ rollup/da_syncer/calldata_blob_source.go | 52 ++++++++++++++++------- 3 files changed, 78 insertions(+), 29 deletions(-) diff --git a/rollup/da_syncer/abi.go b/rollup/da_syncer/abi.go index 21603628a658..8b85f322ff9a 100644 --- a/rollup/da_syncer/abi.go +++ b/rollup/da_syncer/abi.go @@ -12,7 +12,7 @@ import ( // scrollChainMetaData contains ABI of the ScrollChain contract. var scrollChainMetaData = &bind.MetaData{ - ABI: "[{\"inputs\":[{\"internalType\":\"uint64\",\"name\":\"_chainId\",\"type\":\"uint64\"}],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"}],\"name\":\"CommitBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"},{\"indexed\":false,\"internalType\":\"bytes32\",\"name\":\"stateRoot\",\"type\":\"bytes32\"},{\"indexed\":false,\"internalType\":\"bytes32\",\"name\":\"withdrawRoot\",\"type\":\"bytes32\"}],\"name\":\"FinalizeBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint8\",\"name\":\"version\",\"type\":\"uint8\"}],\"name\":\"Initialized\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"previousOwner\",\"type\":\"address\"},{\"indexed\":true,\"internalType\":\"address\",\"name\":\"newOwner\",\"type\":\"address\"}],\"name\":\"OwnershipTransferred\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"}],\"name\":\"Paused\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"batchIndex\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"bytes32\",\"name\":\"batchHash\",\"type\":\"bytes32\"}],\"name\":\"RevertBatch\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"}],\"name\":\"Unpaused\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint256\",\"name\":\"oldMaxNumTxInChunk\",\"type\":\"uint256\"},{\"indexed\":false,\"internalType\":\"uint256\",\"name\":\"newMaxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"UpdateMaxNumTxInChunk\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"},{\"indexed\":false,\"internalType\":\"bool\",\"name\":\"status\",\"type\":\"bool\"}],\"name\":\"UpdateProver\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"account\",\"type\":\"address\"},{\"indexed\":false,\"internalType\":\"bool\",\"name\":\"status\",\"type\":\"bool\"}],\"name\":\"UpdateSequencer\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"address\",\"name\":\"oldVerifier\",\"type\":\"address\"},{\"indexed\":true,\"internalType\":\"address\",\"name\":\"newVerifier\",\"type\":\"address\"}],\"name\":\"UpdateVerifier\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"addProver\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"addSequencer\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint8\",\"name\":\"_version\",\"type\":\"uint8\"},{\"internalType\":\"bytes\",\"name\":\"_parentBatchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes[]\",\"name\":\"_chunks\",\"type\":\"bytes[]\"},{\"internalType\":\"bytes\",\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\"}],\"name\":\"commitBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"committedBatches\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes32\",\"name\":\"_prevStateRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes32\",\"name\":\"_postStateRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes32\",\"name\":\"_withdrawRoot\",\"type\":\"bytes32\"},{\"internalType\":\"bytes\",\"name\":\"_aggrProof\",\"type\":\"bytes\"}],\"name\":\"finalizeBatchWithProof\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"finalizedStateRoots\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"bytes32\",\"name\":\"_stateRoot\",\"type\":\"bytes32\"}],\"name\":\"importGenesisBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_messageQueue\",\"type\":\"address\"},{\"internalType\":\"address\",\"name\":\"_verifier\",\"type\":\"address\"},{\"internalType\":\"uint256\",\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"initialize\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"_batchIndex\",\"type\":\"uint256\"}],\"name\":\"isBatchFinalized\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"name\":\"isProver\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"name\":\"isSequencer\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"lastFinalizedBatchIndex\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"layer2ChainId\",\"outputs\":[{\"internalType\":\"uint64\",\"name\":\"\",\"type\":\"uint64\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"maxNumTxInChunk\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"messageQueue\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"owner\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"paused\",\"outputs\":[{\"internalType\":\"bool\",\"name\":\"\",\"type\":\"bool\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"removeProver\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_account\",\"type\":\"address\"}],\"name\":\"removeSequencer\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"renounceOwnership\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bytes\",\"name\":\"_batchHeader\",\"type\":\"bytes\"},{\"internalType\":\"uint256\",\"name\":\"_count\",\"type\":\"uint256\"}],\"name\":\"revertBatch\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"bool\",\"name\":\"_status\",\"type\":\"bool\"}],\"name\":\"setPause\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"newOwner\",\"type\":\"address\"}],\"name\":\"transferOwnership\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\"}],\"name\":\"updateMaxNumTxInChunk\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"address\",\"name\":\"_newVerifier\",\"type\":\"address\"}],\"name\":\"updateVerifier\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[],\"name\":\"verifier\",\"outputs\":[{\"internalType\":\"address\",\"name\":\"\",\"type\":\"address\"}],\"stateMutability\":\"view\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"withdrawRoots\",\"outputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"stateMutability\":\"view\",\"type\":\"function\"}]", + ABI: "[{\"type\":\"constructor\",\"inputs\":[{\"name\":\"_chainId\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_messageQueue\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_verifier\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addProver\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addSequencer\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"commitBatch\",\"inputs\":[{\"name\":\"_version\",\"type\":\"uint8\",\"internalType\":\"uint8\"},{\"name\":\"_parentBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_chunks\",\"type\":\"bytes[]\",\"internalType\":\"bytes[]\"},{\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"commitBatchWithBlobProof\",\"inputs\":[{\"name\":\"_version\",\"type\":\"uint8\",\"internalType\":\"uint8\"},{\"name\":\"_parentBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_chunks\",\"type\":\"bytes[]\",\"internalType\":\"bytes[]\"},{\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_blobDataProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"committedBatches\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"finalizeBatchWithProof4844\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_postStateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_withdrawRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_blobDataProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_aggrProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"finalizeBundleWithProof\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_postStateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_withdrawRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_aggrProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"finalizedStateRoots\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"importGenesisBatch\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_stateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"initialize\",\"inputs\":[{\"name\":\"_messageQueue\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_verifier\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"isBatchFinalized\",\"inputs\":[{\"name\":\"_batchIndex\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"isProver\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"isSequencer\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"lastFinalizedBatchIndex\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"layer2ChainId\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"maxNumTxInChunk\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"messageQueue\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"owner\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"paused\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"removeProver\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"removeSequencer\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"renounceOwnership\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"revertBatch\",\"inputs\":[{\"name\":\"_firstBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_lastBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"setPause\",\"inputs\":[{\"name\":\"_status\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"transferOwnership\",\"inputs\":[{\"name\":\"newOwner\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"updateMaxNumTxInChunk\",\"inputs\":[{\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"verifier\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"withdrawRoots\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"event\",\"name\":\"CommitBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"FinalizeBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"},{\"name\":\"stateRoot\",\"type\":\"bytes32\",\"indexed\":false,\"internalType\":\"bytes32\"},{\"name\":\"withdrawRoot\",\"type\":\"bytes32\",\"indexed\":false,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Initialized\",\"inputs\":[{\"name\":\"version\",\"type\":\"uint8\",\"indexed\":false,\"internalType\":\"uint8\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"OwnershipTransferred\",\"inputs\":[{\"name\":\"previousOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"newOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Paused\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":false,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"RevertBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Unpaused\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":false,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateMaxNumTxInChunk\",\"inputs\":[{\"name\":\"oldMaxNumTxInChunk\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"},{\"name\":\"newMaxNumTxInChunk\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateProver\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"status\",\"type\":\"bool\",\"indexed\":false,\"internalType\":\"bool\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateSequencer\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"status\",\"type\":\"bool\",\"indexed\":false,\"internalType\":\"bool\"}],\"anonymous\":false},{\"type\":\"error\",\"name\":\"ErrorAccountIsNotEOA\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV0LengthTooSmall\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV1LengthTooSmall\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV3LengthMismatch\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsAlreadyCommitted\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsAlreadyVerified\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsEmpty\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallPointEvaluationPrecompileFailed\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallerIsNotProver\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallerIsNotSequencer\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorFoundMultipleBlobs\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisBatchHasNonZeroField\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisBatchImported\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisDataHashIsZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisParentBatchHashIsNonZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncompleteL2TransactionData\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchHash\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchIndex\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchVersion\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLength\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLengthV0\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLengthV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectChunkLengthV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorLastL1MessageSkipped\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNoBlobFound\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNoBlockInChunkV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNumTxsLessThanNumL1Msgs\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertFinalizedBatch\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertNotStartFromEnd\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertZeroBatches\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorStateRootIsZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorTooManyTxsInOneChunk\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorUnexpectedPointEvaluationPrecompileOutput\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorZeroAddress\",\"inputs\":[]}]", } // L1CommitBatchEvent represents a CommitBatch event raised by the ScrollChain contract. diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index 38b6e7870e4a..621ddc2252ff 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -11,22 +11,26 @@ import ( type BatchQueue struct { // batches is map from batchIndex to batch blocks - batches map[uint64]DAEntry - daQueue *DaQueue - db ethdb.Database + batches map[uint64]DAEntry + daQueue *DaQueue + db ethdb.Database + lastFinalizedBatchIndex uint64 } func NewBatchQueue(daQueue *DaQueue, db ethdb.Database) *BatchQueue { return &BatchQueue{ - batches: make(map[uint64]DAEntry), - daQueue: daQueue, - db: db, + batches: make(map[uint64]DAEntry), + daQueue: daQueue, + db: db, + lastFinalizedBatchIndex: 0, } } // NextBatch finds next finalized batch and returns data, that was committed in that batch func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { - + if batch, ok := bq.getFinalizedBatch(); ok { + return batch, nil + } for { daEntry, err := bq.daQueue.NextDA(ctx) if err != nil { @@ -42,20 +46,42 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { case *RevertBatchDA: bq.deleteBatch(daEntry.BatchIndex) case *FinalizeBatchDA: - ret, ok := bq.batches[daEntry.BatchIndex] - if !ok { - // most probable, we met FinalizeBatch event for already committed batch after restart + if daEntry.BatchIndex > bq.lastFinalizedBatchIndex { + bq.lastFinalizedBatchIndex = daEntry.BatchIndex + } + ret, ok := bq.getFinalizedBatch() + if ok { + return ret, nil + } else { continue } - bq.deleteBatch(daEntry.BatchIndex) - return ret, nil default: return nil, fmt.Errorf("unexpected type of daEntry: %T", daEntry) } } } -// deleteBatch deletes data committed in the batch, because this batch is reverted or finalized +// getFinalizedBatch returns next finalized batch if there is available +func (bq *BatchQueue) getFinalizedBatch() (DAEntry, bool) { + if len(bq.batches) == 0 { + return nil, false + } + var minBatchIndex uint64 = math.MaxUint64 + for index := range bq.batches { + if index < minBatchIndex { + minBatchIndex = index + } + } + if minBatchIndex <= bq.lastFinalizedBatchIndex { + batch, _ := bq.batches[minBatchIndex] + bq.deleteBatch(minBatchIndex) + return batch, true + } else { + return nil, false + } +} + +// deleteBatch deletes data committed in the batch from map, because this batch is reverted or finalized // updates DASyncedL1BlockNumber func (bq *BatchQueue) deleteBatch(batchIndex uint64) { batch, ok := bq.batches[batchIndex] @@ -75,4 +101,5 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { } } rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height-1) + } diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index 901122147c65..e0ffa7e4e547 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -132,6 +132,14 @@ type commitBatchArgs struct { SkippedL1MessageBitmap []byte } +type commitBatchWithBlobProofArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte + BlobDataProof []byte +} + func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { return NewCommitBatchDaV0(0, batchIndex, 0, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil @@ -150,26 +158,40 @@ func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Lo if err != nil { return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) } - values, err := method.Inputs.Unpack(txData[methodIDLength:]) if err != nil { return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) } - var args commitBatchArgs - err = method.Inputs.Copy(&args, values) - if err != nil { - return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - switch args.Version { - case 0: - return ds.decodeDAV0(batchIndex, vLog, &args) - case 1: - return ds.decodeDAV1(batchIndex, vLog, &args) - case 2: - return ds.decodeDAV2(batchIndex, vLog, &args) - default: - return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) + if method.Name == "commitBatch" { + var args commitBatchArgs + err = method.Inputs.Copy(&args, values) + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + switch args.Version { + case 0: + return ds.decodeDAV0(batchIndex, vLog, &args) + case 1: + return ds.decodeDAV1(batchIndex, vLog, &args) + case 2: + return ds.decodeDAV2(batchIndex, vLog, &args) + default: + return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) + } + } else { + var args commitBatchWithBlobProofArgs + err = method.Inputs.Copy(&args, values) + var usedArgs commitBatchArgs = commitBatchArgs{ + Version: args.Version, + ParentBatchHeader: args.ParentBatchHeader, + Chunks: args.Chunks, + SkippedL1MessageBitmap: args.SkippedL1MessageBitmap, + } + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + return ds.decodeDAV2(batchIndex, vLog, &usedArgs) } } From cda4058ac9f84b34b79597903d50172536cdf035 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Wed, 24 Jul 2024 13:08:25 +0200 Subject: [PATCH 27/59] address comments --- cmd/utils/flags.go | 13 +- consensus/clique/clique.go | 4 - core/rawdb/accessors_da_syncer.go | 6 +- eth/backend.go | 11 +- eth/ethconfig/config.go | 3 +- params/config.go | 10 +- rollup/da_syncer/abi.go | 55 ------- rollup/da_syncer/batch_queue.go | 26 +-- .../{ => blob_client}/blob_client.go | 14 +- .../{ => blob_client}/blob_scan_client.go | 20 ++- .../{ => blob_client}/block_native_client.go | 16 +- rollup/da_syncer/block_queue.go | 18 +-- rollup/da_syncer/calldata_blob_source.go | 111 +++++++------ rollup/da_syncer/da.go | 54 +++---- rollup/da_syncer/da_queue.go | 12 +- rollup/da_syncer/da_syncer.go | 8 +- rollup/da_syncer/data_source.go | 10 +- rollup/da_syncer/l1_client.go | 149 ------------------ rollup/da_syncer/syncing_pipeline.go | 65 ++++---- rollup/rollup_sync_service/abi.go | 2 +- rollup/rollup_sync_service/abi_test.go | 4 +- rollup/rollup_sync_service/l1client.go | 63 +++++++- rollup/rollup_sync_service/l1client_test.go | 8 +- .../rollup_sync_service.go | 8 +- .../rollup_sync_service_test.go | 8 +- 25 files changed, 292 insertions(+), 406 deletions(-) delete mode 100644 rollup/da_syncer/abi.go rename rollup/da_syncer/{ => blob_client}/blob_client.go (79%) rename rollup/da_syncer/{ => blob_client}/blob_scan_client.go (87%) rename rollup/da_syncer/{ => blob_client}/block_native_client.go (81%) delete mode 100644 rollup/da_syncer/l1_client.go diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 7451fec99420..654aced7fed2 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -74,6 +74,7 @@ import ( "github.com/scroll-tech/go-ethereum/p2p/netutil" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/da_syncer" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/tracing" "github.com/scroll-tech/go-ethereum/rpc" ) @@ -865,13 +866,13 @@ var ( Name: "da.sync", Usage: "Enable node syncing from DA", } - defaultDa = ethconfig.Defaults.DA.FetcherMode + defaultDA = ethconfig.Defaults.DA.FetcherMode DAModeFlag = TextMarshalerFlag{ Name: "da.mode", Usage: `Da sync sync mode ("l1rpc" or "snapshot")`, - Value: &defaultDa, + Value: &defaultDA, } - defaultBlobSource = ethconfig.Defaults.DA.BLobSource + defaultBlobSource = ethconfig.Defaults.DA.BlobSource DABlobSourceFlag = TextMarshalerFlag{ Name: "da.blob.source", Usage: `Blob data source, currently supported "blobscan" or "blocknative"`, @@ -1598,7 +1599,7 @@ func setEnableRollupVerify(ctx *cli.Context, cfg *ethconfig.Config) { } } -func setDa(ctx *cli.Context, cfg *ethconfig.Config) { +func setDA(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.GlobalIsSet(DASyncEnabledFlag.Name) { cfg.EnableDASyncing = ctx.GlobalBool(DASyncEnabledFlag.Name) if ctx.GlobalIsSet(DAModeFlag.Name) { @@ -1608,7 +1609,7 @@ func setDa(ctx *cli.Context, cfg *ethconfig.Config) { cfg.DA.SnapshotFilePath = ctx.GlobalString(DASnapshotFileFlag.Name) } if ctx.GlobalIsSet(DABlobSourceFlag.Name) { - cfg.DA.BLobSource = *GlobalTextMarshaler(ctx, DABlobSourceFlag.Name).(*da_syncer.BLobSource) + cfg.DA.BlobSource = *GlobalTextMarshaler(ctx, DABlobSourceFlag.Name).(*blob_client.BlobSource) } } } @@ -1688,7 +1689,7 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { setLes(ctx, cfg) setCircuitCapacityCheck(ctx, cfg) setEnableRollupVerify(ctx, cfg) - setDa(ctx, cfg) + setDA(ctx, cfg) setMaxBlockRange(ctx, cfg) // Cap the cache allowance and tune the garbage collector diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index efe10a2e389f..0c79d877e534 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -145,7 +145,6 @@ type SignerFn func(signer accounts.Account, mimeType string, message []byte) ([] // ecrecover extracts the Ethereum account address from a signed header. func ecrecover(header *types.Header, sigcache *lru.ARCCache) (common.Address, error) { - // return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil // If the signature's already cached, return that hash := header.Hash() if address, known := sigcache.Get(hash); known { @@ -353,9 +352,6 @@ func (c *Clique) verifyCascadingFields(chain consensus.ChainHeaderReader, header // Verify the header's EIP-1559 attributes. return err } - if c.config.DaSyncingEnabled { - return nil - } // Retrieve the snapshot needed to verify this header and cache it snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) if err != nil { diff --git a/core/rawdb/accessors_da_syncer.go b/core/rawdb/accessors_da_syncer.go index c4097e59b8e3..96f816685652 100644 --- a/core/rawdb/accessors_da_syncer.go +++ b/core/rawdb/accessors_da_syncer.go @@ -12,7 +12,7 @@ func WriteDASyncedL1BlockNumber(db ethdb.KeyValueWriter, L1BlockNumber uint64) { value := big.NewInt(0).SetUint64(L1BlockNumber).Bytes() if err := db.Put(daSyncedL1BlockNumberKey, value); err != nil { - log.Crit("Failed to update da synced L1 block number", "err", err) + log.Crit("Failed to update DA synced L1 block number", "err", err) } } @@ -23,7 +23,7 @@ func ReadDASyncedL1BlockNumber(db ethdb.Reader) *uint64 { return nil } if err != nil { - log.Crit("Failed to read da synced L1 block number from database", "err", err) + log.Crit("Failed to read DA synced L1 block number from database", "err", err) } if len(data) == 0 { return nil @@ -31,7 +31,7 @@ func ReadDASyncedL1BlockNumber(db ethdb.Reader) *uint64 { number := new(big.Int).SetBytes(data) if !number.IsUint64() { - log.Crit("Unexpected da synced L1 block number in database", "number", number) + log.Crit("Unexpected DA synced L1 block number in database", "number", number) } value := number.Uint64() diff --git a/eth/backend.go b/eth/backend.go index 44b3def3f4c2..fe36f39e5850 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -339,7 +339,7 @@ func (s *Ethereum) APIs() []rpc.API { // Append any APIs exposed explicitly by the consensus engine apis = append(apis, s.engine.APIs(s.BlockChain())...) - if s.handler != nil { + if !s.config.EnableDASyncing { apis = append(apis, rpc.API{ Namespace: "eth", Version: "1.0", @@ -566,7 +566,8 @@ func (s *Ethereum) SyncService() *sync_service.SyncService { return s.syncServic // Protocols returns all the currently configured // network protocols to start. func (s *Ethereum) Protocols() []p2p.Protocol { - if s.handler == nil { + // if DA syncing enabled then we don't create handler + if s.config.EnableDASyncing { return nil } protos := eth.MakeProtocols((*ethHandler)(s.handler), s.networkID, s.ethDialCandidates) @@ -593,7 +594,8 @@ func (s *Ethereum) Start() error { // maxPeers -= s.config.LightPeers //} // Start the networking layer and the light server if requested - if s.handler != nil { + // handler is not enabled when DA syncing enabled + if s.config.EnableDASyncing { s.handler.Start(maxPeers) } return nil @@ -605,7 +607,8 @@ func (s *Ethereum) Stop() error { // Stop all the peer-related stuff first. s.ethDialCandidates.Close() s.snapDialCandidates.Close() - if s.handler != nil { + // handler is not enabled if DA syncing enabled + if !s.config.EnableDASyncing { s.handler.Stop() } diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 0a3ea43b7b75..3e7e1ab99708 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -38,6 +38,7 @@ import ( "github.com/scroll-tech/go-ethereum/node" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/da_syncer" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" ) // FullNodeGPO contains default gasprice oracle settings for full node. @@ -96,7 +97,7 @@ var Defaults = Config{ MaxBlockRange: -1, // Default unconfigured value: no block range limit for backward compatibility DA: da_syncer.Config{ FetcherMode: da_syncer.L1RPC, - BLobSource: da_syncer.BlobScan, + BlobSource: blob_client.BlobScan, }, } diff --git a/params/config.go b/params/config.go index 01e2cdd31dfe..3c3ef43b9bca 100644 --- a/params/config.go +++ b/params/config.go @@ -337,7 +337,7 @@ var ( ScrollChainAddress: common.HexToAddress("0x2D567EcE699Eabe5afCd141eDB7A4f2D0D6ce8a0"), }, DAConfig: &DAConfig{ - BlobScanApiEndpoint: "https://api.sepolia.blobscan.com/blobs/", + BlobScanAPIEndpoint: "https://api.sepolia.blobscan.com/blobs/", }, }, } @@ -379,8 +379,8 @@ var ( ScrollChainAddress: common.HexToAddress("0xa13BAF47339d63B743e7Da8741db5456DAc1E556"), }, DAConfig: &DAConfig{ - BlobScanApiEndpoint: "https://api.blobscan.com/blobs/", - BlockNativeApiEndpoint: "https://api.ethernow.xyz/v1/blob/", + BlobScanAPIEndpoint: "https://api.blobscan.com/blobs/", + BlockNativeAPIEndpoint: "https://api.ethernow.xyz/v1/blob/", }, }, } @@ -673,8 +673,8 @@ type L1Config struct { // DAConfig contains the parameters to sync from DA type DAConfig struct { - BlobScanApiEndpoint string `json:"blobScanApiEndpoint,omitempty"` - BlockNativeApiEndpoint string `json:"blockNativeApiEndpoint,omitempty"` + BlobScanAPIEndpoint string `json:"blobScanApiEndpoint,omitempty"` + BlockNativeAPIEndpoint string `json:"blockNativeApiEndpoint,omitempty"` } func (c *L1Config) String() string { diff --git a/rollup/da_syncer/abi.go b/rollup/da_syncer/abi.go deleted file mode 100644 index 8b85f322ff9a..000000000000 --- a/rollup/da_syncer/abi.go +++ /dev/null @@ -1,55 +0,0 @@ -package da_syncer - -import ( - "fmt" - "math/big" - - "github.com/scroll-tech/go-ethereum/accounts/abi" - "github.com/scroll-tech/go-ethereum/accounts/abi/bind" - "github.com/scroll-tech/go-ethereum/common" - "github.com/scroll-tech/go-ethereum/core/types" -) - -// scrollChainMetaData contains ABI of the ScrollChain contract. -var scrollChainMetaData = &bind.MetaData{ - ABI: "[{\"type\":\"constructor\",\"inputs\":[{\"name\":\"_chainId\",\"type\":\"uint64\",\"internalType\":\"uint64\"},{\"name\":\"_messageQueue\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_verifier\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addProver\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"addSequencer\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"commitBatch\",\"inputs\":[{\"name\":\"_version\",\"type\":\"uint8\",\"internalType\":\"uint8\"},{\"name\":\"_parentBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_chunks\",\"type\":\"bytes[]\",\"internalType\":\"bytes[]\"},{\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"commitBatchWithBlobProof\",\"inputs\":[{\"name\":\"_version\",\"type\":\"uint8\",\"internalType\":\"uint8\"},{\"name\":\"_parentBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_chunks\",\"type\":\"bytes[]\",\"internalType\":\"bytes[]\"},{\"name\":\"_skippedL1MessageBitmap\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_blobDataProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"committedBatches\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"finalizeBatchWithProof4844\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_postStateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_withdrawRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_blobDataProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_aggrProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"finalizeBundleWithProof\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_postStateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_withdrawRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"},{\"name\":\"_aggrProof\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"finalizedStateRoots\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"importGenesisBatch\",\"inputs\":[{\"name\":\"_batchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_stateRoot\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"initialize\",\"inputs\":[{\"name\":\"_messageQueue\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_verifier\",\"type\":\"address\",\"internalType\":\"address\"},{\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"isBatchFinalized\",\"inputs\":[{\"name\":\"_batchIndex\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"isProver\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"isSequencer\",\"inputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"lastFinalizedBatchIndex\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"layer2ChainId\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint64\",\"internalType\":\"uint64\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"maxNumTxInChunk\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"messageQueue\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"owner\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"paused\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"removeProver\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"removeSequencer\",\"inputs\":[{\"name\":\"_account\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"renounceOwnership\",\"inputs\":[],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"revertBatch\",\"inputs\":[{\"name\":\"_firstBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"},{\"name\":\"_lastBatchHeader\",\"type\":\"bytes\",\"internalType\":\"bytes\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"setPause\",\"inputs\":[{\"name\":\"_status\",\"type\":\"bool\",\"internalType\":\"bool\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"transferOwnership\",\"inputs\":[{\"name\":\"newOwner\",\"type\":\"address\",\"internalType\":\"address\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"updateMaxNumTxInChunk\",\"inputs\":[{\"name\":\"_maxNumTxInChunk\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[],\"stateMutability\":\"nonpayable\"},{\"type\":\"function\",\"name\":\"verifier\",\"inputs\":[],\"outputs\":[{\"name\":\"\",\"type\":\"address\",\"internalType\":\"address\"}],\"stateMutability\":\"view\"},{\"type\":\"function\",\"name\":\"withdrawRoots\",\"inputs\":[{\"name\":\"\",\"type\":\"uint256\",\"internalType\":\"uint256\"}],\"outputs\":[{\"name\":\"\",\"type\":\"bytes32\",\"internalType\":\"bytes32\"}],\"stateMutability\":\"view\"},{\"type\":\"event\",\"name\":\"CommitBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"FinalizeBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"},{\"name\":\"stateRoot\",\"type\":\"bytes32\",\"indexed\":false,\"internalType\":\"bytes32\"},{\"name\":\"withdrawRoot\",\"type\":\"bytes32\",\"indexed\":false,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Initialized\",\"inputs\":[{\"name\":\"version\",\"type\":\"uint8\",\"indexed\":false,\"internalType\":\"uint8\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"OwnershipTransferred\",\"inputs\":[{\"name\":\"previousOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"newOwner\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Paused\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":false,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"RevertBatch\",\"inputs\":[{\"name\":\"batchIndex\",\"type\":\"uint256\",\"indexed\":true,\"internalType\":\"uint256\"},{\"name\":\"batchHash\",\"type\":\"bytes32\",\"indexed\":true,\"internalType\":\"bytes32\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"Unpaused\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":false,\"internalType\":\"address\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateMaxNumTxInChunk\",\"inputs\":[{\"name\":\"oldMaxNumTxInChunk\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"},{\"name\":\"newMaxNumTxInChunk\",\"type\":\"uint256\",\"indexed\":false,\"internalType\":\"uint256\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateProver\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"status\",\"type\":\"bool\",\"indexed\":false,\"internalType\":\"bool\"}],\"anonymous\":false},{\"type\":\"event\",\"name\":\"UpdateSequencer\",\"inputs\":[{\"name\":\"account\",\"type\":\"address\",\"indexed\":true,\"internalType\":\"address\"},{\"name\":\"status\",\"type\":\"bool\",\"indexed\":false,\"internalType\":\"bool\"}],\"anonymous\":false},{\"type\":\"error\",\"name\":\"ErrorAccountIsNotEOA\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV0LengthTooSmall\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV1LengthTooSmall\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchHeaderV3LengthMismatch\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsAlreadyCommitted\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsAlreadyVerified\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorBatchIsEmpty\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallPointEvaluationPrecompileFailed\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallerIsNotProver\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorCallerIsNotSequencer\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorFoundMultipleBlobs\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisBatchHasNonZeroField\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisBatchImported\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisDataHashIsZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorGenesisParentBatchHashIsNonZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncompleteL2TransactionData\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchHash\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchIndex\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBatchVersion\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLength\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLengthV0\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectBitmapLengthV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorIncorrectChunkLengthV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorLastL1MessageSkipped\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNoBlobFound\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNoBlockInChunkV1\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorNumTxsLessThanNumL1Msgs\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertFinalizedBatch\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertNotStartFromEnd\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorRevertZeroBatches\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorStateRootIsZero\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorTooManyTxsInOneChunk\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorUnexpectedPointEvaluationPrecompileOutput\",\"inputs\":[]},{\"type\":\"error\",\"name\":\"ErrorZeroAddress\",\"inputs\":[]}]", -} - -// L1CommitBatchEvent represents a CommitBatch event raised by the ScrollChain contract. -type L1CommitBatchEvent struct { - BatchIndex *big.Int - BatchHash common.Hash -} - -// L1RevertBatchEvent represents a RevertBatch event raised by the ScrollChain contract. -type L1RevertBatchEvent struct { - BatchIndex *big.Int - BatchHash common.Hash -} - -// L1FinalizeBatchEvent represents a FinalizeBatch event raised by the ScrollChain contract. -type L1FinalizeBatchEvent struct { - BatchIndex *big.Int - BatchHash common.Hash - StateRoot common.Hash - WithdrawRoot common.Hash -} - -// UnpackLog unpacks a retrieved log into the provided output structure. -func UnpackLog(c *abi.ABI, out interface{}, event string, log types.Log) error { - if log.Topics[0] != c.Events[event].ID { - return fmt.Errorf("event signature mismatch") - } - if len(log.Data) > 0 { - if err := c.UnpackIntoInterface(out, event, log.Data); err != nil { - return err - } - } - var indexed abi.Arguments - for _, arg := range c.Events[event].Inputs { - if arg.Indexed { - indexed = append(indexed, arg) - } - } - return abi.ParseTopics(out, indexed, log.Topics[1:]) -} diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index 621ddc2252ff..b1ae23d04e16 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -12,15 +12,15 @@ import ( type BatchQueue struct { // batches is map from batchIndex to batch blocks batches map[uint64]DAEntry - daQueue *DaQueue + DAQueue *DAQueue db ethdb.Database lastFinalizedBatchIndex uint64 } -func NewBatchQueue(daQueue *DaQueue, db ethdb.Database) *BatchQueue { +func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { return &BatchQueue{ batches: make(map[uint64]DAEntry), - daQueue: daQueue, + DAQueue: DAQueue, db: db, lastFinalizedBatchIndex: 0, } @@ -32,16 +32,16 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { return batch, nil } for { - daEntry, err := bq.daQueue.NextDA(ctx) + daEntry, err := bq.DAQueue.NextDA(ctx) if err != nil { return nil, err } switch daEntry := daEntry.(type) { - case *CommitBatchDaV0: + case *CommitBatchDAV0: bq.batches[daEntry.BatchIndex] = daEntry - case *CommitBatchDaV1: + case *CommitBatchDAV1: bq.batches[daEntry.BatchIndex] = daEntry - case *CommitBatchDaV2: + case *CommitBatchDAV2: bq.batches[daEntry.BatchIndex] = daEntry case *RevertBatchDA: bq.deleteBatch(daEntry.BatchIndex) @@ -73,9 +73,14 @@ func (bq *BatchQueue) getFinalizedBatch() (DAEntry, bool) { } } if minBatchIndex <= bq.lastFinalizedBatchIndex { - batch, _ := bq.batches[minBatchIndex] + batch, ok := bq.batches[minBatchIndex] + + // this won't happen because wew just found minBatchIndex among map keys, but need to leave this check to pass CI + if !ok { + return nil, false + } bq.deleteBatch(minBatchIndex) - return batch, true + return batch, ok } else { return nil, false } @@ -94,12 +99,13 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) return } + // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart var minBatchL1Height uint64 = math.MaxUint64 for _, val := range bq.batches { if val.GetL1BlockNumber() < minBatchL1Height { minBatchL1Height = val.GetL1BlockNumber() } } - rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height-1) + rawdb.WriteDASyncedL1BlockNumber(bq.db, minBatchL1Height-1) } diff --git a/rollup/da_syncer/blob_client.go b/rollup/da_syncer/blob_client/blob_client.go similarity index 79% rename from rollup/da_syncer/blob_client.go rename to rollup/da_syncer/blob_client/blob_client.go index ac718d2e92a5..fbe20ae803ca 100644 --- a/rollup/da_syncer/blob_client.go +++ b/rollup/da_syncer/blob_client/blob_client.go @@ -1,4 +1,4 @@ -package da_syncer +package blob_client import ( "context" @@ -17,21 +17,21 @@ type BlobClient interface { GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) } -type BLobSource int +type BlobSource int const ( // BlobScan - BlobScan BLobSource = iota + BlobScan BlobSource = iota // BlockNative BlockNative ) -func (src BLobSource) IsValid() bool { +func (src BlobSource) IsValid() bool { return src >= BlobScan && src <= BlockNative } // String implements the stringer interface. -func (src BLobSource) String() string { +func (src BlobSource) String() string { switch src { case BlobScan: return "blobscan" @@ -42,7 +42,7 @@ func (src BLobSource) String() string { } } -func (src BLobSource) MarshalText() ([]byte, error) { +func (src BlobSource) MarshalText() ([]byte, error) { switch src { case BlobScan: return []byte("blobscan"), nil @@ -53,7 +53,7 @@ func (src BLobSource) MarshalText() ([]byte, error) { } } -func (src *BLobSource) UnmarshalText(text []byte) error { +func (src *BlobSource) UnmarshalText(text []byte) error { switch string(text) { case "blobscan": *src = BlobScan diff --git a/rollup/da_syncer/blob_scan_client.go b/rollup/da_syncer/blob_client/blob_scan_client.go similarity index 87% rename from rollup/da_syncer/blob_scan_client.go rename to rollup/da_syncer/blob_client/blob_scan_client.go index c65841aa7367..98b190c2214f 100644 --- a/rollup/da_syncer/blob_scan_client.go +++ b/rollup/da_syncer/blob_client/blob_scan_client.go @@ -1,4 +1,4 @@ -package da_syncer +package blob_client import ( "context" @@ -6,25 +6,31 @@ import ( "encoding/json" "fmt" "net/http" + "net/url" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" ) type BlobScanClient struct { - client *http.Client - blobScanApiEndpoint string + client *http.Client + apiEndpoint string } -func newBlobScanClient(blobScanApiEndpoint string) *BlobScanClient { +func NewBlobScanClient(apiEndpoint string) *BlobScanClient { return &BlobScanClient{ - client: http.DefaultClient, - blobScanApiEndpoint: blobScanApiEndpoint, + client: http.DefaultClient, + apiEndpoint: apiEndpoint, } } func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { - req, err := http.NewRequestWithContext(ctx, "GET", c.blobScanApiEndpoint+versionedHash.String(), nil) + // blobscan api docs https://api.blobscan.com/#/blobs/blob-getByBlobId + path, err := url.JoinPath(c.apiEndpoint, versionedHash.String()) + if err != nil { + return nil, fmt.Errorf("failed to join path, err: %w", err) + } + req, err := http.NewRequestWithContext(ctx, "GET", path, nil) if err != nil { return nil, fmt.Errorf("cannot create request, err: %w", err) } diff --git a/rollup/da_syncer/block_native_client.go b/rollup/da_syncer/blob_client/block_native_client.go similarity index 81% rename from rollup/da_syncer/block_native_client.go rename to rollup/da_syncer/blob_client/block_native_client.go index f9cbb7819163..ef3eebaa2972 100644 --- a/rollup/da_syncer/block_native_client.go +++ b/rollup/da_syncer/blob_client/block_native_client.go @@ -1,4 +1,4 @@ -package da_syncer +package blob_client import ( "context" @@ -6,23 +6,29 @@ import ( "encoding/json" "fmt" "net/http" + "net/url" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" ) type BlockNativeClient struct { - blockNativeApiEndpoint string + apiEndpoint string } -func newBlockNativeClient(blockNativeApiEndpoint string) *BlockNativeClient { +func NewBlockNativeClient(apiEndpoint string) *BlockNativeClient { return &BlockNativeClient{ - blockNativeApiEndpoint: blockNativeApiEndpoint, + apiEndpoint: apiEndpoint, } } func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { - resp, err := http.Get(c.blockNativeApiEndpoint + versionedHash.String()) + // blocknative api docs https://docs.blocknative.com/blocknative-data-archive/blob-archive + path, err := url.JoinPath(c.apiEndpoint, versionedHash.String()) + if err != nil { + return nil, fmt.Errorf("failed to join path, err: %w", err) + } + resp, err := http.Get(path) if err != nil { return nil, fmt.Errorf("cannot do request, err: %w", err) } diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index b4ea08e6e8e3..6340bc33db18 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -38,18 +38,18 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { return err } switch daEntry := daEntry.(type) { - case *CommitBatchDaV0: - bq.blocks, err = bq.processDaV0ToBlocks(daEntry) + case *CommitBatchDAV0: + bq.blocks, err = bq.processDAV0ToBlocks(daEntry) if err != nil { return err } - case *CommitBatchDaV1: - bq.blocks, err = bq.processDaV1ToBlocks(daEntry) + case *CommitBatchDAV1: + bq.blocks, err = bq.processDAV1ToBlocks(daEntry) if err != nil { return err } - case *CommitBatchDaV2: - bq.blocks, err = bq.processDaV2ToBlocks(daEntry) + case *CommitBatchDAV2: + bq.blocks, err = bq.processDAV2ToBlocks(daEntry) if err != nil { return err } @@ -59,7 +59,7 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { return nil } -func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Block, error) { +func (bq *BlockQueue) processDAV0ToBlocks(daEntry *CommitBatchDAV0) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped @@ -91,7 +91,7 @@ func (bq *BlockQueue) processDaV0ToBlocks(daEntry *CommitBatchDaV0) ([]*types.Bl return blocks, nil } -func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Block, error) { +func (bq *BlockQueue) processDAV1ToBlocks(daEntry *CommitBatchDAV1) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped @@ -123,7 +123,7 @@ func (bq *BlockQueue) processDaV1ToBlocks(daEntry *CommitBatchDaV1) ([]*types.Bl return blocks, nil } -func (bq *BlockQueue) processDaV2ToBlocks(daEntry *CommitBatchDaV2) ([]*types.Block, error) { +func (bq *BlockQueue) processDAV2ToBlocks(daEntry *CommitBatchDAV2) ([]*types.Block, error) { var blocks []*types.Block l1TxPointer := 0 var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go index e0ffa7e4e547..94487166772c 100644 --- a/rollup/da_syncer/calldata_blob_source.go +++ b/rollup/da_syncer/calldata_blob_source.go @@ -18,16 +18,27 @@ import ( "github.com/scroll-tech/go-ethereum/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" + + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" + "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" ) var ( - callDataBlobSourceFetchBlockRange uint64 = 500 + callDataBlobSourceFetchBlockRange uint64 = 500 + commitBatchEventName = "CommitBatch" + revertBatchEventName = "RevertBatch" + finalizeBatchEventName = "FinalizeBatch" + commitBatchMethodName = "commitBatch" + commitBatchWithBlobProofMethodName = "commitBatchWithBlobProof" + + // the length og method ID at the beginning of transaction data + methodIDLength = 4 ) type CalldataBlobSource struct { ctx context.Context - l1Client *L1Client - blobClient BlobClient + l1Client *rollup_sync_service.L1Client + blobClient blob_client.BlobClient l1height uint64 scrollChainABI *abi.ABI l1CommitBatchEventSignature common.Hash @@ -36,8 +47,8 @@ type CalldataBlobSource struct { db ethdb.Database } -func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) (DataSource, error) { - scrollChainABI, err := scrollChainMetaData.GetAbi() +func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *rollup_sync_service.L1Client, blobClient blob_client.BlobClient, db ethdb.Database) (DataSource, error) { + scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) } @@ -47,26 +58,26 @@ func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *L1Cli blobClient: blobClient, l1height: l1height, scrollChainABI: scrollChainABI, - l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, - l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, + l1CommitBatchEventSignature: scrollChainABI.Events[commitBatchEventName].ID, + l1RevertBatchEventSignature: scrollChainABI.Events[revertBatchEventName].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events[finalizeBatchEventName].ID, db: db, }, nil } func (ds *CalldataBlobSource) NextData() (DA, error) { to := ds.l1height + callDataBlobSourceFetchBlockRange - l1Finalized, err := ds.l1Client.getFinalizedBlockNumber(ds.ctx) + l1Finalized, err := ds.l1Client.GetLatestFinalizedBlockNumber() if err != nil { return nil, fmt.Errorf("cannot get l1height, error: %v", err) } - if to > l1Finalized.Uint64() { - to = l1Finalized.Uint64() + if to > l1Finalized { + to = l1Finalized } if ds.l1height > to { - return nil, sourceExhaustedErr + return nil, errSourceExhausted } - logs, err := ds.l1Client.fetchRollupEventsInRange(ds.ctx, ds.l1height, to) + logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) if err != nil { return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) } @@ -86,8 +97,8 @@ func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { for _, vLog := range logs { switch vLog.Topics[0] { case ds.l1CommitBatchEventSignature: - event := &L1CommitBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { + event := &rollup_sync_service.L1CommitBatchEvent{} + if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) } batchIndex := event.BatchIndex.Uint64() @@ -100,8 +111,8 @@ func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { da = append(da, daEntry) case ds.l1RevertBatchEventSignature: - event := &L1RevertBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { + event := &rollup_sync_service.L1RevertBatchEvent{} + if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) } batchIndex := event.BatchIndex.Uint64() @@ -109,8 +120,8 @@ func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { da = append(da, NewRevertBatchDA(batchIndex)) case ds.l1FinalizeBatchEventSignature: - event := &L1FinalizeBatchEvent{} - if err := UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { + event := &rollup_sync_service.L1FinalizeBatchEvent{} + if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) } batchIndex := event.BatchIndex.Uint64() @@ -132,6 +143,26 @@ type commitBatchArgs struct { SkippedL1MessageBitmap []byte } +func newCommitBatchArgs(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { + var args commitBatchArgs + err := method.Inputs.Copy(&args, values) + return &args, err +} + +func newCommitBatchArgsFromCommitBatchWithProof(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { + var args commitBatchWithBlobProofArgs + err := method.Inputs.Copy(&args, values) + if err != nil { + return nil, err + } + return &commitBatchArgs{ + Version: args.Version, + ParentBatchHeader: args.ParentBatchHeader, + Chunks: args.Chunks, + SkippedL1MessageBitmap: args.SkippedL1MessageBitmap, + }, nil +} + type commitBatchWithBlobProofArgs struct { Version uint8 ParentBatchHeader []byte @@ -142,14 +173,13 @@ type commitBatchWithBlobProofArgs struct { func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { if batchIndex == 0 { - return NewCommitBatchDaV0(0, batchIndex, 0, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil + return NewCommitBatchDAV0(0, batchIndex, 0, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil } - txData, err := ds.l1Client.fetchTxData(ds.ctx, vLog) + txData, err := ds.l1Client.FetchTxData(vLog) if err != nil { return nil, err } - const methodIDLength = 4 if len(txData) < methodIDLength { return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) } @@ -163,37 +193,29 @@ func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Lo return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) } - if method.Name == "commitBatch" { - var args commitBatchArgs - err = method.Inputs.Copy(&args, values) + if method.Name == commitBatchMethodName { + args, err := newCommitBatchArgs(method, values) if err != nil { return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } switch args.Version { case 0: - return ds.decodeDAV0(batchIndex, vLog, &args) + return ds.decodeDAV0(batchIndex, vLog, args) case 1: - return ds.decodeDAV1(batchIndex, vLog, &args) + return ds.decodeDAV1(batchIndex, vLog, args) case 2: - return ds.decodeDAV2(batchIndex, vLog, &args) + return ds.decodeDAV2(batchIndex, vLog, args) default: return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) } - } else { - var args commitBatchWithBlobProofArgs - err = method.Inputs.Copy(&args, values) - var usedArgs commitBatchArgs = commitBatchArgs{ - Version: args.Version, - ParentBatchHeader: args.ParentBatchHeader, - Chunks: args.Chunks, - SkippedL1MessageBitmap: args.SkippedL1MessageBitmap, - } + } else if method.Name == commitBatchWithBlobProofMethodName { + args, err := newCommitBatchArgsFromCommitBatchWithProof(method, values) if err != nil { return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } - return ds.decodeDAV2(batchIndex, vLog, &usedArgs) + return ds.decodeDAV2(batchIndex, vLog, args) } - + return nil, fmt.Errorf("unknown method name: %s", method.Name) } func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { @@ -229,7 +251,7 @@ func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV0(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDAV0(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } @@ -241,7 +263,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) + versionedHash, err := ds.l1Client.FetchTxBlobHash(vLog) if err != nil { return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) } @@ -287,7 +309,7 @@ func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV1(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDAV1(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } @@ -299,7 +321,7 @@ func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, arg return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) } - versionedHash, err := ds.l1Client.fetchTxBlobHash(ds.ctx, vLog) + versionedHash, err := ds.l1Client.FetchTxBlobHash(vLog) if err != nil { return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) } @@ -345,10 +367,11 @@ func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, arg l1Txs = append(l1Txs, l1Tx) currentIndex++ } - da := NewCommitBatchDaV2(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) + da := NewCommitBatchDAV2(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) return da, nil } func getBatchTotalL1MessagePopped(data []byte) uint64 { + // total l1 message popped stored in bytes from 17 to 24, accordingly to codec spec return binary.BigEndian.Uint64(data[17:25]) } diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go index c8daa74f42f4..f2f8bdd9b903 100644 --- a/rollup/da_syncer/da.go +++ b/rollup/da_syncer/da.go @@ -32,8 +32,7 @@ type DAEntry interface { type DA []DAEntry -type CommitBatchDaV0 struct { - DaType DAType +type CommitBatchDAV0 struct { Version uint8 BatchIndex uint64 ParentTotalL1MessagePopped uint64 @@ -44,9 +43,8 @@ type CommitBatchDaV0 struct { L1BlockNumber uint64 } -func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDaV0{ - DaType: CommitBatchV0, +func NewCommitBatchDAV0(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { + return &CommitBatchDAV0{ Version: version, BatchIndex: batchIndex, ParentTotalL1MessagePopped: parentTotalL1MessagePopped, @@ -57,16 +55,15 @@ func NewCommitBatchDaV0(version uint8, batchIndex uint64, parentTotalL1MessagePo } } -func (f *CommitBatchDaV0) DAType() DAType { - return f.DaType +func (f *CommitBatchDAV0) DAType() DAType { + return CommitBatchV0 } -func (f *CommitBatchDaV0) GetL1BlockNumber() uint64 { +func (f *CommitBatchDAV0) GetL1BlockNumber() uint64 { return f.L1BlockNumber } -type CommitBatchDaV1 struct { - DaType DAType +type CommitBatchDAV1 struct { Version uint8 BatchIndex uint64 ParentTotalL1MessagePopped uint64 @@ -77,9 +74,8 @@ type CommitBatchDaV1 struct { L1BlockNumber uint64 } -func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDaV1{ - DaType: CommitBatchV1, +func NewCommitBatchDAV1(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { + return &CommitBatchDAV1{ Version: version, BatchIndex: batchIndex, ParentTotalL1MessagePopped: parentTotalL1MessagePopped, @@ -90,16 +86,15 @@ func NewCommitBatchDaV1(version uint8, batchIndex uint64, parentTotalL1MessagePo } } -func (f *CommitBatchDaV1) DAType() DAType { - return f.DaType +func (f *CommitBatchDAV1) DAType() DAType { + return CommitBatchV1 } -func (f *CommitBatchDaV1) GetL1BlockNumber() uint64 { +func (f *CommitBatchDAV1) GetL1BlockNumber() uint64 { return f.L1BlockNumber } -type CommitBatchDaV2 struct { - DaType DAType +type CommitBatchDAV2 struct { Version uint8 BatchIndex uint64 ParentTotalL1MessagePopped uint64 @@ -110,9 +105,8 @@ type CommitBatchDaV2 struct { L1BlockNumber uint64 } -func NewCommitBatchDaV2(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDaV2{ - DaType: CommitBatchV2, +func NewCommitBatchDAV2(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { + return &CommitBatchDAV2{ Version: version, BatchIndex: batchIndex, ParentTotalL1MessagePopped: parentTotalL1MessagePopped, @@ -123,16 +117,15 @@ func NewCommitBatchDaV2(version uint8, batchIndex uint64, parentTotalL1MessagePo } } -func (f *CommitBatchDaV2) DAType() DAType { - return f.DaType +func (f *CommitBatchDAV2) DAType() DAType { + return CommitBatchV2 } -func (f *CommitBatchDaV2) GetL1BlockNumber() uint64 { +func (f *CommitBatchDAV2) GetL1BlockNumber() uint64 { return f.L1BlockNumber } type RevertBatchDA struct { - DaType DAType BatchIndex uint64 L1BlockNumber uint64 @@ -140,13 +133,12 @@ type RevertBatchDA struct { func NewRevertBatchDA(batchIndex uint64) DAEntry { return &RevertBatchDA{ - DaType: RevertBatch, BatchIndex: batchIndex, } } func (f *RevertBatchDA) DAType() DAType { - return f.DaType + return RevertBatch } func (f *RevertBatchDA) GetL1BlockNumber() uint64 { @@ -154,7 +146,6 @@ func (f *RevertBatchDA) GetL1BlockNumber() uint64 { } type FinalizeBatchDA struct { - DaType DAType BatchIndex uint64 L1BlockNumber uint64 @@ -162,13 +153,12 @@ type FinalizeBatchDA struct { func NewFinalizeBatchDA(batchIndex uint64) DAEntry { return &FinalizeBatchDA{ - DaType: FinalizeBatch, BatchIndex: batchIndex, } } func (f *FinalizeBatchDA) DAType() DAType { - return f.DaType + return FinalizeBatch } func (f *FinalizeBatchDA) GetL1BlockNumber() uint64 { @@ -176,7 +166,6 @@ func (f *FinalizeBatchDA) GetL1BlockNumber() uint64 { } type FinalizeBatchDAV3 struct { - DaType DAType BatchIndex uint64 L1BlockNumber uint64 @@ -184,13 +173,12 @@ type FinalizeBatchDAV3 struct { func NewFinalizeBatchDAV3(batchIndex uint64) DAEntry { return &FinalizeBatchDAV3{ - DaType: FinalizeBatchV3, BatchIndex: batchIndex, } } func (f *FinalizeBatchDAV3) DAType() DAType { - return f.DaType + return FinalizeBatchV3 } func (f *FinalizeBatchDAV3) GetL1BlockNumber() uint64 { diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index 925670eafa05..b1d3d79b8b22 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -4,15 +4,15 @@ import ( "context" ) -type DaQueue struct { +type DAQueue struct { l1height uint64 dataSourceFactory *DataSourceFactory dataSource DataSource da DA } -func NewDaQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DaQueue { - return &DaQueue{ +func NewDAQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DAQueue { + return &DAQueue{ l1height: l1height, dataSourceFactory: dataSourceFactory, dataSource: nil, @@ -20,7 +20,7 @@ func NewDaQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DaQueue } } -func (dq *DaQueue) NextDA(ctx context.Context) (DAEntry, error) { +func (dq *DAQueue) NextDA(ctx context.Context) (DAEntry, error) { for len(dq.da) == 0 { err := dq.getNextData(ctx) if err != nil { @@ -32,7 +32,7 @@ func (dq *DaQueue) NextDA(ctx context.Context) (DAEntry, error) { return daEntry, nil } -func (dq *DaQueue) getNextData(ctx context.Context) error { +func (dq *DAQueue) getNextData(ctx context.Context) error { var err error if dq.dataSource == nil { dq.dataSource, err = dq.dataSourceFactory.OpenDataSource(ctx, dq.l1height) @@ -42,7 +42,7 @@ func (dq *DaQueue) getNextData(ctx context.Context) error { } dq.da, err = dq.dataSource.NextData() // previous dataSource has been exhausted, create new - if err == sourceExhaustedErr { + if err == errSourceExhausted { dq.l1height = dq.dataSource.L1Height() dq.dataSource = nil return dq.getNextData(ctx) diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 5115dcd0c067..e8dbf7859e84 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -11,17 +11,17 @@ import ( "github.com/scroll-tech/go-ethereum/trie" ) -type DaSyncer struct { +type DASyncer struct { blockchain *core.BlockChain } -func NewDaSyncer(blockchain *core.BlockChain) *DaSyncer { - return &DaSyncer{ +func NewDASyncer(blockchain *core.BlockChain) *DASyncer { + return &DASyncer{ blockchain: blockchain, } } -func (s *DaSyncer) SyncOneBlock(block *types.Block) error { +func (s *DASyncer) SyncOneBlock(block *types.Block) error { prevHash := s.blockchain.CurrentBlock().Hash() if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index d3ee9f26ab2d..017796352b8c 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -7,10 +7,12 @@ import ( "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/rollup_sync_service" ) var ( - sourceExhaustedErr = errors.New("data source has been exhausted") + errSourceExhausted = errors.New("data source has been exhausted") ) type DataSource interface { @@ -21,12 +23,12 @@ type DataSource interface { type DataSourceFactory struct { config Config genesisConfig *params.ChainConfig - l1Client *L1Client - blobClient BlobClient + l1Client *rollup_sync_service.L1Client + blobClient blob_client.BlobClient db ethdb.Database } -func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *L1Client, blobClient BlobClient, db ethdb.Database) *DataSourceFactory { +func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.ChainConfig, config Config, l1Client *rollup_sync_service.L1Client, blobClient blob_client.BlobClient, db ethdb.Database) *DataSourceFactory { return &DataSourceFactory{ config: config, genesisConfig: genesisConfig, diff --git a/rollup/da_syncer/l1_client.go b/rollup/da_syncer/l1_client.go deleted file mode 100644 index 8d234c68a9f3..000000000000 --- a/rollup/da_syncer/l1_client.go +++ /dev/null @@ -1,149 +0,0 @@ -package da_syncer - -import ( - "context" - "errors" - "fmt" - "math/big" - - "github.com/scroll-tech/go-ethereum" - "github.com/scroll-tech/go-ethereum/common" - "github.com/scroll-tech/go-ethereum/core/types" - "github.com/scroll-tech/go-ethereum/log" - "github.com/scroll-tech/go-ethereum/params" - "github.com/scroll-tech/go-ethereum/rpc" - - "github.com/scroll-tech/go-ethereum/rollup/sync_service" -) - -// L1Client is a wrapper around EthClient that adds -// methods for conveniently collecting rollup events of ScrollChain contract. -type L1Client struct { - client sync_service.EthClient - scrollChainAddress common.Address - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash -} - -// newL1Client initializes a new L1Client instance with the provided configuration. -// It checks for a valid scrollChainAddress and verifies the chain ID. -func newL1Client(ctx context.Context, genesisConfig *params.ChainConfig, l1Client sync_service.EthClient) (*L1Client, error) { - - scrollChainABI, err := scrollChainMetaData.GetAbi() - if err != nil { - return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) - } - - scrollChainAddress := genesisConfig.Scroll.L1Config.ScrollChainAddress - if scrollChainAddress == (common.Address{}) { - return nil, errors.New("must pass non-zero scrollChainAddress to L1Client") - } - - // sanity check: compare chain IDs - got, err := l1Client.ChainID(ctx) - if err != nil { - return nil, fmt.Errorf("failed to query L1 chain ID, err: %w", err) - } - if got.Cmp(big.NewInt(0).SetUint64(genesisConfig.Scroll.L1Config.L1ChainId)) != 0 { - return nil, fmt.Errorf("unexpected chain ID, expected: %v, got: %v", genesisConfig.Scroll.L1Config.L1ChainId, got) - } - - client := L1Client{ - client: l1Client, - scrollChainAddress: scrollChainAddress, - l1CommitBatchEventSignature: scrollChainABI.Events["CommitBatch"].ID, - l1RevertBatchEventSignature: scrollChainABI.Events["RevertBatch"].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events["FinalizeBatch"].ID, - } - return &client, nil -} - -// fetcRollupEventsInRange retrieves and parses commit/revert/finalize rollup events between block numbers: [from, to]. -func (c *L1Client) fetchRollupEventsInRange(ctx context.Context, from, to uint64) ([]types.Log, error) { - log.Trace("L1Client fetchRollupEventsInRange", "fromBlock", from, "toBlock", to) - - query := ethereum.FilterQuery{ - FromBlock: big.NewInt(int64(from)), // inclusive - ToBlock: big.NewInt(int64(to)), // inclusive - Addresses: []common.Address{ - c.scrollChainAddress, - }, - Topics: make([][]common.Hash, 1), - } - query.Topics[0] = make([]common.Hash, 3) - query.Topics[0][0] = c.l1CommitBatchEventSignature - query.Topics[0][1] = c.l1RevertBatchEventSignature - query.Topics[0][2] = c.l1FinalizeBatchEventSignature - - logs, err := c.client.FilterLogs(ctx, query) - if err != nil { - return nil, fmt.Errorf("failed to filter logs, err: %w", err) - } - return logs, nil -} - -// fetchTxData fetches tx data corresponding to given event log -func (c *L1Client) fetchTxData(ctx context.Context, vLog *types.Log) ([]byte, error) { - tx, _, err := c.client.TransactionByHash(ctx, vLog.TxHash) - if err != nil { - log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", - "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) - block, err := c.client.BlockByHash(ctx, vLog.BlockHash) - if err != nil { - return nil, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) - } - - found := false - for _, txInBlock := range block.Transactions() { - if txInBlock.Hash() == vLog.TxHash { - tx = txInBlock - found = true - break - } - } - if !found { - return nil, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) - } - } - - return tx.Data(), nil -} - -// fetchTxBlobHash fetches tx blob hash corresponding to given event log -func (c *L1Client) fetchTxBlobHash(ctx context.Context, vLog *types.Log) (common.Hash, error) { - tx, _, err := c.client.TransactionByHash(ctx, vLog.TxHash) - if err != nil { - log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", - "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) - block, err := c.client.BlockByHash(ctx, vLog.BlockHash) - if err != nil { - return common.Hash{}, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) - } - - found := false - for _, txInBlock := range block.Transactions() { - if txInBlock.Hash() == vLog.TxHash { - tx = txInBlock - found = true - break - } - } - if !found { - return common.Hash{}, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) - } - } - blobHashes := tx.BlobHashes() - if len(blobHashes) == 0 { - return common.Hash{}, fmt.Errorf("transaction does not contain any blobs, tx hash: %v", vLog.TxHash.Hex()) - } - return blobHashes[0], nil -} - -func (c *L1Client) getFinalizedBlockNumber(ctx context.Context) (*big.Int, error) { - h, err := c.client.HeaderByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) - if err != nil { - return nil, err - } - return h.Number, nil -} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 18d47ef37dd6..5c5e56300886 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -11,14 +11,16 @@ import ( "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" "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/rollup_sync_service" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) // Config is the configuration parameters of data availability syncing. type Config struct { - FetcherMode FetcherMode // mode of fetcher - SnapshotFilePath string // path to snapshot file - BLobSource BLobSource // blob source + FetcherMode FetcherMode // mode of fetcher + SnapshotFilePath string // path to snapshot file + BlobSource blob_client.BlobSource // blob source } // defaultSyncInterval is the frequency at which we query for new rollup event. @@ -30,39 +32,44 @@ type SyncingPipeline struct { db ethdb.Database blockchain *core.BlockChain blockQueue *BlockQueue - daSyncer *DaSyncer + daSyncer *DASyncer } func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, ethClient sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*SyncingPipeline, error) { ctx, cancel := context.WithCancel(ctx) - var err error - l1Client, err := newL1Client(ctx, genesisConfig, ethClient) + scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() + if err != nil { + cancel() + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + + l1Client, err := rollup_sync_service.NewL1Client(ctx, ethClient, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) if err != nil { cancel() return nil, err } - var blobClient BlobClient - switch config.BLobSource { - case BlobScan: - blobClient = newBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanApiEndpoint) - case BlockNative: - blobClient = newBlockNativeClient(genesisConfig.Scroll.DAConfig.BlockNativeApiEndpoint) + var blobClient blob_client.BlobClient + switch config.BlobSource { + case blob_client.BlobScan: + blobClient = blob_client.NewBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanAPIEndpoint) + case blob_client.BlockNative: + blobClient = blob_client.NewBlockNativeClient(genesisConfig.Scroll.DAConfig.BlockNativeAPIEndpoint) default: cancel() - return nil, fmt.Errorf("unknown blob scan client: %d", config.BLobSource) + return nil, fmt.Errorf("unknown blob scan client: %d", config.BlobSource) } dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClient, db) - var syncedL1Height uint64 = l1DeploymentBlock - 1 + syncedL1Height := l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) if from != nil { syncedL1Height = *from } - daQueue := NewDaQueue(syncedL1Height, dataSourceFactory) - batchQueue := NewBatchQueue(daQueue, db) + DAQueue := NewDAQueue(syncedL1Height, dataSourceFactory) + batchQueue := NewBatchQueue(DAQueue, db) blockQueue := NewBlockQueue(batchQueue) - daSyncer := NewDaSyncer(blockchain) + daSyncer := NewDASyncer(blockchain) return &SyncingPipeline{ ctx: ctx, @@ -74,8 +81,8 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi }, nil } -func (sp *SyncingPipeline) Step(ctx context.Context) error { - block, err := sp.blockQueue.NextBlock(ctx) +func (sp *SyncingPipeline) Step() error { + block, err := sp.blockQueue.NextBlock(sp.ctx) if err != nil { return err } @@ -84,10 +91,6 @@ func (sp *SyncingPipeline) Step(ctx context.Context) error { } func (sp *SyncingPipeline) Start() { - if sp == nil { - return - } - log.Info("Starting SyncingPipeline") go func() { @@ -95,7 +98,7 @@ func (sp *SyncingPipeline) Start() { defer syncTicker.Stop() for { - err := sp.Step(sp.ctx) + err := sp.Step() if err != nil { if strings.HasPrefix(err.Error(), "not consecutive block") { log.Warn("syncing pipeline step failed, probably because of restart", "err", err) @@ -107,6 +110,11 @@ func (sp *SyncingPipeline) Start() { case <-sp.ctx.Done(): return case <-syncTicker.C: + select { + case <-sp.ctx.Done(): + return + default: + } continue } } @@ -114,13 +122,6 @@ func (sp *SyncingPipeline) Start() { } func (sp *SyncingPipeline) Stop() { - if sp == nil { - return - } - log.Info("Stopping DaSyncer") - - if sp.cancel != nil { - sp.cancel() - } + sp.cancel() } diff --git a/rollup/rollup_sync_service/abi.go b/rollup/rollup_sync_service/abi.go index 6975001f1870..ff423d5c21bd 100644 --- a/rollup/rollup_sync_service/abi.go +++ b/rollup/rollup_sync_service/abi.go @@ -11,7 +11,7 @@ import ( ) // scrollChainMetaData contains ABI of the ScrollChain contract. -var scrollChainMetaData = &bind.MetaData{ +var ScrollChainMetaData = &bind.MetaData{ ABI: "[{\"anonymous\": false,\"inputs\": [{\"indexed\": true,\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"},{\"indexed\": true,\"internalType\": \"bytes32\",\"name\": \"batchHash\",\"type\": \"bytes32\"}],\"name\": \"CommitBatch\",\"type\": \"event\"},{\"anonymous\": false,\"inputs\": [{\"indexed\": true,\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"},{\"indexed\": true,\"internalType\": \"bytes32\",\"name\": \"batchHash\",\"type\": \"bytes32\"},{\"indexed\": false,\"internalType\": \"bytes32\",\"name\": \"stateRoot\",\"type\": \"bytes32\"},{\"indexed\": false,\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"}],\"name\": \"FinalizeBatch\",\"type\": \"event\"},{\"anonymous\": false,\"inputs\": [{\"indexed\": true,\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"},{\"indexed\": true,\"internalType\": \"bytes32\",\"name\": \"batchHash\",\"type\": \"bytes32\"}],\"name\": \"RevertBatch\",\"type\": \"event\"},{\"anonymous\": false,\"inputs\": [{\"indexed\": false,\"internalType\": \"uint256\",\"name\": \"oldMaxNumTxInChunk\",\"type\": \"uint256\"},{\"indexed\": false,\"internalType\": \"uint256\",\"name\": \"newMaxNumTxInChunk\",\"type\": \"uint256\"}],\"name\": \"UpdateMaxNumTxInChunk\",\"type\": \"event\"},{\"anonymous\": false,\"inputs\": [{\"indexed\": true,\"internalType\": \"address\",\"name\": \"account\",\"type\": \"address\"},{\"indexed\": false,\"internalType\": \"bool\",\"name\": \"status\",\"type\": \"bool\"}],\"name\": \"UpdateProver\",\"type\": \"event\"},{\"anonymous\": false,\"inputs\": [{\"indexed\": true,\"internalType\": \"address\",\"name\": \"account\",\"type\": \"address\"},{\"indexed\": false,\"internalType\": \"bool\",\"name\": \"status\",\"type\": \"bool\"}],\"name\": \"UpdateSequencer\",\"type\": \"event\"},{\"inputs\": [{\"internalType\": \"uint8\",\"name\": \"version\",\"type\": \"uint8\"},{\"internalType\": \"bytes\",\"name\": \"parentBatchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes[]\",\"name\": \"chunks\",\"type\": \"bytes[]\"},{\"internalType\": \"bytes\",\"name\": \"skippedL1MessageBitmap\",\"type\": \"bytes\"}],\"name\": \"commitBatch\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"uint8\",\"name\": \"version\",\"type\": \"uint8\"},{\"internalType\": \"bytes\",\"name\": \"parentBatchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes[]\",\"name\": \"chunks\",\"type\": \"bytes[]\"},{\"internalType\": \"bytes\",\"name\": \"skippedL1MessageBitmap\",\"type\": \"bytes\"},{\"internalType\": \"bytes\",\"name\": \"blobDataProof\",\"type\": \"bytes\"}],\"name\": \"commitBatchWithBlobProof\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"}],\"name\": \"committedBatches\",\"outputs\": [{\"internalType\": \"bytes32\",\"name\": \"\",\"type\": \"bytes32\"}],\"stateMutability\": \"view\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"prevStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"}],\"name\": \"finalizeBatch\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"prevStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes\",\"name\": \"blobDataProof\",\"type\": \"bytes\"}],\"name\": \"finalizeBatch4844\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"prevStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes\",\"name\": \"aggrProof\",\"type\": \"bytes\"}],\"name\": \"finalizeBatchWithProof\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"prevStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes\",\"name\": \"blobDataProof\",\"type\": \"bytes\"},{\"internalType\": \"bytes\",\"name\": \"aggrProof\",\"type\": \"bytes\"}],\"name\": \"finalizeBatchWithProof4844\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"}],\"name\": \"finalizeBundle\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"postStateRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes32\",\"name\": \"withdrawRoot\",\"type\": \"bytes32\"},{\"internalType\": \"bytes\",\"name\": \"aggrProof\",\"type\": \"bytes\"}],\"name\": \"finalizeBundleWithProof\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"}],\"name\": \"finalizedStateRoots\",\"outputs\": [{\"internalType\": \"bytes32\",\"name\": \"\",\"type\": \"bytes32\"}],\"stateMutability\": \"view\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"_batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"bytes32\",\"name\": \"_stateRoot\",\"type\": \"bytes32\"}],\"name\": \"importGenesisBatch\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"}],\"name\": \"isBatchFinalized\",\"outputs\": [{\"internalType\": \"bool\",\"name\": \"\",\"type\": \"bool\"}],\"stateMutability\": \"view\",\"type\": \"function\"},{\"inputs\": [],\"name\": \"lastFinalizedBatchIndex\",\"outputs\": [{\"internalType\": \"uint256\",\"name\": \"\",\"type\": \"uint256\"}],\"stateMutability\": \"view\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"bytes\",\"name\": \"batchHeader\",\"type\": \"bytes\"},{\"internalType\": \"uint256\",\"name\": \"count\",\"type\": \"uint256\"}],\"name\": \"revertBatch\",\"outputs\": [],\"stateMutability\": \"nonpayable\",\"type\": \"function\"},{\"inputs\": [{\"internalType\": \"uint256\",\"name\": \"batchIndex\",\"type\": \"uint256\"}],\"name\": \"withdrawRoots\",\"outputs\": [{\"internalType\": \"bytes32\",\"name\": \"\",\"type\": \"bytes32\"}],\"stateMutability\": \"view\",\"type\": \"function\"}]", } diff --git a/rollup/rollup_sync_service/abi_test.go b/rollup/rollup_sync_service/abi_test.go index d47a2c72e190..550c950bb337 100644 --- a/rollup/rollup_sync_service/abi_test.go +++ b/rollup/rollup_sync_service/abi_test.go @@ -13,7 +13,7 @@ import ( ) func TestEventSignatures(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() if err != nil { t.Fatal("failed to get scroll chain abi", "err", err) } @@ -24,7 +24,7 @@ func TestEventSignatures(t *testing.T) { } func TestUnpackLog(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) mockBatchIndex := big.NewInt(123) diff --git a/rollup/rollup_sync_service/l1client.go b/rollup/rollup_sync_service/l1client.go index 34ffc4db1bc2..2cd5efc625d5 100644 --- a/rollup/rollup_sync_service/l1client.go +++ b/rollup/rollup_sync_service/l1client.go @@ -29,7 +29,7 @@ type L1Client struct { // newL1Client initializes a new L1Client instance with the provided configuration. // It checks for a valid scrollChainAddress and verifies the chain ID. -func newL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId uint64, scrollChainAddress common.Address, scrollChainABI *abi.ABI) (*L1Client, error) { +func NewL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId uint64, scrollChainAddress common.Address, scrollChainABI *abi.ABI) (*L1Client, error) { if scrollChainAddress == (common.Address{}) { return nil, errors.New("must pass non-zero scrollChainAddress to L1Client") } @@ -56,7 +56,7 @@ func newL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId } // fetcRollupEventsInRange retrieves and parses commit/revert/finalize rollup events between block numbers: [from, to]. -func (c *L1Client) fetchRollupEventsInRange(from, to uint64) ([]types.Log, error) { +func (c *L1Client) FetchRollupEventsInRange(from, to uint64) ([]types.Log, error) { log.Trace("L1Client fetchRollupEventsInRange", "fromBlock", from, "toBlock", to) query := ethereum.FilterQuery{ @@ -80,7 +80,7 @@ func (c *L1Client) fetchRollupEventsInRange(from, to uint64) ([]types.Log, error } // getLatestFinalizedBlockNumber fetches the block number of the latest finalized block from the L1 chain. -func (c *L1Client) getLatestFinalizedBlockNumber() (uint64, error) { +func (c *L1Client) GetLatestFinalizedBlockNumber() (uint64, error) { header, err := c.client.HeaderByNumber(c.ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) if err != nil { return 0, err @@ -90,3 +90,60 @@ func (c *L1Client) getLatestFinalizedBlockNumber() (uint64, error) { } return header.Number.Uint64(), nil } + +// fetchTxData fetches tx data corresponding to given event log +func (c *L1Client) FetchTxData(vLog *types.Log) ([]byte, error) { + tx, _, err := c.client.TransactionByHash(c.ctx, vLog.TxHash) + if err != nil { + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := c.client.BlockByHash(c.ctx, vLog.BlockHash) + if err != nil { + return nil, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return nil, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } + } + + return tx.Data(), nil +} + +// fetchTxBlobHash fetches tx blob hash corresponding to given event log +func (c *L1Client) FetchTxBlobHash(vLog *types.Log) (common.Hash, error) { + tx, _, err := c.client.TransactionByHash(c.ctx, vLog.TxHash) + if err != nil { + log.Debug("failed to get transaction by hash, probably an unindexed transaction, fetching the whole block to get the transaction", + "tx hash", vLog.TxHash.Hex(), "block number", vLog.BlockNumber, "block hash", vLog.BlockHash.Hex(), "err", err) + block, err := c.client.BlockByHash(c.ctx, vLog.BlockHash) + if err != nil { + return common.Hash{}, fmt.Errorf("failed to get block by hash, block number: %v, block hash: %v, err: %w", vLog.BlockNumber, vLog.BlockHash.Hex(), err) + } + + found := false + for _, txInBlock := range block.Transactions() { + if txInBlock.Hash() == vLog.TxHash { + tx = txInBlock + found = true + break + } + } + if !found { + return common.Hash{}, fmt.Errorf("transaction not found in the block, tx hash: %v, block number: %v, block hash: %v", vLog.TxHash.Hex(), vLog.BlockNumber, vLog.BlockHash.Hex()) + } + } + blobHashes := tx.BlobHashes() + if len(blobHashes) == 0 { + return common.Hash{}, fmt.Errorf("transaction does not contain any blobs, tx hash: %v", vLog.TxHash.Hex()) + } + return blobHashes[0], nil +} diff --git a/rollup/rollup_sync_service/l1client_test.go b/rollup/rollup_sync_service/l1client_test.go index 8c7bd92f8b11..38719d220f62 100644 --- a/rollup/rollup_sync_service/l1client_test.go +++ b/rollup/rollup_sync_service/l1client_test.go @@ -18,19 +18,19 @@ func TestL1Client(t *testing.T) { ctx := context.Background() mockClient := &mockEthClient{} - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() if err != nil { t.Fatal("failed to get scroll chain abi", "err", err) } scrollChainAddress := common.HexToAddress("0x0123456789abcdef") - l1Client, err := newL1Client(ctx, mockClient, 11155111, scrollChainAddress, scrollChainABI) + l1Client, err := NewL1Client(ctx, mockClient, 11155111, scrollChainAddress, scrollChainABI) require.NoError(t, err, "Failed to initialize L1Client") - blockNumber, err := l1Client.getLatestFinalizedBlockNumber() + blockNumber, err := l1Client.GetLatestFinalizedBlockNumber() assert.NoError(t, err, "Error getting latest confirmed block number") assert.Equal(t, uint64(36), blockNumber, "Unexpected block number") - logs, err := l1Client.fetchRollupEventsInRange(0, blockNumber) + logs, err := l1Client.FetchRollupEventsInRange(0, blockNumber) assert.NoError(t, err, "Error fetching rollup events in range") assert.Empty(t, logs, "Expected no logs from fetchRollupEventsInRange") } diff --git a/rollup/rollup_sync_service/rollup_sync_service.go b/rollup/rollup_sync_service/rollup_sync_service.go index 181e253335f6..cf57ba4cea8b 100644 --- a/rollup/rollup_sync_service/rollup_sync_service.go +++ b/rollup/rollup_sync_service/rollup_sync_service.go @@ -74,12 +74,12 @@ func NewRollupSyncService(ctx context.Context, genesisConfig *params.ChainConfig return nil, fmt.Errorf("missing L1 config in genesis") } - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() if err != nil { return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) } - client, err := newL1Client(ctx, l1Client, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) + client, err := NewL1Client(ctx, l1Client, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) if err != nil { return nil, fmt.Errorf("failed to initialize l1 client: %w", err) } @@ -156,7 +156,7 @@ func (s *RollupSyncService) Stop() { } func (s *RollupSyncService) fetchRollupEvents() { - latestConfirmed, err := s.client.getLatestFinalizedBlockNumber() + latestConfirmed, err := s.client.GetLatestFinalizedBlockNumber() if err != nil { log.Warn("failed to get latest confirmed block number", "err", err) return @@ -176,7 +176,7 @@ func (s *RollupSyncService) fetchRollupEvents() { to = latestConfirmed } - logs, err := s.client.fetchRollupEventsInRange(from, to) + logs, err := s.client.FetchRollupEventsInRange(from, to) if err != nil { log.Error("failed to fetch rollup events in range", "from block", from, "to block", to, "err", err) return diff --git a/rollup/rollup_sync_service/rollup_sync_service_test.go b/rollup/rollup_sync_service/rollup_sync_service_test.go index dea97613fb96..5ab88772d446 100644 --- a/rollup/rollup_sync_service/rollup_sync_service_test.go +++ b/rollup/rollup_sync_service/rollup_sync_service_test.go @@ -51,7 +51,7 @@ func TestRollupSyncServiceStartAndStop(t *testing.T) { } func TestDecodeChunkRangesCodecv0(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -108,7 +108,7 @@ func TestDecodeChunkRangesCodecv0(t *testing.T) { } func TestDecodeChunkRangesCodecv1(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -159,7 +159,7 @@ func TestDecodeChunkRangesCodecv1(t *testing.T) { } func TestDecodeChunkRangesCodecv2(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -210,7 +210,7 @@ func TestDecodeChunkRangesCodecv2(t *testing.T) { } func TestDecodeChunkRangesCodecv3(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ From f3a1c5ddf7028bfc2f02264884a4e0cabce9af67 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Thu, 25 Jul 2024 07:36:29 +0800 Subject: [PATCH 28/59] fix: nil pointer when enabling --da.sync=true --- eth/backend.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eth/backend.go b/eth/backend.go index fe36f39e5850..03c8144a2d30 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -595,7 +595,7 @@ func (s *Ethereum) Start() error { //} // Start the networking layer and the light server if requested // handler is not enabled when DA syncing enabled - if s.config.EnableDASyncing { + if !s.config.EnableDASyncing { s.handler.Start(maxPeers) } return nil From 293109993b95ad91d5176d95df712f75dfeba487 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Thu, 25 Jul 2024 09:03:09 +0800 Subject: [PATCH 29/59] fix: block not being inserted due to Clique verification --- consensus/clique/clique.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index 0c79d877e534..f5c512dff54b 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -352,6 +352,11 @@ func (c *Clique) verifyCascadingFields(chain consensus.ChainHeaderReader, header // Verify the header's EIP-1559 attributes. return err } + + if c.config.DaSyncingEnabled { + return nil + } + // Retrieve the snapshot needed to verify this header and cache it snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) if err != nil { From 1c36c2ce1afae3079e4fd03bb00a25da213543b4 Mon Sep 17 00:00:00 2001 From: Jonas Theis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:50:50 +0800 Subject: [PATCH 30/59] refactor: compose DA types for more cohesion, maintainability and code reuse (#925) * refactor: compose DA types for more cohesion, maintainability and code reuse * feat: simultaneous L1 message sync and L1 sync from DA instead of consecutive (#929) * feat: add exponential backoff utility * feat: add exponential backoff to pipeline * feat: run pipeline and L1 message sync in parallel * Update rollup/da_syncer/syncing_pipeline.go init waitgroup --------- Co-authored-by: Nazarii Denha * update da v2 to use correct blob decoding method * introduce NewCommitBatchDAV1WithBlobDecodeFunc --------- Co-authored-by: Nazarii Denha --- common/backoff/exponential.go | 50 +++ common/backoff/exponential_test.go | 39 ++ eth/backend.go | 20 +- go.mod | 2 +- go.sum | 4 +- rollup/da_syncer/batch_queue.go | 37 +- rollup/da_syncer/block_queue.go | 121 +------ rollup/da_syncer/calldata_blob_source.go | 377 -------------------- rollup/da_syncer/da.go | 186 ---------- rollup/da_syncer/da/calldata_blob_source.go | 220 ++++++++++++ rollup/da_syncer/da/commitV0.go | 160 +++++++++ rollup/da_syncer/da/commitV1.go | 89 +++++ rollup/da_syncer/da/commitV2.go | 40 +++ rollup/da_syncer/da/da.go | 35 ++ rollup/da_syncer/da/finalize.go | 45 +++ rollup/da_syncer/da/revert.go | 24 ++ rollup/da_syncer/da_queue.go | 10 +- rollup/da_syncer/data_source.go | 9 +- rollup/da_syncer/syncing_pipeline.go | 123 +++++-- 19 files changed, 838 insertions(+), 753 deletions(-) create mode 100644 common/backoff/exponential.go create mode 100644 common/backoff/exponential_test.go delete mode 100644 rollup/da_syncer/calldata_blob_source.go delete mode 100644 rollup/da_syncer/da.go create mode 100644 rollup/da_syncer/da/calldata_blob_source.go create mode 100644 rollup/da_syncer/da/commitV0.go create mode 100644 rollup/da_syncer/da/commitV1.go create mode 100644 rollup/da_syncer/da/commitV2.go create mode 100644 rollup/da_syncer/da/da.go create mode 100644 rollup/da_syncer/da/finalize.go create mode 100644 rollup/da_syncer/da/revert.go diff --git a/common/backoff/exponential.go b/common/backoff/exponential.go new file mode 100644 index 000000000000..4e43c4e13346 --- /dev/null +++ b/common/backoff/exponential.go @@ -0,0 +1,50 @@ +package backoff + +import ( + "math" + "math/rand" + "time" +) + +type Exponential struct { + attempt int + + maxJitter time.Duration + + min time.Duration + max time.Duration +} + +func NewExponential(minimum, maximum, maxJitter time.Duration) *Exponential { + return &Exponential{ + min: minimum, + max: maximum, + maxJitter: maxJitter, + } +} + +func (e *Exponential) NextDuration() time.Duration { + var jitter time.Duration + if e.maxJitter > 0 { + jitter = time.Duration(rand.Int63n(e.maxJitter.Nanoseconds())) + } + + minFloat := float64(e.min) + duration := math.Pow(2, float64(e.attempt)) * minFloat + + // limit at configured maximum + if duration > float64(e.max) { + duration = float64(e.max) + } + + e.attempt++ + return time.Duration(duration) + jitter +} + +func (e *Exponential) Reset() { + e.attempt = 0 +} + +func (e *Exponential) Attempt() int { + return e.attempt +} diff --git a/common/backoff/exponential_test.go b/common/backoff/exponential_test.go new file mode 100644 index 000000000000..ff659337a2b0 --- /dev/null +++ b/common/backoff/exponential_test.go @@ -0,0 +1,39 @@ +package backoff + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestExponentialBackoff(t *testing.T) { + t.Run("Multiple attempts", func(t *testing.T) { + e := NewExponential(100*time.Millisecond, 10*time.Second, 0) + expectedDurations := []time.Duration{ + 100 * time.Millisecond, + 200 * time.Millisecond, + 400 * time.Millisecond, + 800 * time.Millisecond, + 1600 * time.Millisecond, + 3200 * time.Millisecond, + 6400 * time.Millisecond, + 10 * time.Second, // capped at max + } + for i, expected := range expectedDurations { + require.Equal(t, expected, e.NextDuration(), "attempt %d", i) + } + }) + + t.Run("Jitter added", func(t *testing.T) { + e := NewExponential(1*time.Second, 10*time.Second, 1*time.Second) + duration := e.NextDuration() + require.GreaterOrEqual(t, duration, 1*time.Second) + require.Less(t, duration, 2*time.Second) + }) + + t.Run("Edge case: min > max", func(t *testing.T) { + e := NewExponential(10*time.Second, 5*time.Second, 0) + require.Equal(t, 5*time.Second, e.NextDuration()) + }) +} diff --git a/eth/backend.go b/eth/backend.go index 03c8144a2d30..90cab3c9e181 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -221,6 +221,18 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl } eth.txPool = core.NewTxPool(config.TxPool, chainConfig, eth.blockchain) + // Initialize and start DA syncing pipeline before SyncService as SyncService is blocking until all L1 messages are loaded. + // We need SyncService to load the L1 messages for DA syncing, but since both sync from last known L1 state, we can + // simply let them run simultaneously. If messages are missing in DA syncing, it will be handled by the syncing pipeline + // by waiting and retrying. + if config.EnableDASyncing { + eth.syncingPipeline, err = da_syncer.NewSyncingPipeline(context.Background(), eth.blockchain, chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) + if err != nil { + return nil, fmt.Errorf("cannot initialize da syncer: %w", err) + } + 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 { @@ -237,14 +249,6 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl eth.rollupSyncService.Start() } - if config.EnableDASyncing { - eth.syncingPipeline, err = da_syncer.NewSyncingPipeline(context.Background(), eth.blockchain, chainConfig, eth.chainDb, l1Client, stack.Config().L1DeploymentBlock, config.DA) - if err != nil { - return nil, fmt.Errorf("cannot initialize da syncer: %w", err) - } - eth.syncingPipeline.Start() - } - // Permit the downloader to use the trie cache allowance during fast sync cacheLimit := cacheConfig.TrieCleanLimit + cacheConfig.TrieDirtyLimit + cacheConfig.SnapshotLimit checkpoint := config.Checkpoint diff --git a/go.mod b/go.mod index aea7d762976c..851cbd50e2ca 100644 --- a/go.mod +++ b/go.mod @@ -50,7 +50,7 @@ require ( github.com/prometheus/tsdb v0.7.1 github.com/rjeczalik/notify v0.9.1 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f + github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 diff --git a/go.sum b/go.sum index 0b5044dffb3e..050f3b35a640 100644 --- a/go.sum +++ b/go.sum @@ -394,8 +394,8 @@ github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncj github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f h1:ZKPhn674+2AgBdIn2ZLGePsUZdM2823m2tJp+JlQf/Y= -github.com/scroll-tech/da-codec v0.1.1-0.20240708084945-cb02d638c45f/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= +github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a h1:2+g6p5dxRQYFjDMBZgBJkem2HpIJdXEIlXzxreTCd4w= +github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= github.com/scroll-tech/zktrie v0.8.4 h1:UagmnZ4Z3ITCk+aUq9NQZJNAwnWl4gSxsLb2Nl7IgRE= github.com/scroll-tech/zktrie v0.8.4/go.mod h1:XvNo7vAk8yxNyTjBDj5WIiFzYW4bx/gJ78+NK6Zn6Uk= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index b1ae23d04e16..aa38d8a64871 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -7,11 +7,12 @@ import ( "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BatchQueue struct { // batches is map from batchIndex to batch blocks - batches map[uint64]DAEntry + batches map[uint64]da.Entry DAQueue *DAQueue db ethdb.Database lastFinalizedBatchIndex uint64 @@ -19,7 +20,7 @@ type BatchQueue struct { func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { return &BatchQueue{ - batches: make(map[uint64]DAEntry), + batches: make(map[uint64]da.Entry), DAQueue: DAQueue, db: db, lastFinalizedBatchIndex: 0, @@ -27,7 +28,7 @@ func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { } // NextBatch finds next finalized batch and returns data, that was committed in that batch -func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { +func (bq *BatchQueue) NextBatch(ctx context.Context) (da.Entry, error) { if batch, ok := bq.getFinalizedBatch(); ok { return batch, nil } @@ -36,18 +37,14 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { if err != nil { return nil, err } - switch daEntry := daEntry.(type) { - case *CommitBatchDAV0: - bq.batches[daEntry.BatchIndex] = daEntry - case *CommitBatchDAV1: - bq.batches[daEntry.BatchIndex] = daEntry - case *CommitBatchDAV2: - bq.batches[daEntry.BatchIndex] = daEntry - case *RevertBatchDA: - bq.deleteBatch(daEntry.BatchIndex) - case *FinalizeBatchDA: - if daEntry.BatchIndex > bq.lastFinalizedBatchIndex { - bq.lastFinalizedBatchIndex = daEntry.BatchIndex + switch daEntry.Type() { + case da.CommitBatchV0Type, da.CommitBatchV1Type, da.CommitBatchV2Type: + bq.batches[daEntry.BatchIndex()] = daEntry + case da.RevertBatchType: + bq.deleteBatch(daEntry.BatchIndex()) + case da.FinalizeBatchType: + if daEntry.BatchIndex() > bq.lastFinalizedBatchIndex { + bq.lastFinalizedBatchIndex = daEntry.BatchIndex() } ret, ok := bq.getFinalizedBatch() if ok { @@ -62,7 +59,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (DAEntry, error) { } // getFinalizedBatch returns next finalized batch if there is available -func (bq *BatchQueue) getFinalizedBatch() (DAEntry, bool) { +func (bq *BatchQueue) getFinalizedBatch() (da.Entry, bool) { if len(bq.batches) == 0 { return nil, false } @@ -93,7 +90,7 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { if !ok { return } - curBatchL1Height := batch.GetL1BlockNumber() + curBatchL1Height := batch.L1BlockNumber() delete(bq.batches, batchIndex) if len(bq.batches) == 0 { rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) @@ -102,10 +99,10 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart var minBatchL1Height uint64 = math.MaxUint64 for _, val := range bq.batches { - if val.GetL1BlockNumber() < minBatchL1Height { - minBatchL1Height = val.GetL1BlockNumber() + if val.L1BlockNumber() < minBatchL1Height { + minBatchL1Height = val.L1BlockNumber() } } - rawdb.WriteDASyncedL1BlockNumber(bq.db, minBatchL1Height-1) + rawdb.WriteDASyncedL1BlockNumber(bq.db, minBatchL1Height-1) } diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 6340bc33db18..abd728912b21 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -3,9 +3,9 @@ package da_syncer import ( "context" "fmt" - "math/big" "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BlockQueue struct { @@ -37,120 +37,17 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { if err != nil { return err } - switch daEntry := daEntry.(type) { - case *CommitBatchDAV0: - bq.blocks, err = bq.processDAV0ToBlocks(daEntry) - if err != nil { - return err - } - case *CommitBatchDAV1: - bq.blocks, err = bq.processDAV1ToBlocks(daEntry) - if err != nil { - return err - } - case *CommitBatchDAV2: - bq.blocks, err = bq.processDAV2ToBlocks(daEntry) - if err != nil { - return err - } - default: - return fmt.Errorf("unexpected type of daEntry: %T", daEntry) - } - return nil -} -func (bq *BlockQueue) processDAV0ToBlocks(daEntry *CommitBatchDAV0) ([]*types.Block, error) { - var blocks []*types.Block - l1TxPointer := 0 - var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped - for _, chunk := range daEntry.Chunks { - for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } - // create txs - // var txs types.Transactions - txs := make(types.Transactions, 0, daBlock.NumTransactions) - // insert l1 msgs - for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) - txs = append(txs, l1Tx) - l1TxPointer++ - } - curL1TxIndex += uint64(daBlock.NumL1Messages) - // insert l2 txs - txs = append(txs, chunk.Transactions[blockId]...) - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) - blocks = append(blocks, block) - } + entryWithBlocks, ok := daEntry.(da.EntryWithBlocks) + // this should never happen because we only receive CommitBatch entries + if !ok { + return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } - return blocks, nil -} -func (bq *BlockQueue) processDAV1ToBlocks(daEntry *CommitBatchDAV1) ([]*types.Block, error) { - var blocks []*types.Block - l1TxPointer := 0 - var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped - for _, chunk := range daEntry.Chunks { - for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } - // create txs - // var txs types.Transactions - txs := make(types.Transactions, 0, daBlock.NumTransactions) - // insert l1 msgs - for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) - txs = append(txs, l1Tx) - l1TxPointer++ - } - curL1TxIndex += uint64(daBlock.NumL1Messages) - // insert l2 txs - txs = append(txs, chunk.Transactions[blockId]...) - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) - blocks = append(blocks, block) - } + bq.blocks, err = entryWithBlocks.Blocks() + if err != nil { + return fmt.Errorf("failed to get blocks from daEntry: %w", err) } - return blocks, nil -} -func (bq *BlockQueue) processDAV2ToBlocks(daEntry *CommitBatchDAV2) ([]*types.Block, error) { - var blocks []*types.Block - l1TxPointer := 0 - var curL1TxIndex uint64 = daEntry.ParentTotalL1MessagePopped - for _, chunk := range daEntry.Chunks { - for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } - // create txs - // var txs types.Transactions - txs := make(types.Transactions, 0, daBlock.NumTransactions) - // insert l1 msgs - for l1TxPointer < len(daEntry.L1Txs) && daEntry.L1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { - l1Tx := types.NewTx(daEntry.L1Txs[l1TxPointer]) - txs = append(txs, l1Tx) - l1TxPointer++ - } - curL1TxIndex += uint64(daBlock.NumL1Messages) - // insert l2 txs - txs = append(txs, chunk.Transactions[blockId]...) - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) - blocks = append(blocks, block) - } - } - return blocks, nil + return nil } diff --git a/rollup/da_syncer/calldata_blob_source.go b/rollup/da_syncer/calldata_blob_source.go deleted file mode 100644 index 94487166772c..000000000000 --- a/rollup/da_syncer/calldata_blob_source.go +++ /dev/null @@ -1,377 +0,0 @@ -package da_syncer - -import ( - "context" - "crypto/sha256" - "encoding/binary" - "fmt" - - "github.com/scroll-tech/da-codec/encoding" - "github.com/scroll-tech/da-codec/encoding/codecv0" - "github.com/scroll-tech/da-codec/encoding/codecv1" - "github.com/scroll-tech/da-codec/encoding/codecv2" - - "github.com/scroll-tech/go-ethereum/accounts/abi" - "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/crypto/kzg4844" - "github.com/scroll-tech/go-ethereum/ethdb" - "github.com/scroll-tech/go-ethereum/log" - - "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" - "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" -) - -var ( - callDataBlobSourceFetchBlockRange uint64 = 500 - commitBatchEventName = "CommitBatch" - revertBatchEventName = "RevertBatch" - finalizeBatchEventName = "FinalizeBatch" - commitBatchMethodName = "commitBatch" - commitBatchWithBlobProofMethodName = "commitBatchWithBlobProof" - - // the length og method ID at the beginning of transaction data - methodIDLength = 4 -) - -type CalldataBlobSource struct { - ctx context.Context - l1Client *rollup_sync_service.L1Client - blobClient blob_client.BlobClient - l1height uint64 - scrollChainABI *abi.ABI - l1CommitBatchEventSignature common.Hash - l1RevertBatchEventSignature common.Hash - l1FinalizeBatchEventSignature common.Hash - db ethdb.Database -} - -func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *rollup_sync_service.L1Client, blobClient blob_client.BlobClient, db ethdb.Database) (DataSource, error) { - scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() - if err != nil { - return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) - } - return &CalldataBlobSource{ - ctx: ctx, - l1Client: l1Client, - blobClient: blobClient, - l1height: l1height, - scrollChainABI: scrollChainABI, - l1CommitBatchEventSignature: scrollChainABI.Events[commitBatchEventName].ID, - l1RevertBatchEventSignature: scrollChainABI.Events[revertBatchEventName].ID, - l1FinalizeBatchEventSignature: scrollChainABI.Events[finalizeBatchEventName].ID, - db: db, - }, nil -} - -func (ds *CalldataBlobSource) NextData() (DA, error) { - to := ds.l1height + callDataBlobSourceFetchBlockRange - l1Finalized, err := ds.l1Client.GetLatestFinalizedBlockNumber() - if err != nil { - return nil, fmt.Errorf("cannot get l1height, error: %v", err) - } - if to > l1Finalized { - to = l1Finalized - } - if ds.l1height > to { - return nil, errSourceExhausted - } - logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) - if err != nil { - return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) - } - da, err := ds.processLogsToDA(logs) - if err == nil { - ds.l1height = to + 1 - } - return da, err -} - -func (ds *CalldataBlobSource) L1Height() uint64 { - return ds.l1height -} - -func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (DA, error) { - var da DA - for _, vLog := range logs { - switch vLog.Topics[0] { - case ds.l1CommitBatchEventSignature: - event := &rollup_sync_service.L1CommitBatchEvent{} - if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "CommitBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new CommitBatch event", "batch index", batchIndex) - - daEntry, err := ds.getCommitBatchDa(batchIndex, &vLog) - if err != nil { - return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) - } - da = append(da, daEntry) - - case ds.l1RevertBatchEventSignature: - event := &rollup_sync_service.L1RevertBatchEvent{} - if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "RevertBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new RevertBatch event", "batch index", batchIndex) - da = append(da, NewRevertBatchDA(batchIndex)) - - case ds.l1FinalizeBatchEventSignature: - event := &rollup_sync_service.L1FinalizeBatchEvent{} - if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, "FinalizeBatch", vLog); err != nil { - return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) - } - batchIndex := event.BatchIndex.Uint64() - log.Trace("found new FinalizeBatch event", "batch index", batchIndex) - - da = append(da, NewFinalizeBatchDA(batchIndex)) - - default: - return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) - } - } - return da, nil -} - -type commitBatchArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte -} - -func newCommitBatchArgs(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { - var args commitBatchArgs - err := method.Inputs.Copy(&args, values) - return &args, err -} - -func newCommitBatchArgsFromCommitBatchWithProof(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { - var args commitBatchWithBlobProofArgs - err := method.Inputs.Copy(&args, values) - if err != nil { - return nil, err - } - return &commitBatchArgs{ - Version: args.Version, - ParentBatchHeader: args.ParentBatchHeader, - Chunks: args.Chunks, - SkippedL1MessageBitmap: args.SkippedL1MessageBitmap, - }, nil -} - -type commitBatchWithBlobProofArgs struct { - Version uint8 - ParentBatchHeader []byte - Chunks [][]byte - SkippedL1MessageBitmap []byte - BlobDataProof []byte -} - -func (ds *CalldataBlobSource) getCommitBatchDa(batchIndex uint64, vLog *types.Log) (DAEntry, error) { - if batchIndex == 0 { - return NewCommitBatchDAV0(0, batchIndex, 0, []byte{}, []*codecv0.DAChunkRawTx{}, []*types.L1MessageTx{}, 0), nil - } - - txData, err := ds.l1Client.FetchTxData(vLog) - if err != nil { - return nil, err - } - if len(txData) < methodIDLength { - return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) - } - - method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) - if err != nil { - return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) - } - values, err := method.Inputs.Unpack(txData[methodIDLength:]) - if err != nil { - return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) - } - - if method.Name == commitBatchMethodName { - args, err := newCommitBatchArgs(method, values) - if err != nil { - return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - switch args.Version { - case 0: - return ds.decodeDAV0(batchIndex, vLog, args) - case 1: - return ds.decodeDAV1(batchIndex, vLog, args) - case 2: - return ds.decodeDAV2(batchIndex, vLog, args) - default: - return nil, fmt.Errorf("failed to decode DA, codec version is unknown: codec version: %d", args.Version) - } - } else if method.Name == commitBatchWithBlobProofMethodName { - args, err := newCommitBatchArgsFromCommitBatchWithProof(method, values) - if err != nil { - return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) - } - return ds.decodeDAV2(batchIndex, vLog, args) - } - return nil, fmt.Errorf("unknown method name: %s", method.Name) -} - -func (ds *CalldataBlobSource) decodeDAV0(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { - var chunks []*codecv0.DAChunkRawTx - var l1Txs []*types.L1MessageTx - chunks, err := codecv0.DecodeDAChunksRawTx(args.Chunks) - if err != nil { - return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - - parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) - totalL1MessagePopped := 0 - for _, chunk := range chunks { - for _, block := range chunk.Blocks { - totalL1MessagePopped += int(block.NumL1Messages) - } - } - skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < totalL1MessagePopped; index++ { - if encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - continue - } - l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - if l1Tx == nil { - return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - da := NewCommitBatchDAV0(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) - return da, nil -} - -func (ds *CalldataBlobSource) decodeDAV1(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { - var chunks []*codecv1.DAChunkRawTx - var l1Txs []*types.L1MessageTx - chunks, err := codecv1.DecodeDAChunksRawTx(args.Chunks) - if err != nil { - return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - - versionedHash, err := ds.l1Client.FetchTxBlobHash(vLog) - if err != nil { - return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) - } - blob, err := ds.blobClient.GetBlobByVersionedHash(ds.ctx, versionedHash) - if err != nil { - return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) - } - // compute blob versioned hash and compare with one from tx - c, err := kzg4844.BlobToCommitment(blob) - if err != nil { - return nil, fmt.Errorf("failed to create blob commitment") - } - blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) - if blobVersionedHash != versionedHash { - return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) - } - // decode txs from blob - err = codecv1.DecodeTxsFromBlob(blob, chunks) - if err != nil { - return nil, fmt.Errorf("failed to decode txs from blob: %w", err) - } - parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) - totalL1MessagePopped := 0 - for _, chunk := range chunks { - for _, block := range chunk.Blocks { - totalL1MessagePopped += int(block.NumL1Messages) - } - } - skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < totalL1MessagePopped; index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - } - l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - if l1Tx == nil { - return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - da := NewCommitBatchDAV1(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) - return da, nil -} - -func (ds *CalldataBlobSource) decodeDAV2(batchIndex uint64, vLog *types.Log, args *commitBatchArgs) (DAEntry, error) { - var chunks []*codecv2.DAChunkRawTx - var l1Txs []*types.L1MessageTx - chunks, err := codecv2.DecodeDAChunksRawTx(args.Chunks) - if err != nil { - return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) - } - - versionedHash, err := ds.l1Client.FetchTxBlobHash(vLog) - if err != nil { - return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) - } - blob, err := ds.blobClient.GetBlobByVersionedHash(ds.ctx, versionedHash) - if err != nil { - return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) - } - // compute blob versioned hash and compare with one from tx - c, err := kzg4844.BlobToCommitment(blob) - if err != nil { - return nil, fmt.Errorf("failed to create blob commitment") - } - blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) - if blobVersionedHash != versionedHash { - return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) - } - // decode txs from blob - err = codecv2.DecodeTxsFromBlob(blob, chunks) - if err != nil { - return nil, fmt.Errorf("failed to decode txs from blob: %w", err) - } - parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(args.ParentBatchHeader) - totalL1MessagePopped := 0 - for _, chunk := range chunks { - for _, block := range chunk.Blocks { - totalL1MessagePopped += int(block.NumL1Messages) - } - } - skippedBitmap, err := encoding.DecodeBitmap(args.SkippedL1MessageBitmap, totalL1MessagePopped) - if err != nil { - return nil, fmt.Errorf("failed to decode bitmap: %v, err: %w", batchIndex, err) - } - // get all necessary l1msgs without skipped - currentIndex := parentTotalL1MessagePopped - for index := 0; index < totalL1MessagePopped; index++ { - for encoding.IsL1MessageSkipped(skippedBitmap, currentIndex-parentTotalL1MessagePopped) { - currentIndex++ - } - l1Tx := rawdb.ReadL1Message(ds.db, currentIndex) - if l1Tx == nil { - return nil, fmt.Errorf("failed to read L1 message from db, l1 message index: %v", currentIndex) - } - l1Txs = append(l1Txs, l1Tx) - currentIndex++ - } - da := NewCommitBatchDAV2(args.Version, batchIndex, parentTotalL1MessagePopped, args.SkippedL1MessageBitmap, chunks, l1Txs, vLog.BlockNumber) - return da, nil -} - -func getBatchTotalL1MessagePopped(data []byte) uint64 { - // total l1 message popped stored in bytes from 17 to 24, accordingly to codec spec - return binary.BigEndian.Uint64(data[17:25]) -} diff --git a/rollup/da_syncer/da.go b/rollup/da_syncer/da.go deleted file mode 100644 index f2f8bdd9b903..000000000000 --- a/rollup/da_syncer/da.go +++ /dev/null @@ -1,186 +0,0 @@ -package da_syncer - -import ( - "github.com/scroll-tech/da-codec/encoding/codecv0" - "github.com/scroll-tech/da-codec/encoding/codecv1" - "github.com/scroll-tech/da-codec/encoding/codecv2" - - "github.com/scroll-tech/go-ethereum/core/types" -) - -type DAType int - -const ( - // CommitBatchV0 contains data of event of CommitBatchV0 - CommitBatchV0 DAType = iota - // CommitBatchV1 contains data of event of CommitBatchV1 - CommitBatchV1 - // CommitBatchV2 contains data of event of CommitBatchV2 - CommitBatchV2 - // RevertBatch contains data of event of RevertBatch - RevertBatch - // FinalizeBatch contains data of event of FinalizeBatch - FinalizeBatch - // FinalizeBatchV3 contains data of event of FinalizeBatch v3 - FinalizeBatchV3 -) - -type DAEntry interface { - DAType() DAType - GetL1BlockNumber() uint64 -} - -type DA []DAEntry - -type CommitBatchDAV0 struct { - Version uint8 - BatchIndex uint64 - ParentTotalL1MessagePopped uint64 - SkippedL1MessageBitmap []byte - Chunks []*codecv0.DAChunkRawTx - L1Txs []*types.L1MessageTx - - L1BlockNumber uint64 -} - -func NewCommitBatchDAV0(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv0.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDAV0{ - Version: version, - BatchIndex: batchIndex, - ParentTotalL1MessagePopped: parentTotalL1MessagePopped, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, - } -} - -func (f *CommitBatchDAV0) DAType() DAType { - return CommitBatchV0 -} - -func (f *CommitBatchDAV0) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} - -type CommitBatchDAV1 struct { - Version uint8 - BatchIndex uint64 - ParentTotalL1MessagePopped uint64 - SkippedL1MessageBitmap []byte - Chunks []*codecv1.DAChunkRawTx - L1Txs []*types.L1MessageTx - - L1BlockNumber uint64 -} - -func NewCommitBatchDAV1(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv1.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDAV1{ - Version: version, - BatchIndex: batchIndex, - ParentTotalL1MessagePopped: parentTotalL1MessagePopped, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, - } -} - -func (f *CommitBatchDAV1) DAType() DAType { - return CommitBatchV1 -} - -func (f *CommitBatchDAV1) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} - -type CommitBatchDAV2 struct { - Version uint8 - BatchIndex uint64 - ParentTotalL1MessagePopped uint64 - SkippedL1MessageBitmap []byte - Chunks []*codecv2.DAChunkRawTx - L1Txs []*types.L1MessageTx - - L1BlockNumber uint64 -} - -func NewCommitBatchDAV2(version uint8, batchIndex uint64, parentTotalL1MessagePopped uint64, skippedL1MessageBitmap []byte, chunks []*codecv2.DAChunkRawTx, l1Txs []*types.L1MessageTx, l1BlockNumber uint64) DAEntry { - return &CommitBatchDAV2{ - Version: version, - BatchIndex: batchIndex, - ParentTotalL1MessagePopped: parentTotalL1MessagePopped, - SkippedL1MessageBitmap: skippedL1MessageBitmap, - Chunks: chunks, - L1Txs: l1Txs, - L1BlockNumber: l1BlockNumber, - } -} - -func (f *CommitBatchDAV2) DAType() DAType { - return CommitBatchV2 -} - -func (f *CommitBatchDAV2) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} - -type RevertBatchDA struct { - BatchIndex uint64 - - L1BlockNumber uint64 -} - -func NewRevertBatchDA(batchIndex uint64) DAEntry { - return &RevertBatchDA{ - BatchIndex: batchIndex, - } -} - -func (f *RevertBatchDA) DAType() DAType { - return RevertBatch -} - -func (f *RevertBatchDA) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} - -type FinalizeBatchDA struct { - BatchIndex uint64 - - L1BlockNumber uint64 -} - -func NewFinalizeBatchDA(batchIndex uint64) DAEntry { - return &FinalizeBatchDA{ - BatchIndex: batchIndex, - } -} - -func (f *FinalizeBatchDA) DAType() DAType { - return FinalizeBatch -} - -func (f *FinalizeBatchDA) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} - -type FinalizeBatchDAV3 struct { - BatchIndex uint64 - - L1BlockNumber uint64 -} - -func NewFinalizeBatchDAV3(batchIndex uint64) DAEntry { - return &FinalizeBatchDAV3{ - BatchIndex: batchIndex, - } -} - -func (f *FinalizeBatchDAV3) DAType() DAType { - return FinalizeBatchV3 -} - -func (f *FinalizeBatchDAV3) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go new file mode 100644 index 000000000000..a8fdd3ea5100 --- /dev/null +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -0,0 +1,220 @@ +package da + +import ( + "context" + "errors" + "fmt" + + "github.com/scroll-tech/go-ethereum/accounts/abi" + "github.com/scroll-tech/go-ethereum/common" + "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/rollup/da_syncer/blob_client" + "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" +) + +const ( + callDataBlobSourceFetchBlockRange uint64 = 500 + commitBatchEventName = "CommitBatch" + revertBatchEventName = "RevertBatch" + finalizeBatchEventName = "FinalizeBatch" + commitBatchMethodName = "commitBatch" + commitBatchWithBlobProofMethodName = "commitBatchWithBlobProof" + + // the length og method ID at the beginning of transaction data + methodIDLength = 4 +) + +var ( + ErrSourceExhausted = errors.New("data source has been exhausted") +) + +type CalldataBlobSource struct { + ctx context.Context + l1Client *rollup_sync_service.L1Client + blobClient blob_client.BlobClient + l1height uint64 + scrollChainABI *abi.ABI + l1CommitBatchEventSignature common.Hash + l1RevertBatchEventSignature common.Hash + l1FinalizeBatchEventSignature common.Hash + db ethdb.Database +} + +func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *rollup_sync_service.L1Client, blobClient blob_client.BlobClient, db ethdb.Database) (*CalldataBlobSource, error) { + scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() + if err != nil { + return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) + } + return &CalldataBlobSource{ + ctx: ctx, + l1Client: l1Client, + blobClient: blobClient, + l1height: l1height, + scrollChainABI: scrollChainABI, + l1CommitBatchEventSignature: scrollChainABI.Events[commitBatchEventName].ID, + l1RevertBatchEventSignature: scrollChainABI.Events[revertBatchEventName].ID, + l1FinalizeBatchEventSignature: scrollChainABI.Events[finalizeBatchEventName].ID, + db: db, + }, nil +} + +func (ds *CalldataBlobSource) NextData() (Entries, error) { + to := ds.l1height + callDataBlobSourceFetchBlockRange + l1Finalized, err := ds.l1Client.GetLatestFinalizedBlockNumber() + if err != nil { + return nil, fmt.Errorf("cannot get l1height, error: %v", err) + } + if to > l1Finalized { + to = l1Finalized + } + if ds.l1height > to { + return nil, ErrSourceExhausted + } + logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) + if err != nil { + return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) + } + da, err := ds.processLogsToDA(logs) + if err == nil { + ds.l1height = to + 1 + } + return da, err +} + +func (ds *CalldataBlobSource) L1Height() uint64 { + return ds.l1height +} + +func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (Entries, error) { + var entries Entries + var entry Entry + var err error + + for _, vLog := range logs { + switch vLog.Topics[0] { + case ds.l1CommitBatchEventSignature: + event := &rollup_sync_service.L1CommitBatchEvent{} + if err = rollup_sync_service.UnpackLog(ds.scrollChainABI, event, commitBatchEventName, vLog); err != nil { + return nil, fmt.Errorf("failed to unpack commit rollup event log, err: %w", err) + } + + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new CommitBatch event", "batch index", batchIndex) + + if entry, err = ds.getCommitBatchDA(batchIndex, &vLog); err != nil { + return nil, fmt.Errorf("failed to get commit batch da: %v, err: %w", batchIndex, err) + } + + case ds.l1RevertBatchEventSignature: + event := &rollup_sync_service.L1RevertBatchEvent{} + if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, revertBatchEventName, vLog); err != nil { + return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) + } + + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new RevertBatchType event", "batch index", batchIndex) + entry = NewRevertBatch(batchIndex) + + case ds.l1FinalizeBatchEventSignature: + event := &rollup_sync_service.L1FinalizeBatchEvent{} + if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, finalizeBatchEventName, vLog); err != nil { + return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) + } + + batchIndex := event.BatchIndex.Uint64() + log.Trace("found new FinalizeBatchType event", "batch index", event.BatchIndex.Uint64()) + entry = NewFinalizeBatch(batchIndex) + + default: + return nil, fmt.Errorf("unknown event, topic: %v, tx hash: %v", vLog.Topics[0].Hex(), vLog.TxHash.Hex()) + } + + entries = append(entries, entry) + } + return entries, nil +} + +type commitBatchArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte +} + +func newCommitBatchArgs(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { + var args commitBatchArgs + err := method.Inputs.Copy(&args, values) + return &args, err +} + +func newCommitBatchArgsFromCommitBatchWithProof(method *abi.Method, values []interface{}) (*commitBatchArgs, error) { + var args commitBatchWithBlobProofArgs + err := method.Inputs.Copy(&args, values) + if err != nil { + return nil, err + } + return &commitBatchArgs{ + Version: args.Version, + ParentBatchHeader: args.ParentBatchHeader, + Chunks: args.Chunks, + SkippedL1MessageBitmap: args.SkippedL1MessageBitmap, + }, nil +} + +type commitBatchWithBlobProofArgs struct { + Version uint8 + ParentBatchHeader []byte + Chunks [][]byte + SkippedL1MessageBitmap []byte + BlobDataProof []byte +} + +func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, vLog *types.Log) (Entry, error) { + if batchIndex == 0 { + return NewCommitBatchDAV0Empty(), nil + } + + txData, err := ds.l1Client.FetchTxData(vLog) + if err != nil { + return nil, err + } + if len(txData) < methodIDLength { + return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) + } + + method, err := ds.scrollChainABI.MethodById(txData[:methodIDLength]) + if err != nil { + return nil, fmt.Errorf("failed to get method by ID, ID: %v, err: %w", txData[:methodIDLength], err) + } + values, err := method.Inputs.Unpack(txData[methodIDLength:]) + if err != nil { + return nil, fmt.Errorf("failed to unpack transaction data using ABI, tx data: %v, err: %w", txData, err) + } + + if method.Name == commitBatchMethodName { + args, err := newCommitBatchArgs(method, values) + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + switch args.Version { + case 0: + return NewCommitBatchDAV0(ds.db, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap, vLog.BlockNumber) + case 1: + return NewCommitBatchDAV1(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + case 2: + return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, 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) + } + } else if method.Name == commitBatchWithBlobProofMethodName { + args, err := newCommitBatchArgsFromCommitBatchWithProof(method, values) + if err != nil { + return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) + } + return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + } + + return nil, fmt.Errorf("unknown method name: %s", method.Name) +} diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go new file mode 100644 index 000000000000..f4305e78ceb0 --- /dev/null +++ b/rollup/da_syncer/da/commitV0.go @@ -0,0 +1,160 @@ +package da + +import ( + "encoding/binary" + "fmt" + "io" + "math/big" + + "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" +) + +type CommitBatchDAV0 struct { + version uint8 + batchIndex uint64 + parentTotalL1MessagePopped uint64 + skippedL1MessageBitmap []byte + chunks []*codecv0.DAChunkRawTx + l1Txs []*types.L1MessageTx + + l1BlockNumber uint64 +} + +func NewCommitBatchDAV0(db ethdb.Database, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + chunks [][]byte, + skippedL1MessageBitmap []byte, + l1BlockNumber uint64, +) (*CommitBatchDAV0, error) { + decodedChunks, err := codecv0.DecodeDAChunksRawTx(chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %d, err: %w", batchIndex, err) + } + + return NewCommitBatchDAV0WithChunks(db, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, l1BlockNumber) +} + +func NewCommitBatchDAV0WithChunks(db ethdb.Database, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + decodedChunks []*codecv0.DAChunkRawTx, + skippedL1MessageBitmap []byte, + l1BlockNumber uint64, +) (*CommitBatchDAV0, error) { + parentTotalL1MessagePopped := getBatchTotalL1MessagePopped(parentBatchHeader) + l1Txs, err := getL1Messages(db, parentTotalL1MessagePopped, skippedL1MessageBitmap, getTotalMessagesPoppedFromChunks(decodedChunks)) + if err != nil { + return nil, fmt.Errorf("failed to get L1 messages for v0 batch %d: %w", batchIndex, err) + } + + return &CommitBatchDAV0{ + version: version, + batchIndex: batchIndex, + parentTotalL1MessagePopped: parentTotalL1MessagePopped, + skippedL1MessageBitmap: skippedL1MessageBitmap, + chunks: decodedChunks, + l1Txs: l1Txs, + l1BlockNumber: l1BlockNumber, + }, nil +} + +func NewCommitBatchDAV0Empty() *CommitBatchDAV0 { + return &CommitBatchDAV0{ + batchIndex: 0, + } +} + +func (c *CommitBatchDAV0) Type() Type { + return CommitBatchV0Type +} + +func (c *CommitBatchDAV0) L1BlockNumber() uint64 { + return c.l1BlockNumber +} + +func (c *CommitBatchDAV0) BatchIndex() uint64 { + return c.batchIndex +} + +func (c *CommitBatchDAV0) Blocks() ([]*types.Block, error) { + var blocks []*types.Block + l1TxPointer := 0 + + curL1TxIndex := c.parentTotalL1MessagePopped + for _, chunk := range c.chunks { + for blockId, daBlock := range chunk.Blocks { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + } + // create txs + // var txs types.Transactions + txs := make(types.Transactions, 0, daBlock.NumTransactions) + // insert l1 msgs + for l1TxPointer < len(c.l1Txs) && c.l1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { + l1Tx := types.NewTx(c.l1Txs[l1TxPointer]) + txs = append(txs, l1Tx) + l1TxPointer++ + } + curL1TxIndex += uint64(daBlock.NumL1Messages) + // insert l2 txs + txs = append(txs, chunk.Transactions[blockId]...) + block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + blocks = append(blocks, block) + } + } + return blocks, nil +} + +func getTotalMessagesPoppedFromChunks(decodedChunks []*codecv0.DAChunkRawTx) int { + totalL1MessagePopped := 0 + for _, chunk := range decodedChunks { + for _, block := range chunk.Blocks { + totalL1MessagePopped += int(block.NumL1Messages) + } + } + return totalL1MessagePopped +} + +func getL1Messages(db ethdb.Database, parentTotalL1MessagePopped uint64, skippedBitmap []byte, totalL1MessagePopped int) ([]*types.L1MessageTx, error) { + var txs []*types.L1MessageTx + + decodedSkippedBitmap, err := encoding.DecodeBitmap(skippedBitmap, totalL1MessagePopped) + if err != nil { + return nil, fmt.Errorf("failed to decode skipped message bitmap: err: %w", err) + } + + // get all necessary l1 messages without skipped + currentIndex := parentTotalL1MessagePopped + for index := 0; index < totalL1MessagePopped; index++ { + if encoding.IsL1MessageSkipped(decodedSkippedBitmap, currentIndex-parentTotalL1MessagePopped) { + currentIndex++ + continue + } + l1Tx := rawdb.ReadL1Message(db, currentIndex) + if l1Tx == nil { + // TODO: returning io.EOF is not the best way to handle this + return nil, io.EOF + } + txs = append(txs, l1Tx) + currentIndex++ + } + + return txs, nil +} + +func getBatchTotalL1MessagePopped(data []byte) uint64 { + // total l1 message popped stored in bytes from 17 to 24, accordingly to codec spec + return binary.BigEndian.Uint64(data[17:25]) +} diff --git a/rollup/da_syncer/da/commitV1.go b/rollup/da_syncer/da/commitV1.go new file mode 100644 index 000000000000..8c45664e7979 --- /dev/null +++ b/rollup/da_syncer/da/commitV1.go @@ -0,0 +1,89 @@ +package da + +import ( + "context" + "crypto/sha256" + "fmt" + + "github.com/scroll-tech/da-codec/encoding/codecv0" + "github.com/scroll-tech/da-codec/encoding/codecv1" + + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" + "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/core/types" + "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, + l1Client *rollup_sync_service.L1Client, + blobClient blob_client.BlobClient, + vLog *types.Log, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + chunks [][]byte, + skippedL1MessageBitmap []byte, +) (*CommitBatchDAV1, error) { + return NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Client, blobClient, vLog, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv1.DecodeTxsFromBlob) +} + +func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, db ethdb.Database, + l1Client *rollup_sync_service.L1Client, + blobClient blob_client.BlobClient, + vLog *types.Log, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + chunks [][]byte, + skippedL1MessageBitmap []byte, + decodeTxsFromBlobFunc func(*kzg4844.Blob, []*codecv0.DAChunkRawTx) error, +) (*CommitBatchDAV1, error) { + decodedChunks, err := codecv1.DecodeDAChunksRawTx(chunks) + if err != nil { + return nil, fmt.Errorf("failed to unpack chunks: %v, err: %w", batchIndex, err) + } + + versionedHash, err := l1Client.FetchTxBlobHash(vLog) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) + } + + blob, err := blobClient.GetBlobByVersionedHash(ctx, versionedHash) + if err != nil { + return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) + } + + // compute blob versioned hash and compare with one from tx + c, err := kzg4844.BlobToCommitment(blob) + if err != nil { + return nil, fmt.Errorf("failed to create blob commitment") + } + blobVersionedHash := common.Hash(kzg4844.CalcBlobHashV1(sha256.New(), &c)) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blobVersionedHash from blob source is not equal to versionedHash from tx, correct versioned hash: %s, fetched blob hash: %s", versionedHash.String(), blobVersionedHash.String()) + } + + // decode txs from blob + err = decodeTxsFromBlobFunc(blob, decodedChunks) + if err != nil { + return nil, fmt.Errorf("failed to decode txs from blob: %w", err) + } + + v0, err := NewCommitBatchDAV0WithChunks(db, version, batchIndex, parentBatchHeader, decodedChunks, skippedL1MessageBitmap, vLog.BlockNumber) + if err != nil { + return nil, err + } + + return &CommitBatchDAV1{v0}, nil +} + +func (c *CommitBatchDAV1) Type() Type { + return CommitBatchV1Type +} diff --git a/rollup/da_syncer/da/commitV2.go b/rollup/da_syncer/da/commitV2.go new file mode 100644 index 000000000000..c1e6d353fc5b --- /dev/null +++ b/rollup/da_syncer/da/commitV2.go @@ -0,0 +1,40 @@ +package da + +import ( + "context" + + "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/rollup_sync_service" + + "github.com/scroll-tech/go-ethereum/core/types" +) + +type CommitBatchDAV2 struct { + *CommitBatchDAV1 +} + +func NewCommitBatchDAV2(ctx context.Context, db ethdb.Database, + l1Client *rollup_sync_service.L1Client, + blobClient blob_client.BlobClient, + vLog *types.Log, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + chunks [][]byte, + skippedL1MessageBitmap []byte, +) (*CommitBatchDAV2, error) { + + v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Client, blobClient, vLog, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv2.DecodeTxsFromBlob) + if err != nil { + return nil, err + } + + return &CommitBatchDAV2{v1}, nil +} + +func (c *CommitBatchDAV2) Type() Type { + return CommitBatchV2Type +} diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go new file mode 100644 index 000000000000..a225b2e76737 --- /dev/null +++ b/rollup/da_syncer/da/da.go @@ -0,0 +1,35 @@ +package da + +import ( + "github.com/scroll-tech/go-ethereum/core/types" +) + +type Type int + +const ( + // CommitBatchV0Type contains data of event of CommitBatchV0Type + CommitBatchV0Type Type = iota + // CommitBatchV1Type contains data of event of CommitBatchV1Type + CommitBatchV1Type + // CommitBatchV2Type contains data of event of CommitBatchV2Type + CommitBatchV2Type + // RevertBatchType contains data of event of RevertBatchType + RevertBatchType + // FinalizeBatchType contains data of event of FinalizeBatchType + FinalizeBatchType + // FinalizeBatchV3Type contains data of event of FinalizeBatchType v3 + FinalizeBatchV3Type +) + +type Entry interface { + Type() Type + BatchIndex() uint64 + L1BlockNumber() uint64 +} + +type EntryWithBlocks interface { + Entry + Blocks() ([]*types.Block, error) +} + +type Entries []Entry diff --git a/rollup/da_syncer/da/finalize.go b/rollup/da_syncer/da/finalize.go new file mode 100644 index 000000000000..9864233fe933 --- /dev/null +++ b/rollup/da_syncer/da/finalize.go @@ -0,0 +1,45 @@ +package da + +type FinalizeBatch struct { + batchIndex uint64 + + l1BlockNumber uint64 +} + +func NewFinalizeBatch(batchIndex uint64) *FinalizeBatch { + return &FinalizeBatch{ + batchIndex: batchIndex, + } +} + +func (f *FinalizeBatch) Type() Type { + return FinalizeBatchType +} + +func (f *FinalizeBatch) L1BlockNumber() uint64 { + return f.l1BlockNumber +} + +func (f *FinalizeBatch) BatchIndex() uint64 { + return f.batchIndex +} + +type FinalizeBatchDAV3 struct { + BatchIndex uint64 + + L1BlockNumber uint64 +} + +func NewFinalizeBatchDAV3(batchIndex uint64) *FinalizeBatchDAV3 { + return &FinalizeBatchDAV3{ + BatchIndex: batchIndex, + } +} + +func (f *FinalizeBatchDAV3) DAType() Type { + return FinalizeBatchV3Type +} + +func (f *FinalizeBatchDAV3) GetL1BlockNumber() uint64 { + return f.L1BlockNumber +} diff --git a/rollup/da_syncer/da/revert.go b/rollup/da_syncer/da/revert.go new file mode 100644 index 000000000000..f02c264039b7 --- /dev/null +++ b/rollup/da_syncer/da/revert.go @@ -0,0 +1,24 @@ +package da + +type RevertBatch struct { + batchIndex uint64 + + l1BlockNumber uint64 +} + +func NewRevertBatch(batchIndex uint64) *RevertBatch { + return &RevertBatch{ + batchIndex: batchIndex, + } +} + +func (r *RevertBatch) Type() Type { + return RevertBatchType +} + +func (r *RevertBatch) L1BlockNumber() uint64 { + return r.l1BlockNumber +} +func (r *RevertBatch) BatchIndex() uint64 { + return r.batchIndex +} diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index b1d3d79b8b22..bdbbbb5428bd 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -2,13 +2,15 @@ package da_syncer import ( "context" + + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type DAQueue struct { l1height uint64 dataSourceFactory *DataSourceFactory dataSource DataSource - da DA + da da.Entries } func NewDAQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DAQueue { @@ -16,11 +18,11 @@ func NewDAQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DAQueue l1height: l1height, dataSourceFactory: dataSourceFactory, dataSource: nil, - da: []DAEntry{}, + da: make(da.Entries, 0), } } -func (dq *DAQueue) NextDA(ctx context.Context) (DAEntry, error) { +func (dq *DAQueue) NextDA(ctx context.Context) (da.Entry, error) { for len(dq.da) == 0 { err := dq.getNextData(ctx) if err != nil { @@ -42,7 +44,7 @@ func (dq *DAQueue) getNextData(ctx context.Context) error { } dq.da, err = dq.dataSource.NextData() // previous dataSource has been exhausted, create new - if err == errSourceExhausted { + if err == da.ErrSourceExhausted { dq.l1height = dq.dataSource.L1Height() dq.dataSource = nil return dq.getNextData(ctx) diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go index 017796352b8c..f417d09af00e 100644 --- a/rollup/da_syncer/data_source.go +++ b/rollup/da_syncer/data_source.go @@ -8,15 +8,12 @@ import ( "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" "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" ) -var ( - errSourceExhausted = errors.New("data source has been exhausted") -) - type DataSource interface { - NextData() (DA, error) + NextData() (da.Entries, error) L1Height() uint64 } @@ -40,7 +37,7 @@ func NewDataSourceFactory(blockchain *core.BlockChain, genesisConfig *params.Cha func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { if ds.config.FetcherMode == L1RPC { - return NewCalldataBlobSource(ctx, l1height, ds.l1Client, ds.blobClient, ds.db) + return da.NewCalldataBlobSource(ctx, l1height, ds.l1Client, ds.blobClient, ds.db) } 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 5c5e56300886..231871bdd069 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -2,10 +2,14 @@ package da_syncer import ( "context" + "errors" "fmt" + "io" "strings" + "sync" "time" + "github.com/scroll-tech/go-ethereum/common/backoff" "github.com/scroll-tech/go-ethereum/core" "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/ethdb" @@ -23,12 +27,12 @@ type Config struct { BlobSource blob_client.BlobSource // blob source } -// defaultSyncInterval is the frequency at which we query for new rollup event. -const defaultSyncInterval = 1 * time.Millisecond - type SyncingPipeline struct { ctx context.Context cancel context.CancelFunc + wg sync.WaitGroup + expBackoff *backoff.Exponential + db ethdb.Database blockchain *core.BlockChain blockQueue *BlockQueue @@ -36,19 +40,16 @@ type SyncingPipeline struct { } func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesisConfig *params.ChainConfig, db ethdb.Database, ethClient sync_service.EthClient, l1DeploymentBlock uint64, config Config) (*SyncingPipeline, error) { - ctx, cancel := context.WithCancel(ctx) - scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() if err != nil { - cancel() return nil, fmt.Errorf("failed to get scroll chain abi: %w", err) } l1Client, err := rollup_sync_service.NewL1Client(ctx, ethClient, genesisConfig.Scroll.L1Config.L1ChainId, genesisConfig.Scroll.L1Config.ScrollChainAddress, scrollChainABI) if err != nil { - cancel() return nil, err } + var blobClient blob_client.BlobClient switch config.BlobSource { case blob_client.BlobScan: @@ -56,7 +57,6 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi case blob_client.BlockNative: blobClient = blob_client.NewBlockNativeClient(genesisConfig.Scroll.DAConfig.BlockNativeAPIEndpoint) default: - cancel() return nil, fmt.Errorf("unknown blob scan client: %d", config.BlobSource) } @@ -66,14 +66,18 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi if from != nil { syncedL1Height = *from } - DAQueue := NewDAQueue(syncedL1Height, dataSourceFactory) - batchQueue := NewBatchQueue(DAQueue, db) + + daQueue := NewDAQueue(syncedL1Height, dataSourceFactory) + batchQueue := NewBatchQueue(daQueue, db) blockQueue := NewBlockQueue(batchQueue) daSyncer := NewDASyncer(blockchain) + ctx, cancel := context.WithCancel(ctx) return &SyncingPipeline{ ctx: ctx, cancel: cancel, + expBackoff: backoff.NewExponential(100*time.Millisecond, 10*time.Second, 100*time.Millisecond), + wg: sync.WaitGroup{}, db: db, blockchain: blockchain, blockQueue: blockQueue, @@ -81,47 +85,92 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi }, nil } -func (sp *SyncingPipeline) Step() error { - block, err := sp.blockQueue.NextBlock(sp.ctx) +func (s *SyncingPipeline) Step() error { + block, err := s.blockQueue.NextBlock(s.ctx) if err != nil { return err } - err = sp.daSyncer.SyncOneBlock(block) + err = s.daSyncer.SyncOneBlock(block) return err } -func (sp *SyncingPipeline) Start() { +func (s *SyncingPipeline) Start() { log.Info("Starting SyncingPipeline") + s.wg.Add(1) go func() { - syncTicker := time.NewTicker(defaultSyncInterval) - defer syncTicker.Stop() - - for { - err := sp.Step() - if err != nil { - if strings.HasPrefix(err.Error(), "not consecutive block") { - log.Warn("syncing pipeline step failed, probably because of restart", "err", err) - } else { - log.Crit("syncing pipeline step failed", "err", err) - } + s.mainLoop() + s.wg.Done() + }() +} + +func (s *SyncingPipeline) mainLoop() { + stepCh := make(chan struct{}, 1) + var delayedStepCh <-chan time.Time + + // 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) { + if delay { + if delayedStepCh == nil { + delayDur := s.expBackoff.NextDuration() + delayedStepCh = time.After(delayDur) + log.Debug("requesting delayed step", "delay", delayDur, "attempt", s.expBackoff.Attempt()) + } else { + log.Debug("ignoring step request because of ongoing delayed step", "attempt", s.expBackoff.Attempt()) } + } else { select { - case <-sp.ctx.Done(): - return - case <-syncTicker.C: - select { - case <-sp.ctx.Done(): - return - default: - } + case stepCh <- struct{}{}: + default: + } + } + } + + // start pipeline + reqStep(false) + + for { + select { + case <-s.ctx.Done(): + return + default: + } + + select { + case <-s.ctx.Done(): + return + case <-delayedStepCh: + delayedStepCh = nil + reqStep(false) + case <-stepCh: + err := s.Step() + if err == nil { + reqStep(false) + s.expBackoff.Reset() + continue + } + + if errors.Is(err, io.EOF) { + reqStep(true) continue } + if errors.Is(err, context.Canceled) { + return + } + + if strings.HasPrefix(err.Error(), "not consecutive block") { + log.Warn("syncing pipeline step failed, probably because of restart", "err", err) + } else { + log.Crit("syncing pipeline step failed", "err", err) + } } - }() + } } -func (sp *SyncingPipeline) Stop() { - log.Info("Stopping DaSyncer") - sp.cancel() +func (s *SyncingPipeline) Stop() { + log.Info("Stopping DaSyncer...") + s.cancel() + s.wg.Wait() + log.Info("Stopped DaSyncer... Done") } From e587c984ec674a48740b74cd96eab11297cbc5cd Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 14:48:32 +0800 Subject: [PATCH 31/59] feat: execute blocks only once --- core/blockchain.go | 45 +++++++++++++++++++++++++++++++++++ rollup/da_syncer/da_syncer.go | 29 +++++++--------------- 2 files changed, 54 insertions(+), 20 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 54a4573e9132..3a8ca512bebb 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1824,6 +1824,51 @@ func (bc *BlockChain) PreprocessBlock(block *types.Block) (common.Hash, types.Bl return receiptSha, bloom, stateRoot, usedGas, nil } +func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types.Header, txs types.Transactions) (WriteStatus, error) { + if !bc.chainmu.TryLock() { + return NonStatTy, errInsertionInterrupted + } + defer bc.chainmu.Unlock() + + statedb, err := state.New(parentBlock.Root(), bc.stateCache, bc.snaps) + if err != nil { + return NonStatTy, err + } + + tempBlock := types.NewBlockWithHeader(header).WithBody(txs, nil) + receipts, logs, gasUsed, err := bc.processor.Process(tempBlock, statedb, bc.vmConfig) + if err != nil { + return NonStatTy, err + } + + header.GasUsed = gasUsed + header.ParentHash = parentBlock.Hash() + header.Root = statedb.GetRootHash() + // Since we're using Clique consensus, we don't have uncles + header.UncleHash = types.EmptyUncleHash + // TODO: extraData and difficulty should be set + + fullBlock := types.NewBlock(header, txs, nil, receipts, trie.NewStackTrie(nil)) + + blockHash := fullBlock.Hash() + // manually replace the block hash in the receipts + for i, receipt := range receipts { + // add block location fields + receipt.BlockHash = blockHash + receipt.BlockNumber = tempBlock.Number() + receipt.TransactionIndex = uint(i) + + for _, l := range receipt.Logs { + l.BlockHash = blockHash + } + } + for _, l := range logs { + l.BlockHash = blockHash + } + + return bc.writeBlockWithState(fullBlock, receipts, logs, statedb, false) +} + // insertSideChain is called when an import batch hits upon a pruned ancestor // error, which happens when a sidechain with a sufficiently old fork-block is // found. diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index e8dbf7859e84..2c34847fc499 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -8,7 +8,6 @@ import ( "github.com/scroll-tech/go-ethereum/core" "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" - "github.com/scroll-tech/go-ethereum/trie" ) type DASyncer struct { @@ -22,32 +21,22 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { } func (s *DASyncer) SyncOneBlock(block *types.Block) error { - prevHash := s.blockchain.CurrentBlock().Hash() - if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { + parentBlock := s.blockchain.CurrentBlock() + if big.NewInt(0).Add(parentBlock.Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) } - header := block.Header() - txs := block.Transactions() - // fill header with all necessary fields - var err error - header.ReceiptHash, header.Bloom, header.Root, header.GasUsed, err = s.blockchain.PreprocessBlock(block) - if err != nil { - return fmt.Errorf("block preprocessing failed, block number: %d, error: %v", block.Number(), err) - } - header.UncleHash = common.HexToHash("0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347") + header := block.Header() header.Difficulty = common.Big1 - header.BaseFee = nil - header.TxHash = types.DeriveSha(txs, trie.NewStackTrie(nil)) - header.ParentHash = prevHash + header.BaseFee = nil // TODO: after Curie we need to fill this correctly + header.ParentHash = parentBlock.Hash() - fullBlock := types.NewBlockWithHeader(header).WithBody(txs, make([]*types.Header, 0)) - - if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { - return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) + if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, header, block.Transactions()); err != nil { + return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.Number(), err) } + if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) } return nil } From 9dc17cb4f7df83adec2aaa7dc1cb99868dd5a522 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 15:18:46 +0800 Subject: [PATCH 32/59] refactor: introduce partial header and partial block for data from DA before execution --- core/blockchain.go | 4 ++-- rollup/da_syncer/block_queue.go | 7 +++---- rollup/da_syncer/da/commitV0.go | 24 ++++++++++++---------- rollup/da_syncer/da/da.go | 36 ++++++++++++++++++++++++++++++++- rollup/da_syncer/da_syncer.go | 20 +++++++----------- 5 files changed, 60 insertions(+), 31 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 3a8ca512bebb..e55f6ce24085 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1835,6 +1835,8 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types return NonStatTy, err } + header.ParentHash = parentBlock.Hash() + tempBlock := types.NewBlockWithHeader(header).WithBody(txs, nil) receipts, logs, gasUsed, err := bc.processor.Process(tempBlock, statedb, bc.vmConfig) if err != nil { @@ -1842,11 +1844,9 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types } header.GasUsed = gasUsed - header.ParentHash = parentBlock.Hash() header.Root = statedb.GetRootHash() // Since we're using Clique consensus, we don't have uncles header.UncleHash = types.EmptyUncleHash - // TODO: extraData and difficulty should be set fullBlock := types.NewBlock(header, txs, nil, receipts, trie.NewStackTrie(nil)) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index abd728912b21..68954da69e2a 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -4,23 +4,22 @@ import ( "context" "fmt" - "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BlockQueue struct { batchQueue *BatchQueue - blocks []*types.Block + blocks []*da.PartialBlock } func NewBlockQueue(batchQueue *BatchQueue) *BlockQueue { return &BlockQueue{ batchQueue: batchQueue, - blocks: []*types.Block{}, + blocks: make([]*da.PartialBlock, 0), } } -func (bq *BlockQueue) NextBlock(ctx context.Context) (*types.Block, error) { +func (bq *BlockQueue) NextBlock(ctx context.Context) (*da.PartialBlock, error) { for len(bq.blocks) == 0 { err := bq.getBlocksFromBatch(ctx) if err != nil { diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index f4305e78ceb0..432c6d8462de 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -4,7 +4,6 @@ import ( "encoding/binary" "fmt" "io" - "math/big" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" @@ -84,20 +83,13 @@ func (c *CommitBatchDAV0) BatchIndex() uint64 { return c.batchIndex } -func (c *CommitBatchDAV0) Blocks() ([]*types.Block, error) { - var blocks []*types.Block +func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { + var blocks []*PartialBlock l1TxPointer := 0 curL1TxIndex := c.parentTotalL1MessagePopped for _, chunk := range c.chunks { for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } // create txs // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) @@ -110,7 +102,17 @@ func (c *CommitBatchDAV0) Blocks() ([]*types.Block, error) { curL1TxIndex += uint64(daBlock.NumL1Messages) // insert l2 txs txs = append(txs, chunk.Transactions[blockId]...) - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + + block := NewPartialBlock( + &PartialHeader{ + Number: daBlock.BlockNumber, + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + //TODO: Difficulty: new(big.Int).SetUint64(10), + //TODO: ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, + }, + txs) blocks = append(blocks, block) } } diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index a225b2e76737..829f8d4712dd 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -1,6 +1,8 @@ package da import ( + "math/big" + "github.com/scroll-tech/go-ethereum/core/types" ) @@ -29,7 +31,39 @@ type Entry interface { type EntryWithBlocks interface { Entry - Blocks() ([]*types.Block, error) + Blocks() ([]*PartialBlock, error) } type Entries []Entry + +type PartialHeader struct { + Number uint64 + Time uint64 + BaseFee *big.Int + GasLimit uint64 + Difficulty *big.Int + ExtraData []byte +} + +func (h *PartialHeader) ToHeader() *types.Header { + return &types.Header{ + Number: big.NewInt(0).SetUint64(h.Number), + Time: h.Time, + BaseFee: h.BaseFee, + GasLimit: h.GasLimit, + Difficulty: h.Difficulty, + Extra: h.ExtraData, + } +} + +type PartialBlock struct { + PartialHeader *PartialHeader + Transactions types.Transactions +} + +func NewPartialBlock(partialHeader *PartialHeader, txs types.Transactions) *PartialBlock { + return &PartialBlock{ + PartialHeader: partialHeader, + Transactions: txs, + } +} diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 2c34847fc499..b1adaa7ea994 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -2,12 +2,10 @@ package da_syncer import ( "fmt" - "math/big" - "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core" - "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type DASyncer struct { @@ -20,23 +18,19 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { } } -func (s *DASyncer) SyncOneBlock(block *types.Block) error { +func (s *DASyncer) SyncOneBlock(block *da.PartialBlock) error { parentBlock := s.blockchain.CurrentBlock() - if big.NewInt(0).Add(parentBlock.Number(), common.Big1).Cmp(block.Number()) != 0 { - return fmt.Errorf("not consecutive block, number: %d", block.Number()) + if parentBlock.NumberU64()+1 != block.PartialHeader.Number { + return fmt.Errorf("not consecutive block, number: %d, chain height: %d", block.PartialHeader.Number, parentBlock.NumberU64()) } - header := block.Header() - header.Difficulty = common.Big1 - header.BaseFee = nil // TODO: after Curie we need to fill this correctly - header.ParentHash = parentBlock.Hash() - - if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, header, block.Transactions()); err != nil { - return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.Number(), err) + if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, block.PartialHeader.ToHeader(), block.Transactions); err != nil { + return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.PartialHeader.Number, err) } if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) } + return nil } From dbdbc88e408dcfbe916a1ac732d71a9aefb72db0 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 15:26:47 +0800 Subject: [PATCH 33/59] minor cleanup --- core/blockchain.go | 21 --------------------- rollup/da_syncer/da/da.go | 4 ++-- 2 files changed, 2 insertions(+), 23 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index e55f6ce24085..e6ec46756dcb 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1803,27 +1803,6 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er return it.index, err } -// PreprocessBlock processes block on top of the chain to calculate receipts, bloom and state root -func (bc *BlockChain) PreprocessBlock(block *types.Block) (common.Hash, types.Bloom, common.Hash, uint64, error) { - // Retrieve the parent block and it's state to execute on top - parent := bc.CurrentBlock().Header() - if parent == nil { - parent = bc.GetHeader(block.ParentHash(), block.NumberU64()-1) - } - statedb, err := state.New(parent.Root, bc.stateCache, bc.snaps) - if err != nil { - return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err - } - receipts, _, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig) - if err != nil { - return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err - } - receiptSha := types.DeriveSha(receipts, trie.NewStackTrie(nil)) - bloom := types.CreateBloom(receipts) - stateRoot := statedb.GetRootHash() - return receiptSha, bloom, stateRoot, usedGas, nil -} - func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types.Header, txs types.Transactions) (WriteStatus, error) { if !bc.chainmu.TryLock() { return NonStatTy, errInsertionInterrupted diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index 829f8d4712dd..fbaa60962758 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -41,7 +41,7 @@ type PartialHeader struct { Time uint64 BaseFee *big.Int GasLimit uint64 - Difficulty *big.Int + Difficulty uint64 ExtraData []byte } @@ -51,7 +51,7 @@ func (h *PartialHeader) ToHeader() *types.Header { Time: h.Time, BaseFee: h.BaseFee, GasLimit: h.GasLimit, - Difficulty: h.Difficulty, + Difficulty: new(big.Int).SetUint64(h.Difficulty), Extra: h.ExtraData, } } From 26dbf42d04a2f6b7a3cc88675f3446f4fecacbaf Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:23:17 +0800 Subject: [PATCH 34/59] feat: fix issue with not specifying difficulty --- rollup/da_syncer/da/commitV0.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 432c6d8462de..481bb60ed124 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -105,12 +105,12 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { block := NewPartialBlock( &PartialHeader{ - Number: daBlock.BlockNumber, - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - //TODO: Difficulty: new(big.Int).SetUint64(10), - //TODO: ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, + Number: daBlock.BlockNumber, + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + Difficulty: 10, // TODO: replace with real difficulty + ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, // TODO: replace with real extra data }, txs) blocks = append(blocks, block) From 0f8e35c54ffc6bc31262bd8c3463d84a046615da Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Thu, 25 Jul 2024 10:57:28 +0800 Subject: [PATCH 35/59] refactor: compose DA types for more cohesion, maintainability and code reuse --- rollup/da_syncer/block_queue.go | 1 + rollup/da_syncer/da/commitV0.go | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 68954da69e2a..9e8aa12a051d 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BlockQueue struct { diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 481bb60ed124..0c6f145dc39f 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -4,6 +4,7 @@ import ( "encoding/binary" "fmt" "io" + "math/big" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" @@ -90,6 +91,13 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { curL1TxIndex := c.parentTotalL1MessagePopped for _, chunk := range c.chunks { for blockId, daBlock := range chunk.Blocks { + // create header + header := types.Header{ + Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + } // create txs // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) From 912af16bd7b121ce37070d40a34514c63f8849e3 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:26:38 +0800 Subject: [PATCH 36/59] feat: implement generic min heap --- common/heap.go | 91 +++++++++++++++++++++++++++++++++++++++++++++ common/heap_test.go | 40 ++++++++++++++++++++ 2 files changed, 131 insertions(+) create mode 100644 common/heap.go create mode 100644 common/heap_test.go diff --git a/common/heap.go b/common/heap.go new file mode 100644 index 000000000000..0906c2edddf1 --- /dev/null +++ b/common/heap.go @@ -0,0 +1,91 @@ +package common + +import ( + "container/heap" +) + +type Heap[T Comparable[T]] struct { + heap innerHeap[T] +} + +func NewHeap[T Comparable[T]]() *Heap[T] { + return &Heap[T]{ + heap: make(innerHeap[T], 0), + } +} + +func (h *Heap[T]) Len() int { + return len(h.heap) +} + +func (h *Heap[T]) Push(element T) { + heapElement := NewHeapElement(element) + heap.Push(&h.heap, heapElement) +} + +func (h *Heap[T]) Pop() T { + return heap.Pop(&h.heap).(*HeapElement[T]).Value() +} + +func (h *Heap[T]) Peek() T { + if h.Len() == 0 { + var empty T + return empty + } + + return h.heap[0].Value() +} + +type innerHeap[T Comparable[T]] []*HeapElement[T] + +func (h innerHeap[T]) Len() int { + return len(h) +} + +func (h innerHeap[T]) Less(i, j int) bool { + return h[i].Value().CompareTo(h[j].Value()) < 0 +} + +func (h innerHeap[T]) Swap(i, j int) { + h[i], h[j] = h[j], h[i] + h[i].index, h[j].index = i, j +} + +func (h *innerHeap[T]) Push(x interface{}) { + data := x.(*HeapElement[T]) + *h = append(*h, data) + data.index = len(*h) - 1 +} + +func (h *innerHeap[T]) Pop() interface{} { + n := len(*h) + element := (*h)[n-1] + (*h)[n-1] = nil // avoid memory leak + *h = (*h)[:n-1] + element.index = -1 + + return element +} + +type Comparable[T any] interface { + CompareTo(other T) int +} + +type HeapElement[T Comparable[T]] struct { + value T + index int +} + +func NewHeapElement[T Comparable[T]](value T) *HeapElement[T] { + return &HeapElement[T]{ + value: value, + } +} + +func (h *HeapElement[T]) Value() T { + return h.value +} + +func (h *HeapElement[T]) Index() int { + return h.index +} diff --git a/common/heap_test.go b/common/heap_test.go new file mode 100644 index 000000000000..ac927c375de4 --- /dev/null +++ b/common/heap_test.go @@ -0,0 +1,40 @@ +package common + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +type Int int + +func (i Int) CompareTo(other Int) int { + if i < other { + return -1 + } else if i > other { + return 1 + } else { + return 0 + } +} + +func TestHeap(t *testing.T) { + h := NewHeap[Int]() + + require.Equal(t, 0, h.Len(), "Heap should be empty initially") + + h.Push(Int(3)) + h.Push(Int(1)) + h.Push(Int(2)) + + require.Equal(t, 3, h.Len(), "Heap should have three elements after pushing") + + require.EqualValues(t, 1, h.Pop(), "Pop should return the smallest element") + require.Equal(t, 2, h.Len(), "Heap should have two elements after popping") + + require.EqualValues(t, 2, h.Pop(), "Pop should return the next smallest element") + require.Equal(t, 1, h.Len(), "Heap should have one element after popping") + + require.EqualValues(t, 3, h.Pop(), "Pop should return the last element") + require.Equal(t, 0, h.Len(), "Heap should be empty after popping all elements") +} From bdb575d404406d3334f6f256bab8411da9547a9e Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:27:31 +0800 Subject: [PATCH 37/59] feat: use generic min heap instead of map in BatchQueue --- rollup/da_syncer/batch_queue.go | 74 +++++++++++++-------------------- rollup/da_syncer/da/commitV0.go | 9 ++++ rollup/da_syncer/da/da.go | 1 + rollup/da_syncer/da/finalize.go | 9 ++++ rollup/da_syncer/da/revert.go | 9 ++++ 5 files changed, 57 insertions(+), 45 deletions(-) diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index aa38d8a64871..b2ad881c0a26 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -3,35 +3,35 @@ package da_syncer import ( "context" "fmt" - "math" + "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core/rawdb" "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BatchQueue struct { - // batches is map from batchIndex to batch blocks - batches map[uint64]da.Entry DAQueue *DAQueue db ethdb.Database lastFinalizedBatchIndex uint64 + batches *common.Heap[da.Entry] } func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { return &BatchQueue{ - batches: make(map[uint64]da.Entry), DAQueue: DAQueue, db: db, lastFinalizedBatchIndex: 0, + batches: common.NewHeap[da.Entry](), } } // NextBatch finds next finalized batch and returns data, that was committed in that batch func (bq *BatchQueue) NextBatch(ctx context.Context) (da.Entry, error) { - if batch, ok := bq.getFinalizedBatch(); ok { + if batch := bq.getFinalizedBatch(); batch != nil { return batch, nil } + for { daEntry, err := bq.DAQueue.NextDA(ctx) if err != nil { @@ -39,18 +39,16 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (da.Entry, error) { } switch daEntry.Type() { case da.CommitBatchV0Type, da.CommitBatchV1Type, da.CommitBatchV2Type: - bq.batches[daEntry.BatchIndex()] = daEntry + bq.batches.Push(daEntry) case da.RevertBatchType: bq.deleteBatch(daEntry.BatchIndex()) case da.FinalizeBatchType: if daEntry.BatchIndex() > bq.lastFinalizedBatchIndex { bq.lastFinalizedBatchIndex = daEntry.BatchIndex() } - ret, ok := bq.getFinalizedBatch() - if ok { - return ret, nil - } else { - continue + + if batch := bq.getFinalizedBatch(); batch != nil { + return batch, nil } default: return nil, fmt.Errorf("unexpected type of daEntry: %T", daEntry) @@ -59,50 +57,36 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (da.Entry, error) { } // getFinalizedBatch returns next finalized batch if there is available -func (bq *BatchQueue) getFinalizedBatch() (da.Entry, bool) { - if len(bq.batches) == 0 { - return nil, false - } - var minBatchIndex uint64 = math.MaxUint64 - for index := range bq.batches { - if index < minBatchIndex { - minBatchIndex = index - } +func (bq *BatchQueue) getFinalizedBatch() da.Entry { + if bq.batches.Len() == 0 { + return nil } - if minBatchIndex <= bq.lastFinalizedBatchIndex { - batch, ok := bq.batches[minBatchIndex] - // this won't happen because wew just found minBatchIndex among map keys, but need to leave this check to pass CI - if !ok { - return nil, false - } - bq.deleteBatch(minBatchIndex) - return batch, ok - } else { - return nil, false - } + batch := bq.batches.Peek() + bq.deleteBatch(batch.BatchIndex()) + + return batch } // deleteBatch deletes data committed in the batch from map, because this batch is reverted or finalized // updates DASyncedL1BlockNumber func (bq *BatchQueue) deleteBatch(batchIndex uint64) { - batch, ok := bq.batches[batchIndex] - if !ok { - return + var batch da.Entry + for batch = bq.batches.Peek(); batch.BatchIndex() <= batchIndex; { + bq.batches.Pop() + + if bq.batches.Len() == 0 { + break + } + batch = bq.batches.Peek() } - curBatchL1Height := batch.L1BlockNumber() - delete(bq.batches, batchIndex) - if len(bq.batches) == 0 { + + if bq.batches.Len() == 0 { + curBatchL1Height := batch.L1BlockNumber() rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) return } - // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart - var minBatchL1Height uint64 = math.MaxUint64 - for _, val := range bq.batches { - if val.L1BlockNumber() < minBatchL1Height { - minBatchL1Height = val.L1BlockNumber() - } - } - rawdb.WriteDASyncedL1BlockNumber(bq.db, minBatchL1Height-1) + // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart + rawdb.WriteDASyncedL1BlockNumber(bq.db, bq.batches.Peek().L1BlockNumber()-1) } diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 0c6f145dc39f..945ae157c538 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -84,6 +84,15 @@ func (c *CommitBatchDAV0) BatchIndex() uint64 { return c.batchIndex } +func (c *CommitBatchDAV0) CompareTo(other Entry) int { + if c.BatchIndex() < other.BatchIndex() { + return -1 + } else if c.BatchIndex() > other.BatchIndex() { + return 1 + } + return 0 +} + func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { var blocks []*PartialBlock l1TxPointer := 0 diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index fbaa60962758..258fdf2d7805 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -27,6 +27,7 @@ type Entry interface { Type() Type BatchIndex() uint64 L1BlockNumber() uint64 + CompareTo(Entry) int } type EntryWithBlocks interface { diff --git a/rollup/da_syncer/da/finalize.go b/rollup/da_syncer/da/finalize.go index 9864233fe933..332caa671f59 100644 --- a/rollup/da_syncer/da/finalize.go +++ b/rollup/da_syncer/da/finalize.go @@ -30,6 +30,15 @@ type FinalizeBatchDAV3 struct { L1BlockNumber uint64 } +func (f *FinalizeBatch) CompareTo(other Entry) int { + if f.BatchIndex() < other.BatchIndex() { + return -1 + } else if f.BatchIndex() > other.BatchIndex() { + return 1 + } + return 0 +} + func NewFinalizeBatchDAV3(batchIndex uint64) *FinalizeBatchDAV3 { return &FinalizeBatchDAV3{ BatchIndex: batchIndex, diff --git a/rollup/da_syncer/da/revert.go b/rollup/da_syncer/da/revert.go index f02c264039b7..d84f22ebaa7b 100644 --- a/rollup/da_syncer/da/revert.go +++ b/rollup/da_syncer/da/revert.go @@ -22,3 +22,12 @@ func (r *RevertBatch) L1BlockNumber() uint64 { func (r *RevertBatch) BatchIndex() uint64 { return r.batchIndex } + +func (r *RevertBatch) CompareTo(other Entry) int { + if r.BatchIndex() < other.BatchIndex() { + return -1 + } else if r.BatchIndex() > other.BatchIndex() { + return 1 + } + return 0 +} From 362d16082215430843bc09eb4779790208854395 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:09:06 +0800 Subject: [PATCH 38/59] fix compile errors after rebase --- rollup/da_syncer/block_queue.go | 1 - rollup/da_syncer/da/commitV0.go | 10 +--------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 9e8aa12a051d..68954da69e2a 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -5,7 +5,6 @@ import ( "fmt" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" - "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BlockQueue struct { diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 945ae157c538..3c0f5d07478f 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -4,7 +4,6 @@ import ( "encoding/binary" "fmt" "io" - "math/big" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" @@ -100,15 +99,7 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { curL1TxIndex := c.parentTotalL1MessagePopped for _, chunk := range c.chunks { for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } // create txs - // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) // insert l1 msgs for l1TxPointer < len(c.l1Txs) && c.l1Txs[l1TxPointer].QueueIndex < curL1TxIndex+uint64(daBlock.NumL1Messages) { @@ -117,6 +108,7 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { l1TxPointer++ } curL1TxIndex += uint64(daBlock.NumL1Messages) + // insert l2 txs txs = append(txs, chunk.Transactions[blockId]...) From 329dd5470575ab58a004863643e59983af950582 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Sun, 4 Aug 2024 16:26:40 +0200 Subject: [PATCH 39/59] disable all p2p networking --- cmd/utils/flags.go | 3 +++ miner/scroll_worker.go | 2 +- node/config.go | 2 ++ node/node.go | 10 +++++++--- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 654aced7fed2..ec23c3835889 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -1325,6 +1325,9 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { setDataDir(ctx, cfg) setSmartCard(ctx, cfg) setL1(ctx, cfg) + if ctx.GlobalIsSet(DASyncEnabledFlag.Name) { + cfg.DaSyncingEnabled = ctx.GlobalBool(DASyncEnabledFlag.Name) + } if ctx.GlobalIsSet(ExternalSignerFlag.Name) { cfg.ExternalSigner = ctx.GlobalString(ExternalSignerFlag.Name) diff --git a/miner/scroll_worker.go b/miner/scroll_worker.go index d7feb9789de8..8a06874043c3 100644 --- a/miner/scroll_worker.go +++ b/miner/scroll_worker.go @@ -152,7 +152,7 @@ func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus } log.Info("created new worker", "CircuitCapacityChecker ID", worker.circuitCapacityChecker.ID) if daSyncingEnabled { - log.Info("worker will not start, because DA syncing is enabled") + log.Info("Worker will not start, because DA syncing is enabled") return worker } diff --git a/node/config.go b/node/config.go index 439b11a2f170..2dc3207dfdca 100644 --- a/node/config.go +++ b/node/config.go @@ -197,6 +197,8 @@ type Config struct { L1Confirmations rpc.BlockNumber `toml:",omitempty"` // L1 bridge deployment block number L1DeploymentBlock uint64 `toml:",omitempty"` + // Is daSyncingEnabled + DaSyncingEnabled bool `toml:",omitempty"` } // IPCEndpoint resolves an IPC endpoint based on a configured value, taking into diff --git a/node/node.go b/node/node.go index ac8c27dde51c..b5aa938e0162 100644 --- a/node/node.go +++ b/node/node.go @@ -262,9 +262,13 @@ func (n *Node) doClose(errs []error) error { // openEndpoints starts all network and RPC endpoints. func (n *Node) openEndpoints() error { // start networking endpoints - n.log.Info("Starting peer-to-peer node", "instance", n.server.Name) - if err := n.server.Start(); err != nil { - return convertFileLockError(err) + if !n.config.DaSyncingEnabled { + n.log.Info("Starting peer-to-peer node", "instance", n.server.Name) + if err := n.server.Start(); err != nil { + return convertFileLockError(err) + } + } else { + n.log.Info("Peer-to-peer node will not start, because DA syncing is enabled") } // start RPC endpoints err := n.startRPC() From d958069c8d2612c11f5dac027445207c9d80c680 Mon Sep 17 00:00:00 2001 From: NazariiDenha Date: Sun, 4 Aug 2024 14:31:51 +0000 Subject: [PATCH 40/59] =?UTF-8?q?chore:=20auto=20version=20bump=E2=80=89[b?= =?UTF-8?q?ot]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- params/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/params/version.go b/params/version.go index d2420d58b647..0b965b500c5e 100644 --- a/params/version.go +++ b/params/version.go @@ -24,7 +24,7 @@ import ( const ( VersionMajor = 5 // Major version component of the current release VersionMinor = 5 // Minor version component of the current release - VersionPatch = 22 // Patch version component of the current release + VersionPatch = 23 // Patch version component of the current release VersionMeta = "mainnet" // Version metadata to append to the version string ) From 69e1a9ea86e51794e3c9445ad9fa7f18e0db2e6f Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Sun, 4 Aug 2024 19:45:33 +0200 Subject: [PATCH 41/59] v3 finalization --- rollup/da_syncer/batch_queue.go | 9 ++++++--- rollup/da_syncer/da/finalize.go | 20 -------------------- 2 files changed, 6 insertions(+), 23 deletions(-) diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index b2ad881c0a26..e8ee7df5afe7 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -63,9 +63,12 @@ func (bq *BatchQueue) getFinalizedBatch() da.Entry { } batch := bq.batches.Peek() - bq.deleteBatch(batch.BatchIndex()) - - return batch + if batch.BatchIndex() <= bq.lastFinalizedBatchIndex { + bq.deleteBatch(batch.BatchIndex()) + return batch + } else { + return nil + } } // deleteBatch deletes data committed in the batch from map, because this batch is reverted or finalized diff --git a/rollup/da_syncer/da/finalize.go b/rollup/da_syncer/da/finalize.go index 332caa671f59..14d6c2a644cb 100644 --- a/rollup/da_syncer/da/finalize.go +++ b/rollup/da_syncer/da/finalize.go @@ -24,12 +24,6 @@ func (f *FinalizeBatch) BatchIndex() uint64 { return f.batchIndex } -type FinalizeBatchDAV3 struct { - BatchIndex uint64 - - L1BlockNumber uint64 -} - func (f *FinalizeBatch) CompareTo(other Entry) int { if f.BatchIndex() < other.BatchIndex() { return -1 @@ -38,17 +32,3 @@ func (f *FinalizeBatch) CompareTo(other Entry) int { } return 0 } - -func NewFinalizeBatchDAV3(batchIndex uint64) *FinalizeBatchDAV3 { - return &FinalizeBatchDAV3{ - BatchIndex: batchIndex, - } -} - -func (f *FinalizeBatchDAV3) DAType() Type { - return FinalizeBatchV3Type -} - -func (f *FinalizeBatchDAV3) GetL1BlockNumber() uint64 { - return f.L1BlockNumber -} From 0fd2a086913a4230498a47854bfc03346b976a7b Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 6 Aug 2024 08:56:10 +0800 Subject: [PATCH 42/59] feat: add shrinking map that shrinks itself after a certain number of deletions are performed --- common/shrinkingmap.go | 66 ++++++++++++++++++ common/shrinkingmap_test.go | 135 ++++++++++++++++++++++++++++++++++++ 2 files changed, 201 insertions(+) create mode 100644 common/shrinkingmap.go create mode 100644 common/shrinkingmap_test.go diff --git a/common/shrinkingmap.go b/common/shrinkingmap.go new file mode 100644 index 000000000000..de117362c54e --- /dev/null +++ b/common/shrinkingmap.go @@ -0,0 +1,66 @@ +package common + +// ShrinkingMap is a map that shrinks itself (by allocating a new map) after a certain number of deletions have been performed. +// If shrinkAfterDeletionsCount is set to <=0, the map will never shrink. +// This is useful to prevent memory leaks in long-running processes that delete a lot of keys from a map. +// See here for more details: https://github.com/golang/go/issues/20135 +type ShrinkingMap[K comparable, V any] struct { + m map[K]V + deletedKeys int + + shrinkAfterDeletionsCount int +} + +func NewShrinkingMap[K comparable, V any](shrinkAfterDeletionsCount int) *ShrinkingMap[K, V] { + return &ShrinkingMap[K, V]{ + m: make(map[K]V), + shrinkAfterDeletionsCount: shrinkAfterDeletionsCount, + } +} + +func (s *ShrinkingMap[K, V]) Set(key K, value V) { + s.m[key] = value +} + +func (s *ShrinkingMap[K, V]) Get(key K) (value V, exists bool) { + value, exists = s.m[key] + return value, exists +} + +func (s *ShrinkingMap[K, V]) Has(key K) bool { + _, exists := s.m[key] + return exists +} + +func (s *ShrinkingMap[K, V]) Delete(key K) (deleted bool) { + if _, exists := s.m[key]; !exists { + return false + } + + delete(s.m, key) + s.deletedKeys++ + + if s.shouldShrink() { + s.shrink() + } + + return true +} + +func (s *ShrinkingMap[K, V]) Size() (size int) { + return len(s.m) +} + +func (s *ShrinkingMap[K, V]) shouldShrink() bool { + return s.shrinkAfterDeletionsCount > 0 && s.deletedKeys >= s.shrinkAfterDeletionsCount +} + +func (s *ShrinkingMap[K, V]) shrink() { + newMap := make(map[K]V, len(s.m)) + for k, v := range s.m { + newMap[k] = v + } + + s.m = newMap + s.deletedKeys = 0 +} diff --git a/common/shrinkingmap_test.go b/common/shrinkingmap_test.go new file mode 100644 index 000000000000..c94a917ee140 --- /dev/null +++ b/common/shrinkingmap_test.go @@ -0,0 +1,135 @@ +package common + +import ( + "fmt" + "runtime" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestShrinkingMap_Shrink(t *testing.T) { + m := NewShrinkingMap[int, int](10) + + for i := 0; i < 100; i++ { + m.Set(i, i) + } + + for i := 0; i < 100; i++ { + val, exists := m.Get(i) + require.Equal(t, true, exists) + require.Equal(t, i, val) + + has := m.Has(i) + require.Equal(t, true, has) + } + + for i := 0; i < 9; i++ { + m.Delete(i) + } + require.Equal(t, 9, m.deletedKeys) + + // Delete the 10th key -> shrinks the map + m.Delete(9) + require.Equal(t, 0, m.deletedKeys) + + for i := 0; i < 100; i++ { + if i < 10 { + val, exists := m.Get(i) + require.Equal(t, false, exists) + require.Equal(t, 0, val) + + has := m.Has(i) + require.Equal(t, false, has) + } else { + val, exists := m.Get(i) + require.Equal(t, true, exists) + require.Equal(t, i, val) + + has := m.Has(i) + require.Equal(t, true, has) + } + } + + require.Equal(t, 90, m.Size()) +} + +func TestNewShrinkingMap_NoShrinking(t *testing.T) { + m := NewShrinkingMap[int, int](0) + for i := 0; i < 10000; i++ { + m.Set(i, i) + } + + for i := 0; i < 10000; i++ { + val, exists := m.Get(i) + require.Equal(t, true, exists) + require.Equal(t, i, val) + + m.Delete(i) + } + + require.Equal(t, 0, m.Size()) + require.Equal(t, 10000, m.deletedKeys) +} + +func TestShrinkingMap_MemoryShrinking(t *testing.T) { + t.Skip("Only for manual testing and memory profiling") + + gcAndPrintAlloc("start") + m := NewShrinkingMap[int, int](10000) + + const mapSize = 1_000_000 + + for i := 0; i < mapSize; i++ { + m.Set(i, i) + } + + gcAndPrintAlloc("after map creation") + + for i := 0; i < mapSize/2; i++ { + m.Delete(i) + } + + gcAndPrintAlloc("after removing half of the elements") + + val, exist := m.Get(mapSize - 1) + require.Equal(t, true, exist) + require.Equal(t, mapSize-1, val) + + gcAndPrintAlloc("end") +} + +func TestShrinkingMap_MemoryNoShrinking(t *testing.T) { + t.Skip("Only for manual testing and memory profiling") + + gcAndPrintAlloc("start") + m := NewShrinkingMap[int, int](0) + + const mapSize = 1_000_000 + + for i := 0; i < mapSize; i++ { + m.Set(i, i) + } + + gcAndPrintAlloc("after map creation") + + for i := 0; i < mapSize/2; i++ { + m.Delete(i) + } + + gcAndPrintAlloc("after removing half of the elements") + + val, exist := m.Get(mapSize - 1) + require.Equal(t, true, exist) + require.Equal(t, mapSize-1, val) + + gcAndPrintAlloc("end") +} + +func gcAndPrintAlloc(prefix string) { + runtime.GC() + + var stats runtime.MemStats + runtime.ReadMemStats(&stats) + fmt.Printf(prefix+", Allocated memory %d KiB\n", stats.Alloc/1024) +} From 3f68f2c4d2856b63904e81afb425d37c7928434b Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 6 Aug 2024 08:57:06 +0800 Subject: [PATCH 43/59] feat: use shrinking map in batch queue to delete specific elements from heap instead of through loop --- common/heap.go | 19 ++++++++++++------- rollup/da_syncer/batch_queue.go | 33 +++++++++++++++++++-------------- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/common/heap.go b/common/heap.go index 0906c2edddf1..c34645c69146 100644 --- a/common/heap.go +++ b/common/heap.go @@ -18,22 +18,27 @@ func (h *Heap[T]) Len() int { return len(h.heap) } -func (h *Heap[T]) Push(element T) { +func (h *Heap[T]) Push(element T) *HeapElement[T] { heapElement := NewHeapElement(element) heap.Push(&h.heap, heapElement) + + return heapElement } -func (h *Heap[T]) Pop() T { - return heap.Pop(&h.heap).(*HeapElement[T]).Value() +func (h *Heap[T]) Pop() *HeapElement[T] { + return heap.Pop(&h.heap).(*HeapElement[T]) } -func (h *Heap[T]) Peek() T { +func (h *Heap[T]) Peek() *HeapElement[T] { if h.Len() == 0 { - var empty T - return empty + return nil } - return h.heap[0].Value() + return h.heap[0] +} + +func (h *Heap[T]) Remove(element *HeapElement[T]) { + heap.Remove(&h.heap, element.index) } type innerHeap[T Comparable[T]] []*HeapElement[T] diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index e8ee7df5afe7..e3c18d2dfe70 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -15,6 +15,7 @@ type BatchQueue struct { db ethdb.Database lastFinalizedBatchIndex uint64 batches *common.Heap[da.Entry] + batchesMap *common.ShrinkingMap[uint64, *common.HeapElement[da.Entry]] } func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { @@ -23,6 +24,7 @@ func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { db: db, lastFinalizedBatchIndex: 0, batches: common.NewHeap[da.Entry](), + batchesMap: common.NewShrinkingMap[uint64, *common.HeapElement[da.Entry]](1000), } } @@ -39,9 +41,9 @@ func (bq *BatchQueue) NextBatch(ctx context.Context) (da.Entry, error) { } switch daEntry.Type() { case da.CommitBatchV0Type, da.CommitBatchV1Type, da.CommitBatchV2Type: - bq.batches.Push(daEntry) + bq.addBatch(daEntry) case da.RevertBatchType: - bq.deleteBatch(daEntry.BatchIndex()) + bq.deleteBatch(daEntry) case da.FinalizeBatchType: if daEntry.BatchIndex() > bq.lastFinalizedBatchIndex { bq.lastFinalizedBatchIndex = daEntry.BatchIndex() @@ -62,28 +64,31 @@ func (bq *BatchQueue) getFinalizedBatch() da.Entry { return nil } - batch := bq.batches.Peek() + batch := bq.batches.Peek().Value() if batch.BatchIndex() <= bq.lastFinalizedBatchIndex { - bq.deleteBatch(batch.BatchIndex()) + bq.deleteBatch(batch) return batch } else { return nil } } +func (bq *BatchQueue) addBatch(batch da.Entry) { + heapElement := bq.batches.Push(batch) + bq.batchesMap.Set(batch.BatchIndex(), heapElement) +} + // deleteBatch deletes data committed in the batch from map, because this batch is reverted or finalized // updates DASyncedL1BlockNumber -func (bq *BatchQueue) deleteBatch(batchIndex uint64) { - var batch da.Entry - for batch = bq.batches.Peek(); batch.BatchIndex() <= batchIndex; { - bq.batches.Pop() - - if bq.batches.Len() == 0 { - break - } - batch = bq.batches.Peek() +func (bq *BatchQueue) deleteBatch(batch da.Entry) { + batchHeapElement, exists := bq.batchesMap.Get(batch.BatchIndex()) + if !exists { + return } + bq.batchesMap.Delete(batch.BatchIndex()) + bq.batches.Remove(batchHeapElement) + if bq.batches.Len() == 0 { curBatchL1Height := batch.L1BlockNumber() rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) @@ -91,5 +96,5 @@ func (bq *BatchQueue) deleteBatch(batchIndex uint64) { } // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart - rawdb.WriteDASyncedL1BlockNumber(bq.db, bq.batches.Peek().L1BlockNumber()-1) + rawdb.WriteDASyncedL1BlockNumber(bq.db, bq.batches.Peek().Value().L1BlockNumber()-1) } From 53bdb1835f3cdc43e68545c33c9135a79a0d16c5 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 14:48:32 +0800 Subject: [PATCH 44/59] feat: execute blocks only once --- core/blockchain.go | 45 +++++++++++++++++++++++++++++++++++ rollup/da_syncer/da_syncer.go | 29 +++++++--------------- 2 files changed, 54 insertions(+), 20 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 54a4573e9132..3a8ca512bebb 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1824,6 +1824,51 @@ func (bc *BlockChain) PreprocessBlock(block *types.Block) (common.Hash, types.Bl return receiptSha, bloom, stateRoot, usedGas, nil } +func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types.Header, txs types.Transactions) (WriteStatus, error) { + if !bc.chainmu.TryLock() { + return NonStatTy, errInsertionInterrupted + } + defer bc.chainmu.Unlock() + + statedb, err := state.New(parentBlock.Root(), bc.stateCache, bc.snaps) + if err != nil { + return NonStatTy, err + } + + tempBlock := types.NewBlockWithHeader(header).WithBody(txs, nil) + receipts, logs, gasUsed, err := bc.processor.Process(tempBlock, statedb, bc.vmConfig) + if err != nil { + return NonStatTy, err + } + + header.GasUsed = gasUsed + header.ParentHash = parentBlock.Hash() + header.Root = statedb.GetRootHash() + // Since we're using Clique consensus, we don't have uncles + header.UncleHash = types.EmptyUncleHash + // TODO: extraData and difficulty should be set + + fullBlock := types.NewBlock(header, txs, nil, receipts, trie.NewStackTrie(nil)) + + blockHash := fullBlock.Hash() + // manually replace the block hash in the receipts + for i, receipt := range receipts { + // add block location fields + receipt.BlockHash = blockHash + receipt.BlockNumber = tempBlock.Number() + receipt.TransactionIndex = uint(i) + + for _, l := range receipt.Logs { + l.BlockHash = blockHash + } + } + for _, l := range logs { + l.BlockHash = blockHash + } + + return bc.writeBlockWithState(fullBlock, receipts, logs, statedb, false) +} + // insertSideChain is called when an import batch hits upon a pruned ancestor // error, which happens when a sidechain with a sufficiently old fork-block is // found. diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index e8dbf7859e84..2c34847fc499 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -8,7 +8,6 @@ import ( "github.com/scroll-tech/go-ethereum/core" "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" - "github.com/scroll-tech/go-ethereum/trie" ) type DASyncer struct { @@ -22,32 +21,22 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { } func (s *DASyncer) SyncOneBlock(block *types.Block) error { - prevHash := s.blockchain.CurrentBlock().Hash() - if big.NewInt(0).Add(s.blockchain.CurrentBlock().Number(), common.Big1).Cmp(block.Number()) != 0 { + parentBlock := s.blockchain.CurrentBlock() + if big.NewInt(0).Add(parentBlock.Number(), common.Big1).Cmp(block.Number()) != 0 { return fmt.Errorf("not consecutive block, number: %d", block.Number()) } - header := block.Header() - txs := block.Transactions() - // fill header with all necessary fields - var err error - header.ReceiptHash, header.Bloom, header.Root, header.GasUsed, err = s.blockchain.PreprocessBlock(block) - if err != nil { - return fmt.Errorf("block preprocessing failed, block number: %d, error: %v", block.Number(), err) - } - header.UncleHash = common.HexToHash("0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347") + header := block.Header() header.Difficulty = common.Big1 - header.BaseFee = nil - header.TxHash = types.DeriveSha(txs, trie.NewStackTrie(nil)) - header.ParentHash = prevHash + header.BaseFee = nil // TODO: after Curie we need to fill this correctly + header.ParentHash = parentBlock.Hash() - fullBlock := types.NewBlockWithHeader(header).WithBody(txs, make([]*types.Header, 0)) - - if _, err := s.blockchain.InsertChainWithoutSealVerification(fullBlock); err != nil { - return fmt.Errorf("cannot insert block, number: %d, error: %v", block.Number(), err) + if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, header, block.Transactions()); err != nil { + return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.Number(), err) } + if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash()) + log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) } return nil } From 5243d59d18df29ec64fcdb87fd6c8747206c9006 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 15:18:46 +0800 Subject: [PATCH 45/59] refactor: introduce partial header and partial block for data from DA before execution --- core/blockchain.go | 4 ++-- rollup/da_syncer/block_queue.go | 7 +++---- rollup/da_syncer/da/commitV0.go | 24 ++++++++++++---------- rollup/da_syncer/da/da.go | 36 ++++++++++++++++++++++++++++++++- rollup/da_syncer/da_syncer.go | 20 +++++++----------- 5 files changed, 60 insertions(+), 31 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 3a8ca512bebb..e55f6ce24085 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1835,6 +1835,8 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types return NonStatTy, err } + header.ParentHash = parentBlock.Hash() + tempBlock := types.NewBlockWithHeader(header).WithBody(txs, nil) receipts, logs, gasUsed, err := bc.processor.Process(tempBlock, statedb, bc.vmConfig) if err != nil { @@ -1842,11 +1844,9 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types } header.GasUsed = gasUsed - header.ParentHash = parentBlock.Hash() header.Root = statedb.GetRootHash() // Since we're using Clique consensus, we don't have uncles header.UncleHash = types.EmptyUncleHash - // TODO: extraData and difficulty should be set fullBlock := types.NewBlock(header, txs, nil, receipts, trie.NewStackTrie(nil)) diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index abd728912b21..68954da69e2a 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -4,23 +4,22 @@ import ( "context" "fmt" - "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type BlockQueue struct { batchQueue *BatchQueue - blocks []*types.Block + blocks []*da.PartialBlock } func NewBlockQueue(batchQueue *BatchQueue) *BlockQueue { return &BlockQueue{ batchQueue: batchQueue, - blocks: []*types.Block{}, + blocks: make([]*da.PartialBlock, 0), } } -func (bq *BlockQueue) NextBlock(ctx context.Context) (*types.Block, error) { +func (bq *BlockQueue) NextBlock(ctx context.Context) (*da.PartialBlock, error) { for len(bq.blocks) == 0 { err := bq.getBlocksFromBatch(ctx) if err != nil { diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index f4305e78ceb0..432c6d8462de 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -4,7 +4,6 @@ import ( "encoding/binary" "fmt" "io" - "math/big" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" @@ -84,20 +83,13 @@ func (c *CommitBatchDAV0) BatchIndex() uint64 { return c.batchIndex } -func (c *CommitBatchDAV0) Blocks() ([]*types.Block, error) { - var blocks []*types.Block +func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { + var blocks []*PartialBlock l1TxPointer := 0 curL1TxIndex := c.parentTotalL1MessagePopped for _, chunk := range c.chunks { for blockId, daBlock := range chunk.Blocks { - // create header - header := types.Header{ - Number: big.NewInt(0).SetUint64(daBlock.BlockNumber), - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - } // create txs // var txs types.Transactions txs := make(types.Transactions, 0, daBlock.NumTransactions) @@ -110,7 +102,17 @@ func (c *CommitBatchDAV0) Blocks() ([]*types.Block, error) { curL1TxIndex += uint64(daBlock.NumL1Messages) // insert l2 txs txs = append(txs, chunk.Transactions[blockId]...) - block := types.NewBlockWithHeader(&header).WithBody(txs, make([]*types.Header, 0)) + + block := NewPartialBlock( + &PartialHeader{ + Number: daBlock.BlockNumber, + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + //TODO: Difficulty: new(big.Int).SetUint64(10), + //TODO: ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, + }, + txs) blocks = append(blocks, block) } } diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index a225b2e76737..829f8d4712dd 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -1,6 +1,8 @@ package da import ( + "math/big" + "github.com/scroll-tech/go-ethereum/core/types" ) @@ -29,7 +31,39 @@ type Entry interface { type EntryWithBlocks interface { Entry - Blocks() ([]*types.Block, error) + Blocks() ([]*PartialBlock, error) } type Entries []Entry + +type PartialHeader struct { + Number uint64 + Time uint64 + BaseFee *big.Int + GasLimit uint64 + Difficulty *big.Int + ExtraData []byte +} + +func (h *PartialHeader) ToHeader() *types.Header { + return &types.Header{ + Number: big.NewInt(0).SetUint64(h.Number), + Time: h.Time, + BaseFee: h.BaseFee, + GasLimit: h.GasLimit, + Difficulty: h.Difficulty, + Extra: h.ExtraData, + } +} + +type PartialBlock struct { + PartialHeader *PartialHeader + Transactions types.Transactions +} + +func NewPartialBlock(partialHeader *PartialHeader, txs types.Transactions) *PartialBlock { + return &PartialBlock{ + PartialHeader: partialHeader, + Transactions: txs, + } +} diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 2c34847fc499..b1adaa7ea994 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -2,12 +2,10 @@ package da_syncer import ( "fmt" - "math/big" - "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/core" - "github.com/scroll-tech/go-ethereum/core/types" "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) type DASyncer struct { @@ -20,23 +18,19 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { } } -func (s *DASyncer) SyncOneBlock(block *types.Block) error { +func (s *DASyncer) SyncOneBlock(block *da.PartialBlock) error { parentBlock := s.blockchain.CurrentBlock() - if big.NewInt(0).Add(parentBlock.Number(), common.Big1).Cmp(block.Number()) != 0 { - return fmt.Errorf("not consecutive block, number: %d", block.Number()) + if parentBlock.NumberU64()+1 != block.PartialHeader.Number { + return fmt.Errorf("not consecutive block, number: %d, chain height: %d", block.PartialHeader.Number, parentBlock.NumberU64()) } - header := block.Header() - header.Difficulty = common.Big1 - header.BaseFee = nil // TODO: after Curie we need to fill this correctly - header.ParentHash = parentBlock.Hash() - - if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, header, block.Transactions()); err != nil { - return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.Number(), err) + if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, block.PartialHeader.ToHeader(), block.Transactions); err != nil { + return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.PartialHeader.Number, err) } if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) } + return nil } From 7018b5b22403c414365cd2ddebb1b4c7936942df Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 26 Jul 2024 15:26:47 +0800 Subject: [PATCH 46/59] minor cleanup --- core/blockchain.go | 21 --------------------- rollup/da_syncer/da/da.go | 4 ++-- 2 files changed, 2 insertions(+), 23 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index e55f6ce24085..e6ec46756dcb 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1803,27 +1803,6 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er return it.index, err } -// PreprocessBlock processes block on top of the chain to calculate receipts, bloom and state root -func (bc *BlockChain) PreprocessBlock(block *types.Block) (common.Hash, types.Bloom, common.Hash, uint64, error) { - // Retrieve the parent block and it's state to execute on top - parent := bc.CurrentBlock().Header() - if parent == nil { - parent = bc.GetHeader(block.ParentHash(), block.NumberU64()-1) - } - statedb, err := state.New(parent.Root, bc.stateCache, bc.snaps) - if err != nil { - return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err - } - receipts, _, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig) - if err != nil { - return common.Hash{}, types.Bloom{}, common.Hash{}, 0, err - } - receiptSha := types.DeriveSha(receipts, trie.NewStackTrie(nil)) - bloom := types.CreateBloom(receipts) - stateRoot := statedb.GetRootHash() - return receiptSha, bloom, stateRoot, usedGas, nil -} - func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types.Header, txs types.Transactions) (WriteStatus, error) { if !bc.chainmu.TryLock() { return NonStatTy, errInsertionInterrupted diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index 829f8d4712dd..fbaa60962758 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -41,7 +41,7 @@ type PartialHeader struct { Time uint64 BaseFee *big.Int GasLimit uint64 - Difficulty *big.Int + Difficulty uint64 ExtraData []byte } @@ -51,7 +51,7 @@ func (h *PartialHeader) ToHeader() *types.Header { Time: h.Time, BaseFee: h.BaseFee, GasLimit: h.GasLimit, - Difficulty: h.Difficulty, + Difficulty: new(big.Int).SetUint64(h.Difficulty), Extra: h.ExtraData, } } From fe33d8380a06da0a168888f4d6d9eac15408368c Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:23:17 +0800 Subject: [PATCH 47/59] feat: fix issue with not specifying difficulty --- rollup/da_syncer/da/commitV0.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 432c6d8462de..481bb60ed124 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -105,12 +105,12 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { block := NewPartialBlock( &PartialHeader{ - Number: daBlock.BlockNumber, - Time: daBlock.Timestamp, - BaseFee: daBlock.BaseFee, - GasLimit: daBlock.GasLimit, - //TODO: Difficulty: new(big.Int).SetUint64(10), - //TODO: ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, + Number: daBlock.BlockNumber, + Time: daBlock.Timestamp, + BaseFee: daBlock.BaseFee, + GasLimit: daBlock.GasLimit, + Difficulty: 10, // TODO: replace with real difficulty + ExtraData: []byte{1, 2, 3, 4, 5, 6, 7, 8}, // TODO: replace with real extra data }, txs) blocks = append(blocks, block) From d3ca47ee99c59d0bffec8c46cbbf6f91d5d89b92 Mon Sep 17 00:00:00 2001 From: Jonas Theis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 6 Aug 2024 11:23:27 +0800 Subject: [PATCH 48/59] feat: implement simple pipeline reset (#941) * feat: implement graceful handling of skipping blocks in the past and resetting of pipeline for too new blocks until we're at the correct block height again * change logging * remove unnecessary check * feat: clear heap and map --- common/heap.go | 4 ++ common/shrinkingmap.go | 5 +++ rollup/da_syncer/batch_queue.go | 15 ++++---- rollup/da_syncer/block_queue.go | 5 +++ rollup/da_syncer/da_queue.go | 20 +++++++--- rollup/da_syncer/da_syncer.go | 18 +++++++-- rollup/da_syncer/syncing_pipeline.go | 56 +++++++++++++++++++++------- 7 files changed, 93 insertions(+), 30 deletions(-) diff --git a/common/heap.go b/common/heap.go index c34645c69146..933c91a04d77 100644 --- a/common/heap.go +++ b/common/heap.go @@ -41,6 +41,10 @@ func (h *Heap[T]) Remove(element *HeapElement[T]) { heap.Remove(&h.heap, element.index) } +func (h *Heap[T]) Clear() { + h.heap = make(innerHeap[T], 0) +} + type innerHeap[T Comparable[T]] []*HeapElement[T] func (h innerHeap[T]) Len() int { diff --git a/common/shrinkingmap.go b/common/shrinkingmap.go index de117362c54e..4bf98f87c2da 100644 --- a/common/shrinkingmap.go +++ b/common/shrinkingmap.go @@ -51,6 +51,11 @@ func (s *ShrinkingMap[K, V]) Size() (size int) { return len(s.m) } +func (s *ShrinkingMap[K, V]) Clear() { + s.m = make(map[K]V) + s.deletedKeys = 0 +} + func (s *ShrinkingMap[K, V]) shouldShrink() bool { return s.shrinkAfterDeletionsCount > 0 && s.deletedKeys >= s.shrinkAfterDeletionsCount } diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index e3c18d2dfe70..e6f6a26c612a 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -89,12 +89,13 @@ func (bq *BatchQueue) deleteBatch(batch da.Entry) { bq.batchesMap.Delete(batch.BatchIndex()) bq.batches.Remove(batchHeapElement) - if bq.batches.Len() == 0 { - curBatchL1Height := batch.L1BlockNumber() - rawdb.WriteDASyncedL1BlockNumber(bq.db, curBatchL1Height) - return - } - // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart - rawdb.WriteDASyncedL1BlockNumber(bq.db, bq.batches.Peek().Value().L1BlockNumber()-1) + // TODO: we should store this information when the batch is done being processed to avoid inconsistencies + rawdb.WriteDASyncedL1BlockNumber(bq.db, batch.L1BlockNumber()-1) +} + +func (bq *BatchQueue) Reset(height uint64) { + bq.batches.Clear() + bq.batchesMap.Clear() + bq.DAQueue.Reset(height) } diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 68954da69e2a..2cd2abb22bb7 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -50,3 +50,8 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { return nil } + +func (bq *BlockQueue) Reset(height uint64) { + bq.blocks = make([]*da.PartialBlock, 0) + bq.batchQueue.Reset(height) +} diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index bdbbbb5428bd..71dabdd020f1 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -2,6 +2,7 @@ package da_syncer import ( "context" + "errors" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) @@ -42,15 +43,24 @@ func (dq *DAQueue) getNextData(ctx context.Context) error { return err } } + dq.da, err = dq.dataSource.NextData() + if err == nil { + return nil + } + // previous dataSource has been exhausted, create new - if err == da.ErrSourceExhausted { + if errors.Is(err, da.ErrSourceExhausted) { dq.l1height = dq.dataSource.L1Height() dq.dataSource = nil return dq.getNextData(ctx) } - if err != nil { - return err - } - return nil + + return err +} + +func (dq *DAQueue) Reset(height uint64) { + dq.l1height = height + dq.dataSource = nil + dq.da = make(da.Entries, 0) } diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index b1adaa7ea994..1ca0d11ee23b 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -8,6 +8,11 @@ import ( "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) +var ( + ErrBlockTooLow = fmt.Errorf("block number is too low") + ErrBlockTooHigh = fmt.Errorf("block number is too high") +) + type DASyncer struct { blockchain *core.BlockChain } @@ -20,16 +25,21 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { func (s *DASyncer) SyncOneBlock(block *da.PartialBlock) error { parentBlock := s.blockchain.CurrentBlock() - if parentBlock.NumberU64()+1 != block.PartialHeader.Number { - return fmt.Errorf("not consecutive block, number: %d, chain height: %d", block.PartialHeader.Number, parentBlock.NumberU64()) + // we expect blocks to be consecutive. block.PartialHeader.Number == parentBlock.Number+1. + if block.PartialHeader.Number <= parentBlock.NumberU64() { + log.Debug("block number is too low", "block number", block.PartialHeader.Number, "parent block number", parentBlock.NumberU64()) + return ErrBlockTooLow + } else if block.PartialHeader.Number > parentBlock.NumberU64()+1 { + log.Debug("block number is too high", "block number", block.PartialHeader.Number, "parent block number", parentBlock.NumberU64()) + return ErrBlockTooHigh } if _, err := s.blockchain.BuildAndWriteBlock(parentBlock, block.PartialHeader.ToHeader(), block.Transactions); err != nil { return fmt.Errorf("failed building and writing block, number: %d, error: %v", block.PartialHeader.Number, err) } - if s.blockchain.CurrentBlock().Header().Number.Uint64()%100 == 0 { - log.Info("inserted block", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) + if s.blockchain.CurrentBlock().Header().Number.Uint64()%1000 == 0 { + log.Info("L1 sync progress", "blockhain height", s.blockchain.CurrentBlock().Header().Number, "block hash", s.blockchain.CurrentBlock().Header().Hash(), "root", s.blockchain.CurrentBlock().Header().Root) } return nil diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 231871bdd069..9d639704700e 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -5,7 +5,6 @@ import ( "errors" "fmt" "io" - "strings" "sync" "time" @@ -33,6 +32,8 @@ type SyncingPipeline struct { wg sync.WaitGroup expBackoff *backoff.Exponential + l1DeploymentBlock uint64 + db ethdb.Database blockchain *core.BlockChain blockQueue *BlockQueue @@ -74,14 +75,15 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi ctx, cancel := context.WithCancel(ctx) return &SyncingPipeline{ - ctx: ctx, - cancel: cancel, - expBackoff: backoff.NewExponential(100*time.Millisecond, 10*time.Second, 100*time.Millisecond), - wg: sync.WaitGroup{}, - db: db, - blockchain: blockchain, - blockQueue: blockQueue, - daSyncer: daSyncer, + ctx: ctx, + cancel: cancel, + expBackoff: backoff.NewExponential(100*time.Millisecond, 10*time.Second, 100*time.Millisecond), + wg: sync.WaitGroup{}, + l1DeploymentBlock: l1DeploymentBlock, + db: db, + blockchain: blockchain, + blockQueue: blockQueue, + daSyncer: daSyncer, }, nil } @@ -107,6 +109,7 @@ func (s *SyncingPipeline) Start() { func (s *SyncingPipeline) mainLoop() { stepCh := make(chan struct{}, 1) var delayedStepCh <-chan time.Time + var resetCounter int // 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. @@ -148,6 +151,7 @@ func (s *SyncingPipeline) mainLoop() { if err == nil { reqStep(false) s.expBackoff.Reset() + resetCounter = 0 continue } @@ -155,15 +159,27 @@ func (s *SyncingPipeline) mainLoop() { reqStep(true) continue } + if errors.Is(err, ErrBlockTooLow) { + // block number returned by the block queue is too low, + // we skip the blocks until we reach the correct block number again. + reqStep(false) + continue + } else if errors.Is(err, ErrBlockTooHigh) { + // block number returned by the block queue is too high, + // reset the pipeline and move backwards from the last L1 block we read + s.reset(resetCounter) + resetCounter++ + reqStep(false) + continue + } + if errors.Is(err, context.Canceled) { + log.Info("syncing pipeline stopped due to cancelled context", "err", err) return } - if strings.HasPrefix(err.Error(), "not consecutive block") { - log.Warn("syncing pipeline step failed, probably because of restart", "err", err) - } else { - log.Crit("syncing pipeline step failed", "err", err) - } + log.Warn("syncing pipeline step failed due to unrecoverable error, stopping pipeline worker", "err", err) + return } } } @@ -174,3 +190,15 @@ func (s *SyncingPipeline) Stop() { s.wg.Wait() log.Info("Stopped DaSyncer... Done") } + +func (s *SyncingPipeline) reset(resetCounter int) { + amount := 100 * uint64(resetCounter) + syncedL1Height := s.l1DeploymentBlock - 1 + from := rawdb.ReadDASyncedL1BlockNumber(s.db) + if from != nil && *from+amount > syncedL1Height { + syncedL1Height = *from - amount + rawdb.WriteDASyncedL1BlockNumber(s.db, syncedL1Height) + } + log.Info("resetting syncing pipeline", "syncedL1Height", syncedL1Height) + s.blockQueue.Reset(syncedL1Height) +} From 19ed11d0a6085558758ffd75e0329b4488db9f3a Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 6 Aug 2024 12:27:36 +0800 Subject: [PATCH 49/59] feat: remove changes to Clique as we're not verifying signatures when reading in pipeline --- consensus/clique/clique.go | 38 ++++++++++++++------------------------ eth/backend.go | 4 +--- params/config.go | 1 - 3 files changed, 15 insertions(+), 28 deletions(-) diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index ffdc09244d5a..448cf75efab0 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -213,9 +213,6 @@ func New(config *params.CliqueConfig, db ethdb.Database) *Clique { // Author implements consensus.Engine, returning the Ethereum address recovered // from the signature in the header's extra-data section. func (c *Clique) Author(header *types.Header) (common.Address, error) { - if c.config.DaSyncingEnabled { - return common.BigToAddress(big.NewInt(0).SetUint64(12345)), nil - } return ecrecover(header, c.signatures) } @@ -271,22 +268,20 @@ func (c *Clique) verifyHeader(chain consensus.ChainHeaderReader, header *types.H if checkpoint && !bytes.Equal(header.Nonce[:], nonceDropVote) { return errInvalidCheckpointVote } - if !c.config.DaSyncingEnabled { - // Check that the extra-data contains both the vanity and signature - if len(header.Extra) < extraVanity { - return errMissingVanity - } - if len(header.Extra) < extraVanity+extraSeal { - return errMissingSignature - } - // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise - signersBytes := len(header.Extra) - extraVanity - extraSeal - if !checkpoint && signersBytes != 0 { - return errExtraSigners - } - if checkpoint && signersBytes%common.AddressLength != 0 { - return errInvalidCheckpointSigners - } + // Check that the extra-data contains both the vanity and signature + if len(header.Extra) < extraVanity { + return errMissingVanity + } + if len(header.Extra) < extraVanity+extraSeal { + return errMissingSignature + } + // Ensure that the extra-data contains a signer list on checkpoint, but none otherwise + signersBytes := len(header.Extra) - extraVanity - extraSeal + if !checkpoint && signersBytes != 0 { + return errExtraSigners + } + if checkpoint && signersBytes%common.AddressLength != 0 { + return errInvalidCheckpointSigners } // Ensure that the mix digest is zero as we don't have fork protection currently if header.MixDigest != (common.Hash{}) { @@ -354,11 +349,6 @@ func (c *Clique) verifyCascadingFields(chain consensus.ChainHeaderReader, header // Verify the header's EIP-1559 attributes. return err } - - if c.config.DaSyncingEnabled { - return nil - } - // Retrieve the snapshot needed to verify this header and cache it snap, err := c.snapshot(chain, number-1, header.ParentHash, parents) if err != nil { diff --git a/eth/backend.go b/eth/backend.go index b28f562024a0..2027ab12d124 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -143,9 +143,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl if _, ok := genesisErr.(*params.ConfigCompatError); genesisErr != nil && !ok { return nil, genesisErr } - if chainConfig.Clique != nil && config.EnableDASyncing { - chainConfig.Clique.DaSyncingEnabled = true - } + log.Info("Initialised chain configuration", "config", chainConfig) if err := pruner.RecoverPruning(stack.ResolvePath(""), chainDb, stack.ResolvePath(config.TrieCleanCacheJournal)); err != nil { diff --git a/params/config.go b/params/config.go index e90f0272b170..d6a1b4c8eea7 100644 --- a/params/config.go +++ b/params/config.go @@ -739,7 +739,6 @@ type CliqueConfig struct { RelaxedPeriod bool `json:"relaxed_period"` // Relaxes the period to be just an upper bound ShadowForkHeight uint64 `json:"shadow_fork_height"` // Allows shadow forking consensus layer at given height ShadowForkSigner common.Address `json:"shadow_fork_signer"` // Sets the address to be the authorized signer after the shadow fork - DaSyncingEnabled bool `json:"da_syncing_enabled"` // Is daSyncingEnabled } // String implements the stringer interface, returning the consensus engine details. From a6461d53e0783af30992929ea0b4d6e91bcd1ccd Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 6 Aug 2024 12:47:19 +0800 Subject: [PATCH 50/59] feat: only request finalized block number when necessary --- rollup/da_syncer/da/calldata_blob_source.go | 22 +++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go index a8fdd3ea5100..19fd1b5eccae 100644 --- a/rollup/da_syncer/da/calldata_blob_source.go +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -40,6 +40,8 @@ type CalldataBlobSource struct { l1RevertBatchEventSignature common.Hash l1FinalizeBatchEventSignature common.Hash db ethdb.Database + + l1Finalized uint64 } func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *rollup_sync_service.L1Client, blobClient blob_client.BlobClient, db ethdb.Database) (*CalldataBlobSource, error) { @@ -61,17 +63,25 @@ func NewCalldataBlobSource(ctx context.Context, l1height uint64, l1Client *rollu } func (ds *CalldataBlobSource) NextData() (Entries, error) { + var err error to := ds.l1height + callDataBlobSourceFetchBlockRange - l1Finalized, err := ds.l1Client.GetLatestFinalizedBlockNumber() - if err != nil { - return nil, fmt.Errorf("cannot get l1height, error: %v", err) - } - if to > l1Finalized { - to = l1Finalized + + // If there's not enough finalized blocks to request up to, we need to query finalized block number. + // Otherwise, we know that there's more finalized blocks than we want to request up to + // -> no need to query finalized block number + if to > ds.l1Finalized { + ds.l1Finalized, err = ds.l1Client.GetLatestFinalizedBlockNumber() + if err != nil { + return nil, fmt.Errorf("failed to query GetLatestFinalizedBlockNumber, error: %v", err) + } + // make sure we don't request more than finalized blocks + to = min(to, ds.l1Finalized) } + if ds.l1height > to { return nil, ErrSourceExhausted } + logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) if err != nil { return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) From 85bbc981aad1ca737806784ed5080ca7714a272d Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Wed, 7 Aug 2024 11:02:49 +0800 Subject: [PATCH 51/59] minor cleanup and comments --- common/backoff/exponential.go | 1 + common/heap.go | 9 +++++++++ core/blockchain.go | 5 +++-- rollup/da_syncer/batch_queue.go | 1 + rollup/da_syncer/block_queue.go | 2 ++ rollup/da_syncer/da/commitV0.go | 3 ++- rollup/da_syncer/da/da.go | 5 +++-- rollup/da_syncer/da_queue.go | 1 + rollup/da_syncer/da_syncer.go | 1 + rollup/da_syncer/syncing_pipeline.go | 4 ++++ 10 files changed, 27 insertions(+), 5 deletions(-) diff --git a/common/backoff/exponential.go b/common/backoff/exponential.go index 4e43c4e13346..e1f9b53a350e 100644 --- a/common/backoff/exponential.go +++ b/common/backoff/exponential.go @@ -6,6 +6,7 @@ import ( "time" ) +// Exponential is a backoff strategy that increases the delay between retries exponentially. type Exponential struct { attempt int diff --git a/common/heap.go b/common/heap.go index 933c91a04d77..67b79a1136d1 100644 --- a/common/heap.go +++ b/common/heap.go @@ -4,6 +4,7 @@ import ( "container/heap" ) +// Heap is a generic min-heap (or max-heap, depending on Comparable behavior) implementation. type Heap[T Comparable[T]] struct { heap innerHeap[T] } @@ -76,10 +77,18 @@ func (h *innerHeap[T]) Pop() interface{} { return element } +// Comparable is an interface for types that can be compared. type Comparable[T any] interface { + // CompareTo compares x with other. + // To create a min heap, return: + // -1 if x < other + // 0 if x == other + // +1 if x > other + // To create a max heap, return the opposite. CompareTo(other T) int } +// HeapElement is a wrapper around the value stored in the heap. type HeapElement[T Comparable[T]] struct { value T index int diff --git a/core/blockchain.go b/core/blockchain.go index e6ec46756dcb..9c2ff37703a3 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1822,10 +1822,11 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types return NonStatTy, err } + // TODO: once we have the extra and difficulty we need to verify the signature of the block with Clique + // This should be done with https://github.com/scroll-tech/go-ethereum/pull/913. + header.GasUsed = gasUsed header.Root = statedb.GetRootHash() - // Since we're using Clique consensus, we don't have uncles - header.UncleHash = types.EmptyUncleHash fullBlock := types.NewBlock(header, txs, nil, receipts, trie.NewStackTrie(nil)) diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go index e6f6a26c612a..7a3d094f6322 100644 --- a/rollup/da_syncer/batch_queue.go +++ b/rollup/da_syncer/batch_queue.go @@ -10,6 +10,7 @@ import ( "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) +// BatchQueue is a pipeline stage that reads all batch events from DAQueue and provides only finalized batches to the next stage. type BatchQueue struct { DAQueue *DAQueue db ethdb.Database diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 2cd2abb22bb7..3c3e68bdcf79 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -7,6 +7,8 @@ import ( "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) +// BlockQueue is a pipeline stage that reads batches from BatchQueue, extracts all da.PartialBlock from it and +// provides them to the next stage one-by-one. type BlockQueue struct { batchQueue *BatchQueue blocks []*da.PartialBlock diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index 3c0f5d07478f..abedfd3231d5 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -155,7 +155,8 @@ func getL1Messages(db ethdb.Database, parentTotalL1MessagePopped uint64, skipped } l1Tx := rawdb.ReadL1Message(db, currentIndex) if l1Tx == nil { - // TODO: returning io.EOF is not the best way to handle this + // message not yet available + // we return io.EOF as this will be handled in the syncing pipeline with a backoff and retry return nil, io.EOF } txs = append(txs, l1Tx) diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index 258fdf2d7805..935f09cfda7c 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -19,10 +19,9 @@ const ( RevertBatchType // FinalizeBatchType contains data of event of FinalizeBatchType FinalizeBatchType - // FinalizeBatchV3Type contains data of event of FinalizeBatchType v3 - FinalizeBatchV3Type ) +// Entry represents a single DA event (commit, revert, finalize). type Entry interface { Type() Type BatchIndex() uint64 @@ -37,6 +36,7 @@ type EntryWithBlocks interface { type Entries []Entry +// PartialHeader represents a partial header (from DA) of a block. type PartialHeader struct { Number uint64 Time uint64 @@ -57,6 +57,7 @@ func (h *PartialHeader) ToHeader() *types.Header { } } +// PartialBlock represents a partial block (from DA). type PartialBlock struct { PartialHeader *PartialHeader Transactions types.Transactions diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index 71dabdd020f1..9ff1b992c3ab 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -7,6 +7,7 @@ import ( "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" ) +// DAQueue is a pipeline stage that reads DA entries from a DataSource and provides them to the next stage. type DAQueue struct { l1height uint64 dataSourceFactory *DataSourceFactory diff --git a/rollup/da_syncer/da_syncer.go b/rollup/da_syncer/da_syncer.go index 1ca0d11ee23b..43926e97d0bf 100644 --- a/rollup/da_syncer/da_syncer.go +++ b/rollup/da_syncer/da_syncer.go @@ -23,6 +23,7 @@ func NewDASyncer(blockchain *core.BlockChain) *DASyncer { } } +// SyncOneBlock receives a PartialBlock, makes sure it's the next block in the chain, executes it and inserts it to the blockchain. func (s *DASyncer) SyncOneBlock(block *da.PartialBlock) error { parentBlock := s.blockchain.CurrentBlock() // we expect blocks to be consecutive. block.PartialHeader.Number == parentBlock.Number+1. diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index 9d639704700e..fb8b2e56d5d8 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -26,6 +26,8 @@ type Config struct { BlobSource blob_client.BlobSource // blob source } +// SyncingPipeline is a derivation pipeline for syncing data from L1 and DA and transform it into +// L2 blocks and chain. type SyncingPipeline struct { ctx context.Context cancel context.CancelFunc @@ -149,6 +151,7 @@ func (s *SyncingPipeline) mainLoop() { case <-stepCh: err := s.Step() if err == nil { + // step succeeded, reset exponential backoff and continue reqStep(false) s.expBackoff.Reset() resetCounter = 0 @@ -156,6 +159,7 @@ func (s *SyncingPipeline) mainLoop() { } if errors.Is(err, io.EOF) { + // pipeline is empty, request a delayed step reqStep(true) continue } From 485c2290a2fc47eaaf2540b0914ce133e49f2d2c Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Wed, 7 Aug 2024 12:26:30 +0200 Subject: [PATCH 52/59] cycle over list of blob clients (#960) * cycle over list of blob clients * change config, remowe sleep, add return error * refactor: flags and config * refactor: simplify GetBlobByVersionedHash in BlobClientList to alternate blob clients --------- Co-authored-by: jonastheis <4181434+jonastheis@users.noreply.github.com> --- cmd/geth/main.go | 3 +- cmd/utils/flags.go | 27 ++++---- eth/ethconfig/config.go | 2 - params/config.go | 16 ----- rollup/da_syncer/blob_client/blob_client.go | 49 -------------- .../da_syncer/blob_client/blob_client_list.go | 65 +++++++++++++++++++ rollup/da_syncer/syncing_pipeline.go | 32 ++++----- 7 files changed, 100 insertions(+), 94 deletions(-) create mode 100644 rollup/da_syncer/blob_client/blob_client_list.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 81d54e127c52..09fa46374c06 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -173,7 +173,8 @@ var ( utils.DASyncEnabledFlag, utils.DAModeFlag, utils.DASnapshotFileFlag, - utils.DABlobSourceFlag, + utils.DABlockNativeAPIEndpointFlag, + utils.DABlobScanAPIEndpointFlag, } rpcFlags = []cli.Flag{ diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 10db538945e5..f3fdbefb7196 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -74,7 +74,6 @@ import ( "github.com/scroll-tech/go-ethereum/p2p/netutil" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/da_syncer" - "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" "github.com/scroll-tech/go-ethereum/rollup/tracing" "github.com/scroll-tech/go-ethereum/rpc" ) @@ -867,7 +866,7 @@ var ( Usage: "peer ids of shadow fork peers", } - // Da syncing settings + // DA syncing settings DASyncEnabledFlag = cli.BoolFlag{ Name: "da.sync", Usage: "Enable node syncing from DA", @@ -875,19 +874,22 @@ var ( defaultDA = ethconfig.Defaults.DA.FetcherMode DAModeFlag = TextMarshalerFlag{ Name: "da.mode", - Usage: `Da sync sync mode ("l1rpc" or "snapshot")`, + Usage: `DA sync mode ("l1rpc" or "snapshot")`, Value: &defaultDA, } - defaultBlobSource = ethconfig.Defaults.DA.BlobSource - DABlobSourceFlag = TextMarshalerFlag{ - Name: "da.blob.source", - Usage: `Blob data source, currently supported "blobscan" or "blocknative"`, - Value: &defaultBlobSource, - } DASnapshotFileFlag = cli.StringFlag{ Name: "da.snapshot.file", Usage: "Snapshot file to sync from da", } + DABlobScanAPIEndpointFlag = cli.StringFlag{ + Name: "da.blob.blobscan", + Usage: "BlobScan blob api endpoint", + Value: ethconfig.Defaults.DA.BlobScanAPIEndpoint, + } + DABlockNativeAPIEndpointFlag = cli.StringFlag{ + Name: "da.blob.blocknative", + Usage: "BlockNative blob api endpoint", + } ) // MakeDataDir retrieves the currently requested data directory, terminating @@ -1617,8 +1619,11 @@ func setDA(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.GlobalIsSet(DASnapshotFileFlag.Name) { cfg.DA.SnapshotFilePath = ctx.GlobalString(DASnapshotFileFlag.Name) } - if ctx.GlobalIsSet(DABlobSourceFlag.Name) { - cfg.DA.BlobSource = *GlobalTextMarshaler(ctx, DABlobSourceFlag.Name).(*blob_client.BlobSource) + if ctx.GlobalIsSet(DABlobScanAPIEndpointFlag.Name) { + cfg.DA.BlobScanAPIEndpoint = ctx.GlobalString(DABlobScanAPIEndpointFlag.Name) + } + if ctx.GlobalIsSet(DABlockNativeAPIEndpointFlag.Name) { + cfg.DA.BlockNativeAPIEndpoint = ctx.GlobalString(DABlockNativeAPIEndpointFlag.Name) } } } diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 5842ab70fd4c..7ce9e720ab5c 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -38,7 +38,6 @@ import ( "github.com/scroll-tech/go-ethereum/node" "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rollup/da_syncer" - "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" ) // FullNodeGPO contains default gasprice oracle settings for full node. @@ -97,7 +96,6 @@ var Defaults = Config{ MaxBlockRange: -1, // Default unconfigured value: no block range limit for backward compatibility DA: da_syncer.Config{ FetcherMode: da_syncer.L1RPC, - BlobSource: blob_client.BlobScan, }, } diff --git a/params/config.go b/params/config.go index d6a1b4c8eea7..1fcd4c748f80 100644 --- a/params/config.go +++ b/params/config.go @@ -336,9 +336,6 @@ var ( NumL1MessagesPerBlock: 10, ScrollChainAddress: common.HexToAddress("0x2D567EcE699Eabe5afCd141eDB7A4f2D0D6ce8a0"), }, - DAConfig: &DAConfig{ - BlobScanAPIEndpoint: "https://api.sepolia.blobscan.com/blobs/", - }, }, } @@ -378,10 +375,6 @@ var ( NumL1MessagesPerBlock: 10, ScrollChainAddress: common.HexToAddress("0xa13BAF47339d63B743e7Da8741db5456DAc1E556"), }, - DAConfig: &DAConfig{ - BlobScanAPIEndpoint: "https://api.blobscan.com/blobs/", - BlockNativeAPIEndpoint: "https://api.ethernow.xyz/v1/blob/", - }, }, } @@ -658,9 +651,6 @@ type ScrollConfig struct { // L1 config L1Config *L1Config `json:"l1Config,omitempty"` - - // DA Config - DAConfig *DAConfig `json:"daConfig,omitempty"` } // L1Config contains the l1 parameters needed to sync l1 contract events (e.g., l1 messages, commit/revert/finalize batches) in the sequencer @@ -671,12 +661,6 @@ type L1Config struct { ScrollChainAddress common.Address `json:"scrollChainAddress,omitempty"` } -// DAConfig contains the parameters to sync from DA -type DAConfig struct { - BlobScanAPIEndpoint string `json:"blobScanApiEndpoint,omitempty"` - BlockNativeAPIEndpoint string `json:"blockNativeApiEndpoint,omitempty"` -} - func (c *L1Config) String() string { if c == nil { return "" diff --git a/rollup/da_syncer/blob_client/blob_client.go b/rollup/da_syncer/blob_client/blob_client.go index fbe20ae803ca..eea486c6fa77 100644 --- a/rollup/da_syncer/blob_client/blob_client.go +++ b/rollup/da_syncer/blob_client/blob_client.go @@ -2,7 +2,6 @@ package blob_client import ( "context" - "fmt" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" @@ -16,51 +15,3 @@ const ( type BlobClient interface { GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) } - -type BlobSource int - -const ( - // BlobScan - BlobScan BlobSource = iota - // BlockNative - BlockNative -) - -func (src BlobSource) IsValid() bool { - return src >= BlobScan && src <= BlockNative -} - -// String implements the stringer interface. -func (src BlobSource) String() string { - switch src { - case BlobScan: - return "blobscan" - case BlockNative: - return "blocknative" - default: - return "unknown" - } -} - -func (src BlobSource) MarshalText() ([]byte, error) { - switch src { - case BlobScan: - return []byte("blobscan"), nil - case BlockNative: - return []byte("blocknative"), nil - default: - return nil, fmt.Errorf("unknown blob source %d", src) - } -} - -func (src *BlobSource) UnmarshalText(text []byte) error { - switch string(text) { - case "blobscan": - *src = BlobScan - case "blocknative": - *src = BlockNative - default: - return fmt.Errorf(`unknown blob source %q, want "blobscan" or "blocknative"`, text) - } - return nil -} diff --git a/rollup/da_syncer/blob_client/blob_client_list.go b/rollup/da_syncer/blob_client/blob_client_list.go new file mode 100644 index 000000000000..b1c11d6d3e4c --- /dev/null +++ b/rollup/da_syncer/blob_client/blob_client_list.go @@ -0,0 +1,65 @@ +package blob_client + +import ( + "context" + "fmt" + "io" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" + "github.com/scroll-tech/go-ethereum/log" +) + +type BlobClientList struct { + list []BlobClient + curPos int +} + +func NewBlobClientList(blobClients ...BlobClient) *BlobClientList { + return &BlobClientList{ + list: blobClients, + curPos: 0, + } +} + +func (c *BlobClientList) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { + if len(c.list) == 0 { + return nil, fmt.Errorf("BlobClientList.GetBlobByVersionedHash: list of BlobClients is empty") + } + + for i := 0; i < len(c.list); i++ { + blob, err := c.list[c.nextPos()].GetBlobByVersionedHash(ctx, versionedHash) + if err != nil { + return blob, nil + } + + // there was an error, try the next blob client in following iteration + log.Warn("BlobClientList: failed to get blob by versioned hash from BlobClient", "err", err, "blob client pos in BlobClientList", c.curPos) + } + + // if we iterated over entire list, return EOF error that will be handled in syncing_pipeline with a backoff and retry + return nil, io.EOF +} + +func (c *BlobClientList) nextPos() int { + c.curPos = (c.curPos + 1) % len(c.list) + return c.curPos +} + +func (c *BlobClientList) AddBlobClient(blobClient BlobClient) { + c.list = append(c.list, blobClient) +} + +func (c *BlobClientList) RemoveBlobClient(blobClient BlobClient) { + c.list = append(c.list, blobClient) + for pos, client := range c.list { + if client == blobClient { + c.list = append(c.list[:pos], c.list[pos+1:]...) + c.curPos %= len(c.list) + return + } + } +} +func (c *BlobClientList) Size() int { + return len(c.list) +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index fb8b2e56d5d8..fea754a01367 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -21,9 +21,10 @@ import ( // Config is the configuration parameters of data availability syncing. type Config struct { - FetcherMode FetcherMode // mode of fetcher - SnapshotFilePath string // path to snapshot file - BlobSource blob_client.BlobSource // blob source + FetcherMode FetcherMode // mode of fetcher + SnapshotFilePath string // path to snapshot file + BlobScanAPIEndpoint string // BlobScan blob api endpoint + BlockNativeAPIEndpoint string // BlockNative blob api endpoint } // SyncingPipeline is a derivation pipeline for syncing data from L1 and DA and transform it into @@ -53,17 +54,18 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi return nil, err } - var blobClient blob_client.BlobClient - switch config.BlobSource { - case blob_client.BlobScan: - blobClient = blob_client.NewBlobScanClient(genesisConfig.Scroll.DAConfig.BlobScanAPIEndpoint) - case blob_client.BlockNative: - blobClient = blob_client.NewBlockNativeClient(genesisConfig.Scroll.DAConfig.BlockNativeAPIEndpoint) - default: - return nil, fmt.Errorf("unknown blob scan client: %d", config.BlobSource) + blobClientList := blob_client.NewBlobClientList() + if config.BlobScanAPIEndpoint != "" { + blobClientList.AddBlobClient(blob_client.NewBlobScanClient(config.BlobScanAPIEndpoint)) + } + if config.BlockNativeAPIEndpoint != "" { + blobClientList.AddBlobClient(blob_client.NewBlockNativeClient(config.BlockNativeAPIEndpoint)) + } + if blobClientList.Size() == 0 { + 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, l1Client, blobClient, db) + dataSourceFactory := NewDataSourceFactory(blockchain, genesisConfig, config, l1Client, blobClientList, db) syncedL1Height := l1DeploymentBlock - 1 from := rawdb.ReadDASyncedL1BlockNumber(db) if from != nil { @@ -99,7 +101,7 @@ func (s *SyncingPipeline) Step() error { } func (s *SyncingPipeline) Start() { - log.Info("Starting SyncingPipeline") + log.Info("sync from DA: starting pipeline") s.wg.Add(1) go func() { @@ -189,10 +191,10 @@ func (s *SyncingPipeline) mainLoop() { } func (s *SyncingPipeline) Stop() { - log.Info("Stopping DaSyncer...") + log.Info("sync from DA: stopping pipeline...") s.cancel() s.wg.Wait() - log.Info("Stopped DaSyncer... Done") + log.Info("sync from DA: stopping pipeline... done") } func (s *SyncingPipeline) reset(resetCounter int) { From 399dcfc1eadc5c1633f25c2c7dc7a7e9b9bb7452 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Tue, 13 Aug 2024 12:31:26 +0800 Subject: [PATCH 53/59] feat: introduce custom errors and mark RPC related errors as temporary so that they can be retried in pipeline --- .../da_syncer/blob_client/blob_client_list.go | 7 ++- rollup/da_syncer/block_queue.go | 5 +- rollup/da_syncer/da/calldata_blob_source.go | 19 +++--- rollup/da_syncer/da/commitV0.go | 11 ++-- rollup/da_syncer/da/da.go | 2 +- rollup/da_syncer/da_queue.go | 5 +- rollup/da_syncer/serrors/errors.go | 62 +++++++++++++++++++ rollup/da_syncer/syncing_pipeline.go | 28 ++++++--- rollup/rollup_sync_service/l1client.go | 2 +- 9 files changed, 111 insertions(+), 30 deletions(-) create mode 100644 rollup/da_syncer/serrors/errors.go diff --git a/rollup/da_syncer/blob_client/blob_client_list.go b/rollup/da_syncer/blob_client/blob_client_list.go index b1c11d6d3e4c..da032c4f972b 100644 --- a/rollup/da_syncer/blob_client/blob_client_list.go +++ b/rollup/da_syncer/blob_client/blob_client_list.go @@ -2,12 +2,13 @@ package blob_client import ( "context" + "errors" "fmt" - "io" "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/crypto/kzg4844" "github.com/scroll-tech/go-ethereum/log" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/serrors" ) type BlobClientList struct { @@ -37,8 +38,8 @@ func (c *BlobClientList) GetBlobByVersionedHash(ctx context.Context, versionedHa log.Warn("BlobClientList: failed to get blob by versioned hash from BlobClient", "err", err, "blob client pos in BlobClientList", c.curPos) } - // if we iterated over entire list, return EOF error that will be handled in syncing_pipeline with a backoff and retry - return nil, io.EOF + // if we iterated over entire list, return a temporary error that will be handled in syncing_pipeline with a backoff and retry + return nil, serrors.NewTemporaryError(errors.New("BlobClientList.GetBlobByVersionedHash: failed to get blob by versioned hash from all BlobClients")) } func (c *BlobClientList) nextPos() int { diff --git a/rollup/da_syncer/block_queue.go b/rollup/da_syncer/block_queue.go index 3c3e68bdcf79..a122d41ab356 100644 --- a/rollup/da_syncer/block_queue.go +++ b/rollup/da_syncer/block_queue.go @@ -45,10 +45,7 @@ func (bq *BlockQueue) getBlocksFromBatch(ctx context.Context) error { return fmt.Errorf("unexpected type of daEntry: %T", daEntry) } - bq.blocks, err = entryWithBlocks.Blocks() - if err != nil { - return fmt.Errorf("failed to get blocks from daEntry: %w", err) - } + bq.blocks = entryWithBlocks.Blocks() return nil } diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go index 19fd1b5eccae..5e973f7774a8 100644 --- a/rollup/da_syncer/da/calldata_blob_source.go +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -11,6 +11,7 @@ import ( "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/log" "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/rollup_sync_service" ) @@ -72,7 +73,7 @@ func (ds *CalldataBlobSource) NextData() (Entries, error) { if to > ds.l1Finalized { ds.l1Finalized, err = ds.l1Client.GetLatestFinalizedBlockNumber() if err != nil { - return nil, fmt.Errorf("failed to query GetLatestFinalizedBlockNumber, error: %v", err) + return nil, serrors.NewTemporaryError(fmt.Errorf("failed to query GetLatestFinalizedBlockNumber, error: %v", err)) } // make sure we don't request more than finalized blocks to = min(to, ds.l1Finalized) @@ -84,13 +85,15 @@ func (ds *CalldataBlobSource) NextData() (Entries, error) { logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) if err != nil { - return nil, fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err) + return nil, serrors.NewTemporaryError(fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err)) } da, err := ds.processLogsToDA(logs) - if err == nil { - ds.l1height = to + 1 + if err != nil { + return nil, serrors.NewTemporaryError(fmt.Errorf("failed to process logs to DA, error: %v", err)) } - return da, err + + ds.l1height = to + 1 + return da, nil } func (ds *CalldataBlobSource) L1Height() uint64 { @@ -119,7 +122,7 @@ func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (Entries, error) case ds.l1RevertBatchEventSignature: event := &rollup_sync_service.L1RevertBatchEvent{} - if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, revertBatchEventName, vLog); err != nil { + if err = rollup_sync_service.UnpackLog(ds.scrollChainABI, event, revertBatchEventName, vLog); err != nil { return nil, fmt.Errorf("failed to unpack revert rollup event log, err: %w", err) } @@ -129,7 +132,7 @@ func (ds *CalldataBlobSource) processLogsToDA(logs []types.Log) (Entries, error) case ds.l1FinalizeBatchEventSignature: event := &rollup_sync_service.L1FinalizeBatchEvent{} - if err := rollup_sync_service.UnpackLog(ds.scrollChainABI, event, finalizeBatchEventName, vLog); err != nil { + if err = rollup_sync_service.UnpackLog(ds.scrollChainABI, event, finalizeBatchEventName, vLog); err != nil { return nil, fmt.Errorf("failed to unpack finalized rollup event log, err: %w", err) } @@ -188,7 +191,7 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, vLog *types.Lo txData, err := ds.l1Client.FetchTxData(vLog) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to fetch tx data, tx hash: %v, err: %w", vLog.TxHash.Hex(), err) } if len(txData) < methodIDLength { return nil, fmt.Errorf("transaction data is too short, length of tx data: %v, minimum length required: %v", len(txData), methodIDLength) diff --git a/rollup/da_syncer/da/commitV0.go b/rollup/da_syncer/da/commitV0.go index abedfd3231d5..66a13786c9cb 100644 --- a/rollup/da_syncer/da/commitV0.go +++ b/rollup/da_syncer/da/commitV0.go @@ -3,7 +3,6 @@ package da import ( "encoding/binary" "fmt" - "io" "github.com/scroll-tech/da-codec/encoding" "github.com/scroll-tech/da-codec/encoding/codecv0" @@ -11,6 +10,7 @@ import ( "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" ) type CommitBatchDAV0 struct { @@ -92,7 +92,7 @@ func (c *CommitBatchDAV0) CompareTo(other Entry) int { return 0 } -func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { +func (c *CommitBatchDAV0) Blocks() []*PartialBlock { var blocks []*PartialBlock l1TxPointer := 0 @@ -125,7 +125,8 @@ func (c *CommitBatchDAV0) Blocks() ([]*PartialBlock, error) { blocks = append(blocks, block) } } - return blocks, nil + + return blocks } func getTotalMessagesPoppedFromChunks(decodedChunks []*codecv0.DAChunkRawTx) int { @@ -156,8 +157,8 @@ func getL1Messages(db ethdb.Database, parentTotalL1MessagePopped uint64, skipped l1Tx := rawdb.ReadL1Message(db, currentIndex) if l1Tx == nil { // message not yet available - // we return io.EOF as this will be handled in the syncing pipeline with a backoff and retry - return nil, io.EOF + // we return serrors.EOFError as this will be handled in the syncing pipeline with a backoff and retry + return nil, serrors.EOFError } txs = append(txs, l1Tx) currentIndex++ diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index 935f09cfda7c..5a55a4fbb513 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -31,7 +31,7 @@ type Entry interface { type EntryWithBlocks interface { Entry - Blocks() ([]*PartialBlock, error) + Blocks() []*PartialBlock } type Entries []Entry diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go index 9ff1b992c3ab..64673a4a646b 100644 --- a/rollup/da_syncer/da_queue.go +++ b/rollup/da_syncer/da_queue.go @@ -5,6 +5,7 @@ import ( "errors" "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/serrors" ) // DAQueue is a pipeline stage that reads DA entries from a DataSource and provides them to the next stage. @@ -54,7 +55,9 @@ func (dq *DAQueue) getNextData(ctx context.Context) error { if errors.Is(err, da.ErrSourceExhausted) { dq.l1height = dq.dataSource.L1Height() dq.dataSource = nil - return dq.getNextData(ctx) + + // we return EOFError to be handled in pipeline + return serrors.EOFError } return err diff --git a/rollup/da_syncer/serrors/errors.go b/rollup/da_syncer/serrors/errors.go new file mode 100644 index 000000000000..aa0426f0771d --- /dev/null +++ b/rollup/da_syncer/serrors/errors.go @@ -0,0 +1,62 @@ +package serrors + +import ( + "fmt" +) + +const ( + temporary Type = iota + eof +) + +var ( + TemporaryError = NewTemporaryError(nil) + EOFError = NewEOFError(nil) +) + +type Type uint8 + +func (t Type) String() string { + switch t { + case temporary: + return "temporary" + case eof: + return "EOF" + default: + return "unknown" + } +} + +type syncError struct { + t Type + err error +} + +func NewTemporaryError(err error) error { + return &syncError{t: temporary, err: err} +} + +func NewEOFError(err error) error { + return &syncError{t: eof, err: err} +} + +func (s *syncError) Error() string { + return fmt.Sprintf("%s: %v", s.t, s.err) +} + +func (s *syncError) Unwrap() error { + return s.err +} + +func (s *syncError) Is(target error) bool { + if target == nil { + return s == nil + } + + targetSyncErr, ok := target.(*syncError) + if !ok { + return false + } + + return s.t == targetSyncErr.t +} diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index fea754a01367..ef66211fe5ee 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "io" "sync" "time" @@ -15,6 +14,7 @@ import ( "github.com/scroll-tech/go-ethereum/log" "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/serrors" "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" "github.com/scroll-tech/go-ethereum/rollup/sync_service" ) @@ -114,6 +114,7 @@ func (s *SyncingPipeline) mainLoop() { stepCh := make(chan struct{}, 1) var delayedStepCh <-chan time.Time var resetCounter int + var tempErrorCounter int // 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. @@ -157,18 +158,32 @@ func (s *SyncingPipeline) mainLoop() { reqStep(false) s.expBackoff.Reset() resetCounter = 0 + tempErrorCounter = 0 continue } - if errors.Is(err, io.EOF) { + if errors.Is(err, serrors.EOFError) { // pipeline is empty, request a delayed step + // TODO: eventually (with state manager) this should not trigger a delayed step because external events will trigger a new step anyway reqStep(true) + tempErrorCounter = 0 continue - } - if errors.Is(err, ErrBlockTooLow) { + } else if errors.Is(err, serrors.TemporaryError) { + log.Warn("syncing pipeline step failed due to temporary error, retrying", "err", err) + if tempErrorCounter > 100 { + log.Warn("syncing pipeline step failed due to 100 consecutive temporary errors, stopping pipeline worker", "last err", err) + return + } + + // temporary error, request a delayed step + reqStep(true) + tempErrorCounter++ + continue + } else if errors.Is(err, ErrBlockTooLow) { // block number returned by the block queue is too low, // we skip the blocks until we reach the correct block number again. reqStep(false) + tempErrorCounter = 0 continue } else if errors.Is(err, ErrBlockTooHigh) { // block number returned by the block queue is too high, @@ -176,10 +191,9 @@ func (s *SyncingPipeline) mainLoop() { s.reset(resetCounter) resetCounter++ reqStep(false) + tempErrorCounter = 0 continue - } - - if errors.Is(err, context.Canceled) { + } else if errors.Is(err, context.Canceled) { log.Info("syncing pipeline stopped due to cancelled context", "err", err) return } diff --git a/rollup/rollup_sync_service/l1client.go b/rollup/rollup_sync_service/l1client.go index 2cd5efc625d5..d2a28f659f10 100644 --- a/rollup/rollup_sync_service/l1client.go +++ b/rollup/rollup_sync_service/l1client.go @@ -55,7 +55,7 @@ func NewL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId return &client, nil } -// fetcRollupEventsInRange retrieves and parses commit/revert/finalize rollup events between block numbers: [from, to]. +// FetchRollupEventsInRange retrieves and parses commit/revert/finalize rollup events between block numbers: [from, to]. func (c *L1Client) FetchRollupEventsInRange(from, to uint64) ([]types.Log, error) { log.Trace("L1Client fetchRollupEventsInRange", "fromBlock", from, "toBlock", to) From 9e1769f48c2c96c65b9a04fc5b1faf2178c8af76 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Thu, 15 Aug 2024 13:18:04 +0200 Subject: [PATCH 54/59] fix blob_client_list --- rollup/da_syncer/blob_client/blob_client_list.go | 2 +- rollup/da_syncer/da/commitV1.go | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/rollup/da_syncer/blob_client/blob_client_list.go b/rollup/da_syncer/blob_client/blob_client_list.go index da032c4f972b..13e48b221f7f 100644 --- a/rollup/da_syncer/blob_client/blob_client_list.go +++ b/rollup/da_syncer/blob_client/blob_client_list.go @@ -30,7 +30,7 @@ func (c *BlobClientList) GetBlobByVersionedHash(ctx context.Context, versionedHa for i := 0; i < len(c.list); i++ { blob, err := c.list[c.nextPos()].GetBlobByVersionedHash(ctx, versionedHash) - if err != nil { + if err == nil { return blob, nil } diff --git a/rollup/da_syncer/da/commitV1.go b/rollup/da_syncer/da/commitV1.go index 8c45664e7979..19512972928c 100644 --- a/rollup/da_syncer/da/commitV1.go +++ b/rollup/da_syncer/da/commitV1.go @@ -59,6 +59,9 @@ func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, db ethdb.Database if err != nil { return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) } + if blob == nil { + return nil, fmt.Errorf("unexpected, blob == nil and err != nil, batch index: %d, versionedHash: %s, blobClient: %T", batchIndex, versionedHash.String(), blobClient) + } // compute blob versioned hash and compare with one from tx c, err := kzg4844.BlobToCommitment(blob) From 06499542d51fb59817b390937000ef751fa20349 Mon Sep 17 00:00:00 2001 From: NazariiDenha Date: Thu, 15 Aug 2024 11:18:44 +0000 Subject: [PATCH 55/59] =?UTF-8?q?chore:=20auto=20version=20bump=E2=80=89[b?= =?UTF-8?q?ot]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- params/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/params/version.go b/params/version.go index 77bb908849c5..0d8aacfff069 100644 --- a/params/version.go +++ b/params/version.go @@ -24,7 +24,7 @@ import ( const ( VersionMajor = 5 // Major version component of the current release VersionMinor = 6 // Minor version component of the current release - VersionPatch = 1 // Patch version component of the current release + VersionPatch = 2 // Patch version component of the current release VersionMeta = "mainnet" // Version metadata to append to the version string ) From 0b2fe3ba2a27bd7d775efee96e5b0bcf49169c17 Mon Sep 17 00:00:00 2001 From: jonastheis <4181434+jonastheis@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:05:54 +0800 Subject: [PATCH 56/59] feat: enable prefetching in BuildAndWriteBlock --- core/blockchain.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/blockchain.go b/core/blockchain.go index 9c2ff37703a3..4a8080dbc8e2 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1814,6 +1814,9 @@ func (bc *BlockChain) BuildAndWriteBlock(parentBlock *types.Block, header *types return NonStatTy, err } + statedb.StartPrefetcher("l1sync") + defer statedb.StopPrefetcher() + header.ParentHash = parentBlock.Hash() tempBlock := types.NewBlockWithHeader(header).WithBody(txs, nil) From 9e3c8381aa1dc22f57ef1afc67df9d35a8ce54c5 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 19 Aug 2024 11:01:08 +0200 Subject: [PATCH 57/59] (follower_node)support beacon node client as blob provider (#988) * support beacon node client as blob provider * fix * fix formatting * use url.JoinPath instead of path * don't move pos each time in blob_client_list --------- Co-authored-by: jonastheis <4181434+jonastheis@users.noreply.github.com> --- cmd/geth/main.go | 1 + cmd/utils/flags.go | 14 +- .../blob_client/beacon_node_client.go | 183 ++++++++++++++++++ rollup/da_syncer/blob_client/blob_client.go | 6 +- .../da_syncer/blob_client/blob_client_list.go | 9 +- .../da_syncer/blob_client/blob_scan_client.go | 42 +--- .../blob_client/block_native_client.go | 11 +- rollup/da_syncer/da/commitV1.go | 2 +- rollup/da_syncer/syncing_pipeline.go | 9 + rollup/rollup_sync_service/l1client.go | 17 +- 10 files changed, 231 insertions(+), 63 deletions(-) create mode 100644 rollup/da_syncer/blob_client/beacon_node_client.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 09fa46374c06..1ec163d97aea 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -175,6 +175,7 @@ var ( utils.DASnapshotFileFlag, utils.DABlockNativeAPIEndpointFlag, utils.DABlobScanAPIEndpointFlag, + utils.DABeaconNodeAPIEndpointFlag, } rpcFlags = []cli.Flag{ diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index f3fdbefb7196..d9be3622a0c5 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -879,16 +879,19 @@ var ( } DASnapshotFileFlag = cli.StringFlag{ Name: "da.snapshot.file", - Usage: "Snapshot file to sync from da", + Usage: "Snapshot file to sync from DA", } DABlobScanAPIEndpointFlag = cli.StringFlag{ Name: "da.blob.blobscan", - Usage: "BlobScan blob api endpoint", - Value: ethconfig.Defaults.DA.BlobScanAPIEndpoint, + Usage: "BlobScan blob API endpoint", } DABlockNativeAPIEndpointFlag = cli.StringFlag{ Name: "da.blob.blocknative", - Usage: "BlockNative blob api endpoint", + Usage: "BlockNative blob API endpoint", + } + DABeaconNodeAPIEndpointFlag = cli.StringFlag{ + Name: "da.blob.beaconnode", + Usage: "Beacon node API endpoint", } ) @@ -1625,6 +1628,9 @@ func setDA(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.GlobalIsSet(DABlockNativeAPIEndpointFlag.Name) { cfg.DA.BlockNativeAPIEndpoint = ctx.GlobalString(DABlockNativeAPIEndpointFlag.Name) } + if ctx.GlobalIsSet(DABeaconNodeAPIEndpointFlag.Name) { + cfg.DA.BeaconNodeAPIEndpoint = ctx.GlobalString(DABeaconNodeAPIEndpointFlag.Name) + } } } diff --git a/rollup/da_syncer/blob_client/beacon_node_client.go b/rollup/da_syncer/blob_client/beacon_node_client.go new file mode 100644 index 000000000000..20e4d876a489 --- /dev/null +++ b/rollup/da_syncer/blob_client/beacon_node_client.go @@ -0,0 +1,183 @@ +package blob_client + +import ( + "context" + "crypto/sha256" + "encoding/json" + "fmt" + "io" + "net/http" + "net/url" + "strconv" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" + "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" +) + +type BeaconNodeClient struct { + apiEndpoint string + l1Client *rollup_sync_service.L1Client + genesisTime uint64 + secondsPerSlot uint64 +} + +var ( + beaconNodeGenesisEndpoint = "/eth/v1/beacon/genesis" + beaconNodeSpecEndpoint = "/eth/v1/config/spec" + beaconNodeBlobEndpoint = "/eth/v1/beacon/blob_sidecars" +) + +func NewBeaconNodeClient(apiEndpoint string, l1Client *rollup_sync_service.L1Client) (*BeaconNodeClient, error) { + // get genesis time + genesisPath, err := url.JoinPath(apiEndpoint, beaconNodeGenesisEndpoint) + if err != nil { + return nil, fmt.Errorf("failed to join path, err: %w", err) + } + resp, err := http.Get(genesisPath) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %w", err) + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { + body, _ := io.ReadAll(resp.Body) + bodyStr := string(body) + return nil, fmt.Errorf("beacon node request failed, status: %s, body: %s", resp.Status, bodyStr) + } + + var genesisResp GenesisResp + err = json.NewDecoder(resp.Body).Decode(&genesisResp) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + genesisTime, err := strconv.ParseUint(genesisResp.Data.GenesisTime, 10, 64) + if err != nil { + return nil, fmt.Errorf("failed to decode genesis time %s, err: %w", genesisResp.Data.GenesisTime, err) + } + + // get seconds per slot from spec + specPath, err := url.JoinPath(apiEndpoint, beaconNodeSpecEndpoint) + if err != nil { + return nil, fmt.Errorf("failed to join path, err: %w", err) + } + resp, err = http.Get(specPath) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %w", err) + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { + body, _ := io.ReadAll(resp.Body) + bodyStr := string(body) + return nil, fmt.Errorf("beacon node request failed, status: %s, body: %s", resp.Status, bodyStr) + } + + var specResp SpecResp + err = json.NewDecoder(resp.Body).Decode(&specResp) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + secondsPerSlot, err := strconv.ParseUint(specResp.Data.SecondsPerSlot, 10, 64) + if err != nil { + return nil, fmt.Errorf("failed to decode seconds per slot %s, err: %w", specResp.Data.SecondsPerSlot, err) + } + if secondsPerSlot == 0 { + return nil, fmt.Errorf("failed to make new BeaconNodeClient, secondsPerSlot is 0") + } + + return &BeaconNodeClient{ + apiEndpoint: apiEndpoint, + l1Client: l1Client, + genesisTime: genesisTime, + secondsPerSlot: secondsPerSlot, + }, nil +} + +func (c *BeaconNodeClient) GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) { + // get block timestamp to calculate slot + header, err := c.l1Client.GetHeaderByNumber(blockNumber) + if err != nil { + return nil, fmt.Errorf("failed to get header by number, err: %w", err) + } + slot := (header.Time - c.genesisTime) / c.secondsPerSlot + + // get blob sidecar for slot + blobSidecarPath, err := url.JoinPath(c.apiEndpoint, beaconNodeBlobEndpoint, fmt.Sprintf("%d", slot)) + if err != nil { + return nil, fmt.Errorf("failed to join path, err: %w", err) + } + resp, err := http.Get(blobSidecarPath) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %w", err) + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { + body, _ := io.ReadAll(resp.Body) + bodyStr := string(body) + return nil, fmt.Errorf("beacon node request failed, status: %s, body: %s", resp.Status, bodyStr) + } + + var blobSidecarResp BlobSidecarResp + err = json.NewDecoder(resp.Body).Decode(&blobSidecarResp) + if err != nil { + return nil, fmt.Errorf("failed to decode result into struct, err: %w", err) + } + + // find blob with desired versionedHash + for _, blob := range blobSidecarResp.Data { + // calculate blob hash from commitment and check it with desired + commitmentBytes := common.FromHex(blob.KzgCommitment) + if len(commitmentBytes) != lenKZGCommitment { + return nil, fmt.Errorf("len of kzg commitment is not correct, expected: %d, got: %d", lenKZGCommitment, len(commitmentBytes)) + } + commitment := kzg4844.Commitment(commitmentBytes) + blobVersionedHash := kzg4844.CalcBlobHashV1(sha256.New(), &commitment) + + if blobVersionedHash == versionedHash { + // found desired blob + blobBytes := common.FromHex(blob.Blob) + if len(blobBytes) != lenBlobBytes { + return nil, fmt.Errorf("len of blob data is not correct, expected: %d, got: %d", lenBlobBytes, len(blobBytes)) + } + + b := kzg4844.Blob(blobBytes) + return &b, nil + } + } + + return nil, fmt.Errorf("missing blob %v in slot %d, block number %d", versionedHash, slot, blockNumber) +} + +type GenesisResp struct { + Data struct { + GenesisTime string `json:"genesis_time"` + } `json:"data"` +} + +type SpecResp struct { + Data struct { + SecondsPerSlot string `json:"SECONDS_PER_SLOT"` + } `json:"data"` +} + +type BlobSidecarResp struct { + Data []struct { + Index string `json:"index"` + Blob string `json:"blob"` + KzgCommitment string `json:"kzg_commitment"` + KzgProof string `json:"kzg_proof"` + SignedBlockHeader struct { + Message struct { + Slot string `json:"slot"` + ProposerIndex string `json:"proposer_index"` + ParentRoot string `json:"parent_root"` + StateRoot string `json:"state_root"` + BodyRoot string `json:"body_root"` + } `json:"message"` + Signature string `json:"signature"` + } `json:"signed_block_header"` + KzgCommitmentInclusionProof []string `json:"kzg_commitment_inclusion_proof"` + } `json:"data"` +} diff --git a/rollup/da_syncer/blob_client/blob_client.go b/rollup/da_syncer/blob_client/blob_client.go index eea486c6fa77..5ce0d16a42cc 100644 --- a/rollup/da_syncer/blob_client/blob_client.go +++ b/rollup/da_syncer/blob_client/blob_client.go @@ -8,10 +8,10 @@ import ( ) const ( - okStatusCode int = 200 - lenBlobBytes int = 131072 + lenBlobBytes int = 131072 + lenKZGCommitment int = 48 ) type BlobClient interface { - GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) + GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) } diff --git a/rollup/da_syncer/blob_client/blob_client_list.go b/rollup/da_syncer/blob_client/blob_client_list.go index 13e48b221f7f..647f5f78ff71 100644 --- a/rollup/da_syncer/blob_client/blob_client_list.go +++ b/rollup/da_syncer/blob_client/blob_client_list.go @@ -23,17 +23,17 @@ func NewBlobClientList(blobClients ...BlobClient) *BlobClientList { } } -func (c *BlobClientList) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { +func (c *BlobClientList) GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) { if len(c.list) == 0 { return nil, fmt.Errorf("BlobClientList.GetBlobByVersionedHash: list of BlobClients is empty") } for i := 0; i < len(c.list); i++ { - blob, err := c.list[c.nextPos()].GetBlobByVersionedHash(ctx, versionedHash) + blob, err := c.list[c.curPos].GetBlobByVersionedHashAndBlockNumber(ctx, versionedHash, blockNumber) if err == nil { return blob, nil } - + c.nextPos() // there was an error, try the next blob client in following iteration log.Warn("BlobClientList: failed to get blob by versioned hash from BlobClient", "err", err, "blob client pos in BlobClientList", c.curPos) } @@ -42,9 +42,8 @@ func (c *BlobClientList) GetBlobByVersionedHash(ctx context.Context, versionedHa return nil, serrors.NewTemporaryError(errors.New("BlobClientList.GetBlobByVersionedHash: failed to get blob by versioned hash from all BlobClients")) } -func (c *BlobClientList) nextPos() int { +func (c *BlobClientList) nextPos() { c.curPos = (c.curPos + 1) % len(c.list) - return c.curPos } func (c *BlobClientList) AddBlobClient(blobClient BlobClient) { diff --git a/rollup/da_syncer/blob_client/blob_scan_client.go b/rollup/da_syncer/blob_client/blob_scan_client.go index 98b190c2214f..49a32d6d8191 100644 --- a/rollup/da_syncer/blob_client/blob_scan_client.go +++ b/rollup/da_syncer/blob_client/blob_scan_client.go @@ -24,7 +24,7 @@ func NewBlobScanClient(apiEndpoint string) *BlobScanClient { } } -func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { +func (c *BlobScanClient) GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) { // blobscan api docs https://api.blobscan.com/#/blobs/blob-getByBlobId path, err := url.JoinPath(c.apiEndpoint, versionedHash.String()) if err != nil { @@ -40,8 +40,8 @@ func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHa return nil, fmt.Errorf("cannot do request, err: %w", err) } defer resp.Body.Close() - if resp.StatusCode != okStatusCode { - if resp.StatusCode == 404 { + if resp.StatusCode != http.StatusOK { + if resp.StatusCode == http.StatusNotFound { return nil, fmt.Errorf("no blob with versioned hash : %s", versionedHash.String()) } var res ErrorRespBlobScan @@ -69,44 +69,10 @@ func (c *BlobScanClient) GetBlobByVersionedHash(ctx context.Context, versionedHa } type BlobRespBlobScan struct { - Commitment string `json:"commitment"` - Proof string `json:"proof"` - Size int `json:"size"` - VersionedHash string `json:"versionedHash"` - Data string `json:"data"` - DataStorageReferences []struct { - BlobStorage string `json:"blobStorage"` - DataReference string `json:"dataReference"` - } `json:"dataStorageReferences"` - Transactions []struct { - Hash string `json:"hash"` - Index int `json:"index"` - Block struct { - Number int `json:"number"` - BlobGasUsed string `json:"blobGasUsed"` - BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` - BlobGasPrice string `json:"blobGasPrice"` - ExcessBlobGas string `json:"excessBlobGas"` - Hash string `json:"hash"` - Timestamp string `json:"timestamp"` - Slot int `json:"slot"` - } `json:"block"` - From string `json:"from"` - To string `json:"to"` - MaxFeePerBlobGas string `json:"maxFeePerBlobGas"` - BlobAsCalldataGasUsed string `json:"blobAsCalldataGasUsed"` - Rollup string `json:"rollup"` - BlobAsCalldataGasFee string `json:"blobAsCalldataGasFee"` - BlobGasBaseFee string `json:"blobGasBaseFee"` - BlobGasMaxFee string `json:"blobGasMaxFee"` - BlobGasUsed string `json:"blobGasUsed"` - } `json:"transactions"` + Data string `json:"data"` } type ErrorRespBlobScan struct { Message string `json:"message"` Code string `json:"code"` - Issues []struct { - Message string `json:"message"` - } `json:"issues"` } diff --git a/rollup/da_syncer/blob_client/block_native_client.go b/rollup/da_syncer/blob_client/block_native_client.go index ef3eebaa2972..32ab290728f7 100644 --- a/rollup/da_syncer/blob_client/block_native_client.go +++ b/rollup/da_syncer/blob_client/block_native_client.go @@ -22,7 +22,7 @@ func NewBlockNativeClient(apiEndpoint string) *BlockNativeClient { } } -func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versionedHash common.Hash) (*kzg4844.Blob, error) { +func (c *BlockNativeClient) GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) { // blocknative api docs https://docs.blocknative.com/blocknative-data-archive/blob-archive path, err := url.JoinPath(c.apiEndpoint, versionedHash.String()) if err != nil { @@ -33,7 +33,7 @@ func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versione return nil, fmt.Errorf("cannot do request, err: %w", err) } defer resp.Body.Close() - if resp.StatusCode != okStatusCode { + if resp.StatusCode != http.StatusOK { var res ErrorRespBlockNative err = json.NewDecoder(resp.Body).Decode(&res) if err != nil { @@ -59,12 +59,7 @@ func (c *BlockNativeClient) GetBlobByVersionedHash(ctx context.Context, versione type BlobRespBlockNative struct { Blob struct { - VersionedHash string `json:"versionedHash"` - Commitment string `json:"commitment"` - Proof string `json:"proof"` - ZeroBytes int `json:"zeroBytes"` - NonZeroBytes int `json:"nonZeroBytes"` - Data string `json:"data"` + Data string `json:"data"` } `json:"blob"` } diff --git a/rollup/da_syncer/da/commitV1.go b/rollup/da_syncer/da/commitV1.go index 19512972928c..d94a046c81df 100644 --- a/rollup/da_syncer/da/commitV1.go +++ b/rollup/da_syncer/da/commitV1.go @@ -55,7 +55,7 @@ func NewCommitBatchDAV1WithBlobDecodeFunc(ctx context.Context, db ethdb.Database return nil, fmt.Errorf("failed to fetch blob hash, err: %w", err) } - blob, err := blobClient.GetBlobByVersionedHash(ctx, versionedHash) + blob, err := blobClient.GetBlobByVersionedHashAndBlockNumber(ctx, versionedHash, vLog.BlockNumber) if err != nil { return nil, fmt.Errorf("failed to fetch blob from blob client, err: %w", err) } diff --git a/rollup/da_syncer/syncing_pipeline.go b/rollup/da_syncer/syncing_pipeline.go index ef66211fe5ee..c99a49210ac1 100644 --- a/rollup/da_syncer/syncing_pipeline.go +++ b/rollup/da_syncer/syncing_pipeline.go @@ -25,6 +25,7 @@ type Config struct { SnapshotFilePath string // path to snapshot file BlobScanAPIEndpoint string // BlobScan blob api endpoint BlockNativeAPIEndpoint string // BlockNative blob api endpoint + BeaconNodeAPIEndpoint string // Beacon node api endpoint } // SyncingPipeline is a derivation pipeline for syncing data from L1 and DA and transform it into @@ -55,6 +56,14 @@ func NewSyncingPipeline(ctx context.Context, blockchain *core.BlockChain, genesi } blobClientList := blob_client.NewBlobClientList() + if config.BeaconNodeAPIEndpoint != "" { + beaconNodeClient, err := blob_client.NewBeaconNodeClient(config.BeaconNodeAPIEndpoint, l1Client) + if err != nil { + log.Warn("failed to create BeaconNodeClient", "err", err) + } else { + blobClientList.AddBlobClient(beaconNodeClient) + } + } if config.BlobScanAPIEndpoint != "" { blobClientList.AddBlobClient(blob_client.NewBlobScanClient(config.BlobScanAPIEndpoint)) } diff --git a/rollup/rollup_sync_service/l1client.go b/rollup/rollup_sync_service/l1client.go index d2a28f659f10..4f636102ddee 100644 --- a/rollup/rollup_sync_service/l1client.go +++ b/rollup/rollup_sync_service/l1client.go @@ -27,7 +27,7 @@ type L1Client struct { l1FinalizeBatchEventSignature common.Hash } -// newL1Client initializes a new L1Client instance with the provided configuration. +// NewL1Client initializes a new L1Client instance with the provided configuration. // It checks for a valid scrollChainAddress and verifies the chain ID. func NewL1Client(ctx context.Context, l1Client sync_service.EthClient, l1ChainId uint64, scrollChainAddress common.Address, scrollChainABI *abi.ABI) (*L1Client, error) { if scrollChainAddress == (common.Address{}) { @@ -79,7 +79,7 @@ func (c *L1Client) FetchRollupEventsInRange(from, to uint64) ([]types.Log, error return logs, nil } -// getLatestFinalizedBlockNumber fetches the block number of the latest finalized block from the L1 chain. +// GetLatestFinalizedBlockNumber fetches the block number of the latest finalized block from the L1 chain. func (c *L1Client) GetLatestFinalizedBlockNumber() (uint64, error) { header, err := c.client.HeaderByNumber(c.ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) if err != nil { @@ -91,7 +91,7 @@ func (c *L1Client) GetLatestFinalizedBlockNumber() (uint64, error) { return header.Number.Uint64(), nil } -// fetchTxData fetches tx data corresponding to given event log +// FetchTxData fetches tx data corresponding to given event log func (c *L1Client) FetchTxData(vLog *types.Log) ([]byte, error) { tx, _, err := c.client.TransactionByHash(c.ctx, vLog.TxHash) if err != nil { @@ -118,7 +118,7 @@ func (c *L1Client) FetchTxData(vLog *types.Log) ([]byte, error) { return tx.Data(), nil } -// fetchTxBlobHash fetches tx blob hash corresponding to given event log +// FetchTxBlobHash fetches tx blob hash corresponding to given event log func (c *L1Client) FetchTxBlobHash(vLog *types.Log) (common.Hash, error) { tx, _, err := c.client.TransactionByHash(c.ctx, vLog.TxHash) if err != nil { @@ -147,3 +147,12 @@ func (c *L1Client) FetchTxBlobHash(vLog *types.Log) (common.Hash, error) { } return blobHashes[0], nil } + +// GetHeaderByNumber fetches the block header by number +func (c *L1Client) GetHeaderByNumber(blockNumber uint64) (*types.Header, error) { + header, err := c.client.HeaderByNumber(c.ctx, big.NewInt(0).SetUint64(blockNumber)) + if err != nil { + return nil, err + } + return header, nil +} From 787c955f38f79158e77d08a0fc8144832b2a0f24 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 26 Aug 2024 12:56:08 +0200 Subject: [PATCH 58/59] update codec version --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 851cbd50e2ca..cfb212274ee9 100644 --- a/go.mod +++ b/go.mod @@ -50,7 +50,7 @@ require ( github.com/prometheus/tsdb v0.7.1 github.com/rjeczalik/notify v0.9.1 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a + github.com/scroll-tech/da-codec v0.1.1-0.20240826104725-b7214d9781ca github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 diff --git a/go.sum b/go.sum index 050f3b35a640..06fcb44be293 100644 --- a/go.sum +++ b/go.sum @@ -396,6 +396,8 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a h1:2+g6p5dxRQYFjDMBZgBJkem2HpIJdXEIlXzxreTCd4w= github.com/scroll-tech/da-codec v0.1.1-0.20240729153040-31de3201bd1a/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= +github.com/scroll-tech/da-codec v0.1.1-0.20240826104725-b7214d9781ca h1:ugAgOpbm84I/uOgGov9hm8X0d0ox+XSk/kCj2QKPLZs= +github.com/scroll-tech/da-codec v0.1.1-0.20240826104725-b7214d9781ca/go.mod h1:O9jsbQGNnTEfyfZg7idevq6jGGSQshX70elX+TRH8vU= github.com/scroll-tech/zktrie v0.8.4 h1:UagmnZ4Z3ITCk+aUq9NQZJNAwnWl4gSxsLb2Nl7IgRE= github.com/scroll-tech/zktrie v0.8.4/go.mod h1:XvNo7vAk8yxNyTjBDj5WIiFzYW4bx/gJ78+NK6Zn6Uk= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= From 4a66bf3f2fdd060e87d17da5bcc74718200244f0 Mon Sep 17 00:00:00 2001 From: Nazarii Denha Date: Mon, 26 Aug 2024 13:03:45 +0200 Subject: [PATCH 59/59] support codec v4 --- rollup/da_syncer/da/calldata_blob_source.go | 10 +++++- rollup/da_syncer/da/commitV4.go | 40 +++++++++++++++++++++ rollup/da_syncer/da/da.go | 2 ++ 3 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 rollup/da_syncer/da/commitV4.go diff --git a/rollup/da_syncer/da/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go index 5e973f7774a8..d946c8096fe8 100644 --- a/rollup/da_syncer/da/calldata_blob_source.go +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -226,7 +226,15 @@ func (ds *CalldataBlobSource) getCommitBatchDA(batchIndex uint64, vLog *types.Lo if err != nil { return nil, fmt.Errorf("failed to decode calldata into commitBatch args, values: %+v, err: %w", values, err) } - return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + switch args.Version { + case 3: + // we can use V2 for version 3, because it's same + return NewCommitBatchDAV2(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, args.Version, batchIndex, args.ParentBatchHeader, args.Chunks, args.SkippedL1MessageBitmap) + case 4: + return NewCommitBatchDAV4(ds.ctx, ds.db, ds.l1Client, ds.blobClient, vLog, 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) + } } return nil, fmt.Errorf("unknown method name: %s", method.Name) diff --git a/rollup/da_syncer/da/commitV4.go b/rollup/da_syncer/da/commitV4.go new file mode 100644 index 000000000000..9b590b2bfff5 --- /dev/null +++ b/rollup/da_syncer/da/commitV4.go @@ -0,0 +1,40 @@ +package da + +import ( + "context" + + "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/rollup_sync_service" + + "github.com/scroll-tech/go-ethereum/core/types" +) + +type CommitBatchDAV4 struct { + *CommitBatchDAV1 +} + +func NewCommitBatchDAV4(ctx context.Context, db ethdb.Database, + l1Client *rollup_sync_service.L1Client, + blobClient blob_client.BlobClient, + vLog *types.Log, + version uint8, + batchIndex uint64, + parentBatchHeader []byte, + chunks [][]byte, + skippedL1MessageBitmap []byte, +) (*CommitBatchDAV2, error) { + + v1, err := NewCommitBatchDAV1WithBlobDecodeFunc(ctx, db, l1Client, blobClient, vLog, version, batchIndex, parentBatchHeader, chunks, skippedL1MessageBitmap, codecv4.DecodeTxsFromBlob) + if err != nil { + return nil, err + } + + return &CommitBatchDAV2{v1}, nil +} + +func (c *CommitBatchDAV4) Type() Type { + return CommitBatchV4Type +} diff --git a/rollup/da_syncer/da/da.go b/rollup/da_syncer/da/da.go index 5a55a4fbb513..5f00e86115a1 100644 --- a/rollup/da_syncer/da/da.go +++ b/rollup/da_syncer/da/da.go @@ -15,6 +15,8 @@ const ( CommitBatchV1Type // CommitBatchV2Type contains data of event of CommitBatchV2Type CommitBatchV2Type + // CommitBatchV4Type contains data of event of CommitBatchV2Type + CommitBatchV4Type // RevertBatchType contains data of event of RevertBatchType RevertBatchType // FinalizeBatchType contains data of event of FinalizeBatchType