From 845449197b689e96df5b2ad7e869b395b44b40c3 Mon Sep 17 00:00:00 2001 From: Jonas Theis <4181434+jonastheis@users.noreply.github.com> Date: Wed, 16 Oct 2024 10:11:51 +0800 Subject: [PATCH] feat: follower node sync from DA rebased to `syncUpstream/active` (#1013) * feat: follower node sync from DA #631 Conflicts: cmd/geth/main.go core/state_processor_test.go core/txpool/legacypool/legacypool.go eth/backend.go eth/ethconfig/config.go eth/gasprice/gasprice_test.go eth/handler.go eth/protocols/eth/broadcast.go eth/protocols/eth/handlers.go go.mod go.sum miner/miner.go miner/miner_test.go miner/scroll_worker.go miner/scroll_worker_test.go params/config.go params/version.go rollup/rollup_sync_service/rollup_sync_service_test.go * fixes after merge * fix: panic after startup due to misconfigured api * fix bug of not calculating block state root * minor adjustment of fix * address review comments * upgrade da-codec to https://github.com/scroll-tech/da-codec/commit/41c6486469676eb88af1131c65dbbd73751458bb * fix go.mod after merge * fix some tests --------- Co-authored-by: Nazarii Denha Co-authored-by: 0xmountaintop <37070449+0xmountaintop@users.noreply.github.com> --- cmd/geth/main.go | 6 + cmd/utils/flags.go | 58 ++++- common/backoff/exponential.go | 51 ++++ common/backoff/exponential_test.go | 39 +++ common/heap.go | 109 ++++++++ common/heap_test.go | 40 +++ common/shrinkingmap.go | 71 ++++++ common/shrinkingmap_test.go | 135 ++++++++++ core/blockchain.go | 53 +++- core/rawdb/accessors_da_syncer.go | 39 +++ core/rawdb/schema.go | 3 + eth/backend.go | 42 ++- eth/ethconfig/config.go | 10 + go.mod | 10 +- go.sum | 149 +---------- miner/miner.go | 10 +- miner/miner_test.go | 2 +- miner/scroll_worker.go | 8 +- miner/worker_test.go | 2 +- node/config.go | 2 + node/node.go | 10 +- rollup/da_syncer/batch_queue.go | 102 ++++++++ .../blob_client/beacon_node_client.go | 192 ++++++++++++++ rollup/da_syncer/blob_client/blob_client.go | 64 +++++ .../da_syncer/blob_client/blob_scan_client.go | 92 +++++++ .../blob_client/block_native_client.go | 85 ++++++ rollup/da_syncer/block_queue.go | 56 ++++ rollup/da_syncer/da/calldata_blob_source.go | 241 ++++++++++++++++++ rollup/da_syncer/da/commitV0.go | 173 +++++++++++++ rollup/da_syncer/da/commitV1.go | 92 +++++++ rollup/da_syncer/da/commitV2.go | 40 +++ rollup/da_syncer/da/commitV4.go | 40 +++ rollup/da_syncer/da/da.go | 73 ++++++ rollup/da_syncer/da/finalize.go | 34 +++ rollup/da_syncer/da/revert.go | 33 +++ rollup/da_syncer/da_queue.go | 70 +++++ rollup/da_syncer/da_syncer.go | 49 ++++ rollup/da_syncer/data_source.go | 44 ++++ rollup/da_syncer/modes.go | 52 ++++ rollup/da_syncer/serrors/errors.go | 62 +++++ rollup/da_syncer/syncing_pipeline.go | 233 +++++++++++++++++ rollup/rollup_sync_service/abi.go | 2 +- rollup/rollup_sync_service/abi_test.go | 4 +- rollup/rollup_sync_service/l1client.go | 78 +++++- rollup/rollup_sync_service/l1client_test.go | 8 +- .../rollup_sync_service.go | 8 +- .../rollup_sync_service_test.go | 8 +- 47 files changed, 2600 insertions(+), 184 deletions(-) create mode 100644 common/backoff/exponential.go create mode 100644 common/backoff/exponential_test.go create mode 100644 common/heap.go create mode 100644 common/heap_test.go create mode 100644 common/shrinkingmap.go create mode 100644 common/shrinkingmap_test.go create mode 100644 core/rawdb/accessors_da_syncer.go create mode 100644 rollup/da_syncer/batch_queue.go create mode 100644 rollup/da_syncer/blob_client/beacon_node_client.go create mode 100644 rollup/da_syncer/blob_client/blob_client.go create mode 100644 rollup/da_syncer/blob_client/blob_scan_client.go create mode 100644 rollup/da_syncer/blob_client/block_native_client.go create mode 100644 rollup/da_syncer/block_queue.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/commitV4.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 create mode 100644 rollup/da_syncer/da_queue.go create mode 100644 rollup/da_syncer/da_syncer.go create mode 100644 rollup/da_syncer/data_source.go create mode 100644 rollup/da_syncer/modes.go create mode 100644 rollup/da_syncer/serrors/errors.go create mode 100644 rollup/da_syncer/syncing_pipeline.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index d76893dd5532..f3303295adca 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -154,6 +154,12 @@ var ( utils.CircuitCapacityCheckWorkersFlag, utils.RollupVerifyEnabledFlag, utils.ShadowforkPeersFlag, + utils.DASyncEnabledFlag, + utils.DAModeFlag, + utils.DASnapshotFileFlag, + utils.DABlockNativeAPIEndpointFlag, + utils.DABlobScanAPIEndpointFlag, + utils.DABeaconNodeAPIEndpointFlag, }, utils.NetworkFlags, utils.DatabaseFlags) rpcFlags = []cli.Flag{ diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 51fb2514591d..25a61c20838e 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -36,6 +36,9 @@ import ( "time" pcsclite "github.com/gballet/go-libpcsclite" + gopsutil "github.com/shirou/gopsutil/mem" + "github.com/urfave/cli/v2" + "github.com/scroll-tech/go-ethereum/accounts" "github.com/scroll-tech/go-ethereum/accounts/keystore" "github.com/scroll-tech/go-ethereum/common" @@ -73,13 +76,12 @@ 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" "github.com/scroll-tech/go-ethereum/trie" "github.com/scroll-tech/go-ethereum/trie/triedb/hashdb" "github.com/scroll-tech/go-ethereum/trie/triedb/pathdb" - gopsutil "github.com/shirou/gopsutil/mem" - "github.com/urfave/cli/v2" ) const ( @@ -1024,6 +1026,33 @@ Please note that --` + MetricsHTTPFlag.Name + ` must be set to start the server. Name: "net.shadowforkpeers", Usage: "peer ids of shadow fork peers", } + + // DA syncing settings + DASyncEnabledFlag = &cli.BoolFlag{ + Name: "da.sync", + Usage: "Enable node syncing from DA", + } + DAModeFlag = &flags.TextMarshalerFlag{ + Name: "da.mode", + Usage: `DA sync mode ("l1rpc" or "snapshot")`, + Value: ðconfig.Defaults.DA.FetcherMode, + } + 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", + } + DABlockNativeAPIEndpointFlag = &cli.StringFlag{ + Name: "da.blob.blocknative", + Usage: "BlockNative blob API endpoint", + } + DABeaconNodeAPIEndpointFlag = &cli.StringFlag{ + Name: "da.blob.beaconnode", + Usage: "Beacon node API endpoint", + } ) var ( @@ -1512,6 +1541,9 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { SetDataDir(ctx, cfg) setSmartCard(ctx, cfg) setL1(ctx, cfg) + if ctx.IsSet(DASyncEnabledFlag.Name) { + cfg.DaSyncingEnabled = ctx.Bool(DASyncEnabledFlag.Name) + } if ctx.IsSet(JWTSecretFlag.Name) { cfg.JWTSecret = ctx.String(JWTSecretFlag.Name) @@ -1766,6 +1798,27 @@ func setEnableRollupVerify(ctx *cli.Context, cfg *ethconfig.Config) { } } +func setDA(ctx *cli.Context, cfg *ethconfig.Config) { + if ctx.IsSet(DASyncEnabledFlag.Name) { + cfg.EnableDASyncing = ctx.Bool(DASyncEnabledFlag.Name) + if ctx.IsSet(DAModeFlag.Name) { + cfg.DA.FetcherMode = *flags.GlobalTextMarshaler(ctx, DAModeFlag.Name).(*da_syncer.FetcherMode) + } + if ctx.IsSet(DASnapshotFileFlag.Name) { + cfg.DA.SnapshotFilePath = ctx.String(DASnapshotFileFlag.Name) + } + if ctx.IsSet(DABlobScanAPIEndpointFlag.Name) { + cfg.DA.BlobScanAPIEndpoint = ctx.String(DABlobScanAPIEndpointFlag.Name) + } + if ctx.IsSet(DABlockNativeAPIEndpointFlag.Name) { + cfg.DA.BlockNativeAPIEndpoint = ctx.String(DABlockNativeAPIEndpointFlag.Name) + } + if ctx.IsSet(DABeaconNodeAPIEndpointFlag.Name) { + cfg.DA.BeaconNodeAPIEndpoint = ctx.String(DABeaconNodeAPIEndpointFlag.Name) + } + } +} + func setMaxBlockRange(ctx *cli.Context, cfg *ethconfig.Config) { if ctx.IsSet(MaxBlockRangeFlag.Name) { cfg.MaxBlockRange = ctx.Int64(MaxBlockRangeFlag.Name) @@ -1831,6 +1884,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) if ctx.IsSet(ShadowforkPeersFlag.Name) { cfg.ShadowForkPeerIDs = ctx.StringSlice(ShadowforkPeersFlag.Name) diff --git a/common/backoff/exponential.go b/common/backoff/exponential.go new file mode 100644 index 000000000000..e1f9b53a350e --- /dev/null +++ b/common/backoff/exponential.go @@ -0,0 +1,51 @@ +package backoff + +import ( + "math" + "math/rand" + "time" +) + +// Exponential is a backoff strategy that increases the delay between retries exponentially. +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/common/heap.go b/common/heap.go new file mode 100644 index 000000000000..67b79a1136d1 --- /dev/null +++ b/common/heap.go @@ -0,0 +1,109 @@ +package common + +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] +} + +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[T] { + heapElement := NewHeapElement(element) + heap.Push(&h.heap, heapElement) + + return heapElement +} + +func (h *Heap[T]) Pop() *HeapElement[T] { + return heap.Pop(&h.heap).(*HeapElement[T]) +} + +func (h *Heap[T]) Peek() *HeapElement[T] { + if h.Len() == 0 { + return nil + } + + return h.heap[0] +} + +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 { + 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 +} + +// 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 +} + +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") +} diff --git a/common/shrinkingmap.go b/common/shrinkingmap.go new file mode 100644 index 000000000000..4bf98f87c2da --- /dev/null +++ b/common/shrinkingmap.go @@ -0,0 +1,71 @@ +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]) 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 +} + +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) +} diff --git a/core/blockchain.go b/core/blockchain.go index 6db2b45d355d..0306f1681b3c 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -28,6 +28,8 @@ import ( "sync/atomic" "time" + "golang.org/x/exp/slices" + "github.com/scroll-tech/go-ethereum/common" "github.com/scroll-tech/go-ethereum/common/lru" "github.com/scroll-tech/go-ethereum/common/mclock" @@ -50,7 +52,6 @@ import ( "github.com/scroll-tech/go-ethereum/trie" "github.com/scroll-tech/go-ethereum/trie/triedb/hashdb" "github.com/scroll-tech/go-ethereum/trie/triedb/pathdb" - "golang.org/x/exp/slices" ) var ( @@ -2024,6 +2025,56 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error) return it.index, err } +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 + } + + statedb.StartPrefetcher("l1sync") + defer statedb.StopPrefetcher() + + 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 { + return NonStatTy, fmt.Errorf("error processing block: %w", 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. + + // finalize and assemble block as fullBlock + header.GasUsed = gasUsed + header.Root = statedb.IntermediateRoot(bc.chainConfig.IsEIP158(header.Number)) + + 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.writeBlockAndSetHead(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/core/rawdb/accessors_da_syncer.go b/core/rawdb/accessors_da_syncer.go new file mode 100644 index 000000000000..96f816685652 --- /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 5a316a42e784..b553045f0a40 100644 --- a/core/rawdb/schema.go +++ b/core/rawdb/schema.go @@ -155,6 +155,9 @@ var ( lastFinalizedBatchIndexKey = []byte("R-finalizedBatchIndex") committedBatchMetaPrefix = []byte("R-cbm") + // 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 ce03e7c1c68c..71c1d0ebe822 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -58,6 +58,7 @@ import ( "github.com/scroll-tech/go-ethereum/params" "github.com/scroll-tech/go-ethereum/rlp" "github.com/scroll-tech/go-ethereum/rollup/ccc" + "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/rpc" @@ -76,6 +77,7 @@ type Ethereum struct { syncService *sync_service.SyncService rollupSyncService *rollup_sync_service.RollupSyncService asyncChecker *ccc.AsyncChecker + syncingPipeline *da_syncer.SyncingPipeline blockchain *core.BlockChain handler *handler @@ -244,6 +246,18 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl return nil, err } + // 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 { @@ -277,7 +291,7 @@ func New(stack *node.Node, config *ethconfig.Config, l1Client sync_service.EthCl return nil, err } - eth.miner = miner.New(eth, &config.Miner, eth.blockchain.Config(), eth.EventMux(), eth.engine, eth.isLocalBlock) + eth.miner = miner.New(eth, &config.Miner, eth.blockchain.Config(), 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} @@ -340,6 +354,12 @@ 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.config.EnableDASyncing { + apis = append(apis, rpc.API{ + Namespace: "eth", + Service: downloader.NewDownloaderAPI(s.handler.downloader, s.eventMux), + }) + } // Append all the local APIs and return return append(apis, []rpc.API{ @@ -349,9 +369,6 @@ func (s *Ethereum) APIs() []rpc.API { }, { Namespace: "miner", Service: NewMinerAPI(s), - }, { - Namespace: "eth", - Service: downloader.NewDownloaderAPI(s.handler.downloader, s.eventMux), }, { Namespace: "admin", Service: NewAdminAPI(s), @@ -528,6 +545,10 @@ 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 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) if !s.blockchain.Config().Scroll.ZktrieEnabled() && s.config.SnapshotCache > 0 { protos = append(protos, snap.MakeProtocols((*snapHandler)(s.handler), s.snapDialCandidates)...) @@ -555,7 +576,10 @@ func (s *Ethereum) Start() error { // maxPeers -= s.config.LightPeers // } // Start the networking layer and the light server if requested - s.handler.Start(maxPeers) + // handler is not enabled when DA syncing enabled + if !s.config.EnableDASyncing { + s.handler.Start(maxPeers) + } return nil } @@ -565,7 +589,10 @@ func (s *Ethereum) Stop() error { // Stop all the peer-related stuff first. s.ethDialCandidates.Close() s.snapDialCandidates.Close() - s.handler.Stop() + // handler is not enabled if DA syncing enabled + if !s.config.EnableDASyncing { + s.handler.Stop() + } // Then stop everything else. s.bloomIndexer.Close() @@ -575,6 +602,9 @@ func (s *Ethereum) Stop() error { if s.config.EnableRollupVerify { s.rollupSyncService.Stop() } + if s.config.EnableDASyncing { + s.syncingPipeline.Stop() + } s.miner.Close() if s.config.CheckCircuitCapacity { s.asyncChecker.Wait() diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index c4f28fdb637d..307656e8350e 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -34,6 +34,7 @@ import ( "github.com/scroll-tech/go-ethereum/ethdb" "github.com/scroll-tech/go-ethereum/miner" "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. @@ -78,6 +79,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, + }, } //go:generate go run github.com/fjl/gencodec -type Config -formats toml -out gen_config.go @@ -182,6 +186,12 @@ type Config struct { // List of peer ids that take part in the shadow-fork ShadowForkPeerIDs []string + + // Enable syncing node from DA + EnableDASyncing bool + + // DA syncer config + DA da_syncer.Config } // CreateConsensusEngine creates a consensus engine for the given chain config. diff --git a/go.mod b/go.mod index 3b3945cd41b9..ab4da0ed3a09 100644 --- a/go.mod +++ b/go.mod @@ -57,9 +57,10 @@ require ( github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 github.com/protolambda/bls12-381-util v0.0.0-20220416220906-d8552aa452c7 github.com/rs/cors v1.7.0 - github.com/scroll-tech/da-codec v0.1.1-0.20240822151711-9e32313056ac + github.com/scroll-tech/da-codec v0.1.1-0.20240902151734-41c648646967 github.com/scroll-tech/zktrie v0.8.4 github.com/shirou/gopsutil v3.21.11+incompatible + github.com/sourcegraph/conc v0.3.0 github.com/status-im/keycard-go v0.2.0 github.com/stretchr/testify v1.9.0 github.com/supranational/blst v0.3.11 @@ -97,7 +98,6 @@ require ( github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.5 // indirect - github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/consensys/bavard v0.1.13 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect @@ -111,7 +111,6 @@ require ( github.com/go-sourcemap/sourcemap v2.1.3+incompatible // indirect github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/google/go-cmp v0.5.9 // indirect github.com/google/go-querystring v1.1.0 // indirect github.com/google/pprof v0.0.0-20230207041349-798e818bf904 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect @@ -119,7 +118,7 @@ require ( github.com/influxdata/line-protocol v0.0.0-20200327222509-2487e7298839 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/kilic/bls12-381 v0.1.0 // indirect - github.com/klauspost/compress v1.15.15 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect github.com/mattn/go-runewidth v0.0.13 // indirect @@ -138,11 +137,12 @@ require ( github.com/rivo/uniseg v0.2.0 // indirect github.com/rogpeppe/go-internal v1.10.0 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect - github.com/sourcegraph/conc v0.3.0 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect + go.uber.org/atomic v1.7.0 // indirect + go.uber.org/multierr v1.9.0 // indirect golang.org/x/mod v0.12.0 // indirect golang.org/x/net v0.17.0 // indirect google.golang.org/protobuf v1.28.1 // indirect diff --git a/go.sum b/go.sum index 487fea0fbd06..e3c2b369d3f9 100644 --- a/go.sum +++ b/go.sum @@ -31,7 +31,6 @@ cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohl cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0 h1:8q4SaHjFsClSvuVne0ID/5Ka8u3fcIHyqkLjcFpNRHQ= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0 h1:vcYCAze6p19qBW7MhZybIsqD8sMV8js0NyQM8JDnVtg= @@ -48,18 +47,12 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= -github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= -github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= -github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= -github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -67,7 +60,6 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= @@ -94,7 +86,6 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwF github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= -github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -122,39 +113,22 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cloudflare/cloudflare-go v0.79.0 h1:ErwCYDjFCYppDJlDJ/5WhsSmzegAUe2+K9qgFyQDg3M= github.com/cloudflare/cloudflare-go v0.79.0/go.mod h1:gkHQf9xEubaQPEuerBuoinR9P8bf8a05Lq0X6WKy1Oc= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= -github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= -github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= github.com/cockroachdb/errors v1.11.1 h1:xSEW75zKaKCWzR3OfxXUxgrk/NtT4G1MiOv5lWZazG8= github.com/cockroachdb/errors v1.11.1/go.mod h1:8MUxA3Gi6b25tYlFEBGLf+D8aISL+M4MIpiWMSNRfxw= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= -github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= github.com/cockroachdb/pebble v1.1.0 h1:pcFh8CdCIt2kmEpK0OIatq67Ln9uGDYY3d5XnE0LJG4= github.com/cockroachdb/pebble v1.1.0/go.mod h1:sEHm5NOXxyiAoKWhoFxT8xMgd/f3RA6qUqQ1BXKrh2E= -github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= -github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= -github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.2 h1:p1EgwI/C7NhT0JmVkwCD2ZBK8j4aeHQX2pMHHBfMQ6w= github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/crate-crypto/go-ipa v0.0.0-20230601170251-1830d0757c80 h1:DuBDHVjgGMPki7bAyh91+3cF1Vh34sAEdH8JQgbc2R0= @@ -174,9 +148,7 @@ github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 h1:YLtO71vCjJRCBcrPMtQ9nqBsqpA1 github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1/go.mod h1:hyedUtir6IdtD/7lIxGeCxkaw7y45JueMRL4DIyJDKs= github.com/deepmap/oapi-codegen v1.6.0 h1:w/d1ntwh91XI0b/8ja7+u5SvA4IFfM0UNNLmiDR1gg0= github.com/deepmap/oapi-codegen v1.6.0/go.mod h1:ryDa9AgbELGeB+YEXE1dR53yAjHwFvE9iAUlWl9Al3M= -github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dlclark/regexp2 v1.4.1-0.20201116162257-a2a8dda75c91/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= github.com/dlclark/regexp2 v1.7.0 h1:7lJfhqlPssTb1WQx4yvTHN0uElPEv52sbaECrAQxjAo= github.com/dlclark/regexp2 v1.7.0/go.mod h1:DHkYz0B9wPfa6wondMfaivmHpzrQ3v9q8cnmRbL6yW8= @@ -189,31 +161,24 @@ github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 h1:qwcF+vdFrvPSEUDSX5R github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127/go.mod h1:QMWlm50DNe14hD7t24KEqZuUdC9sOTy8W6XbCU1mlw4= github.com/dop251/goja_nodejs v0.0.0-20210225215109-d91c329300e7/go.mod h1:hn7BA7c8pLvoGndExHudxTDKZ84Pyvv+90pbBjbTz0Y= github.com/dop251/goja_nodejs v0.0.0-20211022123610-8dd9abb0616d/go.mod h1:DngW8aVqWbuLRMHItjPUyqdj+HWPvnQe8V8y1nDpIbM= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= github.com/ethereum/c-kzg-4844 v1.0.3 h1:IEnbOHwjixW2cTvKRUlAAUOeleV7nNM/umJR+qy4WDs= github.com/ethereum/c-kzg-4844 v1.0.3/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= -github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/fjl/gencodec v0.0.0-20230517082657-f9840df7b83e h1:bBLctRc7kr01YGvaDfgLbTwjFNW5jdp5y5rj8XXBHfY= github.com/fjl/gencodec v0.0.0-20230517082657-f9840df7b83e/go.mod h1:AzA8Lj6YtixmJWL+wkKoBGsLWy9gFrAzi4g+5bCKwpY= github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 h1:FtmdgXiUlNeRsoNMFlKLDt+S+6hbjVMEW6RGQ7aUf7c= github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+uIk2RvXzuaQtkQJzzIx6lSBe1xv7hi0= -github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= github.com/garslo/gogen v0.0.0-20170306192744-1d203ffc1f61 h1:IZqZOB2fydHte3kUgxrzK5E1fW7RQGeDwE8F/ZZnUYc= github.com/garslo/gogen v0.0.0-20170306192744-1d203ffc1f61/go.mod h1:Q0X6pkwTILDlzrGEckF6HKjXe48EgsY/l7K7vhY4MW8= -github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff h1:tY80oXqGNY4FhTFhk+o9oFHGINQ/+vhlm8HFzi6znCI= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/gballet/go-verkle v0.0.0-20230607174250-df487255f46b h1:vMT47RYsrftsHSTQhqXwC3BYflo38OLC3Y4LtXtLyU0= @@ -222,12 +187,9 @@ github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW github.com/getsentry/sentry-go v0.18.0 h1:MtBW5H9QgdcJabtZcuJG80BMOwaBpkRDZkxRkNC1sN0= github.com/getsentry/sentry-go v0.18.0/go.mod h1:Kgon4Mby+FJ7ZWHFUAZgVaIa8sxHtnRJRLTXZr51aKQ= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= -github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= github.com/go-chi/chi/v5 v5.0.0/go.mod h1:BBug9lr0cqtdAhsu6R4AAdvufI0/XBzAQSsUqJpoZOs= -github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= -github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= +github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -237,7 +199,6 @@ github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vb github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= @@ -248,20 +209,13 @@ github.com/go-sourcemap/sourcemap v2.1.3+incompatible/go.mod h1:F8jJfvm2KbVjc5Nq github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-stack/stack v1.8.1 h1:ntEHSVwIt7PNXNpgPmVfMrNhLtgjlmnZha2kOpuRiDw= github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP3XYfe4= -github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= -github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= -github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw= github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= -github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -297,7 +251,6 @@ github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk= github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= -github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= @@ -313,7 +266,6 @@ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/go-querystring v1.1.0 h1:AnCroh3fv4ZBgVIf1Iwtovgjaw/GiKJo8M8yD/fhyJ8= github.com/google/go-querystring v1.1.0/go.mod h1:Kcdr2DB4koayq7X8pmAG4sNG59So17icRSOU623lUBU= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -336,9 +288,7 @@ github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/graph-gophers/graphql-go v1.3.0 h1:Eb9x/q6MFpCLz7jBCiP/WTxjSDrYLR1QY41SORZyNJ0= @@ -352,10 +302,8 @@ github.com/hashicorp/go-hclog v1.2.0 h1:La19f8d7WIlm4ogzNHB0JGqs5AUDAZ2UfCY4sJXc github.com/hashicorp/go-hclog v1.2.0/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ= github.com/hashicorp/go-retryablehttp v0.7.4 h1:ZQgVdpTdAL7WpMIwLzCfbalOcSUdkDZnpUv3/+BxzFA= github.com/hashicorp/go-retryablehttp v0.7.4/go.mod h1:Jy/gPYAdjqffZ/yFGCFV2doI5wjtH1ewM9u8iYVjtX8= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/holiman/billy v0.0.0-20230718173358-1c7e68d277a7 h1:3JQNjnMRil1yD0IfZKHF9GxxWKDJGj8I0IqOUol//sw= github.com/holiman/billy v0.0.0-20230718173358-1c7e68d277a7/go.mod h1:5GuXa7vkL8u9FkFuWdVvfR5ix8hRB7DbOAaYULamFpc= github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= @@ -365,23 +313,16 @@ github.com/holiman/uint256 v1.2.4/go.mod h1:EOMSn4q6Nyt9P6efbI3bueV4e1b3dGlUCXei github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc= github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= -github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20220319035150-800ac71e25c2/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= github.com/iden3/go-iden3-crypto v0.0.15 h1:4MJYlrot1l31Fzlo2sF56u7EVFeHHJkxGXXZCtESgK4= github.com/iden3/go-iden3-crypto v0.0.15/go.mod h1:dLpM4vEPJ3nDHzhWFXDjzkn1qHoBeOT/3UEhXsEsP3E= -github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb-client-go/v2 v2.4.0 h1:HGBfZYStlx3Kqvsv1h2pJixbCl/jhnFtxpKFAv9Tu5k= github.com/influxdata/influxdb-client-go/v2 v2.4.0/go.mod h1:vLNHdxTJkIf2mSLvGrpj8TCcISApPoXkaxP8g9uRlW8= github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c h1:qSHzRbhzK8RdXOsAdfDgO49TtqC1oZ+acxPrkfTxcCs= github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= github.com/influxdata/line-protocol v0.0.0-20200327222509-2487e7298839 h1:W9WBk7wlPfJLvMCdtV4zPulc4uCPrlywQOmbFOhgQNU= github.com/influxdata/line-protocol v0.0.0-20200327222509-2487e7298839/go.mod h1:xaLFMmpvUxqXtVkUJfg9QmT88cDaCJ3ZKgdZ78oO8Qo= -github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= -github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= -github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= -github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jedisct1/go-minisign v0.0.0-20230811132847-661be99b8267 h1:TMtDYDHKYY15rFihtRfck/bfFqNfvcabqvXAFQfAUpY= @@ -397,30 +338,17 @@ github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/karalabe/usb v0.0.2 h1:M6QQBNxF+CQ8OFvxrT90BA0qBOXymndZnk5q235mFc4= github.com/karalabe/usb v0.0.2/go.mod h1:Od972xHfMJowv7NGVDiWVxk2zxnWgjLlJzE+F4F7AGU= -github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= -github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= -github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= -github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= github.com/kilic/bls12-381 v0.1.0 h1:encrdjqKMEvabVQ7qYOKu1OvhqpK4s47wDYtNiPtlp4= github.com/kilic/bls12-381 v0.1.0/go.mod h1:vDTTHJONJ6G+P2R74EhnyotQDTliQDnFEwhdmfzw1ig= -github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.15.15 h1:EF27CXIuDsYJ6mmvtBRlEuB2UVOqHG1tAXgZ7yIO+lw= -github.com/klauspost/compress v1.15.15/go.mod h1:ZcK2JAFqKOpnBlxcLsJzYfrS9X1akm9fHZNnD9+Vo/4= -github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= @@ -435,12 +363,10 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= -github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4FW1e6jwpg= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/matryer/moq v0.0.0-20190312154309-6cfb0558e1bd/go.mod h1:9ELz6aaclSIGnZBoaSLZ3NAl1VTufbOrXBPvtcy6WiQ= @@ -450,7 +376,6 @@ github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= -github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= @@ -462,15 +387,9 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.13 h1:lTGmDsbAYt5DmK6OnoV7EuIF1wEIFAcxld6ypU4OSgU= github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= -github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= -github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= -github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.4.1 h1:CpVNEelQCZBooIPDn+AR3NpivK/TIKU8bDxdASFVQag= github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/mitchellh/pointerstructure v1.2.0 h1:O+i9nHnXS3l/9Wu7r4NrEdwA2VFTicjUEN1uBnDo34A= @@ -483,23 +402,18 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= -github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/naoina/go-stringutil v0.1.0 h1:rCUeRUHjBjGTSHl0VC00jUPLz8/F9dDzYI70Hzifhks= github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416 h1:shk/vn9oCoOTmwcouEdwIeOtOGA/ELRUw/GwvxwfT+0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= -github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= -github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= @@ -507,7 +421,6 @@ github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 h1:oYW+YCJ1pachXTQmzR3rNLYGGz4g/UgFcjb28p/viDM= github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7/go.mod h1:CRroGNssyjTd/qIG2FyxByd2S8JEAZXBl4qUrZf8GS0= github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= @@ -557,34 +470,19 @@ github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjR github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= 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 v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v2.1.0+incompatible h1:j1Wcmh8OrK4Q7GXY+V7SVSY8nUWQxHW5TkBe7YUl+2s= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= -github.com/scroll-tech/da-codec v0.1.1-0.20240822151711-9e32313056ac h1:DjLrqjoOLVFug9ZkAbJYwjtYW51YZE0Num3p4cZXaZs= -github.com/scroll-tech/da-codec v0.1.1-0.20240822151711-9e32313056ac/go.mod h1:D6XEESeNVJkQJlv3eK+FyR+ufPkgVQbJzERylQi53Bs= +github.com/scroll-tech/da-codec v0.1.1-0.20240902151734-41c648646967 h1:FSM0l1n5KszBjPFOnMbSa4pg3zv07DYIU2VnH6BUH34= +github.com/scroll-tech/da-codec v0.1.1-0.20240902151734-41c648646967/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/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v3.21.11+incompatible h1:+1+c1VGhc88SSonWP6foOcLhvnKlUeu/erjjvaPEYiI= github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/sourcegraph/conc v0.3.0 h1:OQTbbt6P72L20UqAkXXuLOj79LfEanQ+YQFNpLA9ySo= github.com/sourcegraph/conc v0.3.0/go.mod h1:Sdozi7LEKbFPqYX2/J+iBAM6HpqSLTASQIKqDmF7Mt0= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/status-im/keycard-go v0.2.0 h1:QDLFswOQu1r5jsycloeQh3bVU8n/NatHHaZobtDnDzA= github.com/status-im/keycard-go v0.2.0/go.mod h1:wlp8ZLbsmrF6g6WjugPAx+IzoLrkdf9+mHxBEeo3Hbg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -606,26 +504,13 @@ github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+F github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/tyler-smith/go-bip39 v1.1.0 h1:5eUemwrMargf3BSLRRCalXT93Ns6pQJIjYQN2nyfOP8= github.com/tyler-smith/go-bip39 v1.1.0/go.mod h1:gUYDtqQw1JS3ZJ8UWVcGTGqqr6YIN3CWg+kkNaLt55U= -github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/urfave/cli/v2 v2.25.7 h1:VAzn5oq403l5pHjc4OhD54+XGO9cdKVL/7lDjF+iKUs= github.com/urfave/cli/v2 v2.25.7/go.mod h1:8qnjx1vcq5s2/wpsqoZFndg2CE5tNFyrTvS6SinrnYQ= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= -github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= -github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= -github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= -github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 h1:bAn7/zixMGCfxrRTfdpNzjtPYqr8smhKouy9mxVdGPU= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8= -github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -638,14 +523,16 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/automaxprocs v1.5.2 h1:2LxUOGiR3O6tw8ui5sZa2LAaHnsviZdVOUZw4fvbnME= go.uber.org/automaxprocs v1.5.2/go.mod h1:eRbA25aqJrxAbsLO0xy5jVwPt7FQnRgjW+efnwa1WM0= +go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= +go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -692,11 +579,9 @@ golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -705,7 +590,6 @@ golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -754,7 +638,6 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -764,7 +647,6 @@ golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -838,15 +720,11 @@ golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= @@ -913,7 +791,6 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -943,7 +820,6 @@ google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7Fc google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -968,8 +844,6 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= @@ -980,9 +854,6 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= -gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= diff --git a/miner/miner.go b/miner/miner.go index d76ec7a37b37..807075471e5f 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -89,7 +89,7 @@ type Miner struct { wg sync.WaitGroup } -func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(header *types.Header) bool) *Miner { +func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(header *types.Header) bool, daSyncingEnabled bool) *Miner { miner := &Miner{ mux: mux, eth: eth, @@ -97,10 +97,12 @@ func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *even exitCh: make(chan struct{}), startCh: make(chan struct{}), 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 7e80abcd404a..2e41c085e504 100644 --- a/miner/miner_test.go +++ b/miner/miner_test.go @@ -335,7 +335,7 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux, func(skipMiner bool)) { // Create event Mux mux := new(event.TypeMux) // Create Miner - miner := New(backend, &config, chainConfig, mux, engine, nil) + miner := New(backend, &config, chainConfig, mux, engine, nil, false) cleanup := func(skipMiner bool) { bc.Stop() engine.Close() diff --git a/miner/scroll_worker.go b/miner/scroll_worker.go index 754ce54ddbe5..6c9e9bbd8341 100644 --- a/miner/scroll_worker.go +++ b/miner/scroll_worker.go @@ -180,7 +180,7 @@ type worker struct { skipTxHash common.Hash } -func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(*types.Header) bool, init bool) *worker { +func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(*types.Header) bool, init bool, daSyncingEnabled bool) *worker { worker := &worker{ config: config, chainConfig: chainConfig, @@ -197,6 +197,12 @@ func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus startCh: make(chan struct{}, 1), reorgCh: make(chan reorgTrigger, 1), } + + if daSyncingEnabled { + log.Info("Worker will not start, because DA syncing is enabled") + return worker + } + worker.asyncChecker = ccc.NewAsyncChecker(worker.chain, config.CCCMaxWorkers, false).WithOnFailingBlock(worker.onBlockFailingCCC) // Subscribe NewTxsEvent for tx pool diff --git a/miner/worker_test.go b/miner/worker_test.go index e3fdcf33cae2..e92705f7b906 100644 --- a/miner/worker_test.go +++ b/miner/worker_test.go @@ -167,7 +167,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.Add(pendingTxs, true, false) - 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/node/config.go b/node/config.go index 93dc612ced99..e75a6d3c70fb 100644 --- a/node/config.go +++ b/node/config.go @@ -218,6 +218,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 00d906845e74..eb2ccee20192 100644 --- a/node/node.go +++ b/node/node.go @@ -265,9 +265,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() diff --git a/rollup/da_syncer/batch_queue.go b/rollup/da_syncer/batch_queue.go new file mode 100644 index 000000000000..7a3d094f6322 --- /dev/null +++ b/rollup/da_syncer/batch_queue.go @@ -0,0 +1,102 @@ +package da_syncer + +import ( + "context" + "fmt" + + "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" +) + +// 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 + lastFinalizedBatchIndex uint64 + batches *common.Heap[da.Entry] + batchesMap *common.ShrinkingMap[uint64, *common.HeapElement[da.Entry]] +} + +func NewBatchQueue(DAQueue *DAQueue, db ethdb.Database) *BatchQueue { + return &BatchQueue{ + DAQueue: DAQueue, + db: db, + lastFinalizedBatchIndex: 0, + batches: common.NewHeap[da.Entry](), + batchesMap: common.NewShrinkingMap[uint64, *common.HeapElement[da.Entry]](1000), + } +} + +// 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 := bq.getFinalizedBatch(); batch != nil { + return batch, nil + } + + for { + daEntry, err := bq.DAQueue.NextDA(ctx) + if err != nil { + return nil, err + } + switch daEntry.Type() { + case da.CommitBatchV0Type, da.CommitBatchV1Type, da.CommitBatchV2Type: + bq.addBatch(daEntry) + case da.RevertBatchType: + bq.deleteBatch(daEntry) + case da.FinalizeBatchType: + if daEntry.BatchIndex() > bq.lastFinalizedBatchIndex { + bq.lastFinalizedBatchIndex = daEntry.BatchIndex() + } + + if batch := bq.getFinalizedBatch(); batch != nil { + return batch, nil + } + default: + return nil, fmt.Errorf("unexpected type of daEntry: %T", daEntry) + } + } +} + +// getFinalizedBatch returns next finalized batch if there is available +func (bq *BatchQueue) getFinalizedBatch() da.Entry { + if bq.batches.Len() == 0 { + return nil + } + + batch := bq.batches.Peek().Value() + if batch.BatchIndex() <= bq.lastFinalizedBatchIndex { + 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(batch da.Entry) { + batchHeapElement, exists := bq.batchesMap.Get(batch.BatchIndex()) + if !exists { + return + } + + bq.batchesMap.Delete(batch.BatchIndex()) + bq.batches.Remove(batchHeapElement) + + // we store here min height of currently loaded batches to be able to start syncing from the same place in case of restart + // 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/blob_client/beacon_node_client.go b/rollup/da_syncer/blob_client/beacon_node_client.go new file mode 100644 index 000000000000..5bfd7b9edf6c --- /dev/null +++ b/rollup/da_syncer/blob_client/beacon_node_client.go @@ -0,0 +1,192 @@ +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, err := io.ReadAll(resp.Body) + if err != nil { + return nil, fmt.Errorf("beacon node request failed with status: %s: could not read response body: %w", resp.Status, err) + } + 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, err := io.ReadAll(resp.Body) + if err != nil { + return nil, fmt.Errorf("beacon node request failed with status: %s: could not read response body: %w", resp.Status, err) + } + 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, err := io.ReadAll(resp.Body) + if err != nil { + return nil, fmt.Errorf("beacon node request failed with status: %s: could not read response body: %w", resp.Status, err) + } + 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 new file mode 100644 index 000000000000..814b1d4faf2d --- /dev/null +++ b/rollup/da_syncer/blob_client/blob_client.go @@ -0,0 +1,64 @@ +package blob_client + +import ( + "context" + "errors" + "fmt" + + "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" +) + +const ( + lenBlobBytes int = 131072 + lenKZGCommitment int = 48 +) + +type BlobClient interface { + GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) +} + +type BlobClients struct { + list []BlobClient + curPos int +} + +func NewBlobClients(blobClients ...BlobClient) *BlobClients { + return &BlobClients{ + list: blobClients, + curPos: 0, + } +} + +func (c *BlobClients) GetBlobByVersionedHashAndBlockNumber(ctx context.Context, versionedHash common.Hash, blockNumber uint64) (*kzg4844.Blob, error) { + if len(c.list) == 0 { + return nil, fmt.Errorf("BlobClients.GetBlobByVersionedHash: list of BlobClients is empty") + } + + for i := 0; i < len(c.list); i++ { + 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("BlobClients: failed to get blob by versioned hash from BlobClient", "err", err, "blob client pos in BlobClients", c.curPos) + } + + // 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("BlobClients.GetBlobByVersionedHash: failed to get blob by versioned hash from all BlobClients")) +} + +func (c *BlobClients) nextPos() { + c.curPos = (c.curPos + 1) % len(c.list) +} + +func (c *BlobClients) AddBlobClient(blobClient BlobClient) { + c.list = append(c.list, blobClient) +} + +func (c *BlobClients) Size() int { + return len(c.list) +} diff --git a/rollup/da_syncer/blob_client/blob_scan_client.go b/rollup/da_syncer/blob_client/blob_scan_client.go new file mode 100644 index 000000000000..24b03bed32b9 --- /dev/null +++ b/rollup/da_syncer/blob_client/blob_scan_client.go @@ -0,0 +1,92 @@ +package blob_client + +import ( + "context" + "crypto/sha256" + "encoding/hex" + "encoding/json" + "fmt" + "net/http" + "net/url" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/common/hexutil" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" +) + +type BlobScanClient struct { + client *http.Client + apiEndpoint string +} + +func NewBlobScanClient(apiEndpoint string) *BlobScanClient { + return &BlobScanClient{ + client: http.DefaultClient, + apiEndpoint: apiEndpoint, + } +} + +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 { + 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) + } + req.Header.Set("accept", "application/json") + resp, err := c.client.Do(req) + if err != nil { + return nil, fmt.Errorf("cannot do request, err: %w", err) + } + defer resp.Body.Close() + 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 + 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 BlobRespBlobScan + + 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.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) + + // sanity check that retrieved blob matches versioned hash + commitment, err := kzg4844.BlobToCommitment(&blob) + if err != nil { + return nil, fmt.Errorf("failed to convert blob to commitment, err: %w", err) + } + + blobVersionedHash := kzg4844.CalcBlobHashV1(sha256.New(), &commitment) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blob versioned hash mismatch, expected: %s, got: %s", versionedHash.String(), hexutil.Encode(blobVersionedHash[:])) + } + + return &blob, nil +} + +type BlobRespBlobScan struct { + Data string `json:"data"` +} + +type ErrorRespBlobScan struct { + Message string `json:"message"` + Code string `json:"code"` +} diff --git a/rollup/da_syncer/blob_client/block_native_client.go b/rollup/da_syncer/blob_client/block_native_client.go new file mode 100644 index 000000000000..ddd574d02d10 --- /dev/null +++ b/rollup/da_syncer/blob_client/block_native_client.go @@ -0,0 +1,85 @@ +package blob_client + +import ( + "context" + "crypto/sha256" + "encoding/hex" + "encoding/json" + "fmt" + "net/http" + "net/url" + + "github.com/scroll-tech/go-ethereum/common" + "github.com/scroll-tech/go-ethereum/common/hexutil" + "github.com/scroll-tech/go-ethereum/crypto/kzg4844" +) + +type BlockNativeClient struct { + apiEndpoint string +} + +func NewBlockNativeClient(apiEndpoint string) *BlockNativeClient { + return &BlockNativeClient{ + apiEndpoint: apiEndpoint, + } +} + +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 { + 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) + } + defer resp.Body.Close() + if resp.StatusCode != http.StatusOK { + 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) + + // sanity check that retrieved blob matches versioned hash + commitment, err := kzg4844.BlobToCommitment(&blob) + if err != nil { + return nil, fmt.Errorf("failed to convert blob to commitment, err: %w", err) + } + + blobVersionedHash := kzg4844.CalcBlobHashV1(sha256.New(), &commitment) + if blobVersionedHash != versionedHash { + return nil, fmt.Errorf("blob versioned hash mismatch, expected: %s, got: %s", versionedHash.String(), hexutil.Encode(blobVersionedHash[:])) + } + + return &blob, nil +} + +type BlobRespBlockNative struct { + Blob struct { + 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/block_queue.go b/rollup/da_syncer/block_queue.go new file mode 100644 index 000000000000..a122d41ab356 --- /dev/null +++ b/rollup/da_syncer/block_queue.go @@ -0,0 +1,56 @@ +package da_syncer + +import ( + "context" + "fmt" + + "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 +} + +func NewBlockQueue(batchQueue *BatchQueue) *BlockQueue { + return &BlockQueue{ + batchQueue: batchQueue, + blocks: make([]*da.PartialBlock, 0), + } +} + +func (bq *BlockQueue) NextBlock(ctx context.Context) (*da.PartialBlock, 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 + } + + 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) + } + + bq.blocks = entryWithBlocks.Blocks() + + 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/calldata_blob_source.go b/rollup/da_syncer/da/calldata_blob_source.go new file mode 100644 index 000000000000..231cc4c1829e --- /dev/null +++ b/rollup/da_syncer/da/calldata_blob_source.go @@ -0,0 +1,241 @@ +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/da_syncer/serrors" + "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 of 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 + + l1Finalized uint64 +} + +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) { + var err error + to := ds.l1height + callDataBlobSourceFetchBlockRange + + // 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, 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) + } + + if ds.l1height > to { + return nil, ErrSourceExhausted + } + + logs, err := ds.l1Client.FetchRollupEventsInRange(ds.l1height, to) + if err != nil { + return nil, serrors.NewTemporaryError(fmt.Errorf("cannot get events, l1height: %d, error: %v", ds.l1height, err)) + } + da, err := ds.processLogsToDA(logs) + if err != nil { + return nil, serrors.NewTemporaryError(fmt.Errorf("failed to process logs to DA, error: %v", err)) + } + + ds.l1height = to + 1 + return da, nil +} + +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, 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) + } + + 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) + } + 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/commitV0.go b/rollup/da_syncer/da/commitV0.go new file mode 100644 index 000000000000..66a13786c9cb --- /dev/null +++ b/rollup/da_syncer/da/commitV0.go @@ -0,0 +1,173 @@ +package da + +import ( + "encoding/binary" + "fmt" + + "github.com/scroll-tech/da-codec/encoding" + "github.com/scroll-tech/da-codec/encoding/codecv0" + + "github.com/scroll-tech/go-ethereum/core/rawdb" + "github.com/scroll-tech/go-ethereum/core/types" + "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/serrors" +) + +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) 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 { + var blocks []*PartialBlock + l1TxPointer := 0 + + curL1TxIndex := c.parentTotalL1MessagePopped + for _, chunk := range c.chunks { + for blockId, daBlock := range chunk.Blocks { + // create txs + 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 := NewPartialBlock( + &PartialHeader{ + 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) + } + } + + return blocks +} + +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 { + // message not yet available + // 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++ + } + + 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..d94a046c81df --- /dev/null +++ b/rollup/da_syncer/da/commitV1.go @@ -0,0 +1,92 @@ +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.GetBlobByVersionedHashAndBlockNumber(ctx, versionedHash, vLog.BlockNumber) + 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) + 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/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 new file mode 100644 index 000000000000..5f00e86115a1 --- /dev/null +++ b/rollup/da_syncer/da/da.go @@ -0,0 +1,73 @@ +package da + +import ( + "math/big" + + "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 + // CommitBatchV4Type contains data of event of CommitBatchV2Type + CommitBatchV4Type + // RevertBatchType contains data of event of RevertBatchType + RevertBatchType + // FinalizeBatchType contains data of event of FinalizeBatchType + FinalizeBatchType +) + +// Entry represents a single DA event (commit, revert, finalize). +type Entry interface { + Type() Type + BatchIndex() uint64 + L1BlockNumber() uint64 + CompareTo(Entry) int +} + +type EntryWithBlocks interface { + Entry + Blocks() []*PartialBlock +} + +type Entries []Entry + +// PartialHeader represents a partial header (from DA) of a block. +type PartialHeader struct { + Number uint64 + Time uint64 + BaseFee *big.Int + GasLimit uint64 + Difficulty uint64 + 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: new(big.Int).SetUint64(h.Difficulty), + Extra: h.ExtraData, + } +} + +// PartialBlock represents a partial block (from DA). +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/finalize.go b/rollup/da_syncer/da/finalize.go new file mode 100644 index 000000000000..14d6c2a644cb --- /dev/null +++ b/rollup/da_syncer/da/finalize.go @@ -0,0 +1,34 @@ +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 +} + +func (f *FinalizeBatch) CompareTo(other Entry) int { + if f.BatchIndex() < other.BatchIndex() { + return -1 + } else if f.BatchIndex() > other.BatchIndex() { + return 1 + } + return 0 +} diff --git a/rollup/da_syncer/da/revert.go b/rollup/da_syncer/da/revert.go new file mode 100644 index 000000000000..d84f22ebaa7b --- /dev/null +++ b/rollup/da_syncer/da/revert.go @@ -0,0 +1,33 @@ +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 +} + +func (r *RevertBatch) CompareTo(other Entry) int { + if r.BatchIndex() < other.BatchIndex() { + return -1 + } else if r.BatchIndex() > other.BatchIndex() { + return 1 + } + return 0 +} diff --git a/rollup/da_syncer/da_queue.go b/rollup/da_syncer/da_queue.go new file mode 100644 index 000000000000..64673a4a646b --- /dev/null +++ b/rollup/da_syncer/da_queue.go @@ -0,0 +1,70 @@ +package da_syncer + +import ( + "context" + "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. +type DAQueue struct { + l1height uint64 + dataSourceFactory *DataSourceFactory + dataSource DataSource + da da.Entries +} + +func NewDAQueue(l1height uint64, dataSourceFactory *DataSourceFactory) *DAQueue { + return &DAQueue{ + l1height: l1height, + dataSourceFactory: dataSourceFactory, + dataSource: nil, + da: make(da.Entries, 0), + } +} + +func (dq *DAQueue) NextDA(ctx context.Context) (da.Entry, error) { + for 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() + if err == nil { + return nil + } + + // previous dataSource has been exhausted, create new + if errors.Is(err, da.ErrSourceExhausted) { + dq.l1height = dq.dataSource.L1Height() + dq.dataSource = nil + + // we return EOFError to be handled in pipeline + return serrors.EOFError + } + + 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 new file mode 100644 index 000000000000..c3c223ff22a9 --- /dev/null +++ b/rollup/da_syncer/da_syncer.go @@ -0,0 +1,49 @@ +package da_syncer + +import ( + "fmt" + + "github.com/scroll-tech/go-ethereum/core" + "github.com/scroll-tech/go-ethereum/log" + "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 +} + +func NewDASyncer(blockchain *core.BlockChain) *DASyncer { + return &DASyncer{ + blockchain: blockchain, + } +} + +// 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 { + currentBlock := s.blockchain.CurrentBlock() + + // we expect blocks to be consecutive. block.PartialHeader.Number == parentBlock.Number+1. + if block.PartialHeader.Number <= currentBlock.Number.Uint64() { + log.Debug("block number is too low", "block number", block.PartialHeader.Number, "parent block number", currentBlock.Number.Uint64()) + return ErrBlockTooLow + } else if block.PartialHeader.Number > currentBlock.Number.Uint64()+1 { + log.Debug("block number is too high", "block number", block.PartialHeader.Number, "parent block number", currentBlock.Number.Uint64()) + return ErrBlockTooHigh + } + + parentBlock := s.blockchain.GetBlockByNumber(currentBlock.Number.Uint64()) + 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().Number.Uint64()%1000 == 0 { + log.Info("L1 sync progress", "blockhain height", s.blockchain.CurrentBlock().Number.Uint64(), "block hash", s.blockchain.CurrentBlock().Hash(), "root", s.blockchain.CurrentBlock().Root) + } + + return nil +} diff --git a/rollup/da_syncer/data_source.go b/rollup/da_syncer/data_source.go new file mode 100644 index 000000000000..f417d09af00e --- /dev/null +++ b/rollup/da_syncer/data_source.go @@ -0,0 +1,44 @@ +package da_syncer + +import ( + "context" + "errors" + + "github.com/scroll-tech/go-ethereum/core" + "github.com/scroll-tech/go-ethereum/ethdb" + "github.com/scroll-tech/go-ethereum/params" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/blob_client" + "github.com/scroll-tech/go-ethereum/rollup/da_syncer/da" + "github.com/scroll-tech/go-ethereum/rollup/rollup_sync_service" +) + +type DataSource interface { + NextData() (da.Entries, error) + L1Height() uint64 +} + +type DataSourceFactory struct { + config Config + genesisConfig *params.ChainConfig + l1Client *rollup_sync_service.L1Client + blobClient blob_client.BlobClient + db ethdb.Database +} + +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, + l1Client: l1Client, + blobClient: blobClient, + db: db, + } +} + +func (ds *DataSourceFactory) OpenDataSource(ctx context.Context, l1height uint64) (DataSource, error) { + if ds.config.FetcherMode == L1RPC { + return da.NewCalldataBlobSource(ctx, l1height, ds.l1Client, ds.blobClient, ds.db) + } else { + return nil, errors.New("snapshot_data_source: not implemented") + } +} diff --git a/rollup/da_syncer/modes.go b/rollup/da_syncer/modes.go new file mode 100644 index 000000000000..bfcc1d1dfba0 --- /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 "l1rpc" or "snapshot"`, text) + } + return nil +} 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 new file mode 100644 index 000000000000..6795f2608e05 --- /dev/null +++ b/rollup/da_syncer/syncing_pipeline.go @@ -0,0 +1,233 @@ +package da_syncer + +import ( + "context" + "errors" + "fmt" + "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" + "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" +) + +// Config is the configuration parameters of data availability syncing. +type Config struct { + FetcherMode FetcherMode // mode of fetcher + 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 +// L2 blocks and chain. +type SyncingPipeline struct { + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + expBackoff *backoff.Exponential + + l1DeploymentBlock uint64 + + 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) { + scrollChainABI, err := rollup_sync_service.ScrollChainMetaData.GetAbi() + if err != nil { + 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 { + return nil, err + } + + blobClientList := blob_client.NewBlobClients() + 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)) + } + if config.BlockNativeAPIEndpoint != "" { + blobClientList.AddBlobClient(blob_client.NewBlockNativeClient(config.BlockNativeAPIEndpoint)) + } + if blobClientList.Size() == 0 { + return nil, errors.New("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, blobClientList, db) + syncedL1Height := l1DeploymentBlock - 1 + from := rawdb.ReadDASyncedL1BlockNumber(db) + if from != nil { + syncedL1Height = *from + } + + 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{}, + l1DeploymentBlock: l1DeploymentBlock, + db: db, + blockchain: blockchain, + blockQueue: blockQueue, + daSyncer: daSyncer, + }, nil +} + +func (s *SyncingPipeline) Step() error { + block, err := s.blockQueue.NextBlock(s.ctx) + if err != nil { + return err + } + err = s.daSyncer.SyncOneBlock(block) + return err +} + +func (s *SyncingPipeline) Start() { + log.Info("sync from DA: starting pipeline") + + s.wg.Add(1) + go func() { + s.mainLoop() + s.wg.Done() + }() +} + +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. + 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 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 { + // step succeeded, reset exponential backoff and continue + reqStep(false) + s.expBackoff.Reset() + resetCounter = 0 + tempErrorCounter = 0 + continue + } + + 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 + } 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, + // reset the pipeline and move backwards from the last L1 block we read + s.reset(resetCounter) + resetCounter++ + reqStep(false) + tempErrorCounter = 0 + continue + } else if errors.Is(err, context.Canceled) { + log.Info("syncing pipeline stopped due to cancelled context", "err", err) + return + } + + log.Warn("syncing pipeline step failed due to unrecoverable error, stopping pipeline worker", "err", err) + return + } + } +} + +func (s *SyncingPipeline) Stop() { + log.Info("sync from DA: stopping pipeline...") + s.cancel() + s.wg.Wait() + log.Info("sync from DA: stopping pipeline... 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) +} 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..4f636102ddee 100644 --- a/rollup/rollup_sync_service/l1client.go +++ b/rollup/rollup_sync_service/l1client.go @@ -27,9 +27,9 @@ 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) { +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") } @@ -55,8 +55,8 @@ 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]. -func (c *L1Client) fetchRollupEventsInRange(from, to uint64) ([]types.Log, error) { +// 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) query := ethereum.FilterQuery{ @@ -79,8 +79,8 @@ 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. -func (c *L1Client) getLatestFinalizedBlockNumber() (uint64, error) { +// 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 { return 0, err @@ -90,3 +90,69 @@ 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 +} + +// 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 +} 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 ec2ca9a85677..6838342adcf5 100644 --- a/rollup/rollup_sync_service/rollup_sync_service.go +++ b/rollup/rollup_sync_service/rollup_sync_service.go @@ -77,12 +77,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) } @@ -175,7 +175,7 @@ func (s *RollupSyncService) fetchRollupEvents() { s.stateMu.Lock() defer s.stateMu.Unlock() - 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 @@ -195,7 +195,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 a045787d6367..83b8c72c3d15 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 TestDecodeBatchVersionAndChunkBlockRangesCodecv0(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -110,7 +110,7 @@ func TestDecodeBatchVersionAndChunkBlockRangesCodecv0(t *testing.T) { } func TestDecodeBatchVersionAndChunkBlockRangesCodecv1(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -163,7 +163,7 @@ func TestDecodeBatchVersionAndChunkBlockRangesCodecv1(t *testing.T) { } func TestDecodeBatchVersionAndChunkBlockRangesCodecv2(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{ @@ -216,7 +216,7 @@ func TestDecodeBatchVersionAndChunkBlockRangesCodecv2(t *testing.T) { } func TestDecodeBatchVersionAndChunkBlockRangesCodecv3(t *testing.T) { - scrollChainABI, err := scrollChainMetaData.GetAbi() + scrollChainABI, err := ScrollChainMetaData.GetAbi() require.NoError(t, err) service := &RollupSyncService{