From d14ed100e8014d2110496b30dccf7fd3fea92d22 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 15:31:08 -0800 Subject: [PATCH 001/126] add missing collection queue --- .../access/ingestion2/collection_syncing.go | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 engine/access/ingestion2/collection_syncing.go diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go new file mode 100644 index 00000000000..4258f3f3993 --- /dev/null +++ b/engine/access/ingestion2/collection_syncing.go @@ -0,0 +1,28 @@ +package ingestion2 + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/irrecoverable" +) + +// Tracks missing collections per height and invokes job callbacks when complete. +type MissingCollectionQueue interface { + EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error + OnIndexedForBlock(blockHeight uint64) // mark done (post‑indexing) + + // On receipt of a collection, MCQ updates internal state and, if a block + // just became complete, returns: (collections, height, true). + // Otherwise, returns (nil, 0, false). + OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, bool) +} + +type CollectionRequester interface { + RequestCollections(ids []flow.Identifier) error +} + +// Implements the job lifecycle for a single block height. +type JobProcessor interface { + ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error + OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester +} From 12339f505fa904932fd5c2a7026146be4d699e71 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 16:36:21 -0800 Subject: [PATCH 002/126] add ingestion2 collection syncer --- engine/access/ingestion2/README.md | 146 +++++++ .../access/ingestion2/collection_syncing.go | 112 ++++++ .../collections/block_collection_indexer.go | 76 ++++ .../ingestion2/collections/job_processor.go | 208 ++++++++++ .../collections/missing_collection_queue.go | 207 ++++++++++ .../missing_collection_queue_test.go | 372 ++++++++++++++++++ .../ingestion2/collections/requester.go | 79 ++++ .../ingestion2/collections/syncer_factory.go | 151 +++++++ engine/access/ingestion2/engine.go | 11 +- engine/access/ingestion2/engine_test.go | 37 +- 10 files changed, 1373 insertions(+), 26 deletions(-) create mode 100644 engine/access/ingestion2/README.md create mode 100644 engine/access/ingestion2/collections/block_collection_indexer.go create mode 100644 engine/access/ingestion2/collections/job_processor.go create mode 100644 engine/access/ingestion2/collections/missing_collection_queue.go create mode 100644 engine/access/ingestion2/collections/missing_collection_queue_test.go create mode 100644 engine/access/ingestion2/collections/requester.go create mode 100644 engine/access/ingestion2/collections/syncer_factory.go diff --git a/engine/access/ingestion2/README.md b/engine/access/ingestion2/README.md new file mode 100644 index 00000000000..f121d8554c9 --- /dev/null +++ b/engine/access/ingestion2/README.md @@ -0,0 +1,146 @@ +Asynchronous Collection Indexing — Design + +1) Purpose & Outcomes + +Goal. Index block collections reliably without overloading the node, even when finalized blocks arrive faster than we can index. + +Outcomes. + +Workers (job processors) stay focused on a small, sliding window of heights just above the latest indexed height. + +Collection retrieval and indexing are decoupled from finalization and from each other. + +Execution Data Indexer (EDI) provides the preferred source for collections; collection requests are only sent when EDI falls behind a configurable threshold. + +2) High-Level Flow + +Finalization happens → a lazy signal wakes the Job Consumer. + +Job Consumer consults: (a) Progress Tracker → highest indexed height; (b) Jobs module → latest safe (head) height. It computes a bounded work window [indexed+1 .. min(indexed+N, head)]. + +For each height in the window, the consumer spins up (or reuses) a Job Processor. + +The Job Processor checks if the block is already indexed. If yes → finish immediately. If not: + +It enqueues the block’s missing collection IDs into MissingCollectionQueue (MCQ) with a completion callback. + +It checks how far EDI lags behind the block height. If the lag exceeds a configured threshold, it triggers CollectionRequester to fetch collections; otherwise, it waits for EDI to deliver them. + +As collections arrive (from requester or EDI), the Job Processor forwards them to MCQ. When MCQ detects a block is now complete, the processor passes the collections to BlockCollectionIndexer to store+index, then calls MCQ to mark the job done. + +Jobs may complete out of order; the progress tracker advances once any gaps below are closed. + +3) Core Interfaces + +BlockCollectionIndexer + +Stores and indexes collections for a given block height; provides the latest indexed height for windowing and fast no-op checks. + +type BlockCollectionIndexer interface { + LatestIndexedHeight() uint64 + + // If blockHeight <= LatestIndexedHeight(), return quickly. + // Otherwise: lock, re-check, then persist+index collections. + // Double-check pattern minimizes lock contention. + OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error +} + +MissingCollectionQueue (MCQ) + +In-memory coordinator for jobs and callbacks. MCQ does not index; it only tracks missing collections per height and announces when a height becomes complete. + +type MissingCollectionQueue interface { + EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error + OnIndexedForBlock(blockHeight uint64) + // On receipt of a collection, MCQ updates internal state and, if a block + // just became complete, returns: (collections, height, true). + // Otherwise, returns (nil, 0, false). + OnReceivedCollection(collectionID flow.Identifier) ([]*flow.Collection, uint64, bool) +} + +CollectionRequester + +Abstracts the engine that requests collections by ID (e.g., from collection nodes). + +type CollectionRequester interface { + RequestCollections(ids []flow.Identifier) error +} + +JobProcessor + +Owns the state of ongoing jobs (delegated to MCQ) and orchestrates request → receive → index → complete. + +type JobProcessor interface { + ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error + OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error +} + +4) Job Consumer (Windowed Throttling) + +Why: Prevent node overload when finalized heights advance rapidly. + +How: + +Reads latestIndexed = BlockCollectionIndexer.LatestIndexedHeight(). + +Reads head = Jobs.Head() (latest height safe to work on). + +Defines window size K. Range: [latestIndexed+1 .. min(latestIndexed+K, head)]. + +Assigns one JobProcessor per height in this range. + +Lazy notification: Finalization pushes a single, coalescing signal to workSignal (buffer size 1). The consumer wakes, recomputes the range, and may ignore new jobs if already at capacity. + +5) Job Lifecycle + +Spawn/Assign. Consumer gives (height, job, doneCb) to a JobProcessor. + +Already Indexed? Processor queries storage. If yes → doneCb() and return. + +Track Missing. MCQ.EnqueueMissingCollections(height, collectionIDs, doneCb). + +Check EDI Lag. Compare height with ediIndexedHeight(). If lag ≤ threshold, wait for EDI; if lag > threshold, trigger CollectionRequester.RequestCollections(ids). + +Receive Collections. + +Processor calls MCQ.OnReceivedCollection(id). + +If complete, processor calls BlockCollectionIndexer.OnReceivedCollectionsForBlock(height, cols) and then MCQ.OnIndexedForBlock(height). + +Progress Advancement. Out-of-order completion allowed; progress advances only when lower gaps close. + +Crash/Restart. On restart, re-created jobs short-circuit if already indexed. + +6) Execution Data Indexer (EDI) Integration + +EDI serves as the primary source of collections. The system dynamically decides whether to fetch collections based on EDI’s progress. + +Lag-based hybrid logic: + +Track ediIndexedHeight() — the latest height for which EDI has collections. + +Define a lag threshold, EDILagThreshold, in number of blocks. + +For block height h: + +If (h - ediIndexedHeight()) <= EDILagThreshold: rely on EDI; no fetching. + +If (h - ediIndexedHeight()) > EDILagThreshold: trigger the collection fetcher to request collections from nodes. + +Behavior summary: + +If EDI is up to date or within threshold → no fetches. + +If EDI is behind beyond threshold → start fetching. + +Setting a very large threshold effectively mimics the previous EDI-only mode. + +Why EDI goes through JobProcessor: To keep job state consistent—update MCQ, suppress redundant fetches, and advance the queue. + +Contention handling: + +If EDI lags → fetching fills the gap. + +If EDI leads → consumer advances; requester stops fetching. + +If close → minimal contention; indexing is faster than receipt. diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go index 4258f3f3993..af9945b99be 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/ingestion2/collection_syncing.go @@ -1,9 +1,18 @@ package ingestion2 import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" ) // Tracks missing collections per height and invokes job callbacks when complete. @@ -17,12 +26,115 @@ type MissingCollectionQueue interface { OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, bool) } +// Requests collections by their IDs. type CollectionRequester interface { RequestCollections(ids []flow.Identifier) error } +// BlockCollectionIndexer stores and indexes collections for a given block height. +type BlockCollectionIndexer interface { + // OnReceivedCollectionsForBlock stores and indexes collections for a given block height. + OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error +} + // Implements the job lifecycle for a single block height. type JobProcessor interface { ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester } + +// EDIHeightProvider provides the latest height for which execution data indexer has collections. +// This can be nil if execution data indexing is disabled. +type EDIHeightProvider interface { + HighestIndexedHeight() (uint64, error) +} + +// Syncer is a component that consumes finalized block jobs and processes them +// to index collections. It uses a job consumer with windowed throttling to prevent node overload. +type Syncer struct { + component.Component + + consumer *jobqueue.ComponentConsumer + jobProcessor JobProcessor + workSignal engine.Notifier +} + +// NewSyncer creates a new Syncer component. +// +// Parameters: +// - log: Logger for the component +// - jobProcessor: JobProcessor implementation for processing collection indexing jobs +// - progressInitializer: Initializer for tracking processed block heights +// - state: Protocol state for reading finalized block information +// - blocks: Blocks storage for reading blocks by height +// - maxProcessing: Maximum number of jobs to process concurrently +// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit +// +// No error returns are expected during normal operation. +func NewSyncer( + log zerolog.Logger, + jobProcessor JobProcessor, + progressInitializer storage.ConsumerProgressInitializer, + state protocol.State, + blocks storage.Blocks, + maxProcessing uint64, // max number of blocks to fetch collections + maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for +) (*Syncer, error) { + workSignal := engine.NewNotifier() + + // Read the default index from the finalized root height + defaultIndex := state.Params().FinalizedRoot().Height + + // Create a Jobs instance that reads finalized blocks by height + jobs := jobqueue.NewFinalizedBlockReader(state, blocks) + + // Create an adapter function that wraps the JobProcessor interface + processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { + err := jobProcessor.ProcessJob(ctx, job, done) + if err != nil { + ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) + } + } + + consumer, err := jobqueue.NewComponentConsumer( + log.With().Str("component", "collection-syncing").Logger(), + workSignal.Channel(), + progressInitializer, + jobs, + defaultIndex, + processorFunc, + maxProcessing, + maxSearchAhead, + ) + if err != nil { + return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) + } + + return &Syncer{ + Component: consumer, + consumer: consumer, + jobProcessor: jobProcessor, + workSignal: workSignal, + }, nil +} + +// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer +// that new work is available. +func (s *Syncer) OnFinalizedBlock() { + s.workSignal.Notify() +} + +// LastProcessedIndex returns the last processed job index. +func (s *Syncer) LastProcessedIndex() uint64 { + return s.consumer.LastProcessedIndex() +} + +// Head returns the highest job index available. +func (s *Syncer) Head() (uint64, error) { + return s.consumer.Head() +} + +// Size returns the number of in-memory jobs that the consumer is processing. +func (s *Syncer) Size() uint { + return s.consumer.Size() +} diff --git a/engine/access/ingestion2/collections/block_collection_indexer.go b/engine/access/ingestion2/collections/block_collection_indexer.go new file mode 100644 index 00000000000..b823a73ce87 --- /dev/null +++ b/engine/access/ingestion2/collections/block_collection_indexer.go @@ -0,0 +1,76 @@ +package collections + +import ( + "fmt" + + "github.com/jordanschalm/lockctx" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/storage" +) + +// blockCollectionIndexerImpl implements BlockCollectionIndexer. +// It stores and indexes collections for a given block height. +type blockCollectionIndexerImpl struct { + metrics module.CollectionExecutedMetric + lockManager lockctx.Manager + db storage.DB + collections storage.Collections +} + +var _ ingestion2.BlockCollectionIndexer = (*blockCollectionIndexerImpl)(nil) + +// NewBlockCollectionIndexer creates a new BlockCollectionIndexer implementation. +// +// Parameters: +// - metrics: Metrics collector for tracking collection indexing +// - lockManager: Lock manager for coordinating database access +// - db: Database for storage operations +// - collections: Collections storage for storing and indexing collections +// +// No error returns are expected during normal operation. +func NewBlockCollectionIndexer( + metrics module.CollectionExecutedMetric, + lockManager lockctx.Manager, + db storage.DB, + collections storage.Collections, +) *blockCollectionIndexerImpl { + return &blockCollectionIndexerImpl{ + metrics: metrics, + lockManager: lockManager, + db: db, + collections: collections, + } +} + +// OnReceivedCollectionsForBlock stores and indexes collections for a given block height. +// +// No error returns are expected during normal operation. +func (bci *blockCollectionIndexerImpl) OnReceivedCollectionsForBlock( + blockHeight uint64, + cols []*flow.Collection, +) error { + // Store and index collections + err := storage.WithLock(bci.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return bci.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + for _, collection := range cols { + // Store the collection, including constituent transactions, and index transactionID -> collectionID + light, err := bci.collections.BatchStoreAndIndexByTransaction(lctx, collection, rw) + if err != nil { + return fmt.Errorf("failed to store collection: %w", err) + } + + bci.metrics.CollectionFinalized(light) + bci.metrics.CollectionExecuted(light) + } + return nil + }) + }) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", blockHeight, err) + } + + return nil +} diff --git a/engine/access/ingestion2/collections/job_processor.go b/engine/access/ingestion2/collections/job_processor.go new file mode 100644 index 00000000000..667f0f94a12 --- /dev/null +++ b/engine/access/ingestion2/collections/job_processor.go @@ -0,0 +1,208 @@ +package collections + +import ( + "errors" + "fmt" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/storage" +) + +// EDIHeightProvider provides the latest height for which execution data indexer has collections. +type EDIHeightProvider interface { + // HighestIndexedHeight returns the highest block height for which EDI has indexed collections. + HighestIndexedHeight() (uint64, error) +} + +// JobProcessor implements the job lifecycle for collection indexing. +// It orchestrates the flow: request → receive → index → complete. +type JobProcessor struct { + mcq ingestion2.MissingCollectionQueue + indexer ingestion2.BlockCollectionIndexer + requester ingestion2.CollectionRequester + blocks storage.Blocks + collections storage.CollectionsReader + ediHeightProvider EDIHeightProvider + ediLagThreshold uint64 +} + +var _ ingestion2.JobProcessor = (*JobProcessor)(nil) + +// NewJobProcessor creates a new JobProcessor. +// +// Parameters: +// - mcq: MissingCollectionQueue for tracking missing collections and callbacks +// - indexer: BlockCollectionIndexer for storing and indexing collections +// - requester: CollectionRequester for requesting collections from the network +// - blocks: Blocks storage for retrieving block data +// - collections: Collections storage reader for checking if collections already exist +// - ediHeightProvider: Provider for EDI's highest indexed height (can be nil if EDI is disabled) +// - ediLagThreshold: Threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. +// Set to a very large value to effectively disable fetching and rely only on EDI. +// +// No error returns are expected during normal operation. +func NewJobProcessor( + mcq ingestion2.MissingCollectionQueue, + indexer ingestion2.BlockCollectionIndexer, + requester ingestion2.CollectionRequester, + blocks storage.Blocks, + collections storage.CollectionsReader, + ediHeightProvider EDIHeightProvider, + ediLagThreshold uint64, +) *JobProcessor { + return &JobProcessor{ + mcq: mcq, + indexer: indexer, + requester: requester, + blocks: blocks, + collections: collections, + ediHeightProvider: ediHeightProvider, + ediLagThreshold: ediLagThreshold, + } +} + +// ProcessJob processes a job for a finalized block. +// It checks if the block is already indexed, and if not, enqueues missing collections +// and optionally requests them based on EDI lag. +// +// No error returns are expected during normal operation. +func (jp *JobProcessor) ProcessJob( + ctx irrecoverable.SignalerContext, + job module.Job, + done func(), +) error { + // Convert job to block + block, err := jobqueue.JobToBlock(job) + if err != nil { + return fmt.Errorf("could not convert job to block: %w", err) + } + + blockHeight := block.Height + + // Get missing collections for this block + missingGuarantees, err := jp.getMissingCollections(blockHeight) + if err != nil { + return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) + } + + // If no missing collections, block is complete + if len(missingGuarantees) == 0 { + done() + return nil + } + + // Extract collection IDs + collectionIDs := make([]flow.Identifier, len(missingGuarantees)) + for i, guarantee := range missingGuarantees { + collectionIDs[i] = guarantee.CollectionID + } + + // Enqueue missing collections with callback + callback := func() { + // When all collections are received and indexed, mark the job as done + done() + } + + err = jp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + if err != nil { + return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) + } + + // Check EDI lag to decide whether to fetch collections + shouldFetch, err := jp.shouldFetchCollections(blockHeight) + if err != nil { + return fmt.Errorf("failed to check EDI lag for block height %d: %w", blockHeight, err) + } + + if shouldFetch { + // Request collections from collection nodes + err = jp.requester.RequestCollections(collectionIDs) + if err != nil { + return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) + } + } + + return nil +} + +// OnReceivedCollectionsForBlock is called by the execution data indexer when collections are received. +// It forwards collections to MCQ and handles block completion. +// +// The blockHeight parameter is provided by EDI to indicate which block these collections belong to. +// Collections are forwarded individually to MCQ, which tracks completion per block. +// +// No error returns are expected during normal operation. +func (jp *JobProcessor) OnReceivedCollectionsForBlock( + blockHeight uint64, + collections []*flow.Collection, +) error { + // Forward each collection to MCQ + // MCQ will track which block each collection belongs to and detect when a block becomes complete + for _, collection := range collections { + receivedCols, height, complete := jp.mcq.OnReceivedCollection(collection) + if complete { + // Block became complete, index it + err := jp.indexer.OnReceivedCollectionsForBlock(height, receivedCols) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", height, err) + } + + // Notify MCQ that the block has been indexed (this invokes the callback) + jp.mcq.OnIndexedForBlock(height) + } + } + + return nil +} + +// getMissingCollections retrieves the block and returns collection guarantees that are missing. +// Only collections that are not already in storage are returned. +func (jp *JobProcessor) getMissingCollections(blockHeight uint64) ([]*flow.CollectionGuarantee, error) { + block, err := jp.blocks.ByHeight(blockHeight) + if err != nil { + return nil, fmt.Errorf("failed to retrieve block at height %d: %w", blockHeight, err) + } + + var missingGuarantees []*flow.CollectionGuarantee + for _, guarantee := range block.Payload.Guarantees { + // Check if collection already exists in storage + _, err := jp.collections.LightByID(guarantee.CollectionID) + if err != nil { + if errors.Is(err, storage.ErrNotFound) { + // Collection is missing + missingGuarantees = append(missingGuarantees, guarantee) + } else { + // Unexpected error + return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) + } + } + // If collection exists, skip it + } + + return missingGuarantees, nil +} + +// shouldFetchCollections determines whether to fetch collections based on EDI lag. +// Returns true if collections should be fetched, false if we should wait for EDI. +func (jp *JobProcessor) shouldFetchCollections(blockHeight uint64) (bool, error) { + // If EDI is not available, always fetch + if jp.ediHeightProvider == nil { + return true, nil + } + + ediHeight, err := jp.ediHeightProvider.HighestIndexedHeight() + if err != nil { + // If we can't get EDI height, err on the side of fetching to avoid blocking + return true, nil + } + + // Calculate lag + lag := blockHeight - ediHeight + + // If lag exceeds threshold, fetch collections + return lag > jp.ediLagThreshold, nil +} diff --git a/engine/access/ingestion2/collections/missing_collection_queue.go b/engine/access/ingestion2/collections/missing_collection_queue.go new file mode 100644 index 00000000000..f136dbab8ef --- /dev/null +++ b/engine/access/ingestion2/collections/missing_collection_queue.go @@ -0,0 +1,207 @@ +package collections + +import ( + "fmt" + "sync" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/model/flow" +) + +// blockJobState tracks the state of a job for a specific block height. +type blockJobState struct { + // missingCollections is a set of collection IDs that are still missing for this block height. + missingCollections map[flow.Identifier]struct{} + // receivedCollections stores the collections that have been received so far, keyed by collection ID. + // This allows us to return all collections when the block becomes complete. + receivedCollections map[flow.Identifier]*flow.Collection + // callback is invoked when all collections for this block height have been received and indexed. + callback func() +} + +// MissingCollectionQueue helps the job processor to keep track of the jobs and their callbacks. +// Note, it DOES NOT index collections directly, instead, it only keeps track of which collections are missing +// for each block height, and when all collections for a block height have been received, it returns the +// collections to the caller for processing (storing and indexing). And let the caller to notify the completion +// of the processing, so that it can mark the job as done by calling the callback. +// This allows the MissingCollectionQueue to be decoupled from the actual processing of the collections, keep +// all states in memory and allow the different callers to hold the lock less time and reduce contention. +// +// The caller is responsible for checking if collections are already in storage before enqueueing them. +// Only collections that are actually missing should be passed to EnqueueMissingCollections. +// +// MissingCollectionQueue is safe for concurrent use. +type MissingCollectionQueue struct { + // mu protects all the maps below. + mu sync.RWMutex + + // blockJobs maps block height to the job state for that height. + blockJobs map[uint64]*blockJobState + + // collectionToHeight maps collection ID to the single block height waiting for that collection. + // This enforces a 1:1 relationship: each collection belongs to exactly one block. + // This allows efficient lookup when a collection is received. + collectionToHeight map[flow.Identifier]uint64 +} + +var _ ingestion2.MissingCollectionQueue = (*MissingCollectionQueue)(nil) + +// NewMissingCollectionQueue creates a new MissingCollectionQueue. +// +// No error returns are expected during normal operation. +func NewMissingCollectionQueue() *MissingCollectionQueue { + return &MissingCollectionQueue{ + blockJobs: make(map[uint64]*blockJobState), + collectionToHeight: make(map[flow.Identifier]uint64), + } +} + +// EnqueueMissingCollections registers missing collections for a block height along with a callback +// that will be invoked when all collections for that height have been received and indexed. +// +// The caller is responsible for checking if collections are already in storage before calling this method. +// Only collections that are actually missing should be passed in collectionIDs. +// +// If the same block height is enqueued multiple times, the previous callback is replaced. +// +// No error returns are expected during normal operation. +func (mcq *MissingCollectionQueue) EnqueueMissingCollections( + blockHeight uint64, + collectionIDs []flow.Identifier, + callback func(), +) error { + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Create the job state with all collections marked as missing. + // The caller has already verified these collections are not in storage. + missingSet := make(map[flow.Identifier]struct{}, len(collectionIDs)) + for _, id := range collectionIDs { + missingSet[id] = struct{}{} + } + + jobState := &blockJobState{ + missingCollections: missingSet, + receivedCollections: make(map[flow.Identifier]*flow.Collection), + callback: callback, + } + + mcq.blockJobs[blockHeight] = jobState + + // Update the collection-to-height mapping, enforcing 1:1 relationship. + for _, collectionID := range collectionIDs { + existingHeight, exists := mcq.collectionToHeight[collectionID] + if exists && existingHeight != blockHeight { + // Collection is already assigned to a different block - this violates the 1:1 constraint. + return fmt.Errorf( + "collection %v is already assigned to block height %d, cannot assign to height %d", + collectionID, existingHeight, blockHeight, + ) + } + mcq.collectionToHeight[collectionID] = blockHeight + } + + return nil +} + +// OnReceivedCollection notifies the queue that a collection has been received. +// It checks if the block height is now complete and returns the collections and height. +// +// The collection parameter should be the actual collection object received from the requester. +// +// Returns: +// - (collections, height, true) if the block height became complete +// - (nil, 0, false) if no block height became complete +// +// No error returns are expected during normal operation. +func (mcq *MissingCollectionQueue) OnReceivedCollection( + collection *flow.Collection, +) ([]*flow.Collection, uint64, bool) { + collectionID := collection.ID() + + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Find the block height waiting for this collection (1:1 relationship). + height, ok := mcq.collectionToHeight[collectionID] + if !ok { + // No block is waiting for this collection. + return nil, 0, false + } + + jobState, exists := mcq.blockJobs[height] + if !exists { + // Job was already completed/removed, clean up the mapping. + delete(mcq.collectionToHeight, collectionID) + return nil, 0, false + } + + // Check if this collection was still missing for this block. + if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { + // Collection was already received or wasn't part of this block's missing set. + // Clean up the mapping since we've already processed this collection. + delete(mcq.collectionToHeight, collectionID) + return nil, 0, false + } + + // Remove from missing set and add to received collections. + delete(jobState.missingCollections, collectionID) + // Store the collection so it can be returned when the block is complete. + jobState.receivedCollections[collectionID] = collection + + // Remove this collection from the collection-to-height mapping since we've processed it. + delete(mcq.collectionToHeight, collectionID) + + // Check if the block is now complete (all collections received). + if len(jobState.missingCollections) == 0 { + // Return all received collections for this block. + collections := make([]*flow.Collection, 0, len(jobState.receivedCollections)) + for _, col := range jobState.receivedCollections { + collections = append(collections, col) + } + return collections, height, true + } + + return nil, 0, false +} + +// OnIndexedForBlock notifies the queue that a block height has been indexed. +// This invokes the callback for that block height and removes it from tracking. +// +// No error returns are expected during normal operation. +func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) { + mcq.mu.Lock() + + jobState, exists := mcq.blockJobs[blockHeight] + if !exists { + // Block was not tracked or already completed (callback already called and job removed). + mcq.mu.Unlock() + return + } + + // Get the callback before removing the job. + callback := jobState.callback + + // Clean up all collection-to-height mappings for collections belonging to this block. + // Clean up from missing collections. + for collectionID := range jobState.missingCollections { + if height, ok := mcq.collectionToHeight[collectionID]; ok && height == blockHeight { + delete(mcq.collectionToHeight, collectionID) + } + } + // Clean up from received collections. + for collectionID := range jobState.receivedCollections { + if height, ok := mcq.collectionToHeight[collectionID]; ok && height == blockHeight { + delete(mcq.collectionToHeight, collectionID) + } + } + + // Remove the job state from the queue before calling the callback. + // This ensures the height is removed from tracking once the callback is invoked. + delete(mcq.blockJobs, blockHeight) + + // Release the lock before calling the callback to avoid holding it during callback execution. + mcq.mu.Unlock() + callback() + // Note: We manually unlocked above, so we don't use defer here to avoid double-unlock. +} diff --git a/engine/access/ingestion2/collections/missing_collection_queue_test.go b/engine/access/ingestion2/collections/missing_collection_queue_test.go new file mode 100644 index 00000000000..682f2d5267e --- /dev/null +++ b/engine/access/ingestion2/collections/missing_collection_queue_test.go @@ -0,0 +1,372 @@ +package collections + +import ( + "sync" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestMissingCollectionQueue_EnqueueMissingCollections(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackCalled := false + callback := func() { + callbackCalled = true + } + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + require.False(t, callbackCalled, "callback should not be called yet") +} + +func TestMissingCollectionQueue_OnReceivedCollection_CompleteBlock(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackCalled := false + callback := func() { + callbackCalled = true + } + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + + // Receive first two collections - block should not be complete yet + receivedCols, height, complete := mcq.OnReceivedCollection(collections[0]) + require.False(t, complete) + require.Nil(t, receivedCols) + require.Equal(t, uint64(0), height) + + receivedCols, height, complete = mcq.OnReceivedCollection(collections[1]) + require.False(t, complete) + require.Nil(t, receivedCols) + require.Equal(t, uint64(0), height) + + // Receive last collection - block should now be complete + receivedCols, height, complete = mcq.OnReceivedCollection(collections[2]) + require.True(t, complete) + require.NotNil(t, receivedCols) + require.Equal(t, blockHeight, height) + require.Len(t, receivedCols, 3) + + // Verify all collections are present + receivedMap := make(map[flow.Identifier]*flow.Collection) + for _, col := range receivedCols { + receivedMap[col.ID()] = col + } + for _, expectedCol := range collections { + require.Contains(t, receivedMap, expectedCol.ID()) + require.Equal(t, expectedCol, receivedMap[expectedCol.ID()]) + } + + require.False(t, callbackCalled, "callback should not be called until OnIndexedForBlock") +} + +func TestMissingCollectionQueue_OnReceivedCollection_UntrackedCollection(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + collection := unittest.CollectionFixture(1) + + // Receive collection that was never enqueued + receivedCols, height, complete := mcq.OnReceivedCollection(&collection) + require.False(t, complete) + require.Nil(t, receivedCols) + require.Equal(t, uint64(0), height) +} + +func TestMissingCollectionQueue_OnReceivedCollection_DuplicateCollection(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(2) + collectionIDs := []flow.Identifier{collections[0].ID(), collections[1].ID()} + + callback := func() {} + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + + // Receive first collection + receivedCols, height, complete := mcq.OnReceivedCollection(collections[0]) + require.False(t, complete) + + // Receive same collection again - should be ignored + receivedCols, height, complete = mcq.OnReceivedCollection(collections[0]) + require.False(t, complete) + require.Nil(t, receivedCols) + require.Equal(t, uint64(0), height) +} + +func TestMissingCollectionQueue_OnIndexedForBlock_InvokesCallback(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(2) + collectionIDs := []flow.Identifier{collections[0].ID(), collections[1].ID()} + + callbackCalled := false + var callbackMutex sync.Mutex + callback := func() { + callbackMutex.Lock() + defer callbackMutex.Unlock() + callbackCalled = true + } + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + + // Receive all collections to complete the block + _, _, complete := mcq.OnReceivedCollection(collections[0]) + require.False(t, complete) + + _, _, complete = mcq.OnReceivedCollection(collections[1]) + require.True(t, complete) + + // Index the block - should invoke callback + mcq.OnIndexedForBlock(blockHeight) + + callbackMutex.Lock() + require.True(t, callbackCalled, "callback should have been called") + callbackMutex.Unlock() +} + +func TestMissingCollectionQueue_OnIndexedForBlock_UntrackedBlock(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + + // Try to index a block that was never enqueued - should be a no-op + mcq.OnIndexedForBlock(blockHeight) + // Should not panic +} + +func TestMissingCollectionQueue_OnIndexedForBlock_AlreadyIndexed(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(1) + collectionIDs := []flow.Identifier{collections[0].ID()} + + callbackCallCount := 0 + callback := func() { + callbackCallCount++ + } + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + + // Receive collection to complete the block + _, _, complete := mcq.OnReceivedCollection(collections[0]) + require.True(t, complete) + + // Index the block first time + mcq.OnIndexedForBlock(blockHeight) + require.Equal(t, 1, callbackCallCount) + + // Try to index again - should be a no-op + mcq.OnIndexedForBlock(blockHeight) + require.Equal(t, 1, callbackCallCount, "callback should not be called again") +} + +func TestMissingCollectionQueue_EnqueueSameBlockHeight_ReplacesCallback(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(1) + collectionIDs := []flow.Identifier{collections[0].ID()} + + firstCallbackCalled := false + firstCallback := func() { + firstCallbackCalled = true + } + + secondCallbackCalled := false + secondCallback := func() { + secondCallbackCalled = true + } + + // Enqueue first time + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, firstCallback) + require.NoError(t, err) + + // Enqueue same block height again - should replace callback + err = mcq.EnqueueMissingCollections(blockHeight, collectionIDs, secondCallback) + require.NoError(t, err) + + // Complete and index + _, _, complete := mcq.OnReceivedCollection(collections[0]) + require.True(t, complete) + + mcq.OnIndexedForBlock(blockHeight) + + require.False(t, firstCallbackCalled, "first callback should not be called") + require.True(t, secondCallbackCalled, "second callback should be called") +} + +func TestMissingCollectionQueue_CollectionToBlock_OneToOneRelationship(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + collection := unittest.CollectionFixture(1) + collectionID := collection.ID() + + blockHeight1 := uint64(100) + blockHeight2 := uint64(200) + + callback1 := func() {} + callback2 := func() {} + + // Assign collection to first block + err := mcq.EnqueueMissingCollections(blockHeight1, []flow.Identifier{collectionID}, callback1) + require.NoError(t, err) + + // Try to assign same collection to different block - should error + err = mcq.EnqueueMissingCollections(blockHeight2, []flow.Identifier{collectionID}, callback2) + require.Error(t, err) + require.Contains(t, err.Error(), "already assigned") +} + +func TestMissingCollectionQueue_MultipleBlocks_Independent(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + + blockHeight1 := uint64(100) + blockHeight2 := uint64(200) + + collections1 := unittest.CollectionListFixture(2) + collections2 := unittest.CollectionListFixture(2) + + collectionIDs1 := []flow.Identifier{collections1[0].ID(), collections1[1].ID()} + collectionIDs2 := []flow.Identifier{collections2[0].ID(), collections2[1].ID()} + + callback1Called := false + callback1 := func() { + callback1Called = true + } + + callback2Called := false + callback2 := func() { + callback2Called = true + } + + // Enqueue both blocks + err := mcq.EnqueueMissingCollections(blockHeight1, collectionIDs1, callback1) + require.NoError(t, err) + + err = mcq.EnqueueMissingCollections(blockHeight2, collectionIDs2, callback2) + require.NoError(t, err) + + // Complete block 1 + _, _, complete := mcq.OnReceivedCollection(collections1[0]) + require.False(t, complete) + + receivedCols, height, complete := mcq.OnReceivedCollection(collections1[1]) + require.True(t, complete) + require.Equal(t, blockHeight1, height) + require.NotNil(t, receivedCols) + + mcq.OnIndexedForBlock(blockHeight1) + require.True(t, callback1Called) + require.False(t, callback2Called) + + // Complete block 2 + _, _, complete = mcq.OnReceivedCollection(collections2[0]) + require.False(t, complete) + + receivedCols, height, complete = mcq.OnReceivedCollection(collections2[1]) + require.True(t, complete) + require.Equal(t, blockHeight2, height) + require.NotNil(t, receivedCols) + + mcq.OnIndexedForBlock(blockHeight2) + require.True(t, callback2Called) +} + +func TestMissingCollectionQueue_ConcurrentAccess(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + collections := unittest.CollectionListFixture(10) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackCalled := false + var callbackMutex sync.Mutex + callback := func() { + callbackMutex.Lock() + defer callbackMutex.Unlock() + callbackCalled = true + } + + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + + // Receive collections concurrently + var wg sync.WaitGroup + for _, col := range collections { + wg.Add(1) + go func(c *flow.Collection) { + defer wg.Done() + mcq.OnReceivedCollection(c) + }(col) + } + wg.Wait() + + // One of the receives should have completed the block + // We can't predict which one, but we can check that eventually the block is complete + // by trying to index it + mcq.OnIndexedForBlock(blockHeight) + + callbackMutex.Lock() + require.True(t, callbackCalled) + callbackMutex.Unlock() +} + +func TestMissingCollectionQueue_EmptyCollectionList(t *testing.T) { + t.Parallel() + + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + + callback := func() { + // Callback for empty collection list + } + + // Enqueue with no missing collections + err := mcq.EnqueueMissingCollections(blockHeight, []flow.Identifier{}, callback) + require.NoError(t, err) + + // Block should be immediately complete (no collections to wait for) + // But we can't test this directly since OnReceivedCollection won't be called + // The caller should handle this case by checking if all collections are already in storage +} + diff --git a/engine/access/ingestion2/collections/requester.go b/engine/access/ingestion2/collections/requester.go new file mode 100644 index 00000000000..b7b48012f9b --- /dev/null +++ b/engine/access/ingestion2/collections/requester.go @@ -0,0 +1,79 @@ +package collections + +import ( + "errors" + "fmt" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +var _ ingestion2.CollectionRequester = (*CollectionRequester)(nil) + +// CollectionRequester requests collections from collection nodes on the network. +// It implements the ingestion2.CollectionRequester interface. +type CollectionRequester struct { + requester module.Requester + state protocol.State + guarantees storage.Guarantees +} + +// NewCollectionRequester creates a new CollectionRequester. +// +// Parameters: +// - requester: The requester engine for requesting entities from the network +// - state: Protocol state for finding guarantors +// - guarantees: Guarantees storage for retrieving collection guarantees by collection ID +// +// No error returns are expected during normal operation. +func NewCollectionRequester( + requester module.Requester, + state protocol.State, + guarantees storage.Guarantees, +) *CollectionRequester { + return &CollectionRequester{ + requester: requester, + state: state, + guarantees: guarantees, + } +} + +// RequestCollections requests collections by their IDs from collection nodes on the network. +// For each collection, it finds the guarantors and requests the collection from them. +// +// No error returns are expected during normal operation. +func (cr *CollectionRequester) RequestCollections(ids []flow.Identifier) error { + for _, collectionID := range ids { + // Retrieve the guarantee for this collection + guarantee, err := cr.guarantees.ByCollectionID(collectionID) + if err != nil { + // If guarantee is not found, we can't determine guarantors, so skip this collection + // This can happen if the collection hasn't been finalized yet or if it's from a fork + if errors.Is(err, storage.ErrNotFound) { + continue + } + return fmt.Errorf("failed to retrieve guarantee for collection %v: %w", collectionID, err) + } + + // Find guarantors for this guarantee + guarantors, err := protocol.FindGuarantors(cr.state, guarantee) + if err != nil { + // Failed to find guarantors - this could happen if the reference block is unknown + // or if the cluster is not found. Skip this collection rather than failing entirely. + continue + } + + // Request the collection from the guarantors + cr.requester.EntityByID(collectionID, filter.HasNodeID[flow.Identity](guarantors...)) + } + + // Force immediate dispatch of all pending requests + cr.requester.Force() + + return nil +} + diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/ingestion2/collections/syncer_factory.go new file mode 100644 index 00000000000..7ffdc27587b --- /dev/null +++ b/engine/access/ingestion2/collections/syncer_factory.go @@ -0,0 +1,151 @@ +package collections + +import ( + "fmt" + + "github.com/jordanschalm/lockctx" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/common/requester" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// CreateSyncerConfig holds configuration parameters for creating a Syncer. +type CreateSyncerConfig struct { + // MaxProcessing is the maximum number of jobs to process concurrently. + MaxProcessing uint64 + // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. + MaxSearchAhead uint64 + // EDILagThreshold is the threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. + // Set to a very large value to effectively disable fetching and rely only on EDI. + EDILagThreshold uint64 +} + +// CreateSyncer creates a new ingestion2.Syncer component with all its dependencies. +// This function is in the collections package to avoid import cycles: +// - collections package already imports ingestion2 (for interfaces) +// - CreateSyncer needs to create concrete types from collections package +// - Placing it in ingestion2 would create: ingestion2 -> collections -> ingestion2 (cycle) +// +// Parameters: +// - log: Logger for the component +// - engineRegistry: Engine registry for creating the requester engine +// - state: Protocol state +// - me: Local node identity +// - blocks: Blocks storage +// - collections: Collections storage +// - guarantees: Guarantees storage +// - db: Database for storage operations +// - lockManager: Lock manager for coordinating database access +// - processedFinalizedBlockHeight: Initializer for tracking processed block heights +// - collectionExecutedMetric: Metrics collector for tracking collection indexing +// - ediHeightProvider: Provider for EDI's highest indexed height (can be nil if EDI is disabled) +// - config: Configuration for the syncer +// +// No error returns are expected during normal operation. +func CreateSyncer( + log zerolog.Logger, + engineRegistry network.EngineRegistry, + state protocol.State, + me module.Local, + blocks storage.Blocks, + collections storage.Collections, + guarantees storage.Guarantees, + db storage.DB, + lockManager lockctx.Manager, + processedFinalizedBlockHeight storage.ConsumerProgressInitializer, + collectionExecutedMetric module.CollectionExecutedMetric, + ediHeightProvider ingestion2.EDIHeightProvider, + config CreateSyncerConfig, +) (*ingestion2.Syncer, error) { + // Create requester engine for requesting collections + requestEng, err := requester.New( + log.With().Str("entity", "collection").Logger(), + metrics.NewNoopCollector(), // TODO: pass proper metrics if available + engineRegistry, + me, + state, + channels.RequestCollections, + filter.HasRole[flow.Identity](flow.RoleCollection), + func() flow.Entity { return new(flow.Collection) }, + ) + if err != nil { + return nil, fmt.Errorf("could not create requester engine: %w", err) + } + + // Create MissingCollectionQueue + mcq := NewMissingCollectionQueue() + + // Create BlockCollectionIndexer + indexer := NewBlockCollectionIndexer( + collectionExecutedMetric, + lockManager, + db, + collections, + ) + + // Create CollectionRequester + collectionRequester := NewCollectionRequester( + requestEng, + state, + guarantees, + ) + + // Create JobProcessor + jobProcessor := NewJobProcessor( + mcq, + indexer, + collectionRequester, + blocks, + collections, + ediHeightProvider, + config.EDILagThreshold, + ) + + // Register handler for received collections + requestEng.WithHandle(func(originID flow.Identifier, entity flow.Entity) { + collection, ok := entity.(*flow.Collection) + if !ok { + return + } + + // Forward collection to MCQ + receivedCols, height, complete := mcq.OnReceivedCollection(collection) + if complete { + // Block became complete, index it + err := indexer.OnReceivedCollectionsForBlock(height, receivedCols) + if err != nil { + log.Error().Err(err).Uint64("height", height).Msg("failed to index collections for block") + return + } + + // Notify MCQ that the block has been indexed (this invokes the callback) + mcq.OnIndexedForBlock(height) + } + }) + + // Create Syncer using ingestion2.NewSyncer + syncer, err := ingestion2.NewSyncer( + log, + jobProcessor, + processedFinalizedBlockHeight, + state, + blocks, + config.MaxProcessing, + config.MaxSearchAhead, + ) + if err != nil { + return nil, fmt.Errorf("could not create syncer: %w", err) + } + + return syncer, nil +} + diff --git a/engine/access/ingestion2/engine.go b/engine/access/ingestion2/engine.go index ad2fbc449d4..80b69f7d33d 100644 --- a/engine/access/ingestion2/engine.go +++ b/engine/access/ingestion2/engine.go @@ -16,7 +16,6 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -36,7 +35,7 @@ type Engine struct { log zerolog.Logger finalizedBlockProcessor *FinalizedBlockProcessor - collectionSyncer *collections.Syncer + collectionSyncer *Syncer messageHandler *engine.MessageHandler executionReceiptsQueue *engine.FifoMessageStore @@ -50,7 +49,7 @@ func New( log zerolog.Logger, net network.EngineRegistry, finalizedBlockProcessor *FinalizedBlockProcessor, - collectionSyncer *collections.Syncer, + collectionSyncer *Syncer, receipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, ) (*Engine, error) { @@ -85,8 +84,7 @@ func New( // engine notifies workers when new data is available so that they can start processing them. builder := component.NewComponentManagerBuilder(). AddWorker(e.messageHandlerLoop). - AddWorker(e.finalizedBlockProcessor.StartWorkerLoop). - AddWorker(e.collectionSyncer.WorkerLoop) + AddWorker(e.finalizedBlockProcessor.StartWorkerLoop) e.ComponentManager = builder.Build() // engine gets execution receipts from channels.ReceiveReceipts channel @@ -178,6 +176,7 @@ func (e *Engine) persistExecutionReceipt(receipt *flow.ExecutionReceipt) error { // OnFinalizedBlock is called by the follower engine after a block has been finalized and the state has been updated. // Receives block finalized events from the finalization distributor and forwards them to the consumer. -func (e *Engine) OnFinalizedBlock(_ *model.Block) { +func (e *Engine) OnFinalizedBlock(block *model.Block) { e.finalizedBlockProcessor.Notify() + e.collectionSyncer.OnFinalizedBlock() } diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index 7e22207b5ac..f3326ee13ff 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -14,7 +14,6 @@ import ( "github.com/stretchr/testify/suite" hotmodel "github.com/onflow/flow-go/consensus/hotstuff/model" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -200,7 +199,7 @@ func (s *Suite) TestComponentShutdown() { // initEngineAndSyncer create new instance of ingestion engine and collection collectionSyncer. // It waits until the ingestion engine starts. -func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *collections.Syncer) { +func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *Syncer) { processedHeightInitializer := store.NewConsumerProgress(s.db, module.ConsumeProgressIngestionEngineBlockHeight) lastFullBlockHeight, err := store.NewConsumerProgress(s.db, module.ConsumeProgressLastFullBlockHeight).Initialize(s.finalizedBlock.Height) @@ -209,36 +208,34 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeight) require.NoError(s.T(), err) - indexer, err := collections.NewIndexer( + // TODO: Update test to use new ingestion2 architecture with JobProcessor + // This test needs to be refactored to use the new JobProcessor-based architecture. + // For now, we skip creating the new Syncer since FinalizedBlockProcessor still uses the old collections.Syncer + // and the test architecture needs to be updated to match the new design. + + // Create a placeholder syncer for the Engine (new ingestion2.Syncer) + // Note: This won't work properly until the test is fully refactored + syncer, err := NewSyncer( s.log, - s.db, - s.collectionExecutedMetric, + nil, // jobProcessor - needs to be created with proper dependencies + processedHeightInitializer, s.proto.state, s.blocks, - s.collections, - s.lastFullBlockHeight, - s.lockManager, - ) - require.NoError(s.T(), err) - - syncer := collections.NewSyncer( - s.log, - s.request, - s.proto.state, - s.collections, - s.lastFullBlockHeight, - indexer, - nil, + 10, // maxProcessing + 0, // maxSearchAhead ) require.NoError(s.T(), err) + // TODO: FinalizedBlockProcessor still uses old collections.Syncer + // This needs to be updated to work with the new architecture + // For now, pass nil and skip this part of the test blockProcessor, err := NewFinalizedBlockProcessor( s.log, s.proto.state, s.blocks, s.results, processedHeightInitializer, - syncer, + nil, // TODO: Update FinalizedBlockProcessor to not require old syncer s.collectionExecutedMetric, ) require.NoError(s.T(), err) From a7a02530908a13cd376e42ba58ae32e5fbc5cc5e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 16:47:31 -0800 Subject: [PATCH 003/126] update ingestion2 engine --- .../access/ingestion2/collection_syncing.go | 98 +--------------- .../collections/edi_height_provider.go | 29 +++++ .../ingestion2/collections/job_processor.go | 2 +- .../access/ingestion2/collections/syncer.go | 109 ++++++++++++++++++ .../ingestion2/collections/syncer_factory.go | 19 ++- engine/access/ingestion2/engine.go | 4 +- engine/access/ingestion2/engine_test.go | 7 +- 7 files changed, 163 insertions(+), 105 deletions(-) create mode 100644 engine/access/ingestion2/collections/edi_height_provider.go create mode 100644 engine/access/ingestion2/collections/syncer.go diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go index af9945b99be..145ccc6a925 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/ingestion2/collection_syncing.go @@ -1,18 +1,10 @@ package ingestion2 import ( - "fmt" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/jobqueue" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" ) // Tracks missing collections per height and invokes job callbacks when complete. @@ -51,90 +43,10 @@ type EDIHeightProvider interface { // Syncer is a component that consumes finalized block jobs and processes them // to index collections. It uses a job consumer with windowed throttling to prevent node overload. -type Syncer struct { +type Syncer interface { component.Component - - consumer *jobqueue.ComponentConsumer - jobProcessor JobProcessor - workSignal engine.Notifier -} - -// NewSyncer creates a new Syncer component. -// -// Parameters: -// - log: Logger for the component -// - jobProcessor: JobProcessor implementation for processing collection indexing jobs -// - progressInitializer: Initializer for tracking processed block heights -// - state: Protocol state for reading finalized block information -// - blocks: Blocks storage for reading blocks by height -// - maxProcessing: Maximum number of jobs to process concurrently -// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit -// -// No error returns are expected during normal operation. -func NewSyncer( - log zerolog.Logger, - jobProcessor JobProcessor, - progressInitializer storage.ConsumerProgressInitializer, - state protocol.State, - blocks storage.Blocks, - maxProcessing uint64, // max number of blocks to fetch collections - maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for -) (*Syncer, error) { - workSignal := engine.NewNotifier() - - // Read the default index from the finalized root height - defaultIndex := state.Params().FinalizedRoot().Height - - // Create a Jobs instance that reads finalized blocks by height - jobs := jobqueue.NewFinalizedBlockReader(state, blocks) - - // Create an adapter function that wraps the JobProcessor interface - processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { - err := jobProcessor.ProcessJob(ctx, job, done) - if err != nil { - ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) - } - } - - consumer, err := jobqueue.NewComponentConsumer( - log.With().Str("component", "collection-syncing").Logger(), - workSignal.Channel(), - progressInitializer, - jobs, - defaultIndex, - processorFunc, - maxProcessing, - maxSearchAhead, - ) - if err != nil { - return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) - } - - return &Syncer{ - Component: consumer, - consumer: consumer, - jobProcessor: jobProcessor, - workSignal: workSignal, - }, nil -} - -// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer -// that new work is available. -func (s *Syncer) OnFinalizedBlock() { - s.workSignal.Notify() -} - -// LastProcessedIndex returns the last processed job index. -func (s *Syncer) LastProcessedIndex() uint64 { - return s.consumer.LastProcessedIndex() -} - -// Head returns the highest job index available. -func (s *Syncer) Head() (uint64, error) { - return s.consumer.Head() -} - -// Size returns the number of in-memory jobs that the consumer is processing. -func (s *Syncer) Size() uint { - return s.consumer.Size() + OnFinalizedBlock() + LastProcessedIndex() uint64 + Head() (uint64, error) + Size() uint } diff --git a/engine/access/ingestion2/collections/edi_height_provider.go b/engine/access/ingestion2/collections/edi_height_provider.go new file mode 100644 index 00000000000..83162e75651 --- /dev/null +++ b/engine/access/ingestion2/collections/edi_height_provider.go @@ -0,0 +1,29 @@ +package collections + +import ( + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" +) + +var _ ingestion2.EDIHeightProvider = (*ProcessedHeightRecorderWrapper)(nil) + +// ProcessedHeightRecorderWrapper wraps execution_data.ProcessedHeightRecorder to implement +// ingestion2.EDIHeightProvider interface. +type ProcessedHeightRecorderWrapper struct { + recorder execution_data.ProcessedHeightRecorder +} + +// NewProcessedHeightRecorderWrapper creates a new wrapper that implements EDIHeightProvider +// by wrapping the given ProcessedHeightRecorder. +func NewProcessedHeightRecorderWrapper(recorder execution_data.ProcessedHeightRecorder) *ProcessedHeightRecorderWrapper { + return &ProcessedHeightRecorderWrapper{ + recorder: recorder, + } +} + +// HighestIndexedHeight returns the highest block height for which EDI has indexed collections. +// It wraps the ProcessedHeightRecorder's HighestCompleteHeight method. +func (p *ProcessedHeightRecorderWrapper) HighestIndexedHeight() (uint64, error) { + return p.recorder.HighestCompleteHeight(), nil +} + diff --git a/engine/access/ingestion2/collections/job_processor.go b/engine/access/ingestion2/collections/job_processor.go index 667f0f94a12..e05683c6c2b 100644 --- a/engine/access/ingestion2/collections/job_processor.go +++ b/engine/access/ingestion2/collections/job_processor.go @@ -26,7 +26,7 @@ type JobProcessor struct { requester ingestion2.CollectionRequester blocks storage.Blocks collections storage.CollectionsReader - ediHeightProvider EDIHeightProvider + ediHeightProvider EDIHeightProvider // when set as nil, EDI is considered disabled ediLagThreshold uint64 } diff --git a/engine/access/ingestion2/collections/syncer.go b/engine/access/ingestion2/collections/syncer.go new file mode 100644 index 00000000000..247b30d6e3a --- /dev/null +++ b/engine/access/ingestion2/collections/syncer.go @@ -0,0 +1,109 @@ +package collections + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +var _ ingestion2.Syncer = (*Syncer)(nil) + +// Syncer is a component that consumes finalized block jobs and processes them +// to index collections. It uses a job consumer with windowed throttling to prevent node overload. +type Syncer struct { + component.Component + + consumer *jobqueue.ComponentConsumer + jobProcessor ingestion2.JobProcessor + workSignal engine.Notifier +} + +// NewSyncer creates a new Syncer component. +// +// Parameters: +// - log: Logger for the component +// - jobProcessor: JobProcessor implementation for processing collection indexing jobs +// - progressInitializer: Initializer for tracking processed block heights +// - state: Protocol state for reading finalized block information +// - blocks: Blocks storage for reading blocks by height +// - maxProcessing: Maximum number of jobs to process concurrently +// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit +// +// No error returns are expected during normal operation. +func NewSyncer( + log zerolog.Logger, + jobProcessor ingestion2.JobProcessor, + progressInitializer storage.ConsumerProgressInitializer, + state protocol.State, + blocks storage.Blocks, + maxProcessing uint64, // max number of blocks to fetch collections + maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for +) (*Syncer, error) { + workSignal := engine.NewNotifier() + + // Read the default index from the finalized root height + defaultIndex := state.Params().FinalizedRoot().Height + + // Create a Jobs instance that reads finalized blocks by height + jobs := jobqueue.NewFinalizedBlockReader(state, blocks) + + // Create an adapter function that wraps the JobProcessor interface + processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { + err := jobProcessor.ProcessJob(ctx, job, done) + if err != nil { + ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) + } + } + + consumer, err := jobqueue.NewComponentConsumer( + log.With().Str("component", "collection-syncing").Logger(), + workSignal.Channel(), + progressInitializer, + jobs, + defaultIndex, + processorFunc, + maxProcessing, + maxSearchAhead, + ) + if err != nil { + return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) + } + + return &Syncer{ + Component: consumer, + consumer: consumer, + jobProcessor: jobProcessor, + workSignal: workSignal, + }, nil +} + +// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer +// that new work is available. +func (s *Syncer) OnFinalizedBlock() { + s.workSignal.Notify() +} + +// LastProcessedIndex returns the last processed job index. +func (s *Syncer) LastProcessedIndex() uint64 { + return s.consumer.LastProcessedIndex() +} + +// Head returns the highest job index available. +func (s *Syncer) Head() (uint64, error) { + return s.consumer.Head() +} + +// Size returns the number of in-memory jobs that the consumer is processing. +func (s *Syncer) Size() uint { + return s.consumer.Size() +} + diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/ingestion2/collections/syncer_factory.go index 7ffdc27587b..497a9c2c53c 100644 --- a/engine/access/ingestion2/collections/syncer_factory.go +++ b/engine/access/ingestion2/collections/syncer_factory.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" @@ -29,7 +30,7 @@ type CreateSyncerConfig struct { EDILagThreshold uint64 } -// CreateSyncer creates a new ingestion2.Syncer component with all its dependencies. +// CreateSyncer creates a new Syncer component with all its dependencies. // This function is in the collections package to avoid import cycles: // - collections package already imports ingestion2 (for interfaces) // - CreateSyncer needs to create concrete types from collections package @@ -47,7 +48,7 @@ type CreateSyncerConfig struct { // - lockManager: Lock manager for coordinating database access // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionExecutedMetric: Metrics collector for tracking collection indexing -// - ediHeightProvider: Provider for EDI's highest indexed height (can be nil if EDI is disabled) +// - processedHeightRecorder: Recorder for execution data processed heights (can be nil if EDI is disabled) // - config: Configuration for the syncer // // No error returns are expected during normal operation. @@ -63,9 +64,9 @@ func CreateSyncer( lockManager lockctx.Manager, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, - ediHeightProvider ingestion2.EDIHeightProvider, + processedHeightRecorder execution_data.ProcessedHeightRecorder, config CreateSyncerConfig, -) (*ingestion2.Syncer, error) { +) (*Syncer, error) { // Create requester engine for requesting collections requestEng, err := requester.New( log.With().Str("entity", "collection").Logger(), @@ -99,6 +100,12 @@ func CreateSyncer( guarantees, ) + // Wrap ProcessedHeightRecorder as EDIHeightProvider if provided + var ediHeightProvider ingestion2.EDIHeightProvider + if processedHeightRecorder != nil { + ediHeightProvider = NewProcessedHeightRecorderWrapper(processedHeightRecorder) + } + // Create JobProcessor jobProcessor := NewJobProcessor( mcq, @@ -132,8 +139,8 @@ func CreateSyncer( } }) - // Create Syncer using ingestion2.NewSyncer - syncer, err := ingestion2.NewSyncer( + // Create Syncer + syncer, err := NewSyncer( log, jobProcessor, processedFinalizedBlockHeight, diff --git a/engine/access/ingestion2/engine.go b/engine/access/ingestion2/engine.go index 80b69f7d33d..b0ff39f52f0 100644 --- a/engine/access/ingestion2/engine.go +++ b/engine/access/ingestion2/engine.go @@ -35,7 +35,7 @@ type Engine struct { log zerolog.Logger finalizedBlockProcessor *FinalizedBlockProcessor - collectionSyncer *Syncer + collectionSyncer Syncer messageHandler *engine.MessageHandler executionReceiptsQueue *engine.FifoMessageStore @@ -49,7 +49,7 @@ func New( log zerolog.Logger, net network.EngineRegistry, finalizedBlockProcessor *FinalizedBlockProcessor, - collectionSyncer *Syncer, + collectionSyncer Syncer, receipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, ) (*Engine, error) { diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index f3326ee13ff..40040188297 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -14,6 +14,7 @@ import ( "github.com/stretchr/testify/suite" hotmodel "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/engine/access/ingestion2/collections" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -199,7 +200,7 @@ func (s *Suite) TestComponentShutdown() { // initEngineAndSyncer create new instance of ingestion engine and collection collectionSyncer. // It waits until the ingestion engine starts. -func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *Syncer) { +func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, Syncer) { processedHeightInitializer := store.NewConsumerProgress(s.db, module.ConsumeProgressIngestionEngineBlockHeight) lastFullBlockHeight, err := store.NewConsumerProgress(s.db, module.ConsumeProgressLastFullBlockHeight).Initialize(s.finalizedBlock.Height) @@ -213,9 +214,9 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, // For now, we skip creating the new Syncer since FinalizedBlockProcessor still uses the old collections.Syncer // and the test architecture needs to be updated to match the new design. - // Create a placeholder syncer for the Engine (new ingestion2.Syncer) + // Create a placeholder syncer for the Engine (new collections.Syncer) // Note: This won't work properly until the test is fully refactored - syncer, err := NewSyncer( + syncer, err := collections.NewSyncer( s.log, nil, // jobProcessor - needs to be created with proper dependencies processedHeightInitializer, From 6a9ccd50fd596d93628fd119a45318d264233d35 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 16:49:50 -0800 Subject: [PATCH 004/126] fix tests --- engine/access/ingestion2/engine_test.go | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index 40040188297..2a32c3a9ed7 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -14,7 +14,6 @@ import ( "github.com/stretchr/testify/suite" hotmodel "github.com/onflow/flow-go/consensus/hotstuff/model" - "github.com/onflow/flow-go/engine/access/ingestion2/collections" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -213,19 +212,9 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, // This test needs to be refactored to use the new JobProcessor-based architecture. // For now, we skip creating the new Syncer since FinalizedBlockProcessor still uses the old collections.Syncer // and the test architecture needs to be updated to match the new design. - - // Create a placeholder syncer for the Engine (new collections.Syncer) - // Note: This won't work properly until the test is fully refactored - syncer, err := collections.NewSyncer( - s.log, - nil, // jobProcessor - needs to be created with proper dependencies - processedHeightInitializer, - s.proto.state, - s.blocks, - 10, // maxProcessing - 0, // maxSearchAhead - ) - require.NoError(s.T(), err) + // The syncer will be created using collections.CreateSyncer in the actual node builder. + // For testing, we'll use nil for now until the test is fully refactored. + var syncer Syncer = nil // TODO: FinalizedBlockProcessor still uses old collections.Syncer // This needs to be updated to work with the new architecture From 4277a725b04347877a523a5053678d54890e2caa Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 17:30:49 -0800 Subject: [PATCH 005/126] update ingestion2 and indexer --- .../access/ingestion2/collection_syncing.go | 6 +- .../ingestion2/collections/job_processor.go | 49 +- .../collections/missing_collection_queue.go | 31 +- .../missing_collection_queue_test.go | 372 -------------- .../ingestion2/collections/syncer_factory.go | 18 +- engine/access/ingestion2/engine_test.go | 468 ------------------ .../indexer/indexer_core.go | 8 +- 7 files changed, 67 insertions(+), 885 deletions(-) delete mode 100644 engine/access/ingestion2/collections/missing_collection_queue_test.go delete mode 100644 engine/access/ingestion2/engine_test.go diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go index 145ccc6a925..90d169621e9 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/ingestion2/collection_syncing.go @@ -10,12 +10,15 @@ import ( // Tracks missing collections per height and invokes job callbacks when complete. type MissingCollectionQueue interface { EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error - OnIndexedForBlock(blockHeight uint64) // mark done (post‑indexing) + OnIndexedForBlock(blockHeight uint64) bool // mark done (post‑indexing), returns true if height existed // On receipt of a collection, MCQ updates internal state and, if a block // just became complete, returns: (collections, height, true). // Otherwise, returns (nil, 0, false). OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, bool) + + // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). + IsHeightQueued(height uint64) bool } // Requests collections by their IDs. @@ -32,6 +35,7 @@ type BlockCollectionIndexer interface { // Implements the job lifecycle for a single block height. type JobProcessor interface { ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error + OnReceiveCollection(collection *flow.Collection) error OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester } diff --git a/engine/access/ingestion2/collections/job_processor.go b/engine/access/ingestion2/collections/job_processor.go index e05683c6c2b..a67a82bd47d 100644 --- a/engine/access/ingestion2/collections/job_processor.go +++ b/engine/access/ingestion2/collections/job_processor.go @@ -129,33 +129,54 @@ func (jp *JobProcessor) ProcessJob( return nil } +// OnReceiveCollection is called when a collection is received from the requester. +// It passes the collection to MCQ, and if it completes a block, indexes it and marks it as done. +// +// No error returns are expected during normal operation. +func (jp *JobProcessor) OnReceiveCollection(collection *flow.Collection) error { + // Pass collection to MCQ + collections, height, complete := jp.mcq.OnReceivedCollection(collection) + if !complete { + // Block is not complete yet, nothing more to do + return nil + } + + // Block became complete, index it + err := jp.indexer.OnReceivedCollectionsForBlock(height, collections) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", height, err) + } + + // Mark the block as indexed (this invokes the callback) + jp.mcq.OnIndexedForBlock(height) + + return nil +} + // OnReceivedCollectionsForBlock is called by the execution data indexer when collections are received. // It forwards collections to MCQ and handles block completion. // // The blockHeight parameter is provided by EDI to indicate which block these collections belong to. -// Collections are forwarded individually to MCQ, which tracks completion per block. // // No error returns are expected during normal operation. func (jp *JobProcessor) OnReceivedCollectionsForBlock( blockHeight uint64, collections []*flow.Collection, ) error { - // Forward each collection to MCQ - // MCQ will track which block each collection belongs to and detect when a block becomes complete - for _, collection := range collections { - receivedCols, height, complete := jp.mcq.OnReceivedCollection(collection) - if complete { - // Block became complete, index it - err := jp.indexer.OnReceivedCollectionsForBlock(height, receivedCols) - if err != nil { - return fmt.Errorf("failed to index collections for block height %d: %w", height, err) - } + // Mark the block as indexed (this invokes the callback if the height exists) + queued := jp.mcq.IsHeightQueued(blockHeight) + if !queued { + // If the height is not queued, nothing to do + return nil + } - // Notify MCQ that the block has been indexed (this invokes the callback) - jp.mcq.OnIndexedForBlock(height) - } + err := jp.indexer.OnReceivedCollectionsForBlock(blockHeight, collections) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", blockHeight, err) } + jp.mcq.OnIndexedForBlock(blockHeight) + return nil } diff --git a/engine/access/ingestion2/collections/missing_collection_queue.go b/engine/access/ingestion2/collections/missing_collection_queue.go index f136dbab8ef..6c9b1035420 100644 --- a/engine/access/ingestion2/collections/missing_collection_queue.go +++ b/engine/access/ingestion2/collections/missing_collection_queue.go @@ -63,8 +63,6 @@ func NewMissingCollectionQueue() *MissingCollectionQueue { // Only collections that are actually missing should be passed in collectionIDs. // // If the same block height is enqueued multiple times, the previous callback is replaced. -// -// No error returns are expected during normal operation. func (mcq *MissingCollectionQueue) EnqueueMissingCollections( blockHeight uint64, collectionIDs []flow.Identifier, @@ -112,8 +110,6 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( // Returns: // - (collections, height, true) if the block height became complete // - (nil, 0, false) if no block height became complete -// -// No error returns are expected during normal operation. func (mcq *MissingCollectionQueue) OnReceivedCollection( collection *flow.Collection, ) ([]*flow.Collection, uint64, bool) { @@ -131,16 +127,15 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( jobState, exists := mcq.blockJobs[height] if !exists { - // Job was already completed/removed, clean up the mapping. - delete(mcq.collectionToHeight, collectionID) + // Job was already completed/removed. + // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. return nil, 0, false } // Check if this collection was still missing for this block. if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { // Collection was already received or wasn't part of this block's missing set. - // Clean up the mapping since we've already processed this collection. - delete(mcq.collectionToHeight, collectionID) + // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. return nil, 0, false } @@ -149,8 +144,7 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( // Store the collection so it can be returned when the block is complete. jobState.receivedCollections[collectionID] = collection - // Remove this collection from the collection-to-height mapping since we've processed it. - delete(mcq.collectionToHeight, collectionID) + // Don't delete from collectionToHeight - the mapping is kept until OnIndexedForBlock cleans it up. // Check if the block is now complete (all collections received). if len(jobState.missingCollections) == 0 { @@ -165,18 +159,28 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( return nil, 0, false } +// IsHeightQueued returns true if the given height has queued collections +// Returns false if the height is not tracked +func (mcq *MissingCollectionQueue) IsHeightQueued(height uint64) bool { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + _, exists := mcq.blockJobs[height] + return exists +} + // OnIndexedForBlock notifies the queue that a block height has been indexed. // This invokes the callback for that block height and removes it from tracking. // -// No error returns are expected during normal operation. -func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) { +// Returns true if the height existed and was processed, false if the height was not tracked. +func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) bool { mcq.mu.Lock() jobState, exists := mcq.blockJobs[blockHeight] if !exists { // Block was not tracked or already completed (callback already called and job removed). mcq.mu.Unlock() - return + return false } // Get the callback before removing the job. @@ -204,4 +208,5 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) { mcq.mu.Unlock() callback() // Note: We manually unlocked above, so we don't use defer here to avoid double-unlock. + return true } diff --git a/engine/access/ingestion2/collections/missing_collection_queue_test.go b/engine/access/ingestion2/collections/missing_collection_queue_test.go deleted file mode 100644 index 682f2d5267e..00000000000 --- a/engine/access/ingestion2/collections/missing_collection_queue_test.go +++ /dev/null @@ -1,372 +0,0 @@ -package collections - -import ( - "sync" - "testing" - - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestMissingCollectionQueue_EnqueueMissingCollections(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(3) - collectionIDs := make([]flow.Identifier, len(collections)) - for i, col := range collections { - collectionIDs[i] = col.ID() - } - - callbackCalled := false - callback := func() { - callbackCalled = true - } - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - require.False(t, callbackCalled, "callback should not be called yet") -} - -func TestMissingCollectionQueue_OnReceivedCollection_CompleteBlock(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(3) - collectionIDs := make([]flow.Identifier, len(collections)) - for i, col := range collections { - collectionIDs[i] = col.ID() - } - - callbackCalled := false - callback := func() { - callbackCalled = true - } - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - - // Receive first two collections - block should not be complete yet - receivedCols, height, complete := mcq.OnReceivedCollection(collections[0]) - require.False(t, complete) - require.Nil(t, receivedCols) - require.Equal(t, uint64(0), height) - - receivedCols, height, complete = mcq.OnReceivedCollection(collections[1]) - require.False(t, complete) - require.Nil(t, receivedCols) - require.Equal(t, uint64(0), height) - - // Receive last collection - block should now be complete - receivedCols, height, complete = mcq.OnReceivedCollection(collections[2]) - require.True(t, complete) - require.NotNil(t, receivedCols) - require.Equal(t, blockHeight, height) - require.Len(t, receivedCols, 3) - - // Verify all collections are present - receivedMap := make(map[flow.Identifier]*flow.Collection) - for _, col := range receivedCols { - receivedMap[col.ID()] = col - } - for _, expectedCol := range collections { - require.Contains(t, receivedMap, expectedCol.ID()) - require.Equal(t, expectedCol, receivedMap[expectedCol.ID()]) - } - - require.False(t, callbackCalled, "callback should not be called until OnIndexedForBlock") -} - -func TestMissingCollectionQueue_OnReceivedCollection_UntrackedCollection(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - collection := unittest.CollectionFixture(1) - - // Receive collection that was never enqueued - receivedCols, height, complete := mcq.OnReceivedCollection(&collection) - require.False(t, complete) - require.Nil(t, receivedCols) - require.Equal(t, uint64(0), height) -} - -func TestMissingCollectionQueue_OnReceivedCollection_DuplicateCollection(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(2) - collectionIDs := []flow.Identifier{collections[0].ID(), collections[1].ID()} - - callback := func() {} - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - - // Receive first collection - receivedCols, height, complete := mcq.OnReceivedCollection(collections[0]) - require.False(t, complete) - - // Receive same collection again - should be ignored - receivedCols, height, complete = mcq.OnReceivedCollection(collections[0]) - require.False(t, complete) - require.Nil(t, receivedCols) - require.Equal(t, uint64(0), height) -} - -func TestMissingCollectionQueue_OnIndexedForBlock_InvokesCallback(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(2) - collectionIDs := []flow.Identifier{collections[0].ID(), collections[1].ID()} - - callbackCalled := false - var callbackMutex sync.Mutex - callback := func() { - callbackMutex.Lock() - defer callbackMutex.Unlock() - callbackCalled = true - } - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - - // Receive all collections to complete the block - _, _, complete := mcq.OnReceivedCollection(collections[0]) - require.False(t, complete) - - _, _, complete = mcq.OnReceivedCollection(collections[1]) - require.True(t, complete) - - // Index the block - should invoke callback - mcq.OnIndexedForBlock(blockHeight) - - callbackMutex.Lock() - require.True(t, callbackCalled, "callback should have been called") - callbackMutex.Unlock() -} - -func TestMissingCollectionQueue_OnIndexedForBlock_UntrackedBlock(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - - // Try to index a block that was never enqueued - should be a no-op - mcq.OnIndexedForBlock(blockHeight) - // Should not panic -} - -func TestMissingCollectionQueue_OnIndexedForBlock_AlreadyIndexed(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(1) - collectionIDs := []flow.Identifier{collections[0].ID()} - - callbackCallCount := 0 - callback := func() { - callbackCallCount++ - } - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - - // Receive collection to complete the block - _, _, complete := mcq.OnReceivedCollection(collections[0]) - require.True(t, complete) - - // Index the block first time - mcq.OnIndexedForBlock(blockHeight) - require.Equal(t, 1, callbackCallCount) - - // Try to index again - should be a no-op - mcq.OnIndexedForBlock(blockHeight) - require.Equal(t, 1, callbackCallCount, "callback should not be called again") -} - -func TestMissingCollectionQueue_EnqueueSameBlockHeight_ReplacesCallback(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(1) - collectionIDs := []flow.Identifier{collections[0].ID()} - - firstCallbackCalled := false - firstCallback := func() { - firstCallbackCalled = true - } - - secondCallbackCalled := false - secondCallback := func() { - secondCallbackCalled = true - } - - // Enqueue first time - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, firstCallback) - require.NoError(t, err) - - // Enqueue same block height again - should replace callback - err = mcq.EnqueueMissingCollections(blockHeight, collectionIDs, secondCallback) - require.NoError(t, err) - - // Complete and index - _, _, complete := mcq.OnReceivedCollection(collections[0]) - require.True(t, complete) - - mcq.OnIndexedForBlock(blockHeight) - - require.False(t, firstCallbackCalled, "first callback should not be called") - require.True(t, secondCallbackCalled, "second callback should be called") -} - -func TestMissingCollectionQueue_CollectionToBlock_OneToOneRelationship(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - collection := unittest.CollectionFixture(1) - collectionID := collection.ID() - - blockHeight1 := uint64(100) - blockHeight2 := uint64(200) - - callback1 := func() {} - callback2 := func() {} - - // Assign collection to first block - err := mcq.EnqueueMissingCollections(blockHeight1, []flow.Identifier{collectionID}, callback1) - require.NoError(t, err) - - // Try to assign same collection to different block - should error - err = mcq.EnqueueMissingCollections(blockHeight2, []flow.Identifier{collectionID}, callback2) - require.Error(t, err) - require.Contains(t, err.Error(), "already assigned") -} - -func TestMissingCollectionQueue_MultipleBlocks_Independent(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - - blockHeight1 := uint64(100) - blockHeight2 := uint64(200) - - collections1 := unittest.CollectionListFixture(2) - collections2 := unittest.CollectionListFixture(2) - - collectionIDs1 := []flow.Identifier{collections1[0].ID(), collections1[1].ID()} - collectionIDs2 := []flow.Identifier{collections2[0].ID(), collections2[1].ID()} - - callback1Called := false - callback1 := func() { - callback1Called = true - } - - callback2Called := false - callback2 := func() { - callback2Called = true - } - - // Enqueue both blocks - err := mcq.EnqueueMissingCollections(blockHeight1, collectionIDs1, callback1) - require.NoError(t, err) - - err = mcq.EnqueueMissingCollections(blockHeight2, collectionIDs2, callback2) - require.NoError(t, err) - - // Complete block 1 - _, _, complete := mcq.OnReceivedCollection(collections1[0]) - require.False(t, complete) - - receivedCols, height, complete := mcq.OnReceivedCollection(collections1[1]) - require.True(t, complete) - require.Equal(t, blockHeight1, height) - require.NotNil(t, receivedCols) - - mcq.OnIndexedForBlock(blockHeight1) - require.True(t, callback1Called) - require.False(t, callback2Called) - - // Complete block 2 - _, _, complete = mcq.OnReceivedCollection(collections2[0]) - require.False(t, complete) - - receivedCols, height, complete = mcq.OnReceivedCollection(collections2[1]) - require.True(t, complete) - require.Equal(t, blockHeight2, height) - require.NotNil(t, receivedCols) - - mcq.OnIndexedForBlock(blockHeight2) - require.True(t, callback2Called) -} - -func TestMissingCollectionQueue_ConcurrentAccess(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - collections := unittest.CollectionListFixture(10) - collectionIDs := make([]flow.Identifier, len(collections)) - for i, col := range collections { - collectionIDs[i] = col.ID() - } - - callbackCalled := false - var callbackMutex sync.Mutex - callback := func() { - callbackMutex.Lock() - defer callbackMutex.Unlock() - callbackCalled = true - } - - err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) - require.NoError(t, err) - - // Receive collections concurrently - var wg sync.WaitGroup - for _, col := range collections { - wg.Add(1) - go func(c *flow.Collection) { - defer wg.Done() - mcq.OnReceivedCollection(c) - }(col) - } - wg.Wait() - - // One of the receives should have completed the block - // We can't predict which one, but we can check that eventually the block is complete - // by trying to index it - mcq.OnIndexedForBlock(blockHeight) - - callbackMutex.Lock() - require.True(t, callbackCalled) - callbackMutex.Unlock() -} - -func TestMissingCollectionQueue_EmptyCollectionList(t *testing.T) { - t.Parallel() - - mcq := NewMissingCollectionQueue() - blockHeight := uint64(100) - - callback := func() { - // Callback for empty collection list - } - - // Enqueue with no missing collections - err := mcq.EnqueueMissingCollections(blockHeight, []flow.Identifier{}, callback) - require.NoError(t, err) - - // Block should be immediately complete (no collections to wait for) - // But we can't test this directly since OnReceivedCollection won't be called - // The caller should handle this case by checking if all collections are already in storage -} - diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/ingestion2/collections/syncer_factory.go index 497a9c2c53c..05ba4e1c09d 100644 --- a/engine/access/ingestion2/collections/syncer_factory.go +++ b/engine/access/ingestion2/collections/syncer_factory.go @@ -124,18 +124,11 @@ func CreateSyncer( return } - // Forward collection to MCQ - receivedCols, height, complete := mcq.OnReceivedCollection(collection) - if complete { - // Block became complete, index it - err := indexer.OnReceivedCollectionsForBlock(height, receivedCols) - if err != nil { - log.Error().Err(err).Uint64("height", height).Msg("failed to index collections for block") - return - } - - // Notify MCQ that the block has been indexed (this invokes the callback) - mcq.OnIndexedForBlock(height) + // Forward collection to JobProcessor, which handles MCQ, indexing, and completion + err := jobProcessor.OnReceiveCollection(collection) + if err != nil { + log.Fatal().Err(err).Msg("failed to process received collection") + return } }) @@ -155,4 +148,3 @@ func CreateSyncer( return syncer, nil } - diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go deleted file mode 100644 index 2a32c3a9ed7..00000000000 --- a/engine/access/ingestion2/engine_test.go +++ /dev/null @@ -1,468 +0,0 @@ -package ingestion2 - -import ( - "context" - "os" - "sync" - "testing" - "time" - - "github.com/jordanschalm/lockctx" - "github.com/rs/zerolog" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - hotmodel "github.com/onflow/flow-go/consensus/hotstuff/model" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/counters" - downloadermock "github.com/onflow/flow-go/module/executiondatasync/execution_data/mock" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/mempool/stdmap" - "github.com/onflow/flow-go/module/metrics" - modulemock "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/module/signature" - "github.com/onflow/flow-go/module/state_synchronization/indexer" - "github.com/onflow/flow-go/network/channels" - mocknetwork "github.com/onflow/flow-go/network/mock" - protocol "github.com/onflow/flow-go/state/protocol/mock" - "github.com/onflow/flow-go/storage" - storagemock "github.com/onflow/flow-go/storage/mock" - "github.com/onflow/flow-go/storage/operation/pebbleimpl" - "github.com/onflow/flow-go/storage/store" - "github.com/onflow/flow-go/utils/unittest" - "github.com/onflow/flow-go/utils/unittest/mocks" -) - -type Suite struct { - suite.Suite - - // protocol state - proto struct { - state *protocol.FollowerState - snapshot *protocol.Snapshot - params *protocol.Params - } - - me *modulemock.Local - net *mocknetwork.EngineRegistry - request *modulemock.Requester - obsIdentity *flow.Identity - provider *mocknetwork.Engine - blocks *storagemock.Blocks - headers *storagemock.Headers - collections *storagemock.Collections - transactions *storagemock.Transactions - receipts *storagemock.ExecutionReceipts - results *storagemock.ExecutionResults - seals *storagemock.Seals - - conduit *mocknetwork.Conduit - downloader *downloadermock.Downloader - sealedBlock *flow.Header - finalizedBlock *flow.Header - log zerolog.Logger - blockMap map[uint64]*flow.Block - rootBlock *flow.Block - - collectionExecutedMetric *indexer.CollectionExecutedMetricImpl - - ctx context.Context - cancel context.CancelFunc - - db storage.DB - dbDir string - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter - lockManager lockctx.Manager -} - -func TestIngestEngine(t *testing.T) { - suite.Run(t, new(Suite)) -} - -// TearDownTest stops the engine and cleans up the db -func (s *Suite) TearDownTest() { - s.cancel() - err := os.RemoveAll(s.dbDir) - s.Require().NoError(err) -} - -func (s *Suite) SetupTest() { - s.log = unittest.Logger() - s.ctx, s.cancel = context.WithCancel(context.Background()) - pdb, dbDir := unittest.TempPebbleDB(s.T()) - s.db = pebbleimpl.ToDB(pdb) - s.dbDir = dbDir - s.lockManager = storage.NewTestingLockManager() - - s.obsIdentity = unittest.IdentityFixture(unittest.WithRole(flow.RoleAccess)) - - s.blocks = storagemock.NewBlocks(s.T()) - // mock out protocol state - s.proto.state = new(protocol.FollowerState) - s.proto.snapshot = new(protocol.Snapshot) - s.proto.params = new(protocol.Params) - s.finalizedBlock = unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) - s.proto.state.On("Identity").Return(s.obsIdentity, nil) - s.proto.state.On("Params").Return(s.proto.params) - s.proto.snapshot.On("Head").Return( - func() *flow.Header { - return s.finalizedBlock - }, - nil, - ).Maybe() - - s.me = modulemock.NewLocal(s.T()) - s.me.On("NodeID").Return(s.obsIdentity.NodeID).Maybe() - s.net = mocknetwork.NewEngineRegistry(s.T()) - conduit := mocknetwork.NewConduit(s.T()) - s.net.On("Register", channels.ReceiveReceipts, mock.Anything). - Return(conduit, nil). - Once() - s.request = modulemock.NewRequester(s.T()) - s.provider = mocknetwork.NewEngine(s.T()) - s.blocks = storagemock.NewBlocks(s.T()) - s.headers = storagemock.NewHeaders(s.T()) - s.collections = new(storagemock.Collections) - s.receipts = new(storagemock.ExecutionReceipts) - s.transactions = new(storagemock.Transactions) - s.results = new(storagemock.ExecutionResults) - collectionsToMarkFinalized := stdmap.NewTimes(100) - collectionsToMarkExecuted := stdmap.NewTimes(100) - blocksToMarkExecuted := stdmap.NewTimes(100) - blockTransactions := stdmap.NewIdentifierMap(100) - - s.proto.state.On("Identity").Return(s.obsIdentity, nil) - s.proto.state.On("Params").Return(s.proto.params) - - blockCount := 5 - s.blockMap = make(map[uint64]*flow.Block, blockCount) - s.rootBlock = unittest.Block.Genesis(flow.Emulator) - parent := s.rootBlock.ToHeader() - - for i := 0; i < blockCount; i++ { - block := unittest.BlockWithParentFixture(parent) - // update for next iteration - parent = block.ToHeader() - s.blockMap[block.Height] = block - } - s.finalizedBlock = parent - - s.blocks.On("ByHeight", mock.AnythingOfType("uint64")).Return( - mocks.ConvertStorageOutput( - mocks.StorageMapGetter(s.blockMap), - func(block *flow.Block) *flow.Block { return block }, - ), - ).Maybe() - - s.proto.snapshot.On("Head").Return( - func() *flow.Header { - return s.finalizedBlock - }, - nil, - ).Maybe() - s.proto.state.On("Final").Return(s.proto.snapshot, nil) - - // Mock the finalized root block header with height 0. - header := unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) - s.proto.params.On("FinalizedRoot").Return(header, nil) - - var err error - s.collectionExecutedMetric, err = indexer.NewCollectionExecutedMetricImpl( - s.log, - metrics.NewNoopCollector(), - collectionsToMarkFinalized, - collectionsToMarkExecuted, - blocksToMarkExecuted, - s.collections, - s.blocks, - blockTransactions, - ) - require.NoError(s.T(), err) -} - -func (s *Suite) TestComponentShutdown() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - // start then shut down the engine - unittest.AssertClosesBefore(s.T(), eng.Ready(), 10*time.Millisecond) - s.cancel() - unittest.AssertClosesBefore(s.T(), eng.Done(), 10*time.Millisecond) - - err := eng.Process(channels.ReceiveReceipts, unittest.IdentifierFixture(), &flow.ExecutionReceipt{}) - s.Assert().ErrorIs(err, component.ErrComponentShutdown) -} - -// initEngineAndSyncer create new instance of ingestion engine and collection collectionSyncer. -// It waits until the ingestion engine starts. -func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, Syncer) { - processedHeightInitializer := store.NewConsumerProgress(s.db, module.ConsumeProgressIngestionEngineBlockHeight) - - lastFullBlockHeight, err := store.NewConsumerProgress(s.db, module.ConsumeProgressLastFullBlockHeight).Initialize(s.finalizedBlock.Height) - require.NoError(s.T(), err) - - s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeight) - require.NoError(s.T(), err) - - // TODO: Update test to use new ingestion2 architecture with JobProcessor - // This test needs to be refactored to use the new JobProcessor-based architecture. - // For now, we skip creating the new Syncer since FinalizedBlockProcessor still uses the old collections.Syncer - // and the test architecture needs to be updated to match the new design. - // The syncer will be created using collections.CreateSyncer in the actual node builder. - // For testing, we'll use nil for now until the test is fully refactored. - var syncer Syncer = nil - - // TODO: FinalizedBlockProcessor still uses old collections.Syncer - // This needs to be updated to work with the new architecture - // For now, pass nil and skip this part of the test - blockProcessor, err := NewFinalizedBlockProcessor( - s.log, - s.proto.state, - s.blocks, - s.results, - processedHeightInitializer, - nil, // TODO: Update FinalizedBlockProcessor to not require old syncer - s.collectionExecutedMetric, - ) - require.NoError(s.T(), err) - - eng, err := New( - s.log, - s.net, - blockProcessor, - syncer, - s.receipts, - s.collectionExecutedMetric, - ) - - require.NoError(s.T(), err) - - eng.ComponentManager.Start(ctx) - <-eng.Ready() - - return eng, syncer -} - -// mockCollectionsForBlock mocks collections for block -func (s *Suite) mockCollectionsForBlock(block *flow.Block) { - // we should query the block once and index the guarantee payload once - for _, g := range block.Payload.Guarantees { - collection := unittest.CollectionFixture(1) - light := collection.Light() - s.collections.On("LightByID", g.CollectionID).Return(light, nil).Twice() - } -} - -// generateBlock prepares block with payload and specified guarantee.SignerIndices -func (s *Suite) generateBlock(clusterCommittee flow.IdentitySkeletonList, snap *protocol.Snapshot) *flow.Block { - block := unittest.BlockFixture( - unittest.Block.WithPayload(unittest.PayloadFixture( - unittest.WithGuarantees(unittest.CollectionGuaranteesFixture(4)...), - unittest.WithExecutionResults(unittest.ExecutionResultFixture()), - unittest.WithSeals(unittest.Seal.Fixture()), - )), - ) - - refBlockID := unittest.IdentifierFixture() - for _, guarantee := range block.Payload.Guarantees { - guarantee.ReferenceBlockID = refBlockID - // guarantee signers must be cluster committee members, so that access will fetch collection from - // the signers that are specified by guarantee.SignerIndices - indices, err := signature.EncodeSignersToIndices(clusterCommittee.NodeIDs(), clusterCommittee.NodeIDs()) - require.NoError(s.T(), err) - guarantee.SignerIndices = indices - } - - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - return block -} - -// TestOnFinalizedBlock checks that when a block is received, a request for each individual collection is made -func (s *Suite) TestOnFinalizedBlockSingle() { - cluster := protocol.NewCluster(s.T()) - epoch := protocol.NewCommittedEpoch(s.T()) - epochs := protocol.NewEpochQuery(s.T()) - snap := protocol.NewSnapshot(s.T()) - - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs.On("Current").Return(epoch, nil) - snap.On("Epochs").Return(epochs) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - cluster.On("Members").Return(clusterCommittee, nil) - - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - block := s.generateBlock(clusterCommittee, snap) - block.Height = s.finalizedBlock.Height + 1 - s.blockMap[block.Height] = block - s.mockCollectionsForBlock(block) - s.finalizedBlock = block.ToHeader() - - hotstuffBlock := hotmodel.Block{ - BlockID: block.ID(), - } - - // expect that the block storage is indexed with each of the collection guarantee - s.blocks.On("IndexBlockContainingCollectionGuarantees", block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() - for _, seal := range block.Payload.Seals { - s.results.On("Index", seal.BlockID, seal.ResultID).Return(nil).Once() - } - - missingCollectionCount := 4 - wg := sync.WaitGroup{} - wg.Add(missingCollectionCount) - - for _, cg := range block.Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Run(func(args mock.Arguments) { - // Ensure the test does not complete its work faster than necessary - wg.Done() - }).Once() - } - s.request.On("Force").Return().Once() - - // process the block through the finalized callback - eng.OnFinalizedBlock(&hotstuffBlock) - - unittest.RequireReturnsBefore(s.T(), wg.Wait, 100*time.Millisecond, "expect to process new block before timeout") - - // assert that the block was retrieved and all collections were requested - s.headers.AssertExpectations(s.T()) - s.request.AssertNumberOfCalls(s.T(), "EntityByID", len(block.Payload.Guarantees)) - s.results.AssertNumberOfCalls(s.T(), "Index", len(block.Payload.Seals)) -} - -// TestOnFinalizedBlockSeveralBlocksAhead checks OnFinalizedBlock with a block several blocks newer than the last block processed -func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { - cluster := protocol.NewCluster(s.T()) - epoch := protocol.NewCommittedEpoch(s.T()) - epochs := protocol.NewEpochQuery(s.T()) - snap := protocol.NewSnapshot(s.T()) - - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs.On("Current").Return(epoch, nil) - snap.On("Epochs").Return(epochs) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - cluster.On("Members").Return(clusterCommittee, nil) - - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - newBlocksCount := 3 - startHeight := s.finalizedBlock.Height + 1 - blocks := make([]*flow.Block, newBlocksCount) - - // generate the test blocks, cgs and collections - for i := 0; i < newBlocksCount; i++ { - block := s.generateBlock(clusterCommittee, snap) - block.Height = startHeight + uint64(i) - s.blockMap[block.Height] = block - blocks[i] = block - s.mockCollectionsForBlock(block) - s.finalizedBlock = block.ToHeader() - } - - // latest of all the new blocks which are newer than the last block processed - latestBlock := blocks[2] - - // block several blocks newer than the last block processed - hotstuffBlock := hotmodel.Block{ - BlockID: latestBlock.ID(), - } - - missingCollectionCountPerBlock := 4 - wg := sync.WaitGroup{} - wg.Add(missingCollectionCountPerBlock * newBlocksCount) - - // expected all new blocks after last block processed - for _, block := range blocks { - s.blocks.On("IndexBlockContainingCollectionGuarantees", block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() - - for _, cg := range block.Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Run(func(args mock.Arguments) { - // Ensure the test does not complete its work faster than necessary, so we can check all expected results - wg.Done() - }).Once() - } - s.request.On("Force").Return().Once() - - for _, seal := range block.Payload.Seals { - s.results.On("Index", seal.BlockID, seal.ResultID).Return(nil).Once() - } - } - - eng.OnFinalizedBlock(&hotstuffBlock) - - unittest.RequireReturnsBefore(s.T(), wg.Wait, 100*time.Millisecond, "expect to process all blocks before timeout") - - expectedEntityByIDCalls := 0 - expectedIndexCalls := 0 - for _, block := range blocks { - expectedEntityByIDCalls += len(block.Payload.Guarantees) - expectedIndexCalls += len(block.Payload.Seals) - } - - s.headers.AssertExpectations(s.T()) - s.blocks.AssertNumberOfCalls(s.T(), "IndexBlockContainingCollectionGuarantees", newBlocksCount) - s.request.AssertNumberOfCalls(s.T(), "EntityByID", expectedEntityByIDCalls) - s.results.AssertNumberOfCalls(s.T(), "Index", expectedIndexCalls) -} - -// TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed -func (s *Suite) TestExecutionReceiptsAreIndexed() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - collection := unittest.CollectionFixture(5) - light := collection.Light() - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", &collection).Return(light, nil).Once() - block := unittest.BlockFixture( - unittest.Block.WithHeight(0), - unittest.Block.WithPayload( - unittest.PayloadFixture(unittest.WithGuarantees([]*flow.CollectionGuarantee{}...)), - ), - ) - s.blocks.On("ByID", mock.Anything).Return(block, nil) - - // for each transaction in the collection, we should store it - needed := make(map[flow.Identifier]struct{}) - for _, txID := range light.Transactions { - needed[txID] = struct{}{} - } - s.transactions.On("Store", mock.Anything).Return(nil).Run( - func(args mock.Arguments) { - tx := args.Get(0).(*flow.TransactionBody) - _, pending := needed[tx.ID()] - s.Assert().True(pending, "tx not pending (%x)", tx.ID()) - }, - ) - er1 := unittest.ExecutionReceiptFixture() - er2 := unittest.ExecutionReceiptFixture() - - s.receipts.On("Store", mock.Anything).Return(nil) - s.blocks.On("ByID", er1.ExecutionResult.BlockID).Return(nil, storage.ErrNotFound) - - s.receipts.On("Store", mock.Anything).Return(nil) - s.blocks.On("ByID", er2.ExecutionResult.BlockID).Return(nil, storage.ErrNotFound) - - err := eng.persistExecutionReceipt(er1) - require.NoError(s.T(), err) - - err = eng.persistExecutionReceipt(er2) - require.NoError(s.T(), err) - - s.receipts.AssertExpectations(s.T()) - s.results.AssertExpectations(s.T()) - s.receipts.AssertExpectations(s.T()) -} diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index b2d92e65e69..705ca437d3d 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -11,7 +11,7 @@ import ( "github.com/onflow/flow-core-contracts/lib/go/templates" - "github.com/onflow/flow-go/engine/access/ingestion/collections" + "github.com/onflow/flow-go/engine/access/ingestion2" "github.com/onflow/flow-go/fvm/blueprints" "github.com/onflow/flow-go/fvm/storage/derived" "github.com/onflow/flow-go/fvm/systemcontracts" @@ -31,7 +31,7 @@ type IndexerCore struct { fvmEnv templates.Environment metrics module.ExecutionStateIndexerMetrics collectionExecutedMetric module.CollectionExecutedMetric - collectionIndexer collections.CollectionIndexer + collectionIndexer ingestion2.JobProcessor registers storage.RegisterIndex headers storage.Headers @@ -63,7 +63,7 @@ func New( scheduledTransactions storage.ScheduledTransactions, chainID flow.ChainID, derivedChainData *derived.DerivedChainData, - collectionIndexer collections.CollectionIndexer, + collectionIndexer ingestion2.JobProcessor, collectionExecutedMetric module.CollectionExecutedMetric, lockManager lockctx.Manager, ) *IndexerCore { @@ -231,7 +231,7 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti // index all standard (non-system) collections standardCollections := data.StandardCollections() if len(standardCollections) > 0 { - err := c.collectionIndexer.IndexCollections(standardCollections) + err := c.collectionIndexer.OnReceivedCollectionsForBlock(header.Height, standardCollections) if err != nil { return fmt.Errorf("could not index collections: %w", err) } From 59684ee74aa6e145fb58ee23ddd9d025b14c946b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 6 Nov 2025 20:39:03 -0800 Subject: [PATCH 006/126] update access node builder --- .../node_builder/access_node_builder.go | 112 +++++++++++++----- .../ingestion2/collections/syncer_factory.go | 17 ++- 2 files changed, 100 insertions(+), 29 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 6e776b4d05f..6451e4a1e3d 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -40,9 +40,10 @@ import ( recovery "github.com/onflow/flow-go/consensus/recovery/protocol" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/index" - "github.com/onflow/flow-go/engine/access/ingestion" "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" + "github.com/onflow/flow-go/engine/access/ingestion2" + ingestion2collections "github.com/onflow/flow-go/engine/access/ingestion2/collections" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" "github.com/onflow/flow-go/engine/access/rest/router" @@ -170,6 +171,7 @@ type AccessNodeConfig struct { PublicNetworkConfig PublicNetworkConfig TxResultCacheSize uint executionDataIndexingEnabled bool + ediLagThreshold uint64 registersDBPath string checkpointFile string scriptExecutorConfig query.QueryConfig @@ -275,6 +277,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { MaxRetryDelay: edrequester.DefaultMaxRetryDelay, }, executionDataIndexingEnabled: false, + ediLagThreshold: 100, executionDataPrunerHeightRangeTarget: 0, executionDataPrunerThreshold: pruner.DefaultThreshold, executionDataPruningInterval: pruner.DefaultPruningInterval, @@ -333,6 +336,8 @@ type FlowAccessNodeBuilder struct { ExecutionIndexerCore *indexer.IndexerCore CollectionIndexer *collections.Indexer CollectionSyncer *collections.Syncer + JobProcessor ingestion2.JobProcessor + CollectionSyncer2 ingestion2.Syncer ScriptExecutor *backend.ScriptExecutor RegistersAsyncStore *execution.RegistersAsyncStore Reporter *index.Reporter @@ -360,7 +365,7 @@ type FlowAccessNodeBuilder struct { SyncEngineParticipantsProviderFactory func() module.IdentifierProvider // engines - IngestEng *ingestion.Engine + IngestEng *ingestion2.Engine RequestEng *requester.Engine FollowerEng *followereng.ComplianceEngine SyncEng *synceng.Engine @@ -942,6 +947,11 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess return nil, fmt.Errorf("could not create derived chain data: %w", err) } + // JobProcessor should already be created in the "collection syncer and job processor" module + if builder.JobProcessor == nil { + return nil, fmt.Errorf("JobProcessor must be created before execution data indexer") + } + builder.ExecutionIndexerCore = indexer.New( builder.Logger, metrics.NewExecutionStateIndexerCollector(), @@ -955,7 +965,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess notNil(builder.scheduledTransactions), builder.RootChainID, indexerDerivedChainData, - notNil(builder.CollectionIndexer), + notNil(builder.JobProcessor), notNil(builder.collectionExecutedMetric), node.StorageLockMgr, ) @@ -1413,6 +1423,10 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "execution-data-indexing-enabled", defaultConfig.executionDataIndexingEnabled, "whether to enable the execution data indexing") + flags.Uint64Var(&builder.ediLagThreshold, + "edi-lag-threshold", + defaultConfig.ediLagThreshold, + "threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. Default: 100") flags.StringVar(&builder.registersDBPath, "execution-state-dir", defaultConfig.registersDBPath, "directory to use for execution-state database") flags.StringVar(&builder.checkpointFile, "execution-state-checkpoint", defaultConfig.checkpointFile, "execution-state checkpoint file") @@ -1951,6 +1965,45 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { processedFinalizedBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) return nil }). + Module("collection syncer and job processor", func(node *cmd.NodeConfig) error { + // Get ProcessedHeightRecorder from ExecutionIndexer if available + // Note: ExecutionIndexer may not exist yet, so we'll pass nil and update later if needed + var processedHeightRecorder execution_data.ProcessedHeightRecorder + if builder.ExecutionIndexer != nil { + processedHeightRecorder = builder.ExecutionIndexer + } + + // Create syncer and job processor + syncerResult, err := ingestion2collections.CreateSyncer( + node.Logger, + node.EngineRegistry, + node.State, + node.Me, + node.Storage.Blocks, + notNil(builder.collections), + node.Storage.Guarantees, + builder.ProtocolDB, + node.StorageLockMgr, + processedFinalizedBlockHeight, + notNil(builder.collectionExecutedMetric), + processedHeightRecorder, + ingestion2collections.CreateSyncerConfig{ + MaxProcessing: 10, // TODO: make configurable + MaxSearchAhead: 0, // TODO: make configurable + EDILagThreshold: builder.ediLagThreshold, + }, + ) + if err != nil { + return fmt.Errorf("could not create collection syncer: %w", err) + } + + // Store the results for use in other components + builder.JobProcessor = syncerResult.JobProcessor + builder.CollectionSyncer2 = syncerResult.Syncer + builder.RequestEng = syncerResult.RequestEng + + return nil + }). Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { rootBlockHeight := node.State.Params().FinalizedRoot().Height @@ -2198,20 +2251,10 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return builder.RpcEng, nil }). Component("requester engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - requestEng, err := requester.New( - node.Logger.With().Str("entity", "collection").Logger(), - node.Metrics.Engine, - node.EngineRegistry, - node.Me, - node.State, - channels.RequestCollections, - filter.HasRole[flow.Identity](flow.RoleCollection), - func() flow.Entity { return new(flow.Collection) }, - ) - if err != nil { - return nil, fmt.Errorf("could not create requester engine: %w", err) + // RequestEng should already be created in the "collection syncer and job processor" module + if builder.RequestEng == nil { + return nil, fmt.Errorf("RequestEng must be created before requester engine component") } - builder.RequestEng = requestEng collectionIndexer, err := collections.NewIndexer( node.Logger, @@ -2266,29 +2309,44 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { ) } - ingestEng, err := ingestion.New( + // CollectionSyncer2 should already be created in the "collection syncer and job processor" module + if builder.CollectionSyncer2 == nil { + return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") + } + + // Create FinalizedBlockProcessor (still uses old syncer temporarily) + finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( node.Logger, - node.EngineRegistry, node.State, - node.Me, node.Storage.Blocks, node.Storage.Results, - node.Storage.Receipts, processedFinalizedBlockHeight, - notNil(builder.CollectionSyncer), - notNil(builder.CollectionIndexer), + notNil(builder.CollectionSyncer), // TODO: update FinalizedBlockProcessor to not need this notNil(builder.collectionExecutedMetric), - notNil(builder.TxResultErrorMessagesCore), ) if err != nil { - return nil, err + return nil, fmt.Errorf("could not create finalized block processor: %w", err) + } + + // Create ingestion2 engine + ingestEng, err := ingestion2.New( + node.Logger, + node.EngineRegistry, + finalizedBlockProcessor, + builder.CollectionSyncer2, + node.Storage.Receipts, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) } + builder.IngestEng = ingestEng - ingestionDependable.Init(builder.IngestEng) - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.IngestEng.OnFinalizedBlock) + ingestionDependable.Init(ingestEng) + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) - return builder.IngestEng, nil + return ingestEng, nil }) if builder.storeTxResultErrorMessages { diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/ingestion2/collections/syncer_factory.go index 05ba4e1c09d..0353a453fdc 100644 --- a/engine/access/ingestion2/collections/syncer_factory.go +++ b/engine/access/ingestion2/collections/syncer_factory.go @@ -30,6 +30,13 @@ type CreateSyncerConfig struct { EDILagThreshold uint64 } +// CreateSyncerResult holds the results of CreateSyncer. +type CreateSyncerResult struct { + Syncer *Syncer + JobProcessor ingestion2.JobProcessor + RequestEng *requester.Engine +} + // CreateSyncer creates a new Syncer component with all its dependencies. // This function is in the collections package to avoid import cycles: // - collections package already imports ingestion2 (for interfaces) @@ -51,6 +58,8 @@ type CreateSyncerConfig struct { // - processedHeightRecorder: Recorder for execution data processed heights (can be nil if EDI is disabled) // - config: Configuration for the syncer // +// Returns both the Syncer and JobProcessor so they can be reused in other components. +// // No error returns are expected during normal operation. func CreateSyncer( log zerolog.Logger, @@ -66,7 +75,7 @@ func CreateSyncer( collectionExecutedMetric module.CollectionExecutedMetric, processedHeightRecorder execution_data.ProcessedHeightRecorder, config CreateSyncerConfig, -) (*Syncer, error) { +) (*CreateSyncerResult, error) { // Create requester engine for requesting collections requestEng, err := requester.New( log.With().Str("entity", "collection").Logger(), @@ -146,5 +155,9 @@ func CreateSyncer( return nil, fmt.Errorf("could not create syncer: %w", err) } - return syncer, nil + return &CreateSyncerResult{ + Syncer: syncer, + JobProcessor: jobProcessor, + RequestEng: requestEng, + }, nil } From 3353177177b227c515621dde1e018bdba7c8a913 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 10 Nov 2025 10:29:04 -0800 Subject: [PATCH 007/126] simplify the job queue --- .../node_builder/access_node_builder.go | 28 ++--- .../access/ingestion2/collection_syncing.go | 16 +-- .../collections/block_collection_indexer.go | 2 +- .../collections/edi_height_provider.go | 5 +- .../ingestion2/collections/job_processor.go | 102 +++--------------- .../access/ingestion2/collections/syncer.go | 6 +- .../ingestion2/collections/syncer_factory.go | 14 --- .../ingestion2/finalized_block_processor.go | 9 -- module/jobqueue.go | 9 +- 9 files changed, 50 insertions(+), 141 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 6451e4a1e3d..034eef4609c 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -171,7 +171,6 @@ type AccessNodeConfig struct { PublicNetworkConfig PublicNetworkConfig TxResultCacheSize uint executionDataIndexingEnabled bool - ediLagThreshold uint64 registersDBPath string checkpointFile string scriptExecutorConfig query.QueryConfig @@ -277,7 +276,6 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { MaxRetryDelay: edrequester.DefaultMaxRetryDelay, }, executionDataIndexingEnabled: false, - ediLagThreshold: 100, executionDataPrunerHeightRangeTarget: 0, executionDataPrunerThreshold: pruner.DefaultThreshold, executionDataPruningInterval: pruner.DefaultPruningInterval, @@ -1423,10 +1421,6 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "execution-data-indexing-enabled", defaultConfig.executionDataIndexingEnabled, "whether to enable the execution data indexing") - flags.Uint64Var(&builder.ediLagThreshold, - "edi-lag-threshold", - defaultConfig.ediLagThreshold, - "threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. Default: 100") flags.StringVar(&builder.registersDBPath, "execution-state-dir", defaultConfig.registersDBPath, "directory to use for execution-state database") flags.StringVar(&builder.checkpointFile, "execution-state-checkpoint", defaultConfig.checkpointFile, "execution-state checkpoint file") @@ -1737,6 +1731,8 @@ func (builder *FlowAccessNodeBuilder) enqueueRelayNetwork() { func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { var processedFinalizedBlockHeight storage.ConsumerProgressInitializer + var fetchAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer + var syncAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer if builder.executionDataSyncEnabled { @@ -1961,18 +1957,13 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.lightTransactionResults) return nil }). - Module("processed finalized block height consumer progress", func(node *cmd.NodeConfig) error { + Module("block height consumer progress", func(node *cmd.NodeConfig) error { processedFinalizedBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) + fetchAndIndexedCollectionsBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + syncAndIndexedCollectionsBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight) return nil }). Module("collection syncer and job processor", func(node *cmd.NodeConfig) error { - // Get ProcessedHeightRecorder from ExecutionIndexer if available - // Note: ExecutionIndexer may not exist yet, so we'll pass nil and update later if needed - var processedHeightRecorder execution_data.ProcessedHeightRecorder - if builder.ExecutionIndexer != nil { - processedHeightRecorder = builder.ExecutionIndexer - } - // Create syncer and job processor syncerResult, err := ingestion2collections.CreateSyncer( node.Logger, @@ -1984,13 +1975,11 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.Storage.Guarantees, builder.ProtocolDB, node.StorageLockMgr, - processedFinalizedBlockHeight, + fetchAndIndexedCollectionsBlockHeight, notNil(builder.collectionExecutedMetric), - processedHeightRecorder, ingestion2collections.CreateSyncerConfig{ - MaxProcessing: 10, // TODO: make configurable - MaxSearchAhead: 0, // TODO: make configurable - EDILagThreshold: builder.ediLagThreshold, + MaxProcessing: 10, // TODO: make configurable + MaxSearchAhead: 0, // TODO: make configurable }, ) if err != nil { @@ -2321,7 +2310,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.Storage.Blocks, node.Storage.Results, processedFinalizedBlockHeight, - notNil(builder.CollectionSyncer), // TODO: update FinalizedBlockProcessor to not need this notNil(builder.collectionExecutedMetric), ) if err != nil { diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go index 90d169621e9..3d06e6e9255 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/ingestion2/collection_syncing.go @@ -28,21 +28,21 @@ type CollectionRequester interface { // BlockCollectionIndexer stores and indexes collections for a given block height. type BlockCollectionIndexer interface { - // OnReceivedCollectionsForBlock stores and indexes collections for a given block height. - OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error + // IndexCollectionsForBlock stores and indexes collections for a given block height. + IndexCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error } // Implements the job lifecycle for a single block height. type JobProcessor interface { - ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error - OnReceiveCollection(collection *flow.Collection) error - OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester + ProcessJobConcurrently(ctx irrecoverable.SignalerContext, job module.Job, done func()) error + // OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester } // EDIHeightProvider provides the latest height for which execution data indexer has collections. // This can be nil if execution data indexing is disabled. type EDIHeightProvider interface { - HighestIndexedHeight() (uint64, error) + HighestIndexedHeight() uint64 + GetExecutionDataByHeight(height uint64) ([]*flow.Collection, error) } // Syncer is a component that consumes finalized block jobs and processes them @@ -54,3 +54,7 @@ type Syncer interface { Head() (uint64, error) Size() uint } + +type ExecutionDataProcessor interface { + OnNewExectuionData() +} diff --git a/engine/access/ingestion2/collections/block_collection_indexer.go b/engine/access/ingestion2/collections/block_collection_indexer.go index b823a73ce87..f36a908fe8d 100644 --- a/engine/access/ingestion2/collections/block_collection_indexer.go +++ b/engine/access/ingestion2/collections/block_collection_indexer.go @@ -48,7 +48,7 @@ func NewBlockCollectionIndexer( // OnReceivedCollectionsForBlock stores and indexes collections for a given block height. // // No error returns are expected during normal operation. -func (bci *blockCollectionIndexerImpl) OnReceivedCollectionsForBlock( +func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( blockHeight uint64, cols []*flow.Collection, ) error { diff --git a/engine/access/ingestion2/collections/edi_height_provider.go b/engine/access/ingestion2/collections/edi_height_provider.go index 83162e75651..c52016ea44c 100644 --- a/engine/access/ingestion2/collections/edi_height_provider.go +++ b/engine/access/ingestion2/collections/edi_height_provider.go @@ -23,7 +23,6 @@ func NewProcessedHeightRecorderWrapper(recorder execution_data.ProcessedHeightRe // HighestIndexedHeight returns the highest block height for which EDI has indexed collections. // It wraps the ProcessedHeightRecorder's HighestCompleteHeight method. -func (p *ProcessedHeightRecorderWrapper) HighestIndexedHeight() (uint64, error) { - return p.recorder.HighestCompleteHeight(), nil +func (p *ProcessedHeightRecorderWrapper) HighestIndexedHeight() uint64 { + return p.recorder.HighestCompleteHeight() } - diff --git a/engine/access/ingestion2/collections/job_processor.go b/engine/access/ingestion2/collections/job_processor.go index a67a82bd47d..cb789c409af 100644 --- a/engine/access/ingestion2/collections/job_processor.go +++ b/engine/access/ingestion2/collections/job_processor.go @@ -12,22 +12,16 @@ import ( "github.com/onflow/flow-go/storage" ) -// EDIHeightProvider provides the latest height for which execution data indexer has collections. -type EDIHeightProvider interface { - // HighestIndexedHeight returns the highest block height for which EDI has indexed collections. - HighestIndexedHeight() (uint64, error) -} - // JobProcessor implements the job lifecycle for collection indexing. // It orchestrates the flow: request → receive → index → complete. +// TODO: rename to fetch_job_processor type JobProcessor struct { - mcq ingestion2.MissingCollectionQueue - indexer ingestion2.BlockCollectionIndexer - requester ingestion2.CollectionRequester - blocks storage.Blocks - collections storage.CollectionsReader - ediHeightProvider EDIHeightProvider // when set as nil, EDI is considered disabled - ediLagThreshold uint64 + mcq ingestion2.MissingCollectionQueue + indexer ingestion2.BlockCollectionIndexer + requester ingestion2.CollectionRequester + blocks storage.Blocks + collections storage.CollectionsReader + newExecutionDataIndexed <-chan struct{} } var _ ingestion2.JobProcessor = (*JobProcessor)(nil) @@ -40,8 +34,6 @@ var _ ingestion2.JobProcessor = (*JobProcessor)(nil) // - requester: CollectionRequester for requesting collections from the network // - blocks: Blocks storage for retrieving block data // - collections: Collections storage reader for checking if collections already exist -// - ediHeightProvider: Provider for EDI's highest indexed height (can be nil if EDI is disabled) -// - ediLagThreshold: Threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. // Set to a very large value to effectively disable fetching and rely only on EDI. // // No error returns are expected during normal operation. @@ -51,17 +43,13 @@ func NewJobProcessor( requester ingestion2.CollectionRequester, blocks storage.Blocks, collections storage.CollectionsReader, - ediHeightProvider EDIHeightProvider, - ediLagThreshold uint64, ) *JobProcessor { return &JobProcessor{ - mcq: mcq, - indexer: indexer, - requester: requester, - blocks: blocks, - collections: collections, - ediHeightProvider: ediHeightProvider, - ediLagThreshold: ediLagThreshold, + mcq: mcq, + indexer: indexer, + requester: requester, + blocks: blocks, + collections: collections, } } @@ -70,7 +58,7 @@ func NewJobProcessor( // and optionally requests them based on EDI lag. // // No error returns are expected during normal operation. -func (jp *JobProcessor) ProcessJob( +func (jp *JobProcessor) ProcessJobConcurrently( ctx irrecoverable.SignalerContext, job module.Job, done func(), @@ -112,18 +100,10 @@ func (jp *JobProcessor) ProcessJob( return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) } - // Check EDI lag to decide whether to fetch collections - shouldFetch, err := jp.shouldFetchCollections(blockHeight) + // Request collections from collection nodes + err = jp.requester.RequestCollections(collectionIDs) if err != nil { - return fmt.Errorf("failed to check EDI lag for block height %d: %w", blockHeight, err) - } - - if shouldFetch { - // Request collections from collection nodes - err = jp.requester.RequestCollections(collectionIDs) - if err != nil { - return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) - } + return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) } return nil @@ -142,7 +122,7 @@ func (jp *JobProcessor) OnReceiveCollection(collection *flow.Collection) error { } // Block became complete, index it - err := jp.indexer.OnReceivedCollectionsForBlock(height, collections) + err := jp.indexer.IndexCollectionsForBlock(height, collections) if err != nil { return fmt.Errorf("failed to index collections for block height %d: %w", height, err) } @@ -153,33 +133,6 @@ func (jp *JobProcessor) OnReceiveCollection(collection *flow.Collection) error { return nil } -// OnReceivedCollectionsForBlock is called by the execution data indexer when collections are received. -// It forwards collections to MCQ and handles block completion. -// -// The blockHeight parameter is provided by EDI to indicate which block these collections belong to. -// -// No error returns are expected during normal operation. -func (jp *JobProcessor) OnReceivedCollectionsForBlock( - blockHeight uint64, - collections []*flow.Collection, -) error { - // Mark the block as indexed (this invokes the callback if the height exists) - queued := jp.mcq.IsHeightQueued(blockHeight) - if !queued { - // If the height is not queued, nothing to do - return nil - } - - err := jp.indexer.OnReceivedCollectionsForBlock(blockHeight, collections) - if err != nil { - return fmt.Errorf("failed to index collections for block height %d: %w", blockHeight, err) - } - - jp.mcq.OnIndexedForBlock(blockHeight) - - return nil -} - // getMissingCollections retrieves the block and returns collection guarantees that are missing. // Only collections that are not already in storage are returned. func (jp *JobProcessor) getMissingCollections(blockHeight uint64) ([]*flow.CollectionGuarantee, error) { @@ -206,24 +159,3 @@ func (jp *JobProcessor) getMissingCollections(blockHeight uint64) ([]*flow.Colle return missingGuarantees, nil } - -// shouldFetchCollections determines whether to fetch collections based on EDI lag. -// Returns true if collections should be fetched, false if we should wait for EDI. -func (jp *JobProcessor) shouldFetchCollections(blockHeight uint64) (bool, error) { - // If EDI is not available, always fetch - if jp.ediHeightProvider == nil { - return true, nil - } - - ediHeight, err := jp.ediHeightProvider.HighestIndexedHeight() - if err != nil { - // If we can't get EDI height, err on the side of fetching to avoid blocking - return true, nil - } - - // Calculate lag - lag := blockHeight - ediHeight - - // If lag exceeds threshold, fetch collections - return lag > jp.ediLagThreshold, nil -} diff --git a/engine/access/ingestion2/collections/syncer.go b/engine/access/ingestion2/collections/syncer.go index 247b30d6e3a..a32350ca3fb 100644 --- a/engine/access/ingestion2/collections/syncer.go +++ b/engine/access/ingestion2/collections/syncer.go @@ -58,7 +58,7 @@ func NewSyncer( // Create an adapter function that wraps the JobProcessor interface processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { - err := jobProcessor.ProcessJob(ctx, job, done) + err := jobProcessor.ProcessJobConcurrently(ctx, job, done) if err != nil { ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) } @@ -93,17 +93,19 @@ func (s *Syncer) OnFinalizedBlock() { } // LastProcessedIndex returns the last processed job index. +// Optional methods, not required for operation but useful for monitoring. func (s *Syncer) LastProcessedIndex() uint64 { return s.consumer.LastProcessedIndex() } // Head returns the highest job index available. +// Optional methods, not required for operation but useful for monitoring. func (s *Syncer) Head() (uint64, error) { return s.consumer.Head() } // Size returns the number of in-memory jobs that the consumer is processing. +// Optional methods, not required for operation but useful for monitoring. func (s *Syncer) Size() uint { return s.consumer.Size() } - diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/ingestion2/collections/syncer_factory.go index 0353a453fdc..68cb9924d2d 100644 --- a/engine/access/ingestion2/collections/syncer_factory.go +++ b/engine/access/ingestion2/collections/syncer_factory.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" @@ -25,9 +24,6 @@ type CreateSyncerConfig struct { MaxProcessing uint64 // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. MaxSearchAhead uint64 - // EDILagThreshold is the threshold in blocks. If (blockHeight - ediHeight) > threshold, fetch collections. - // Set to a very large value to effectively disable fetching and rely only on EDI. - EDILagThreshold uint64 } // CreateSyncerResult holds the results of CreateSyncer. @@ -55,7 +51,6 @@ type CreateSyncerResult struct { // - lockManager: Lock manager for coordinating database access // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionExecutedMetric: Metrics collector for tracking collection indexing -// - processedHeightRecorder: Recorder for execution data processed heights (can be nil if EDI is disabled) // - config: Configuration for the syncer // // Returns both the Syncer and JobProcessor so they can be reused in other components. @@ -73,7 +68,6 @@ func CreateSyncer( lockManager lockctx.Manager, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, - processedHeightRecorder execution_data.ProcessedHeightRecorder, config CreateSyncerConfig, ) (*CreateSyncerResult, error) { // Create requester engine for requesting collections @@ -109,12 +103,6 @@ func CreateSyncer( guarantees, ) - // Wrap ProcessedHeightRecorder as EDIHeightProvider if provided - var ediHeightProvider ingestion2.EDIHeightProvider - if processedHeightRecorder != nil { - ediHeightProvider = NewProcessedHeightRecorderWrapper(processedHeightRecorder) - } - // Create JobProcessor jobProcessor := NewJobProcessor( mcq, @@ -122,8 +110,6 @@ func CreateSyncer( collectionRequester, blocks, collections, - ediHeightProvider, - config.EDILagThreshold, ) // Register handler for received collections diff --git a/engine/access/ingestion2/finalized_block_processor.go b/engine/access/ingestion2/finalized_block_processor.go index 0c6700450bb..8c89d3e6c22 100644 --- a/engine/access/ingestion2/finalized_block_processor.go +++ b/engine/access/ingestion2/finalized_block_processor.go @@ -6,7 +6,6 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -50,7 +49,6 @@ type FinalizedBlockProcessor struct { executionResults storage.ExecutionResults - collectionSyncer *collections.Syncer collectionExecutedMetric module.CollectionExecutedMetric } @@ -64,7 +62,6 @@ func NewFinalizedBlockProcessor( blocks storage.Blocks, executionResults storage.ExecutionResults, finalizedProcessedHeight storage.ConsumerProgressInitializer, - syncer *collections.Syncer, collectionExecutedMetric module.CollectionExecutedMetric, ) (*FinalizedBlockProcessor, error) { reader := jobqueue.NewFinalizedBlockReader(state, blocks) @@ -79,7 +76,6 @@ func NewFinalizedBlockProcessor( blocks: blocks, executionResults: executionResults, consumerNotifier: consumerNotifier, - collectionSyncer: syncer, collectionExecutedMetric: collectionExecutedMetric, } @@ -159,11 +155,6 @@ func (p *FinalizedBlockProcessor) indexFinalizedBlock(block *flow.Block) error { } } - err = p.collectionSyncer.RequestCollectionsForBlock(block.Height, block.Payload.Guarantees) - if err != nil { - return fmt.Errorf("could not request collections for block: %w", err) - } - p.collectionExecutedMetric.BlockFinalized(block) return nil diff --git a/module/jobqueue.go b/module/jobqueue.go index 9150956504b..3aae850dac0 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -13,7 +13,14 @@ const ( ConsumeProgressExecutionDataIndexerBlockHeight = "ConsumeProgressExecutionDataIndexerBlockHeight" - ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" + ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" + + // the highest block height that has indexed collections from fetching missing collections from LN + ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight" + + // the highest block height that has indexed collections from syncing execution data from EN + ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight" + ConsumeProgressEngineTxErrorMessagesBlockHeight = "ConsumeProgressEngineTxErrorMessagesBlockHeight" ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" ) From 92fe3563e59e117c06479b454efcc1b74de173f2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 10 Nov 2025 10:29:21 -0800 Subject: [PATCH 008/126] add execution data processor --- .../collections/execution_data_processor.go | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100644 engine/access/ingestion2/collections/execution_data_processor.go diff --git a/engine/access/ingestion2/collections/execution_data_processor.go b/engine/access/ingestion2/collections/execution_data_processor.go new file mode 100644 index 00000000000..9b8e8e987df --- /dev/null +++ b/engine/access/ingestion2/collections/execution_data_processor.go @@ -0,0 +1,75 @@ +package collections + +import ( + "fmt" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/irrecoverable" +) + +type ExecutionDataProcessor struct { + newExecutionDataIndexed chan struct{} + ediHeightProvider ingestion2.EDIHeightProvider + indexer ingestion2.BlockCollectionIndexer + // state + processedHeight *counters.PersistentStrictMonotonicCounter +} + +var _ ingestion2.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) + +func NewExecutionDataProcessor( + ediHeightProvider ingestion2.EDIHeightProvider, + processedHeight *counters.PersistentStrictMonotonicCounter, +) *ExecutionDataProcessor { + return &ExecutionDataProcessor{ + newExecutionDataIndexed: make(chan struct{}, 1), + ediHeightProvider: ediHeightProvider, + processedHeight: processedHeight, + } +} + +func (edp *ExecutionDataProcessor) OnNewExectuionData() { + select { + case edp.newExecutionDataIndexed <- struct{}{}: + default: + // if the channel is full, no need to block, just return. + // once the worker loop processes the buffered signal, it will + // process the next height all the way to the highest available height. + } +} + +func (edp *ExecutionDataProcessor) WrokerLoop(ctx irrecoverable.SignalerContext) error { + // using a single threaded loop to index each exectuion for height + // since indexing collections is blocking anyway, and reading the execution data + // is quick, because we cache for 100 heights. + for { + select { + case <-ctx.Done(): + return nil + case <-edp.newExecutionDataIndexed: + highestAvailableHeight := edp.ediHeightProvider.HighestIndexedHeight() + lowestMissing := edp.processedHeight.Value() + 1 + + for height := lowestMissing; height <= highestAvailableHeight; height++ { + collections, err := edp.ediHeightProvider.GetExecutionDataByHeight(height) + if err != nil { + return fmt.Errorf("failed to get execution data for height %d: %w", height, err) + } + + // TODO: since both fetcher and exectuion data processor are the data source of + // collections, before indexing the collections, double check if it was indexed + // by the fetcher already by simply comparing the missing height with the + // fetcher's lowest height. + // if fetcher's lowest height is higher than the missing height, it means the collections + // has been indexed by the fetcher already, no need to index again. + // And make sure reading the fetcher's lowest height is cheap operation (only hitting RW lock) + + err = edp.indexer.IndexCollectionsForBlock(height, collections) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", height, err) + } + } + } + } +} From 79076c45f92ce31b606a081f02e546816740fed1 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 10 Nov 2025 13:05:28 -0800 Subject: [PATCH 009/126] update exectuion data processor --- .../access/ingestion2/collection_syncing.go | 4 +- .../collections/edi_height_provider.go | 45 +++++++++++++------ .../collections/execution_data_processor.go | 2 +- 3 files changed, 35 insertions(+), 16 deletions(-) diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/ingestion2/collection_syncing.go index 3d06e6e9255..af3b4d7b7f1 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/ingestion2/collection_syncing.go @@ -1,6 +1,8 @@ package ingestion2 import ( + "context" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -42,7 +44,7 @@ type JobProcessor interface { // This can be nil if execution data indexing is disabled. type EDIHeightProvider interface { HighestIndexedHeight() uint64 - GetExecutionDataByHeight(height uint64) ([]*flow.Collection, error) + GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) } // Syncer is a component that consumes finalized block jobs and processes them diff --git a/engine/access/ingestion2/collections/edi_height_provider.go b/engine/access/ingestion2/collections/edi_height_provider.go index c52016ea44c..19115b04d0c 100644 --- a/engine/access/ingestion2/collections/edi_height_provider.go +++ b/engine/access/ingestion2/collections/edi_height_provider.go @@ -1,28 +1,45 @@ package collections import ( + "context" + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) -var _ ingestion2.EDIHeightProvider = (*ProcessedHeightRecorderWrapper)(nil) +var _ ingestion2.EDIHeightProvider = (*ediHeightProvider)(nil) -// ProcessedHeightRecorderWrapper wraps execution_data.ProcessedHeightRecorder to implement -// ingestion2.EDIHeightProvider interface. -type ProcessedHeightRecorderWrapper struct { - recorder execution_data.ProcessedHeightRecorder +// ediHeightProvider implements EDIHeightProvider by querying ExecutionDataCache. +type ediHeightProvider struct { + cache execution_data.ExecutionDataCache + highestExectuionDataHeight counters.Reader } -// NewProcessedHeightRecorderWrapper creates a new wrapper that implements EDIHeightProvider -// by wrapping the given ProcessedHeightRecorder. -func NewProcessedHeightRecorderWrapper(recorder execution_data.ProcessedHeightRecorder) *ProcessedHeightRecorderWrapper { - return &ProcessedHeightRecorderWrapper{ - recorder: recorder, +// NewEDIHeightProvider creates a new EDIHeightProvider that reads from the given ExecutionDataCache. +// The headers storage is used to determine the search range for finding available heights. +func NewEDIHeightProvider( + cache execution_data.ExecutionDataCache, + highestExectuionDataHeight counters.Reader, +) *ediHeightProvider { + return &ediHeightProvider{ + cache: cache, + highestExectuionDataHeight: highestExectuionDataHeight, } } -// HighestIndexedHeight returns the highest block height for which EDI has indexed collections. -// It wraps the ProcessedHeightRecorder's HighestCompleteHeight method. -func (p *ProcessedHeightRecorderWrapper) HighestIndexedHeight() uint64 { - return p.recorder.HighestCompleteHeight() +// HighestIndexedHeight returns the highest block height for which execution data is available. +func (p *ediHeightProvider) HighestIndexedHeight() uint64 { + return p.highestExectuionDataHeight.Value() +} + +// GetExecutionDataByHeight returns the execution data for the given block height. +func (p *ediHeightProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { + blockExecutionData, err := p.cache.ByHeight(ctx, height) + if err != nil { + return nil, err + } + + return blockExecutionData.StandardCollections(), nil } diff --git a/engine/access/ingestion2/collections/execution_data_processor.go b/engine/access/ingestion2/collections/execution_data_processor.go index 9b8e8e987df..8ba77e230e6 100644 --- a/engine/access/ingestion2/collections/execution_data_processor.go +++ b/engine/access/ingestion2/collections/execution_data_processor.go @@ -52,7 +52,7 @@ func (edp *ExecutionDataProcessor) WrokerLoop(ctx irrecoverable.SignalerContext) lowestMissing := edp.processedHeight.Value() + 1 for height := lowestMissing; height <= highestAvailableHeight; height++ { - collections, err := edp.ediHeightProvider.GetExecutionDataByHeight(height) + collections, err := edp.ediHeightProvider.GetExecutionDataByHeight(ctx, height) if err != nil { return fmt.Errorf("failed to get execution data for height %d: %w", height, err) } From c4ffce4595d6891b1e7fd1eda8ddfac511ad8537 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 10 Nov 2025 14:37:24 -0800 Subject: [PATCH 010/126] update exectuion data processor --- .../access/ingestion2/collections/edi_height_provider.go | 8 ++++---- .../ingestion2/collections/execution_data_processor.go | 2 ++ .../requester/execution_data_requester.go | 1 + 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/engine/access/ingestion2/collections/edi_height_provider.go b/engine/access/ingestion2/collections/edi_height_provider.go index 19115b04d0c..526861a8b7b 100644 --- a/engine/access/ingestion2/collections/edi_height_provider.go +++ b/engine/access/ingestion2/collections/edi_height_provider.go @@ -4,8 +4,8 @@ import ( "context" "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) @@ -14,14 +14,14 @@ var _ ingestion2.EDIHeightProvider = (*ediHeightProvider)(nil) // ediHeightProvider implements EDIHeightProvider by querying ExecutionDataCache. type ediHeightProvider struct { cache execution_data.ExecutionDataCache - highestExectuionDataHeight counters.Reader + highestExectuionDataHeight tracker.ExecutionDataTracker } // NewEDIHeightProvider creates a new EDIHeightProvider that reads from the given ExecutionDataCache. // The headers storage is used to determine the search range for finding available heights. func NewEDIHeightProvider( cache execution_data.ExecutionDataCache, - highestExectuionDataHeight counters.Reader, + highestExectuionDataHeight tracker.ExecutionDataTracker, ) *ediHeightProvider { return &ediHeightProvider{ cache: cache, @@ -31,7 +31,7 @@ func NewEDIHeightProvider( // HighestIndexedHeight returns the highest block height for which execution data is available. func (p *ediHeightProvider) HighestIndexedHeight() uint64 { - return p.highestExectuionDataHeight.Value() + return p.highestExectuionDataHeight.GetHighestHeight() } // GetExecutionDataByHeight returns the execution data for the given block height. diff --git a/engine/access/ingestion2/collections/execution_data_processor.go b/engine/access/ingestion2/collections/execution_data_processor.go index 8ba77e230e6..0a5173715f6 100644 --- a/engine/access/ingestion2/collections/execution_data_processor.go +++ b/engine/access/ingestion2/collections/execution_data_processor.go @@ -20,11 +20,13 @@ var _ ingestion2.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( ediHeightProvider ingestion2.EDIHeightProvider, + indexer ingestion2.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, ) *ExecutionDataProcessor { return &ExecutionDataProcessor{ newExecutionDataIndexed: make(chan struct{}, 1), ediHeightProvider: ediHeightProvider, + indexer: indexer, processedHeight: processedHeight, } } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 35bb21a08a9..237366ef8a7 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -146,6 +146,7 @@ func New( downloader execution_data.Downloader, execDataCache *cache.ExecutionDataCache, processedHeight storage.ConsumerProgressInitializer, + // ? (leo) what's the difference from processedHeight? processedNotifications storage.ConsumerProgressInitializer, state protocol.State, headers storage.Headers, From aefe0f2d680a51ccc0f36342875a49c0efba45d3 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 10 Nov 2025 14:48:46 -0800 Subject: [PATCH 011/126] update exectuion data processor factory --- .../collections/execution_data_processor.go | 38 +++++++++++++--- .../execution_data_processor_factory.go | 45 +++++++++++++++++++ 2 files changed, 76 insertions(+), 7 deletions(-) create mode 100644 engine/access/ingestion2/collections/execution_data_processor_factory.go diff --git a/engine/access/ingestion2/collections/execution_data_processor.go b/engine/access/ingestion2/collections/execution_data_processor.go index 0a5173715f6..e2703e42742 100644 --- a/engine/access/ingestion2/collections/execution_data_processor.go +++ b/engine/access/ingestion2/collections/execution_data_processor.go @@ -4,11 +4,13 @@ import ( "fmt" "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" ) type ExecutionDataProcessor struct { + component.Component newExecutionDataIndexed chan struct{} ediHeightProvider ingestion2.EDIHeightProvider indexer ingestion2.BlockCollectionIndexer @@ -17,18 +19,28 @@ type ExecutionDataProcessor struct { } var _ ingestion2.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) +var _ component.Component = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( ediHeightProvider ingestion2.EDIHeightProvider, indexer ingestion2.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, ) *ExecutionDataProcessor { - return &ExecutionDataProcessor{ + edp := &ExecutionDataProcessor{ newExecutionDataIndexed: make(chan struct{}, 1), ediHeightProvider: ediHeightProvider, indexer: indexer, processedHeight: processedHeight, } + + // Build component manager with worker loop + cm := component.NewComponentManagerBuilder(). + AddWorker(edp.workerLoop). + Build() + + edp.Component = cm + + return edp } func (edp *ExecutionDataProcessor) OnNewExectuionData() { @@ -41,14 +53,16 @@ func (edp *ExecutionDataProcessor) OnNewExectuionData() { } } -func (edp *ExecutionDataProcessor) WrokerLoop(ctx irrecoverable.SignalerContext) error { - // using a single threaded loop to index each exectuion for height +func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + // using a single threaded loop to index each execution for height // since indexing collections is blocking anyway, and reading the execution data // is quick, because we cache for 100 heights. for { select { case <-ctx.Done(): - return nil + return case <-edp.newExecutionDataIndexed: highestAvailableHeight := edp.ediHeightProvider.HighestIndexedHeight() lowestMissing := edp.processedHeight.Value() + 1 @@ -56,10 +70,11 @@ func (edp *ExecutionDataProcessor) WrokerLoop(ctx irrecoverable.SignalerContext) for height := lowestMissing; height <= highestAvailableHeight; height++ { collections, err := edp.ediHeightProvider.GetExecutionDataByHeight(ctx, height) if err != nil { - return fmt.Errorf("failed to get execution data for height %d: %w", height, err) + ctx.Throw(fmt.Errorf("failed to get execution data for height %d: %w", height, err)) + return } - // TODO: since both fetcher and exectuion data processor are the data source of + // TODO: since both fetcher and execution data processor are the data source of // collections, before indexing the collections, double check if it was indexed // by the fetcher already by simply comparing the missing height with the // fetcher's lowest height. @@ -68,8 +83,17 @@ func (edp *ExecutionDataProcessor) WrokerLoop(ctx irrecoverable.SignalerContext) // And make sure reading the fetcher's lowest height is cheap operation (only hitting RW lock) err = edp.indexer.IndexCollectionsForBlock(height, collections) + // TODO: handle already exists + if err != nil { + ctx.Throw(fmt.Errorf("failed to index collections for block height %d: %w", height, err)) + return + } + + // Update processed height after successful indexing + err = edp.processedHeight.Set(height) if err != nil { - return fmt.Errorf("failed to index collections for block height %d: %w", height, err) + ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) + return } } } diff --git a/engine/access/ingestion2/collections/execution_data_processor_factory.go b/engine/access/ingestion2/collections/execution_data_processor_factory.go new file mode 100644 index 00000000000..c2651401e76 --- /dev/null +++ b/engine/access/ingestion2/collections/execution_data_processor_factory.go @@ -0,0 +1,45 @@ +package collections + +import ( + "fmt" + + "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/subscription/tracker" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage" +) + +// CreateExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. +// +// Parameters: +// - cache: Execution data cache for retrieving execution data by height +// - executionDataTracker: Tracker for execution data that provides the highest available height +// - processedHeight: Consumer progress for tracking processed heights +// - indexer: Block collection indexer for indexing collections +// +// Returns: +// - *ExecutionDataProcessor: A new ExecutionDataProcessor instance +// - error: An error if the processor could not be created +// +// No errors are expected during normal operation. +func CreateExecutionDataProcessor( + cache execution_data.ExecutionDataCache, + executionDataTracker tracker.ExecutionDataTracker, + processedHeight storage.ConsumerProgress, + indexer ingestion2.BlockCollectionIndexer, +) (*ExecutionDataProcessor, error) { + // Create EDI height provider + ediHeightProvider := NewEDIHeightProvider(cache, executionDataTracker) + + // Convert ConsumerProgress to PersistentStrictMonotonicCounter + processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) + if err != nil { + return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) + } + + // Create the execution data processor + processor := NewExecutionDataProcessor(ediHeightProvider, indexer, processedHeightCounter) + + return processor, nil +} From 7ff67724f3b1990a7a3c516349e18aa4877533f0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 12 Nov 2025 18:03:50 -0800 Subject: [PATCH 012/126] refactor collection sync --- .../node_builder/access_node_builder.go | 147 +++++++++++----- .../node_builder/collection_ingestion.go | 135 +++++++++++++++ .../collection_sync.go} | 17 +- .../collections/block_collection_indexer.go | 4 +- .../collections/edi_height_provider.go | 4 +- .../collections/execution_data_processor.go | 12 +- .../execution_data_processor_factory.go | 4 +- .../collections/job_processor.go | 16 +- .../collections/missing_collection_queue.go | 4 +- .../collections/requester.go | 7 +- .../collections/syncer.go | 8 +- .../factory/execution_data_processor.go | 46 +++++ .../factory/syncer.go} | 29 ++-- engine/access/finalized_indexer/processor.go | 138 +++++++++++++++ .../{ingestion2 => ingest_receipt}/engine.go | 24 +-- engine/access/ingestion2/README.md | 146 ---------------- .../ingestion2/finalized_block_processor.go | 161 ------------------ module/jobqueue.go | 15 +- .../indexer/indexer_core.go | 30 ---- 19 files changed, 491 insertions(+), 456 deletions(-) create mode 100644 cmd/access/node_builder/collection_ingestion.go rename engine/access/{ingestion2/collection_syncing.go => collection_sync/collection_sync.go} (94%) rename engine/access/{ingestion2 => collection_sync}/collections/block_collection_indexer.go (94%) rename engine/access/{ingestion2 => collection_sync}/collections/edi_height_provider.go (92%) rename engine/access/{ingestion2 => collection_sync}/collections/execution_data_processor.go (89%) rename engine/access/{ingestion2 => collection_sync}/collections/execution_data_processor_factory.go (93%) rename engine/access/{ingestion2 => collection_sync}/collections/job_processor.go (91%) rename engine/access/{ingestion2 => collection_sync}/collections/missing_collection_queue.go (98%) rename engine/access/{ingestion2 => collection_sync}/collections/requester.go (92%) rename engine/access/{ingestion2 => collection_sync}/collections/syncer.go (94%) create mode 100644 engine/access/collection_sync/factory/execution_data_processor.go rename engine/access/{ingestion2/collections/syncer_factory.go => collection_sync/factory/syncer.go} (84%) create mode 100644 engine/access/finalized_indexer/processor.go rename engine/access/{ingestion2 => ingest_receipt}/engine.go (84%) delete mode 100644 engine/access/ingestion2/README.md delete mode 100644 engine/access/ingestion2/finalized_block_processor.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 034eef4609c..665ff1bf336 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -44,6 +44,7 @@ import ( "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" "github.com/onflow/flow-go/engine/access/ingestion2" ingestion2collections "github.com/onflow/flow-go/engine/access/ingestion2/collections" + ingestion2factory "github.com/onflow/flow-go/engine/access/ingestion2/factory" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" "github.com/onflow/flow-go/engine/access/rest/router" @@ -327,6 +328,7 @@ type FlowAccessNodeBuilder struct { ExecutionDataDownloader execution_data.Downloader PublicBlobService network.BlobService ExecutionDataRequester state_synchronization.ExecutionDataRequester + ExecutionDataDistributor *edrequester.ExecutionDataDistributor ExecutionDataStore execution_data.ExecutionDataStore ExecutionDataBlobstore blobs.Blobstore ExecutionDataCache *execdatacache.ExecutionDataCache @@ -738,6 +740,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } execDataDistributor = edrequester.NewExecutionDataDistributor() + builder.ExecutionDataDistributor = execDataDistributor // Execution Data cache with a downloader as the backend. This is used by the requester // to download and cache execution data for each block. It shares a cache backend instance @@ -945,11 +948,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess return nil, fmt.Errorf("could not create derived chain data: %w", err) } - // JobProcessor should already be created in the "collection syncer and job processor" module - if builder.JobProcessor == nil { - return nil, fmt.Errorf("JobProcessor must be created before execution data indexer") - } - builder.ExecutionIndexerCore = indexer.New( builder.Logger, metrics.NewExecutionStateIndexerCollector(), @@ -963,7 +961,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess notNil(builder.scheduledTransactions), builder.RootChainID, indexerDerivedChainData, - notNil(builder.JobProcessor), notNil(builder.collectionExecutedMetric), node.StorageLockMgr, ) @@ -1965,7 +1962,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { }). Module("collection syncer and job processor", func(node *cmd.NodeConfig) error { // Create syncer and job processor - syncerResult, err := ingestion2collections.CreateSyncer( + syncerResult, err := ingestion2factory.CreateSyncer( node.Logger, node.EngineRegistry, node.State, @@ -1977,7 +1974,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.StorageLockMgr, fetchAndIndexedCollectionsBlockHeight, notNil(builder.collectionExecutedMetric), - ingestion2collections.CreateSyncerConfig{ + ingestion2factory.CreateSyncerConfig{ MaxProcessing: 10, // TODO: make configurable MaxSearchAhead: 0, // TODO: make configurable }, @@ -2286,56 +2283,122 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.RequestEng.WithHandle(collectionSyncer.OnCollectionDownloaded) return builder.RequestEng, nil - }). - Component("ingestion engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.storeTxResultErrorMessages { - builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( - node.Logger, - notNil(builder.txResultErrorMessageProvider), - builder.transactionResultErrorMessages, - notNil(builder.ExecNodeIdentitiesProvider), - node.StorageLockMgr, - ) + }) + if builder.executionDataSyncEnabled { + builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + // ExecutionDataCache should already be created in BuildExecutionSyncComponents + if builder.ExecutionDataCache == nil { + return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") } - // CollectionSyncer2 should already be created in the "collection syncer and job processor" module - if builder.CollectionSyncer2 == nil { - return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") + // Create execution data tracker for the processor + // This is similar to the one created in state stream engine but used for collection indexing + broadcaster := engine.NewBroadcaster() + highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() + if err != nil { + return nil, fmt.Errorf("could not get highest consecutive height: %w", err) } - // Create FinalizedBlockProcessor (still uses old syncer temporarily) - finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( - node.Logger, + useIndex := builder.executionDataIndexingEnabled + executionDataTracker := subscriptiontracker.NewExecutionDataTracker( + builder.Logger, node.State, - node.Storage.Blocks, - node.Storage.Results, - processedFinalizedBlockHeight, - notNil(builder.collectionExecutedMetric), + builder.executionDataConfig.InitialBlockHeight, + node.Storage.Headers, + broadcaster, + highestAvailableHeight, + builder.EventsIndex, + useIndex, ) + + // Initialize processed height + rootBlockHeight := node.State.Params().FinalizedRoot().Height + processedHeight, err := syncAndIndexedCollectionsBlockHeight.Initialize(rootBlockHeight) if err != nil { - return nil, fmt.Errorf("could not create finalized block processor: %w", err) + return nil, fmt.Errorf("could not initialize processed height: %w", err) } - // Create ingestion2 engine - ingestEng, err := ingestion2.New( - node.Logger, - node.EngineRegistry, - finalizedBlockProcessor, - builder.CollectionSyncer2, - node.Storage.Receipts, + // Get BlockCollectionIndexer from syncer result + // The indexer is created in the CreateSyncer call, we need to extract it + // For now, we'll create a new one since we need it for the processor + blockCollectionIndexer := ingestion2collections.NewBlockCollectionIndexer( notNil(builder.collectionExecutedMetric), + node.StorageLockMgr, + builder.ProtocolDB, + notNil(builder.collections), + ) + + // Create execution data processor + executionDataProcessor, err := ingestion2factory.CreateExecutionDataProcessor( + notNil(builder.ExecutionDataCache), + executionDataTracker, + processedHeight, + blockCollectionIndexer, ) if err != nil { - return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) + return nil, fmt.Errorf("could not create execution data processor: %w", err) } - builder.IngestEng = ingestEng - - ingestionDependable.Init(ingestEng) - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) + // Setup requester to notify processor when new execution data is received + if builder.ExecutionDataDistributor != nil { + builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { + executionDataTracker.OnExecutionData(executionData) + executionDataProcessor.OnNewExectuionData() + }) + } - return ingestEng, nil + return executionDataProcessor, nil }) + } + builder.Component("ingestion engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + if builder.storeTxResultErrorMessages { + builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( + node.Logger, + notNil(builder.txResultErrorMessageProvider), + builder.transactionResultErrorMessages, + notNil(builder.ExecNodeIdentitiesProvider), + node.StorageLockMgr, + ) + } + + // CollectionSyncer2 should already be created in the "collection syncer and job processor" module + if builder.CollectionSyncer2 == nil { + return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") + } + + // Create FinalizedBlockProcessor (still uses old syncer temporarily) + finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( + node.Logger, + node.State, + node.Storage.Blocks, + node.Storage.Results, + processedFinalizedBlockHeight, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create finalized block processor: %w", err) + } + + // Create ingestion2 engine + ingestEng, err := ingestion2.New( + node.Logger, + node.EngineRegistry, + finalizedBlockProcessor, + builder.CollectionSyncer2, + node.Storage.Receipts, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) + } + + builder.IngestEng = ingestEng + + ingestionDependable.Init(ingestEng) + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) + + return ingestEng, nil + }) if builder.storeTxResultErrorMessages { builder. diff --git a/cmd/access/node_builder/collection_ingestion.go b/cmd/access/node_builder/collection_ingestion.go new file mode 100644 index 00000000000..27f70694a5c --- /dev/null +++ b/cmd/access/node_builder/collection_ingestion.go @@ -0,0 +1,135 @@ +package node_builder + +import ( + "fmt" + + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/ingestion2" + ingestion2collections "github.com/onflow/flow-go/engine/access/ingestion2/collections" + ingestion2factory "github.com/onflow/flow-go/engine/access/ingestion2/factory" + subscriptiontracker "github.com/onflow/flow-go/engine/access/subscription/tracker" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage/store" +) + +// InitializeIngestionEngine creates and initializes the ingestion2 engine with collections.Syncer. +// This function should be called from within a Component function where node is available. +func InitializeIngestionEngine(builder *FlowAccessNodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + // CollectionSyncer2 should already be created in the "collection syncer and job processor" module + if builder.CollectionSyncer2 == nil { + return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") + } + + // Create processedFinalizedBlockHeight if not already created + processedFinalizedBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) + + // Create FinalizedBlockProcessor + finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( + node.Logger, + node.State, + node.Storage.Blocks, + node.Storage.Results, + processedFinalizedBlockHeight, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create finalized block processor: %w", err) + } + + // Create ingestion2 engine + ingestEng, err := ingestion2.New( + node.Logger, + node.EngineRegistry, + finalizedBlockProcessor, + builder.CollectionSyncer2, + node.Storage.Receipts, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) + } + + builder.IngestEng = ingestEng + + // Create and initialize ingestionDependable if IndexerDependencies exists + if builder.IndexerDependencies != nil { + ingestionDependable := module.NewProxiedReadyDoneAware() + builder.IndexerDependencies.Add(ingestionDependable) + ingestionDependable.Init(ingestEng) + } + + // Add OnFinalizedBlock consumer if FollowerDistributor exists + if builder.FollowerDistributor != nil { + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) + } + + return ingestEng, nil +} + +// InitializeExecutionDataCollectionIndexer creates and initializes the execution data collection indexer +// with ExecutionDataProcessor. This function should be called from within a Component function where node is available. +func InitializeExecutionDataCollectionIndexer(builder *FlowAccessNodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + // ExecutionDataCache should already be created in BuildExecutionSyncComponents + if builder.ExecutionDataCache == nil { + return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") + } + + // Create execution data tracker for the processor + // This is similar to the one created in state stream engine but used for collection indexing + broadcaster := engine.NewBroadcaster() + highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() + if err != nil { + return nil, fmt.Errorf("could not get highest consecutive height: %w", err) + } + + useIndex := builder.executionDataIndexingEnabled + executionDataTracker := subscriptiontracker.NewExecutionDataTracker( + builder.Logger, + node.State, + builder.executionDataConfig.InitialBlockHeight, + node.Storage.Headers, + broadcaster, + highestAvailableHeight, + builder.EventsIndex, + useIndex, + ) + + // Initialize processed height + rootBlockHeight := node.State.Params().FinalizedRoot().Height + syncAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight) + processedHeight, err := syncAndIndexedCollectionsBlockHeight.Initialize(rootBlockHeight) + if err != nil { + return nil, fmt.Errorf("could not initialize processed height: %w", err) + } + + // Create BlockCollectionIndexer + blockCollectionIndexer := ingestion2collections.NewBlockCollectionIndexer( + notNil(builder.collectionExecutedMetric), + node.StorageLockMgr, + builder.ProtocolDB, + notNil(builder.collections), + ) + + // Create execution data processor + executionDataProcessor, err := ingestion2factory.CreateExecutionDataProcessor( + notNil(builder.ExecutionDataCache), + executionDataTracker, + processedHeight, + blockCollectionIndexer, + ) + if err != nil { + return nil, fmt.Errorf("could not create execution data processor: %w", err) + } + + // Setup requester to notify processor when new execution data is received + if builder.ExecutionDataDistributor != nil { + builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { + executionDataTracker.OnExecutionData(executionData) + executionDataProcessor.OnNewExectuionData() + }) + } + + return executionDataProcessor, nil +} diff --git a/engine/access/ingestion2/collection_syncing.go b/engine/access/collection_sync/collection_sync.go similarity index 94% rename from engine/access/ingestion2/collection_syncing.go rename to engine/access/collection_sync/collection_sync.go index af3b4d7b7f1..5ea215ec1cb 100644 --- a/engine/access/ingestion2/collection_syncing.go +++ b/engine/access/collection_sync/collection_sync.go @@ -1,4 +1,4 @@ -package ingestion2 +package collection_sync import ( "context" @@ -37,14 +37,6 @@ type BlockCollectionIndexer interface { // Implements the job lifecycle for a single block height. type JobProcessor interface { ProcessJobConcurrently(ctx irrecoverable.SignalerContext, job module.Job, done func()) error - // OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error // called by EDI or requester -} - -// EDIHeightProvider provides the latest height for which execution data indexer has collections. -// This can be nil if execution data indexing is disabled. -type EDIHeightProvider interface { - HighestIndexedHeight() uint64 - GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) } // Syncer is a component that consumes finalized block jobs and processes them @@ -57,6 +49,13 @@ type Syncer interface { Size() uint } +// EDIHeightProvider provides the latest height for which execution data indexer has collections. +// This can be nil if execution data indexing is disabled. +type EDIHeightProvider interface { + HighestIndexedHeight() uint64 + GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) +} + type ExecutionDataProcessor interface { OnNewExectuionData() } diff --git a/engine/access/ingestion2/collections/block_collection_indexer.go b/engine/access/collection_sync/collections/block_collection_indexer.go similarity index 94% rename from engine/access/ingestion2/collections/block_collection_indexer.go rename to engine/access/collection_sync/collections/block_collection_indexer.go index f36a908fe8d..ac92bc16cb8 100644 --- a/engine/access/ingestion2/collections/block_collection_indexer.go +++ b/engine/access/collection_sync/collections/block_collection_indexer.go @@ -5,7 +5,7 @@ import ( "github.com/jordanschalm/lockctx" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/storage" @@ -20,7 +20,7 @@ type blockCollectionIndexerImpl struct { collections storage.Collections } -var _ ingestion2.BlockCollectionIndexer = (*blockCollectionIndexerImpl)(nil) +var _ collection_sync.BlockCollectionIndexer = (*blockCollectionIndexerImpl)(nil) // NewBlockCollectionIndexer creates a new BlockCollectionIndexer implementation. // diff --git a/engine/access/ingestion2/collections/edi_height_provider.go b/engine/access/collection_sync/collections/edi_height_provider.go similarity index 92% rename from engine/access/ingestion2/collections/edi_height_provider.go rename to engine/access/collection_sync/collections/edi_height_provider.go index 526861a8b7b..ae0697c48b9 100644 --- a/engine/access/ingestion2/collections/edi_height_provider.go +++ b/engine/access/collection_sync/collections/edi_height_provider.go @@ -3,13 +3,13 @@ package collections import ( "context" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) -var _ ingestion2.EDIHeightProvider = (*ediHeightProvider)(nil) +var _ collection_sync.EDIHeightProvider = (*ediHeightProvider)(nil) // ediHeightProvider implements EDIHeightProvider by querying ExecutionDataCache. type ediHeightProvider struct { diff --git a/engine/access/ingestion2/collections/execution_data_processor.go b/engine/access/collection_sync/collections/execution_data_processor.go similarity index 89% rename from engine/access/ingestion2/collections/execution_data_processor.go rename to engine/access/collection_sync/collections/execution_data_processor.go index e2703e42742..f7191c13138 100644 --- a/engine/access/ingestion2/collections/execution_data_processor.go +++ b/engine/access/collection_sync/collections/execution_data_processor.go @@ -3,7 +3,7 @@ package collections import ( "fmt" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" @@ -12,18 +12,18 @@ import ( type ExecutionDataProcessor struct { component.Component newExecutionDataIndexed chan struct{} - ediHeightProvider ingestion2.EDIHeightProvider - indexer ingestion2.BlockCollectionIndexer + ediHeightProvider collection_sync.EDIHeightProvider + indexer collection_sync.BlockCollectionIndexer // state processedHeight *counters.PersistentStrictMonotonicCounter } -var _ ingestion2.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) +var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) var _ component.Component = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( - ediHeightProvider ingestion2.EDIHeightProvider, - indexer ingestion2.BlockCollectionIndexer, + ediHeightProvider collection_sync.EDIHeightProvider, + indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ diff --git a/engine/access/ingestion2/collections/execution_data_processor_factory.go b/engine/access/collection_sync/collections/execution_data_processor_factory.go similarity index 93% rename from engine/access/ingestion2/collections/execution_data_processor_factory.go rename to engine/access/collection_sync/collections/execution_data_processor_factory.go index c2651401e76..ef76df69fcd 100644 --- a/engine/access/ingestion2/collections/execution_data_processor_factory.go +++ b/engine/access/collection_sync/collections/execution_data_processor_factory.go @@ -3,7 +3,7 @@ package collections import ( "fmt" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" @@ -27,7 +27,7 @@ func CreateExecutionDataProcessor( cache execution_data.ExecutionDataCache, executionDataTracker tracker.ExecutionDataTracker, processedHeight storage.ConsumerProgress, - indexer ingestion2.BlockCollectionIndexer, + indexer collection_sync.BlockCollectionIndexer, ) (*ExecutionDataProcessor, error) { // Create EDI height provider ediHeightProvider := NewEDIHeightProvider(cache, executionDataTracker) diff --git a/engine/access/ingestion2/collections/job_processor.go b/engine/access/collection_sync/collections/job_processor.go similarity index 91% rename from engine/access/ingestion2/collections/job_processor.go rename to engine/access/collection_sync/collections/job_processor.go index cb789c409af..7911bd65aa0 100644 --- a/engine/access/ingestion2/collections/job_processor.go +++ b/engine/access/collection_sync/collections/job_processor.go @@ -4,7 +4,7 @@ import ( "errors" "fmt" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" @@ -16,15 +16,15 @@ import ( // It orchestrates the flow: request → receive → index → complete. // TODO: rename to fetch_job_processor type JobProcessor struct { - mcq ingestion2.MissingCollectionQueue - indexer ingestion2.BlockCollectionIndexer - requester ingestion2.CollectionRequester + mcq collection_sync.MissingCollectionQueue + indexer collection_sync.BlockCollectionIndexer + requester collection_sync.CollectionRequester blocks storage.Blocks collections storage.CollectionsReader newExecutionDataIndexed <-chan struct{} } -var _ ingestion2.JobProcessor = (*JobProcessor)(nil) +var _ collection_sync.JobProcessor = (*JobProcessor)(nil) // NewJobProcessor creates a new JobProcessor. // @@ -38,9 +38,9 @@ var _ ingestion2.JobProcessor = (*JobProcessor)(nil) // // No error returns are expected during normal operation. func NewJobProcessor( - mcq ingestion2.MissingCollectionQueue, - indexer ingestion2.BlockCollectionIndexer, - requester ingestion2.CollectionRequester, + mcq collection_sync.MissingCollectionQueue, + indexer collection_sync.BlockCollectionIndexer, + requester collection_sync.CollectionRequester, blocks storage.Blocks, collections storage.CollectionsReader, ) *JobProcessor { diff --git a/engine/access/ingestion2/collections/missing_collection_queue.go b/engine/access/collection_sync/collections/missing_collection_queue.go similarity index 98% rename from engine/access/ingestion2/collections/missing_collection_queue.go rename to engine/access/collection_sync/collections/missing_collection_queue.go index 6c9b1035420..45530b102b7 100644 --- a/engine/access/ingestion2/collections/missing_collection_queue.go +++ b/engine/access/collection_sync/collections/missing_collection_queue.go @@ -4,7 +4,7 @@ import ( "fmt" "sync" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" ) @@ -44,7 +44,7 @@ type MissingCollectionQueue struct { collectionToHeight map[flow.Identifier]uint64 } -var _ ingestion2.MissingCollectionQueue = (*MissingCollectionQueue)(nil) +var _ collection_sync.MissingCollectionQueue = (*MissingCollectionQueue)(nil) // NewMissingCollectionQueue creates a new MissingCollectionQueue. // diff --git a/engine/access/ingestion2/collections/requester.go b/engine/access/collection_sync/collections/requester.go similarity index 92% rename from engine/access/ingestion2/collections/requester.go rename to engine/access/collection_sync/collections/requester.go index b7b48012f9b..0e873e48eb5 100644 --- a/engine/access/ingestion2/collections/requester.go +++ b/engine/access/collection_sync/collections/requester.go @@ -4,7 +4,7 @@ import ( "errors" "fmt" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" @@ -12,10 +12,10 @@ import ( "github.com/onflow/flow-go/storage" ) -var _ ingestion2.CollectionRequester = (*CollectionRequester)(nil) +var _ collection_sync.CollectionRequester = (*CollectionRequester)(nil) // CollectionRequester requests collections from collection nodes on the network. -// It implements the ingestion2.CollectionRequester interface. +// It implements the collection_sync.CollectionRequester interface. type CollectionRequester struct { requester module.Requester state protocol.State @@ -76,4 +76,3 @@ func (cr *CollectionRequester) RequestCollections(ids []flow.Identifier) error { return nil } - diff --git a/engine/access/ingestion2/collections/syncer.go b/engine/access/collection_sync/collections/syncer.go similarity index 94% rename from engine/access/ingestion2/collections/syncer.go rename to engine/access/collection_sync/collections/syncer.go index a32350ca3fb..c56a32b35d7 100644 --- a/engine/access/ingestion2/collections/syncer.go +++ b/engine/access/collection_sync/collections/syncer.go @@ -6,7 +6,7 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" @@ -15,7 +15,7 @@ import ( "github.com/onflow/flow-go/storage" ) -var _ ingestion2.Syncer = (*Syncer)(nil) +var _ collection_sync.Syncer = (*Syncer)(nil) // Syncer is a component that consumes finalized block jobs and processes them // to index collections. It uses a job consumer with windowed throttling to prevent node overload. @@ -23,7 +23,7 @@ type Syncer struct { component.Component consumer *jobqueue.ComponentConsumer - jobProcessor ingestion2.JobProcessor + jobProcessor collection_sync.JobProcessor workSignal engine.Notifier } @@ -41,7 +41,7 @@ type Syncer struct { // No error returns are expected during normal operation. func NewSyncer( log zerolog.Logger, - jobProcessor ingestion2.JobProcessor, + jobProcessor collection_sync.JobProcessor, progressInitializer storage.ConsumerProgressInitializer, state protocol.State, blocks storage.Blocks, diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go new file mode 100644 index 00000000000..6ee99165d3c --- /dev/null +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -0,0 +1,46 @@ +package factory + +import ( + "fmt" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/collections" + "github.com/onflow/flow-go/engine/access/subscription/tracker" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage" +) + +// CreateExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. +// +// Parameters: +// - cache: Execution data cache for retrieving execution data by height +// - executionDataTracker: Tracker for execution data that provides the highest available height +// - processedHeight: Consumer progress for tracking processed heights +// - indexer: Block collection indexer for indexing collections +// +// Returns: +// - *ExecutionDataProcessor: A new ExecutionDataProcessor instance +// - error: An error if the processor could not be created +// +// No errors are expected during normal operation. +func CreateExecutionDataProcessor( + cache execution_data.ExecutionDataCache, + executionDataTracker tracker.ExecutionDataTracker, + processedHeight storage.ConsumerProgress, + indexer collection_sync.BlockCollectionIndexer, +) (*collections.ExecutionDataProcessor, error) { + // Create EDI height provider + ediHeightProvider := collections.NewEDIHeightProvider(cache, executionDataTracker) + + // Convert ConsumerProgress to PersistentStrictMonotonicCounter + processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) + if err != nil { + return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) + } + + // Create the execution data processor + processor := collections.NewExecutionDataProcessor(ediHeightProvider, indexer, processedHeightCounter) + + return processor, nil +} diff --git a/engine/access/ingestion2/collections/syncer_factory.go b/engine/access/collection_sync/factory/syncer.go similarity index 84% rename from engine/access/ingestion2/collections/syncer_factory.go rename to engine/access/collection_sync/factory/syncer.go index 68cb9924d2d..939897079c3 100644 --- a/engine/access/ingestion2/collections/syncer_factory.go +++ b/engine/access/collection_sync/factory/syncer.go @@ -1,4 +1,4 @@ -package collections +package factory import ( "fmt" @@ -6,7 +6,8 @@ import ( "github.com/jordanschalm/lockctx" "github.com/rs/zerolog" - "github.com/onflow/flow-go/engine/access/ingestion2" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/collections" "github.com/onflow/flow-go/engine/common/requester" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" @@ -28,16 +29,16 @@ type CreateSyncerConfig struct { // CreateSyncerResult holds the results of CreateSyncer. type CreateSyncerResult struct { - Syncer *Syncer - JobProcessor ingestion2.JobProcessor + Syncer *collections.Syncer + JobProcessor collection_sync.JobProcessor RequestEng *requester.Engine } // CreateSyncer creates a new Syncer component with all its dependencies. // This function is in the collections package to avoid import cycles: -// - collections package already imports ingestion2 (for interfaces) +// - collections package already imports collection_sync (for interfaces) // - CreateSyncer needs to create concrete types from collections package -// - Placing it in ingestion2 would create: ingestion2 -> collections -> ingestion2 (cycle) +// - Placing it in collection_sync would create: collection_sync -> collections -> collection_sync (cycle) // // Parameters: // - log: Logger for the component @@ -62,7 +63,7 @@ func CreateSyncer( state protocol.State, me module.Local, blocks storage.Blocks, - collections storage.Collections, + collStore storage.Collections, guarantees storage.Guarantees, db storage.DB, lockManager lockctx.Manager, @@ -86,30 +87,30 @@ func CreateSyncer( } // Create MissingCollectionQueue - mcq := NewMissingCollectionQueue() + mcq := collections.NewMissingCollectionQueue() // Create BlockCollectionIndexer - indexer := NewBlockCollectionIndexer( + indexer := collections.NewBlockCollectionIndexer( collectionExecutedMetric, lockManager, db, - collections, + collStore, ) // Create CollectionRequester - collectionRequester := NewCollectionRequester( + collectionRequester := collections.NewCollectionRequester( requestEng, state, guarantees, ) // Create JobProcessor - jobProcessor := NewJobProcessor( + jobProcessor := collections.NewJobProcessor( mcq, indexer, collectionRequester, blocks, - collections, + collStore, ) // Register handler for received collections @@ -128,7 +129,7 @@ func CreateSyncer( }) // Create Syncer - syncer, err := NewSyncer( + syncer, err := collections.NewSyncer( log, jobProcessor, processedFinalizedBlockHeight, diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go new file mode 100644 index 00000000000..e4a7348b8c5 --- /dev/null +++ b/engine/access/finalized_indexer/processor.go @@ -0,0 +1,138 @@ +package finalized_indexer + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// FinalizedBlockProcessor handles processing of finalized blocks, +// including indexing and syncing of related collections and execution results. +// +// FinalizedBlockProcessor processes finalized blocks sequentially using a simple loop +// that iterates from the last processed height to the latest finalized block height. +// When notified of a new finalized block, it processes all blocks up to the current +// finalized head height. +type FinalizedBlockProcessor struct { + log zerolog.Logger + + state protocol.State + blocks storage.Blocks + processedHeight *counters.PersistentStrictMonotonicCounter + + blockFinalizedNotifier chan struct{} + collectionExecutedMetric module.CollectionExecutedMetric +} + +// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor. +// +// No errors are expected during normal operations. +func NewFinalizedBlockProcessor( + log zerolog.Logger, + state protocol.State, + blocks storage.Blocks, + finalizedProcessedHeight storage.ConsumerProgressInitializer, + collectionExecutedMetric module.CollectionExecutedMetric, +) (*FinalizedBlockProcessor, error) { + finalizedBlock, err := state.Final().Head() + if err != nil { + return nil, fmt.Errorf("could not get finalized block header: %w", err) + } + + processedHeightProgress, err := finalizedProcessedHeight.Initialize(finalizedBlock.Height) + if err != nil { + return nil, fmt.Errorf("could not initialize processed height: %w", err) + } + + processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeightProgress) + if err != nil { + return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) + } + + processor := &FinalizedBlockProcessor{ + log: log, + state: state, + blocks: blocks, + processedHeight: processedHeightCounter, + blockFinalizedNotifier: make(chan struct{}, 1), + collectionExecutedMetric: collectionExecutedMetric, + } + + return processor, nil +} + +// OnBlockFinalized notifies the processor that a new finalized block is available for processing. +func (p *FinalizedBlockProcessor) OnBlockFinalized() { + select { + case p.blockFinalizedNotifier <- struct{}{}: + default: + // if the channel is full, no need to block, just return. + // once the worker loop processes the buffered signal, it will + // process the next height all the way to the highest finalized height. + } +} + +// StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. +// It uses a single-threaded loop to process each finalized block height sequentially. +func (p *FinalizedBlockProcessor) StartWorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + for { + select { + case <-ctx.Done(): + return + case <-p.blockFinalizedNotifier: + finalizedHead, err := p.state.Final().Head() + if err != nil { + ctx.Throw(fmt.Errorf("failed to get finalized head: %w", err)) + return + } + + highestFinalizedHeight := finalizedHead.Height + lowestMissing := p.processedHeight.Value() + 1 + + for height := lowestMissing; height <= highestFinalizedHeight; height++ { + block, err := p.blocks.ByHeight(height) + if err != nil { + ctx.Throw(fmt.Errorf("failed to get block at height %d: %w", height, err)) + return + } + + err = p.indexForFinalizedBlock(block) + if err != nil { + ctx.Throw(fmt.Errorf("failed to index finalized block at height %d: %w", height, err)) + return + } + + // Update processed height after successful indexing + err = p.processedHeight.Set(height) + if err != nil { + ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) + return + } + } + } + } +} + +// indexForFinalizedBlock indexes the given finalized block’s collection guarantees +// +// No errors are expected during normal operations. +func (p *FinalizedBlockProcessor) indexForFinalizedBlock(block *flow.Block) error { + err := p.blocks.IndexBlockContainingCollectionGuarantees(block.ID(), flow.GetIDs(block.Payload.Guarantees)) + if err != nil { + return fmt.Errorf("could not index block for collections: %w", err) + } + + p.collectionExecutedMetric.BlockFinalized(block) + + return nil +} diff --git a/engine/access/ingestion2/engine.go b/engine/access/ingest_receipt/engine.go similarity index 84% rename from engine/access/ingestion2/engine.go rename to engine/access/ingest_receipt/engine.go index b0ff39f52f0..c1508457b04 100644 --- a/engine/access/ingestion2/engine.go +++ b/engine/access/ingest_receipt/engine.go @@ -1,4 +1,4 @@ -// Package ingestion2 implements a modular ingestion engine responsible for +// Package finalized_indexer implements a modular ingestion engine responsible for // orchestrating the processing of finalized blockchain data and receiving // execution receipts from the network. // @@ -6,7 +6,7 @@ // - Receiving and persisting execution receipts from the network. // - Subscribing to finalized block events. // - Synchronizing collections associated with finalized blocks. -package ingestion2 +package ingest_receipt import ( "context" @@ -14,7 +14,6 @@ import ( "github.com/rs/zerolog" - "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/flow" @@ -34,9 +33,6 @@ type Engine struct { log zerolog.Logger - finalizedBlockProcessor *FinalizedBlockProcessor - collectionSyncer Syncer - messageHandler *engine.MessageHandler executionReceiptsQueue *engine.FifoMessageStore receipts storage.ExecutionReceipts @@ -48,8 +44,6 @@ var _ network.MessageProcessor = (*Engine)(nil) func New( log zerolog.Logger, net network.EngineRegistry, - finalizedBlockProcessor *FinalizedBlockProcessor, - collectionSyncer Syncer, receipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, ) (*Engine, error) { @@ -71,9 +65,7 @@ func New( ) e := &Engine{ - log: log.With().Str("engine", "ingestion2").Logger(), - finalizedBlockProcessor: finalizedBlockProcessor, - collectionSyncer: collectionSyncer, + log: log.With().Str("engine", "ingest_receipt").Logger(), messageHandler: messageHandler, executionReceiptsQueue: executionReceiptsQueue, receipts: receipts, @@ -83,8 +75,7 @@ func New( // register our workers which are basically consumers of different kinds of data. // engine notifies workers when new data is available so that they can start processing them. builder := component.NewComponentManagerBuilder(). - AddWorker(e.messageHandlerLoop). - AddWorker(e.finalizedBlockProcessor.StartWorkerLoop) + AddWorker(e.messageHandlerLoop) e.ComponentManager = builder.Build() // engine gets execution receipts from channels.ReceiveReceipts channel @@ -173,10 +164,3 @@ func (e *Engine) persistExecutionReceipt(receipt *flow.ExecutionReceipt) error { e.collectionExecutedMetric.ExecutionReceiptReceived(receipt) return nil } - -// OnFinalizedBlock is called by the follower engine after a block has been finalized and the state has been updated. -// Receives block finalized events from the finalization distributor and forwards them to the consumer. -func (e *Engine) OnFinalizedBlock(block *model.Block) { - e.finalizedBlockProcessor.Notify() - e.collectionSyncer.OnFinalizedBlock() -} diff --git a/engine/access/ingestion2/README.md b/engine/access/ingestion2/README.md deleted file mode 100644 index f121d8554c9..00000000000 --- a/engine/access/ingestion2/README.md +++ /dev/null @@ -1,146 +0,0 @@ -Asynchronous Collection Indexing — Design - -1) Purpose & Outcomes - -Goal. Index block collections reliably without overloading the node, even when finalized blocks arrive faster than we can index. - -Outcomes. - -Workers (job processors) stay focused on a small, sliding window of heights just above the latest indexed height. - -Collection retrieval and indexing are decoupled from finalization and from each other. - -Execution Data Indexer (EDI) provides the preferred source for collections; collection requests are only sent when EDI falls behind a configurable threshold. - -2) High-Level Flow - -Finalization happens → a lazy signal wakes the Job Consumer. - -Job Consumer consults: (a) Progress Tracker → highest indexed height; (b) Jobs module → latest safe (head) height. It computes a bounded work window [indexed+1 .. min(indexed+N, head)]. - -For each height in the window, the consumer spins up (or reuses) a Job Processor. - -The Job Processor checks if the block is already indexed. If yes → finish immediately. If not: - -It enqueues the block’s missing collection IDs into MissingCollectionQueue (MCQ) with a completion callback. - -It checks how far EDI lags behind the block height. If the lag exceeds a configured threshold, it triggers CollectionRequester to fetch collections; otherwise, it waits for EDI to deliver them. - -As collections arrive (from requester or EDI), the Job Processor forwards them to MCQ. When MCQ detects a block is now complete, the processor passes the collections to BlockCollectionIndexer to store+index, then calls MCQ to mark the job done. - -Jobs may complete out of order; the progress tracker advances once any gaps below are closed. - -3) Core Interfaces - -BlockCollectionIndexer - -Stores and indexes collections for a given block height; provides the latest indexed height for windowing and fast no-op checks. - -type BlockCollectionIndexer interface { - LatestIndexedHeight() uint64 - - // If blockHeight <= LatestIndexedHeight(), return quickly. - // Otherwise: lock, re-check, then persist+index collections. - // Double-check pattern minimizes lock contention. - OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error -} - -MissingCollectionQueue (MCQ) - -In-memory coordinator for jobs and callbacks. MCQ does not index; it only tracks missing collections per height and announces when a height becomes complete. - -type MissingCollectionQueue interface { - EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error - OnIndexedForBlock(blockHeight uint64) - // On receipt of a collection, MCQ updates internal state and, if a block - // just became complete, returns: (collections, height, true). - // Otherwise, returns (nil, 0, false). - OnReceivedCollection(collectionID flow.Identifier) ([]*flow.Collection, uint64, bool) -} - -CollectionRequester - -Abstracts the engine that requests collections by ID (e.g., from collection nodes). - -type CollectionRequester interface { - RequestCollections(ids []flow.Identifier) error -} - -JobProcessor - -Owns the state of ongoing jobs (delegated to MCQ) and orchestrates request → receive → index → complete. - -type JobProcessor interface { - ProcessJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) error - OnReceivedCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error -} - -4) Job Consumer (Windowed Throttling) - -Why: Prevent node overload when finalized heights advance rapidly. - -How: - -Reads latestIndexed = BlockCollectionIndexer.LatestIndexedHeight(). - -Reads head = Jobs.Head() (latest height safe to work on). - -Defines window size K. Range: [latestIndexed+1 .. min(latestIndexed+K, head)]. - -Assigns one JobProcessor per height in this range. - -Lazy notification: Finalization pushes a single, coalescing signal to workSignal (buffer size 1). The consumer wakes, recomputes the range, and may ignore new jobs if already at capacity. - -5) Job Lifecycle - -Spawn/Assign. Consumer gives (height, job, doneCb) to a JobProcessor. - -Already Indexed? Processor queries storage. If yes → doneCb() and return. - -Track Missing. MCQ.EnqueueMissingCollections(height, collectionIDs, doneCb). - -Check EDI Lag. Compare height with ediIndexedHeight(). If lag ≤ threshold, wait for EDI; if lag > threshold, trigger CollectionRequester.RequestCollections(ids). - -Receive Collections. - -Processor calls MCQ.OnReceivedCollection(id). - -If complete, processor calls BlockCollectionIndexer.OnReceivedCollectionsForBlock(height, cols) and then MCQ.OnIndexedForBlock(height). - -Progress Advancement. Out-of-order completion allowed; progress advances only when lower gaps close. - -Crash/Restart. On restart, re-created jobs short-circuit if already indexed. - -6) Execution Data Indexer (EDI) Integration - -EDI serves as the primary source of collections. The system dynamically decides whether to fetch collections based on EDI’s progress. - -Lag-based hybrid logic: - -Track ediIndexedHeight() — the latest height for which EDI has collections. - -Define a lag threshold, EDILagThreshold, in number of blocks. - -For block height h: - -If (h - ediIndexedHeight()) <= EDILagThreshold: rely on EDI; no fetching. - -If (h - ediIndexedHeight()) > EDILagThreshold: trigger the collection fetcher to request collections from nodes. - -Behavior summary: - -If EDI is up to date or within threshold → no fetches. - -If EDI is behind beyond threshold → start fetching. - -Setting a very large threshold effectively mimics the previous EDI-only mode. - -Why EDI goes through JobProcessor: To keep job state consistent—update MCQ, suppress redundant fetches, and advance the queue. - -Contention handling: - -If EDI lags → fetching fills the gap. - -If EDI leads → consumer advances; requester stops fetching. - -If close → minimal contention; indexing is faster than receipt. diff --git a/engine/access/ingestion2/finalized_block_processor.go b/engine/access/ingestion2/finalized_block_processor.go deleted file mode 100644 index 8c89d3e6c22..00000000000 --- a/engine/access/ingestion2/finalized_block_processor.go +++ /dev/null @@ -1,161 +0,0 @@ -package ingestion2 - -import ( - "fmt" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/jobqueue" - "github.com/onflow/flow-go/module/util" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -const ( - // finalizedBlockProcessorWorkerCount defines the number of workers that - // concurrently process finalized blocks in the job queue. - // MUST be 1 to ensure sequential processing - finalizedBlockProcessorWorkerCount = 1 - - // searchAhead is a number of blocks that should be processed ahead by jobqueue - // MUST be 1 to ensure sequential processing - searchAhead = 1 -) - -// FinalizedBlockProcessor handles processing of finalized blocks, -// including indexing and syncing of related collections and execution results. -// -// FinalizedBlockProcessor is designed to handle the ingestion of finalized Flow blocks -// in a scalable and decoupled manner. It uses a jobqueue.ComponentConsumer to consume -// and process finalized block jobs asynchronously. This design enables the processor -// to handle high-throughput block finalization events without blocking other parts -// of the system. -// -// The processor relies on a notifier (engine.Notifier) to signal when a new finalized -// block is available, which triggers the job consumer to process it. The actual -// processing involves indexing block-to-collection and block-to-execution-result -// mappings, as well as requesting the associated collections. -type FinalizedBlockProcessor struct { - log zerolog.Logger - - consumer *jobqueue.ComponentConsumer - consumerNotifier engine.Notifier - blocks storage.Blocks - - executionResults storage.ExecutionResults - - collectionExecutedMetric module.CollectionExecutedMetric -} - -// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor, -// setting up job consumer infrastructure to handle finalized block processing. -// -// No errors are expected during normal operations. -func NewFinalizedBlockProcessor( - log zerolog.Logger, - state protocol.State, - blocks storage.Blocks, - executionResults storage.ExecutionResults, - finalizedProcessedHeight storage.ConsumerProgressInitializer, - collectionExecutedMetric module.CollectionExecutedMetric, -) (*FinalizedBlockProcessor, error) { - reader := jobqueue.NewFinalizedBlockReader(state, blocks) - finalizedBlock, err := state.Final().Head() - if err != nil { - return nil, fmt.Errorf("could not get finalized block header: %w", err) - } - - consumerNotifier := engine.NewNotifier() - processor := &FinalizedBlockProcessor{ - log: log, - blocks: blocks, - executionResults: executionResults, - consumerNotifier: consumerNotifier, - collectionExecutedMetric: collectionExecutedMetric, - } - - processor.consumer, err = jobqueue.NewComponentConsumer( - log.With().Str("module", "ingestion_block_consumer").Logger(), - consumerNotifier.Channel(), - finalizedProcessedHeight, - reader, - finalizedBlock.Height, - processor.processFinalizedBlockJobCallback, - finalizedBlockProcessorWorkerCount, - searchAhead, - ) - if err != nil { - return nil, fmt.Errorf("error creating finalized block jobqueue: %w", err) - } - - return processor, nil -} - -// Notify notifies the processor that a new finalized block is available for processing. -func (p *FinalizedBlockProcessor) Notify() { - p.consumerNotifier.Notify() -} - -// StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. -func (p *FinalizedBlockProcessor) StartWorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - p.consumer.Start(ctx) - - err := util.WaitClosed(ctx, p.consumer.Ready()) - if err == nil { - ready() - } - - <-p.consumer.Done() -} - -// processFinalizedBlockJobCallback is a jobqueue callback that processes a finalized block job. -func (p *FinalizedBlockProcessor) processFinalizedBlockJobCallback( - ctx irrecoverable.SignalerContext, - job module.Job, - done func(), -) { - block, err := jobqueue.JobToBlock(job) - if err != nil { - ctx.Throw(fmt.Errorf("failed to convert job to block: %w", err)) - return - } - - err = p.indexFinalizedBlock(block) - if err != nil { - p.log.Error().Err(err). - Str("job_id", string(job.ID())). - Msg("unexpected error during finalized block processing job") - ctx.Throw(fmt.Errorf("failed to index finalized block: %w", err)) - return - } - - done() -} - -// indexFinalizedBlock indexes the given finalized block’s collection guarantees and execution results, -// and requests related collections from the syncer. -// -// No errors are expected during normal operations. -func (p *FinalizedBlockProcessor) indexFinalizedBlock(block *flow.Block) error { - err := p.blocks.IndexBlockContainingCollectionGuarantees(block.ID(), flow.GetIDs(block.Payload.Guarantees)) - if err != nil { - return fmt.Errorf("could not index block for collections: %w", err) - } - - // loop through seals and index ID -> result ID - for _, seal := range block.Payload.Seals { - err := p.executionResults.Index(seal.BlockID, seal.ResultID) - if err != nil { - return fmt.Errorf("could not index block for execution result: %w", err) - } - } - - p.collectionExecutedMetric.BlockFinalized(block) - - return nil -} diff --git a/module/jobqueue.go b/module/jobqueue.go index 3aae850dac0..0a27480881e 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -8,21 +8,28 @@ const ( ConsumeProgressVerificationBlockHeight = "ConsumeProgressVerificationBlockHeight" ConsumeProgressVerificationChunkIndex = "ConsumeProgressVerificationChunkIndex" - ConsumeProgressExecutionDataRequesterBlockHeight = "ConsumeProgressExecutionDataRequesterBlockHeight" + // for each sealed block height, request and store the execution data + ConsumeProgressExecutionDataRequesterBlockHeight = "ConsumeProgressExecutionDataRequesterBlockHeight" + + // for OnExecutionDataFetched notification distributor to distribute the events ConsumeProgressExecutionDataRequesterNotification = "ConsumeProgressExecutionDataRequesterNotification" ConsumeProgressExecutionDataIndexerBlockHeight = "ConsumeProgressExecutionDataIndexerBlockHeight" + // for AN ingestion engine to index guarantee by block id ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" - // the highest block height that has indexed collections from fetching missing collections from LN + // (new)the highest block height that has indexed collections from fetching missing collections from LN ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight" - // the highest block height that has indexed collections from syncing execution data from EN + // (new)the highest block height that has indexed collections from syncing execution data from EN ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight" + // on finalized block, fetch tx error message from EN ConsumeProgressEngineTxErrorMessagesBlockHeight = "ConsumeProgressEngineTxErrorMessagesBlockHeight" - ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" + + // fetch collections + ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" ) // JobID is a unique ID of the job. diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index 705ca437d3d..71a311c2110 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-core-contracts/lib/go/templates" - "github.com/onflow/flow-go/engine/access/ingestion2" "github.com/onflow/flow-go/fvm/blueprints" "github.com/onflow/flow-go/fvm/storage/derived" "github.com/onflow/flow-go/fvm/systemcontracts" @@ -31,7 +30,6 @@ type IndexerCore struct { fvmEnv templates.Environment metrics module.ExecutionStateIndexerMetrics collectionExecutedMetric module.CollectionExecutedMetric - collectionIndexer ingestion2.JobProcessor registers storage.RegisterIndex headers storage.Headers @@ -63,7 +61,6 @@ func New( scheduledTransactions storage.ScheduledTransactions, chainID flow.ChainID, derivedChainData *derived.DerivedChainData, - collectionIndexer ingestion2.JobProcessor, collectionExecutedMetric module.CollectionExecutedMetric, lockManager lockctx.Manager, ) *IndexerCore { @@ -93,7 +90,6 @@ func New( serviceAddress: chainID.Chain().ServiceAddress(), derivedChainData: derivedChainData, - collectionIndexer: collectionIndexer, collectionExecutedMetric: collectionExecutedMetric, lockManager: lockManager, } @@ -219,32 +215,6 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti return nil }) - g.Go(func() error { - start := time.Now() - - // Note: the access ingestion engine also indexes collections, starting when the block is - // finalized. This process can fall behind due to the node being offline, resource issues - // or network congestion. This indexer ensures that collections are never farther behind - // than the latest indexed block. Calling the collection handler with a collection that - // has already been indexed is a noop. - - // index all standard (non-system) collections - standardCollections := data.StandardCollections() - if len(standardCollections) > 0 { - err := c.collectionIndexer.OnReceivedCollectionsForBlock(header.Height, standardCollections) - if err != nil { - return fmt.Errorf("could not index collections: %w", err) - } - } - - lg.Debug(). - Int("collection_count", len(standardCollections)). - Dur("duration_ms", time.Since(start)). - Msg("indexed collections") - - return nil - }) - g.Go(func() error { start := time.Now() From c723fc9d7f235446e3d64df99cdcc6756cf63eac Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 12 Nov 2025 20:46:02 -0800 Subject: [PATCH 013/126] refactor with fetcher --- .../node_builder/access_node_builder.go | 204 ++++++------------ .../node_builder/collection_ingestion.go | 135 ------------ .../access/collection_sync/collection_sync.go | 4 +- .../execution_data_processor_factory.go | 45 ---- .../processor.go} | 28 +-- .../provider.go} | 20 +- .../factory/execution_data_processor.go | 10 +- .../access/collection_sync/factory/syncer.go | 41 +--- .../block_collection_indexer.go | 8 +- .../{collections => fetcher}/job_processor.go | 2 +- .../missing_collection_queue.go | 2 +- .../{collections => fetcher}/requester.go | 2 +- .../{collections => fetcher}/syncer.go | 4 +- engine/access/finalized_indexer/processor.go | 21 +- 14 files changed, 141 insertions(+), 385 deletions(-) delete mode 100644 cmd/access/node_builder/collection_ingestion.go delete mode 100644 engine/access/collection_sync/collections/execution_data_processor_factory.go rename engine/access/collection_sync/{collections/execution_data_processor.go => execution_data_index/processor.go} (72%) rename engine/access/collection_sync/{collections/edi_height_provider.go => execution_data_index/provider.go} (63%) rename engine/access/collection_sync/{collections => fetcher}/block_collection_indexer.go (90%) rename engine/access/collection_sync/{collections => fetcher}/job_processor.go (99%) rename engine/access/collection_sync/{collections => fetcher}/missing_collection_queue.go (99%) rename engine/access/collection_sync/{collections => fetcher}/requester.go (99%) rename engine/access/collection_sync/{collections => fetcher}/syncer.go (98%) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 665ff1bf336..5d3e0ecd9fb 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -39,12 +39,14 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/verification" recovery "github.com/onflow/flow-go/consensus/recovery/protocol" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/factory" + collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" + "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" + "github.com/onflow/flow-go/engine/access/finalized_indexer" "github.com/onflow/flow-go/engine/access/index" - "github.com/onflow/flow-go/engine/access/ingestion/collections" + "github.com/onflow/flow-go/engine/access/ingest_receipt" "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" - "github.com/onflow/flow-go/engine/access/ingestion2" - ingestion2collections "github.com/onflow/flow-go/engine/access/ingestion2/collections" - ingestion2factory "github.com/onflow/flow-go/engine/access/ingestion2/factory" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" "github.com/onflow/flow-go/engine/access/rest/router" @@ -334,10 +336,7 @@ type FlowAccessNodeBuilder struct { ExecutionDataCache *execdatacache.ExecutionDataCache ExecutionIndexer *indexer.Indexer ExecutionIndexerCore *indexer.IndexerCore - CollectionIndexer *collections.Indexer - CollectionSyncer *collections.Syncer - JobProcessor ingestion2.JobProcessor - CollectionSyncer2 ingestion2.Syncer + blockCollectionIndexer collection_sync.BlockCollectionIndexer ScriptExecutor *backend.ScriptExecutor RegistersAsyncStore *execution.RegistersAsyncStore Reporter *index.Reporter @@ -365,11 +364,9 @@ type FlowAccessNodeBuilder struct { SyncEngineParticipantsProviderFactory func() module.IdentifierProvider // engines - IngestEng *ingestion2.Engine - RequestEng *requester.Engine - FollowerEng *followereng.ComplianceEngine - SyncEng *synceng.Engine - StateStreamEng *statestreambackend.Engine + CollectionRequesterEngine *requester.Engine + FollowerEng *followereng.ComplianceEngine + StateStreamEng *statestreambackend.Engine // grpc servers secureGrpcServer *grpcserver.GrpcServer @@ -380,7 +377,6 @@ type FlowAccessNodeBuilder struct { nodeBackend *backend.Backend ExecNodeIdentitiesProvider *commonrpc.ExecutionNodeIdentitiesProvider - TxResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore txResultErrorMessageProvider error_messages.Provider } @@ -544,10 +540,9 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) } - builder.SyncEng = sync builder.FollowerDistributor.AddFinalizationConsumer(sync) - return builder.SyncEng, nil + return sync, nil }) return builder @@ -1730,14 +1725,11 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { var processedFinalizedBlockHeight storage.ConsumerProgressInitializer var fetchAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer var syncAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer - var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer if builder.executionDataSyncEnabled { builder.BuildExecutionSyncComponents() } - ingestionDependable := module.NewProxiedReadyDoneAware() - builder.IndexerDependencies.Add(ingestionDependable) versionControlDependable := module.NewProxiedReadyDoneAware() builder.IndexerDependencies.Add(versionControlDependable) stopControlDependable := module.NewProxiedReadyDoneAware() @@ -1960,34 +1952,13 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { syncAndIndexedCollectionsBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight) return nil }). - Module("collection syncer and job processor", func(node *cmd.NodeConfig) error { - // Create syncer and job processor - syncerResult, err := ingestion2factory.CreateSyncer( - node.Logger, - node.EngineRegistry, - node.State, - node.Me, - node.Storage.Blocks, - notNil(builder.collections), - node.Storage.Guarantees, - builder.ProtocolDB, - node.StorageLockMgr, - fetchAndIndexedCollectionsBlockHeight, + Module("block collection indexer", func(node *cmd.NodeConfig) error { + builder.blockCollectionIndexer = fetcher.NewBlockCollectionIndexer( notNil(builder.collectionExecutedMetric), - ingestion2factory.CreateSyncerConfig{ - MaxProcessing: 10, // TODO: make configurable - MaxSearchAhead: 0, // TODO: make configurable - }, + node.StorageLockMgr, + builder.ProtocolDB, + notNil(builder.collections), ) - if err != nil { - return fmt.Errorf("could not create collection syncer: %w", err) - } - - // Store the results for use in other components - builder.JobProcessor = syncerResult.JobProcessor - builder.CollectionSyncer2 = syncerResult.Syncer - builder.RequestEng = syncerResult.RequestEng - return nil }). Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { @@ -2235,54 +2206,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.RpcEng.OnFinalizedBlock) return builder.RpcEng, nil - }). - Component("requester engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // RequestEng should already be created in the "collection syncer and job processor" module - if builder.RequestEng == nil { - return nil, fmt.Errorf("RequestEng must be created before requester engine component") - } - - collectionIndexer, err := collections.NewIndexer( - node.Logger, - builder.ProtocolDB, - notNil(builder.collectionExecutedMetric), - node.State, - node.Storage.Blocks, - notNil(builder.collections), - lastFullBlockHeight, - node.StorageLockMgr, - ) - if err != nil { - return nil, fmt.Errorf("could not create collection indexer: %w", err) - } - builder.CollectionIndexer = collectionIndexer - - // the collection syncer has support for indexing collections from execution data if the - // syncer falls behind. This is only needed if the execution state indexing is disabled, - // since it will also index collections. - var executionDataSyncer *collections.ExecutionDataSyncer - if builder.executionDataSyncEnabled && !builder.executionDataIndexingEnabled { - executionDataSyncer = collections.NewExecutionDataSyncer( - node.Logger, - notNil(builder.ExecutionDataCache), - collectionIndexer, - ) - } - - collectionSyncer := collections.NewSyncer( - node.Logger, - builder.RequestEng, - node.State, - notNil(builder.collections), - lastFullBlockHeight, - collectionIndexer, - executionDataSyncer, - ) - builder.CollectionSyncer = collectionSyncer - - builder.RequestEng.WithHandle(collectionSyncer.OnCollectionDownloaded) - - return builder.RequestEng, nil }) if builder.executionDataSyncEnabled { builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -2318,22 +2241,12 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil, fmt.Errorf("could not initialize processed height: %w", err) } - // Get BlockCollectionIndexer from syncer result - // The indexer is created in the CreateSyncer call, we need to extract it - // For now, we'll create a new one since we need it for the processor - blockCollectionIndexer := ingestion2collections.NewBlockCollectionIndexer( - notNil(builder.collectionExecutedMetric), - node.StorageLockMgr, - builder.ProtocolDB, - notNil(builder.collections), - ) - // Create execution data processor - executionDataProcessor, err := ingestion2factory.CreateExecutionDataProcessor( + executionDataProcessor, err := collection_syncfactory.CreateExecutionDataProcessor( notNil(builder.ExecutionDataCache), executionDataTracker, processedHeight, - blockCollectionIndexer, + notNil(builder.blockCollectionIndexer), ) if err != nil { return nil, fmt.Errorf("could not create execution data processor: %w", err) @@ -2350,28 +2263,12 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return executionDataProcessor, nil }) } - builder.Component("ingestion engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.storeTxResultErrorMessages { - builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( - node.Logger, - notNil(builder.txResultErrorMessageProvider), - builder.transactionResultErrorMessages, - notNil(builder.ExecNodeIdentitiesProvider), - node.StorageLockMgr, - ) - } - // CollectionSyncer2 should already be created in the "collection syncer and job processor" module - if builder.CollectionSyncer2 == nil { - return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") - } - - // Create FinalizedBlockProcessor (still uses old syncer temporarily) - finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( + builder.Component("finalized block indexer", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + finalizedBlockProcessor, err := finalized_indexer.NewFinalizedBlockProcessor( node.Logger, node.State, node.Storage.Blocks, - node.Storage.Results, processedFinalizedBlockHeight, notNil(builder.collectionExecutedMetric), ) @@ -2379,34 +2276,73 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil, fmt.Errorf("could not create finalized block processor: %w", err) } - // Create ingestion2 engine - ingestEng, err := ingestion2.New( + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(finalizedBlockProcessor.OnBlockFinalized) + + return finalizedBlockProcessor, nil + }) + + builder.Component("ingest receipt", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return ingest_receipt.New( node.Logger, node.EngineRegistry, - finalizedBlockProcessor, - builder.CollectionSyncer2, node.Storage.Receipts, notNil(builder.collectionExecutedMetric), ) + }) + + builder.Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + // skip if execution data sync is enabled + // Create syncer and requesterEng + requesterEng, syncer, err := factory.CreateSyncer( + node.Logger, + node.EngineRegistry, + node.State, + node.Me, + node.Storage.Blocks, + notNil(builder.collections), + node.Storage.Guarantees, + builder.ProtocolDB, + notNil(builder.blockCollectionIndexer), + fetchAndIndexedCollectionsBlockHeight, + notNil(builder.collectionExecutedMetric), + collection_syncfactory.CreateSyncerConfig{ + MaxProcessing: 10, // TODO: make configurable + MaxSearchAhead: 0, // TODO: make configurable + }, + ) + if err != nil { - return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) + return nil, fmt.Errorf("could not create collection syncer: %w", err) } - builder.IngestEng = ingestEng + builder.CollectionRequesterEngine = requesterEng - ingestionDependable.Init(ingestEng) - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) + return syncer, nil + }) - return ingestEng, nil + builder.Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return builder.CollectionRequesterEngine, nil }) if builder.storeTxResultErrorMessages { + var txResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore + var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer builder. + Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { + txResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( + node.Logger, + notNil(builder.txResultErrorMessageProvider), + builder.transactionResultErrorMessages, + notNil(builder.ExecNodeIdentitiesProvider), + node.StorageLockMgr, + ) + return nil + }). AdminCommand("backfill-tx-error-messages", func(config *cmd.NodeConfig) commands.AdminCommand { return storageCommands.NewBackfillTxErrorMessagesCommand( builder.Logger, builder.State, - builder.TxResultErrorMessagesCore, + txResultErrorMessagesCore, ) }). Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { @@ -2431,7 +2367,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.State, node.Storage.Headers, processedTxErrorMessagesBlockHeight, - builder.TxResultErrorMessagesCore, + txResultErrorMessagesCore, ) if err != nil { return nil, err diff --git a/cmd/access/node_builder/collection_ingestion.go b/cmd/access/node_builder/collection_ingestion.go deleted file mode 100644 index 27f70694a5c..00000000000 --- a/cmd/access/node_builder/collection_ingestion.go +++ /dev/null @@ -1,135 +0,0 @@ -package node_builder - -import ( - "fmt" - - "github.com/onflow/flow-go/cmd" - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/access/ingestion2" - ingestion2collections "github.com/onflow/flow-go/engine/access/ingestion2/collections" - ingestion2factory "github.com/onflow/flow-go/engine/access/ingestion2/factory" - subscriptiontracker "github.com/onflow/flow-go/engine/access/subscription/tracker" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" - "github.com/onflow/flow-go/storage/store" -) - -// InitializeIngestionEngine creates and initializes the ingestion2 engine with collections.Syncer. -// This function should be called from within a Component function where node is available. -func InitializeIngestionEngine(builder *FlowAccessNodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // CollectionSyncer2 should already be created in the "collection syncer and job processor" module - if builder.CollectionSyncer2 == nil { - return nil, fmt.Errorf("CollectionSyncer2 must be created before ingestion engine") - } - - // Create processedFinalizedBlockHeight if not already created - processedFinalizedBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) - - // Create FinalizedBlockProcessor - finalizedBlockProcessor, err := ingestion2.NewFinalizedBlockProcessor( - node.Logger, - node.State, - node.Storage.Blocks, - node.Storage.Results, - processedFinalizedBlockHeight, - notNil(builder.collectionExecutedMetric), - ) - if err != nil { - return nil, fmt.Errorf("could not create finalized block processor: %w", err) - } - - // Create ingestion2 engine - ingestEng, err := ingestion2.New( - node.Logger, - node.EngineRegistry, - finalizedBlockProcessor, - builder.CollectionSyncer2, - node.Storage.Receipts, - notNil(builder.collectionExecutedMetric), - ) - if err != nil { - return nil, fmt.Errorf("could not create ingestion2 engine: %w", err) - } - - builder.IngestEng = ingestEng - - // Create and initialize ingestionDependable if IndexerDependencies exists - if builder.IndexerDependencies != nil { - ingestionDependable := module.NewProxiedReadyDoneAware() - builder.IndexerDependencies.Add(ingestionDependable) - ingestionDependable.Init(ingestEng) - } - - // Add OnFinalizedBlock consumer if FollowerDistributor exists - if builder.FollowerDistributor != nil { - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(ingestEng.OnFinalizedBlock) - } - - return ingestEng, nil -} - -// InitializeExecutionDataCollectionIndexer creates and initializes the execution data collection indexer -// with ExecutionDataProcessor. This function should be called from within a Component function where node is available. -func InitializeExecutionDataCollectionIndexer(builder *FlowAccessNodeBuilder, node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // ExecutionDataCache should already be created in BuildExecutionSyncComponents - if builder.ExecutionDataCache == nil { - return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") - } - - // Create execution data tracker for the processor - // This is similar to the one created in state stream engine but used for collection indexing - broadcaster := engine.NewBroadcaster() - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } - - useIndex := builder.executionDataIndexingEnabled - executionDataTracker := subscriptiontracker.NewExecutionDataTracker( - builder.Logger, - node.State, - builder.executionDataConfig.InitialBlockHeight, - node.Storage.Headers, - broadcaster, - highestAvailableHeight, - builder.EventsIndex, - useIndex, - ) - - // Initialize processed height - rootBlockHeight := node.State.Params().FinalizedRoot().Height - syncAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight) - processedHeight, err := syncAndIndexedCollectionsBlockHeight.Initialize(rootBlockHeight) - if err != nil { - return nil, fmt.Errorf("could not initialize processed height: %w", err) - } - - // Create BlockCollectionIndexer - blockCollectionIndexer := ingestion2collections.NewBlockCollectionIndexer( - notNil(builder.collectionExecutedMetric), - node.StorageLockMgr, - builder.ProtocolDB, - notNil(builder.collections), - ) - - // Create execution data processor - executionDataProcessor, err := ingestion2factory.CreateExecutionDataProcessor( - notNil(builder.ExecutionDataCache), - executionDataTracker, - processedHeight, - blockCollectionIndexer, - ) - if err != nil { - return nil, fmt.Errorf("could not create execution data processor: %w", err) - } - - // Setup requester to notify processor when new execution data is received - if builder.ExecutionDataDistributor != nil { - builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { - executionDataTracker.OnExecutionData(executionData) - executionDataProcessor.OnNewExectuionData() - }) - } - - return executionDataProcessor, nil -} diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 5ea215ec1cb..5972cb44680 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -49,9 +49,9 @@ type Syncer interface { Size() uint } -// EDIHeightProvider provides the latest height for which execution data indexer has collections. +// ExecutionDataProvider provides the latest height for which execution data indexer has collections. // This can be nil if execution data indexing is disabled. -type EDIHeightProvider interface { +type ExecutionDataProvider interface { HighestIndexedHeight() uint64 GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) } diff --git a/engine/access/collection_sync/collections/execution_data_processor_factory.go b/engine/access/collection_sync/collections/execution_data_processor_factory.go deleted file mode 100644 index ef76df69fcd..00000000000 --- a/engine/access/collection_sync/collections/execution_data_processor_factory.go +++ /dev/null @@ -1,45 +0,0 @@ -package collections - -import ( - "fmt" - - "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/subscription/tracker" - "github.com/onflow/flow-go/module/counters" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" - "github.com/onflow/flow-go/storage" -) - -// CreateExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. -// -// Parameters: -// - cache: Execution data cache for retrieving execution data by height -// - executionDataTracker: Tracker for execution data that provides the highest available height -// - processedHeight: Consumer progress for tracking processed heights -// - indexer: Block collection indexer for indexing collections -// -// Returns: -// - *ExecutionDataProcessor: A new ExecutionDataProcessor instance -// - error: An error if the processor could not be created -// -// No errors are expected during normal operation. -func CreateExecutionDataProcessor( - cache execution_data.ExecutionDataCache, - executionDataTracker tracker.ExecutionDataTracker, - processedHeight storage.ConsumerProgress, - indexer collection_sync.BlockCollectionIndexer, -) (*ExecutionDataProcessor, error) { - // Create EDI height provider - ediHeightProvider := NewEDIHeightProvider(cache, executionDataTracker) - - // Convert ConsumerProgress to PersistentStrictMonotonicCounter - processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) - if err != nil { - return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) - } - - // Create the execution data processor - processor := NewExecutionDataProcessor(ediHeightProvider, indexer, processedHeightCounter) - - return processor, nil -} diff --git a/engine/access/collection_sync/collections/execution_data_processor.go b/engine/access/collection_sync/execution_data_index/processor.go similarity index 72% rename from engine/access/collection_sync/collections/execution_data_processor.go rename to engine/access/collection_sync/execution_data_index/processor.go index f7191c13138..e91d4be3748 100644 --- a/engine/access/collection_sync/collections/execution_data_processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -1,4 +1,4 @@ -package collections +package execution_data_index import ( "fmt" @@ -12,7 +12,7 @@ import ( type ExecutionDataProcessor struct { component.Component newExecutionDataIndexed chan struct{} - ediHeightProvider collection_sync.EDIHeightProvider + provider collection_sync.ExecutionDataProvider indexer collection_sync.BlockCollectionIndexer // state processedHeight *counters.PersistentStrictMonotonicCounter @@ -22,17 +22,21 @@ var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) var _ component.Component = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( - ediHeightProvider collection_sync.EDIHeightProvider, + provider collection_sync.ExecutionDataProvider, indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ newExecutionDataIndexed: make(chan struct{}, 1), - ediHeightProvider: ediHeightProvider, + provider: provider, indexer: indexer, processedHeight: processedHeight, } + // Initialize the channel so that even if no new execution data comes in, + // the worker loop can still be triggered to process any existing data. + edp.newExecutionDataIndexed <- struct{}{} + // Build component manager with worker loop cm := component.NewComponentManagerBuilder(). AddWorker(edp.workerLoop). @@ -64,23 +68,23 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, case <-ctx.Done(): return case <-edp.newExecutionDataIndexed: - highestAvailableHeight := edp.ediHeightProvider.HighestIndexedHeight() + highestAvailableHeight := edp.provider.HighestIndexedHeight() lowestMissing := edp.processedHeight.Value() + 1 for height := lowestMissing; height <= highestAvailableHeight; height++ { - collections, err := edp.ediHeightProvider.GetExecutionDataByHeight(ctx, height) + collections, err := edp.provider.GetExecutionDataByHeight(ctx, height) if err != nil { ctx.Throw(fmt.Errorf("failed to get execution data for height %d: %w", height, err)) return } - // TODO: since both fetcher and execution data processor are the data source of + // TODO: since both collections and execution data processor are the data source of // collections, before indexing the collections, double check if it was indexed - // by the fetcher already by simply comparing the missing height with the - // fetcher's lowest height. - // if fetcher's lowest height is higher than the missing height, it means the collections - // has been indexed by the fetcher already, no need to index again. - // And make sure reading the fetcher's lowest height is cheap operation (only hitting RW lock) + // by the collections already by simply comparing the missing height with the + // collections's lowest height. + // if collections's lowest height is higher than the missing height, it means the collections + // has been indexed by the collections already, no need to index again. + // And make sure reading the collections's lowest height is cheap operation (only hitting RW lock) err = edp.indexer.IndexCollectionsForBlock(height, collections) // TODO: handle already exists diff --git a/engine/access/collection_sync/collections/edi_height_provider.go b/engine/access/collection_sync/execution_data_index/provider.go similarity index 63% rename from engine/access/collection_sync/collections/edi_height_provider.go rename to engine/access/collection_sync/execution_data_index/provider.go index ae0697c48b9..7928b3cc504 100644 --- a/engine/access/collection_sync/collections/edi_height_provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -1,4 +1,4 @@ -package collections +package execution_data_index import ( "context" @@ -9,33 +9,33 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) -var _ collection_sync.EDIHeightProvider = (*ediHeightProvider)(nil) +var _ collection_sync.ExecutionDataProvider = (*executionDataProvider)(nil) -// ediHeightProvider implements EDIHeightProvider by querying ExecutionDataCache. -type ediHeightProvider struct { +// executionDataProvider implements ExecutionDataProvider by querying ExecutionDataCache. +type executionDataProvider struct { cache execution_data.ExecutionDataCache highestExectuionDataHeight tracker.ExecutionDataTracker } -// NewEDIHeightProvider creates a new EDIHeightProvider that reads from the given ExecutionDataCache. +// NewExecutionDataProvider creates a new ExecutionDataProvider that reads from the given ExecutionDataCache. // The headers storage is used to determine the search range for finding available heights. -func NewEDIHeightProvider( +func NewExecutionDataProvider( cache execution_data.ExecutionDataCache, highestExectuionDataHeight tracker.ExecutionDataTracker, -) *ediHeightProvider { - return &ediHeightProvider{ +) *executionDataProvider { + return &executionDataProvider{ cache: cache, highestExectuionDataHeight: highestExectuionDataHeight, } } // HighestIndexedHeight returns the highest block height for which execution data is available. -func (p *ediHeightProvider) HighestIndexedHeight() uint64 { +func (p *executionDataProvider) HighestIndexedHeight() uint64 { return p.highestExectuionDataHeight.GetHighestHeight() } // GetExecutionDataByHeight returns the execution data for the given block height. -func (p *ediHeightProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { +func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { blockExecutionData, err := p.cache.ByHeight(ctx, height) if err != nil { return nil, err diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index 6ee99165d3c..8581ea868ef 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -4,7 +4,7 @@ import ( "fmt" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/collection_sync/collections" + "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" @@ -29,9 +29,9 @@ func CreateExecutionDataProcessor( executionDataTracker tracker.ExecutionDataTracker, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, -) (*collections.ExecutionDataProcessor, error) { - // Create EDI height provider - ediHeightProvider := collections.NewEDIHeightProvider(cache, executionDataTracker) +) (*execution_data_index.ExecutionDataProcessor, error) { + // Create execution data provider + executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataTracker) // Convert ConsumerProgress to PersistentStrictMonotonicCounter processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) @@ -40,7 +40,7 @@ func CreateExecutionDataProcessor( } // Create the execution data processor - processor := collections.NewExecutionDataProcessor(ediHeightProvider, indexer, processedHeightCounter) + processor := execution_data_index.NewExecutionDataProcessor(executionDataProvider, indexer, processedHeightCounter) return processor, nil } diff --git a/engine/access/collection_sync/factory/syncer.go b/engine/access/collection_sync/factory/syncer.go index 939897079c3..e50b3232887 100644 --- a/engine/access/collection_sync/factory/syncer.go +++ b/engine/access/collection_sync/factory/syncer.go @@ -3,11 +3,10 @@ package factory import ( "fmt" - "github.com/jordanschalm/lockctx" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/collection_sync/collections" + "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" "github.com/onflow/flow-go/engine/common/requester" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" @@ -27,13 +26,6 @@ type CreateSyncerConfig struct { MaxSearchAhead uint64 } -// CreateSyncerResult holds the results of CreateSyncer. -type CreateSyncerResult struct { - Syncer *collections.Syncer - JobProcessor collection_sync.JobProcessor - RequestEng *requester.Engine -} - // CreateSyncer creates a new Syncer component with all its dependencies. // This function is in the collections package to avoid import cycles: // - collections package already imports collection_sync (for interfaces) @@ -49,7 +41,6 @@ type CreateSyncerResult struct { // - collections: Collections storage // - guarantees: Guarantees storage // - db: Database for storage operations -// - lockManager: Lock manager for coordinating database access // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionExecutedMetric: Metrics collector for tracking collection indexing // - config: Configuration for the syncer @@ -66,11 +57,11 @@ func CreateSyncer( collStore storage.Collections, guarantees storage.Guarantees, db storage.DB, - lockManager lockctx.Manager, + indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, config CreateSyncerConfig, -) (*CreateSyncerResult, error) { +) (*requester.Engine, collection_sync.Syncer, error) { // Create requester engine for requesting collections requestEng, err := requester.New( log.With().Str("entity", "collection").Logger(), @@ -83,29 +74,21 @@ func CreateSyncer( func() flow.Entity { return new(flow.Collection) }, ) if err != nil { - return nil, fmt.Errorf("could not create requester engine: %w", err) + return nil, nil, fmt.Errorf("could not create requester engine: %w", err) } // Create MissingCollectionQueue - mcq := collections.NewMissingCollectionQueue() - - // Create BlockCollectionIndexer - indexer := collections.NewBlockCollectionIndexer( - collectionExecutedMetric, - lockManager, - db, - collStore, - ) + mcq := fetcher.NewMissingCollectionQueue() // Create CollectionRequester - collectionRequester := collections.NewCollectionRequester( + collectionRequester := fetcher.NewCollectionRequester( requestEng, state, guarantees, ) // Create JobProcessor - jobProcessor := collections.NewJobProcessor( + jobProcessor := fetcher.NewJobProcessor( mcq, indexer, collectionRequester, @@ -129,7 +112,7 @@ func CreateSyncer( }) // Create Syncer - syncer, err := collections.NewSyncer( + syncer, err := fetcher.NewSyncer( log, jobProcessor, processedFinalizedBlockHeight, @@ -139,12 +122,8 @@ func CreateSyncer( config.MaxSearchAhead, ) if err != nil { - return nil, fmt.Errorf("could not create syncer: %w", err) + return nil, nil, fmt.Errorf("could not create syncer: %w", err) } - return &CreateSyncerResult{ - Syncer: syncer, - JobProcessor: jobProcessor, - RequestEng: requestEng, - }, nil + return requestEng, syncer, nil } diff --git a/engine/access/collection_sync/collections/block_collection_indexer.go b/engine/access/collection_sync/fetcher/block_collection_indexer.go similarity index 90% rename from engine/access/collection_sync/collections/block_collection_indexer.go rename to engine/access/collection_sync/fetcher/block_collection_indexer.go index ac92bc16cb8..1ae72770566 100644 --- a/engine/access/collection_sync/collections/block_collection_indexer.go +++ b/engine/access/collection_sync/fetcher/block_collection_indexer.go @@ -1,4 +1,4 @@ -package collections +package fetcher import ( "fmt" @@ -12,7 +12,7 @@ import ( ) // blockCollectionIndexerImpl implements BlockCollectionIndexer. -// It stores and indexes collections for a given block height. +// It stores and indexes fetcher for a given block height. type blockCollectionIndexerImpl struct { metrics module.CollectionExecutedMetric lockManager lockctx.Manager @@ -28,7 +28,7 @@ var _ collection_sync.BlockCollectionIndexer = (*blockCollectionIndexerImpl)(nil // - metrics: Metrics collector for tracking collection indexing // - lockManager: Lock manager for coordinating database access // - db: Database for storage operations -// - collections: Collections storage for storing and indexing collections +// - collections: collections storage for storing and indexing collections // // No error returns are expected during normal operation. func NewBlockCollectionIndexer( @@ -45,7 +45,7 @@ func NewBlockCollectionIndexer( } } -// OnReceivedCollectionsForBlock stores and indexes collections for a given block height. +// IndexCollectionsForBlock stores and indexes collections for a given block height. // // No error returns are expected during normal operation. func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( diff --git a/engine/access/collection_sync/collections/job_processor.go b/engine/access/collection_sync/fetcher/job_processor.go similarity index 99% rename from engine/access/collection_sync/collections/job_processor.go rename to engine/access/collection_sync/fetcher/job_processor.go index 7911bd65aa0..280b420fea2 100644 --- a/engine/access/collection_sync/collections/job_processor.go +++ b/engine/access/collection_sync/fetcher/job_processor.go @@ -1,4 +1,4 @@ -package collections +package fetcher import ( "errors" diff --git a/engine/access/collection_sync/collections/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go similarity index 99% rename from engine/access/collection_sync/collections/missing_collection_queue.go rename to engine/access/collection_sync/fetcher/missing_collection_queue.go index 45530b102b7..90f253075db 100644 --- a/engine/access/collection_sync/collections/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -1,4 +1,4 @@ -package collections +package fetcher import ( "fmt" diff --git a/engine/access/collection_sync/collections/requester.go b/engine/access/collection_sync/fetcher/requester.go similarity index 99% rename from engine/access/collection_sync/collections/requester.go rename to engine/access/collection_sync/fetcher/requester.go index 0e873e48eb5..64870314514 100644 --- a/engine/access/collection_sync/collections/requester.go +++ b/engine/access/collection_sync/fetcher/requester.go @@ -1,4 +1,4 @@ -package collections +package fetcher import ( "errors" diff --git a/engine/access/collection_sync/collections/syncer.go b/engine/access/collection_sync/fetcher/syncer.go similarity index 98% rename from engine/access/collection_sync/collections/syncer.go rename to engine/access/collection_sync/fetcher/syncer.go index c56a32b35d7..a1ce305f5f2 100644 --- a/engine/access/collection_sync/collections/syncer.go +++ b/engine/access/collection_sync/fetcher/syncer.go @@ -1,4 +1,4 @@ -package collections +package fetcher import ( "fmt" @@ -27,6 +27,8 @@ type Syncer struct { workSignal engine.Notifier } +var _ component.Component = (*Syncer)(nil) + // NewSyncer creates a new Syncer component. // // Parameters: diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index e4a7348b8c5..6b38f41c812 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -5,6 +5,7 @@ import ( "github.com/rs/zerolog" + "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -22,6 +23,8 @@ import ( // When notified of a new finalized block, it processes all blocks up to the current // finalized head height. type FinalizedBlockProcessor struct { + component.Component + log zerolog.Logger state protocol.State @@ -32,6 +35,8 @@ type FinalizedBlockProcessor struct { collectionExecutedMetric module.CollectionExecutedMetric } +var _ component.Component = (*FinalizedBlockProcessor)(nil) + // NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor. // // No errors are expected during normal operations. @@ -66,11 +71,21 @@ func NewFinalizedBlockProcessor( collectionExecutedMetric: collectionExecutedMetric, } + // Initialize the channel so that even if no new finalized blocks come in, + // the worker loop can still be triggered to process any existing finalized blocks. + processor.blockFinalizedNotifier <- struct{}{} + + cm := component.NewComponentManagerBuilder(). + AddWorker(processor.workerLoop). + Build() + + processor.Component = cm + return processor, nil } // OnBlockFinalized notifies the processor that a new finalized block is available for processing. -func (p *FinalizedBlockProcessor) OnBlockFinalized() { +func (p *FinalizedBlockProcessor) OnBlockFinalized(_ *model.Block) { select { case p.blockFinalizedNotifier <- struct{}{}: default: @@ -80,9 +95,9 @@ func (p *FinalizedBlockProcessor) OnBlockFinalized() { } } -// StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. +// workerLoop begins processing of finalized blocks and signals readiness when initialization is complete. // It uses a single-threaded loop to process each finalized block height sequentially. -func (p *FinalizedBlockProcessor) StartWorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { +func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { ready() for { From 69f2da94c3171b0ddbb082c69876fcb13a658b80 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 07:11:51 -0800 Subject: [PATCH 014/126] simplify the finalizer --- .../node_builder/access_node_builder.go | 166 ++++++++++-------- engine/access/finalized_indexer/processor.go | 162 +++++++++-------- engine/access/rpc/backend/backend.go | 3 +- .../backend/transactions/status/deriver.go | 4 +- 4 files changed, 171 insertions(+), 164 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5d3e0ecd9fb..f984c8b535c 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -1722,10 +1722,6 @@ func (builder *FlowAccessNodeBuilder) enqueueRelayNetwork() { } func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { - var processedFinalizedBlockHeight storage.ConsumerProgressInitializer - var fetchAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer - var syncAndIndexedCollectionsBlockHeight storage.ConsumerProgressInitializer - if builder.executionDataSyncEnabled { builder.BuildExecutionSyncComponents() } @@ -1734,7 +1730,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.IndexerDependencies.Add(versionControlDependable) stopControlDependable := module.NewProxiedReadyDoneAware() builder.IndexerDependencies.Add(stopControlDependable) - var lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + var lastFullBlockHeight counters.Reader + var collectionIndexedHeight storage.ConsumerProgress builder. BuildConsensusFollower(). @@ -1946,12 +1943,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.lightTransactionResults) return nil }). - Module("block height consumer progress", func(node *cmd.NodeConfig) error { - processedFinalizedBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) - fetchAndIndexedCollectionsBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) - syncAndIndexedCollectionsBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight) - return nil - }). Module("block collection indexer", func(node *cmd.NodeConfig) error { builder.blockCollectionIndexer = fetcher.NewBlockCollectionIndexer( notNil(builder.collectionExecutedMetric), @@ -1974,6 +1965,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return fmt.Errorf("failed to initialize monotonic consumer progress: %w", err) } + collectionIndexedHeight = progress + return nil }). Component("version control", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -2234,18 +2227,11 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { useIndex, ) - // Initialize processed height - rootBlockHeight := node.State.Params().FinalizedRoot().Height - processedHeight, err := syncAndIndexedCollectionsBlockHeight.Initialize(rootBlockHeight) - if err != nil { - return nil, fmt.Errorf("could not initialize processed height: %w", err) - } - // Create execution data processor executionDataProcessor, err := collection_syncfactory.CreateExecutionDataProcessor( notNil(builder.ExecutionDataCache), executionDataTracker, - processedHeight, + collectionIndexedHeight, notNil(builder.blockCollectionIndexer), ) if err != nil { @@ -2264,65 +2250,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { }) } - builder.Component("finalized block indexer", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - finalizedBlockProcessor, err := finalized_indexer.NewFinalizedBlockProcessor( - node.Logger, - node.State, - node.Storage.Blocks, - processedFinalizedBlockHeight, - notNil(builder.collectionExecutedMetric), - ) - if err != nil { - return nil, fmt.Errorf("could not create finalized block processor: %w", err) - } - - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(finalizedBlockProcessor.OnBlockFinalized) - - return finalizedBlockProcessor, nil - }) - - builder.Component("ingest receipt", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return ingest_receipt.New( - node.Logger, - node.EngineRegistry, - node.Storage.Receipts, - notNil(builder.collectionExecutedMetric), - ) - }) - - builder.Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // skip if execution data sync is enabled - // Create syncer and requesterEng - requesterEng, syncer, err := factory.CreateSyncer( - node.Logger, - node.EngineRegistry, - node.State, - node.Me, - node.Storage.Blocks, - notNil(builder.collections), - node.Storage.Guarantees, - builder.ProtocolDB, - notNil(builder.blockCollectionIndexer), - fetchAndIndexedCollectionsBlockHeight, - notNil(builder.collectionExecutedMetric), - collection_syncfactory.CreateSyncerConfig{ - MaxProcessing: 10, // TODO: make configurable - MaxSearchAhead: 0, // TODO: make configurable - }, - ) - - if err != nil { - return nil, fmt.Errorf("could not create collection syncer: %w", err) - } - - builder.CollectionRequesterEngine = requesterEng - - return syncer, nil - }) - - builder.Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return builder.CollectionRequesterEngine, nil - }) + builder.Component("finalized block indexer", createFinalizedBlockIndexer(builder)) + builder.Component("ingest receipt", createIngestReceiptEngine(builder)) + createCollectionSyncFetcher(builder) if builder.storeTxResultErrorMessages { var txResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore @@ -2570,6 +2500,86 @@ func (builder *FlowAccessNodeBuilder) initPublicLibp2pNode(networkKey crypto.Pri return libp2pNode, nil } +func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + processedFinalizedBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) + + finalizedBlockProcessor, err := finalized_indexer.NewFinalizedBlockProcessor( + node.Logger, + node.State, + node.Storage.Blocks, + processedFinalizedBlockHeight, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create finalized block processor: %w", err) + } + + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(finalizedBlockProcessor.OnBlockFinalized) + + return finalizedBlockProcessor, nil + } +} + +func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { + builder. + Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + if builder.executionDataSyncEnabled { + // skip if execution data sync is enabled + return &module.NoopReadyDoneAware{}, nil + } + + // fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) + + // skip if execution data sync is enabled + // Create syncer and requesterEng + requesterEng, syncer, err := factory.CreateSyncer( + node.Logger, + node.EngineRegistry, + node.State, + node.Me, + node.Storage.Blocks, + notNil(builder.collections), + node.Storage.Guarantees, + builder.ProtocolDB, + notNil(builder.blockCollectionIndexer), + fetchAndIndexedCollectionsBlockHeight, + notNil(builder.collectionExecutedMetric), + collection_syncfactory.CreateSyncerConfig{ + MaxProcessing: 10, // TODO: make configurable + MaxSearchAhead: 20, // TODO: make configurable + }, + ) + + if err != nil { + return nil, fmt.Errorf("could not create collection syncer: %w", err) + } + + builder.CollectionRequesterEngine = requesterEng + + return syncer, nil + }). + Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + if builder.executionDataSyncEnabled { + return &module.NoopReadyDoneAware{}, nil + } + return builder.CollectionRequesterEngine, nil + }) + +} + +func createIngestReceiptEngine(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return ingest_receipt.New( + node.Logger, + node.EngineRegistry, + node.Storage.Receipts, + notNil(builder.collectionExecutedMetric), + ) + } +} + // notNil ensures that the input is not nil and returns it // the usage is to ensure the dependencies are initialized before initializing a module. // for instance, the IngestionEngine depends on storage.Collections, which is initialized in a diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index 6b38f41c812..6a3afdd3cb9 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -6,38 +6,53 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/module/util" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" ) +const ( + // finalizedBlockProcessorWorkerCount defines the number of workers that + // concurrently process finalized blocks in the job queue. + // MUST be 1 to ensure sequential processing + finalizedBlockProcessorWorkerCount = 1 + + // searchAhead is a number of blocks that should be processed ahead by jobqueue + // MUST be 1 to ensure sequential processing + searchAhead = 1 +) + // FinalizedBlockProcessor handles processing of finalized blocks, // including indexing and syncing of related collections and execution results. // -// FinalizedBlockProcessor processes finalized blocks sequentially using a simple loop -// that iterates from the last processed height to the latest finalized block height. -// When notified of a new finalized block, it processes all blocks up to the current -// finalized head height. +// FinalizedBlockProcessor is designed to handle the ingestion of finalized Flow blocks +// in a scalable and decoupled manner. It uses a jobqueue.ComponentConsumer to consume +// and process finalized block jobs asynchronously. This design enables the processor +// to handle high-throughput block finalization events without blocking other parts +// of the system. +// +// The processor relies on a notifier (engine.Notifier) to signal when a new finalized +// block is available, which triggers the job consumer to process it. The actual +// processing involves indexing block-to-collection and block-to-execution-result +// mappings, as well as requesting the associated collections. type FinalizedBlockProcessor struct { - component.Component - log zerolog.Logger - state protocol.State - blocks storage.Blocks - processedHeight *counters.PersistentStrictMonotonicCounter + consumer *jobqueue.ComponentConsumer + blockFinalizedNotifier engine.Notifier + blocks storage.Blocks - blockFinalizedNotifier chan struct{} collectionExecutedMetric module.CollectionExecutedMetric } -var _ component.Component = (*FinalizedBlockProcessor)(nil) - -// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor. +// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor, +// setting up job consumer infrastructure to handle finalized block processing. // // No errors are expected during normal operations. func NewFinalizedBlockProcessor( @@ -47,95 +62,76 @@ func NewFinalizedBlockProcessor( finalizedProcessedHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, ) (*FinalizedBlockProcessor, error) { + reader := jobqueue.NewFinalizedBlockReader(state, blocks) finalizedBlock, err := state.Final().Head() if err != nil { return nil, fmt.Errorf("could not get finalized block header: %w", err) } - processedHeightProgress, err := finalizedProcessedHeight.Initialize(finalizedBlock.Height) - if err != nil { - return nil, fmt.Errorf("could not initialize processed height: %w", err) - } - - processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeightProgress) - if err != nil { - return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) - } - + blockFinalizedNotifier := engine.NewNotifier() processor := &FinalizedBlockProcessor{ log: log, - state: state, blocks: blocks, - processedHeight: processedHeightCounter, - blockFinalizedNotifier: make(chan struct{}, 1), + blockFinalizedNotifier: blockFinalizedNotifier, collectionExecutedMetric: collectionExecutedMetric, } - // Initialize the channel so that even if no new finalized blocks come in, - // the worker loop can still be triggered to process any existing finalized blocks. - processor.blockFinalizedNotifier <- struct{}{} - - cm := component.NewComponentManagerBuilder(). - AddWorker(processor.workerLoop). - Build() - - processor.Component = cm + processor.consumer, err = jobqueue.NewComponentConsumer( + log.With().Str("module", "ingestion_block_consumer").Logger(), + blockFinalizedNotifier.Channel(), + finalizedProcessedHeight, + reader, + finalizedBlock.Height, + processor.processFinalizedBlockJobCallback, + finalizedBlockProcessorWorkerCount, + searchAhead, + ) + if err != nil { + return nil, fmt.Errorf("error creating finalized block jobqueue: %w", err) + } return processor, nil } -// OnBlockFinalized notifies the processor that a new finalized block is available for processing. +// Notify notifies the processor that a new finalized block is available for processing. func (p *FinalizedBlockProcessor) OnBlockFinalized(_ *model.Block) { - select { - case p.blockFinalizedNotifier <- struct{}{}: - default: - // if the channel is full, no need to block, just return. - // once the worker loop processes the buffered signal, it will - // process the next height all the way to the highest finalized height. + p.blockFinalizedNotifier.Notify() +} + +// StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. +func (p *FinalizedBlockProcessor) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + p.consumer.Start(ctx) + + err := util.WaitClosed(ctx, p.consumer.Ready()) + if err == nil { + ready() } + + <-p.consumer.Done() } -// workerLoop begins processing of finalized blocks and signals readiness when initialization is complete. -// It uses a single-threaded loop to process each finalized block height sequentially. -func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - ready() - - for { - select { - case <-ctx.Done(): - return - case <-p.blockFinalizedNotifier: - finalizedHead, err := p.state.Final().Head() - if err != nil { - ctx.Throw(fmt.Errorf("failed to get finalized head: %w", err)) - return - } - - highestFinalizedHeight := finalizedHead.Height - lowestMissing := p.processedHeight.Value() + 1 - - for height := lowestMissing; height <= highestFinalizedHeight; height++ { - block, err := p.blocks.ByHeight(height) - if err != nil { - ctx.Throw(fmt.Errorf("failed to get block at height %d: %w", height, err)) - return - } - - err = p.indexForFinalizedBlock(block) - if err != nil { - ctx.Throw(fmt.Errorf("failed to index finalized block at height %d: %w", height, err)) - return - } - - // Update processed height after successful indexing - err = p.processedHeight.Set(height) - if err != nil { - ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) - return - } - } - } +// processFinalizedBlockJobCallback is a jobqueue callback that processes a finalized block job. +func (p *FinalizedBlockProcessor) processFinalizedBlockJobCallback( + ctx irrecoverable.SignalerContext, + job module.Job, + done func(), +) { + finalizedBlock, err := jobqueue.JobToBlock(job) + if err != nil { + ctx.Throw(fmt.Errorf("failed to convert job to finalizedBlock: %w", err)) + return + } + + err = p.indexForFinalizedBlock(finalizedBlock) + if err != nil { + p.log.Error().Err(err). + Str("job_id", string(job.ID())). + Msg("unexpected error during finalized block processing job") + ctx.Throw(fmt.Errorf("failed to index finalized block: %w", err)) + return } + + done() } // indexForFinalizedBlock indexes the given finalized block’s collection guarantees diff --git a/engine/access/rpc/backend/backend.go b/engine/access/rpc/backend/backend.go index c7bc39052e8..ef4d197f4b3 100644 --- a/engine/access/rpc/backend/backend.go +++ b/engine/access/rpc/backend/backend.go @@ -113,7 +113,8 @@ type Params struct { EventsIndex *index.EventsIndex TxResultQueryMode query_mode.IndexQueryMode TxResultsIndex *index.TransactionResultsIndex - LastFullBlockHeight *counters.PersistentStrictMonotonicCounter + CollectionIndexedHeight storage.ConsumerProgress + LastFullBlockHeight counters.Reader IndexReporter state_synchronization.IndexReporter VersionControl *version.VersionControl ExecNodeIdentitiesProvider *rpc.ExecutionNodeIdentitiesProvider diff --git a/engine/access/rpc/backend/transactions/status/deriver.go b/engine/access/rpc/backend/transactions/status/deriver.go index d32b9d26357..7c16c1e9549 100644 --- a/engine/access/rpc/backend/transactions/status/deriver.go +++ b/engine/access/rpc/backend/transactions/status/deriver.go @@ -9,10 +9,10 @@ import ( type TxStatusDeriver struct { state protocol.State - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + lastFullBlockHeight counters.Reader } -func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight *counters.PersistentStrictMonotonicCounter) *TxStatusDeriver { +func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight counters.Reader) *TxStatusDeriver { return &TxStatusDeriver{ state: state, lastFullBlockHeight: lastFullBlockHeight, From c9f1ffab8fd6d5f1f70cb6a3c677e96e9e110e09 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 07:13:42 -0800 Subject: [PATCH 015/126] add component to finalized indexer processor --- engine/access/finalized_indexer/processor.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index 6a3afdd3cb9..ed605e09ab7 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -43,6 +43,7 @@ const ( // mappings, as well as requesting the associated collections. type FinalizedBlockProcessor struct { log zerolog.Logger + component.Component consumer *jobqueue.ComponentConsumer blockFinalizedNotifier engine.Notifier @@ -90,6 +91,13 @@ func NewFinalizedBlockProcessor( return nil, fmt.Errorf("error creating finalized block jobqueue: %w", err) } + // Build component manager with worker loop + cm := component.NewComponentManagerBuilder(). + AddWorker(processor.workerLoop). + Build() + + processor.Component = cm + return processor, nil } @@ -99,7 +107,7 @@ func (p *FinalizedBlockProcessor) OnBlockFinalized(_ *model.Block) { } // StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. -func (p *FinalizedBlockProcessor) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { +func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { p.consumer.Start(ctx) err := util.WaitClosed(ctx, p.consumer.Ready()) From 1c43552c0684d805eb68a41c7d2a4598d6d544a4 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 07:18:35 -0800 Subject: [PATCH 016/126] rename syncer to fetcher --- .../node_builder/access_node_builder.go | 10 +++---- .../access/collection_sync/collection_sync.go | 4 +-- .../access/collection_sync/factory/syncer.go | 26 +++++++++---------- .../fetcher/{syncer.go => fetcher.go} | 24 ++++++++--------- 4 files changed, 32 insertions(+), 32 deletions(-) rename engine/access/collection_sync/fetcher/{syncer.go => fetcher.go} (87%) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index f984c8b535c..5d655b9316a 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2533,8 +2533,8 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) // skip if execution data sync is enabled - // Create syncer and requesterEng - requesterEng, syncer, err := factory.CreateSyncer( + // Create fetcher and requesterEng + requesterEng, fetcher, err := factory.CreateFetcher( node.Logger, node.EngineRegistry, node.State, @@ -2546,19 +2546,19 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { notNil(builder.blockCollectionIndexer), fetchAndIndexedCollectionsBlockHeight, notNil(builder.collectionExecutedMetric), - collection_syncfactory.CreateSyncerConfig{ + collection_syncfactory.CreateFetcherConfig{ MaxProcessing: 10, // TODO: make configurable MaxSearchAhead: 20, // TODO: make configurable }, ) if err != nil { - return nil, fmt.Errorf("could not create collection syncer: %w", err) + return nil, fmt.Errorf("could not create collection fetcher: %w", err) } builder.CollectionRequesterEngine = requesterEng - return syncer, nil + return fetcher, nil }). Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { if builder.executionDataSyncEnabled { diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 5972cb44680..e0e26663c94 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -39,9 +39,9 @@ type JobProcessor interface { ProcessJobConcurrently(ctx irrecoverable.SignalerContext, job module.Job, done func()) error } -// Syncer is a component that consumes finalized block jobs and processes them +// Fetcher is a component that consumes finalized block jobs and processes them // to index collections. It uses a job consumer with windowed throttling to prevent node overload. -type Syncer interface { +type Fetcher interface { component.Component OnFinalizedBlock() LastProcessedIndex() uint64 diff --git a/engine/access/collection_sync/factory/syncer.go b/engine/access/collection_sync/factory/syncer.go index e50b3232887..3a099af387b 100644 --- a/engine/access/collection_sync/factory/syncer.go +++ b/engine/access/collection_sync/factory/syncer.go @@ -18,18 +18,18 @@ import ( "github.com/onflow/flow-go/storage" ) -// CreateSyncerConfig holds configuration parameters for creating a Syncer. -type CreateSyncerConfig struct { +// CreateFetcherConfig holds configuration parameters for creating a Fetcher. +type CreateFetcherConfig struct { // MaxProcessing is the maximum number of jobs to process concurrently. MaxProcessing uint64 // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. MaxSearchAhead uint64 } -// CreateSyncer creates a new Syncer component with all its dependencies. +// CreateFetcher creates a new Fetcher component with all its dependencies. // This function is in the collections package to avoid import cycles: // - collections package already imports collection_sync (for interfaces) -// - CreateSyncer needs to create concrete types from collections package +// - CreateFetcher needs to create concrete types from collections package // - Placing it in collection_sync would create: collection_sync -> collections -> collection_sync (cycle) // // Parameters: @@ -43,12 +43,12 @@ type CreateSyncerConfig struct { // - db: Database for storage operations // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionExecutedMetric: Metrics collector for tracking collection indexing -// - config: Configuration for the syncer +// - config: Configuration for the fetcher // -// Returns both the Syncer and JobProcessor so they can be reused in other components. +// Returns both the Fetcher and JobProcessor so they can be reused in other components. // // No error returns are expected during normal operation. -func CreateSyncer( +func CreateFetcher( log zerolog.Logger, engineRegistry network.EngineRegistry, state protocol.State, @@ -60,8 +60,8 @@ func CreateSyncer( indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, - config CreateSyncerConfig, -) (*requester.Engine, collection_sync.Syncer, error) { + config CreateFetcherConfig, +) (*requester.Engine, collection_sync.Fetcher, error) { // Create requester engine for requesting collections requestEng, err := requester.New( log.With().Str("entity", "collection").Logger(), @@ -111,8 +111,8 @@ func CreateSyncer( } }) - // Create Syncer - syncer, err := fetcher.NewSyncer( + // Create Fetcher + collectionFetcher, err := fetcher.NewFetcher( log, jobProcessor, processedFinalizedBlockHeight, @@ -122,8 +122,8 @@ func CreateSyncer( config.MaxSearchAhead, ) if err != nil { - return nil, nil, fmt.Errorf("could not create syncer: %w", err) + return nil, nil, fmt.Errorf("could not create fetcher: %w", err) } - return requestEng, syncer, nil + return requestEng, collectionFetcher, nil } diff --git a/engine/access/collection_sync/fetcher/syncer.go b/engine/access/collection_sync/fetcher/fetcher.go similarity index 87% rename from engine/access/collection_sync/fetcher/syncer.go rename to engine/access/collection_sync/fetcher/fetcher.go index a1ce305f5f2..46587167abc 100644 --- a/engine/access/collection_sync/fetcher/syncer.go +++ b/engine/access/collection_sync/fetcher/fetcher.go @@ -15,11 +15,11 @@ import ( "github.com/onflow/flow-go/storage" ) -var _ collection_sync.Syncer = (*Syncer)(nil) +var _ collection_sync.Fetcher = (*Fetcher)(nil) -// Syncer is a component that consumes finalized block jobs and processes them +// Fetcher is a component that consumes finalized block jobs and processes them // to index collections. It uses a job consumer with windowed throttling to prevent node overload. -type Syncer struct { +type Fetcher struct { component.Component consumer *jobqueue.ComponentConsumer @@ -27,9 +27,9 @@ type Syncer struct { workSignal engine.Notifier } -var _ component.Component = (*Syncer)(nil) +var _ component.Component = (*Fetcher)(nil) -// NewSyncer creates a new Syncer component. +// NewFetcher creates a new Fetcher component. // // Parameters: // - log: Logger for the component @@ -41,7 +41,7 @@ var _ component.Component = (*Syncer)(nil) // - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit // // No error returns are expected during normal operation. -func NewSyncer( +func NewFetcher( log zerolog.Logger, jobProcessor collection_sync.JobProcessor, progressInitializer storage.ConsumerProgressInitializer, @@ -49,7 +49,7 @@ func NewSyncer( blocks storage.Blocks, maxProcessing uint64, // max number of blocks to fetch collections maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for -) (*Syncer, error) { +) (*Fetcher, error) { workSignal := engine.NewNotifier() // Read the default index from the finalized root height @@ -80,7 +80,7 @@ func NewSyncer( return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) } - return &Syncer{ + return &Fetcher{ Component: consumer, consumer: consumer, jobProcessor: jobProcessor, @@ -90,24 +90,24 @@ func NewSyncer( // OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer // that new work is available. -func (s *Syncer) OnFinalizedBlock() { +func (s *Fetcher) OnFinalizedBlock() { s.workSignal.Notify() } // LastProcessedIndex returns the last processed job index. // Optional methods, not required for operation but useful for monitoring. -func (s *Syncer) LastProcessedIndex() uint64 { +func (s *Fetcher) LastProcessedIndex() uint64 { return s.consumer.LastProcessedIndex() } // Head returns the highest job index available. // Optional methods, not required for operation but useful for monitoring. -func (s *Syncer) Head() (uint64, error) { +func (s *Fetcher) Head() (uint64, error) { return s.consumer.Head() } // Size returns the number of in-memory jobs that the consumer is processing. // Optional methods, not required for operation but useful for monitoring. -func (s *Syncer) Size() uint { +func (s *Fetcher) Size() uint { return s.consumer.Size() } From 66e639bcffc260efc41b9c05ecf05961b3e3d505 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 08:27:35 -0800 Subject: [PATCH 017/126] add comment --- .../node_builder/access_node_builder.go | 2 + .../access/collection_sync/fetcher/syncer.go | 113 ++++++++++++++++++ .../requester/execution_data_requester.go | 8 +- 3 files changed, 122 insertions(+), 1 deletion(-) create mode 100644 engine/access/collection_sync/fetcher/syncer.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5d655b9316a..2bd03830dcb 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -1960,6 +1960,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return err } + // TODO: won't work lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(progress) if err != nil { return fmt.Errorf("failed to initialize monotonic consumer progress: %w", err) @@ -2530,6 +2531,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { } // fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) // skip if execution data sync is enabled diff --git a/engine/access/collection_sync/fetcher/syncer.go b/engine/access/collection_sync/fetcher/syncer.go new file mode 100644 index 00000000000..46587167abc --- /dev/null +++ b/engine/access/collection_sync/fetcher/syncer.go @@ -0,0 +1,113 @@ +package fetcher + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +var _ collection_sync.Fetcher = (*Fetcher)(nil) + +// Fetcher is a component that consumes finalized block jobs and processes them +// to index collections. It uses a job consumer with windowed throttling to prevent node overload. +type Fetcher struct { + component.Component + + consumer *jobqueue.ComponentConsumer + jobProcessor collection_sync.JobProcessor + workSignal engine.Notifier +} + +var _ component.Component = (*Fetcher)(nil) + +// NewFetcher creates a new Fetcher component. +// +// Parameters: +// - log: Logger for the component +// - jobProcessor: JobProcessor implementation for processing collection indexing jobs +// - progressInitializer: Initializer for tracking processed block heights +// - state: Protocol state for reading finalized block information +// - blocks: Blocks storage for reading blocks by height +// - maxProcessing: Maximum number of jobs to process concurrently +// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit +// +// No error returns are expected during normal operation. +func NewFetcher( + log zerolog.Logger, + jobProcessor collection_sync.JobProcessor, + progressInitializer storage.ConsumerProgressInitializer, + state protocol.State, + blocks storage.Blocks, + maxProcessing uint64, // max number of blocks to fetch collections + maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for +) (*Fetcher, error) { + workSignal := engine.NewNotifier() + + // Read the default index from the finalized root height + defaultIndex := state.Params().FinalizedRoot().Height + + // Create a Jobs instance that reads finalized blocks by height + jobs := jobqueue.NewFinalizedBlockReader(state, blocks) + + // Create an adapter function that wraps the JobProcessor interface + processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { + err := jobProcessor.ProcessJobConcurrently(ctx, job, done) + if err != nil { + ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) + } + } + + consumer, err := jobqueue.NewComponentConsumer( + log.With().Str("component", "collection-syncing").Logger(), + workSignal.Channel(), + progressInitializer, + jobs, + defaultIndex, + processorFunc, + maxProcessing, + maxSearchAhead, + ) + if err != nil { + return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) + } + + return &Fetcher{ + Component: consumer, + consumer: consumer, + jobProcessor: jobProcessor, + workSignal: workSignal, + }, nil +} + +// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer +// that new work is available. +func (s *Fetcher) OnFinalizedBlock() { + s.workSignal.Notify() +} + +// LastProcessedIndex returns the last processed job index. +// Optional methods, not required for operation but useful for monitoring. +func (s *Fetcher) LastProcessedIndex() uint64 { + return s.consumer.LastProcessedIndex() +} + +// Head returns the highest job index available. +// Optional methods, not required for operation but useful for monitoring. +func (s *Fetcher) Head() (uint64, error) { + return s.consumer.Head() +} + +// Size returns the number of in-memory jobs that the consumer is processing. +// Optional methods, not required for operation but useful for monitoring. +func (s *Fetcher) Size() uint { + return s.consumer.Size() +} diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 237366ef8a7..7b3697c07b8 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -146,7 +146,6 @@ func New( downloader execution_data.Downloader, execDataCache *cache.ExecutionDataCache, processedHeight storage.ConsumerProgressInitializer, - // ? (leo) what's the difference from processedHeight? processedNotifications storage.ConsumerProgressInitializer, state protocol.State, headers storage.Headers, @@ -216,6 +215,13 @@ func New( }, ) + // TODO (leo): we don't have to keep the notification distributor, because if we add a new notification + // consumer, we would like to consume from the beginning instead of the last consumed height + // of the existing consumer. + // without the notification distributor, each notification consumer can also be + // simplified as a component with a signal channel as notification, and a worker loop to consume + // notifications and iterate all the way to the blockConsumer.LatestProcessedIndex(). + // notificationConsumer consumes `OnExecutionDataFetched` events, and ensures its consumer // receives this event in consecutive block height order. // It listens to events from `executionDataNotifier`, which is delivered when From 28d459a9c981e78eb6487f996085dccf036d2f32 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 09:24:16 -0800 Subject: [PATCH 018/126] refactor last full block height --- .../node_builder/access_node_builder.go | 22 +++- .../access/collection_sync/collection_sync.go | 7 +- .../execution_data_index/processor.go | 5 + .../factory/{syncer.go => fetcher.go} | 0 .../factory/progress_reader.go | 62 ++++++++++ .../access/collection_sync/fetcher/fetcher.go | 13 +- .../access/collection_sync/fetcher/syncer.go | 113 ------------------ engine/access/rpc/backend/backend.go | 4 +- .../backend/transactions/status/deriver.go | 8 +- 9 files changed, 97 insertions(+), 137 deletions(-) rename engine/access/collection_sync/factory/{syncer.go => fetcher.go} (100%) create mode 100644 engine/access/collection_sync/factory/progress_reader.go delete mode 100644 engine/access/collection_sync/fetcher/syncer.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 2bd03830dcb..c4729f30b43 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -80,7 +80,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/chainsync" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/executiondatasync/execution_data" execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" @@ -368,6 +367,10 @@ type FlowAccessNodeBuilder struct { FollowerEng *followereng.ComplianceEngine StateStreamEng *statestreambackend.Engine + // for tx status deriver to know about the highest full block (a block with all collections synced) + // backed by either collection fetcher to execution data syncing + lastFullBlockHeight *factory.ProgressReader + // grpc servers secureGrpcServer *grpcserver.GrpcServer unsecureGrpcServer *grpcserver.GrpcServer @@ -1730,7 +1733,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.IndexerDependencies.Add(versionControlDependable) stopControlDependable := module.NewProxiedReadyDoneAware() builder.IndexerDependencies.Add(stopControlDependable) - var lastFullBlockHeight counters.Reader var collectionIndexedHeight storage.ConsumerProgress builder. @@ -1960,12 +1962,14 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return err } - // TODO: won't work - lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(progress) + lastProgress, err := progress.ProcessedIndex() if err != nil { - return fmt.Errorf("failed to initialize monotonic consumer progress: %w", err) + return fmt.Errorf("failed to get last processed index for last full block height: %w", err) } + // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher + builder.lastFullBlockHeight = factory.NewProgressReader(lastProgress) + collectionIndexedHeight = progress return nil @@ -2158,7 +2162,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { EventsIndex: notNil(builder.EventsIndex), TxResultQueryMode: txResultQueryMode, TxResultsIndex: notNil(builder.TxResultsIndex), - LastFullBlockHeight: lastFullBlockHeight, + LastFullBlockHeight: notNil(builder.lastFullBlockHeight), IndexReporter: indexReporter, VersionControl: notNil(builder.VersionControl), ExecNodeIdentitiesProvider: notNil(builder.ExecNodeIdentitiesProvider), @@ -2239,6 +2243,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil, fmt.Errorf("could not create execution data processor: %w", err) } + // Store and register with ProgressReader + builder.lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) + // Setup requester to notify processor when new execution data is received if builder.ExecutionDataDistributor != nil { builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { @@ -2560,6 +2567,9 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder.CollectionRequesterEngine = requesterEng + // Store and register with ProgressReader + builder.lastFullBlockHeight.SetCollectionFetcher(fetcher) + return fetcher, nil }). Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index e0e26663c94..9950ce9da00 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -43,9 +43,8 @@ type JobProcessor interface { // to index collections. It uses a job consumer with windowed throttling to prevent node overload. type Fetcher interface { component.Component + ProgressReader OnFinalizedBlock() - LastProcessedIndex() uint64 - Head() (uint64, error) Size() uint } @@ -59,3 +58,7 @@ type ExecutionDataProvider interface { type ExecutionDataProcessor interface { OnNewExectuionData() } + +type ProgressReader interface { + ProcessedHeight() uint64 +} diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index e91d4be3748..d5586e75e1e 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -19,6 +19,7 @@ type ExecutionDataProcessor struct { } var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) +var _ collection_sync.ProgressReader = (*ExecutionDataProcessor)(nil) var _ component.Component = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( @@ -103,3 +104,7 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, } } } + +func (edp *ExecutionDataProcessor) ProcessedHeight() uint64 { + return edp.processedHeight.Value() +} diff --git a/engine/access/collection_sync/factory/syncer.go b/engine/access/collection_sync/factory/fetcher.go similarity index 100% rename from engine/access/collection_sync/factory/syncer.go rename to engine/access/collection_sync/factory/fetcher.go diff --git a/engine/access/collection_sync/factory/progress_reader.go b/engine/access/collection_sync/factory/progress_reader.go new file mode 100644 index 00000000000..ce65016fa4c --- /dev/null +++ b/engine/access/collection_sync/factory/progress_reader.go @@ -0,0 +1,62 @@ +package factory + +import "github.com/onflow/flow-go/engine/access/collection_sync" + +// ProgressReader aggregates progress from multiple backends and returns the maximum +// processed height. It can be initialized with an optional lastProgress value and +// two optional backends: executionDataProcessor and collectionFetcher. +type ProgressReader struct { + lastProgress uint64 + executionDataProcessor collection_sync.ProgressReader + collectionFetcher collection_sync.ProgressReader +} + +var _ collection_sync.ProgressReader = (*ProgressReader)(nil) + +// NewProgressReader creates a new ProgressReader initialized with lastProgress. +// Backends can be added using SetExecutionDataProcessor and SetCollectionFetcher. +func NewProgressReader(lastProgress uint64) *ProgressReader { + return &ProgressReader{ + lastProgress: lastProgress, + executionDataProcessor: nil, + collectionFetcher: nil, + } +} + +// SetExecutionDataProcessor sets the execution data processor backend. +func (pr *ProgressReader) SetExecutionDataProcessor(backend collection_sync.ProgressReader) { + pr.executionDataProcessor = backend +} + +// SetCollectionFetcher sets the collection fetcher backend. +func (pr *ProgressReader) SetCollectionFetcher(backend collection_sync.ProgressReader) { + pr.collectionFetcher = backend +} + +// ProcessedHeight returns the maximum processed height from the available backends. +// If both backends are available, it returns the maximum of their progress. +// If only one backend is available, it returns that backend's progress. +// If neither backend is available, it returns lastProgress. +func (pr *ProgressReader) ProcessedHeight() uint64 { + hasExecutionData := pr.executionDataProcessor != nil + hasCollectionFetcher := pr.collectionFetcher != nil + + if hasExecutionData && hasCollectionFetcher { + execHeight := pr.executionDataProcessor.ProcessedHeight() + collectionHeight := pr.collectionFetcher.ProcessedHeight() + if execHeight > collectionHeight { + return execHeight + } + return collectionHeight + } + + if hasExecutionData { + return pr.executionDataProcessor.ProcessedHeight() + } + + if hasCollectionFetcher { + return pr.collectionFetcher.ProcessedHeight() + } + + return pr.lastProgress +} diff --git a/engine/access/collection_sync/fetcher/fetcher.go b/engine/access/collection_sync/fetcher/fetcher.go index 46587167abc..25e71c07ae8 100644 --- a/engine/access/collection_sync/fetcher/fetcher.go +++ b/engine/access/collection_sync/fetcher/fetcher.go @@ -15,8 +15,6 @@ import ( "github.com/onflow/flow-go/storage" ) -var _ collection_sync.Fetcher = (*Fetcher)(nil) - // Fetcher is a component that consumes finalized block jobs and processes them // to index collections. It uses a job consumer with windowed throttling to prevent node overload. type Fetcher struct { @@ -27,6 +25,8 @@ type Fetcher struct { workSignal engine.Notifier } +var _ collection_sync.Fetcher = (*Fetcher)(nil) +var _ collection_sync.ProgressReader = (*Fetcher)(nil) var _ component.Component = (*Fetcher)(nil) // NewFetcher creates a new Fetcher component. @@ -95,17 +95,10 @@ func (s *Fetcher) OnFinalizedBlock() { } // LastProcessedIndex returns the last processed job index. -// Optional methods, not required for operation but useful for monitoring. -func (s *Fetcher) LastProcessedIndex() uint64 { +func (s *Fetcher) ProcessedHeight() uint64 { return s.consumer.LastProcessedIndex() } -// Head returns the highest job index available. -// Optional methods, not required for operation but useful for monitoring. -func (s *Fetcher) Head() (uint64, error) { - return s.consumer.Head() -} - // Size returns the number of in-memory jobs that the consumer is processing. // Optional methods, not required for operation but useful for monitoring. func (s *Fetcher) Size() uint { diff --git a/engine/access/collection_sync/fetcher/syncer.go b/engine/access/collection_sync/fetcher/syncer.go deleted file mode 100644 index 46587167abc..00000000000 --- a/engine/access/collection_sync/fetcher/syncer.go +++ /dev/null @@ -1,113 +0,0 @@ -package fetcher - -import ( - "fmt" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/jobqueue" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -var _ collection_sync.Fetcher = (*Fetcher)(nil) - -// Fetcher is a component that consumes finalized block jobs and processes them -// to index collections. It uses a job consumer with windowed throttling to prevent node overload. -type Fetcher struct { - component.Component - - consumer *jobqueue.ComponentConsumer - jobProcessor collection_sync.JobProcessor - workSignal engine.Notifier -} - -var _ component.Component = (*Fetcher)(nil) - -// NewFetcher creates a new Fetcher component. -// -// Parameters: -// - log: Logger for the component -// - jobProcessor: JobProcessor implementation for processing collection indexing jobs -// - progressInitializer: Initializer for tracking processed block heights -// - state: Protocol state for reading finalized block information -// - blocks: Blocks storage for reading blocks by height -// - maxProcessing: Maximum number of jobs to process concurrently -// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit -// -// No error returns are expected during normal operation. -func NewFetcher( - log zerolog.Logger, - jobProcessor collection_sync.JobProcessor, - progressInitializer storage.ConsumerProgressInitializer, - state protocol.State, - blocks storage.Blocks, - maxProcessing uint64, // max number of blocks to fetch collections - maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for -) (*Fetcher, error) { - workSignal := engine.NewNotifier() - - // Read the default index from the finalized root height - defaultIndex := state.Params().FinalizedRoot().Height - - // Create a Jobs instance that reads finalized blocks by height - jobs := jobqueue.NewFinalizedBlockReader(state, blocks) - - // Create an adapter function that wraps the JobProcessor interface - processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { - err := jobProcessor.ProcessJobConcurrently(ctx, job, done) - if err != nil { - ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) - } - } - - consumer, err := jobqueue.NewComponentConsumer( - log.With().Str("component", "collection-syncing").Logger(), - workSignal.Channel(), - progressInitializer, - jobs, - defaultIndex, - processorFunc, - maxProcessing, - maxSearchAhead, - ) - if err != nil { - return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) - } - - return &Fetcher{ - Component: consumer, - consumer: consumer, - jobProcessor: jobProcessor, - workSignal: workSignal, - }, nil -} - -// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer -// that new work is available. -func (s *Fetcher) OnFinalizedBlock() { - s.workSignal.Notify() -} - -// LastProcessedIndex returns the last processed job index. -// Optional methods, not required for operation but useful for monitoring. -func (s *Fetcher) LastProcessedIndex() uint64 { - return s.consumer.LastProcessedIndex() -} - -// Head returns the highest job index available. -// Optional methods, not required for operation but useful for monitoring. -func (s *Fetcher) Head() (uint64, error) { - return s.consumer.Head() -} - -// Size returns the number of in-memory jobs that the consumer is processing. -// Optional methods, not required for operation but useful for monitoring. -func (s *Fetcher) Size() uint { - return s.consumer.Size() -} diff --git a/engine/access/rpc/backend/backend.go b/engine/access/rpc/backend/backend.go index ef4d197f4b3..05bb1d9c2e6 100644 --- a/engine/access/rpc/backend/backend.go +++ b/engine/access/rpc/backend/backend.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/access" "github.com/onflow/flow-go/access/validator" "github.com/onflow/flow-go/cmd/build" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" "github.com/onflow/flow-go/engine/access/rpc/backend/accounts" "github.com/onflow/flow-go/engine/access/rpc/backend/common" @@ -34,7 +35,6 @@ import ( accessmodel "github.com/onflow/flow-go/model/access" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/state/protocol" @@ -114,7 +114,7 @@ type Params struct { TxResultQueryMode query_mode.IndexQueryMode TxResultsIndex *index.TransactionResultsIndex CollectionIndexedHeight storage.ConsumerProgress - LastFullBlockHeight counters.Reader + LastFullBlockHeight collection_sync.ProgressReader IndexReporter state_synchronization.IndexReporter VersionControl *version.VersionControl ExecNodeIdentitiesProvider *rpc.ExecutionNodeIdentitiesProvider diff --git a/engine/access/rpc/backend/transactions/status/deriver.go b/engine/access/rpc/backend/transactions/status/deriver.go index 7c16c1e9549..af43d882d28 100644 --- a/engine/access/rpc/backend/transactions/status/deriver.go +++ b/engine/access/rpc/backend/transactions/status/deriver.go @@ -1,18 +1,18 @@ package status import ( + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/state/protocol" ) type TxStatusDeriver struct { state protocol.State - lastFullBlockHeight counters.Reader + lastFullBlockHeight collection_sync.ProgressReader } -func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight counters.Reader) *TxStatusDeriver { +func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight collection_sync.ProgressReader) *TxStatusDeriver { return &TxStatusDeriver{ state: state, lastFullBlockHeight: lastFullBlockHeight, @@ -48,7 +48,7 @@ func (t *TxStatusDeriver) DeriveUnknownTransactionStatus(refBlockID flow.Identif // the last full height is the height where we have received all // collections for all blocks with a lower height - fullHeight := t.lastFullBlockHeight.Value() + fullHeight := t.lastFullBlockHeight.ProcessedHeight() // if we have received collections for all blocks up to the expiry block, the transaction is expired if isExpired(refHeight, fullHeight) { From 96caf344ddc18b49aa0c3f760fa994c9feb47c6d Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 09:28:28 -0800 Subject: [PATCH 019/126] make job consumer LastProcessedIndex and Size to be non blocking --- module/jobqueue/consumer.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/module/jobqueue/consumer.go b/module/jobqueue/consumer.go index 035f625dfaf..a09e617a2fe 100644 --- a/module/jobqueue/consumer.go +++ b/module/jobqueue/consumer.go @@ -18,7 +18,7 @@ type Worker interface { } type Consumer struct { - mu sync.Mutex + mu sync.RWMutex log zerolog.Logger // Storage @@ -124,16 +124,16 @@ func (c *Consumer) Stop() { // Size returns number of in-memory jobs that consumer is processing. func (c *Consumer) Size() uint { - c.mu.Lock() - defer c.mu.Unlock() + c.mu.RLock() + defer c.mu.RUnlock() return uint(len(c.processings)) } // LastProcessedIndex returns the last processed job index func (c *Consumer) LastProcessedIndex() uint64 { - c.mu.Lock() - defer c.mu.Unlock() + c.mu.RLock() + defer c.mu.RUnlock() return c.processedIndex } From 31889f850eafd7ff8372d40deab7ebca65bee42b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:00:51 -0800 Subject: [PATCH 020/126] fix for observer --- cmd/observer/node_builder/observer_builder.go | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index e45564043c4..aae4d039abf 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -38,7 +38,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/apiproxy" "github.com/onflow/flow-go/engine/access/index" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/engine/access/rest" restapiproxy "github.com/onflow/flow-go/engine/access/rest/apiproxy" "github.com/onflow/flow-go/engine/access/rest/router" @@ -70,7 +69,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/chainsync" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/executiondatasync/execution_data" execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" @@ -1440,31 +1438,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS return nil, fmt.Errorf("could not create derived chain data: %w", err) } - rootBlockHeight := node.State.Params().FinalizedRoot().Height - progress, err := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) - if err != nil { - return nil, fmt.Errorf("could not create last full block height consumer progress: %w", err) - } - - lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter(progress) - if err != nil { - return nil, fmt.Errorf("could not create last full block height counter: %w", err) - } - var collectionExecutedMetric module.CollectionExecutedMetric = metrics.NewNoopCollector() - collectionIndexer, err := collections.NewIndexer( - builder.Logger, - builder.ProtocolDB, - collectionExecutedMetric, - builder.State, - builder.Storage.Blocks, - builder.Storage.Collections, - lastFullBlockHeight, - builder.StorageLockMgr, - ) - if err != nil { - return nil, fmt.Errorf("could not create collection indexer: %w", err) - } builder.ExecutionIndexerCore = indexer.New( builder.Logger, @@ -1479,7 +1453,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.scheduledTransactions, builder.RootChainID, indexerDerivedChainData, - collectionIndexer, collectionExecutedMetric, node.StorageLockMgr, ) From eb4f9f517ec1cc1676efa8cda935085c524c607b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:05:45 -0800 Subject: [PATCH 021/126] fix backend test --- engine/access/rpc/backend/backend_test.go | 14 +++++++++++++- engine/access/rpc/backend/script_executor_test.go | 1 - .../transactions/stream/stream_backend_test.go | 14 +++++++++++++- .../transactions/transactions_functional_test.go | 2 +- .../rpc/backend/transactions/transactions_test.go | 14 +++++++++++++- 5 files changed, 40 insertions(+), 5 deletions(-) diff --git a/engine/access/rpc/backend/backend_test.go b/engine/access/rpc/backend/backend_test.go index abe3983006d..238a813096d 100644 --- a/engine/access/rpc/backend/backend_test.go +++ b/engine/access/rpc/backend/backend_test.go @@ -24,6 +24,7 @@ import ( "github.com/onflow/flow-go/cmd/build" accessmock "github.com/onflow/flow-go/engine/access/mock" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/rpc/backend/common" "github.com/onflow/flow-go/engine/access/rpc/backend/events" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" @@ -63,6 +64,17 @@ var eventEncodingVersions = []entitiesproto.EventEncodingVersion{ entitiesproto.EventEncodingVersion_CCF_V0, } +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + type Suite struct { suite.Suite @@ -2052,7 +2064,7 @@ func (suite *Suite) defaultBackendParams() Params { TxResultQueryMode: query_mode.IndexQueryModeExecutionNodesOnly, EventQueryMode: query_mode.IndexQueryModeExecutionNodesOnly, ScriptExecutionMode: query_mode.IndexQueryModeExecutionNodesOnly, - LastFullBlockHeight: suite.lastFullBlockHeight, + LastFullBlockHeight: &progressReaderAdapter{counter: suite.lastFullBlockHeight}, VersionControl: suite.versionControl, ExecNodeIdentitiesProvider: commonrpc.NewExecutionNodeIdentitiesProvider( suite.log, diff --git a/engine/access/rpc/backend/script_executor_test.go b/engine/access/rpc/backend/script_executor_test.go index 9d127da6810..20e17c959a7 100644 --- a/engine/access/rpc/backend/script_executor_test.go +++ b/engine/access/rpc/backend/script_executor_test.go @@ -142,7 +142,6 @@ func (s *ScriptExecutorSuite) SetupTest() { nil, s.chain.ChainID(), derivedChainData, - nil, metrics.NewNoopCollector(), lockManager, ) diff --git a/engine/access/rpc/backend/transactions/stream/stream_backend_test.go b/engine/access/rpc/backend/transactions/stream/stream_backend_test.go index 5dc9a64c982..f2ef3373ec7 100644 --- a/engine/access/rpc/backend/transactions/stream/stream_backend_test.go +++ b/engine/access/rpc/backend/transactions/stream/stream_backend_test.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/access/validator" validatormock "github.com/onflow/flow-go/access/validator/mock" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" access "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" @@ -54,6 +55,17 @@ import ( "github.com/onflow/flow-go/utils/unittest/mocks" ) +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + // TransactionStreamSuite represents a suite for testing transaction status-related functionality in the Flow blockchain. type TransactionStreamSuite struct { suite.Suite @@ -212,7 +224,7 @@ func (s *TransactionStreamSuite) initializeBackend() { txStatusDeriver := txstatus.NewTxStatusDeriver( s.state, - s.lastFullBlockHeight, + &progressReaderAdapter{counter: s.lastFullBlockHeight}, ) nodeCommunicator := node_communicator.NewNodeCommunicator(false) diff --git a/engine/access/rpc/backend/transactions/transactions_functional_test.go b/engine/access/rpc/backend/transactions/transactions_functional_test.go index e12a58c7446..b8b6f9d2135 100644 --- a/engine/access/rpc/backend/transactions/transactions_functional_test.go +++ b/engine/access/rpc/backend/transactions/transactions_functional_test.go @@ -238,7 +238,7 @@ func (s *TransactionsFunctionalSuite) SetupTest() { s.Require().NoError(err) // Instantiate intermediate components - s.txStatusDeriver = txstatus.NewTxStatusDeriver(s.state, s.lastFullBlockHeight) + s.txStatusDeriver = txstatus.NewTxStatusDeriver(s.state, &progressReaderAdapter{counter: s.lastFullBlockHeight}) s.mockState = protocolmock.NewState(s.T()) s.nodeProvider = commonrpc.NewExecutionNodeIdentitiesProvider(s.log, s.mockState, s.receipts, nil, nil) diff --git a/engine/access/rpc/backend/transactions/transactions_test.go b/engine/access/rpc/backend/transactions/transactions_test.go index 6ebb861a551..1847fe5e259 100644 --- a/engine/access/rpc/backend/transactions/transactions_test.go +++ b/engine/access/rpc/backend/transactions/transactions_test.go @@ -16,6 +16,7 @@ import ( "github.com/onflow/flow/protobuf/go/flow/entities" "github.com/onflow/flow-go/access/validator" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" @@ -46,6 +47,17 @@ func TestTransactionsBackend(t *testing.T) { suite.Run(t, new(Suite)) } +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + type Suite struct { suite.Suite @@ -187,7 +199,7 @@ func (suite *Suite) defaultTransactionsParams() Params { txStatusDeriver := txstatus.NewTxStatusDeriver( suite.state, - suite.lastFullBlockHeight, + &progressReaderAdapter{counter: suite.lastFullBlockHeight}, ) validatorBlocks := validator.NewProtocolStateBlocks(suite.state, suite.indexReporter) From 3b5fc1d833dd4369324ad6351e8b612bae36578b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:08:46 -0800 Subject: [PATCH 022/126] fix execution script test --- module/execution/scripts_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/module/execution/scripts_test.go b/module/execution/scripts_test.go index af0b3749642..648f3d01221 100644 --- a/module/execution/scripts_test.go +++ b/module/execution/scripts_test.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/fvm/storage/snapshot" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/state_synchronization/indexer" synctest "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" @@ -180,7 +181,7 @@ func (s *scriptTestSuite) SetupTest() { index := indexer.New( logger, - metrics.NewNoopCollector(), + module.ExecutionStateIndexerMetrics(metrics.NewNoopCollector()), nil, s.registerIndex, headers, @@ -191,8 +192,7 @@ func (s *scriptTestSuite) SetupTest() { nil, flow.Testnet, derivedChainData, - nil, - nil, + metrics.NewNoopCollector(), lockManager, ) From 06b83b0764d84f377846ac113891c9a82b73070c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:11:16 -0800 Subject: [PATCH 023/126] fix indexer tests --- module/state_synchronization/indexer/indexer_core_test.go | 8 +------- module/state_synchronization/indexer/indexer_test.go | 2 -- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index 5a30285dcf6..6ba56701fb8 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -190,7 +190,7 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.indexer = New( log, - metrics.NewNoopCollector(), + module.ExecutionStateIndexerMetrics(metrics.NewNoopCollector()), db, i.registers, i.headers, @@ -201,7 +201,6 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.scheduledTransactions, i.g.ChainID(), derivedChainData, - i.collectionIndexer, collectionExecutedMetric, lockManager, ) @@ -250,7 +249,6 @@ func TestExecutionState_IndexBlockData(t *testing.T) { assert.ElementsMatch(t, tf.ExpectedRegisterEntries, entries) }). Return(nil) - test.collectionIndexer.On("IndexCollections", tf.ExpectedCollections).Return(nil).Once() for txID, scheduledTxID := range tf.ExpectedScheduledTransactions { test.scheduledTransactions.On("BatchIndex", mock.Anything, blockID, txID, scheduledTxID, mock.Anything). Return(func(lctx lockctx.Proof, blockID flow.Identifier, txID flow.Identifier, scheduledTxID uint64, batch storage.ReaderBatchWriter) error { @@ -367,7 +365,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -403,7 +400,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -432,7 +428,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -478,7 +473,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index 7efd98f680f..6f6733a73f3 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -201,7 +201,6 @@ func TestIndexer_Success(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("IndexCollections", ed.StandardCollections()).Return(nil).Once() test.indexTest.registers.On("Store", flow.RegisterEntries{}, block.Height).Return(nil).Once() } @@ -243,7 +242,6 @@ func TestIndexer_Failure(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("IndexCollections", ed.StandardCollections()).Return(nil).Once() // return an error on the last block to trigger the error path if block.Height == lastHeight { From 743d56da6e0ede64a5c27100f050985f212ba529 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:15:45 -0800 Subject: [PATCH 024/126] fix lint --- .../access/collection_sync/fetcher/job_processor.go | 11 +++++------ engine/access/rpc/backend/backend_test.go | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/engine/access/collection_sync/fetcher/job_processor.go b/engine/access/collection_sync/fetcher/job_processor.go index 280b420fea2..fd64e550bc0 100644 --- a/engine/access/collection_sync/fetcher/job_processor.go +++ b/engine/access/collection_sync/fetcher/job_processor.go @@ -16,12 +16,11 @@ import ( // It orchestrates the flow: request → receive → index → complete. // TODO: rename to fetch_job_processor type JobProcessor struct { - mcq collection_sync.MissingCollectionQueue - indexer collection_sync.BlockCollectionIndexer - requester collection_sync.CollectionRequester - blocks storage.Blocks - collections storage.CollectionsReader - newExecutionDataIndexed <-chan struct{} + mcq collection_sync.MissingCollectionQueue + indexer collection_sync.BlockCollectionIndexer + requester collection_sync.CollectionRequester + blocks storage.Blocks + collections storage.CollectionsReader } var _ collection_sync.JobProcessor = (*JobProcessor)(nil) diff --git a/engine/access/rpc/backend/backend_test.go b/engine/access/rpc/backend/backend_test.go index 238a813096d..80216716afa 100644 --- a/engine/access/rpc/backend/backend_test.go +++ b/engine/access/rpc/backend/backend_test.go @@ -23,8 +23,8 @@ import ( "google.golang.org/grpc/status" "github.com/onflow/flow-go/cmd/build" - accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/collection_sync" + accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/common" "github.com/onflow/flow-go/engine/access/rpc/backend/events" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" From 91fe006bee9b8243a0c78fbb54b58d6d39b22913 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 10:38:06 -0800 Subject: [PATCH 025/126] add metrics --- .../node_builder/access_node_builder.go | 7 ++++ .../execution_data_index/processor.go | 9 ++++ .../factory/execution_data_processor.go | 5 ++- .../access/collection_sync/factory/fetcher.go | 3 ++ .../access/collection_sync/fetcher/fetcher.go | 18 +++++++- module/metrics.go | 7 ++++ module/metrics/collection_sync.go | 41 +++++++++++++++++++ module/metrics/noop.go | 5 +++ 8 files changed, 92 insertions(+), 3 deletions(-) create mode 100644 module/metrics/collection_sync.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index c4729f30b43..aac0e12dbf2 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -321,6 +321,7 @@ type FlowAccessNodeBuilder struct { RestMetrics *metrics.RestCollector AccessMetrics module.AccessMetrics PingMetrics module.PingMetrics + CollectionSyncMetrics module.CollectionSyncMetrics Committee hotstuff.DynamicCommittee Finalized *flow.Header // latest finalized block that the node knows of at startup time Pending []*flow.ProposalHeader @@ -1876,6 +1877,10 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.PingMetrics = metrics.NewPingCollector() return nil }). + Module("collection sync metrics", func(node *cmd.NodeConfig) error { + builder.CollectionSyncMetrics = metrics.NewCollectionSyncCollector() + return nil + }). Module("server certificate", func(node *cmd.NodeConfig) error { // generate the server certificate that will be served by the GRPC server x509Certificate, err := grpcutils.X509Certificate(node.NetworkKey) @@ -2238,6 +2243,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { executionDataTracker, collectionIndexedHeight, notNil(builder.blockCollectionIndexer), + builder.CollectionSyncMetrics, ) if err != nil { return nil, fmt.Errorf("could not create execution data processor: %w", err) @@ -2555,6 +2561,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { notNil(builder.blockCollectionIndexer), fetchAndIndexedCollectionsBlockHeight, notNil(builder.collectionExecutedMetric), + builder.CollectionSyncMetrics, collection_syncfactory.CreateFetcherConfig{ MaxProcessing: 10, // TODO: make configurable MaxSearchAhead: 20, // TODO: make configurable diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index d5586e75e1e..52b98d586b8 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -4,6 +4,7 @@ import ( "fmt" "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" @@ -16,6 +17,7 @@ type ExecutionDataProcessor struct { indexer collection_sync.BlockCollectionIndexer // state processedHeight *counters.PersistentStrictMonotonicCounter + metrics module.CollectionSyncMetrics } var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) @@ -26,12 +28,14 @@ func NewExecutionDataProcessor( provider collection_sync.ExecutionDataProvider, indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, + metrics module.CollectionSyncMetrics, // optional metrics collector ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ newExecutionDataIndexed: make(chan struct{}, 1), provider: provider, indexer: indexer, processedHeight: processedHeight, + metrics: metrics, } // Initialize the channel so that even if no new execution data comes in, @@ -100,6 +104,11 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) return } + + // Update metrics if available + if edp.metrics != nil { + edp.metrics.CollectionSyncedHeight(height) + } } } } diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index 8581ea868ef..b0fa6aa0621 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -6,6 +6,7 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/engine/access/subscription/tracker" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" @@ -18,6 +19,7 @@ import ( // - executionDataTracker: Tracker for execution data that provides the highest available height // - processedHeight: Consumer progress for tracking processed heights // - indexer: Block collection indexer for indexing collections +// - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress // // Returns: // - *ExecutionDataProcessor: A new ExecutionDataProcessor instance @@ -29,6 +31,7 @@ func CreateExecutionDataProcessor( executionDataTracker tracker.ExecutionDataTracker, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, + collectionSyncMetrics module.CollectionSyncMetrics, // optional metrics collector ) (*execution_data_index.ExecutionDataProcessor, error) { // Create execution data provider executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataTracker) @@ -40,7 +43,7 @@ func CreateExecutionDataProcessor( } // Create the execution data processor - processor := execution_data_index.NewExecutionDataProcessor(executionDataProvider, indexer, processedHeightCounter) + processor := execution_data_index.NewExecutionDataProcessor(executionDataProvider, indexer, processedHeightCounter, collectionSyncMetrics) return processor, nil } diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 3a099af387b..e25c35aea3f 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -43,6 +43,7 @@ type CreateFetcherConfig struct { // - db: Database for storage operations // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionExecutedMetric: Metrics collector for tracking collection indexing +// - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress // - config: Configuration for the fetcher // // Returns both the Fetcher and JobProcessor so they can be reused in other components. @@ -60,6 +61,7 @@ func CreateFetcher( indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, + collectionSyncMetrics module.CollectionSyncMetrics, // optional metrics collector config CreateFetcherConfig, ) (*requester.Engine, collection_sync.Fetcher, error) { // Create requester engine for requesting collections @@ -120,6 +122,7 @@ func CreateFetcher( blocks, config.MaxProcessing, config.MaxSearchAhead, + collectionSyncMetrics, ) if err != nil { return nil, nil, fmt.Errorf("could not create fetcher: %w", err) diff --git a/engine/access/collection_sync/fetcher/fetcher.go b/engine/access/collection_sync/fetcher/fetcher.go index 25e71c07ae8..31c6f52abf4 100644 --- a/engine/access/collection_sync/fetcher/fetcher.go +++ b/engine/access/collection_sync/fetcher/fetcher.go @@ -23,6 +23,7 @@ type Fetcher struct { consumer *jobqueue.ComponentConsumer jobProcessor collection_sync.JobProcessor workSignal engine.Notifier + metrics module.CollectionSyncMetrics } var _ collection_sync.Fetcher = (*Fetcher)(nil) @@ -39,6 +40,7 @@ var _ component.Component = (*Fetcher)(nil) // - blocks: Blocks storage for reading blocks by height // - maxProcessing: Maximum number of jobs to process concurrently // - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit +// - metrics: Optional metrics collector for reporting collection fetched height // // No error returns are expected during normal operation. func NewFetcher( @@ -49,6 +51,7 @@ func NewFetcher( blocks storage.Blocks, maxProcessing uint64, // max number of blocks to fetch collections maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for + metrics module.CollectionSyncMetrics, // optional metrics collector ) (*Fetcher, error) { workSignal := engine.NewNotifier() @@ -80,12 +83,23 @@ func NewFetcher( return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) } - return &Fetcher{ + f := &Fetcher{ Component: consumer, consumer: consumer, jobProcessor: jobProcessor, workSignal: workSignal, - }, nil + metrics: metrics, + } + + // Set up post-notifier to update metrics when a job is done + if metrics != nil { + consumer.SetPostNotifier(func(jobID module.JobID) { + height := f.ProcessedHeight() + metrics.CollectionFetchedHeight(height) + }) + } + + return f, nil } // OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer diff --git a/module/metrics.go b/module/metrics.go index 39497cdf777..1a865f0dcd9 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -1214,6 +1214,13 @@ type CollectionExecutedMetric interface { UpdateLastFullBlockHeight(height uint64) } +type CollectionSyncMetrics interface { + // CollectionFetchedHeight reports the highest block height for which collections have been fetched + CollectionFetchedHeight(height uint64) + // CollectionSyncedHeight reports the highest block height for which collections have been synced from execution data + CollectionSyncedHeight(height uint64) +} + type MachineAccountMetrics interface { // AccountBalance reports the current balance of the machine account. AccountBalance(bal float64) diff --git a/module/metrics/collection_sync.go b/module/metrics/collection_sync.go new file mode 100644 index 00000000000..89e9efe0d32 --- /dev/null +++ b/module/metrics/collection_sync.go @@ -0,0 +1,41 @@ +package metrics + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/onflow/flow-go/module" +) + +type CollectionSyncCollector struct { + collectionFetchedHeight prometheus.Gauge + collectionSyncedHeight prometheus.Gauge +} + +var _ module.CollectionSyncMetrics = (*CollectionSyncCollector)(nil) + +func NewCollectionSyncCollector() *CollectionSyncCollector { + return &CollectionSyncCollector{ + collectionFetchedHeight: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "collection_fetched_height", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the highest block height for which collections have been fetched", + }), + collectionSyncedHeight: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "collection_synced_height", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the highest block height for which collections have been synced from execution data", + }), + } +} + +func (c *CollectionSyncCollector) CollectionFetchedHeight(height uint64) { + c.collectionFetchedHeight.Set(float64(height)) +} + +func (c *CollectionSyncCollector) CollectionSyncedHeight(height uint64) { + c.collectionSyncedHeight.Set(float64(height)) +} + diff --git a/module/metrics/noop.go b/module/metrics/noop.go index 3de4e382651..69eb594ae27 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -397,6 +397,11 @@ func (nc *NoopCollector) CollectionExecuted(light *flow.LightCollection) {} func (nc *NoopCollector) ExecutionReceiptReceived(r *flow.ExecutionReceipt) { } +var _ module.CollectionSyncMetrics = (*NoopCollector)(nil) + +func (nc *NoopCollector) CollectionFetchedHeight(height uint64) {} +func (nc *NoopCollector) CollectionSyncedHeight(height uint64) {} + func (nc *NoopCollector) AccountBalance(bal float64) {} func (nc *NoopCollector) RecommendedMinBalance(bal float64) {} func (nc *NoopCollector) IsMisconfigured(misconfigured bool) {} From 52149f648afa9da821057e1f5ec0ba75c03d9860 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 13:37:59 -0800 Subject: [PATCH 026/126] update transaction and collections storage --- .../node_builder/access_node_builder.go | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index aac0e12dbf2..b76bb962341 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -354,8 +354,6 @@ type FlowAccessNodeBuilder struct { events storage.Events lightTransactionResults storage.LightTransactionResults transactionResultErrorMessages storage.TransactionResultErrorMessages - transactions storage.Transactions - collections storage.Collections scheduledTransactions storage.ScheduledTransactions // The sync engine participants provider is the libp2p peer store for the access node @@ -583,14 +581,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess AdminCommand("read-execution-data", func(config *cmd.NodeConfig) commands.AdminCommand { return stateSyncCommands.NewReadExecutionDataCommand(builder.ExecutionDataStore) }). - Module("transactions and collections storage", func(node *cmd.NodeConfig) error { - transactions := store.NewTransactions(node.Metrics.Cache, node.ProtocolDB) - collections := store.NewCollections(node.ProtocolDB, transactions) - builder.transactions = transactions - builder.collections = collections - - return nil - }). Module("execution data datastore and blobstore", func(node *cmd.NodeConfig) error { var err error builder.ExecutionDatastoreManager, err = edstorage.CreateDatastoreManager( @@ -954,8 +944,8 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess notNil(builder.Storage.RegisterIndex), notNil(builder.Storage.Headers), notNil(builder.events), - notNil(builder.collections), - notNil(builder.transactions), + notNil(builder.Storage.Collections), + notNil(builder.Storage.Transactions), notNil(builder.lightTransactionResults), notNil(builder.scheduledTransactions), builder.RootChainID, @@ -1686,7 +1676,8 @@ func (builder *FlowAccessNodeBuilder) Initialize() error { builder.EnqueueNetworkInit() builder.AdminCommand("get-transactions", func(conf *cmd.NodeConfig) commands.AdminCommand { - return storageCommands.NewGetTransactionsCommand(conf.State, conf.Storage.Payloads, notNil(builder.collections)) + return storageCommands.NewGetTransactionsCommand(conf.State, conf.Storage.Payloads, + notNil(builder.Storage.Collections)) }) // if this is an access node that supports public followers, enqueue the public network @@ -1863,7 +1854,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.CollectionsToMarkFinalized, builder.CollectionsToMarkExecuted, builder.BlocksToMarkExecuted, - builder.collections, + builder.Storage.Collections, builder.Storage.Blocks, builder.BlockTransactions, ) @@ -1955,7 +1946,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { notNil(builder.collectionExecutedMetric), node.StorageLockMgr, builder.ProtocolDB, - notNil(builder.collections), + notNil(builder.Storage.Collections), ) return nil }). @@ -2138,8 +2129,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { HistoricalAccessNodes: notNil(builder.HistoricalAccessRPCs), Blocks: node.Storage.Blocks, Headers: node.Storage.Headers, - Collections: notNil(builder.collections), - Transactions: notNil(builder.transactions), + Collections: node.Storage.Collections, + Transactions: node.Storage.Transactions, ExecutionReceipts: node.Storage.Receipts, ExecutionResults: node.Storage.Results, TxResultErrorMessages: builder.transactionResultErrorMessages, // might be nil @@ -2555,7 +2546,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { node.State, node.Me, node.Storage.Blocks, - notNil(builder.collections), + node.Storage.Collections, node.Storage.Guarantees, builder.ProtocolDB, notNil(builder.blockCollectionIndexer), From ef5635bd754ebd7ca00511c91a3646770499324c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 13:40:28 -0800 Subject: [PATCH 027/126] fix benchmark tool --- integration/benchmark/cmd/manual/main.go | 46 ++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/integration/benchmark/cmd/manual/main.go b/integration/benchmark/cmd/manual/main.go index 1a5c7f8be6d..96a8204415d 100644 --- a/integration/benchmark/cmd/manual/main.go +++ b/integration/benchmark/cmd/manual/main.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/integration/benchmark" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/utils/unittest" ) @@ -59,8 +60,34 @@ func main() { } log = log.Level(lvl) + // Create context for metrics server + metricsCtx, metricsCancel := context.WithCancel(context.Background()) + + // Create SignalerContext and start metrics server server := metrics.NewServer(log, *metricport) + signalerCtx, errChan := irrecoverable.WithSignaler(metricsCtx) + go func() { + select { + case err, ok := <-errChan: + if !ok { + // Channel was closed without an error (shouldn't happen, but handle gracefully) + return + } + if err != nil { + log.Fatal().Err(err).Msg("metrics server encountered irrecoverable error") + } + case <-metricsCtx.Done(): + return + } + }() + server.Start(signalerCtx) <-server.Ready() + + // Ensure metrics server shuts down gracefully on exit + defer func() { + metricsCancel() + <-server.Done() + }() loaderMetrics := metrics.NewLoaderCollector() ctx, cancel := context.WithCancel(context.Background()) @@ -88,6 +115,7 @@ func main() { accessNodeAddrs := strings.Split(*accessNodes, ",") clients := make([]access.Client, 0, len(accessNodeAddrs)) for _, addr := range accessNodeAddrs { + log.Info().Str("addr", addr).Msg("connecting to access node") client, err := client.NewClient( addr, client.WithGRPCDialOptions( @@ -101,6 +129,18 @@ func main() { if err != nil { log.Fatal().Str("addr", addr).Err(err).Msgf("unable to initialize flow client") } + log.Info().Str("addr", addr).Msg("successfully connected to access node") + + // Test connectivity with a timeout + testCtx, testCancel := context.WithTimeout(ctx, 5*time.Second) + defer testCancel() + log.Info().Str("addr", addr).Msg("testing access node connectivity") + _, err = client.GetLatestBlockHeader(testCtx, true) + if err != nil { + log.Fatal().Str("addr", addr).Err(err).Msg("access node is not responding - is it running?") + } + log.Info().Str("addr", addr).Msg("access node is responding") + clients = append(clients, client) } @@ -120,6 +160,11 @@ func main() { statsLogger.Start() defer statsLogger.Stop() + log.Info(). + Int("number_of_accounts", int(maxTPS)**accountMultiplierFlag). + Uint("max_tps", maxTPS). + Msg("initializing load generator (this may take a while if creating many accounts)") + lg, err := benchmark.New( ctx, log, @@ -145,6 +190,7 @@ func main() { if err != nil { log.Fatal().Err(err).Msg("unable to create new cont load generator") } + log.Info().Msg("load generator initialized successfully") defer lg.Stop() for i, c := range loadCases { From 11f32a27a2640bf049ffde95adf3dc439d43eb4e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 14:43:09 -0800 Subject: [PATCH 028/126] add logs --- .../node_builder/access_node_builder.go | 2 ++ .../execution_data_index/processor.go | 15 +++++++++++ .../factory/execution_data_processor.go | 6 ++++- .../access/collection_sync/factory/fetcher.go | 8 +++--- .../collection_sync/fetcher/job_processor.go | 26 ++++++++++++++++++- 5 files changed, 52 insertions(+), 5 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b76bb962341..dea6a8231d2 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2230,6 +2230,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { // Create execution data processor executionDataProcessor, err := collection_syncfactory.CreateExecutionDataProcessor( + builder.Logger, notNil(builder.ExecutionDataCache), executionDataTracker, collectionIndexedHeight, @@ -2542,6 +2543,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { // Create fetcher and requesterEng requesterEng, fetcher, err := factory.CreateFetcher( node.Logger, + node.Metrics.Engine, node.EngineRegistry, node.State, node.Me, diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index 52b98d586b8..149d8ddbf1a 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -3,6 +3,8 @@ package execution_data_index import ( "fmt" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -12,6 +14,7 @@ import ( type ExecutionDataProcessor struct { component.Component + log zerolog.Logger newExecutionDataIndexed chan struct{} provider collection_sync.ExecutionDataProvider indexer collection_sync.BlockCollectionIndexer @@ -25,12 +28,14 @@ var _ collection_sync.ProgressReader = (*ExecutionDataProcessor)(nil) var _ component.Component = (*ExecutionDataProcessor)(nil) func NewExecutionDataProcessor( + log zerolog.Logger, provider collection_sync.ExecutionDataProvider, indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, metrics module.CollectionSyncMetrics, // optional metrics collector ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ + log: log.With().Str("component", "coll_sync_ed_processor").Logger(), newExecutionDataIndexed: make(chan struct{}, 1), provider: provider, indexer: indexer, @@ -105,6 +110,16 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, return } + // Log progress for each height with all relevant information + edp.log.Info(). + Uint64("indexed", height). + Uint64("lowest_missing", lowestMissing). + Uint64("highest_available", highestAvailableHeight). + Uint64("processed_count", height-lowestMissing+1). + Uint64("remaining_count", highestAvailableHeight-height). + Uint64("total_to_process", highestAvailableHeight-lowestMissing+1). + Msg("indexed execution data progress") + // Update metrics if available if edp.metrics != nil { edp.metrics.CollectionSyncedHeight(height) diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index b0fa6aa0621..1593f21f394 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -3,6 +3,8 @@ package factory import ( "fmt" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/engine/access/subscription/tracker" @@ -15,6 +17,7 @@ import ( // CreateExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. // // Parameters: +// - log: Logger for the component // - cache: Execution data cache for retrieving execution data by height // - executionDataTracker: Tracker for execution data that provides the highest available height // - processedHeight: Consumer progress for tracking processed heights @@ -27,6 +30,7 @@ import ( // // No errors are expected during normal operation. func CreateExecutionDataProcessor( + log zerolog.Logger, cache execution_data.ExecutionDataCache, executionDataTracker tracker.ExecutionDataTracker, processedHeight storage.ConsumerProgress, @@ -43,7 +47,7 @@ func CreateExecutionDataProcessor( } // Create the execution data processor - processor := execution_data_index.NewExecutionDataProcessor(executionDataProvider, indexer, processedHeightCounter, collectionSyncMetrics) + processor := execution_data_index.NewExecutionDataProcessor(log, executionDataProvider, indexer, processedHeightCounter, collectionSyncMetrics) return processor, nil } diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index e25c35aea3f..76fcd527e9e 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/state/protocol" @@ -34,6 +33,7 @@ type CreateFetcherConfig struct { // // Parameters: // - log: Logger for the component +// - engineMetrics: Metrics collector for the requester engine // - engineRegistry: Engine registry for creating the requester engine // - state: Protocol state // - me: Local node identity @@ -51,6 +51,7 @@ type CreateFetcherConfig struct { // No error returns are expected during normal operation. func CreateFetcher( log zerolog.Logger, + engineMetrics module.EngineMetrics, engineRegistry network.EngineRegistry, state protocol.State, me module.Local, @@ -67,7 +68,7 @@ func CreateFetcher( // Create requester engine for requesting collections requestEng, err := requester.New( log.With().Str("entity", "collection").Logger(), - metrics.NewNoopCollector(), // TODO: pass proper metrics if available + engineMetrics, engineRegistry, me, state, @@ -91,6 +92,7 @@ func CreateFetcher( // Create JobProcessor jobProcessor := fetcher.NewJobProcessor( + log.With().Str("component", "coll_sync_fetcher").Logger(), mcq, indexer, collectionRequester, @@ -106,7 +108,7 @@ func CreateFetcher( } // Forward collection to JobProcessor, which handles MCQ, indexing, and completion - err := jobProcessor.OnReceiveCollection(collection) + err := jobProcessor.OnReceiveCollection(originID, collection) if err != nil { log.Fatal().Err(err).Msg("failed to process received collection") return diff --git a/engine/access/collection_sync/fetcher/job_processor.go b/engine/access/collection_sync/fetcher/job_processor.go index fd64e550bc0..5618989af59 100644 --- a/engine/access/collection_sync/fetcher/job_processor.go +++ b/engine/access/collection_sync/fetcher/job_processor.go @@ -4,6 +4,8 @@ import ( "errors" "fmt" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -16,6 +18,7 @@ import ( // It orchestrates the flow: request → receive → index → complete. // TODO: rename to fetch_job_processor type JobProcessor struct { + log zerolog.Logger mcq collection_sync.MissingCollectionQueue indexer collection_sync.BlockCollectionIndexer requester collection_sync.CollectionRequester @@ -28,6 +31,7 @@ var _ collection_sync.JobProcessor = (*JobProcessor)(nil) // NewJobProcessor creates a new JobProcessor. // // Parameters: +// - log: Logger for the component // - mcq: MissingCollectionQueue for tracking missing collections and callbacks // - indexer: BlockCollectionIndexer for storing and indexing collections // - requester: CollectionRequester for requesting collections from the network @@ -37,6 +41,7 @@ var _ collection_sync.JobProcessor = (*JobProcessor)(nil) // // No error returns are expected during normal operation. func NewJobProcessor( + log zerolog.Logger, mcq collection_sync.MissingCollectionQueue, indexer collection_sync.BlockCollectionIndexer, requester collection_sync.CollectionRequester, @@ -44,6 +49,7 @@ func NewJobProcessor( collections storage.CollectionsReader, ) *JobProcessor { return &JobProcessor{ + log: log, mcq: mcq, indexer: indexer, requester: requester, @@ -112,9 +118,27 @@ func (jp *JobProcessor) ProcessJobConcurrently( // It passes the collection to MCQ, and if it completes a block, indexes it and marks it as done. // // No error returns are expected during normal operation. -func (jp *JobProcessor) OnReceiveCollection(collection *flow.Collection) error { +func (jp *JobProcessor) OnReceiveCollection(originID flow.Identifier, collection *flow.Collection) error { + collectionID := collection.ID() + // Pass collection to MCQ collections, height, complete := jp.mcq.OnReceivedCollection(collection) + + // Log collection receipt and whether it completes a block + if complete { + jp.log.Info(). + Hex("collection_id", collectionID[:]). + Hex("origin_id", originID[:]). + Uint64("block_height", height). + Int("collections_count", len(collections)). + Msg("received collection completing block to be indexed") + } else { + jp.log.Debug(). + Hex("collection_id", collectionID[:]). + Hex("origin_id", originID[:]). + Msg("received collection (block not yet complete)") + } + if !complete { // Block is not complete yet, nothing more to do return nil From 9ee10b7b7de107f6cf3afbd5c75e93cdeac72477 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 14:57:50 -0800 Subject: [PATCH 029/126] fix lint --- module/metrics/collection_sync.go | 1 - 1 file changed, 1 deletion(-) diff --git a/module/metrics/collection_sync.go b/module/metrics/collection_sync.go index 89e9efe0d32..37abc1cb6ff 100644 --- a/module/metrics/collection_sync.go +++ b/module/metrics/collection_sync.go @@ -38,4 +38,3 @@ func (c *CollectionSyncCollector) CollectionFetchedHeight(height uint64) { func (c *CollectionSyncCollector) CollectionSyncedHeight(height uint64) { c.collectionSyncedHeight.Set(float64(height)) } - From c4a867c238774a1c152bf09573397595ff54e968 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 15:09:55 -0800 Subject: [PATCH 030/126] add logs --- engine/access/collection_sync/fetcher/job_processor.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/engine/access/collection_sync/fetcher/job_processor.go b/engine/access/collection_sync/fetcher/job_processor.go index 5618989af59..eac20d2855f 100644 --- a/engine/access/collection_sync/fetcher/job_processor.go +++ b/engine/access/collection_sync/fetcher/job_processor.go @@ -49,7 +49,7 @@ func NewJobProcessor( collections storage.CollectionsReader, ) *JobProcessor { return &JobProcessor{ - log: log, + log: log.With().Str("component", "coll_fetcher").Logger(), mcq: mcq, indexer: indexer, requester: requester, @@ -75,6 +75,8 @@ func (jp *JobProcessor) ProcessJobConcurrently( } blockHeight := block.Height + jp.log.Debug().Uint64("block_height", blockHeight). + Msg("processing collection indexing job for finalized block") // Get missing collections for this block missingGuarantees, err := jp.getMissingCollections(blockHeight) From 8664c48ee31f8b85ead068e805ed70c57c94a777 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 13 Nov 2025 15:47:31 -0800 Subject: [PATCH 031/126] update metrics --- engine/access/collection_sync/fetcher/fetcher.go | 14 ++++++++------ .../collection_sync/fetcher/job_processor.go | 8 +++----- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/engine/access/collection_sync/fetcher/fetcher.go b/engine/access/collection_sync/fetcher/fetcher.go index 31c6f52abf4..d4e5c7aa8f6 100644 --- a/engine/access/collection_sync/fetcher/fetcher.go +++ b/engine/access/collection_sync/fetcher/fetcher.go @@ -91,14 +91,16 @@ func NewFetcher( metrics: metrics, } - // Set up post-notifier to update metrics when a job is done - if metrics != nil { - consumer.SetPostNotifier(func(jobID module.JobID) { - height := f.ProcessedHeight() - metrics.CollectionFetchedHeight(height) - }) + if metrics == nil { + return nil, fmt.Errorf("collection sync metrics not provided") } + // Set up post-notifier to update metrics when a job is done + consumer.SetPostNotifier(func(jobID module.JobID) { + height := f.ProcessedHeight() + metrics.CollectionFetchedHeight(height) + }) + return f, nil } diff --git a/engine/access/collection_sync/fetcher/job_processor.go b/engine/access/collection_sync/fetcher/job_processor.go index eac20d2855f..d2a0f1a1a34 100644 --- a/engine/access/collection_sync/fetcher/job_processor.go +++ b/engine/access/collection_sync/fetcher/job_processor.go @@ -76,7 +76,7 @@ func (jp *JobProcessor) ProcessJobConcurrently( blockHeight := block.Height jp.log.Debug().Uint64("block_height", blockHeight). - Msg("processing collection indexing job for finalized block") + Msg("processing collection fetching job for finalized block") // Get missing collections for this block missingGuarantees, err := jp.getMissingCollections(blockHeight) @@ -97,10 +97,8 @@ func (jp *JobProcessor) ProcessJobConcurrently( } // Enqueue missing collections with callback - callback := func() { - // When all collections are received and indexed, mark the job as done - done() - } + // When all collections are received and indexed, mark the job as done + callback := done err = jp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) if err != nil { From 7f25f253ee80598feed60c857d0024c1ffc4b7f9 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 08:59:08 -0800 Subject: [PATCH 032/126] update comments for collection sync metrics --- module/metrics.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/module/metrics.go b/module/metrics.go index 1a865f0dcd9..e13b8c3ad5a 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -1215,9 +1215,12 @@ type CollectionExecutedMetric interface { } type CollectionSyncMetrics interface { - // CollectionFetchedHeight reports the highest block height for which collections have been fetched + // CollectionFetchedHeight reports the highest consecutive block height for + // which collections have been fetched from collection nodes CollectionFetchedHeight(height uint64) - // CollectionSyncedHeight reports the highest block height for which collections have been synced from execution data + + // CollectionSyncedHeight reports the highest consecutive block height for + // which collections have been synced from execution data CollectionSyncedHeight(height uint64) } From ee66ca9fd21f351ebc896957c90f7da990b060a3 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 09:18:21 -0800 Subject: [PATCH 033/126] update comments --- cmd/access/node_builder/access_node_builder.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index dea6a8231d2..a79b89f3613 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2532,10 +2532,19 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { if builder.executionDataSyncEnabled { // skip if execution data sync is enabled + // because the execution data contains the collections, so no need to fetch them separately. + // otherwise, if both fetching and syncing are enabled, they might slow down each other, + // because the database operation requires locking. return &module.NoopReadyDoneAware{}, nil } - // fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + // TODO (leo): switch to module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // to implement hybrid sync mode in the future + // in the hybrid sync mode, the fetcher will store its progress under a different key, + // and and only active if either of the following condition is met: + // 1) execution data sync is disabled + // 2) execution data sync is enabled and exectuion data sync height is far behind + // the latest finalized height and the execution data sync is not updating. fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) From 075e7cd3488d4c994354d8be27a7ec5c429e7b2b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 10:12:22 -0800 Subject: [PATCH 034/126] rename job processor to block processor --- .../access/collection_sync/collection_sync.go | 5 +- .../access/collection_sync/factory/fetcher.go | 12 ++-- .../{job_processor.go => block_processor.go} | 55 ++++++++----------- .../fetcher/{fetcher.go => engine.go} | 38 ++++++++++--- module/jobqueue/finalized_block_reader.go | 11 +++- module/jobqueue/sealed_header_reader.go | 10 +++- 6 files changed, 76 insertions(+), 55 deletions(-) rename engine/access/collection_sync/fetcher/{job_processor.go => block_processor.go} (77%) rename engine/access/collection_sync/fetcher/{fetcher.go => engine.go} (76%) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 9950ce9da00..9756c7a2e5e 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -4,7 +4,6 @@ import ( "context" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" ) @@ -35,8 +34,8 @@ type BlockCollectionIndexer interface { } // Implements the job lifecycle for a single block height. -type JobProcessor interface { - ProcessJobConcurrently(ctx irrecoverable.SignalerContext, job module.Job, done func()) error +type BlockProcessor interface { + FetchCollections(ctx irrecoverable.SignalerContext, block *flow.Block, done func()) error } // Fetcher is a component that consumes finalized block jobs and processes them diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 76fcd527e9e..27fb52484a2 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -46,7 +46,7 @@ type CreateFetcherConfig struct { // - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress // - config: Configuration for the fetcher // -// Returns both the Fetcher and JobProcessor so they can be reused in other components. +// Returns both the Fetcher and BlockProcessor so they can be reused in other components. // // No error returns are expected during normal operation. func CreateFetcher( @@ -90,8 +90,8 @@ func CreateFetcher( guarantees, ) - // Create JobProcessor - jobProcessor := fetcher.NewJobProcessor( + // Create BlockProcessor + blockProcessor := fetcher.NewBlockProcessor( log.With().Str("component", "coll_sync_fetcher").Logger(), mcq, indexer, @@ -107,8 +107,8 @@ func CreateFetcher( return } - // Forward collection to JobProcessor, which handles MCQ, indexing, and completion - err := jobProcessor.OnReceiveCollection(originID, collection) + // Forward collection to BlockProcessor, which handles MCQ, indexing, and completion + err := blockProcessor.OnReceiveCollection(originID, collection) if err != nil { log.Fatal().Err(err).Msg("failed to process received collection") return @@ -118,7 +118,7 @@ func CreateFetcher( // Create Fetcher collectionFetcher, err := fetcher.NewFetcher( log, - jobProcessor, + blockProcessor, processedFinalizedBlockHeight, state, blocks, diff --git a/engine/access/collection_sync/fetcher/job_processor.go b/engine/access/collection_sync/fetcher/block_processor.go similarity index 77% rename from engine/access/collection_sync/fetcher/job_processor.go rename to engine/access/collection_sync/fetcher/block_processor.go index d2a0f1a1a34..ac2501ccd65 100644 --- a/engine/access/collection_sync/fetcher/job_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -8,16 +8,13 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/jobqueue" "github.com/onflow/flow-go/storage" ) -// JobProcessor implements the job lifecycle for collection indexing. +// BlockProcessor implements the job lifecycle for collection indexing. // It orchestrates the flow: request → receive → index → complete. -// TODO: rename to fetch_job_processor -type JobProcessor struct { +type BlockProcessor struct { log zerolog.Logger mcq collection_sync.MissingCollectionQueue indexer collection_sync.BlockCollectionIndexer @@ -26,9 +23,9 @@ type JobProcessor struct { collections storage.CollectionsReader } -var _ collection_sync.JobProcessor = (*JobProcessor)(nil) +var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) -// NewJobProcessor creates a new JobProcessor. +// NewBlockProcessor creates a new BlockProcessor. // // Parameters: // - log: Logger for the component @@ -40,15 +37,15 @@ var _ collection_sync.JobProcessor = (*JobProcessor)(nil) // Set to a very large value to effectively disable fetching and rely only on EDI. // // No error returns are expected during normal operation. -func NewJobProcessor( +func NewBlockProcessor( log zerolog.Logger, mcq collection_sync.MissingCollectionQueue, indexer collection_sync.BlockCollectionIndexer, requester collection_sync.CollectionRequester, blocks storage.Blocks, collections storage.CollectionsReader, -) *JobProcessor { - return &JobProcessor{ +) *BlockProcessor { + return &BlockProcessor{ log: log.With().Str("component", "coll_fetcher").Logger(), mcq: mcq, indexer: indexer, @@ -58,28 +55,22 @@ func NewJobProcessor( } } -// ProcessJob processes a job for a finalized block. +// FetchCollections processes a block for collection fetching. // It checks if the block is already indexed, and if not, enqueues missing collections // and optionally requests them based on EDI lag. // // No error returns are expected during normal operation. -func (jp *JobProcessor) ProcessJobConcurrently( +func (bp *BlockProcessor) FetchCollections( ctx irrecoverable.SignalerContext, - job module.Job, + block *flow.Block, done func(), ) error { - // Convert job to block - block, err := jobqueue.JobToBlock(job) - if err != nil { - return fmt.Errorf("could not convert job to block: %w", err) - } - blockHeight := block.Height - jp.log.Debug().Uint64("block_height", blockHeight). + bp.log.Debug().Uint64("block_height", blockHeight). Msg("processing collection fetching job for finalized block") // Get missing collections for this block - missingGuarantees, err := jp.getMissingCollections(blockHeight) + missingGuarantees, err := bp.getMissingCollections(blockHeight) if err != nil { return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) } @@ -100,13 +91,13 @@ func (jp *JobProcessor) ProcessJobConcurrently( // When all collections are received and indexed, mark the job as done callback := done - err = jp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + err = bp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) if err != nil { return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) } // Request collections from collection nodes - err = jp.requester.RequestCollections(collectionIDs) + err = bp.requester.RequestCollections(collectionIDs) if err != nil { return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) } @@ -118,22 +109,22 @@ func (jp *JobProcessor) ProcessJobConcurrently( // It passes the collection to MCQ, and if it completes a block, indexes it and marks it as done. // // No error returns are expected during normal operation. -func (jp *JobProcessor) OnReceiveCollection(originID flow.Identifier, collection *flow.Collection) error { +func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collection *flow.Collection) error { collectionID := collection.ID() // Pass collection to MCQ - collections, height, complete := jp.mcq.OnReceivedCollection(collection) + collections, height, complete := bp.mcq.OnReceivedCollection(collection) // Log collection receipt and whether it completes a block if complete { - jp.log.Info(). + bp.log.Info(). Hex("collection_id", collectionID[:]). Hex("origin_id", originID[:]). Uint64("block_height", height). Int("collections_count", len(collections)). Msg("received collection completing block to be indexed") } else { - jp.log.Debug(). + bp.log.Debug(). Hex("collection_id", collectionID[:]). Hex("origin_id", originID[:]). Msg("received collection (block not yet complete)") @@ -145,21 +136,21 @@ func (jp *JobProcessor) OnReceiveCollection(originID flow.Identifier, collection } // Block became complete, index it - err := jp.indexer.IndexCollectionsForBlock(height, collections) + err := bp.indexer.IndexCollectionsForBlock(height, collections) if err != nil { return fmt.Errorf("failed to index collections for block height %d: %w", height, err) } // Mark the block as indexed (this invokes the callback) - jp.mcq.OnIndexedForBlock(height) + bp.mcq.OnIndexedForBlock(height) return nil } // getMissingCollections retrieves the block and returns collection guarantees that are missing. // Only collections that are not already in storage are returned. -func (jp *JobProcessor) getMissingCollections(blockHeight uint64) ([]*flow.CollectionGuarantee, error) { - block, err := jp.blocks.ByHeight(blockHeight) +func (bp *BlockProcessor) getMissingCollections(blockHeight uint64) ([]*flow.CollectionGuarantee, error) { + block, err := bp.blocks.ByHeight(blockHeight) if err != nil { return nil, fmt.Errorf("failed to retrieve block at height %d: %w", blockHeight, err) } @@ -167,7 +158,7 @@ func (jp *JobProcessor) getMissingCollections(blockHeight uint64) ([]*flow.Colle var missingGuarantees []*flow.CollectionGuarantee for _, guarantee := range block.Payload.Guarantees { // Check if collection already exists in storage - _, err := jp.collections.LightByID(guarantee.CollectionID) + _, err := bp.collections.LightByID(guarantee.CollectionID) if err != nil { if errors.Is(err, storage.ErrNotFound) { // Collection is missing diff --git a/engine/access/collection_sync/fetcher/fetcher.go b/engine/access/collection_sync/fetcher/engine.go similarity index 76% rename from engine/access/collection_sync/fetcher/fetcher.go rename to engine/access/collection_sync/fetcher/engine.go index d4e5c7aa8f6..d69431418ed 100644 --- a/engine/access/collection_sync/fetcher/fetcher.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -2,6 +2,7 @@ package fetcher import ( "fmt" + "sync" "github.com/rs/zerolog" @@ -20,10 +21,12 @@ import ( type Fetcher struct { component.Component - consumer *jobqueue.ComponentConsumer - jobProcessor collection_sync.JobProcessor - workSignal engine.Notifier - metrics module.CollectionSyncMetrics + consumer *jobqueue.ComponentConsumer + jobProcessor collection_sync.BlockProcessor + workSignal engine.Notifier + metrics module.CollectionSyncMetrics + lastReportedMu sync.Mutex + lastReportedHeight uint64 } var _ collection_sync.Fetcher = (*Fetcher)(nil) @@ -34,7 +37,7 @@ var _ component.Component = (*Fetcher)(nil) // // Parameters: // - log: Logger for the component -// - jobProcessor: JobProcessor implementation for processing collection indexing jobs +// - jobProcessor: BlockProcessor implementation for processing collection indexing jobs // - progressInitializer: Initializer for tracking processed block heights // - state: Protocol state for reading finalized block information // - blocks: Blocks storage for reading blocks by height @@ -45,7 +48,7 @@ var _ component.Component = (*Fetcher)(nil) // No error returns are expected during normal operation. func NewFetcher( log zerolog.Logger, - jobProcessor collection_sync.JobProcessor, + jobProcessor collection_sync.BlockProcessor, progressInitializer storage.ConsumerProgressInitializer, state protocol.State, blocks storage.Blocks, @@ -59,11 +62,18 @@ func NewFetcher( defaultIndex := state.Params().FinalizedRoot().Height // Create a Jobs instance that reads finalized blocks by height + // each job is a finalized block jobs := jobqueue.NewFinalizedBlockReader(state, blocks) - // Create an adapter function that wraps the JobProcessor interface + // Create an adapter function that wraps the BlockProcessor interface processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { - err := jobProcessor.ProcessJobConcurrently(ctx, job, done) + // Convert job to block + block, err := jobs.ConvertJobToBlock(job) + if err != nil { + ctx.Throw(fmt.Errorf("could not convert job to block: %w", err)) + return + } + err = jobProcessor.FetchCollections(ctx, block, done) if err != nil { ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) } @@ -96,9 +106,19 @@ func NewFetcher( } // Set up post-notifier to update metrics when a job is done + // Only update metrics when the processed height actually changes, since processedIndex + // only advances when consecutive jobs complete, not on every individual job completion. consumer.SetPostNotifier(func(jobID module.JobID) { height := f.ProcessedHeight() - metrics.CollectionFetchedHeight(height) + + f.lastReportedMu.Lock() + if height > f.lastReportedHeight { + f.lastReportedHeight = height + f.lastReportedMu.Unlock() + metrics.CollectionFetchedHeight(height) + } else { + f.lastReportedMu.Unlock() + } }) return f, nil diff --git a/module/jobqueue/finalized_block_reader.go b/module/jobqueue/finalized_block_reader.go index 952fef7555c..e84dd31ed0e 100644 --- a/module/jobqueue/finalized_block_reader.go +++ b/module/jobqueue/finalized_block_reader.go @@ -27,7 +27,7 @@ func NewFinalizedBlockReader(state protocol.State, blocks storage.Blocks) *Final // AtIndex returns the block job at the given index. // The block job at an index is just the finalized block at that index (i.e., height). -func (r FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { +func (r *FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { block, err := r.blockByHeight(index) if err != nil { return nil, fmt.Errorf("could not get block by index %v: %w", index, err) @@ -35,8 +35,13 @@ func (r FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { return BlockToJob(block), nil } +// ConvertJobToBlock converts a job to a block. +func (r *FinalizedBlockReader) ConvertJobToBlock(job module.Job) (*flow.Block, error) { + return JobToBlock(job) +} + // blockByHeight returns the block at the given height. -func (r FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) { +func (r *FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) { block, err := r.blocks.ByHeight(height) if err != nil { return nil, fmt.Errorf("could not get block by height %d: %w", height, err) @@ -46,7 +51,7 @@ func (r FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) } // Head returns the last finalized height as job index. -func (r FinalizedBlockReader) Head() (uint64, error) { +func (r *FinalizedBlockReader) Head() (uint64, error) { header, err := r.state.Final().Head() if err != nil { return 0, fmt.Errorf("could not get header of last finalized block: %w", err) diff --git a/module/jobqueue/sealed_header_reader.go b/module/jobqueue/sealed_header_reader.go index 4a12f76f0cc..e534e657f96 100644 --- a/module/jobqueue/sealed_header_reader.go +++ b/module/jobqueue/sealed_header_reader.go @@ -3,6 +3,7 @@ package jobqueue import ( "fmt" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" @@ -28,7 +29,7 @@ func NewSealedBlockHeaderReader(state protocol.State, headers storage.Headers) * // The block header job at an index is just the finalized block header at that index (i.e., height). // Error returns: // - storage.ErrNotFound if the provided index is not sealed -func (r SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { +func (r *SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { sealed, err := r.Head() if err != nil { return nil, fmt.Errorf("could not get last sealed block height: %w", err) @@ -48,8 +49,13 @@ func (r SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { return BlockHeaderToJob(header), nil } +// ConvertJobToBlockHeader converts a job to a block header. +func (r *SealedBlockHeaderReader) ConvertJobToBlockHeader(job module.Job) (*flow.Header, error) { + return JobToBlockHeader(job) +} + // Head returns the last sealed height as job index. -func (r SealedBlockHeaderReader) Head() (uint64, error) { +func (r *SealedBlockHeaderReader) Head() (uint64, error) { header, err := r.state.Sealed().Head() if err != nil { return 0, fmt.Errorf("could not get header of last sealed block: %w", err) From 4770fb7dabaf7079fb24111eccae3921e654db45 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 10:20:02 -0800 Subject: [PATCH 035/126] move indexer --- cmd/access/node_builder/access_node_builder.go | 4 ++-- .../block_collection_indexer.go => indexer/indexer.go} | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) rename engine/access/collection_sync/{fetcher/block_collection_indexer.go => indexer/indexer.go} (99%) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index a79b89f3613..cf9f71e4a71 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -42,7 +42,7 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/factory" collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" - "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" + collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" "github.com/onflow/flow-go/engine/access/finalized_indexer" "github.com/onflow/flow-go/engine/access/index" "github.com/onflow/flow-go/engine/access/ingest_receipt" @@ -1942,7 +1942,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil }). Module("block collection indexer", func(node *cmd.NodeConfig) error { - builder.blockCollectionIndexer = fetcher.NewBlockCollectionIndexer( + builder.blockCollectionIndexer = collsyncindexer.NewBlockCollectionIndexer( notNil(builder.collectionExecutedMetric), node.StorageLockMgr, builder.ProtocolDB, diff --git a/engine/access/collection_sync/fetcher/block_collection_indexer.go b/engine/access/collection_sync/indexer/indexer.go similarity index 99% rename from engine/access/collection_sync/fetcher/block_collection_indexer.go rename to engine/access/collection_sync/indexer/indexer.go index 1ae72770566..d33aaa9b94b 100644 --- a/engine/access/collection_sync/fetcher/block_collection_indexer.go +++ b/engine/access/collection_sync/indexer/indexer.go @@ -1,4 +1,4 @@ -package fetcher +package indexer import ( "fmt" From 8feb65e22b5ea2191e8270929fd9a5ff3477aff0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 10:53:35 -0800 Subject: [PATCH 036/126] update comments --- cmd/access/node_builder/access_node_builder.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cf9f71e4a71..5dbc0248a18 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2235,7 +2235,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { executionDataTracker, collectionIndexedHeight, notNil(builder.blockCollectionIndexer), - builder.CollectionSyncMetrics, + notNil(builder.CollectionSyncMetrics), ) if err != nil { return nil, fmt.Errorf("could not create execution data processor: %w", err) @@ -2563,7 +2563,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { notNil(builder.blockCollectionIndexer), fetchAndIndexedCollectionsBlockHeight, notNil(builder.collectionExecutedMetric), - builder.CollectionSyncMetrics, + notNil(builder.CollectionSyncMetrics), collection_syncfactory.CreateFetcherConfig{ MaxProcessing: 10, // TODO: make configurable MaxSearchAhead: 20, // TODO: make configurable From 2feb1700e2761888f83341538e91ec95b603a19f Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 10:55:12 -0800 Subject: [PATCH 037/126] update metrics --- engine/access/collection_sync/collection_sync.go | 3 +++ .../execution_data_index/processor.go | 12 ++++-------- engine/access/collection_sync/factory/fetcher.go | 2 +- engine/access/collection_sync/fetcher/engine.go | 10 +--------- 4 files changed, 9 insertions(+), 18 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 9756c7a2e5e..8f93dcf353c 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -30,6 +30,7 @@ type CollectionRequester interface { // BlockCollectionIndexer stores and indexes collections for a given block height. type BlockCollectionIndexer interface { // IndexCollectionsForBlock stores and indexes collections for a given block height. + // No error is exepcted during normal operation. IndexCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error } @@ -54,10 +55,12 @@ type ExecutionDataProvider interface { GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) } +// ExecutionDataProcessor processes execution data when new execution data is available. type ExecutionDataProcessor interface { OnNewExectuionData() } +// ProgressReader provides the current progress of collection fetching/indexing. type ProgressReader interface { ProcessedHeight() uint64 } diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index 149d8ddbf1a..be3c8204c72 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -18,9 +18,9 @@ type ExecutionDataProcessor struct { newExecutionDataIndexed chan struct{} provider collection_sync.ExecutionDataProvider indexer collection_sync.BlockCollectionIndexer + metrics module.CollectionSyncMetrics // state processedHeight *counters.PersistentStrictMonotonicCounter - metrics module.CollectionSyncMetrics } var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) @@ -32,15 +32,15 @@ func NewExecutionDataProcessor( provider collection_sync.ExecutionDataProvider, indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, - metrics module.CollectionSyncMetrics, // optional metrics collector + metrics module.CollectionSyncMetrics, ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ log: log.With().Str("component", "coll_sync_ed_processor").Logger(), newExecutionDataIndexed: make(chan struct{}, 1), provider: provider, indexer: indexer, - processedHeight: processedHeight, metrics: metrics, + processedHeight: processedHeight, } // Initialize the channel so that even if no new execution data comes in, @@ -97,7 +97,6 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, // And make sure reading the collections's lowest height is cheap operation (only hitting RW lock) err = edp.indexer.IndexCollectionsForBlock(height, collections) - // TODO: handle already exists if err != nil { ctx.Throw(fmt.Errorf("failed to index collections for block height %d: %w", height, err)) return @@ -120,10 +119,7 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, Uint64("total_to_process", highestAvailableHeight-lowestMissing+1). Msg("indexed execution data progress") - // Update metrics if available - if edp.metrics != nil { - edp.metrics.CollectionSyncedHeight(height) - } + edp.metrics.CollectionSyncedHeight(height) } } } diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 27fb52484a2..b34791995ce 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -62,7 +62,7 @@ func CreateFetcher( indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, collectionExecutedMetric module.CollectionExecutedMetric, - collectionSyncMetrics module.CollectionSyncMetrics, // optional metrics collector + collectionSyncMetrics module.CollectionSyncMetrics, config CreateFetcherConfig, ) (*requester.Engine, collection_sync.Fetcher, error) { // Create requester engine for requesting collections diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index d69431418ed..7270c25576b 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -110,15 +110,7 @@ func NewFetcher( // only advances when consecutive jobs complete, not on every individual job completion. consumer.SetPostNotifier(func(jobID module.JobID) { height := f.ProcessedHeight() - - f.lastReportedMu.Lock() - if height > f.lastReportedHeight { - f.lastReportedHeight = height - f.lastReportedMu.Unlock() - metrics.CollectionFetchedHeight(height) - } else { - f.lastReportedMu.Unlock() - } + metrics.CollectionFetchedHeight(height) }) return f, nil From 232a60290a9ef1aa82a9069ae893dab0e66115fa Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 11:19:50 -0800 Subject: [PATCH 038/126] refactor missing collections queue --- .../access/collection_sync/collection_sync.go | 2 +- .../fetcher/block_processor.go | 14 ++++++---- .../fetcher/missing_collection_queue.go | 28 +++++++++---------- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 8f93dcf353c..f6d57c8e4dc 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -11,7 +11,7 @@ import ( // Tracks missing collections per height and invokes job callbacks when complete. type MissingCollectionQueue interface { EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error - OnIndexedForBlock(blockHeight uint64) bool // mark done (post‑indexing), returns true if height existed + OnIndexedForBlock(blockHeight uint64) (func(), bool) // On receipt of a collection, MCQ updates internal state and, if a block // just became complete, returns: (collections, height, true). diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index ac2501ccd65..fcc4c2fdbb8 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -87,11 +87,12 @@ func (bp *BlockProcessor) FetchCollections( collectionIDs[i] = guarantee.CollectionID } - // Enqueue missing collections with callback + // Enqueue missing collections with notifyJobCompletion // When all collections are received and indexed, mark the job as done - callback := done + notifyJobCompletion := done - err = bp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + // the notifyJobCompletion callback will be returned in the OnReceiveCollection method + err = bp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, notifyJobCompletion) if err != nil { return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) } @@ -141,8 +142,11 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti return fmt.Errorf("failed to index collections for block height %d: %w", height, err) } - // Mark the block as indexed (this invokes the callback) - bp.mcq.OnIndexedForBlock(height) + // Mark the block as indexed + notifyJobCompletion, ok := bp.mcq.OnIndexedForBlock(height) + if ok { + notifyJobCompletion() + } return nil } diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 90f253075db..0019351b757 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -169,23 +169,25 @@ func (mcq *MissingCollectionQueue) IsHeightQueued(height uint64) bool { return exists } -// OnIndexedForBlock notifies the queue that a block height has been indexed. -// This invokes the callback for that block height and removes it from tracking. +// OnIndexedForBlock notifies the queue that a block height has been indexed, +// removes that block height from tracking, and return the callback for caller to +// invoke. // -// Returns true if the height existed and was processed, false if the height was not tracked. -func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) bool { +// Returns: +// (callback, true) if the height existed and was processed; +// (nil, false) if the height was not tracked. +// +// Note, caller should invoke the returned callback if not nil. +func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func(), bool) { mcq.mu.Lock() + defer mcq.mu.Unlock() jobState, exists := mcq.blockJobs[blockHeight] if !exists { - // Block was not tracked or already completed (callback already called and job removed). - mcq.mu.Unlock() - return false + // Block was not tracked or already completed + return nil, false } - // Get the callback before removing the job. - callback := jobState.callback - // Clean up all collection-to-height mappings for collections belonging to this block. // Clean up from missing collections. for collectionID := range jobState.missingCollections { @@ -204,9 +206,5 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) bool { // This ensures the height is removed from tracking once the callback is invoked. delete(mcq.blockJobs, blockHeight) - // Release the lock before calling the callback to avoid holding it during callback execution. - mcq.mu.Unlock() - callback() - // Note: We manually unlocked above, so we don't use defer here to avoid double-unlock. - return true + return jobState.callback, true } From f5bda6262b0d69dc62ae2d21d133738e62f850e2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 11:45:11 -0800 Subject: [PATCH 039/126] add hotstuff distributor interface --- consensus/hotstuff/distributor.go | 8 ++++++++ .../notifications/pubsub/finalization_distributor.go | 1 + 2 files changed, 9 insertions(+) create mode 100644 consensus/hotstuff/distributor.go diff --git a/consensus/hotstuff/distributor.go b/consensus/hotstuff/distributor.go new file mode 100644 index 00000000000..34544a54697 --- /dev/null +++ b/consensus/hotstuff/distributor.go @@ -0,0 +1,8 @@ +package hotstuff + +import "github.com/onflow/flow-go/consensus/hotstuff/model" + +type Distributor interface { + AddOnBlockFinalizedConsumer(consumer func(block *model.Block)) + AddOnBlockIncorporatedConsumer(consumer func(block *model.Block)) +} diff --git a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go index e351575c122..195632d433e 100644 --- a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go +++ b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go @@ -22,6 +22,7 @@ type FinalizationDistributor struct { } var _ hotstuff.FinalizationConsumer = (*FinalizationDistributor)(nil) +var _ hotstuff.Distributor = (*FinalizationDistributor)(nil) func NewFinalizationDistributor() *FinalizationDistributor { return &FinalizationDistributor{} From 25e54a81f3baa4e46666e697dea1e68356ed8585 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 14 Nov 2025 11:45:29 -0800 Subject: [PATCH 040/126] access node builder should subscribe for finalization events --- cmd/access/node_builder/access_node_builder.go | 4 ++-- .../access/collection_sync/factory/fetcher.go | 7 +++++++ engine/access/collection_sync/fetcher/engine.go | 4 ++-- engine/access/finalized_indexer/processor.go | 17 ++++++++--------- 4 files changed, 19 insertions(+), 13 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5dbc0248a18..0a6344ad193 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2515,14 +2515,13 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. node.State, node.Storage.Blocks, processedFinalizedBlockHeight, + builder.FollowerDistributor, notNil(builder.collectionExecutedMetric), ) if err != nil { return nil, fmt.Errorf("could not create finalized block processor: %w", err) } - builder.FollowerDistributor.AddOnBlockFinalizedConsumer(finalizedBlockProcessor.OnBlockFinalized) - return finalizedBlockProcessor, nil } } @@ -2562,6 +2561,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder.ProtocolDB, notNil(builder.blockCollectionIndexer), fetchAndIndexedCollectionsBlockHeight, + builder.FollowerDistributor, notNil(builder.collectionExecutedMetric), notNil(builder.CollectionSyncMetrics), collection_syncfactory.CreateFetcherConfig{ diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index b34791995ce..fc192f26d70 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -5,6 +5,8 @@ import ( "github.com/rs/zerolog" + "github.com/onflow/flow-go/consensus/hotstuff" + "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" "github.com/onflow/flow-go/engine/common/requester" @@ -61,6 +63,7 @@ func CreateFetcher( db storage.DB, indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, + distributor hotstuff.Distributor, collectionExecutedMetric module.CollectionExecutedMetric, collectionSyncMetrics module.CollectionSyncMetrics, config CreateFetcherConfig, @@ -130,5 +133,9 @@ func CreateFetcher( return nil, nil, fmt.Errorf("could not create fetcher: %w", err) } + distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { + collectionFetcher.OnFinalizedBlock() + }) + return requestEng, collectionFetcher, nil } diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index 7270c25576b..e542596a1b5 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -73,6 +73,7 @@ func NewFetcher( ctx.Throw(fmt.Errorf("could not convert job to block: %w", err)) return } + err = jobProcessor.FetchCollections(ctx, block, done) if err != nil { ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) @@ -109,8 +110,7 @@ func NewFetcher( // Only update metrics when the processed height actually changes, since processedIndex // only advances when consecutive jobs complete, not on every individual job completion. consumer.SetPostNotifier(func(jobID module.JobID) { - height := f.ProcessedHeight() - metrics.CollectionFetchedHeight(height) + metrics.CollectionFetchedHeight(f.ProcessedHeight()) }) return f, nil diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index ed605e09ab7..9e9d2e33f47 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -5,6 +5,7 @@ import ( "github.com/rs/zerolog" + "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" @@ -45,9 +46,8 @@ type FinalizedBlockProcessor struct { log zerolog.Logger component.Component - consumer *jobqueue.ComponentConsumer - blockFinalizedNotifier engine.Notifier - blocks storage.Blocks + consumer *jobqueue.ComponentConsumer + blocks storage.Blocks collectionExecutedMetric module.CollectionExecutedMetric } @@ -61,6 +61,7 @@ func NewFinalizedBlockProcessor( state protocol.State, blocks storage.Blocks, finalizedProcessedHeight storage.ConsumerProgressInitializer, + distributor hotstuff.Distributor, collectionExecutedMetric module.CollectionExecutedMetric, ) (*FinalizedBlockProcessor, error) { reader := jobqueue.NewFinalizedBlockReader(state, blocks) @@ -73,7 +74,6 @@ func NewFinalizedBlockProcessor( processor := &FinalizedBlockProcessor{ log: log, blocks: blocks, - blockFinalizedNotifier: blockFinalizedNotifier, collectionExecutedMetric: collectionExecutedMetric, } @@ -91,6 +91,10 @@ func NewFinalizedBlockProcessor( return nil, fmt.Errorf("error creating finalized block jobqueue: %w", err) } + distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { + blockFinalizedNotifier.Notify() + }) + // Build component manager with worker loop cm := component.NewComponentManagerBuilder(). AddWorker(processor.workerLoop). @@ -101,11 +105,6 @@ func NewFinalizedBlockProcessor( return processor, nil } -// Notify notifies the processor that a new finalized block is available for processing. -func (p *FinalizedBlockProcessor) OnBlockFinalized(_ *model.Block) { - p.blockFinalizedNotifier.Notify() -} - // StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { p.consumer.Start(ctx) From 75c7ed3e377c2c2e56d0f2f88deb19882744681c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sun, 16 Nov 2025 11:19:28 -0800 Subject: [PATCH 041/126] use flag to config collectoin fetcher configs --- .../node_builder/access_node_builder.go | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0a6344ad193..569a1a77d29 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -40,7 +40,6 @@ import ( recovery "github.com/onflow/flow-go/consensus/recovery/protocol" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/collection_sync/factory" collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" "github.com/onflow/flow-go/engine/access/finalized_indexer" @@ -187,6 +186,8 @@ type AccessNodeConfig struct { stopControlEnabled bool registerDBPruneThreshold uint64 scheduledCallbacksEnabled bool + collectionFetcherMaxProcessing uint64 + collectionFetcherMaxSearchAhead uint64 } type PublicNetworkConfig struct { @@ -295,6 +296,8 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { stopControlEnabled: false, registerDBPruneThreshold: 0, scheduledCallbacksEnabled: fvm.DefaultScheduledCallbacksEnabled, + collectionFetcherMaxProcessing: 10, + collectionFetcherMaxSearchAhead: 20, } } @@ -368,7 +371,7 @@ type FlowAccessNodeBuilder struct { // for tx status deriver to know about the highest full block (a block with all collections synced) // backed by either collection fetcher to execution data syncing - lastFullBlockHeight *factory.ProgressReader + lastFullBlockHeight *collection_syncfactory.ProgressReader // grpc servers secureGrpcServer *grpcserver.GrpcServer @@ -1203,6 +1206,14 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "", defaultConfig.rpcConf.CollectionAddr, "the address (of the collection node) to send transactions to") + flags.Uint64Var(&builder.collectionFetcherMaxProcessing, + "collection-fetcher-max-processing", + defaultConfig.collectionFetcherMaxProcessing, + "maximum number of collection fetcher requests to process concurrently") + flags.Uint64Var(&builder.collectionFetcherMaxSearchAhead, + "collection-fetcher-max-search-ahead", + defaultConfig.collectionFetcherMaxSearchAhead, + "maximum number of blocks to search ahead when fetching collections") flags.StringVarP(&builder.ExecutionNodeAddress, "script-addr", "s", @@ -1964,7 +1975,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher - builder.lastFullBlockHeight = factory.NewProgressReader(lastProgress) + builder.lastFullBlockHeight = collection_syncfactory.NewProgressReader(lastProgress) collectionIndexedHeight = progress @@ -2549,7 +2560,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { // skip if execution data sync is enabled // Create fetcher and requesterEng - requesterEng, fetcher, err := factory.CreateFetcher( + requesterEng, fetcher, err := collection_syncfactory.CreateFetcher( node.Logger, node.Metrics.Engine, node.EngineRegistry, @@ -2565,8 +2576,8 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { notNil(builder.collectionExecutedMetric), notNil(builder.CollectionSyncMetrics), collection_syncfactory.CreateFetcherConfig{ - MaxProcessing: 10, // TODO: make configurable - MaxSearchAhead: 20, // TODO: make configurable + MaxProcessing: builder.collectionFetcherMaxProcessing, + MaxSearchAhead: builder.collectionFetcherMaxSearchAhead, }, ) From a9348ee0042bde011554e6c349ac422e8cebedb0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 08:45:00 -0800 Subject: [PATCH 042/126] add missing collection queue metrics --- .../access/collection_sync/collection_sync.go | 5 +++++ .../fetcher/block_processor.go | 5 +++++ .../access/collection_sync/fetcher/engine.go | 20 +++++++++---------- .../fetcher/missing_collection_queue.go | 9 +++++++++ module/metrics.go | 3 +++ module/metrics/collection_sync.go | 15 ++++++++++++-- 6 files changed, 45 insertions(+), 12 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index f6d57c8e4dc..88aaff55132 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -20,6 +20,9 @@ type MissingCollectionQueue interface { // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). IsHeightQueued(height uint64) bool + + // Size returns the number of missing collections currently in the queue. + Size() uint } // Requests collections by their IDs. @@ -37,6 +40,8 @@ type BlockCollectionIndexer interface { // Implements the job lifecycle for a single block height. type BlockProcessor interface { FetchCollections(ctx irrecoverable.SignalerContext, block *flow.Block, done func()) error + // MissingCollectionQueueSize returns the number of missing collections currently in the queue. + MissingCollectionQueueSize() uint } // Fetcher is a component that consumes finalized block jobs and processes them diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index fcc4c2fdbb8..828536e59fc 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -177,3 +177,8 @@ func (bp *BlockProcessor) getMissingCollections(blockHeight uint64) ([]*flow.Col return missingGuarantees, nil } + +// MissingCollectionQueueSize returns the number of missing collections currently in the queue. +func (bp *BlockProcessor) MissingCollectionQueueSize() uint { + return bp.mcq.Size() +} diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index e542596a1b5..2458f7efb13 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -22,7 +22,7 @@ type Fetcher struct { component.Component consumer *jobqueue.ComponentConsumer - jobProcessor collection_sync.BlockProcessor + blockProcessor collection_sync.BlockProcessor workSignal engine.Notifier metrics module.CollectionSyncMetrics lastReportedMu sync.Mutex @@ -37,7 +37,7 @@ var _ component.Component = (*Fetcher)(nil) // // Parameters: // - log: Logger for the component -// - jobProcessor: BlockProcessor implementation for processing collection indexing jobs +// - blockProcessor: BlockProcessor implementation for processing collection indexing jobs // - progressInitializer: Initializer for tracking processed block heights // - state: Protocol state for reading finalized block information // - blocks: Blocks storage for reading blocks by height @@ -48,8 +48,7 @@ var _ component.Component = (*Fetcher)(nil) // No error returns are expected during normal operation. func NewFetcher( log zerolog.Logger, - jobProcessor collection_sync.BlockProcessor, - progressInitializer storage.ConsumerProgressInitializer, + blockProcessor collection_sync.BlockProcessor, progressInitializer storage.ConsumerProgressInitializer, state protocol.State, blocks storage.Blocks, maxProcessing uint64, // max number of blocks to fetch collections @@ -74,7 +73,7 @@ func NewFetcher( return } - err = jobProcessor.FetchCollections(ctx, block, done) + err = blockProcessor.FetchCollections(ctx, block, done) if err != nil { ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) } @@ -95,11 +94,11 @@ func NewFetcher( } f := &Fetcher{ - Component: consumer, - consumer: consumer, - jobProcessor: jobProcessor, - workSignal: workSignal, - metrics: metrics, + Component: consumer, + consumer: consumer, + blockProcessor: blockProcessor, + workSignal: workSignal, + metrics: metrics, } if metrics == nil { @@ -111,6 +110,7 @@ func NewFetcher( // only advances when consecutive jobs complete, not on every individual job completion. consumer.SetPostNotifier(func(jobID module.JobID) { metrics.CollectionFetchedHeight(f.ProcessedHeight()) + metrics.MissingCollectionQueueSize(f.blockProcessor.MissingCollectionQueueSize()) }) return f, nil diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 0019351b757..6206c46fa90 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -208,3 +208,12 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func() return jobState.callback, true } + +// Size returns the number of missing collections currently in the queue. +// This is the total number of collections across all block heights that are still missing. +func (mcq *MissingCollectionQueue) Size() uint { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + return uint(len(mcq.collectionToHeight)) +} diff --git a/module/metrics.go b/module/metrics.go index e13b8c3ad5a..ea173d0b772 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -1222,6 +1222,9 @@ type CollectionSyncMetrics interface { // CollectionSyncedHeight reports the highest consecutive block height for // which collections have been synced from execution data CollectionSyncedHeight(height uint64) + + // MissingCollectionQueueSize reports the number of missing collections currently in the queue + MissingCollectionQueueSize(size uint) } type MachineAccountMetrics interface { diff --git a/module/metrics/collection_sync.go b/module/metrics/collection_sync.go index 37abc1cb6ff..548a8304e4f 100644 --- a/module/metrics/collection_sync.go +++ b/module/metrics/collection_sync.go @@ -8,8 +8,9 @@ import ( ) type CollectionSyncCollector struct { - collectionFetchedHeight prometheus.Gauge - collectionSyncedHeight prometheus.Gauge + collectionFetchedHeight prometheus.Gauge + collectionSyncedHeight prometheus.Gauge + missingCollectionQueueSize prometheus.Gauge } var _ module.CollectionSyncMetrics = (*CollectionSyncCollector)(nil) @@ -28,6 +29,12 @@ func NewCollectionSyncCollector() *CollectionSyncCollector { Subsystem: "collection_sync", Help: "the highest block height for which collections have been synced from execution data", }), + missingCollectionQueueSize: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "missing_collection_queue_size", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the number of missing collections currently in the queue", + }), } } @@ -38,3 +45,7 @@ func (c *CollectionSyncCollector) CollectionFetchedHeight(height uint64) { func (c *CollectionSyncCollector) CollectionSyncedHeight(height uint64) { c.collectionSyncedHeight.Set(float64(height)) } + +func (c *CollectionSyncCollector) MissingCollectionQueueSize(size uint) { + c.missingCollectionQueueSize.Set(float64(size)) +} From cc662436ee7a5c1ca1049e67a1ce79d97ed2bf15 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 08:50:31 -0800 Subject: [PATCH 043/126] fix lint --- consensus/hotstuff/mocks/distributor.go | 37 +++++++++++++++++++++++ module/metrics/noop.go | 1 + module/mock/collection_sync_metrics.go | 39 +++++++++++++++++++++++++ 3 files changed, 77 insertions(+) create mode 100644 consensus/hotstuff/mocks/distributor.go create mode 100644 module/mock/collection_sync_metrics.go diff --git a/consensus/hotstuff/mocks/distributor.go b/consensus/hotstuff/mocks/distributor.go new file mode 100644 index 00000000000..b914f899c9f --- /dev/null +++ b/consensus/hotstuff/mocks/distributor.go @@ -0,0 +1,37 @@ +// Code generated by mockery. DO NOT EDIT. + +package mocks + +import ( + model "github.com/onflow/flow-go/consensus/hotstuff/model" + mock "github.com/stretchr/testify/mock" +) + +// Distributor is an autogenerated mock type for the Distributor type +type Distributor struct { + mock.Mock +} + +// AddOnBlockFinalizedConsumer provides a mock function with given fields: consumer +func (_m *Distributor) AddOnBlockFinalizedConsumer(consumer func(*model.Block)) { + _m.Called(consumer) +} + +// AddOnBlockIncorporatedConsumer provides a mock function with given fields: consumer +func (_m *Distributor) AddOnBlockIncorporatedConsumer(consumer func(*model.Block)) { + _m.Called(consumer) +} + +// NewDistributor creates a new instance of Distributor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDistributor(t interface { + mock.TestingT + Cleanup(func()) +}) *Distributor { + mock := &Distributor{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/module/metrics/noop.go b/module/metrics/noop.go index 69eb594ae27..bdf38738853 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -401,6 +401,7 @@ var _ module.CollectionSyncMetrics = (*NoopCollector)(nil) func (nc *NoopCollector) CollectionFetchedHeight(height uint64) {} func (nc *NoopCollector) CollectionSyncedHeight(height uint64) {} +func (nc *NoopCollector) MissingCollectionQueueSize(size uint) {} func (nc *NoopCollector) AccountBalance(bal float64) {} func (nc *NoopCollector) RecommendedMinBalance(bal float64) {} diff --git a/module/mock/collection_sync_metrics.go b/module/mock/collection_sync_metrics.go new file mode 100644 index 00000000000..42270a81944 --- /dev/null +++ b/module/mock/collection_sync_metrics.go @@ -0,0 +1,39 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +import mock "github.com/stretchr/testify/mock" + +// CollectionSyncMetrics is an autogenerated mock type for the CollectionSyncMetrics type +type CollectionSyncMetrics struct { + mock.Mock +} + +// CollectionFetchedHeight provides a mock function with given fields: height +func (_m *CollectionSyncMetrics) CollectionFetchedHeight(height uint64) { + _m.Called(height) +} + +// CollectionSyncedHeight provides a mock function with given fields: height +func (_m *CollectionSyncMetrics) CollectionSyncedHeight(height uint64) { + _m.Called(height) +} + +// MissingCollectionQueueSize provides a mock function with given fields: size +func (_m *CollectionSyncMetrics) MissingCollectionQueueSize(size uint) { + _m.Called(size) +} + +// NewCollectionSyncMetrics creates a new instance of CollectionSyncMetrics. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewCollectionSyncMetrics(t interface { + mock.TestingT + Cleanup(func()) +}) *CollectionSyncMetrics { + mock := &CollectionSyncMetrics{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} From 3ed17bbf9cf9d171e76310603e8b235a35041dd4 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 08:55:25 -0800 Subject: [PATCH 044/126] remove unused var --- engine/access/collection_sync/fetcher/engine.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index 2458f7efb13..eba803fbeea 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -2,7 +2,6 @@ package fetcher import ( "fmt" - "sync" "github.com/rs/zerolog" @@ -21,12 +20,10 @@ import ( type Fetcher struct { component.Component - consumer *jobqueue.ComponentConsumer - blockProcessor collection_sync.BlockProcessor - workSignal engine.Notifier - metrics module.CollectionSyncMetrics - lastReportedMu sync.Mutex - lastReportedHeight uint64 + consumer *jobqueue.ComponentConsumer + blockProcessor collection_sync.BlockProcessor + workSignal engine.Notifier + metrics module.CollectionSyncMetrics } var _ collection_sync.Fetcher = (*Fetcher)(nil) From 6a4448117d1bc704312df56325a0c5a4c94531b8 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 10:07:35 -0800 Subject: [PATCH 045/126] add todo --- engine/access/finalized_indexer/processor.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index 9e9d2e33f47..9aa57d3e976 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -77,6 +77,8 @@ func NewFinalizedBlockProcessor( collectionExecutedMetric: collectionExecutedMetric, } + // TODO (leo): no need to use job queue consumer, instead, we could use a simple + // for loop to go through from next unprocessed height to the latest height. processor.consumer, err = jobqueue.NewComponentConsumer( log.With().Str("module", "ingestion_block_consumer").Logger(), blockFinalizedNotifier.Channel(), From 389273fd4c8f45efa4ec4a25a9192cd2fea4a69b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 19:12:06 -0800 Subject: [PATCH 046/126] fix tests --- storage/locks.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/storage/locks.go b/storage/locks.go index 60451824d86..88a7c3313e7 100644 --- a/storage/locks.go +++ b/storage/locks.go @@ -92,6 +92,12 @@ var LockGroupExecutionBootstrap = []string{ LockIndexStateCommitment, } +// deprecated (leo), only used by engine/access/ingestion/engine, which is deprecated +var LockGroupAccessFinalizingBlock = []string{ + LockIndexBlockByPayloadGuarantees, + LockIndexExecutionResult, +} + var LockGroupExecutionSaveExecutionResult = []string{ LockIndexChunkDataPackByChunkID, LockInsertEvent, @@ -149,6 +155,7 @@ func addLocks(builder lockctx.DAGPolicyBuilder, locks []string) { func makeLockPolicy() lockctx.Policy { builder := lockctx.NewDAGPolicyBuilder() + addLocks(builder, LockGroupAccessFinalizingBlock) // deprecated (leo) addLocks(builder, LockGroupAccessStateSyncIndexBlockData) addLocks(builder, LockGroupAccessOptimisticSyncBlockPersist) addLocks(builder, LockGroupExecutionBootstrap) From b65768ab0c76ed952d9953b3de3219d6e4bb33e7 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 17 Nov 2025 19:42:01 -0800 Subject: [PATCH 047/126] fix linter --- cmd/access/node_builder/access_node_builder.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0fd200db44c..e05a929d004 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2520,9 +2520,11 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. finalizedBlockProcessor, err := finalized_indexer.NewFinalizedBlockProcessor( node.Logger, node.State, + node.StorageLockMgr, + node.ProtocolDB, node.Storage.Blocks, processedFinalizedBlockHeight, - builder.FollowerDistributor, + builder.FollowerDistributor.FinalizationDistributor, notNil(builder.collectionExecutedMetric), ) if err != nil { From 1ae635573aa451258450a5e92f058dd8a3e2242c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 10:26:42 -0800 Subject: [PATCH 048/126] update ProgressReader --- .../factory/progress_reader.go | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/engine/access/collection_sync/factory/progress_reader.go b/engine/access/collection_sync/factory/progress_reader.go index ce65016fa4c..9c3fa71dfea 100644 --- a/engine/access/collection_sync/factory/progress_reader.go +++ b/engine/access/collection_sync/factory/progress_reader.go @@ -34,8 +34,8 @@ func (pr *ProgressReader) SetCollectionFetcher(backend collection_sync.ProgressR } // ProcessedHeight returns the maximum processed height from the available backends. -// If both backends are available, it returns the maximum of their progress. -// If only one backend is available, it returns that backend's progress. +// If both backends are available, it returns the maximum of their progress and lastProgress. +// If only one backend is available, it returns the maximum of that backend's progress and lastProgress. // If neither backend is available, it returns lastProgress. func (pr *ProgressReader) ProcessedHeight() uint64 { hasExecutionData := pr.executionDataProcessor != nil @@ -44,18 +44,30 @@ func (pr *ProgressReader) ProcessedHeight() uint64 { if hasExecutionData && hasCollectionFetcher { execHeight := pr.executionDataProcessor.ProcessedHeight() collectionHeight := pr.collectionFetcher.ProcessedHeight() - if execHeight > collectionHeight { - return execHeight + max := execHeight + if collectionHeight > max { + max = collectionHeight } - return collectionHeight + if pr.lastProgress > max { + max = pr.lastProgress + } + return max } if hasExecutionData { - return pr.executionDataProcessor.ProcessedHeight() + execHeight := pr.executionDataProcessor.ProcessedHeight() + if pr.lastProgress > execHeight { + return pr.lastProgress + } + return execHeight } if hasCollectionFetcher { - return pr.collectionFetcher.ProcessedHeight() + collectionHeight := pr.collectionFetcher.ProcessedHeight() + if pr.lastProgress > collectionHeight { + return pr.lastProgress + } + return collectionHeight } return pr.lastProgress From a5ba7b0441612657f90b205fe98800adfbf5b5ca Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 14:08:04 -0800 Subject: [PATCH 049/126] fix wait for executed to allow tx to be sealed --- integration/testnet/client.go | 31 ++++++++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/integration/testnet/client.go b/integration/testnet/client.go index f12330dfc6a..56c4124e019 100644 --- a/integration/testnet/client.go +++ b/integration/testnet/client.go @@ -304,9 +304,9 @@ func (c *Client) WaitForSealed(ctx context.Context, id sdk.Identifier) (*sdk.Tra return c.waitForStatus(ctx, id, sdk.TransactionStatusSealed) } -// WaitForExecuted waits for the transaction to be executed, then returns the result. +// WaitForExecuted waits for the transaction to be executed or sealed, then returns the result. func (c *Client) WaitForExecuted(ctx context.Context, id sdk.Identifier) (*sdk.TransactionResult, error) { - return c.waitForStatus(ctx, id, sdk.TransactionStatusExecuted) + return c.waitForStatuses(ctx, id, sdk.TransactionStatusExecuted, sdk.TransactionStatusSealed) } // WaitUntilIndexed blocks until the node has indexed the given height. @@ -331,11 +331,32 @@ func (c *Client) waitForStatus( id sdk.Identifier, targetStatus sdk.TransactionStatus, ) (*sdk.TransactionResult, error) { - fmt.Printf("Waiting for transaction %s to be %v...\n", id, targetStatus) + return c.waitForStatuses(ctx, id, targetStatus) +} + +// waitForStatuses waits for the transaction to be in one of the target statuses, then returns the result. +func (c *Client) waitForStatuses( + ctx context.Context, + id sdk.Identifier, + targetStatuses ...sdk.TransactionStatus, +) (*sdk.TransactionResult, error) { + statusSet := make(map[sdk.TransactionStatus]bool, len(targetStatuses)) + for _, status := range targetStatuses { + statusSet[status] = true + } + + var statusStr string + if len(targetStatuses) == 1 { + statusStr = targetStatuses[0].String() + } else { + statusStr = fmt.Sprintf("%v", targetStatuses) + } + fmt.Printf("Waiting for transaction %s to be %s...\n", id, statusStr) + errCount := 0 var result *sdk.TransactionResult var err error - for result == nil || (result.Status != targetStatus) { + for result == nil || !statusSet[result.Status] { childCtx, cancel := context.WithTimeout(ctx, time.Second*30) result, err = c.client.GetTransactionResult(childCtx, id) cancel() @@ -354,7 +375,7 @@ func (c *Client) waitForStatus( } fmt.Println() - fmt.Printf("(Wait for Seal) Transaction %s %s\n", id, targetStatus) + fmt.Printf("Transaction %s is %s\n", id, result.Status) return result, err } From a9de92ae005847b42cbdf646dc22d31287d311b1 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 14:12:33 -0800 Subject: [PATCH 050/126] disable ingest receipt engine --- cmd/access/node_builder/access_node_builder.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index e05a929d004..93fbafd0633 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2264,7 +2264,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } builder.Component("finalized block indexer", createFinalizedBlockIndexer(builder)) - builder.Component("ingest receipt", createIngestReceiptEngine(builder)) + // leo: ingest receipt allows AN to store receipts sent from the EN, + // but this is unnecessary, because access node can get the receipts from sealed block + // builder.Component("ingest receipt", createIngestReceiptEngine(builder)) createCollectionSyncFetcher(builder) if builder.storeTxResultErrorMessages { From 232e6e0b4b0029ccb2d0b77a1628f88c54eab287 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 14:22:52 -0800 Subject: [PATCH 051/126] comment out unused ingest receipt engine --- .../node_builder/access_node_builder.go | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 93fbafd0633..209f1fd9e25 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -44,7 +44,6 @@ import ( collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" "github.com/onflow/flow-go/engine/access/finalized_indexer" "github.com/onflow/flow-go/engine/access/index" - "github.com/onflow/flow-go/engine/access/ingest_receipt" "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" @@ -2601,16 +2600,16 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { } -func createIngestReceiptEngine(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - return ingest_receipt.New( - node.Logger, - node.EngineRegistry, - node.Storage.Receipts, - notNil(builder.collectionExecutedMetric), - ) - } -} +// func createIngestReceiptEngine(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +// return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +// return ingest_receipt.New( +// node.Logger, +// node.EngineRegistry, +// node.Storage.Receipts, +// notNil(builder.collectionExecutedMetric), +// ) +// } +// } // notNil ensures that the input is not nil and returns it // the usage is to ensure the dependencies are initialized before initializing a module. From 271c2711989670436f43b855f28fe41b7165c354 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 17:21:49 -0800 Subject: [PATCH 052/126] always enable collection fetcher to pass integration tests --- cmd/access/node_builder/access_node_builder.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 209f1fd9e25..b1ef4058b33 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2539,13 +2539,14 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder. Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.executionDataSyncEnabled { - // skip if execution data sync is enabled - // because the execution data contains the collections, so no need to fetch them separately. - // otherwise, if both fetching and syncing are enabled, they might slow down each other, - // because the database operation requires locking. - return &module.NoopReadyDoneAware{}, nil - } + // leo: temp removed for integration tests + // if builder.executionDataSyncEnabled { + // // skip if execution data sync is enabled + // // because the execution data contains the collections, so no need to fetch them separately. + // // otherwise, if both fetching and syncing are enabled, they might slow down each other, + // // because the database operation requires locking. + // return &module.NoopReadyDoneAware{}, nil + // } // TODO (leo): switch to module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight // to implement hybrid sync mode in the future From 3b813ebe8dd4598f48bcc5ecb7fcd76ef598f7a7 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 18 Nov 2025 17:31:12 -0800 Subject: [PATCH 053/126] fix lint --- integration/scripts/send-to-loki.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/scripts/send-to-loki.go b/integration/scripts/send-to-loki.go index a5908ea9716..962a2a354cf 100644 --- a/integration/scripts/send-to-loki.go +++ b/integration/scripts/send-to-loki.go @@ -139,4 +139,3 @@ func main() { os.Exit(1) } } - From 16f1c1f3bff9f1a69024919ac7cee194a9d844ec Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 10:03:00 -0800 Subject: [PATCH 054/126] fix cohort3 --- .../tests/access/cohort3/access_circuit_breaker_test.go | 3 +-- integration/tests/access/cohort3/collection_indexing_test.go | 5 ++--- module/metrics/collection_sync.go | 3 +++ 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/integration/tests/access/cohort3/access_circuit_breaker_test.go b/integration/tests/access/cohort3/access_circuit_breaker_test.go index e172be9515d..e69dc6c81c2 100644 --- a/integration/tests/access/cohort3/access_circuit_breaker_test.go +++ b/integration/tests/access/cohort3/access_circuit_breaker_test.go @@ -73,8 +73,7 @@ func (s *AccessCircuitBreakerSuite) SetupTest() { nodeConfigs = append(nodeConfigs, exeConfig) // need one dummy verification node (unused ghost) - verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel), testnet.AsGhost()) - nodeConfigs = append(nodeConfigs, verConfig) + nodeConfigs = append(nodeConfigs, testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel))) // need one controllable collection node collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel), testnet.WithAdditionalFlag("--hotstuff-proposal-duration=100ms")) diff --git a/integration/tests/access/cohort3/collection_indexing_test.go b/integration/tests/access/cohort3/collection_indexing_test.go index 230155efadf..e53ac9eac0c 100644 --- a/integration/tests/access/cohort3/collection_indexing_test.go +++ b/integration/tests/access/cohort3/collection_indexing_test.go @@ -12,12 +12,11 @@ import ( "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" ) // This suite tests collection syncing using the ingestion engine and the indexer. -const lastFullBlockMetric = "access_ingestion_last_full_finalized_block_height" - func TestCollectionIndexing(t *testing.T) { suite.Run(t, new(CollectionIndexingSuite)) } @@ -114,7 +113,7 @@ func (s *CollectionIndexingSuite) Test() { func (s *CollectionIndexingSuite) getLastFullHeight(containerName string) (uint64, error) { node := s.net.ContainerByName(containerName) metricsURL := fmt.Sprintf("http://0.0.0.0:%s/metrics", node.Port(testnet.MetricsPort)) - values := s.net.GetMetricFromContainer(s.T(), containerName, metricsURL, lastFullBlockMetric) + values := s.net.GetMetricFromContainer(s.T(), containerName, metricsURL, metrics.MetricsCollectionSyncedHeight) if len(values) == 0 { return 0, fmt.Errorf("no values found") diff --git a/module/metrics/collection_sync.go b/module/metrics/collection_sync.go index 548a8304e4f..f112e371266 100644 --- a/module/metrics/collection_sync.go +++ b/module/metrics/collection_sync.go @@ -15,6 +15,9 @@ type CollectionSyncCollector struct { var _ module.CollectionSyncMetrics = (*CollectionSyncCollector)(nil) +// to be used by test case, must be consistent with collectionSyncedHeight +const MetricsCollectionSyncedHeight = "access_collection_sync_collection_synced_height" + func NewCollectionSyncCollector() *CollectionSyncCollector { return &CollectionSyncCollector{ collectionFetchedHeight: promauto.NewGauge(prometheus.GaugeOpts{ From e1d1480e046a49af2e598d5857981147c3f5ff27 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 10:05:38 -0800 Subject: [PATCH 055/126] remove ghost verification node --- integration/tests/access/cohort4/grpc_compression_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/integration/tests/access/cohort4/grpc_compression_test.go b/integration/tests/access/cohort4/grpc_compression_test.go index 807262350dd..fcb585fe768 100644 --- a/integration/tests/access/cohort4/grpc_compression_test.go +++ b/integration/tests/access/cohort4/grpc_compression_test.go @@ -54,8 +54,7 @@ func (s *AccessGRPCSuite) SetupTest() { exeConfig := testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)) nodeConfigs = append(nodeConfigs, exeConfig) - // need one dummy verification node (unused ghost) - verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel), testnet.AsGhost()) + verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel)) nodeConfigs = append(nodeConfigs, verConfig) // need one controllable collection node From 84072c0bfa6a92ff1af978cb14deb10b53c88719 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 11:13:13 -0800 Subject: [PATCH 056/126] fixed --- .../node_builder/access_node_builder.go | 137 ++++++++++++------ integration/tests/execution/suite.go | 12 +- 2 files changed, 106 insertions(+), 43 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b1ef4058b33..1d5d24ef87d 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2102,33 +2102,15 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } - preferredENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.PreferredExecutionNodeIDs) - if err != nil { - return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for preferred EN map: %w", err) - } - - fixedENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.FixedExecutionNodeIDs) - if err != nil { - return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) - } - - builder.ExecNodeIdentitiesProvider = commonrpc.NewExecutionNodeIdentitiesProvider( - node.Logger, - node.State, - node.Storage.Receipts, - preferredENIdentifiers, - fixedENIdentifiers, - ) + // ExecNodeIdentitiesProvider is initialized in the "execution node identities provider" module + // to ensure it's available before modules that depend on it. Verify it's initialized here. + _ = notNil(builder.ExecNodeIdentitiesProvider) nodeCommunicator := node_communicator.NewNodeCommunicator(backendConfig.CircuitBreakerConfig.Enabled) - builder.txResultErrorMessageProvider = error_messages.NewTxErrorMessageProvider( - node.Logger, - builder.transactionResultErrorMessages, // might be nil - notNil(builder.TxResultsIndex), - connFactory, - nodeCommunicator, - notNil(builder.ExecNodeIdentitiesProvider), - ) + + // txResultErrorMessageProvider is initialized in the "transaction result error message provider" module + // to ensure it's available before modules that depend on it. Verify it's initialized here. + _ = notNil(builder.txResultErrorMessageProvider) builder.nodeBackend, err = backend.New(backend.Params{ State: node.State, @@ -2268,15 +2250,95 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { // builder.Component("ingest receipt", createIngestReceiptEngine(builder)) createCollectionSyncFetcher(builder) + // Initialize ExecNodeIdentitiesProvider as a module so it's available before components run. + // This is needed because the "transaction result error messages storage" module depends on it. + builder.Module("execution node identities provider", func(node *cmd.NodeConfig) error { + backendConfig := builder.rpcConf.BackendConfig + + preferredENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.PreferredExecutionNodeIDs) + if err != nil { + return fmt.Errorf("failed to convert node id string to Flow Identifier for preferred EN map: %w", err) + } + + fixedENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.FixedExecutionNodeIDs) + if err != nil { + return fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) + } + + builder.ExecNodeIdentitiesProvider = commonrpc.NewExecutionNodeIdentitiesProvider( + node.Logger, + node.State, + node.Storage.Receipts, + preferredENIdentifiers, + fixedENIdentifiers, + ) + + return nil + }) + + // Initialize txResultErrorMessageProvider as a module so it's available before components run. + // This is needed because the "transaction result error messages storage" module depends on it. + builder.Module("transaction result error message provider", func(node *cmd.NodeConfig) error { + backendConfig := builder.rpcConf.BackendConfig + accessMetrics := builder.AccessMetrics + cacheSize := int(backendConfig.ConnectionPoolSize) + + var connBackendCache *rpcConnection.Cache + var err error + if cacheSize > 0 { + connBackendCache, err = rpcConnection.NewCache(node.Logger, accessMetrics, cacheSize) + if err != nil { + return fmt.Errorf("could not initialize connection cache: %w", err) + } + } + + connFactory := &rpcConnection.ConnectionFactoryImpl{ + AccessConfig: backendConfig.AccessConfig, + CollectionConfig: backendConfig.CollectionConfig, + ExecutionConfig: backendConfig.ExecutionConfig, + AccessMetrics: accessMetrics, + Log: node.Logger, + Manager: rpcConnection.NewManager( + node.Logger, + accessMetrics, + connBackendCache, + backendConfig.CircuitBreakerConfig, + builder.rpcConf.CompressorName, + ), + } + + nodeCommunicator := node_communicator.NewNodeCommunicator(backendConfig.CircuitBreakerConfig.Enabled) + builder.txResultErrorMessageProvider = error_messages.NewTxErrorMessageProvider( + node.Logger, + builder.transactionResultErrorMessages, // might be nil + notNil(builder.TxResultsIndex), + connFactory, + nodeCommunicator, + notNil(builder.ExecNodeIdentitiesProvider), + ) + + return nil + }) + if builder.storeTxResultErrorMessages { var txResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer builder. + // Initialize transactionResultErrorMessages storage first, as it's needed by the core Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { + builder.transactionResultErrorMessages = store.NewTransactionResultErrorMessages( + node.Metrics.Cache, + node.ProtocolDB, + bstorage.DefaultCacheSize, + ) + return nil + }). + // Create the core after storage is initialized + Module("transaction result error messages core", func(node *cmd.NodeConfig) error { txResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( node.Logger, notNil(builder.txResultErrorMessageProvider), - builder.transactionResultErrorMessages, + notNil(builder.transactionResultErrorMessages), notNil(builder.ExecNodeIdentitiesProvider), node.StorageLockMgr, ) @@ -2289,14 +2351,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { txResultErrorMessagesCore, ) }). - Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { - builder.transactionResultErrorMessages = store.NewTransactionResultErrorMessages( - node.Metrics.Cache, - node.ProtocolDB, - bstorage.DefaultCacheSize, - ) - return nil - }). Module("processed error messages block height consumer progress", func(node *cmd.NodeConfig) error { processedTxErrorMessagesBlockHeight = store.NewConsumerProgress( builder.ProtocolDB, @@ -2539,14 +2593,13 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder. Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // leo: temp removed for integration tests - // if builder.executionDataSyncEnabled { - // // skip if execution data sync is enabled - // // because the execution data contains the collections, so no need to fetch them separately. - // // otherwise, if both fetching and syncing are enabled, they might slow down each other, - // // because the database operation requires locking. - // return &module.NoopReadyDoneAware{}, nil - // } + if builder.executionDataSyncEnabled { + // skip if execution data sync is enabled + // because the execution data contains the collections, so no need to fetch them separately. + // otherwise, if both fetching and syncing are enabled, they might slow down each other, + // because the database operation requires locking. + return &module.NoopReadyDoneAware{}, nil + } // TODO (leo): switch to module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight // to implement hybrid sync mode in the future diff --git a/integration/tests/execution/suite.go b/integration/tests/execution/suite.go index 5d69a79b82b..41b14ca8893 100644 --- a/integration/tests/execution/suite.go +++ b/integration/tests/execution/suite.go @@ -114,7 +114,17 @@ func (s *Suite) SetupTest() { s.nodeConfigs = nil s.accessClient = nil - s.nodeConfigs = append(s.nodeConfigs, testnet.NewNodeConfig(flow.RoleAccess)) + s.nodeConfigs = append(s.nodeConfigs, testnet.NewNodeConfig(flow.RoleAccess, + // this integration tests didn't include verification node, so + // no block will be sealed. Since AN only sync execution data for sealed block, + // the collection syncing which relys on execution data sync will not work. + // and the collection syncing is required to report the correct transaction results, + // which the integration test need on (WaitForExecuted), otherwise the tx will stuck + // on "Status: Pending" + // we disable execution data sync on AN so that AN will fetch collections from LN instead. + testnet.WithAdditionalFlag("--execution-data-sync-enabled=false"), + ), + ) // generate the four consensus identities s.nodeIDs = unittest.IdentifierListFixture(4) From e1ebb8c7e85853d7a578238b749fa9ca6f4cebe2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 11:22:37 -0800 Subject: [PATCH 057/126] add logs --- .../access/cohort4/access_store_tx_error_messages_test.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go index e5278418382..1b62371ebfd 100644 --- a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go +++ b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go @@ -23,8 +23,6 @@ import ( "github.com/onflow/flow-go/storage/store" ) -const maxReceiptHeightMetric = "access_ingestion_max_receipt_height" - func TestAccessStoreTxErrorMessages(t *testing.T) { suite.Run(t, new(AccessStoreTxErrorMessagesSuite)) } @@ -112,20 +110,25 @@ func (s *AccessStoreTxErrorMessagesSuite) TestAccessStoreTxErrorMessages() { defer cancel() // Create and send a transaction that will result in an error. + s.T().Log("Creating and sending transaction that will result in an error...") txResult := s.createAndSendTxWithTxError() + s.T().Logf("Transaction sent with ID: %s", txResult.TransactionID.String()) client, err := s.net.ContainerByName(s.accessContainerName).TestnetClient() s.Require().NoError(err) // wait until the node has indexed a few blocks past the transaction block height + s.T().Logf("Waiting until access node has indexed up to block height: %d", txResult.BlockHeight+10) err = client.WaitUntilIndexed(ctx, txResult.BlockHeight+10) s.Require().NoError(err) + s.T().Logf("Access node has indexed up to block height: %d", txResult.BlockHeight+10) // Stop the network containers before checking the results. s.net.StopContainers() txResults := []*sdk.TransactionResult{txResult} txErrorMessages := s.fetchTxErrorMessages(txResults, s.accessContainerName) + s.T().Logf("Fetched transaction error messages from the database.") s.verifyTxErrorMessage(txResults, txErrorMessages) } From cde51989d852551f40aa600382f32202f322de47 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 11:46:09 -0800 Subject: [PATCH 058/126] remove scripts --- scripts/update-cadence.sh | 12 ------------ scripts/update-core-contracts.sh | 15 --------------- 2 files changed, 27 deletions(-) delete mode 100755 scripts/update-cadence.sh delete mode 100755 scripts/update-core-contracts.sh diff --git a/scripts/update-cadence.sh b/scripts/update-cadence.sh deleted file mode 100755 index 509f29ba582..00000000000 --- a/scripts/update-cadence.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/sh -# -# This script updates all cadence dependencies to a new version. -# Specify the desired version as the only argument when running the script: -# ./scripts/update-cadence.sh v1.2.3 - -go get github.com/onflow/cadence@$1 -cd integration -go get github.com/onflow/cadence@$1 -cd ../insecure/ -go get github.com/onflow/cadence@$1 -cd .. diff --git a/scripts/update-core-contracts.sh b/scripts/update-core-contracts.sh deleted file mode 100755 index a2a294d0270..00000000000 --- a/scripts/update-core-contracts.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/sh -# -# This script updates all flow-core-contracts dependencies to a new version. -# Specify the desired version as the only argument when running the script: -# ./scripts/update-core-contracts.sh v1.2.3 - -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd integration -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd ../insecure/ -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd .. From 029e8e068e2604a4bcb8cd5518b483b5ecf8043a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 19 Nov 2025 11:53:45 -0800 Subject: [PATCH 059/126] fix indexer core tests --- module/state_synchronization/indexer/indexer_core_test.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index 0b0b5f89494..814c8d8c653 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -351,18 +351,14 @@ func TestExecutionState_IndexBlockData(t *testing.T) { test.results.On("BatchStore", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Unset() test.scheduledTransactions.On("BatchIndex", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Unset() test.registers.On("Store", mock.Anything, mock.Anything).Unset() - test.collectionIndexer.On("IndexCollections", mock.Anything).Unset() // setup mocks to behave as they would if the block was already indexed. // tx results and scheduled transactions will not be called since events returned an error. + // The second goroutine that processes registers will still run and call Store. test.events. On("BatchStore", mocks.MatchLock(storage.LockInsertEvent), blockID, []flow.EventsList{tf.ExpectedEvents}, mock.Anything). Return(storage.ErrAlreadyExists). Once() - test.collectionIndexer. - On("IndexCollections", tf.ExpectedCollections). - Return(nil). - Once() test.registers. On("Store", mock.Anything, tf.Block.Height). Return(nil). From cee0e8ffd05c5650c01be2b08472b262cceb4061 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 08:19:23 -0800 Subject: [PATCH 060/126] add collection_fetch flag --- .../node_builder/access_node_builder.go | 65 +++++++++++++++++-- 1 file changed, 58 insertions(+), 7 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 61d0f087a51..b1ddf7e16a6 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -185,6 +185,7 @@ type AccessNodeConfig struct { registerDBPruneThreshold uint64 collectionFetcherMaxProcessing uint64 collectionFetcherMaxSearchAhead uint64 + collectionFetch string } type PublicNetworkConfig struct { @@ -294,6 +295,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { registerDBPruneThreshold: 0, collectionFetcherMaxProcessing: 10, collectionFetcherMaxSearchAhead: 20, + collectionFetch: "default", } } @@ -1334,6 +1336,10 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "public-network-execution-data-sync-enabled", defaultConfig.publicNetworkExecutionDataEnabled, "[experimental] whether to enable the execution data sync protocol on public network") + flags.StringVar(&builder.collectionFetch, + "collection-fetch", + defaultConfig.collectionFetch, + "collection fetch mode: 'default' (normal behavior), 'always_on' (always create fetcher even with execution data sync), 'fetch_only' (create fetcher but skip execution data processor)") flags.StringVar(&builder.executionDataDir, "execution-data-dir", defaultConfig.executionDataDir, "directory to use for Execution Data database") flags.Uint64Var(&builder.executionDataStartHeight, "execution-data-start-height", @@ -1522,6 +1528,9 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { if builder.supportsObserver && (builder.PublicNetworkConfig.BindAddress == cmd.NotSet || builder.PublicNetworkConfig.BindAddress == "") { return errors.New("public-network-address must be set if supports-observer is true") } + if builder.collectionFetch != "default" && builder.collectionFetch != "always_on" && builder.collectionFetch != "fetch_only" { + return fmt.Errorf("collection-fetch must be one of [default, always_on, fetch_only], got: %s", builder.collectionFetch) + } if builder.executionDataSyncEnabled { if builder.executionDataConfig.FetchTimeout <= 0 { return errors.New("execution-data-fetch-timeout must be greater than 0") @@ -2190,8 +2199,13 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return builder.RpcEng, nil }) - if builder.executionDataSyncEnabled { - builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + if builder.executionDataSyncEnabled && builder.collectionFetch != "fetch_only" { + node.Logger.Info(). + Str("collection_fetch_mode", builder.collectionFetch). + Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). + Msg("creating execution data processor") + // ExecutionDataCache should already be created in BuildExecutionSyncComponents if builder.ExecutionDataCache == nil { return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") @@ -2242,8 +2256,23 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } return executionDataProcessor, nil - }) - } + } + + // Log when execution data processor is not created + if !builder.executionDataSyncEnabled { + node.Logger.Info(). + Str("collection_fetch_mode", builder.collectionFetch). + Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). + Msg("execution data processor not created: execution data sync is disabled") + } else if builder.collectionFetch == "fetch_only" { + node.Logger.Info(). + Str("collection_fetch_mode", builder.collectionFetch). + Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). + Msg("execution data processor not created: collection fetch mode is 'fetch_only'") + } + + return &module.NoopReadyDoneAware{}, nil + }) builder.Component("finalized block indexer", createFinalizedBlockIndexer(builder)) // leo: ingest receipt allows AN to store receipts sent from the EN, @@ -2594,14 +2623,31 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder. Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.executionDataSyncEnabled { - // skip if execution data sync is enabled + // Create fetcher if: + // 1. collectionFetch is "always_on" (always create, even with execution data sync) + // 2. collectionFetch is "fetch_only" (always create) + // 3. collectionFetch is "default" and execution data sync is disabled + shouldCreateFetcher := builder.collectionFetch == "always_on" || + builder.collectionFetch == "fetch_only" || + (builder.collectionFetch == "default" && !builder.executionDataSyncEnabled) + + if !shouldCreateFetcher { + // skip if execution data sync is enabled and not in always_on or fetch_only mode // because the execution data contains the collections, so no need to fetch them separately. // otherwise, if both fetching and syncing are enabled, they might slow down each other, // because the database operation requires locking. + node.Logger.Info(). + Str("collection_fetch_mode", builder.collectionFetch). + Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). + Msg("collection sync fetcher not created: execution data sync is enabled and collection fetch mode is not 'always_on' or 'fetch_only'") return &module.NoopReadyDoneAware{}, nil } + node.Logger.Info(). + Str("collection_fetch_mode", builder.collectionFetch). + Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). + Msg("creating collection sync fetcher") + // TODO (leo): switch to module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight // to implement hybrid sync mode in the future // in the hybrid sync mode, the fetcher will store its progress under a different key, @@ -2647,7 +2693,12 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { return fetcher, nil }). Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.executionDataSyncEnabled { + // Create requester if fetcher was created (same conditions as fetcher) + shouldCreateRequester := builder.collectionFetch == "always_on" || + builder.collectionFetch == "fetch_only" || + (builder.collectionFetch == "default" && !builder.executionDataSyncEnabled) + + if !shouldCreateRequester { return &module.NoopReadyDoneAware{}, nil } return builder.CollectionRequesterEngine, nil From 58228beec624a44430b54b536636d84215d26a34 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 09:20:59 -0800 Subject: [PATCH 061/126] improve getMissingCollections --- .../fetcher/block_processor.go | 26 +++++++------------ storage/collections.go | 6 +++++ storage/operation/collections.go | 8 ++++++ storage/store/collections.go | 8 ++++++ 4 files changed, 32 insertions(+), 16 deletions(-) diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 828536e59fc..68258a9a2d1 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -1,7 +1,6 @@ package fetcher import ( - "errors" "fmt" "github.com/rs/zerolog" @@ -70,7 +69,7 @@ func (bp *BlockProcessor) FetchCollections( Msg("processing collection fetching job for finalized block") // Get missing collections for this block - missingGuarantees, err := bp.getMissingCollections(blockHeight) + missingGuarantees, err := bp.getMissingCollections(block) if err != nil { return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) } @@ -153,24 +152,19 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti // getMissingCollections retrieves the block and returns collection guarantees that are missing. // Only collections that are not already in storage are returned. -func (bp *BlockProcessor) getMissingCollections(blockHeight uint64) ([]*flow.CollectionGuarantee, error) { - block, err := bp.blocks.ByHeight(blockHeight) - if err != nil { - return nil, fmt.Errorf("failed to retrieve block at height %d: %w", blockHeight, err) - } - +func (bp *BlockProcessor) getMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) { var missingGuarantees []*flow.CollectionGuarantee for _, guarantee := range block.Payload.Guarantees { // Check if collection already exists in storage - _, err := bp.collections.LightByID(guarantee.CollectionID) + exists, err := bp.collections.ExistByID(guarantee.CollectionID) if err != nil { - if errors.Is(err, storage.ErrNotFound) { - // Collection is missing - missingGuarantees = append(missingGuarantees, guarantee) - } else { - // Unexpected error - return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) - } + // Unexpected error + return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) + } + + if !exists { + // Collection is missing + missingGuarantees = append(missingGuarantees, guarantee) } // If collection exists, skip it } diff --git a/storage/collections.go b/storage/collections.go index b8b9e026e30..51d8e9c3787 100644 --- a/storage/collections.go +++ b/storage/collections.go @@ -22,6 +22,12 @@ type CollectionsReader interface { // - `storage.ErrNotFound` if no light collection was found. LightByID(collID flow.Identifier) (*flow.LightCollection, error) + // ExistByID checks whether a collection with the given ID exists in storage. + // Returns (true, nil) if it exists, + // Returns (false, nil) if it does not exist. + // No errors are expected during normal operation. + ExistByID(collID flow.Identifier) (bool, error) + // LightByTransactionID returns a reduced representation of the collection // holding the given transaction ID. The reduced collection references the // constituent transactions by their hashes. diff --git a/storage/operation/collections.go b/storage/operation/collections.go index b2d20c9f953..01942382be3 100644 --- a/storage/operation/collections.go +++ b/storage/operation/collections.go @@ -32,6 +32,14 @@ func RetrieveCollection(r storage.Reader, collID flow.Identifier, collection *fl return RetrieveByKey(r, MakePrefix(codeCollection, collID), collection) } +// CollectionExists checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func CollectionExists(r storage.Reader, collID flow.Identifier) (bool, error) { + return KeyExists(r, MakePrefix(codeCollection, collID)) +} + // RemoveCollection removes a collection from the storage. // CAUTION: this is for recovery purposes only, and should not be used during normal operations! // It returns nil if the collection does not exist. diff --git a/storage/store/collections.go b/storage/store/collections.go index c07b2901d8e..0a04f3aa407 100644 --- a/storage/store/collections.go +++ b/storage/store/collections.go @@ -109,6 +109,14 @@ func (c *Collections) LightByID(colID flow.Identifier) (*flow.LightCollection, e return &collection, nil } +// ExistByID checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func (c *Collections) ExistByID(colID flow.Identifier) (bool, error) { + return operation.CollectionExists(c.db.Reader(), colID) +} + // Remove removes a collection from the database, including all constituent transactions and // indices inserted by Store. // Remove does not error if the collection does not exist From 5bd3c88620cc20a58f0ed4554001fd585fbe7661 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 09:27:16 -0800 Subject: [PATCH 062/126] simplify block_processor --- .../node_builder/access_node_builder.go | 1 - .../access/collection_sync/collection_sync.go | 4 ++ .../access/collection_sync/factory/fetcher.go | 4 -- .../fetcher/block_processor.go | 50 ++++--------------- .../access/collection_sync/indexer/indexer.go | 23 +++++++++ storage/mock/collections.go | 28 +++++++++++ storage/mock/collections_reader.go | 28 +++++++++++ 7 files changed, 92 insertions(+), 46 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b1ddf7e16a6..0142387dd9e 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2667,7 +2667,6 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { node.State, node.Me, node.Storage.Blocks, - node.Storage.Collections, node.Storage.Guarantees, builder.ProtocolDB, notNil(builder.blockCollectionIndexer), diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 88aaff55132..25ef5c6fc59 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -35,6 +35,10 @@ type BlockCollectionIndexer interface { // IndexCollectionsForBlock stores and indexes collections for a given block height. // No error is exepcted during normal operation. IndexCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error + + // GetMissingCollections retrieves the block and returns collection guarantees that are missing. + // Only collections that are not already in storage are returned. + GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) } // Implements the job lifecycle for a single block height. diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index fc192f26d70..67555a2e9d8 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -40,7 +40,6 @@ type CreateFetcherConfig struct { // - state: Protocol state // - me: Local node identity // - blocks: Blocks storage -// - collections: Collections storage // - guarantees: Guarantees storage // - db: Database for storage operations // - processedFinalizedBlockHeight: Initializer for tracking processed block heights @@ -58,7 +57,6 @@ func CreateFetcher( state protocol.State, me module.Local, blocks storage.Blocks, - collStore storage.Collections, guarantees storage.Guarantees, db storage.DB, indexer collection_sync.BlockCollectionIndexer, @@ -99,8 +97,6 @@ func CreateFetcher( mcq, indexer, collectionRequester, - blocks, - collStore, ) // Register handler for received collections diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 68258a9a2d1..71a7696a37a 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -8,18 +8,15 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/storage" ) // BlockProcessor implements the job lifecycle for collection indexing. // It orchestrates the flow: request → receive → index → complete. type BlockProcessor struct { - log zerolog.Logger - mcq collection_sync.MissingCollectionQueue - indexer collection_sync.BlockCollectionIndexer - requester collection_sync.CollectionRequester - blocks storage.Blocks - collections storage.CollectionsReader + log zerolog.Logger + mcq collection_sync.MissingCollectionQueue + indexer collection_sync.BlockCollectionIndexer + requester collection_sync.CollectionRequester } var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) @@ -31,9 +28,6 @@ var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) // - mcq: MissingCollectionQueue for tracking missing collections and callbacks // - indexer: BlockCollectionIndexer for storing and indexing collections // - requester: CollectionRequester for requesting collections from the network -// - blocks: Blocks storage for retrieving block data -// - collections: Collections storage reader for checking if collections already exist -// Set to a very large value to effectively disable fetching and rely only on EDI. // // No error returns are expected during normal operation. func NewBlockProcessor( @@ -41,16 +35,12 @@ func NewBlockProcessor( mcq collection_sync.MissingCollectionQueue, indexer collection_sync.BlockCollectionIndexer, requester collection_sync.CollectionRequester, - blocks storage.Blocks, - collections storage.CollectionsReader, ) *BlockProcessor { return &BlockProcessor{ - log: log.With().Str("component", "coll_fetcher").Logger(), - mcq: mcq, - indexer: indexer, - requester: requester, - blocks: blocks, - collections: collections, + log: log.With().Str("component", "coll_fetcher").Logger(), + mcq: mcq, + indexer: indexer, + requester: requester, } } @@ -69,7 +59,7 @@ func (bp *BlockProcessor) FetchCollections( Msg("processing collection fetching job for finalized block") // Get missing collections for this block - missingGuarantees, err := bp.getMissingCollections(block) + missingGuarantees, err := bp.indexer.GetMissingCollections(block) if err != nil { return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) } @@ -150,28 +140,6 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti return nil } -// getMissingCollections retrieves the block and returns collection guarantees that are missing. -// Only collections that are not already in storage are returned. -func (bp *BlockProcessor) getMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) { - var missingGuarantees []*flow.CollectionGuarantee - for _, guarantee := range block.Payload.Guarantees { - // Check if collection already exists in storage - exists, err := bp.collections.ExistByID(guarantee.CollectionID) - if err != nil { - // Unexpected error - return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) - } - - if !exists { - // Collection is missing - missingGuarantees = append(missingGuarantees, guarantee) - } - // If collection exists, skip it - } - - return missingGuarantees, nil -} - // MissingCollectionQueueSize returns the number of missing collections currently in the queue. func (bp *BlockProcessor) MissingCollectionQueueSize() uint { return bp.mcq.Size() diff --git a/engine/access/collection_sync/indexer/indexer.go b/engine/access/collection_sync/indexer/indexer.go index d33aaa9b94b..8c8d1fbfb83 100644 --- a/engine/access/collection_sync/indexer/indexer.go +++ b/engine/access/collection_sync/indexer/indexer.go @@ -74,3 +74,26 @@ func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( return nil } + +// GetMissingCollections retrieves the block and returns collection guarantees that are missing. +// Only collections that are not already in storage are returned. +// No error returns are expected during normal operation. +func (bci *blockCollectionIndexerImpl) GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) { + var missingGuarantees []*flow.CollectionGuarantee + for _, guarantee := range block.Payload.Guarantees { + // Check if collection already exists in storage + exists, err := bci.collections.ExistByID(guarantee.CollectionID) + if err != nil { + // Unexpected error + return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) + } + + if !exists { + // Collection is missing + missingGuarantees = append(missingGuarantees, guarantee) + } + // If collection exists, skip it + } + + return missingGuarantees, nil +} diff --git a/storage/mock/collections.go b/storage/mock/collections.go index 592ba5fba26..579f21b3dff 100644 --- a/storage/mock/collections.go +++ b/storage/mock/collections.go @@ -76,6 +76,34 @@ func (_m *Collections) ByID(collID flow.Identifier) (*flow.Collection, error) { return r0, r1 } +// ExistByID provides a mock function with given fields: collID +func (_m *Collections) ExistByID(collID flow.Identifier) (bool, error) { + ret := _m.Called(collID) + + if len(ret) == 0 { + panic("no return value specified for ExistByID") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(collID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(collID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(collID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // LightByID provides a mock function with given fields: collID func (_m *Collections) LightByID(collID flow.Identifier) (*flow.LightCollection, error) { ret := _m.Called(collID) diff --git a/storage/mock/collections_reader.go b/storage/mock/collections_reader.go index 3a8e071d6bc..f8d337ea5cf 100644 --- a/storage/mock/collections_reader.go +++ b/storage/mock/collections_reader.go @@ -42,6 +42,34 @@ func (_m *CollectionsReader) ByID(collID flow.Identifier) (*flow.Collection, err return r0, r1 } +// ExistByID provides a mock function with given fields: collID +func (_m *CollectionsReader) ExistByID(collID flow.Identifier) (bool, error) { + ret := _m.Called(collID) + + if len(ret) == 0 { + panic("no return value specified for ExistByID") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(collID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(collID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(collID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // LightByID provides a mock function with given fields: collID func (_m *CollectionsReader) LightByID(collID flow.Identifier) (*flow.LightCollection, error) { ret := _m.Called(collID) From 9d5fdbfa325fdcda28e3776148666c1b09cb0d9d Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 09:38:22 -0800 Subject: [PATCH 063/126] fix lint --- engine/execution/ingestion/mocks/collection_store.go | 5 +++++ storage/inmemory/collections_reader.go | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/engine/execution/ingestion/mocks/collection_store.go b/engine/execution/ingestion/mocks/collection_store.go index 1404d5baf58..6585be844d4 100644 --- a/engine/execution/ingestion/mocks/collection_store.go +++ b/engine/execution/ingestion/mocks/collection_store.go @@ -29,6 +29,11 @@ func (m *MockCollectionStore) ByID(id flow.Identifier) (*flow.Collection, error) return c, nil } +func (m *MockCollectionStore) ExistByID(id flow.Identifier) (bool, error) { + _, exists := m.byID[id] + return exists, nil +} + func (m *MockCollectionStore) Store(c *flow.Collection) (*flow.LightCollection, error) { m.byID[c.ID()] = c return c.Light(), nil diff --git a/storage/inmemory/collections_reader.go b/storage/inmemory/collections_reader.go index 1efc5bc16d7..dfce61b7818 100644 --- a/storage/inmemory/collections_reader.go +++ b/storage/inmemory/collections_reader.go @@ -61,6 +61,15 @@ func (c *CollectionsReader) LightByID(collID flow.Identifier) (*flow.LightCollec return val, nil } +// ExistByID checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func (c *CollectionsReader) ExistByID(collID flow.Identifier) (bool, error) { + _, exists := c.collections[collID] + return exists, nil +} + // LightByTransactionID returns the collection for the given transaction ID. Only retrieves transaction hashes. // // Expected error returns during normal operation: From fea0f6572493897d117c1617e7fc2c5407a63940 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 10:22:39 -0800 Subject: [PATCH 064/126] add comments --- engine/access/collection_sync/collection_sync.go | 5 +++-- .../collection_sync/fetcher/block_processor.go | 8 +++++++- engine/access/collection_sync/indexer/indexer.go | 14 +++++--------- storage/collections.go | 1 + 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 25ef5c6fc59..32ab29b5cb6 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -36,8 +36,9 @@ type BlockCollectionIndexer interface { // No error is exepcted during normal operation. IndexCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error - // GetMissingCollections retrieves the block and returns collection guarantees that are missing. - // Only collections that are not already in storage are returned. + // GetMissingCollections retrieves the block and returns collection guarantees that whose collections + // are missing in storage. + // Only garantees whose collections that are not already in storage are returned. GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) } diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 71a7696a37a..93c849d410d 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -64,7 +64,13 @@ func (bp *BlockProcessor) FetchCollections( return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) } - // If no missing collections, block is complete + // If there are no missing collections, this block is considered complete. + // Caution: This relies on the assumption that: + // whenever a collection exists in storage, all of its transactions must have already been indexed. + // This assumption currently holds because transaction indexing by collection is always performed + // in the same batch that stores the collection (via collections.BatchStoreAndIndexByTransaction). + // Note: when we receives a collection, we need it in memory, and don't index until all collections + // of the block are received. if len(missingGuarantees) == 0 { done() return nil diff --git a/engine/access/collection_sync/indexer/indexer.go b/engine/access/collection_sync/indexer/indexer.go index 8c8d1fbfb83..ae7dc6ce55b 100644 --- a/engine/access/collection_sync/indexer/indexer.go +++ b/engine/access/collection_sync/indexer/indexer.go @@ -49,11 +49,11 @@ func NewBlockCollectionIndexer( // // No error returns are expected during normal operation. func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( - blockHeight uint64, + _ uint64, cols []*flow.Collection, ) error { // Store and index collections - err := storage.WithLock(bci.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return storage.WithLock(bci.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { return bci.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { for _, collection := range cols { // Store the collection, including constituent transactions, and index transactionID -> collectionID @@ -68,15 +68,11 @@ func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( return nil }) }) - if err != nil { - return fmt.Errorf("failed to index collections for block height %d: %w", blockHeight, err) - } - - return nil } -// GetMissingCollections retrieves the block and returns collection guarantees that are missing. -// Only collections that are not already in storage are returned. +// GetMissingCollections retrieves the block and returns collection guarantees that whose collections +// are missing in storage. +// Only garantees whose collections that are not already in storage are returned. // No error returns are expected during normal operation. func (bci *blockCollectionIndexerImpl) GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) { var missingGuarantees []*flow.CollectionGuarantee diff --git a/storage/collections.go b/storage/collections.go index 51d8e9c3787..044964a33ca 100644 --- a/storage/collections.go +++ b/storage/collections.go @@ -54,6 +54,7 @@ type Collections interface { // StoreAndIndexByTransaction stores the collection and indexes it by transaction. // This is used by access node storing collections for finalized blocks. // + // deprecated, only used by AN ingestion engine, which is deprecated // CAUTION: current approach is NOT BFT and needs to be revised in the future. // Honest clusters ensure a transaction can only belong to one collection. However, in rare // cases, the collector clusters can exceed byzantine thresholds -- making it possible to From d0b2fc5b8eac7715c9e8477507cdca9e02277f68 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 7 Nov 2025 11:21:57 -0800 Subject: [PATCH 065/126] optimize batch store same collection --- storage/store/collections.go | 21 ++++++++--- storage/store/collections_test.go | 61 +++++++++++++++++++++++++++++++ 2 files changed, 77 insertions(+), 5 deletions(-) diff --git a/storage/store/collections.go b/storage/store/collections.go index 0a04f3aa407..47cb44270e6 100644 --- a/storage/store/collections.go +++ b/storage/store/collections.go @@ -187,11 +187,8 @@ func (c *Collections) BatchStoreAndIndexByTransaction(lctx lockctx.Proof, collec light := collection.Light() collectionID := light.ID() - err := operation.UpsertCollection(rw.Writer(), light) - if err != nil { - return nil, fmt.Errorf("could not insert collection: %w", err) - } - + // First, check if all transactions are already indexed and consistent + someTransactionIndexed := false for _, txID := range light.Transactions { var differentColTxIsIn flow.Identifier // The following is not BFT, because we can't handle the case where a transaction is included @@ -217,6 +214,20 @@ func (c *Collections) BatchStoreAndIndexByTransaction(lctx lockctx.Proof, collec if err != nil { return nil, fmt.Errorf("could not insert transaction ID: %w", err) } + someTransactionIndexed = true + } + + if !someTransactionIndexed { + // All transactions are already indexed and point to this collection. + // Since the index is always added along with the collection and transactions, + // this means the collection and its transactions have already been stored. + // Abort early to avoid redundant database writes. + return light, nil + } + + err := operation.UpsertCollection(rw.Writer(), light) + if err != nil { + return nil, fmt.Errorf("could not insert collection: %w", err) } // Store individual transactions diff --git a/storage/store/collections_test.go b/storage/store/collections_test.go index 8ab091c67b4..45576fe3199 100644 --- a/storage/store/collections_test.go +++ b/storage/store/collections_test.go @@ -187,3 +187,64 @@ func TestCollections_ConcurrentIndexByTx(t *testing.T) { assert.True(t, indexedCollection.ID() == col1.ID() || indexedCollection.ID() == col2.ID(), "Expected one of the collections to be indexed") }) } + +// TestCollections_BatchStoreAndIndexByTransaction_EarlyAbort verifies that +// BatchStoreAndIndexByTransaction aborts early when all transactions are already +// indexed and point to the same collection, avoiding redundant database writes. +func TestCollections_BatchStoreAndIndexByTransaction_EarlyAbort(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + // Create a collection with multiple transactions + collection := unittest.CollectionFixture(3) + expectedLight := collection.Light() + + // First, store the collection and index it by transaction + err := unittest.WithLock(t, lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + _, err := collections.BatchStoreAndIndexByTransaction(lctx, &collection, rw) + return err + }) + }) + require.NoError(t, err) + + // Verify the collection was stored + actualLight, err := collections.LightByID(collection.ID()) + require.NoError(t, err) + assert.Equal(t, expectedLight, actualLight) + + // Verify all transactions are indexed + for _, tx := range collection.Transactions { + collLight, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + assert.Equal(t, collection.ID(), collLight.ID()) + } + + // Try to store the same collection again - should abort early + err = unittest.WithLock(t, lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + light, err := collections.BatchStoreAndIndexByTransaction(lctx, &collection, rw) + require.NoError(t, err) + // Should return the light collection without error + assert.Equal(t, expectedLight, light) + return err + }) + }) + require.NoError(t, err) + + // Verify the collection still exists and is unchanged + actualLight, err = collections.LightByID(collection.ID()) + require.NoError(t, err) + assert.Equal(t, expectedLight, actualLight) + + // Verify all transactions are still indexed correctly + for _, tx := range collection.Transactions { + collLight, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + assert.Equal(t, collection.ID(), collLight.ID()) + } + }) +} From ca8695cbb1ba41e59bbb5b9d2059683386a33903 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 11:05:41 -0800 Subject: [PATCH 066/126] add benchmark --- .../indexer/indexer_benchmark_test.go | 144 ++++++++++++++++++ 1 file changed, 144 insertions(+) create mode 100644 engine/access/collection_sync/indexer/indexer_benchmark_test.go diff --git a/engine/access/collection_sync/indexer/indexer_benchmark_test.go b/engine/access/collection_sync/indexer/indexer_benchmark_test.go new file mode 100644 index 00000000000..463e443a1c9 --- /dev/null +++ b/engine/access/collection_sync/indexer/indexer_benchmark_test.go @@ -0,0 +1,144 @@ +package indexer + +import ( + "flag" + "sync" + "testing" + "time" + + "github.com/cockroachdb/pebble/v2" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/pebbleimpl" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" +) + +var ( + collectionsPerHeight = flag.Int("collections-per-height", 5, "Number of collections per height") + transactionsPerCollection = flag.Int("transactions-per-collection", 2, "Number of transactions per collection") +) + +// BenchmarkIndexCollectionsForBlock_Sequential benchmarks IndexCollectionsForBlock +// when called sequentially for multiple heights. +// It measures how many heights per second can be indexed. +// +// Benchmark results for 30s (Apple M1 Pro, Pebble storage, Nov 20, 2025): +// - Default config (5 collections/height, 2 transactions/collection): ~197.9 heights/sec +// - Higher load (10 collections/height, 5 transactions/collection): ~168.0 heights/sec +func BenchmarkIndexCollectionsForBlock_Sequential(b *testing.B) { + unittest.RunWithPebbleDB(b, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + indexer := NewBlockCollectionIndexer( + metrics, + lockManager, + db, + collections, + ) + + // Pre-generate collections for all heights + collectionsByHeight := make([][]*flow.Collection, b.N) + for height := 0; height < b.N; height++ { + cols := make([]*flow.Collection, *collectionsPerHeight) + for i := 0; i < *collectionsPerHeight; i++ { + col := unittest.CollectionFixture(*transactionsPerCollection) + cols[i] = &col + } + collectionsByHeight[height] = cols + } + + b.ResetTimer() + b.ReportAllocs() + + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + + // Report heights per second + b.ReportMetric(float64(b.N)/b.Elapsed().Seconds(), "heights/sec") + }) +} + +// BenchmarkIndexCollectionsForBlock_Concurrent benchmarks IndexCollectionsForBlock +// with 2 concurrent threads, each independently calling IndexCollectionsForBlock +// sequentially for the same height range. Both threads use the same data for each height. +// It measures how many heights per second each thread can process. +// +// Benchmark results for 30s (Apple M1 Pro, Pebble storage, Nov 20, 2025): +// - Default config (5 collections/height, 2 transactions/collection): +// Thread 1: ~185.9 heights/sec, Thread 2: ~185.9 heights/sec +func BenchmarkIndexCollectionsForBlock_Concurrent(b *testing.B) { + unittest.RunWithPebbleDB(b, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + // Create separate indexer instances for each thread + indexer := NewBlockCollectionIndexer( + metrics, + lockManager, + db, + collections, + ) + + // Pre-generate collections for all heights (shared between threads) + collectionsByHeight := make([][]*flow.Collection, b.N) + for height := 0; height < b.N; height++ { + cols := make([]*flow.Collection, *collectionsPerHeight) + for i := 0; i < *collectionsPerHeight; i++ { + col := unittest.CollectionFixture(*transactionsPerCollection) + cols[i] = &col + } + collectionsByHeight[height] = cols + } + + b.ResetTimer() + b.ReportAllocs() + + var wg sync.WaitGroup + var thread1Elapsed, thread2Elapsed time.Duration + + // Thread 1: process heights sequentially + wg.Add(1) + go func() { + defer wg.Done() + start := time.Now() + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + thread1Elapsed = time.Since(start) + }() + + // Thread 2: process the same heights sequentially + wg.Add(1) + go func() { + defer wg.Done() + start := time.Now() + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + thread2Elapsed = time.Since(start) + }() + + wg.Wait() + + // Report heights per second for each thread + thread1HeightsPerSec := float64(b.N) / thread1Elapsed.Seconds() + thread2HeightsPerSec := float64(b.N) / thread2Elapsed.Seconds() + b.ReportMetric(thread1HeightsPerSec, "heights/sec-thread1") + b.ReportMetric(thread2HeightsPerSec, "heights/sec-thread2") + }) +} From 43c0402c9949dd031d9be2efeb64da973776da93 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 12:03:28 -0800 Subject: [PATCH 067/126] change flags default to execution_and_collection --- .../node_builder/access_node_builder.go | 56 ++++++++++--------- 1 file changed, 29 insertions(+), 27 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 0142387dd9e..adf5471208d 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -185,7 +185,7 @@ type AccessNodeConfig struct { registerDBPruneThreshold uint64 collectionFetcherMaxProcessing uint64 collectionFetcherMaxSearchAhead uint64 - collectionFetch string + collectionSync string } type PublicNetworkConfig struct { @@ -295,7 +295,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { registerDBPruneThreshold: 0, collectionFetcherMaxProcessing: 10, collectionFetcherMaxSearchAhead: 20, - collectionFetch: "default", + collectionSync: "execution_and_collection", } } @@ -1336,10 +1336,12 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "public-network-execution-data-sync-enabled", defaultConfig.publicNetworkExecutionDataEnabled, "[experimental] whether to enable the execution data sync protocol on public network") - flags.StringVar(&builder.collectionFetch, - "collection-fetch", - defaultConfig.collectionFetch, - "collection fetch mode: 'default' (normal behavior), 'always_on' (always create fetcher even with execution data sync), 'fetch_only' (create fetcher but skip execution data processor)") + flags.StringVar(&builder.collectionSync, + "collection-sync", + defaultConfig.collectionSync, + "collection sync mode: 'execution_first' (only fetch from execution nodes if execution data syncing is on, "+ + "otherwise fetch from collection nodes), 'execution_and_collection' (fetch from both collection nodes and execution nodes), "+ + "'collection_only' (only fetch from collection nodes)") flags.StringVar(&builder.executionDataDir, "execution-data-dir", defaultConfig.executionDataDir, "directory to use for Execution Data database") flags.Uint64Var(&builder.executionDataStartHeight, "execution-data-start-height", @@ -1528,8 +1530,8 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { if builder.supportsObserver && (builder.PublicNetworkConfig.BindAddress == cmd.NotSet || builder.PublicNetworkConfig.BindAddress == "") { return errors.New("public-network-address must be set if supports-observer is true") } - if builder.collectionFetch != "default" && builder.collectionFetch != "always_on" && builder.collectionFetch != "fetch_only" { - return fmt.Errorf("collection-fetch must be one of [default, always_on, fetch_only], got: %s", builder.collectionFetch) + if builder.collectionSync != "execution_first" && builder.collectionSync != "execution_and_collection" && builder.collectionSync != "collection_only" { + return fmt.Errorf("collection-sync must be one of [execution_first, execution_and_collection, collection_only], got: %s", builder.collectionSync) } if builder.executionDataSyncEnabled { if builder.executionDataConfig.FetchTimeout <= 0 { @@ -2200,9 +2202,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return builder.RpcEng, nil }) builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.executionDataSyncEnabled && builder.collectionFetch != "fetch_only" { + if builder.executionDataSyncEnabled && builder.collectionSync != "collection_only" { node.Logger.Info(). - Str("collection_fetch_mode", builder.collectionFetch). + Str("collection_sync_mode", builder.collectionSync). Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). Msg("creating execution data processor") @@ -2261,14 +2263,14 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { // Log when execution data processor is not created if !builder.executionDataSyncEnabled { node.Logger.Info(). - Str("collection_fetch_mode", builder.collectionFetch). + Str("collection_sync_mode", builder.collectionSync). Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). Msg("execution data processor not created: execution data sync is disabled") - } else if builder.collectionFetch == "fetch_only" { + } else if builder.collectionSync == "collection_only" { node.Logger.Info(). - Str("collection_fetch_mode", builder.collectionFetch). + Str("collection_sync_mode", builder.collectionSync). Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("execution data processor not created: collection fetch mode is 'fetch_only'") + Msg("execution data processor not created: collection sync mode is 'collection_only'") } return &module.NoopReadyDoneAware{}, nil @@ -2624,27 +2626,27 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder. Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Create fetcher if: - // 1. collectionFetch is "always_on" (always create, even with execution data sync) - // 2. collectionFetch is "fetch_only" (always create) - // 3. collectionFetch is "default" and execution data sync is disabled - shouldCreateFetcher := builder.collectionFetch == "always_on" || - builder.collectionFetch == "fetch_only" || - (builder.collectionFetch == "default" && !builder.executionDataSyncEnabled) + // 1. collectionSync is "execution_and_collection" (always create, even with execution data sync) + // 2. collectionSync is "collection_only" (always create) + // 3. collectionSync is "execution_first" and execution data sync is disabled + shouldCreateFetcher := builder.collectionSync == "execution_and_collection" || + builder.collectionSync == "collection_only" || + (builder.collectionSync == "execution_first" && !builder.executionDataSyncEnabled) if !shouldCreateFetcher { - // skip if execution data sync is enabled and not in always_on or fetch_only mode + // skip if execution data sync is enabled and not in execution_and_collection or collection_only mode // because the execution data contains the collections, so no need to fetch them separately. // otherwise, if both fetching and syncing are enabled, they might slow down each other, // because the database operation requires locking. node.Logger.Info(). - Str("collection_fetch_mode", builder.collectionFetch). + Str("collection_sync_mode", builder.collectionSync). Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("collection sync fetcher not created: execution data sync is enabled and collection fetch mode is not 'always_on' or 'fetch_only'") + Msg("collection sync fetcher not created: execution data sync is enabled and collection sync mode is not 'execution_and_collection' or 'collection_only'") return &module.NoopReadyDoneAware{}, nil } node.Logger.Info(). - Str("collection_fetch_mode", builder.collectionFetch). + Str("collection_sync_mode", builder.collectionSync). Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). Msg("creating collection sync fetcher") @@ -2693,9 +2695,9 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { }). Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Create requester if fetcher was created (same conditions as fetcher) - shouldCreateRequester := builder.collectionFetch == "always_on" || - builder.collectionFetch == "fetch_only" || - (builder.collectionFetch == "default" && !builder.executionDataSyncEnabled) + shouldCreateRequester := builder.collectionSync == "execution_and_collection" || + builder.collectionSync == "collection_only" || + (builder.collectionSync == "execution_first" && !builder.executionDataSyncEnabled) if !shouldCreateRequester { return &module.NoopReadyDoneAware{}, nil From 1ea00c226d0075b14c07f8ebdd6fa002cee520f0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 16:45:08 -0800 Subject: [PATCH 068/126] refactor execution data requester's HighestConsecutiveHeight --- .../node_builder/access_node_builder.go | 29 ++----------------- cmd/observer/node_builder/observer_builder.go | 6 +--- .../execution_data_index/provider.go | 8 ++--- .../factory/execution_data_processor.go | 6 ++-- .../integration_unsecure_grpc_server_test.go | 15 +++++++++- .../backend/backend_executiondata_test.go | 15 +++++++++- .../tracker/execution_data_tracker.go | 26 +++-------------- .../execution_data_requester.go | 12 ++++---- .../state_synchronization/indexer/indexer.go | 2 +- .../indexer/indexer_test.go | 4 +-- .../requester/execution_data_requester.go | 15 +++------- .../requester/jobs/execution_data_reader.go | 11 +++---- .../jobs/execution_data_reader_test.go | 13 ++++----- 13 files changed, 66 insertions(+), 96 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5d08bbe5f24..849e6c8ae9a 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -1028,10 +1028,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } builder.stateStreamConf.RpcMetricsEnabled = builder.rpcMetricsEnabled - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } broadcaster := engine.NewBroadcaster() eventQueryMode, err := query_mode.ParseIndexQueryMode(builder.rpcConf.BackendConfig.EventQueryMode) @@ -1050,7 +1046,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.executionDataConfig.InitialBlockHeight, node.Storage.Headers, broadcaster, - highestAvailableHeight, + notNil(builder.ExecutionDataRequester), builder.EventsIndex, useIndex, ) @@ -2213,31 +2209,11 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") } - // Create execution data tracker for the processor - // This is similar to the one created in state stream engine but used for collection indexing - broadcaster := engine.NewBroadcaster() - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } - - useIndex := builder.executionDataIndexingEnabled - executionDataTracker := subscriptiontracker.NewExecutionDataTracker( - builder.Logger, - node.State, - builder.executionDataConfig.InitialBlockHeight, - node.Storage.Headers, - broadcaster, - highestAvailableHeight, - builder.EventsIndex, - useIndex, - ) - // Create execution data processor executionDataProcessor, err := collection_syncfactory.CreateExecutionDataProcessor( builder.Logger, notNil(builder.ExecutionDataCache), - executionDataTracker, + notNil(builder.ExecutionDataRequester), collectionIndexedHeight, notNil(builder.blockCollectionIndexer), notNil(builder.CollectionSyncMetrics), @@ -2252,7 +2228,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { // Setup requester to notify processor when new execution data is received if builder.ExecutionDataDistributor != nil { builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { - executionDataTracker.OnExecutionData(executionData) executionDataProcessor.OnNewExectuionData() }) } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 7120a710623..0a669b58617 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1529,10 +1529,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS } builder.stateStreamConf.RpcMetricsEnabled = builder.rpcMetricsEnabled - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } broadcaster := engine.NewBroadcaster() eventQueryMode, err := query_mode.ParseIndexQueryMode(builder.rpcConf.BackendConfig.EventQueryMode) @@ -1551,7 +1547,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.executionDataConfig.InitialBlockHeight, node.Storage.Headers, broadcaster, - highestAvailableHeight, + builder.ExecutionDataRequester, builder.EventsIndex, useIndex, ) diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go index 7928b3cc504..ab832c50fdb 100644 --- a/engine/access/collection_sync/execution_data_index/provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -4,9 +4,9 @@ import ( "context" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" ) var _ collection_sync.ExecutionDataProvider = (*executionDataProvider)(nil) @@ -14,14 +14,14 @@ var _ collection_sync.ExecutionDataProvider = (*executionDataProvider)(nil) // executionDataProvider implements ExecutionDataProvider by querying ExecutionDataCache. type executionDataProvider struct { cache execution_data.ExecutionDataCache - highestExectuionDataHeight tracker.ExecutionDataTracker + highestExectuionDataHeight state_synchronization.ExecutionDataIndexedHeight } // NewExecutionDataProvider creates a new ExecutionDataProvider that reads from the given ExecutionDataCache. // The headers storage is used to determine the search range for finding available heights. func NewExecutionDataProvider( cache execution_data.ExecutionDataCache, - highestExectuionDataHeight tracker.ExecutionDataTracker, + highestExectuionDataHeight state_synchronization.ExecutionDataIndexedHeight, ) *executionDataProvider { return &executionDataProvider{ cache: cache, @@ -31,7 +31,7 @@ func NewExecutionDataProvider( // HighestIndexedHeight returns the highest block height for which execution data is available. func (p *executionDataProvider) HighestIndexedHeight() uint64 { - return p.highestExectuionDataHeight.GetHighestHeight() + return p.highestExectuionDataHeight.HighestConsecutiveHeight() } // GetExecutionDataByHeight returns the execution data for the given block height. diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index 1593f21f394..f4753b13c7e 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -7,10 +7,10 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" - "github.com/onflow/flow-go/engine/access/subscription/tracker" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/storage" ) @@ -32,13 +32,13 @@ import ( func CreateExecutionDataProcessor( log zerolog.Logger, cache execution_data.ExecutionDataCache, - executionDataTracker tracker.ExecutionDataTracker, + executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, collectionSyncMetrics module.CollectionSyncMetrics, // optional metrics collector ) (*execution_data_index.ExecutionDataProcessor, error) { // Create execution data provider - executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataTracker) + executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataIndexedHeight) // Convert ConsumerProgress to PersistentStrictMonotonicCounter processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) diff --git a/engine/access/integration_unsecure_grpc_server_test.go b/engine/access/integration_unsecure_grpc_server_test.go index edd0c9f2f9b..1ed816ded3d 100644 --- a/engine/access/integration_unsecure_grpc_server_test.go +++ b/engine/access/integration_unsecure_grpc_server_test.go @@ -41,6 +41,7 @@ import ( "github.com/onflow/flow-go/module/mempool/herocache" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/network" protocol "github.com/onflow/flow-go/state/protocol/mock" "github.com/onflow/flow-go/storage" @@ -49,6 +50,17 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +// mockExecutionDataIndexedHeight is a simple implementation of ExecutionDataIndexedHeight for testing +type mockExecutionDataIndexedHeight struct { + height uint64 +} + +func (m *mockExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + return m.height +} + +var _ state_synchronization.ExecutionDataIndexedHeight = (*mockExecutionDataIndexedHeight)(nil) + // SameGRPCPortTestSuite verifies both AccessAPI and ExecutionDataAPI client continue to work when configured // on the same port type SameGRPCPortTestSuite struct { @@ -264,13 +276,14 @@ func (suite *SameGRPCPortTestSuite) SetupTest() { eventIndexer := index.NewEventsIndex(index.NewReporter(), suite.events) + highestHeight := &mockExecutionDataIndexedHeight{height: rootBlock.Height} suite.executionDataTracker = tracker.NewExecutionDataTracker( suite.log, suite.state, rootBlock.Height, suite.headers, nil, - rootBlock.Height, + highestHeight, eventIndexer, false, ) diff --git a/engine/access/state_stream/backend/backend_executiondata_test.go b/engine/access/state_stream/backend/backend_executiondata_test.go index f7587affa6d..e026615ff0c 100644 --- a/engine/access/state_stream/backend/backend_executiondata_test.go +++ b/engine/access/state_stream/backend/backend_executiondata_test.go @@ -29,6 +29,7 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" "github.com/onflow/flow-go/module/mempool/herocache" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/state_synchronization" protocolmock "github.com/onflow/flow-go/state/protocol/mock" "github.com/onflow/flow-go/storage" storagemock "github.com/onflow/flow-go/storage/mock" @@ -87,6 +88,17 @@ type executionDataTestType struct { startHeight uint64 } +// mockExecutionDataIndexedHeight is a simple implementation of ExecutionDataIndexedHeight for testing +type mockExecutionDataIndexedHeight struct { + height uint64 +} + +func (m *mockExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + return m.height +} + +var _ state_synchronization.ExecutionDataIndexedHeight = (*mockExecutionDataIndexedHeight)(nil) + func TestBackendExecutionDataSuite(t *testing.T) { suite.Run(t, new(BackendExecutionDataSuite)) } @@ -265,13 +277,14 @@ func (s *BackendExecutionDataSuite) SetupBackend(useEventsIndex bool) { require.NoError(s.T(), err) // create real execution data tracker to use GetStartHeight from it, instead of mocking + highestHeight := &mockExecutionDataIndexedHeight{height: s.rootBlock.Height} s.executionDataTrackerReal = tracker.NewExecutionDataTracker( s.logger, s.state, s.rootBlock.Height, s.headers, s.broadcaster, - s.rootBlock.Height, + highestHeight, s.eventsIndex, useEventsIndex, ) diff --git a/engine/access/subscription/tracker/execution_data_tracker.go b/engine/access/subscription/tracker/execution_data_tracker.go index f797e775652..7637c14e0df 100644 --- a/engine/access/subscription/tracker/execution_data_tracker.go +++ b/engine/access/subscription/tracker/execution_data_tracker.go @@ -12,13 +12,11 @@ import ( "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/module/state_synchronization/indexer" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/utils/logging" ) const ( @@ -68,13 +66,12 @@ var _ ExecutionDataTracker = (*ExecutionDataTrackerImpl)(nil) type ExecutionDataTrackerImpl struct { BaseTracker log zerolog.Logger - headers storage.Headers broadcaster *engine.Broadcaster indexReporter state_synchronization.IndexReporter useIndex bool // highestHeight contains the highest consecutive block height that we have consecutive execution data for - highestHeight counters.StrictMonotonicCounter + highestHeight state_synchronization.ExecutionDataIndexedHeight } // NewExecutionDataTracker creates a new ExecutionDataTrackerImpl instance. @@ -97,16 +94,15 @@ func NewExecutionDataTracker( rootHeight uint64, headers storage.Headers, broadcaster *engine.Broadcaster, - highestAvailableFinalizedHeight uint64, + highestHeight state_synchronization.ExecutionDataIndexedHeight, indexReporter state_synchronization.IndexReporter, useIndex bool, ) *ExecutionDataTrackerImpl { return &ExecutionDataTrackerImpl{ BaseTracker: NewBaseTrackerImpl(rootHeight, state, headers), log: log, - headers: headers, broadcaster: broadcaster, - highestHeight: counters.NewMonotonicCounter(highestAvailableFinalizedHeight), + highestHeight: highestHeight, indexReporter: indexReporter, useIndex: useIndex, } @@ -221,25 +217,11 @@ func (e *ExecutionDataTrackerImpl) GetStartHeightFromLatest(ctx context.Context) // GetHighestHeight returns the highest height that we have consecutive execution data for. func (e *ExecutionDataTrackerImpl) GetHighestHeight() uint64 { - return e.highestHeight.Value() + return e.highestHeight.HighestConsecutiveHeight() } // OnExecutionData is used to notify the tracker when a new execution data is received. func (e *ExecutionDataTrackerImpl) OnExecutionData(executionData *execution_data.BlockExecutionDataEntity) { - log := e.log.With().Hex("block_id", logging.ID(executionData.BlockID)).Logger() - - log.Trace().Msg("received execution data") - - header, err := e.headers.ByBlockID(executionData.BlockID) - if err != nil { - // if the execution data is available, the block must be locally finalized - log.Fatal().Err(err).Msg("failed to notify of new execution data") - return - } - - // sets the highest height for which execution data is available. - _ = e.highestHeight.Set(header.Height) - e.broadcaster.Publish() } diff --git a/module/state_synchronization/execution_data_requester.go b/module/state_synchronization/execution_data_requester.go index dde19f6da52..d9e9725c1a3 100644 --- a/module/state_synchronization/execution_data_requester.go +++ b/module/state_synchronization/execution_data_requester.go @@ -8,13 +8,15 @@ import ( // OnExecutionDataReceivedConsumer is a callback that is called ExecutionData is received for a new block type OnExecutionDataReceivedConsumer func(*execution_data.BlockExecutionDataEntity) +type ExecutionDataIndexedHeight interface { + // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData + // has been received. + HighestConsecutiveHeight() uint64 +} + // ExecutionDataRequester is a component that syncs ExecutionData from the network, and exposes // a callback that is called when a new ExecutionData is received type ExecutionDataRequester interface { component.Component - - // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData - // has been received. - // This method must only be called after the component is Ready. If it is called early, an error is returned. - HighestConsecutiveHeight() (uint64, error) + ExecutionDataIndexedHeight } diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index db164a2c12d..c1b374aab13 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -69,7 +69,7 @@ func NewIndexer( registers storage.RegisterIndex, indexer *IndexerCore, executionCache *cache.ExecutionDataCache, - executionDataLatestHeight func() (uint64, error), + executionDataLatestHeight func() uint64, processedHeightInitializer storage.ConsumerProgressInitializer, ) (*Indexer, error) { r := &Indexer{ diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index 4c8187a86e2..6d8dc5a1c05 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -90,8 +90,8 @@ func newIndexerTest(t *testing.T, g *fixtures.GeneratorSuite, blocks []*flow.Blo return test } -func (w *indexerTest) latestHeight() (uint64, error) { - return w.last().Height, nil +func (w *indexerTest) latestHeight() uint64 { + return w.last().Height } func (w *indexerTest) last() *flow.Block { diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index ebdc12d719e..2fa64205f47 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -212,8 +212,8 @@ func New( // here by the notification job consumer to discover new jobs. // Note: we don't want to notify notificationConsumer for a block if it has not downloaded // execution data yet. - func() (uint64, error) { - return e.blockConsumer.LastProcessedIndex(), nil + func() uint64 { + return e.blockConsumer.LastProcessedIndex() }, ) @@ -270,15 +270,8 @@ func (e *executionDataRequester) onBlockFinalized(*model.Block) { // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData // has been received. // This method must only be called after the component is Ready. If it is called early, an error is returned. -func (e *executionDataRequester) HighestConsecutiveHeight() (uint64, error) { - select { - case <-e.blockConsumer.Ready(): - default: - // LastProcessedIndex is not meaningful until the component has completed startup - return 0, fmt.Errorf("HighestConsecutiveHeight must not be called before the component is ready") - } - - return e.blockConsumer.LastProcessedIndex(), nil +func (e *executionDataRequester) HighestConsecutiveHeight() uint64 { + return e.blockConsumer.LastProcessedIndex() } // runBlockConsumer runs the blockConsumer component diff --git a/module/state_synchronization/requester/jobs/execution_data_reader.go b/module/state_synchronization/requester/jobs/execution_data_reader.go index 721356ac7df..b63171f15ae 100644 --- a/module/state_synchronization/requester/jobs/execution_data_reader.go +++ b/module/state_synchronization/requester/jobs/execution_data_reader.go @@ -27,7 +27,7 @@ type ExecutionDataReader struct { store *cache.ExecutionDataCache fetchTimeout time.Duration - highestConsecutiveHeight func() (uint64, error) + highestConsecutiveHeight func() uint64 // TODO: refactor this to accept a context in AtIndex instead of storing it on the struct. // This requires also refactoring jobqueue.Consumer @@ -38,7 +38,7 @@ type ExecutionDataReader struct { func NewExecutionDataReader( store *cache.ExecutionDataCache, fetchTimeout time.Duration, - highestConsecutiveHeight func() (uint64, error), + highestConsecutiveHeight func() uint64, ) *ExecutionDataReader { return &ExecutionDataReader{ store: store, @@ -62,10 +62,7 @@ func (r *ExecutionDataReader) AtIndex(height uint64) (module.Job, error) { } // data for the requested height or a lower height, has not been downloaded yet. - highestHeight, err := r.highestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("failed to get highest height: %w", err) - } + highestHeight := r.highestConsecutiveHeight() if height > highestHeight { return nil, storage.ErrNotFound @@ -88,5 +85,5 @@ func (r *ExecutionDataReader) AtIndex(height uint64) (module.Job, error) { // Head returns the highest consecutive block height with downloaded execution data func (r *ExecutionDataReader) Head() (uint64, error) { - return r.highestConsecutiveHeight() + return r.highestConsecutiveHeight(), nil } diff --git a/module/state_synchronization/requester/jobs/execution_data_reader_test.go b/module/state_synchronization/requester/jobs/execution_data_reader_test.go index 74d3ee8e655..49fa56a923c 100644 --- a/module/state_synchronization/requester/jobs/execution_data_reader_test.go +++ b/module/state_synchronization/requester/jobs/execution_data_reader_test.go @@ -99,9 +99,7 @@ func (suite *ExecutionDataReaderSuite) reset() { suite.reader = NewExecutionDataReader( cache, suite.fetchTimeout, - func() (uint64, error) { - return suite.highestAvailableHeight(), nil - }, + suite.highestAvailableHeight, ) } @@ -189,11 +187,12 @@ func (suite *ExecutionDataReaderSuite) TestAtIndex() { } func (suite *ExecutionDataReaderSuite) TestHead() { + expectedIndex := uint64(15) + suite.highestAvailableHeight = func() uint64 { + return expectedIndex + } + suite.reset() suite.runTest(func() { - expectedIndex := uint64(15) - suite.highestAvailableHeight = func() uint64 { - return expectedIndex - } index, err := suite.reader.Head() assert.NoError(suite.T(), err) assert.Equal(suite.T(), expectedIndex, index) From 835e793d388b6899b39ce64f5e19ac9da9ca85bf Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 17:13:01 -0800 Subject: [PATCH 069/126] update mocks --- .../mock/execution_data_indexed_height.go | 42 +++++++++++++++++++ .../mock/execution_data_requester.go | 14 +------ 2 files changed, 44 insertions(+), 12 deletions(-) create mode 100644 module/state_synchronization/mock/execution_data_indexed_height.go diff --git a/module/state_synchronization/mock/execution_data_indexed_height.go b/module/state_synchronization/mock/execution_data_indexed_height.go new file mode 100644 index 00000000000..1cb8b94492a --- /dev/null +++ b/module/state_synchronization/mock/execution_data_indexed_height.go @@ -0,0 +1,42 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +import mock "github.com/stretchr/testify/mock" + +// ExecutionDataIndexedHeight is an autogenerated mock type for the ExecutionDataIndexedHeight type +type ExecutionDataIndexedHeight struct { + mock.Mock +} + +// HighestConsecutiveHeight provides a mock function with no fields +func (_m *ExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for HighestConsecutiveHeight") + } + + var r0 uint64 + if rf, ok := ret.Get(0).(func() uint64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(uint64) + } + + return r0 +} + +// NewExecutionDataIndexedHeight creates a new instance of ExecutionDataIndexedHeight. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewExecutionDataIndexedHeight(t interface { + mock.TestingT + Cleanup(func()) +}) *ExecutionDataIndexedHeight { + mock := &ExecutionDataIndexedHeight{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/module/state_synchronization/mock/execution_data_requester.go b/module/state_synchronization/mock/execution_data_requester.go index 40afda9ff30..06c0bf8921c 100644 --- a/module/state_synchronization/mock/execution_data_requester.go +++ b/module/state_synchronization/mock/execution_data_requester.go @@ -33,7 +33,7 @@ func (_m *ExecutionDataRequester) Done() <-chan struct{} { } // HighestConsecutiveHeight provides a mock function with no fields -func (_m *ExecutionDataRequester) HighestConsecutiveHeight() (uint64, error) { +func (_m *ExecutionDataRequester) HighestConsecutiveHeight() uint64 { ret := _m.Called() if len(ret) == 0 { @@ -41,23 +41,13 @@ func (_m *ExecutionDataRequester) HighestConsecutiveHeight() (uint64, error) { } var r0 uint64 - var r1 error - if rf, ok := ret.Get(0).(func() (uint64, error)); ok { - return rf() - } if rf, ok := ret.Get(0).(func() uint64); ok { r0 = rf() } else { r0 = ret.Get(0).(uint64) } - if rf, ok := ret.Get(1).(func() error); ok { - r1 = rf() - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // Ready provides a mock function with no fields From 1cf295cef4518a2867a4682d5967da33f35e914e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 17:29:21 -0800 Subject: [PATCH 070/126] add todo --- .../node_builder/access_node_builder.go | 24 +++++++++---------- module/jobqueue.go | 3 --- 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 849e6c8ae9a..809e5802918 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2226,11 +2226,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) // Setup requester to notify processor when new execution data is received - if builder.ExecutionDataDistributor != nil { - builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { - executionDataProcessor.OnNewExectuionData() - }) - } + builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { + executionDataProcessor.OnNewExectuionData() + }) return executionDataProcessor, nil } @@ -2625,14 +2623,14 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). Msg("creating collection sync fetcher") - // TODO (leo): switch to module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight - // to implement hybrid sync mode in the future - // in the hybrid sync mode, the fetcher will store its progress under a different key, - // and and only active if either of the following condition is met: - // 1) execution data sync is disabled - // 2) execution data sync is enabled and exectuion data sync height is far behind - // the latest finalized height and the execution data sync is not updating. - + // if execution data sync is disabled, or collection sync mode is "collection_only", + // then the fetcher is the only component updating the last full block height, + // so it can use the module.ConsumeProgressLastFullBlockHeight as the progress tracker. + // TODO(leo): if both execution data sync and collection fetcher are enabled, + // then the fetcher should use a different progress tracker to avoid contention. + // use ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight to track progress in that case. + // but before using that, make sure the progress is synced with the execution data + // processor.(module.ConsumeProgressLastFullBlockHeight) fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) // skip if execution data sync is enabled diff --git a/module/jobqueue.go b/module/jobqueue.go index 0a27480881e..4528a75206b 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -22,9 +22,6 @@ const ( // (new)the highest block height that has indexed collections from fetching missing collections from LN ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight" - // (new)the highest block height that has indexed collections from syncing execution data from EN - ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessSyncAndIndexedCollectionsBlockHeight" - // on finalized block, fetch tx error message from EN ConsumeProgressEngineTxErrorMessagesBlockHeight = "ConsumeProgressEngineTxErrorMessagesBlockHeight" From a4dd3b3ca81c338a0dc13fb7e485dd28c3ea75b0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 20 Nov 2025 17:42:01 -0800 Subject: [PATCH 071/126] fetcher to use different consumer progress to avoid contention --- .../node_builder/access_node_builder.go | 68 +++++++++++++++---- 1 file changed, 56 insertions(+), 12 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 809e5802918..4f96a5c9b84 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -1966,6 +1966,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { rootBlockHeight := node.State.Params().FinalizedRoot().Height + // Initialize ConsumeProgressLastFullBlockHeight progress, err := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) if err != nil { return err @@ -1976,8 +1977,58 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return fmt.Errorf("failed to get last processed index for last full block height: %w", err) } + // Sync ConsumeProgressLastFullBlockHeight and ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // by taking the max value of each and updating both + fetchAndIndexedTracker := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + fetchAndIndexed, err := fetchAndIndexedTracker.Initialize(rootBlockHeight) + if err != nil { + return fmt.Errorf("failed to initialize fetch and indexed collections block height tracker: %w", err) + } + fetchAndIndexedValue, err := fetchAndIndexed.ProcessedIndex() + if err != nil { + return fmt.Errorf("failed to get fetch and indexed collections block height: %w", err) + } + + // Take the max of both values + maxValue := max(lastProgress, fetchAndIndexedValue) + + // Update both trackers if needed + if lastProgress < maxValue { + if err := progress.SetProcessedIndex(maxValue); err != nil { + return fmt.Errorf("failed to update last full block height: %w", err) + } + node.Logger.Info(). + Uint64("old_value", lastProgress). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressLastFullBlockHeight). + Msg("synced collection sync progress tracker") + } + + if fetchAndIndexedValue < maxValue { + if err := fetchAndIndexed.SetProcessedIndex(maxValue); err != nil { + return fmt.Errorf("failed to update fetch and indexed collections block height: %w", err) + } + node.Logger.Info(). + Uint64("old_value", fetchAndIndexedValue). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight). + Msg("synced collection sync progress tracker") + } + + if lastProgress == maxValue && fetchAndIndexedValue == maxValue { + node.Logger.Info(). + Uint64("value", maxValue). + Msg("collection sync progress trackers already in sync") + } + + // Get the final synced value for ProgressReader + finalProgress, err := progress.ProcessedIndex() + if err != nil { + return fmt.Errorf("failed to get final synced progress: %w", err) + } + // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher - builder.lastFullBlockHeight = collection_syncfactory.NewProgressReader(lastProgress) + builder.lastFullBlockHeight = collection_syncfactory.NewProgressReader(finalProgress) collectionIndexedHeight = progress @@ -2623,17 +2674,10 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). Msg("creating collection sync fetcher") - // if execution data sync is disabled, or collection sync mode is "collection_only", - // then the fetcher is the only component updating the last full block height, - // so it can use the module.ConsumeProgressLastFullBlockHeight as the progress tracker. - // TODO(leo): if both execution data sync and collection fetcher are enabled, - // then the fetcher should use a different progress tracker to avoid contention. - // use ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight to track progress in that case. - // but before using that, make sure the progress is synced with the execution data - // processor.(module.ConsumeProgressLastFullBlockHeight) - fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight) - - // skip if execution data sync is enabled + // Fetcher always uses ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // to avoid contention with execution data processor which uses ConsumeProgressLastFullBlockHeight + fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + // Create fetcher and requesterEng requesterEng, fetcher, err := collection_syncfactory.CreateFetcher( node.Logger, From 84b37afc8708fd0671235564e0b10681ebf5439f Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 10:05:54 -0800 Subject: [PATCH 072/126] move component initialization functions from access_node_builder to factory functions --- .../node_builder/access_node_builder.go | 225 +++---------- .../collection_sync/factory/access_builder.go | 302 ++++++++++++++++++ .../factory/collection_sync_mode.go | 59 ++++ .../factory/execution_data_processor.go | 4 +- .../access/collection_sync/factory/fetcher.go | 4 +- 5 files changed, 411 insertions(+), 183 deletions(-) create mode 100644 engine/access/collection_sync/factory/access_builder.go create mode 100644 engine/access/collection_sync/factory/collection_sync_mode.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 4f96a5c9b84..30dd41c9bf2 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -61,7 +61,6 @@ import ( "github.com/onflow/flow-go/engine/access/subscription" subscriptiontracker "github.com/onflow/flow-go/engine/access/subscription/tracker" followereng "github.com/onflow/flow-go/engine/common/follower" - "github.com/onflow/flow-go/engine/common/requester" commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/stop" synceng "github.com/onflow/flow-go/engine/common/synchronization" @@ -185,7 +184,7 @@ type AccessNodeConfig struct { registerDBPruneThreshold uint64 collectionFetcherMaxProcessing uint64 collectionFetcherMaxSearchAhead uint64 - collectionSync string + collectionSync collection_syncfactory.CollectionSyncMode } type PublicNetworkConfig struct { @@ -295,7 +294,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { registerDBPruneThreshold: 0, collectionFetcherMaxProcessing: 10, collectionFetcherMaxSearchAhead: 20, - collectionSync: "execution_and_collection", + collectionSync: collection_syncfactory.CollectionSyncModeExecutionAndCollection, } } @@ -363,9 +362,8 @@ type FlowAccessNodeBuilder struct { SyncEngineParticipantsProviderFactory func() module.IdentifierProvider // engines - CollectionRequesterEngine *requester.Engine - FollowerEng *followereng.ComplianceEngine - StateStreamEng *statestreambackend.Engine + FollowerEng *followereng.ComplianceEngine + StateStreamEng *statestreambackend.Engine // for tx status deriver to know about the highest full block (a block with all collections synced) // backed by either collection fetcher to execution data syncing @@ -1148,6 +1146,7 @@ func (builder *FlowAccessNodeBuilder) ParseFlags() error { } func (builder *FlowAccessNodeBuilder) extraFlags() { + var collectionSyncStr string builder.ExtraFlags(func(flags *pflag.FlagSet) { defaultConfig := DefaultAccessNodeConfig() @@ -1331,9 +1330,9 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "public-network-execution-data-sync-enabled", defaultConfig.publicNetworkExecutionDataEnabled, "[experimental] whether to enable the execution data sync protocol on public network") - flags.StringVar(&builder.collectionSync, + flags.StringVar(&collectionSyncStr, "collection-sync", - defaultConfig.collectionSync, + defaultConfig.collectionSync.String(), "collection sync mode: 'execution_first' (only fetch from execution nodes if execution data syncing is on, "+ "otherwise fetch from collection nodes), 'execution_and_collection' (fetch from both collection nodes and execution nodes), "+ "'collection_only' (only fetch from collection nodes)") @@ -1525,9 +1524,12 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { if builder.supportsObserver && (builder.PublicNetworkConfig.BindAddress == cmd.NotSet || builder.PublicNetworkConfig.BindAddress == "") { return errors.New("public-network-address must be set if supports-observer is true") } - if builder.collectionSync != "execution_first" && builder.collectionSync != "execution_and_collection" && builder.collectionSync != "collection_only" { - return fmt.Errorf("collection-sync must be one of [execution_first, execution_and_collection, collection_only], got: %s", builder.collectionSync) + // Parse collection sync mode from string flag + collectionSyncMode, err := collection_syncfactory.ParseCollectionSyncMode(collectionSyncStr) + if err != nil { + return err } + builder.collectionSync = collectionSyncMode if builder.executionDataSyncEnabled { if builder.executionDataConfig.FetchTimeout <= 0 { return errors.New("execution-data-fetch-timeout must be greater than 0") @@ -1964,74 +1966,16 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil }). Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { - rootBlockHeight := node.State.Params().FinalizedRoot().Height - - // Initialize ConsumeProgressLastFullBlockHeight - progress, err := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) + result, err := collection_syncfactory.CreateProcessedLastFullBlockHeightModule( + node.Logger, + node.State, + builder.ProtocolDB, + ) if err != nil { return err } - - lastProgress, err := progress.ProcessedIndex() - if err != nil { - return fmt.Errorf("failed to get last processed index for last full block height: %w", err) - } - - // Sync ConsumeProgressLastFullBlockHeight and ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight - // by taking the max value of each and updating both - fetchAndIndexedTracker := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) - fetchAndIndexed, err := fetchAndIndexedTracker.Initialize(rootBlockHeight) - if err != nil { - return fmt.Errorf("failed to initialize fetch and indexed collections block height tracker: %w", err) - } - fetchAndIndexedValue, err := fetchAndIndexed.ProcessedIndex() - if err != nil { - return fmt.Errorf("failed to get fetch and indexed collections block height: %w", err) - } - - // Take the max of both values - maxValue := max(lastProgress, fetchAndIndexedValue) - - // Update both trackers if needed - if lastProgress < maxValue { - if err := progress.SetProcessedIndex(maxValue); err != nil { - return fmt.Errorf("failed to update last full block height: %w", err) - } - node.Logger.Info(). - Uint64("old_value", lastProgress). - Uint64("new_value", maxValue). - Str("tracker", module.ConsumeProgressLastFullBlockHeight). - Msg("synced collection sync progress tracker") - } - - if fetchAndIndexedValue < maxValue { - if err := fetchAndIndexed.SetProcessedIndex(maxValue); err != nil { - return fmt.Errorf("failed to update fetch and indexed collections block height: %w", err) - } - node.Logger.Info(). - Uint64("old_value", fetchAndIndexedValue). - Uint64("new_value", maxValue). - Str("tracker", module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight). - Msg("synced collection sync progress tracker") - } - - if lastProgress == maxValue && fetchAndIndexedValue == maxValue { - node.Logger.Info(). - Uint64("value", maxValue). - Msg("collection sync progress trackers already in sync") - } - - // Get the final synced value for ProgressReader - finalProgress, err := progress.ProcessedIndex() - if err != nil { - return fmt.Errorf("failed to get final synced progress: %w", err) - } - - // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher - builder.lastFullBlockHeight = collection_syncfactory.NewProgressReader(finalProgress) - - collectionIndexedHeight = progress - + builder.lastFullBlockHeight = result.LastFullBlockHeight + collectionIndexedHeight = result.CollectionIndexedHeight return nil }). Component("version control", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -2249,55 +2193,22 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { }) builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - if builder.executionDataSyncEnabled && builder.collectionSync != "collection_only" { - node.Logger.Info(). - Str("collection_sync_mode", builder.collectionSync). - Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("creating execution data processor") - - // ExecutionDataCache should already be created in BuildExecutionSyncComponents - if builder.ExecutionDataCache == nil { - return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") - } - - // Create execution data processor - executionDataProcessor, err := collection_syncfactory.CreateExecutionDataProcessor( - builder.Logger, - notNil(builder.ExecutionDataCache), - notNil(builder.ExecutionDataRequester), - collectionIndexedHeight, - notNil(builder.blockCollectionIndexer), - notNil(builder.CollectionSyncMetrics), - ) - if err != nil { - return nil, fmt.Errorf("could not create execution data processor: %w", err) - } - - // Store and register with ProgressReader - builder.lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) - - // Setup requester to notify processor when new execution data is received - builder.ExecutionDataDistributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { - executionDataProcessor.OnNewExectuionData() - }) - - return executionDataProcessor, nil - } - - // Log when execution data processor is not created - if !builder.executionDataSyncEnabled { - node.Logger.Info(). - Str("collection_sync_mode", builder.collectionSync). - Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("execution data processor not created: execution data sync is disabled") - } else if builder.collectionSync == "collection_only" { - node.Logger.Info(). - Str("collection_sync_mode", builder.collectionSync). - Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("execution data processor not created: collection sync mode is 'collection_only'") + processor, err := collection_syncfactory.CreateExecutionDataProcessorComponent( + builder.Logger, + builder.executionDataSyncEnabled, + builder.collectionSync, + notNil(builder.ExecutionDataCache), + notNil(builder.ExecutionDataRequester), + notNil(collectionIndexedHeight), + notNil(builder.blockCollectionIndexer), + notNil(builder.CollectionSyncMetrics), + notNil(builder.lastFullBlockHeight), + notNil(builder.ExecutionDataDistributor), + ) + if err != nil { + return nil, err } - - return &module.NoopReadyDoneAware{}, nil + return processor, nil }) builder.Component("finalized block indexer", createFinalizedBlockIndexer(builder)) @@ -2647,40 +2558,13 @@ func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd. } func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { + var requesterEng module.ReadyDoneAware builder. Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // Create fetcher if: - // 1. collectionSync is "execution_and_collection" (always create, even with execution data sync) - // 2. collectionSync is "collection_only" (always create) - // 3. collectionSync is "execution_first" and execution data sync is disabled - shouldCreateFetcher := builder.collectionSync == "execution_and_collection" || - builder.collectionSync == "collection_only" || - (builder.collectionSync == "execution_first" && !builder.executionDataSyncEnabled) - - if !shouldCreateFetcher { - // skip if execution data sync is enabled and not in execution_and_collection or collection_only mode - // because the execution data contains the collections, so no need to fetch them separately. - // otherwise, if both fetching and syncing are enabled, they might slow down each other, - // because the database operation requires locking. - node.Logger.Info(). - Str("collection_sync_mode", builder.collectionSync). - Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("collection sync fetcher not created: execution data sync is enabled and collection sync mode is not 'execution_and_collection' or 'collection_only'") - return &module.NoopReadyDoneAware{}, nil - } - - node.Logger.Info(). - Str("collection_sync_mode", builder.collectionSync). - Bool("execution_data_sync_enabled", builder.executionDataSyncEnabled). - Msg("creating collection sync fetcher") - - // Fetcher always uses ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight - // to avoid contention with execution data processor which uses ConsumeProgressLastFullBlockHeight - fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) - - // Create fetcher and requesterEng - requesterEng, fetcher, err := collection_syncfactory.CreateFetcher( + result, err := collection_syncfactory.CreateCollectionSyncFetcherComponent( node.Logger, + builder.executionDataSyncEnabled, + builder.collectionSync, node.Metrics.Engine, node.EngineRegistry, node.State, @@ -2689,39 +2573,22 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { node.Storage.Guarantees, builder.ProtocolDB, notNil(builder.blockCollectionIndexer), - fetchAndIndexedCollectionsBlockHeight, builder.FollowerDistributor, notNil(builder.collectionExecutedMetric), notNil(builder.CollectionSyncMetrics), - collection_syncfactory.CreateFetcherConfig{ - MaxProcessing: builder.collectionFetcherMaxProcessing, - MaxSearchAhead: builder.collectionFetcherMaxSearchAhead, - }, + builder.collectionFetcherMaxProcessing, + builder.collectionFetcherMaxSearchAhead, + notNil(builder.lastFullBlockHeight), ) - if err != nil { - return nil, fmt.Errorf("could not create collection fetcher: %w", err) + return nil, err } - - builder.CollectionRequesterEngine = requesterEng - - // Store and register with ProgressReader - builder.lastFullBlockHeight.SetCollectionFetcher(fetcher) - - return fetcher, nil + requesterEng = result.Requester + return result.Fetcher, nil }). - Component("collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // Create requester if fetcher was created (same conditions as fetcher) - shouldCreateRequester := builder.collectionSync == "execution_and_collection" || - builder.collectionSync == "collection_only" || - (builder.collectionSync == "execution_first" && !builder.executionDataSyncEnabled) - - if !shouldCreateRequester { - return &module.NoopReadyDoneAware{}, nil - } - return builder.CollectionRequesterEngine, nil + Component("collection_sync collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return requesterEng, nil }) - } // func createIngestReceiptEngine(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go new file mode 100644 index 00000000000..0766cdb1da6 --- /dev/null +++ b/engine/access/collection_sync/factory/access_builder.go @@ -0,0 +1,302 @@ +package factory + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/consensus/hotstuff" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" + edrequester "github.com/onflow/flow-go/module/state_synchronization/requester" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/store" +) + +// ProcessedLastFullBlockHeightModuleResult contains the results from creating the processed last full block height module. +type ProcessedLastFullBlockHeightModuleResult struct { + LastFullBlockHeight *ProgressReader + CollectionIndexedHeight storage.ConsumerProgress +} + +// CreateProcessedLastFullBlockHeightModule initializes and syncs the progress trackers for collection sync. +// +// Parameters: +// - log: Logger for logging operations +// - state: Protocol state to get root block height +// - db: Database for storing progress +// +// Returns: +// - The result containing the ProgressReader and collection indexed height +// - An error if the initialization fails +func CreateProcessedLastFullBlockHeightModule( + log zerolog.Logger, + state protocol.State, + db storage.DB, +) (*ProcessedLastFullBlockHeightModuleResult, error) { + rootBlockHeight := state.Params().FinalizedRoot().Height + + // Initialize ConsumeProgressLastFullBlockHeight + progress, err := store.NewConsumerProgress(db, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) + if err != nil { + return nil, err + } + + lastProgress, err := progress.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get last processed index for last full block height: %w", err) + } + + // Sync ConsumeProgressLastFullBlockHeight and ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // by taking the max value of each and updating both + fetchAndIndexedTracker := store.NewConsumerProgress(db, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + fetchAndIndexed, err := fetchAndIndexedTracker.Initialize(rootBlockHeight) + if err != nil { + return nil, fmt.Errorf("failed to initialize fetch and indexed collections block height tracker: %w", err) + } + fetchAndIndexedValue, err := fetchAndIndexed.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get fetch and indexed collections block height: %w", err) + } + + // Take the max of both values + maxValue := max(lastProgress, fetchAndIndexedValue) + + // Update both trackers if needed + if lastProgress < maxValue { + if err := progress.SetProcessedIndex(maxValue); err != nil { + return nil, fmt.Errorf("failed to update last full block height: %w", err) + } + log.Info(). + Uint64("old_value", lastProgress). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressLastFullBlockHeight). + Msg("synced collection sync progress tracker") + } + + if fetchAndIndexedValue < maxValue { + if err := fetchAndIndexed.SetProcessedIndex(maxValue); err != nil { + return nil, fmt.Errorf("failed to update fetch and indexed collections block height: %w", err) + } + log.Info(). + Uint64("old_value", fetchAndIndexedValue). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight). + Msg("synced collection sync progress tracker") + } + + if lastProgress == maxValue && fetchAndIndexedValue == maxValue { + log.Info(). + Uint64("value", maxValue). + Msg("collection sync progress trackers already in sync") + } + + // Get the final synced value for ProgressReader + finalProgress, err := progress.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get final synced progress: %w", err) + } + + // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher + return &ProcessedLastFullBlockHeightModuleResult{ + LastFullBlockHeight: NewProgressReader(finalProgress), + CollectionIndexedHeight: progress, + }, nil +} + +// CreateExecutionDataProcessorComponent creates an execution data processor component. +// It creates an execution data processor if execution data sync is enabled and collection sync mode is not "collection_only". +// +// Parameters: +// - log: Logger for logging operations +// - executionDataSyncEnabled: Whether execution data sync is enabled +// - collectionSyncMode: The collection sync mode +// - executionDataCache: Execution data cache +// - executionDataRequester: Execution data requester +// - collectionIndexedHeight: Consumer progress for collection indexed height +// - blockCollectionIndexer: Block collection indexer +// - collectionSyncMetrics: Collection sync metrics +// - lastFullBlockHeight: Progress reader to register the processor with +// - distributor: Execution data distributor to notify on new execution data +// +// Returns: +// - The processor component (or NoopReadyDoneAware if not created) +// - An error if the processor creation fails +func CreateExecutionDataProcessorComponent( + log zerolog.Logger, + executionDataSyncEnabled bool, + collectionSyncMode CollectionSyncMode, + executionDataCache execution_data.ExecutionDataCache, + executionDataRequester state_synchronization.ExecutionDataRequester, + collectionIndexedHeight storage.ConsumerProgress, + blockCollectionIndexer collection_sync.BlockCollectionIndexer, + collectionSyncMetrics module.CollectionSyncMetrics, + lastFullBlockHeight *ProgressReader, + distributor *edrequester.ExecutionDataDistributor, +) (module.ReadyDoneAware, error) { + shouldCreate := collectionSyncMode.ShouldCreateExecutionDataProcessor(executionDataSyncEnabled) + if !shouldCreate { + // Log when execution data processor is not created + if !executionDataSyncEnabled { + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("execution data processor not created: execution data sync is disabled") + } else if collectionSyncMode == CollectionSyncModeCollectionOnly { + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("execution data processor not created: collection sync mode is 'collection_only'") + } + + return &module.NoopReadyDoneAware{}, nil + } + + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("creating execution data processor") + + if executionDataCache == nil { + return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") + } + + // Create execution data processor + executionDataProcessor, err := createExecutionDataProcessor( + log, + executionDataCache, + executionDataRequester, + collectionIndexedHeight, + blockCollectionIndexer, + collectionSyncMetrics, + ) + if err != nil { + return nil, fmt.Errorf("could not create execution data processor: %w", err) + } + + // Register with ProgressReader + lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) + + distributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { + executionDataProcessor.OnNewExectuionData() + }) + + return executionDataProcessor, nil +} + +// CollectionSyncFetcherComponentResult contains the results from creating the collection sync fetcher component. +type CollectionSyncFetcherComponentResult struct { + Fetcher module.ReadyDoneAware + Requester module.ReadyDoneAware +} + +// CreateCollectionSyncFetcherComponent creates a collection fetcher and requester engine based on the collection sync mode. +// +// Parameters: +// - log: Logger for logging operations +// - executionDataSyncEnabled: Whether execution data sync is enabled +// - collectionSyncMode: The collection sync mode +// - engineMetrics: Engine metrics +// - engineRegistry: Engine registry +// - state: Protocol state +// - me: Local node identity +// - blocks: Blocks storage +// - guarantees: Guarantees storage +// - db: Database for storage operations +// - blockCollectionIndexer: Block collection indexer +// - followerDistributor: Follower distributor +// - collectionExecutedMetric: Collection executed metric +// - collectionSyncMetrics: Collection sync metrics +// - maxProcessing: Maximum number of concurrent processing jobs +// - maxSearchAhead: Maximum number of blocks to search ahead +// - lastFullBlockHeight: Progress reader to register the fetcher with +// +// Returns: +// - The result containing the fetcher component, requester component, and requester engine +// - An error if the fetcher creation fails +func CreateCollectionSyncFetcherComponent( + log zerolog.Logger, + executionDataSyncEnabled bool, + collectionSyncMode CollectionSyncMode, + engineMetrics module.EngineMetrics, + engineRegistry network.EngineRegistry, + state protocol.State, + me module.Local, + blocks storage.Blocks, + guarantees storage.Guarantees, + db storage.DB, + blockCollectionIndexer collection_sync.BlockCollectionIndexer, + followerDistributor hotstuff.Distributor, + collectionExecutedMetric module.CollectionExecutedMetric, + collectionSyncMetrics module.CollectionSyncMetrics, + maxProcessing uint64, + maxSearchAhead uint64, + lastFullBlockHeight *ProgressReader, +) (*CollectionSyncFetcherComponentResult, error) { + // Create fetcher if: + // 1. collectionSync is "execution_and_collection" (always create, even with execution data sync) + // 2. collectionSync is "collection_only" (always create) + // 3. collectionSync is "execution_first" and execution data sync is disabled + shouldCreateFetcher := collectionSyncMode.ShouldCreateFetcher(executionDataSyncEnabled) + + if !shouldCreateFetcher { + // skip if execution data sync is enabled and not in execution_and_collection or collection_only mode + // because the execution data contains the collections, so no need to fetch them separately. + // otherwise, if both fetching and syncing are enabled, they might slow down each other, + // because the database operation requires locking. + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("collection sync fetcher not created: execution data sync is enabled and collection sync mode is not 'execution_and_collection' or 'collection_only'") + return &CollectionSyncFetcherComponentResult{ + Fetcher: &module.NoopReadyDoneAware{}, + Requester: &module.NoopReadyDoneAware{}, + }, nil + } + + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("creating collection sync fetcher") + + // Fetcher always uses ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // to avoid contention with execution data processor which uses ConsumeProgressLastFullBlockHeight + fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(db, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + + // Create fetcher and requesterEng + requesterEng, fetcher, err := createFetcher( + log, + engineMetrics, + engineRegistry, + state, + me, + blocks, + guarantees, + db, + blockCollectionIndexer, + fetchAndIndexedCollectionsBlockHeight, + followerDistributor, + collectionExecutedMetric, + collectionSyncMetrics, + CreateFetcherConfig{ + MaxProcessing: maxProcessing, + MaxSearchAhead: maxSearchAhead, + }, + ) + + if err != nil { + return nil, fmt.Errorf("could not create collection fetcher: %w", err) + } + + // Register with ProgressReader + lastFullBlockHeight.SetCollectionFetcher(fetcher) + + return &CollectionSyncFetcherComponentResult{ + Fetcher: fetcher, + Requester: requesterEng, + }, nil +} diff --git a/engine/access/collection_sync/factory/collection_sync_mode.go b/engine/access/collection_sync/factory/collection_sync_mode.go new file mode 100644 index 00000000000..6fe7ebd482a --- /dev/null +++ b/engine/access/collection_sync/factory/collection_sync_mode.go @@ -0,0 +1,59 @@ +package factory + +import "fmt" + +// CollectionSyncMode represents the mode for collection synchronization. +type CollectionSyncMode string + +const ( + // CollectionSyncModeExecutionFirst fetches from execution nodes first if execution data syncing is enabled, + // otherwise fetches from collection nodes. + CollectionSyncModeExecutionFirst CollectionSyncMode = "execution_first" + // CollectionSyncModeExecutionAndCollection fetches from both collection nodes and execution nodes. + CollectionSyncModeExecutionAndCollection CollectionSyncMode = "execution_and_collection" + // CollectionSyncModeCollectionOnly only fetches from collection nodes. + CollectionSyncModeCollectionOnly CollectionSyncMode = "collection_only" +) + +// String returns the string representation of the CollectionSyncMode. +func (m CollectionSyncMode) String() string { + return string(m) +} + +// ParseCollectionSyncMode parses a string into a CollectionSyncMode. +func ParseCollectionSyncMode(s string) (CollectionSyncMode, error) { + switch s { + case string(CollectionSyncModeExecutionFirst): + return CollectionSyncModeExecutionFirst, nil + case string(CollectionSyncModeExecutionAndCollection): + return CollectionSyncModeExecutionAndCollection, nil + case string(CollectionSyncModeCollectionOnly): + return CollectionSyncModeCollectionOnly, nil + default: + return "", fmt.Errorf("invalid collection sync mode: %s, must be one of [execution_first, execution_and_collection, collection_only]", s) + } +} + +// ShouldCreateFetcher returns whether a collection fetcher should be created based on the sync mode +// and whether execution data sync is enabled. +// +// A fetcher should be created if: +// - The mode is ExecutionAndCollection (always create, even with execution data sync) +// - The mode is CollectionOnly (always create) +// - The mode is ExecutionFirst and execution data sync is disabled +func (m CollectionSyncMode) ShouldCreateFetcher(executionDataSyncEnabled bool) bool { + return m == CollectionSyncModeExecutionAndCollection || + m == CollectionSyncModeCollectionOnly || + (m == CollectionSyncModeExecutionFirst && !executionDataSyncEnabled) +} + +// ShouldCreateExecutionDataProcessor returns whether an execution data processor should be created +// based on the sync mode and whether execution data sync is enabled. +// +// An execution data processor should be created if: +// - Execution data sync is enabled AND +// - The mode is NOT CollectionOnly (since CollectionOnly mode only fetches from collection nodes) +func (m CollectionSyncMode) ShouldCreateExecutionDataProcessor(executionDataSyncEnabled bool) bool { + return executionDataSyncEnabled && m != CollectionSyncModeCollectionOnly +} + diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index f4753b13c7e..9d1a8e01a6f 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -14,7 +14,7 @@ import ( "github.com/onflow/flow-go/storage" ) -// CreateExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. +// createExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. // // Parameters: // - log: Logger for the component @@ -29,7 +29,7 @@ import ( // - error: An error if the processor could not be created // // No errors are expected during normal operation. -func CreateExecutionDataProcessor( +func createExecutionDataProcessor( log zerolog.Logger, cache execution_data.ExecutionDataCache, executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 67555a2e9d8..f8ea7764a10 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -27,7 +27,7 @@ type CreateFetcherConfig struct { MaxSearchAhead uint64 } -// CreateFetcher creates a new Fetcher component with all its dependencies. +// createFetcher creates a new Fetcher component with all its dependencies. // This function is in the collections package to avoid import cycles: // - collections package already imports collection_sync (for interfaces) // - CreateFetcher needs to create concrete types from collections package @@ -50,7 +50,7 @@ type CreateFetcherConfig struct { // Returns both the Fetcher and BlockProcessor so they can be reused in other components. // // No error returns are expected during normal operation. -func CreateFetcher( +func createFetcher( log zerolog.Logger, engineMetrics module.EngineMetrics, engineRegistry network.EngineRegistry, From 4198d32a341345dde256ab0d6efd720a1e78833a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 10:08:14 -0800 Subject: [PATCH 073/126] fix lint --- engine/access/collection_sync/factory/collection_sync_mode.go | 1 - 1 file changed, 1 deletion(-) diff --git a/engine/access/collection_sync/factory/collection_sync_mode.go b/engine/access/collection_sync/factory/collection_sync_mode.go index 6fe7ebd482a..53042319bca 100644 --- a/engine/access/collection_sync/factory/collection_sync_mode.go +++ b/engine/access/collection_sync/factory/collection_sync_mode.go @@ -56,4 +56,3 @@ func (m CollectionSyncMode) ShouldCreateFetcher(executionDataSyncEnabled bool) b func (m CollectionSyncMode) ShouldCreateExecutionDataProcessor(executionDataSyncEnabled bool) bool { return executionDataSyncEnabled && m != CollectionSyncModeCollectionOnly } - From c53a0e368b40185be9e5c5d9a7264e78bb612402 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 10:11:42 -0800 Subject: [PATCH 074/126] add collection sync mode tests --- .../factory/collection_sync_mode_test.go | 205 ++++++++++++++++++ 1 file changed, 205 insertions(+) create mode 100644 engine/access/collection_sync/factory/collection_sync_mode_test.go diff --git a/engine/access/collection_sync/factory/collection_sync_mode_test.go b/engine/access/collection_sync/factory/collection_sync_mode_test.go new file mode 100644 index 00000000000..7353b94f6fe --- /dev/null +++ b/engine/access/collection_sync/factory/collection_sync_mode_test.go @@ -0,0 +1,205 @@ +package factory + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// TestCollectionSyncMode_String tests the String method for all CollectionSyncMode constants. +func TestCollectionSyncMode_String(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + expected string + }{ + { + name: "execution_first", + mode: CollectionSyncModeExecutionFirst, + expected: "execution_first", + }, + { + name: "execution_and_collection", + mode: CollectionSyncModeExecutionAndCollection, + expected: "execution_and_collection", + }, + { + name: "collection_only", + mode: CollectionSyncModeCollectionOnly, + expected: "collection_only", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + assert.Equal(t, test.expected, test.mode.String()) + }) + } +} + +// TestParseCollectionSyncMode_Valid tests the ParseCollectionSyncMode function with valid inputs. +func TestParseCollectionSyncMode_Valid(t *testing.T) { + tests := map[string]CollectionSyncMode{ + "execution_first": CollectionSyncModeExecutionFirst, + "execution_and_collection": CollectionSyncModeExecutionAndCollection, + "collection_only": CollectionSyncModeCollectionOnly, + } + + for input, expectedMode := range tests { + t.Run(input, func(t *testing.T) { + mode, err := ParseCollectionSyncMode(input) + require.NoError(t, err) + assert.Equal(t, expectedMode, mode) + }) + } +} + +// TestParseCollectionSyncMode_Invalid tests the ParseCollectionSyncMode function with invalid inputs. +func TestParseCollectionSyncMode_Invalid(t *testing.T) { + invalidInputs := []string{ + "", + "unknown", + "invalid", + "execution-first", // wrong separator + "executionFirst", // wrong format + "EXECUTION_FIRST", // wrong case + "execution_and_collection_extra", // extra suffix + } + + for _, input := range invalidInputs { + t.Run(input, func(t *testing.T) { + mode, err := ParseCollectionSyncMode(input) + assert.Error(t, err) + assert.Empty(t, mode) + expectedErr := fmt.Errorf("invalid collection sync mode: %s, must be one of [execution_first, execution_and_collection, collection_only]", input) + assert.EqualError(t, err, expectedErr.Error()) + }) + } +} + +// TestCollectionSyncMode_ShouldCreateFetcher tests the ShouldCreateFetcher method with all combinations +// of modes and execution data sync enabled/disabled. +func TestCollectionSyncMode_ShouldCreateFetcher(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string + }{ + { + name: "execution_first_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: true, + expectedShouldCreate: false, + description: "should not create fetcher when execution data sync is enabled", + }, + { + name: "execution_first_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should create fetcher when execution data sync is disabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should always create fetcher for execution_and_collection mode", + }, + { + name: "execution_and_collection_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should always create fetcher for execution_and_collection mode", + }, + { + name: "collection_only_with_execution_data_sync_enabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should always create fetcher for collection_only mode", + }, + { + name: "collection_only_with_execution_data_sync_disabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should always create fetcher for collection_only mode", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + result := test.mode.ShouldCreateFetcher(test.executionDataSyncEnabled) + assert.Equal(t, test.expectedShouldCreate, result, test.description) + }) + } +} + +// TestCollectionSyncMode_ShouldCreateExecutionDataProcessor tests the ShouldCreateExecutionDataProcessor method +// with all combinations of modes and execution data sync enabled/disabled. +func TestCollectionSyncMode_ShouldCreateExecutionDataProcessor(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string + }{ + { + name: "execution_first_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should create processor when execution data sync is enabled", + }, + { + name: "execution_first_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor when execution data sync is disabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should create processor when execution data sync is enabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor when execution data sync is disabled", + }, + { + name: "collection_only_with_execution_data_sync_enabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: true, + expectedShouldCreate: false, + description: "should not create processor for collection_only mode even with execution data sync enabled", + }, + { + name: "collection_only_with_execution_data_sync_disabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor for collection_only mode", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + result := test.mode.ShouldCreateExecutionDataProcessor(test.executionDataSyncEnabled) + assert.Equal(t, test.expectedShouldCreate, result, test.description) + }) + } +} + From 00f5c1a0a16a6e9b1c0bebc92e5d827fd93018de Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 10:36:29 -0800 Subject: [PATCH 075/126] add indexer tests --- .../collection_sync/indexer/indexer_test.go | 134 ++++++++++++++++++ 1 file changed, 134 insertions(+) create mode 100644 engine/access/collection_sync/indexer/indexer_test.go diff --git a/engine/access/collection_sync/indexer/indexer_test.go b/engine/access/collection_sync/indexer/indexer_test.go new file mode 100644 index 00000000000..812d70ebd47 --- /dev/null +++ b/engine/access/collection_sync/indexer/indexer_test.go @@ -0,0 +1,134 @@ +package indexer + +import ( + "testing" + + "github.com/cockroachdb/pebble/v2" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + modulemock "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/pebbleimpl" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/fixtures" +) + +// TestIndexCollectionsForBlock verifies the complete indexing workflow: +// +// Verification Process: +// 1. Initial State Check: +// - Before indexing, GetMissingCollections returns all collections as missing +// - This confirms the indexer correctly identifies unindexed collections +// +// 2. First Indexing: +// - IndexCollectionsForBlock successfully stores and indexes all collections +// - Collections are persisted to storage and can be retrieved by ID +// - Transactions are stored and can be queried by transactions.ByID +// - Transaction-to-collection index exists and can be queried by collections.LightByTransactionID +// - Metrics are called for each collection (CollectionFinalized, CollectionExecuted) +// +// 3. Post-Indexing Verification: +// - GetMissingCollections returns empty (no missing collections) +// - This confirms indexing was successful and collections are now available +// +// 4. Idempotency Check: +// - Indexing the same block again succeeds without error +// - GetMissingCollections still returns empty after second indexing +// - This confirms the operation is idempotent and safe to retry +func TestIndexCollectionsForBlock(t *testing.T) { + unittest.RunWithPebbleDB(t, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + collectionMetrics := modulemock.NewCollectionExecutedMetric(t) + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + indexer := NewBlockCollectionIndexer( + collectionMetrics, + lockManager, + db, + collections, + ) + + g := fixtures.NewGeneratorSuite() + height := uint64(100) + collectionList := g.Collections().List(3) + + // Create guarantees and block before indexing + guarantees := make([]*flow.CollectionGuarantee, len(collectionList)) + for i, collection := range collectionList { + guarantee := g.Guarantees().Fixture(fixtures.Guarantee.WithCollectionID(collection.ID())) + guarantees[i] = guarantee + } + + payload := g.Payloads().Fixture( + fixtures.Payload.WithGuarantees(guarantees...), + ) + block := g.Blocks().Fixture( + fixtures.Block.WithPayload(payload), + ) + + // Step 1: Initial State Check - Before indexing, GetMissingCollections should return all collections as missing + missing, err := indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 3) + require.Equal(t, guarantees[0].CollectionID, missing[0].CollectionID) + require.Equal(t, guarantees[1].CollectionID, missing[1].CollectionID) + require.Equal(t, guarantees[2].CollectionID, missing[2].CollectionID) + + // Step 2: First Indexing - Expect metrics to be called for each collection + for _, collection := range collectionList { + light := collection.Light() + collectionMetrics.On("CollectionFinalized", light).Once() + collectionMetrics.On("CollectionExecuted", light).Once() + } + + err = indexer.IndexCollectionsForBlock(height, collectionList) + require.NoError(t, err) + + // Step 2 (continued): Verify all collections are stored and can be retrieved + for _, collection := range collectionList { + stored, err := collections.ByID(collection.ID()) + require.NoError(t, err) + require.Equal(t, collection.ID(), stored.ID()) + } + + // Step 2 (continued): Verify all transactions are stored and can be queried by ID + for _, collection := range collectionList { + for _, tx := range collection.Transactions { + storedTx, err := transactions.ByID(tx.ID()) + require.NoError(t, err) + require.Equal(t, tx.ID(), storedTx.ID()) + } + } + + // Step 2 (continued): Verify transaction-to-collection index exists and can be queried + for _, collection := range collectionList { + for _, tx := range collection.Transactions { + lightCollection, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + require.Equal(t, collection.ID(), lightCollection.ID()) + } + } + + // Step 3: Post-Indexing Verification - After indexing, GetMissingCollections should return empty + missing, err = indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 0) + + collectionMetrics.AssertExpectations(t) + + // Step 4: Idempotency Check - Indexing again is idempotent and should not error + err = indexer.IndexCollectionsForBlock(height, collectionList) + require.NoError(t, err) + + // Step 4 (continued): After second indexing, GetMissingCollections should still return empty + missing, err = indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 0) + }) +} From 8c72128f6fb77373850258d219cc08e47906ab04 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 11:24:58 -0800 Subject: [PATCH 076/126] update finalized indexer comments --- engine/access/finalized_indexer/processor.go | 25 +++++++++++--------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index d2365eacc9d..8c4ed2896c4 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -16,19 +16,22 @@ import ( "github.com/onflow/flow-go/storage" ) -// FinalizedBlockProcessor handles processing of finalized blocks, -// including indexing and syncing of related collections and execution results. +// FinalizedBlockProcessor processes finalized blocks and builds a collection-to-finalized-block index. // -// FinalizedBlockProcessor is designed to handle the ingestion of finalized Flow blocks -// in a scalable and decoupled manner. It uses a worker loop with a for loop to iterate -// through heights sequentially, processing each finalized block. This design enables -// the processor to handle high-throughput block finalization events without blocking -// other parts of the system. +// The processor iterates through each finalized block sequentially and indexes which finalized block +// contains each collection. This index is necessary because while a collection can belong to multiple +// unfinalized blocks, each collection belongs to exactly one finalized block. This uniqueness property +// enables efficient transaction result lookups. // -// The processor relies on the distributor to signal when a new finalized -// block is available, which triggers the worker loop to process any unprocessed blocks. -// The actual processing involves indexing block-to-collection and block-to-execution-result -// mappings, as well as requesting the associated collections. +// The collection-to-block index is used by the GetTransactionResult API to locate the block containing +// a transaction. The lookup process is: transaction -> collection (via collection sync indexer) -> +// finalized block (via this indexer). The transaction-to-collection index is built by the collection +// sync indexer, which indexes either sealed or finalized blocks. When indexing finalized blocks, the +// transaction-to-collection mapping is also unique. +// +// The processor uses a worker loop that processes blocks sequentially by height, triggered by the +// distributor when new blocks are finalized. This design enables high-throughput block finalization +// handling without blocking other system components. type FinalizedBlockProcessor struct { log zerolog.Logger component.Component From d4052415fe5f3a4a6f402978a81edc486a3e2bcc Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 14:21:33 -0800 Subject: [PATCH 077/126] use debug level log --- engine/access/finalized_indexer/processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index 8c4ed2896c4..235e4f2067a 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -153,7 +153,7 @@ func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, } // Log progress for each height with all relevant information - p.log.Info(). + p.log.Debug(). Uint64("indexed", height). Uint64("lowest_missing", lowestMissing). Uint64("highest_available", highestAvailableHeight). From 8d8740e3fa86ddc4647f8f9be59ad4070ab34cf5 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 14:26:19 -0800 Subject: [PATCH 078/126] fix lint --- .../factory/collection_sync_mode_test.go | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/engine/access/collection_sync/factory/collection_sync_mode_test.go b/engine/access/collection_sync/factory/collection_sync_mode_test.go index 7353b94f6fe..2938a40b655 100644 --- a/engine/access/collection_sync/factory/collection_sync_mode_test.go +++ b/engine/access/collection_sync/factory/collection_sync_mode_test.go @@ -42,9 +42,9 @@ func TestCollectionSyncMode_String(t *testing.T) { // TestParseCollectionSyncMode_Valid tests the ParseCollectionSyncMode function with valid inputs. func TestParseCollectionSyncMode_Valid(t *testing.T) { tests := map[string]CollectionSyncMode{ - "execution_first": CollectionSyncModeExecutionFirst, - "execution_and_collection": CollectionSyncModeExecutionAndCollection, - "collection_only": CollectionSyncModeCollectionOnly, + "execution_first": CollectionSyncModeExecutionFirst, + "execution_and_collection": CollectionSyncModeExecutionAndCollection, + "collection_only": CollectionSyncModeCollectionOnly, } for input, expectedMode := range tests { @@ -62,9 +62,9 @@ func TestParseCollectionSyncMode_Invalid(t *testing.T) { "", "unknown", "invalid", - "execution-first", // wrong separator - "executionFirst", // wrong format - "EXECUTION_FIRST", // wrong case + "execution-first", // wrong separator + "executionFirst", // wrong format + "EXECUTION_FIRST", // wrong case "execution_and_collection_extra", // extra suffix } @@ -83,17 +83,17 @@ func TestParseCollectionSyncMode_Invalid(t *testing.T) { // of modes and execution data sync enabled/disabled. func TestCollectionSyncMode_ShouldCreateFetcher(t *testing.T) { tests := []struct { - name string - mode CollectionSyncMode - executionDataSyncEnabled bool - expectedShouldCreate bool - description string + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string }{ { name: "execution_first_with_execution_data_sync_enabled", mode: CollectionSyncModeExecutionFirst, executionDataSyncEnabled: true, - expectedShouldCreate: false, + expectedShouldCreate: false, description: "should not create fetcher when execution data sync is enabled", }, { @@ -145,11 +145,11 @@ func TestCollectionSyncMode_ShouldCreateFetcher(t *testing.T) { // with all combinations of modes and execution data sync enabled/disabled. func TestCollectionSyncMode_ShouldCreateExecutionDataProcessor(t *testing.T) { tests := []struct { - name string - mode CollectionSyncMode - executionDataSyncEnabled bool - expectedShouldCreate bool - description string + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string }{ { name: "execution_first_with_execution_data_sync_enabled", @@ -162,7 +162,7 @@ func TestCollectionSyncMode_ShouldCreateExecutionDataProcessor(t *testing.T) { name: "execution_first_with_execution_data_sync_disabled", mode: CollectionSyncModeExecutionFirst, executionDataSyncEnabled: false, - expectedShouldCreate: false, + expectedShouldCreate: false, description: "should not create processor when execution data sync is disabled", }, { @@ -202,4 +202,3 @@ func TestCollectionSyncMode_ShouldCreateExecutionDataProcessor(t *testing.T) { }) } } - From 743e7b2d4bcf022230852e0f52b18de1a297c850 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 15:38:46 -0800 Subject: [PATCH 079/126] fix tests --- engine/access/collection_sync/indexer/indexer_test.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/engine/access/collection_sync/indexer/indexer_test.go b/engine/access/collection_sync/indexer/indexer_test.go index 812d70ebd47..2fd8b972b70 100644 --- a/engine/access/collection_sync/indexer/indexer_test.go +++ b/engine/access/collection_sync/indexer/indexer_test.go @@ -123,9 +123,18 @@ func TestIndexCollectionsForBlock(t *testing.T) { collectionMetrics.AssertExpectations(t) // Step 4: Idempotency Check - Indexing again is idempotent and should not error + // Set up expectations for the second call (metrics are still called even if collections already exist) + for _, collection := range collectionList { + light := collection.Light() + collectionMetrics.On("CollectionFinalized", light).Once() + collectionMetrics.On("CollectionExecuted", light).Once() + } + err = indexer.IndexCollectionsForBlock(height, collectionList) require.NoError(t, err) + collectionMetrics.AssertExpectations(t) + // Step 4 (continued): After second indexing, GetMissingCollections should still return empty missing, err = indexer.GetMissingCollections(block) require.NoError(t, err) From af7ec617014c1f332a7c1188c6f2853d284b7dc6 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 15:59:56 -0800 Subject: [PATCH 080/126] refactor using engine.Notifier --- .../execution_data_index/processor.go | 19 +++++++------------ engine/access/finalized_indexer/processor.go | 19 +++++++------------ 2 files changed, 14 insertions(+), 24 deletions(-) diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index be3c8204c72..bb225f7d6e2 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -5,6 +5,7 @@ import ( "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -15,7 +16,7 @@ import ( type ExecutionDataProcessor struct { component.Component log zerolog.Logger - newExecutionDataIndexed chan struct{} + newExecutionDataIndexed engine.Notifier provider collection_sync.ExecutionDataProvider indexer collection_sync.BlockCollectionIndexer metrics module.CollectionSyncMetrics @@ -36,16 +37,16 @@ func NewExecutionDataProcessor( ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ log: log.With().Str("component", "coll_sync_ed_processor").Logger(), - newExecutionDataIndexed: make(chan struct{}, 1), + newExecutionDataIndexed: engine.NewNotifier(), provider: provider, indexer: indexer, metrics: metrics, processedHeight: processedHeight, } - // Initialize the channel so that even if no new execution data comes in, + // Initialize the notifier so that even if no new execution data comes in, // the worker loop can still be triggered to process any existing data. - edp.newExecutionDataIndexed <- struct{}{} + edp.newExecutionDataIndexed.Notify() // Build component manager with worker loop cm := component.NewComponentManagerBuilder(). @@ -58,13 +59,7 @@ func NewExecutionDataProcessor( } func (edp *ExecutionDataProcessor) OnNewExectuionData() { - select { - case edp.newExecutionDataIndexed <- struct{}{}: - default: - // if the channel is full, no need to block, just return. - // once the worker loop processes the buffered signal, it will - // process the next height all the way to the highest available height. - } + edp.newExecutionDataIndexed.Notify() } func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { @@ -77,7 +72,7 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, select { case <-ctx.Done(): return - case <-edp.newExecutionDataIndexed: + case <-edp.newExecutionDataIndexed.Channel(): highestAvailableHeight := edp.provider.HighestIndexedHeight() lowestMissing := edp.processedHeight.Value() + 1 diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go index 235e4f2067a..b7ec6e384b4 100644 --- a/engine/access/finalized_indexer/processor.go +++ b/engine/access/finalized_indexer/processor.go @@ -8,6 +8,7 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" @@ -36,7 +37,7 @@ type FinalizedBlockProcessor struct { log zerolog.Logger component.Component - newBlockFinalized chan struct{} + newBlockFinalized engine.Notifier state protocol.State blocks storage.Blocks db storage.DB @@ -73,7 +74,7 @@ func NewFinalizedBlockProcessor( processor := &FinalizedBlockProcessor{ log: log.With().Str("component", "finalized_block_processor").Logger(), - newBlockFinalized: make(chan struct{}, 1), + newBlockFinalized: engine.NewNotifier(), state: state, db: db, lockManager: lockManager, @@ -82,18 +83,12 @@ func NewFinalizedBlockProcessor( collectionExecutedMetric: collectionExecutedMetric, } - // Initialize the channel so that even if no new blocks are finalized, + // Initialize the notifier so that even if no new blocks are finalized, // the worker loop can still be triggered to process any existing blocks. - processor.newBlockFinalized <- struct{}{} + processor.newBlockFinalized.Notify() distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { - select { - case processor.newBlockFinalized <- struct{}{}: - default: - // if the channel is full, no need to block, just return. - // once the worker loop processes the buffered signal, it will - // process the next height all the way to the highest available height. - } + processor.newBlockFinalized.Notify() }) // Build component manager with worker loop @@ -117,7 +112,7 @@ func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, select { case <-ctx.Done(): return - case <-p.newBlockFinalized: + case <-p.newBlockFinalized.Channel(): finalizedHeader, err := p.state.Final().Head() if err != nil { ctx.Throw(fmt.Errorf("failed to get finalized block header: %w", err)) From 237973d50e5562def95324eaf68afe6dbebcc5e0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 16:16:16 -0800 Subject: [PATCH 081/126] address review comments --- .../collection_sync/factory/access_builder.go | 2 -- engine/access/collection_sync/factory/fetcher.go | 4 ---- engine/access/collection_sync/fetcher/engine.go | 13 +++++++++---- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 0766cdb1da6..43e3054c3e5 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -276,11 +276,9 @@ func CreateCollectionSyncFetcherComponent( me, blocks, guarantees, - db, blockCollectionIndexer, fetchAndIndexedCollectionsBlockHeight, followerDistributor, - collectionExecutedMetric, collectionSyncMetrics, CreateFetcherConfig{ MaxProcessing: maxProcessing, diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index f8ea7764a10..8dcefae8634 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -41,9 +41,7 @@ type CreateFetcherConfig struct { // - me: Local node identity // - blocks: Blocks storage // - guarantees: Guarantees storage -// - db: Database for storage operations // - processedFinalizedBlockHeight: Initializer for tracking processed block heights -// - collectionExecutedMetric: Metrics collector for tracking collection indexing // - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress // - config: Configuration for the fetcher // @@ -58,11 +56,9 @@ func createFetcher( me module.Local, blocks storage.Blocks, guarantees storage.Guarantees, - db storage.DB, indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, distributor hotstuff.Distributor, - collectionExecutedMetric module.CollectionExecutedMetric, collectionSyncMetrics module.CollectionSyncMetrics, config CreateFetcherConfig, ) (*requester.Engine, collection_sync.Fetcher, error) { diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index eba803fbeea..fc69bfadc8c 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -62,6 +62,15 @@ func NewFetcher( jobs := jobqueue.NewFinalizedBlockReader(state, blocks) // Create an adapter function that wraps the BlockProcessor interface + // For each finalized block, a worker is responsible for fetching all collections for that block. + // Since a block may contain multiple collections and fetching is asynchronous, tracking which + // collections have been received requires stateful management. The blockProcessor handles this + // by fetching collections and maintaining state about which collections are still missing. + // It also invokes the done callback to notify the job consumer when the worker has completed + // processing all collections for the block. + // + // The processor function translates a job (finalized block) and passes it to the blockProcessor + // to fetch and index the associated collections. processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { // Convert job to block block, err := jobs.ConvertJobToBlock(job) @@ -98,10 +107,6 @@ func NewFetcher( metrics: metrics, } - if metrics == nil { - return nil, fmt.Errorf("collection sync metrics not provided") - } - // Set up post-notifier to update metrics when a job is done // Only update metrics when the processed height actually changes, since processedIndex // only advances when consecutive jobs complete, not on every individual job completion. From a31eb4fabe54d5339bfbc1473b88e378d9736c44 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 16:22:46 -0800 Subject: [PATCH 082/126] use sealed root height --- engine/access/collection_sync/fetcher/engine.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index fc69bfadc8c..edc5a52c415 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -54,8 +54,8 @@ func NewFetcher( ) (*Fetcher, error) { workSignal := engine.NewNotifier() - // Read the default index from the finalized root height - defaultIndex := state.Params().FinalizedRoot().Height + // Read the default index from the sealed root height + defaultIndex := state.Params().SealedRoot().Height // Create a Jobs instance that reads finalized blocks by height // each job is a finalized block From a6d6c08105178ad715038e770e646aaec206254a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 16:27:15 -0800 Subject: [PATCH 083/126] update progress reader --- .../collection_sync/factory/progress_reader.go | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/engine/access/collection_sync/factory/progress_reader.go b/engine/access/collection_sync/factory/progress_reader.go index 9c3fa71dfea..1756b2b23c8 100644 --- a/engine/access/collection_sync/factory/progress_reader.go +++ b/engine/access/collection_sync/factory/progress_reader.go @@ -3,9 +3,11 @@ package factory import "github.com/onflow/flow-go/engine/access/collection_sync" // ProgressReader aggregates progress from multiple backends and returns the maximum -// processed height. It can be initialized with an optional lastProgress value and +// processed height. It can be initialized with an readonly lastProgress value and // two optional backends: executionDataProcessor and collectionFetcher. type ProgressReader struct { + // this is a readonly value set at initialization so that if ProcessedHeight was + // called before any backend was set, it can still return a meaningful value lastProgress uint64 executionDataProcessor collection_sync.ProgressReader collectionFetcher collection_sync.ProgressReader @@ -44,14 +46,7 @@ func (pr *ProgressReader) ProcessedHeight() uint64 { if hasExecutionData && hasCollectionFetcher { execHeight := pr.executionDataProcessor.ProcessedHeight() collectionHeight := pr.collectionFetcher.ProcessedHeight() - max := execHeight - if collectionHeight > max { - max = collectionHeight - } - if pr.lastProgress > max { - max = pr.lastProgress - } - return max + return max(execHeight, collectionHeight, pr.lastProgress) } if hasExecutionData { From aedc1db374cbcea9a887346ef4478c3c6f3652ed Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 16:38:40 -0800 Subject: [PATCH 084/126] update comments --- .../execution_data_index/processor.go | 18 +++++++++--------- .../execution_data_index/provider.go | 3 +++ .../access/collection_sync/factory/fetcher.go | 4 ---- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index bb225f7d6e2..7a8b145517c 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -77,20 +77,18 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, lowestMissing := edp.processedHeight.Value() + 1 for height := lowestMissing; height <= highestAvailableHeight; height++ { + // TODO: This logic supports ingesting execution data from sealed blocks. Once support is + // added for syncing execution data for unsealed results, this logic will need to be updated + // to account for execution forks. collections, err := edp.provider.GetExecutionDataByHeight(ctx, height) if err != nil { ctx.Throw(fmt.Errorf("failed to get execution data for height %d: %w", height, err)) return } - // TODO: since both collections and execution data processor are the data source of - // collections, before indexing the collections, double check if it was indexed - // by the collections already by simply comparing the missing height with the - // collections's lowest height. - // if collections's lowest height is higher than the missing height, it means the collections - // has been indexed by the collections already, no need to index again. - // And make sure reading the collections's lowest height is cheap operation (only hitting RW lock) - + // Note: the collections might have been indexed by fetcher engine already, + // but IndexCollectionsForBlock will handle deduplication by first check if the collections already exist, + // if so, it will skip indexing them again. err = edp.indexer.IndexCollectionsForBlock(height, collections) if err != nil { ctx.Throw(fmt.Errorf("failed to index collections for block height %d: %w", height, err)) @@ -105,7 +103,7 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, } // Log progress for each height with all relevant information - edp.log.Info(). + edp.log.Debug(). Uint64("indexed", height). Uint64("lowest_missing", lowestMissing). Uint64("highest_available", highestAvailableHeight). @@ -120,6 +118,8 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, } } +// ProcessedHeight returns the highest consecutive height for which execution data has been processed, +// meaning the collections for that height have been indexed. func (edp *ExecutionDataProcessor) ProcessedHeight() uint64 { return edp.processedHeight.Value() } diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go index ab832c50fdb..82635006123 100644 --- a/engine/access/collection_sync/execution_data_index/provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -41,5 +41,8 @@ func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, he return nil, err } + // TODO: to support processing data from unsealed blocks, we should check here that the collections + // returned match the guarantees in the block. + return blockExecutionData.StandardCollections(), nil } diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 8dcefae8634..c3d7034ef53 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -28,10 +28,6 @@ type CreateFetcherConfig struct { } // createFetcher creates a new Fetcher component with all its dependencies. -// This function is in the collections package to avoid import cycles: -// - collections package already imports collection_sync (for interfaces) -// - CreateFetcher needs to create concrete types from collections package -// - Placing it in collection_sync would create: collection_sync -> collections -> collection_sync (cycle) // // Parameters: // - log: Logger for the component From 6578186deff7a95c761e95252a5ae704304467ab Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 16:45:03 -0800 Subject: [PATCH 085/126] use verification node in execution integration tests --- integration/tests/execution/suite.go | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/integration/tests/execution/suite.go b/integration/tests/execution/suite.go index 41b14ca8893..54c405b1769 100644 --- a/integration/tests/execution/suite.go +++ b/integration/tests/execution/suite.go @@ -114,17 +114,7 @@ func (s *Suite) SetupTest() { s.nodeConfigs = nil s.accessClient = nil - s.nodeConfigs = append(s.nodeConfigs, testnet.NewNodeConfig(flow.RoleAccess, - // this integration tests didn't include verification node, so - // no block will be sealed. Since AN only sync execution data for sealed block, - // the collection syncing which relys on execution data sync will not work. - // and the collection syncing is required to report the correct transaction results, - // which the integration test need on (WaitForExecuted), otherwise the tx will stuck - // on "Status: Pending" - // we disable execution data sync on AN so that AN will fetch collections from LN instead. - testnet.WithAdditionalFlag("--execution-data-sync-enabled=false"), - ), - ) + s.nodeConfigs = append(s.nodeConfigs, testnet.NewNodeConfig(flow.RoleAccess)) // generate the four consensus identities s.nodeIDs = unittest.IdentifierListFixture(4) @@ -154,11 +144,10 @@ func (s *Suite) SetupTest() { ) s.nodeConfigs = append(s.nodeConfigs, coll1Config, coll2Config) - // add the ghost (verification) node config + // add the verification node config s.ghostID = unittest.IdentifierFixture() ghostConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithID(s.ghostID), - testnet.AsGhost(), testnet.WithLogLevel(zerolog.InfoLevel)) s.nodeConfigs = append(s.nodeConfigs, ghostConfig) From 8f08af373b4e99c103c27829e578cf6a7f6db072 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 21 Nov 2025 17:16:34 -0800 Subject: [PATCH 086/126] fix execution integration tests --- integration/tests/execution/suite.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration/tests/execution/suite.go b/integration/tests/execution/suite.go index 54c405b1769..3a2c3e2062a 100644 --- a/integration/tests/execution/suite.go +++ b/integration/tests/execution/suite.go @@ -144,10 +144,12 @@ func (s *Suite) SetupTest() { ) s.nodeConfigs = append(s.nodeConfigs, coll1Config, coll2Config) - // add the verification node config + // add the verification node config as ghost + // note: no block will be sealed without verification node s.ghostID = unittest.IdentifierFixture() ghostConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithID(s.ghostID), + testnet.AsGhost(), testnet.WithLogLevel(zerolog.InfoLevel)) s.nodeConfigs = append(s.nodeConfigs, ghostConfig) From f555c73343d175e2414be2787e8eb8e2e10ab9a1 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 22 Nov 2025 11:24:35 -0800 Subject: [PATCH 087/126] refactor missing collections queue --- .../fetcher/missing_collection_queue.go | 56 +++++++++++++++---- 1 file changed, 44 insertions(+), 12 deletions(-) diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 6206c46fa90..0ef64f6c373 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -62,7 +62,7 @@ func NewMissingCollectionQueue() *MissingCollectionQueue { // The caller is responsible for checking if collections are already in storage before calling this method. // Only collections that are actually missing should be passed in collectionIDs. // -// If the same block height is enqueued multiple times, the previous callback is replaced. +// Returns an error if the block height is already enqueued to prevent overwriting existing jobs. func (mcq *MissingCollectionQueue) EnqueueMissingCollections( blockHeight uint64, collectionIDs []flow.Identifier, @@ -71,6 +71,14 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( mcq.mu.Lock() defer mcq.mu.Unlock() + // Check if block height is already enqueued to prevent overwriting. + if _, exists := mcq.blockJobs[blockHeight]; exists { + return fmt.Errorf( + "block height %d is already enqueued, cannot overwrite existing job", + blockHeight, + ) + } + // Create the job state with all collections marked as missing. // The caller has already verified these collections are not in storage. missingSet := make(map[flow.Identifier]struct{}, len(collectionIDs)) @@ -178,35 +186,59 @@ func (mcq *MissingCollectionQueue) IsHeightQueued(height uint64) bool { // (nil, false) if the height was not tracked. // // Note, caller should invoke the returned callback if not nil. +// +// Behavior: OnIndexedForBlock can return the callback even before the block is +// complete (i.e., before all collections have been received). This allows the caller +// to index a block with partial collections if needed. After indexing: +// - The block is removed from tracking (IsHeightQueued returns false) +// - All collection-to-height mappings for this block are cleaned up +// - Any remaining missing collections are removed from tracking +// - Subsequent OnReceivedCollection calls for collections belonging to this block +// will return (nil, 0, false) because the block has been removed func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func(), bool) { mcq.mu.Lock() defer mcq.mu.Unlock() - jobState, exists := mcq.blockJobs[blockHeight] + // Clean up all collection-to-height mappings and remove the block job. + // This ensures the height is removed from tracking once the callback is invoked. + jobState, exists := mcq.cleanupCollectionMappingsForHeight(blockHeight) if !exists { // Block was not tracked or already completed return nil, false } - // Clean up all collection-to-height mappings for collections belonging to this block. + // Get the callback from the job state. + return jobState.callback, true +} + +// cleanupCollectionMappingsForHeight removes all collection-to-height mappings for collections +// belonging to the specified block height and removes the block job from tracking. +// This includes both missing and received collections. +// +// Returns the job state and a bool indicating if the block height existed. +// +// This method must be called while holding the write lock (mcq.mu.Lock()). +func (mcq *MissingCollectionQueue) cleanupCollectionMappingsForHeight( + blockHeight uint64, +) (*blockJobState, bool) { + jobState, exists := mcq.blockJobs[blockHeight] + if !exists { + return nil, false + } + // Clean up from missing collections. for collectionID := range jobState.missingCollections { - if height, ok := mcq.collectionToHeight[collectionID]; ok && height == blockHeight { - delete(mcq.collectionToHeight, collectionID) - } + delete(mcq.collectionToHeight, collectionID) } // Clean up from received collections. for collectionID := range jobState.receivedCollections { - if height, ok := mcq.collectionToHeight[collectionID]; ok && height == blockHeight { - delete(mcq.collectionToHeight, collectionID) - } + delete(mcq.collectionToHeight, collectionID) } - // Remove the job state from the queue before calling the callback. - // This ensures the height is removed from tracking once the callback is invoked. + // Remove the job state from the queue. delete(mcq.blockJobs, blockHeight) - return jobState.callback, true + return jobState, true } // Size returns the number of missing collections currently in the queue. From 768f401d9d591875b09cd086a78719876c16994c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 22 Nov 2025 12:13:29 -0800 Subject: [PATCH 088/126] add pruner for missing collections queue --- .../fetcher/missing_collection_queue.go | 113 ++++++++++++++++-- 1 file changed, 101 insertions(+), 12 deletions(-) diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 0ef64f6c373..d2cad8840aa 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -8,17 +8,6 @@ import ( "github.com/onflow/flow-go/model/flow" ) -// blockJobState tracks the state of a job for a specific block height. -type blockJobState struct { - // missingCollections is a set of collection IDs that are still missing for this block height. - missingCollections map[flow.Identifier]struct{} - // receivedCollections stores the collections that have been received so far, keyed by collection ID. - // This allows us to return all collections when the block becomes complete. - receivedCollections map[flow.Identifier]*flow.Collection - // callback is invoked when all collections for this block height have been received and indexed. - callback func() -} - // MissingCollectionQueue helps the job processor to keep track of the jobs and their callbacks. // Note, it DOES NOT index collections directly, instead, it only keeps track of which collections are missing // for each block height, and when all collections for a block height have been received, it returns the @@ -42,6 +31,38 @@ type MissingCollectionQueue struct { // This enforces a 1:1 relationship: each collection belongs to exactly one block. // This allows efficient lookup when a collection is received. collectionToHeight map[flow.Identifier]uint64 + + // pruner keeps track of the lowest and highest missing heights for pruning. + // it is updated whenever a new height is added or removed from blockJobs. + // this is useful because: + // 1. it prevents memory leak by allowing us to prune old heights that are no longer needed. + // 2. if missing collections has been indexed by the execution data index processor, then + // we don't need to fetcher to fetch them, and we can prune the jobs up to that height. + pruner *pruner +} + +// blockJobState tracks the state of a job for a specific block height. +type blockJobState struct { + // missingCollections is a set of collection IDs that are still missing for this block height. + missingCollections map[flow.Identifier]struct{} + // receivedCollections stores the collections that have been received so far, keyed by collection ID. + // This allows us to return all collections when the block becomes complete. + receivedCollections map[flow.Identifier]*flow.Collection + // callback is invoked when all collections for this block height have been received and indexed. + callback func() +} + +// pruner keeps track of the lowest and highest missing heights to decide +// what range can be pruned. +type pruner struct { + lowest uint64 // lowest missing height + highest uint64 // highest missing height +} + +// pruneRange represents a range of heights to prune, inclusive. +type pruneRange struct { + from uint64 + to uint64 } var _ collection_sync.MissingCollectionQueue = (*MissingCollectionQueue)(nil) @@ -53,6 +74,7 @@ func NewMissingCollectionQueue() *MissingCollectionQueue { return &MissingCollectionQueue{ blockJobs: make(map[uint64]*blockJobState), collectionToHeight: make(map[flow.Identifier]uint64), + pruner: &pruner{lowest: 0, highest: 0}, } } @@ -93,6 +115,7 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( } mcq.blockJobs[blockHeight] = jobState + mcq.pruner.AddHeight(blockHeight) // Update the collection-to-height mapping, enforcing 1:1 relationship. for _, collectionID := range collectionIDs { @@ -100,7 +123,7 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( if exists && existingHeight != blockHeight { // Collection is already assigned to a different block - this violates the 1:1 constraint. return fmt.Errorf( - "collection %v is already assigned to block height %d, cannot assign to height %d", + "fatal: collection %v is already assigned to block height %d, cannot assign to height %d", collectionID, existingHeight, blockHeight, ) } @@ -199,6 +222,7 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func() mcq.mu.Lock() defer mcq.mu.Unlock() + mcq.pruner.RemoveHeight(blockHeight) // Clean up all collection-to-height mappings and remove the block job. // This ensures the height is removed from tracking once the callback is invoked. jobState, exists := mcq.cleanupCollectionMappingsForHeight(blockHeight) @@ -237,10 +261,41 @@ func (mcq *MissingCollectionQueue) cleanupCollectionMappingsForHeight( // Remove the job state from the queue. delete(mcq.blockJobs, blockHeight) + mcq.pruner.RemoveHeight(blockHeight) return jobState, true } +// PruneUpToHeight removes all block jobs up to and including the specified height +// and returns their callbacks. The pruner ensures that heights are only pruned once. +// +// Returns a slice of callbacks for all pruned block heights. Callers should invoke +// these callbacks to notify that the blocks have been pruned. +func (mcq *MissingCollectionQueue) PruneUpToHeight(height uint64) []func() { + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Use pruner to determine the range to prune. + pruneRange, shouldPrune := mcq.pruner.RemoveUpToHeight(height) + if !shouldPrune { + return nil + } + + // Calculate the maximum number of callbacks (upper bound: all heights in range may have jobs). + capacity := max(0, int(pruneRange.to-pruneRange.from+1)) + + // Collect callbacks for all heights in the prune range. + callbacks := make([]func(), 0, capacity) + for h := pruneRange.from; h <= pruneRange.to; h++ { + jobState, exists := mcq.cleanupCollectionMappingsForHeight(h) + if exists { + callbacks = append(callbacks, jobState.callback) + } + } + + return callbacks +} + // Size returns the number of missing collections currently in the queue. // This is the total number of collections across all block heights that are still missing. func (mcq *MissingCollectionQueue) Size() uint { @@ -249,3 +304,37 @@ func (mcq *MissingCollectionQueue) Size() uint { return uint(len(mcq.collectionToHeight)) } + +// AddHeight updates the pruner with a new height. +func (pc *pruner) AddHeight(height uint64) { + pc.lowest = min(height, pc.lowest) + pc.highest = max(height, pc.highest) +} + +// RemoveHeight removes a single height from tracking and +// advances the lowest height if needed. +func (pc *pruner) RemoveHeight(height uint64) { + if height == pc.lowest && + pc.lowest < pc.highest { // only advance if there are higher heights + pc.lowest++ + } +} + +// RemoveUpToHeight removes all heights up to and including the specified height +// and returns the range that was pruned. +func (pc *pruner) RemoveUpToHeight(highestConsecutiveCompletedHeight uint64) (pruneRange, bool) { + if highestConsecutiveCompletedHeight <= pc.lowest { + // No update needed + return pruneRange{}, false + } + + pruneFrom := pc.lowest + pruneTo := highestConsecutiveCompletedHeight + + pc.lowest = highestConsecutiveCompletedHeight + 1 + if highestConsecutiveCompletedHeight > pc.highest { + pc.highest = highestConsecutiveCompletedHeight + } + + return pruneRange{from: pruneFrom, to: pruneTo}, true +} From afbf8bc1c64a9bade70a6cd7aedee3c66d437f0b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 22 Nov 2025 12:43:12 -0800 Subject: [PATCH 089/126] add test cases for missing collection queue --- .../fetcher/missing_collection_queue_test.go | 146 ++++++++++++++++++ 1 file changed, 146 insertions(+) create mode 100644 engine/access/collection_sync/fetcher/missing_collection_queue_test.go diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go new file mode 100644 index 00000000000..bb8d8be5660 --- /dev/null +++ b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go @@ -0,0 +1,146 @@ +package fetcher + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +// TestMissingCollectionQueue_CompleteBlockLifecycle tests the complete lifecycle of a block: +// 1. Initially IsHeightQueued returns false +// 2. After enqueuing, IsHeightQueued returns true +// 3. Enqueuing twice should error (not idempotent - prevents overwriting) +// 4. Receiving one collection doesn't complete the block, IsHeightQueued still returns true +// 5. Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) +// 6. OnIndexedForBlock returns the callback +// 7. After indexing, IsHeightQueued returns false +// 8. OnReceivedCollection for that block returns false because the block has been removed +func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + + // Create test collections + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackInvoked := false + callback := func() { + callbackInvoked = true + } + + // Step 1: Initially IsHeightQueued returns false + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued initially") + + // Step 2: After enqueuing, IsHeightQueued returns true + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should be queued after enqueuing") + + // Step 3: Enqueuing twice should error (prevents overwriting) + anotherCallback := func() {} + err = mcq.EnqueueMissingCollections(blockHeight, collectionIDs, anotherCallback) + require.Error(t, err) + assert.Contains(t, err.Error(), "already enqueued") + assert.Contains(t, err.Error(), "cannot overwrite existing job") + + // Step 4: Receiving one collection doesn't complete the block, IsHeightQueued still returns true + collectionsReturned, heightReturned, complete := mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") + assert.False(t, complete, "block should not be complete with only one collection") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued after receiving one collection") + + // Step 5: Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) + collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[1]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.False(t, complete, "block should not be complete with only two collections") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") + + // Receive the last collection - block should now be complete + collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[2]) + assert.NotNil(t, collectionsReturned, "should return collections when block is complete") + assert.Equal(t, 3, len(collectionsReturned), "should return all 3 collections") + assert.Equal(t, blockHeight, heightReturned, "should return correct block height") + assert.True(t, complete, "block should be complete after receiving all collections") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued (not yet indexed)") + + // Step 6: OnIndexedForBlock returns the callback + returnedCallback, exists := mcq.OnIndexedForBlock(blockHeight) + assert.NotNil(t, returnedCallback, "should return callback") + assert.True(t, exists, "block should exist") + assert.False(t, callbackInvoked, "callback should not be invoked yet") + + // Step 7: After indexing, IsHeightQueued returns false + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued after indexing") + + // Invoke the callback + returnedCallback() + assert.True(t, callbackInvoked, "callback should be invoked") + + // Step 8: OnReceivedCollection for that block returns false because the block has been removed + collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") + assert.False(t, complete, "should not indicate completion for removed block") +} + +// TestMissingCollectionQueue_IndexBeforeBlockCompletion tests that OnIndexedForBlock +// can return the callback even before the block is complete (i.e., before all collections have been received). +func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { + mcq := NewMissingCollectionQueue() + blockHeight := uint64(200) + + // Create test collections + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackInvoked := false + callback := func() { + callbackInvoked = true + } + + // Enqueue block with 3 collections + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should be queued") + + // Receive only one collection (block is not complete) + collectionsReturned, heightReturned, complete := mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.False(t, complete, "block should not be complete with only one collection") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") + + // OnIndexedForBlock can return the callback even before the block is complete + returnedCallback, exists := mcq.OnIndexedForBlock(blockHeight) + assert.NotNil(t, returnedCallback, "should return callback even when block is not complete") + assert.True(t, exists, "block should exist") + assert.False(t, callbackInvoked, "callback should not be invoked yet") + + // After indexing, the block is removed from tracking + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued after indexing") + + // Verify that remaining collections cannot be received (block has been removed) + collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[1]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") + assert.False(t, complete, "should not indicate completion for removed block") + + collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[2]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.False(t, complete, "should not indicate completion for removed block") + + // Invoke the callback + returnedCallback() + assert.True(t, callbackInvoked, "callback should be invoked") +} + From 3dcb69203a8654c6949f76ca9355f583e896501e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 22 Nov 2025 15:45:40 -0800 Subject: [PATCH 090/126] refactor pruner --- .../access/collection_sync/collection_sync.go | 11 +- .../fetcher/block_processor.go | 9 ++ .../fetcher/missing_collection_queue.go | 107 +++--------------- 3 files changed, 36 insertions(+), 91 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 32ab29b5cb6..0b6d35042dd 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -10,7 +10,10 @@ import ( // Tracks missing collections per height and invokes job callbacks when complete. type MissingCollectionQueue interface { + // EnqueueMissingCollections tracks the given missing collection IDs for the given block height. EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error + + // OnIndexedForBlock returns the callback function for the given block height OnIndexedForBlock(blockHeight uint64) (func(), bool) // On receipt of a collection, MCQ updates internal state and, if a block @@ -18,6 +21,9 @@ type MissingCollectionQueue interface { // Otherwise, returns (nil, 0, false). OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, bool) + // PruneUpToHeight removes all tracked heights up to and including the given height. + PruneUpToHeight(height uint64) (callbacks []func()) + // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). IsHeightQueued(height uint64) bool @@ -42,11 +48,14 @@ type BlockCollectionIndexer interface { GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) } -// Implements the job lifecycle for a single block height. +// BlockProcessor processes blocks to fetch and index their collections. type BlockProcessor interface { + // RequestCollectionsForBlock requests all missing collections for the given block. FetchCollections(ctx irrecoverable.SignalerContext, block *flow.Block, done func()) error // MissingCollectionQueueSize returns the number of missing collections currently in the queue. MissingCollectionQueueSize() uint + // PruneUpToHeight removes all tracked heights up to and including the given height. + PruneUpToHeight(height uint64) } // Fetcher is a component that consumes finalized block jobs and processes them diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 93c849d410d..c725e0e26b8 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -150,3 +150,12 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti func (bp *BlockProcessor) MissingCollectionQueueSize() uint { return bp.mcq.Size() } + +// PruneUpToHeight removes all tracked heights up to and including the given height. +func (bp *BlockProcessor) PruneUpToHeight(height uint64) { + callbacks := bp.mcq.PruneUpToHeight(height) + // notify job completion + for _, cb := range callbacks { + cb() + } +} diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index d2cad8840aa..06735f57f80 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -31,14 +31,6 @@ type MissingCollectionQueue struct { // This enforces a 1:1 relationship: each collection belongs to exactly one block. // This allows efficient lookup when a collection is received. collectionToHeight map[flow.Identifier]uint64 - - // pruner keeps track of the lowest and highest missing heights for pruning. - // it is updated whenever a new height is added or removed from blockJobs. - // this is useful because: - // 1. it prevents memory leak by allowing us to prune old heights that are no longer needed. - // 2. if missing collections has been indexed by the execution data index processor, then - // we don't need to fetcher to fetch them, and we can prune the jobs up to that height. - pruner *pruner } // blockJobState tracks the state of a job for a specific block height. @@ -52,19 +44,6 @@ type blockJobState struct { callback func() } -// pruner keeps track of the lowest and highest missing heights to decide -// what range can be pruned. -type pruner struct { - lowest uint64 // lowest missing height - highest uint64 // highest missing height -} - -// pruneRange represents a range of heights to prune, inclusive. -type pruneRange struct { - from uint64 - to uint64 -} - var _ collection_sync.MissingCollectionQueue = (*MissingCollectionQueue)(nil) // NewMissingCollectionQueue creates a new MissingCollectionQueue. @@ -74,7 +53,6 @@ func NewMissingCollectionQueue() *MissingCollectionQueue { return &MissingCollectionQueue{ blockJobs: make(map[uint64]*blockJobState), collectionToHeight: make(map[flow.Identifier]uint64), - pruner: &pruner{lowest: 0, highest: 0}, } } @@ -115,7 +93,6 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( } mcq.blockJobs[blockHeight] = jobState - mcq.pruner.AddHeight(blockHeight) // Update the collection-to-height mapping, enforcing 1:1 relationship. for _, collectionID := range collectionIDs { @@ -222,7 +199,6 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func() mcq.mu.Lock() defer mcq.mu.Unlock() - mcq.pruner.RemoveHeight(blockHeight) // Clean up all collection-to-height mappings and remove the block job. // This ensures the height is removed from tracking once the callback is invoked. jobState, exists := mcq.cleanupCollectionMappingsForHeight(blockHeight) @@ -235,6 +211,23 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func() return jobState.callback, true } +// PruneUpToHeight removes all block jobs and their collection mappings +// for block heights less than or equal to the specified height. +func (mcq *MissingCollectionQueue) PruneUpToHeight(height uint64) []func() { + mcq.mu.Lock() + defer mcq.mu.Unlock() + + callbacks := make([]func(), 0, len(mcq.blockJobs)) + for blockHeight := range mcq.blockJobs { + if blockHeight <= height { + jobState, _ := mcq.cleanupCollectionMappingsForHeight(blockHeight) + callbacks = append(callbacks, jobState.callback) + } + } + + return callbacks +} + // cleanupCollectionMappingsForHeight removes all collection-to-height mappings for collections // belonging to the specified block height and removes the block job from tracking. // This includes both missing and received collections. @@ -259,43 +252,11 @@ func (mcq *MissingCollectionQueue) cleanupCollectionMappingsForHeight( delete(mcq.collectionToHeight, collectionID) } - // Remove the job state from the queue. delete(mcq.blockJobs, blockHeight) - mcq.pruner.RemoveHeight(blockHeight) return jobState, true } -// PruneUpToHeight removes all block jobs up to and including the specified height -// and returns their callbacks. The pruner ensures that heights are only pruned once. -// -// Returns a slice of callbacks for all pruned block heights. Callers should invoke -// these callbacks to notify that the blocks have been pruned. -func (mcq *MissingCollectionQueue) PruneUpToHeight(height uint64) []func() { - mcq.mu.Lock() - defer mcq.mu.Unlock() - - // Use pruner to determine the range to prune. - pruneRange, shouldPrune := mcq.pruner.RemoveUpToHeight(height) - if !shouldPrune { - return nil - } - - // Calculate the maximum number of callbacks (upper bound: all heights in range may have jobs). - capacity := max(0, int(pruneRange.to-pruneRange.from+1)) - - // Collect callbacks for all heights in the prune range. - callbacks := make([]func(), 0, capacity) - for h := pruneRange.from; h <= pruneRange.to; h++ { - jobState, exists := mcq.cleanupCollectionMappingsForHeight(h) - if exists { - callbacks = append(callbacks, jobState.callback) - } - } - - return callbacks -} - // Size returns the number of missing collections currently in the queue. // This is the total number of collections across all block heights that are still missing. func (mcq *MissingCollectionQueue) Size() uint { @@ -304,37 +265,3 @@ func (mcq *MissingCollectionQueue) Size() uint { return uint(len(mcq.collectionToHeight)) } - -// AddHeight updates the pruner with a new height. -func (pc *pruner) AddHeight(height uint64) { - pc.lowest = min(height, pc.lowest) - pc.highest = max(height, pc.highest) -} - -// RemoveHeight removes a single height from tracking and -// advances the lowest height if needed. -func (pc *pruner) RemoveHeight(height uint64) { - if height == pc.lowest && - pc.lowest < pc.highest { // only advance if there are higher heights - pc.lowest++ - } -} - -// RemoveUpToHeight removes all heights up to and including the specified height -// and returns the range that was pruned. -func (pc *pruner) RemoveUpToHeight(highestConsecutiveCompletedHeight uint64) (pruneRange, bool) { - if highestConsecutiveCompletedHeight <= pc.lowest { - // No update needed - return pruneRange{}, false - } - - pruneFrom := pc.lowest - pruneTo := highestConsecutiveCompletedHeight - - pc.lowest = highestConsecutiveCompletedHeight + 1 - if highestConsecutiveCompletedHeight > pc.highest { - pc.highest = highestConsecutiveCompletedHeight - } - - return pruneRange{from: pruneFrom, to: pruneTo}, true -} From 17190cb494e3fd5b001d03773e9bda5f27759b8a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 22 Nov 2025 17:14:57 -0800 Subject: [PATCH 091/126] refactor with onIndexedCallback --- .../execution_data_index/processor.go | 13 ++++++------- .../collection_sync/factory/access_builder.go | 4 +++- .../factory/execution_data_processor.go | 7 +++---- engine/access/collection_sync/factory/fetcher.go | 2 +- .../collection_sync/fetcher/block_processor.go | 2 +- engine/access/collection_sync/fetcher/engine.go | 2 +- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index 7a8b145517c..39e4962dd77 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -7,7 +7,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" @@ -19,9 +18,9 @@ type ExecutionDataProcessor struct { newExecutionDataIndexed engine.Notifier provider collection_sync.ExecutionDataProvider indexer collection_sync.BlockCollectionIndexer - metrics module.CollectionSyncMetrics // state - processedHeight *counters.PersistentStrictMonotonicCounter + processedHeight *counters.PersistentStrictMonotonicCounter + onIndexedCallback func(uint64) } var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) @@ -33,15 +32,15 @@ func NewExecutionDataProcessor( provider collection_sync.ExecutionDataProvider, indexer collection_sync.BlockCollectionIndexer, processedHeight *counters.PersistentStrictMonotonicCounter, - metrics module.CollectionSyncMetrics, + onIndexedCallback func(uint64), ) *ExecutionDataProcessor { edp := &ExecutionDataProcessor{ - log: log.With().Str("component", "coll_sync_ed_processor").Logger(), + log: log.With().Str("coll_sync", "data_processor").Logger(), newExecutionDataIndexed: engine.NewNotifier(), provider: provider, indexer: indexer, - metrics: metrics, processedHeight: processedHeight, + onIndexedCallback: onIndexedCallback, } // Initialize the notifier so that even if no new execution data comes in, @@ -112,7 +111,7 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, Uint64("total_to_process", highestAvailableHeight-lowestMissing+1). Msg("indexed execution data progress") - edp.metrics.CollectionSyncedHeight(height) + edp.onIndexedCallback(height) } } } diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 43e3054c3e5..2e9205c0019 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -172,7 +172,9 @@ func CreateExecutionDataProcessorComponent( executionDataRequester, collectionIndexedHeight, blockCollectionIndexer, - collectionSyncMetrics, + func(indexedHeight uint64) { + collectionSyncMetrics.CollectionSyncedHeight(indexedHeight) + }, ) if err != nil { return nil, fmt.Errorf("could not create execution data processor: %w", err) diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index 9d1a8e01a6f..25091b14803 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -7,7 +7,6 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" - "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" @@ -22,7 +21,7 @@ import ( // - executionDataTracker: Tracker for execution data that provides the highest available height // - processedHeight: Consumer progress for tracking processed heights // - indexer: Block collection indexer for indexing collections -// - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress +// - onIndexedCallback: Callback function to be called when a block's execution data has been indexed // // Returns: // - *ExecutionDataProcessor: A new ExecutionDataProcessor instance @@ -35,7 +34,7 @@ func createExecutionDataProcessor( executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, - collectionSyncMetrics module.CollectionSyncMetrics, // optional metrics collector + onIndexedCallback func(uint64), ) (*execution_data_index.ExecutionDataProcessor, error) { // Create execution data provider executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataIndexedHeight) @@ -47,7 +46,7 @@ func createExecutionDataProcessor( } // Create the execution data processor - processor := execution_data_index.NewExecutionDataProcessor(log, executionDataProvider, indexer, processedHeightCounter, collectionSyncMetrics) + processor := execution_data_index.NewExecutionDataProcessor(log, executionDataProvider, indexer, processedHeightCounter, onIndexedCallback) return processor, nil } diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index c3d7034ef53..3c9791a6785 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -85,7 +85,7 @@ func createFetcher( // Create BlockProcessor blockProcessor := fetcher.NewBlockProcessor( - log.With().Str("component", "coll_sync_fetcher").Logger(), + log, mcq, indexer, collectionRequester, diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index c725e0e26b8..845661cd39e 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -37,7 +37,7 @@ func NewBlockProcessor( requester collection_sync.CollectionRequester, ) *BlockProcessor { return &BlockProcessor{ - log: log.With().Str("component", "coll_fetcher").Logger(), + log: log.With().Str("coll_sync", "fetcher_processor").Logger(), mcq: mcq, indexer: indexer, requester: requester, diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index edc5a52c415..f6e862a68e2 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -86,7 +86,7 @@ func NewFetcher( } consumer, err := jobqueue.NewComponentConsumer( - log.With().Str("component", "collection-syncing").Logger(), + log.With().Str("coll_sync", "fetcher").Logger(), workSignal.Channel(), progressInitializer, jobs, From 964ee3c4aa7fdd207556ab683d181255696402ad Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 24 Nov 2025 07:17:31 -0800 Subject: [PATCH 092/126] fix lint --- .../fetcher/missing_collection_queue_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go index bb8d8be5660..a1760ca2c81 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go @@ -58,7 +58,7 @@ func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued after receiving one collection") // Step 5: Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) - collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[1]) + collectionsReturned, _, complete = mcq.OnReceivedCollection(collections[1]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") assert.False(t, complete, "block should not be complete with only two collections") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") @@ -117,6 +117,7 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { // Receive only one collection (block is not complete) collectionsReturned, heightReturned, complete := mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") assert.False(t, complete, "block should not be complete with only one collection") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") @@ -135,7 +136,7 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") assert.False(t, complete, "should not indicate completion for removed block") - collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[2]) + collectionsReturned, _, complete = mcq.OnReceivedCollection(collections[2]) assert.Nil(t, collectionsReturned, "should not return collections for removed block") assert.False(t, complete, "should not indicate completion for removed block") @@ -143,4 +144,3 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { returnedCallback() assert.True(t, callbackInvoked, "callback should be invoked") } - From ec6cdc900d07dd51565c819a50380e21fc2a185f Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 24 Nov 2025 08:43:45 -0800 Subject: [PATCH 093/126] refactor with RequestCollectionsByGuarantees --- .../node_builder/access_node_builder.go | 1 - .../access/collection_sync/collection_sync.go | 4 +- .../collection_sync/factory/access_builder.go | 3 -- .../access/collection_sync/factory/fetcher.go | 3 -- .../fetcher/block_processor.go | 4 +- .../access/collection_sync/fetcher/engine.go | 2 +- .../collection_sync/fetcher/requester.go | 37 +++++-------------- 7 files changed, 14 insertions(+), 40 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 30dd41c9bf2..b17a6208c07 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2570,7 +2570,6 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { node.State, node.Me, node.Storage.Blocks, - node.Storage.Guarantees, builder.ProtocolDB, notNil(builder.blockCollectionIndexer), builder.FollowerDistributor, diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 0b6d35042dd..80fdadf7084 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -31,9 +31,9 @@ type MissingCollectionQueue interface { Size() uint } -// Requests collections by their IDs. +// Requests collections by their guarantees. type CollectionRequester interface { - RequestCollections(ids []flow.Identifier) error + RequestCollectionsByGuarantees(guarantees []*flow.CollectionGuarantee) error } // BlockCollectionIndexer stores and indexes collections for a given block height. diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 2e9205c0019..0e638e787a5 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -207,7 +207,6 @@ type CollectionSyncFetcherComponentResult struct { // - state: Protocol state // - me: Local node identity // - blocks: Blocks storage -// - guarantees: Guarantees storage // - db: Database for storage operations // - blockCollectionIndexer: Block collection indexer // - followerDistributor: Follower distributor @@ -229,7 +228,6 @@ func CreateCollectionSyncFetcherComponent( state protocol.State, me module.Local, blocks storage.Blocks, - guarantees storage.Guarantees, db storage.DB, blockCollectionIndexer collection_sync.BlockCollectionIndexer, followerDistributor hotstuff.Distributor, @@ -277,7 +275,6 @@ func CreateCollectionSyncFetcherComponent( state, me, blocks, - guarantees, blockCollectionIndexer, fetchAndIndexedCollectionsBlockHeight, followerDistributor, diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 3c9791a6785..5a55672617b 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -36,7 +36,6 @@ type CreateFetcherConfig struct { // - state: Protocol state // - me: Local node identity // - blocks: Blocks storage -// - guarantees: Guarantees storage // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress // - config: Configuration for the fetcher @@ -51,7 +50,6 @@ func createFetcher( state protocol.State, me module.Local, blocks storage.Blocks, - guarantees storage.Guarantees, indexer collection_sync.BlockCollectionIndexer, processedFinalizedBlockHeight storage.ConsumerProgressInitializer, distributor hotstuff.Distributor, @@ -80,7 +78,6 @@ func createFetcher( collectionRequester := fetcher.NewCollectionRequester( requestEng, state, - guarantees, ) // Create BlockProcessor diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 845661cd39e..09c4c0ae697 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -76,7 +76,7 @@ func (bp *BlockProcessor) FetchCollections( return nil } - // Extract collection IDs + // Extract collection IDs for enqueueing collectionIDs := make([]flow.Identifier, len(missingGuarantees)) for i, guarantee := range missingGuarantees { collectionIDs[i] = guarantee.CollectionID @@ -93,7 +93,7 @@ func (bp *BlockProcessor) FetchCollections( } // Request collections from collection nodes - err = bp.requester.RequestCollections(collectionIDs) + err = bp.requester.RequestCollectionsByGuarantees(missingGuarantees) if err != nil { return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) } diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index f6e862a68e2..d7e4ae76aba 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -48,7 +48,7 @@ func NewFetcher( blockProcessor collection_sync.BlockProcessor, progressInitializer storage.ConsumerProgressInitializer, state protocol.State, blocks storage.Blocks, - maxProcessing uint64, // max number of blocks to fetch collections + maxProcessing uint64, // max number of blocks to fetch collections concurrently maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for metrics module.CollectionSyncMetrics, // optional metrics collector ) (*Fetcher, error) { diff --git a/engine/access/collection_sync/fetcher/requester.go b/engine/access/collection_sync/fetcher/requester.go index 64870314514..bf7b2613c2a 100644 --- a/engine/access/collection_sync/fetcher/requester.go +++ b/engine/access/collection_sync/fetcher/requester.go @@ -1,15 +1,11 @@ package fetcher import ( - "errors" - "fmt" - "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" ) var _ collection_sync.CollectionRequester = (*CollectionRequester)(nil) @@ -17,9 +13,8 @@ var _ collection_sync.CollectionRequester = (*CollectionRequester)(nil) // CollectionRequester requests collections from collection nodes on the network. // It implements the collection_sync.CollectionRequester interface. type CollectionRequester struct { - requester module.Requester - state protocol.State - guarantees storage.Guarantees + requester module.Requester + state protocol.State } // NewCollectionRequester creates a new CollectionRequester. @@ -27,38 +22,24 @@ type CollectionRequester struct { // Parameters: // - requester: The requester engine for requesting entities from the network // - state: Protocol state for finding guarantors -// - guarantees: Guarantees storage for retrieving collection guarantees by collection ID // // No error returns are expected during normal operation. func NewCollectionRequester( requester module.Requester, state protocol.State, - guarantees storage.Guarantees, ) *CollectionRequester { return &CollectionRequester{ - requester: requester, - state: state, - guarantees: guarantees, + requester: requester, + state: state, } } -// RequestCollections requests collections by their IDs from collection nodes on the network. -// For each collection, it finds the guarantors and requests the collection from them. +// RequestCollectionsByGuarantees requests collections by their guarantees from collection nodes on the network. +// For each guarantee, it finds the guarantors and requests the collection from them. // // No error returns are expected during normal operation. -func (cr *CollectionRequester) RequestCollections(ids []flow.Identifier) error { - for _, collectionID := range ids { - // Retrieve the guarantee for this collection - guarantee, err := cr.guarantees.ByCollectionID(collectionID) - if err != nil { - // If guarantee is not found, we can't determine guarantors, so skip this collection - // This can happen if the collection hasn't been finalized yet or if it's from a fork - if errors.Is(err, storage.ErrNotFound) { - continue - } - return fmt.Errorf("failed to retrieve guarantee for collection %v: %w", collectionID, err) - } - +func (cr *CollectionRequester) RequestCollectionsByGuarantees(guarantees []*flow.CollectionGuarantee) error { + for _, guarantee := range guarantees { // Find guarantors for this guarantee guarantors, err := protocol.FindGuarantors(cr.state, guarantee) if err != nil { @@ -68,7 +49,7 @@ func (cr *CollectionRequester) RequestCollections(ids []flow.Identifier) error { } // Request the collection from the guarantors - cr.requester.EntityByID(collectionID, filter.HasNodeID[flow.Identity](guarantors...)) + cr.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) } // Force immediate dispatch of all pending requests From 1436a6b9a5dd63abb050c13fb45c24131d310a5b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 24 Nov 2025 09:58:41 -0800 Subject: [PATCH 094/126] initilaize metrics to avoid spikes --- engine/access/collection_sync/factory/access_builder.go | 3 +++ engine/access/collection_sync/fetcher/engine.go | 3 +++ 2 files changed, 6 insertions(+) diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 0e638e787a5..e575280990e 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -183,6 +183,9 @@ func CreateExecutionDataProcessorComponent( // Register with ProgressReader lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) + // Initialize collection synced height metric to avoid spikes in dashboard + collectionSyncMetrics.CollectionSyncedHeight(executionDataProcessor.ProcessedHeight()) + distributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { executionDataProcessor.OnNewExectuionData() }) diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index d7e4ae76aba..9d78f83f74d 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -115,6 +115,9 @@ func NewFetcher( metrics.MissingCollectionQueueSize(f.blockProcessor.MissingCollectionQueueSize()) }) + // report the initial metrics, otherwise it creates spikes in dashboard + metrics.CollectionFetchedHeight(f.ProcessedHeight()) + return f, nil } From c7d5b5008ff9b340643707b5d0a5a1b84efdb5f0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 25 Nov 2025 12:06:25 -0800 Subject: [PATCH 095/126] use sealed root height --- engine/access/collection_sync/factory/access_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index e575280990e..5b11698434f 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -38,7 +38,7 @@ func CreateProcessedLastFullBlockHeightModule( state protocol.State, db storage.DB, ) (*ProcessedLastFullBlockHeightModuleResult, error) { - rootBlockHeight := state.Params().FinalizedRoot().Height + rootBlockHeight := state.Params().SealedRoot().Height // Initialize ConsumeProgressLastFullBlockHeight progress, err := store.NewConsumerProgress(db, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) From 1fe260735f1281a3351efd3059cdc841c58df077 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 26 Nov 2025 09:20:52 -0800 Subject: [PATCH 096/126] add back last full height metrics --- cmd/access/node_builder/access_node_builder.go | 2 ++ .../access/collection_sync/factory/access_builder.go | 10 ++++++++++ engine/access/collection_sync/factory/fetcher.go | 10 +++++++++- engine/access/collection_sync/fetcher/engine.go | 8 ++++++++ 4 files changed, 29 insertions(+), 1 deletion(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b17a6208c07..6a10c8b00b0 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2203,6 +2203,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { notNil(builder.blockCollectionIndexer), notNil(builder.CollectionSyncMetrics), notNil(builder.lastFullBlockHeight), + notNil(builder.AccessMetrics), notNil(builder.ExecutionDataDistributor), ) if err != nil { @@ -2578,6 +2579,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { builder.collectionFetcherMaxProcessing, builder.collectionFetcherMaxSearchAhead, notNil(builder.lastFullBlockHeight), + notNil(builder.AccessMetrics), ) if err != nil { return nil, err diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 5b11698434f..864eebb2e16 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -121,6 +121,7 @@ func CreateProcessedLastFullBlockHeightModule( // - blockCollectionIndexer: Block collection indexer // - collectionSyncMetrics: Collection sync metrics // - lastFullBlockHeight: Progress reader to register the processor with +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height // - distributor: Execution data distributor to notify on new execution data // // Returns: @@ -136,6 +137,7 @@ func CreateExecutionDataProcessorComponent( blockCollectionIndexer collection_sync.BlockCollectionIndexer, collectionSyncMetrics module.CollectionSyncMetrics, lastFullBlockHeight *ProgressReader, + accessMetrics module.AccessMetrics, distributor *edrequester.ExecutionDataDistributor, ) (module.ReadyDoneAware, error) { shouldCreate := collectionSyncMode.ShouldCreateExecutionDataProcessor(executionDataSyncEnabled) @@ -174,6 +176,8 @@ func CreateExecutionDataProcessorComponent( blockCollectionIndexer, func(indexedHeight uint64) { collectionSyncMetrics.CollectionSyncedHeight(indexedHeight) + // Also update last_full_finalized_block_height metric with the max of both heights + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) }, ) if err != nil { @@ -185,6 +189,8 @@ func CreateExecutionDataProcessorComponent( // Initialize collection synced height metric to avoid spikes in dashboard collectionSyncMetrics.CollectionSyncedHeight(executionDataProcessor.ProcessedHeight()) + // Initialize last_full_finalized_block_height metric + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) distributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { executionDataProcessor.OnNewExectuionData() @@ -218,6 +224,7 @@ type CollectionSyncFetcherComponentResult struct { // - maxProcessing: Maximum number of concurrent processing jobs // - maxSearchAhead: Maximum number of blocks to search ahead // - lastFullBlockHeight: Progress reader to register the fetcher with +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height // // Returns: // - The result containing the fetcher component, requester component, and requester engine @@ -239,6 +246,7 @@ func CreateCollectionSyncFetcherComponent( maxProcessing uint64, maxSearchAhead uint64, lastFullBlockHeight *ProgressReader, + accessMetrics module.AccessMetrics, ) (*CollectionSyncFetcherComponentResult, error) { // Create fetcher if: // 1. collectionSync is "execution_and_collection" (always create, even with execution data sync) @@ -282,6 +290,8 @@ func CreateCollectionSyncFetcherComponent( fetchAndIndexedCollectionsBlockHeight, followerDistributor, collectionSyncMetrics, + lastFullBlockHeight, + accessMetrics, CreateFetcherConfig{ MaxProcessing: maxProcessing, MaxSearchAhead: maxSearchAhead, diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 5a55672617b..c0e0caaf447 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -38,6 +38,8 @@ type CreateFetcherConfig struct { // - blocks: Blocks storage // - processedFinalizedBlockHeight: Initializer for tracking processed block heights // - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress +// - lastFullBlockHeight: Progress reader to get the max of fetcher and syncer heights +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height // - config: Configuration for the fetcher // // Returns both the Fetcher and BlockProcessor so they can be reused in other components. @@ -54,6 +56,8 @@ func createFetcher( processedFinalizedBlockHeight storage.ConsumerProgressInitializer, distributor hotstuff.Distributor, collectionSyncMetrics module.CollectionSyncMetrics, + lastFullBlockHeight collection_sync.ProgressReader, + accessMetrics module.AccessMetrics, config CreateFetcherConfig, ) (*requester.Engine, collection_sync.Fetcher, error) { // Create requester engine for requesting collections @@ -103,7 +107,7 @@ func createFetcher( } }) - // Create Fetcher + // Create Fetcher with callback to update last_full_finalized_block_height metric collectionFetcher, err := fetcher.NewFetcher( log, blockProcessor, @@ -113,6 +117,10 @@ func createFetcher( config.MaxProcessing, config.MaxSearchAhead, collectionSyncMetrics, + func() { + // Update last_full_finalized_block_height metric with the max of both heights + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + }, ) if err != nil { return nil, nil, fmt.Errorf("could not create fetcher: %w", err) diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index 9d78f83f74d..c28008012c1 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -41,6 +41,7 @@ var _ component.Component = (*Fetcher)(nil) // - maxProcessing: Maximum number of jobs to process concurrently // - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit // - metrics: Optional metrics collector for reporting collection fetched height +// - onHeightUpdated: Optional callback to be called when processed height is updated // // No error returns are expected during normal operation. func NewFetcher( @@ -51,6 +52,7 @@ func NewFetcher( maxProcessing uint64, // max number of blocks to fetch collections concurrently maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for metrics module.CollectionSyncMetrics, // optional metrics collector + onHeightUpdated func(), // optional callback when processed height is updated ) (*Fetcher, error) { workSignal := engine.NewNotifier() @@ -113,10 +115,16 @@ func NewFetcher( consumer.SetPostNotifier(func(jobID module.JobID) { metrics.CollectionFetchedHeight(f.ProcessedHeight()) metrics.MissingCollectionQueueSize(f.blockProcessor.MissingCollectionQueueSize()) + if onHeightUpdated != nil { + onHeightUpdated() + } }) // report the initial metrics, otherwise it creates spikes in dashboard metrics.CollectionFetchedHeight(f.ProcessedHeight()) + if onHeightUpdated != nil { + onHeightUpdated() + } return f, nil } From 5df74e84d637b1d0b4978baa011e20eeab4a222b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 26 Nov 2025 09:40:14 -0800 Subject: [PATCH 097/126] add pebble compaction listener --- storage/pebble/config.go | 5 ++ storage/pebble/event_listener.go | 145 +++++++++++++++++++++++++++++++ 2 files changed, 150 insertions(+) create mode 100644 storage/pebble/event_listener.go diff --git a/storage/pebble/config.go b/storage/pebble/config.go index 06e8497d50d..be0d4f5dcba 100644 --- a/storage/pebble/config.go +++ b/storage/pebble/config.go @@ -11,6 +11,7 @@ import ( // DefaultPebbleOptions returns an optimized set of pebble options. // This is mostly copied form pebble's nightly performance benchmark. func DefaultPebbleOptions(logger zerolog.Logger, cache *pebble.Cache, comparer *pebble.Comparer) *pebble.Options { + eventListener := NewCompactionEventListener(logger) opts := &pebble.Options{ Cache: cache, Comparer: comparer, @@ -32,6 +33,10 @@ func DefaultPebbleOptions(logger zerolog.Logger, cache *pebble.Cache, comparer * // The default is 1. MaxConcurrentCompactions: func() int { return 4 }, Logger: util.NewLogger(logger), + EventListener: &pebble.EventListener{ + CompactionBegin: eventListener.CompactionBegin, + CompactionEnd: eventListener.CompactionEnd, + }, } for i := 0; i < len(opts.Levels); i++ { diff --git a/storage/pebble/event_listener.go b/storage/pebble/event_listener.go new file mode 100644 index 00000000000..92381e704d6 --- /dev/null +++ b/storage/pebble/event_listener.go @@ -0,0 +1,145 @@ +package pebble + +import ( + "time" + + "github.com/cockroachdb/pebble/v2" + "github.com/rs/zerolog" +) + +const ( + // DefaultMinCompactionSizeBytes is the default minimum input size (200MB) for logging compactions. + // Small compactions below this threshold are ignored. + DefaultMinCompactionSizeBytes int64 = 200 << 20 // 200 MB + + // DefaultMinCompactionDuration is the default minimum duration (1 second) for logging compactions. + // Fast compactions below this threshold are ignored. + DefaultMinCompactionDuration = 10 * time.Second + + // DefaultMinCompactionFiles is the default minimum number of input files (5) for logging compactions. + // Compactions with fewer files are ignored. + DefaultMinCompactionFiles = 5 +) + +// CompactionEventListener implements pebble.EventListener to log compaction events. +// Only significant compactions (large size, long duration, or many files) are logged. +type CompactionEventListener struct { + logger zerolog.Logger + minSizeBytes int64 + minDuration time.Duration + minFiles int +} + +// NewCompactionEventListener creates a new CompactionEventListener with default thresholds. +func NewCompactionEventListener(logger zerolog.Logger) *CompactionEventListener { + return NewCompactionEventListenerWithThresholds( + logger, + DefaultMinCompactionSizeBytes, + DefaultMinCompactionDuration, + DefaultMinCompactionFiles, + ) +} + +// NewCompactionEventListenerWithThresholds creates a new CompactionEventListener with custom thresholds. +func NewCompactionEventListenerWithThresholds( + logger zerolog.Logger, + minSizeBytes int64, + minDuration time.Duration, + minFiles int, +) *CompactionEventListener { + return &CompactionEventListener{ + logger: logger.With().Str("module", "pebble_compaction").Logger(), + minSizeBytes: minSizeBytes, + minDuration: minDuration, + minFiles: minFiles, + } +} + +// calculateTotalSize calculates the total size of tables in a level. +func calculateTotalSize(level pebble.LevelInfo) int64 { + var totalSize int64 + for _, table := range level.Tables { + totalSize += int64(table.Size) + } + return totalSize +} + +// isSignificantCompaction checks if a compaction is significant enough to log +// based on size, file count, or duration (for end events). +func (e *CompactionEventListener) isSignificantCompaction( + totalInputSize int64, + totalInputFiles int, + duration time.Duration, +) bool { + // Log if size exceeds threshold + if totalInputSize >= e.minSizeBytes { + return true + } + // Log if file count exceeds threshold + if totalInputFiles >= e.minFiles { + return true + } + // Log if duration exceeds threshold (only available for end events) + if duration > 0 && duration >= e.minDuration { + return true + } + return false +} + +// CompactionBegin logs when a significant compaction starts. +func (e *CompactionEventListener) CompactionBegin(info pebble.CompactionInfo) { + var totalInputFiles int + var totalInputSize int64 + var inputLevels []int + + for _, level := range info.Input { + totalInputFiles += len(level.Tables) + totalInputSize += calculateTotalSize(level) + inputLevels = append(inputLevels, level.Level) + } + + // Only log if compaction is significant (size or file count threshold) + if !e.isSignificantCompaction(totalInputSize, totalInputFiles, 0) { + return + } + + e.logger.Info(). + Int("job", info.JobID). + Str("reason", info.Reason). + Ints("input_levels", inputLevels). + Int("output_level", info.Output.Level). + Int("num_input_files", totalInputFiles). + Int64("input_size_bytes", totalInputSize). + Msg("compaction started") +} + +// CompactionEnd logs when a significant compaction ends. +func (e *CompactionEventListener) CompactionEnd(info pebble.CompactionInfo) { + var totalInputFiles int + var totalInputSize int64 + var inputLevels []int + + for _, level := range info.Input { + totalInputFiles += len(level.Tables) + totalInputSize += calculateTotalSize(level) + inputLevels = append(inputLevels, level.Level) + } + + // Only log if compaction is significant (size, file count, or duration threshold) + if !e.isSignificantCompaction(totalInputSize, totalInputFiles, info.Duration) { + return + } + + outputSize := calculateTotalSize(info.Output) + + e.logger.Info(). + Int("job", info.JobID). + Ints("input_levels", inputLevels). + Int("output_level", info.Output.Level). + Int("num_input_files", totalInputFiles). + Int("num_output_files", len(info.Output.Tables)). + Int64("input_size_bytes", totalInputSize). + Int64("output_size_bytes", outputSize). + Int64("duration_micros", info.Duration.Microseconds()). + Msg("compaction finished") +} From 079a2aaa7fdf3640be28a53f37ab81fb93ae223a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 26 Nov 2025 10:37:42 -0800 Subject: [PATCH 098/126] log with seconds --- storage/pebble/event_listener.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/storage/pebble/event_listener.go b/storage/pebble/event_listener.go index 92381e704d6..1cbf45d3aa5 100644 --- a/storage/pebble/event_listener.go +++ b/storage/pebble/event_listener.go @@ -8,15 +8,15 @@ import ( ) const ( - // DefaultMinCompactionSizeBytes is the default minimum input size (200MB) for logging compactions. + // DefaultMinCompactionSizeBytes is the default minimum input size for logging compactions. // Small compactions below this threshold are ignored. DefaultMinCompactionSizeBytes int64 = 200 << 20 // 200 MB - // DefaultMinCompactionDuration is the default minimum duration (1 second) for logging compactions. + // DefaultMinCompactionDuration is the default minimum duration for logging compactions. // Fast compactions below this threshold are ignored. - DefaultMinCompactionDuration = 10 * time.Second + DefaultMinCompactionDuration = 5 * time.Second - // DefaultMinCompactionFiles is the default minimum number of input files (5) for logging compactions. + // DefaultMinCompactionFiles is the default minimum number of input files for logging compactions. // Compactions with fewer files are ignored. DefaultMinCompactionFiles = 5 ) @@ -103,7 +103,7 @@ func (e *CompactionEventListener) CompactionBegin(info pebble.CompactionInfo) { return } - e.logger.Info(). + e.logger.Debug(). Int("job", info.JobID). Str("reason", info.Reason). Ints("input_levels", inputLevels). @@ -140,6 +140,6 @@ func (e *CompactionEventListener) CompactionEnd(info pebble.CompactionInfo) { Int("num_output_files", len(info.Output.Tables)). Int64("input_size_bytes", totalInputSize). Int64("output_size_bytes", outputSize). - Int64("duration_micros", info.Duration.Microseconds()). + Float64("duration_seconds", info.Duration.Seconds()). Msg("compaction finished") } From 50293fd8f627a65a51d21b035886d88ddc86fc16 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 2 Dec 2025 09:55:35 -0800 Subject: [PATCH 099/126] log missing collection --- .../node_builder/access_node_builder.go | 5 +- .../access/collection_sync/collection_sync.go | 7 +-- .../access/collection_sync/factory/config.go | 12 +++++ .../access/collection_sync/factory/fetcher.go | 10 +--- .../fetcher/block_processor.go | 13 +++-- .../access/collection_sync/fetcher/engine.go | 3 +- .../fetcher/missing_collection_queue.go | 52 +++++++++++++------ .../fetcher/missing_collection_queue_test.go | 14 ++--- 8 files changed, 75 insertions(+), 41 deletions(-) create mode 100644 engine/access/collection_sync/factory/config.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 6a10c8b00b0..10996c77153 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -40,6 +40,7 @@ import ( recovery "github.com/onflow/flow-go/consensus/recovery/protocol" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/factory" collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" "github.com/onflow/flow-go/engine/access/finalized_indexer" @@ -292,8 +293,8 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { storeTxResultErrorMessages: false, stopControlEnabled: false, registerDBPruneThreshold: 0, - collectionFetcherMaxProcessing: 10, - collectionFetcherMaxSearchAhead: 20, + collectionFetcherMaxProcessing: factory.DefaultMaxProcessing, + collectionFetcherMaxSearchAhead: factory.DefaultMaxSearchAhead, collectionSync: collection_syncfactory.CollectionSyncModeExecutionAndCollection, } } diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 80fdadf7084..c57c3e5c640 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -17,9 +17,10 @@ type MissingCollectionQueue interface { OnIndexedForBlock(blockHeight uint64) (func(), bool) // On receipt of a collection, MCQ updates internal state and, if a block - // just became complete, returns: (collections, height, true). - // Otherwise, returns (nil, 0, false). - OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, bool) + // just became complete, returns: (collections, height, missingCollectionID, true). + // Otherwise, returns (nil, height, missingCollectionID, false). + // missingCollectionID is an arbitrary ID from the remaining missing collections, or ZeroID if none. + OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, flow.Identifier, bool) // PruneUpToHeight removes all tracked heights up to and including the given height. PruneUpToHeight(height uint64) (callbacks []func()) diff --git a/engine/access/collection_sync/factory/config.go b/engine/access/collection_sync/factory/config.go new file mode 100644 index 00000000000..83c231c68e1 --- /dev/null +++ b/engine/access/collection_sync/factory/config.go @@ -0,0 +1,12 @@ +package factory + +// CreateFetcherConfig holds configuration parameters for creating a Fetcher. +type CreateFetcherConfig struct { + // MaxProcessing is the maximum number of jobs to process concurrently. + MaxProcessing uint64 + // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. + MaxSearchAhead uint64 +} + +const DefaultMaxProcessing = 10 +const DefaultMaxSearchAhead = 20 diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index c0e0caaf447..0637dddb534 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -19,14 +19,6 @@ import ( "github.com/onflow/flow-go/storage" ) -// CreateFetcherConfig holds configuration parameters for creating a Fetcher. -type CreateFetcherConfig struct { - // MaxProcessing is the maximum number of jobs to process concurrently. - MaxProcessing uint64 - // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. - MaxSearchAhead uint64 -} - // createFetcher creates a new Fetcher component with all its dependencies. // // Parameters: @@ -126,6 +118,8 @@ func createFetcher( return nil, nil, fmt.Errorf("could not create fetcher: %w", err) } + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { collectionFetcher.OnFinalizedBlock() }) diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 09c4c0ae697..f5ea1951923 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -83,10 +83,11 @@ func (bp *BlockProcessor) FetchCollections( } // Enqueue missing collections with notifyJobCompletion - // When all collections are received and indexed, mark the job as done + // When all collections are received and indexed, the notifyJobCompletion callback + // will be returned in the OnIndexedForBlock method, calling it will notify the job + // queue that the job is complete. notifyJobCompletion := done - // the notifyJobCompletion callback will be returned in the OnReceiveCollection method err = bp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, notifyJobCompletion) if err != nil { return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) @@ -109,7 +110,7 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti collectionID := collection.ID() // Pass collection to MCQ - collections, height, complete := bp.mcq.OnReceivedCollection(collection) + collections, height, missingCol, complete := bp.mcq.OnReceivedCollection(collection) // Log collection receipt and whether it completes a block if complete { @@ -123,11 +124,17 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti bp.log.Debug(). Hex("collection_id", collectionID[:]). Hex("origin_id", originID[:]). + // there might be multiple missing collections, just log one of them + Hex("missing_col", missingCol[:]). + Uint64("block_height", height). Msg("received collection (block not yet complete)") } + // OnReceiveCollection might be called multiple times for the same collection + // but complete will only be true once per block height. if !complete { // Block is not complete yet, nothing more to do + // or block is complete, but another thread is indexing it return nil } diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index c28008012c1..439bf89def2 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -46,7 +46,8 @@ var _ component.Component = (*Fetcher)(nil) // No error returns are expected during normal operation. func NewFetcher( log zerolog.Logger, - blockProcessor collection_sync.BlockProcessor, progressInitializer storage.ConsumerProgressInitializer, + blockProcessor collection_sync.BlockProcessor, + progressInitializer storage.ConsumerProgressInitializer, state protocol.State, blocks storage.Blocks, maxProcessing uint64, // max number of blocks to fetch collections concurrently diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 06735f57f80..b142e9af5e0 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -56,6 +56,15 @@ func NewMissingCollectionQueue() *MissingCollectionQueue { } } +// pickOne returns an arbitrary key from the given map. +// The map must be non-empty. The selection is non-deterministic due to Go's map iteration order. +func pickOne(m map[flow.Identifier]struct{}) flow.Identifier { + for id := range m { + return id + } + return flow.ZeroID +} + // EnqueueMissingCollections registers missing collections for a block height along with a callback // that will be invoked when all collections for that height have been received and indexed. // @@ -116,12 +125,14 @@ func (mcq *MissingCollectionQueue) EnqueueMissingCollections( // The collection parameter should be the actual collection object received from the requester. // // Returns: -// - (collections, height, true) if the block height became complete -// - (nil, 0, false) if no block height became complete +// - (collections, height, missingCollectionID, true) if the block height became complete +// - (nil, 0, missingCollectionID, false) if no block height became complete +// missingCollectionID is an arbitrary ID from the remaining missing collections, or ZeroID if none. func (mcq *MissingCollectionQueue) OnReceivedCollection( collection *flow.Collection, -) ([]*flow.Collection, uint64, bool) { +) ([]*flow.Collection, uint64, flow.Identifier, bool) { collectionID := collection.ID() + missingCol := flow.ZeroID mcq.mu.Lock() defer mcq.mu.Unlock() @@ -130,21 +141,29 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( height, ok := mcq.collectionToHeight[collectionID] if !ok { // No block is waiting for this collection. - return nil, 0, false + return nil, 0, missingCol, false } jobState, exists := mcq.blockJobs[height] if !exists { // Job was already completed/removed. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, false + return nil, 0, missingCol, false + } + + if len(jobState.missingCollections) > 0 { + // pick a random missing collection to return + // useful for logging/debugging purposes + // in case fetching is stuck, it's useful to know which collections are still missing + // we don't need to return all missing collections, just one is enough + missingCol = pickOne(jobState.missingCollections) } // Check if this collection was still missing for this block. if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { // Collection was already received or wasn't part of this block's missing set. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, false + return nil, 0, missingCol, false } // Remove from missing set and add to received collections. @@ -155,16 +174,16 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( // Don't delete from collectionToHeight - the mapping is kept until OnIndexedForBlock cleans it up. // Check if the block is now complete (all collections received). - if len(jobState.missingCollections) == 0 { - // Return all received collections for this block. - collections := make([]*flow.Collection, 0, len(jobState.receivedCollections)) - for _, col := range jobState.receivedCollections { - collections = append(collections, col) - } - return collections, height, true + if len(jobState.missingCollections) > 0 { + return nil, 0, missingCol, false } - return nil, 0, false + // Return all received collections for this block. + collections := make([]*flow.Collection, 0, len(jobState.receivedCollections)) + for _, col := range jobState.receivedCollections { + collections = append(collections, col) + } + return collections, height, flow.ZeroID, true } // IsHeightQueued returns true if the given height has queued collections @@ -257,11 +276,10 @@ func (mcq *MissingCollectionQueue) cleanupCollectionMappingsForHeight( return jobState, true } -// Size returns the number of missing collections currently in the queue. -// This is the total number of collections across all block heights that are still missing. +// Size returns the number of incomplete jobs currently in the queue. func (mcq *MissingCollectionQueue) Size() uint { mcq.mu.RLock() defer mcq.mu.RUnlock() - return uint(len(mcq.collectionToHeight)) + return uint(len(mcq.blockJobs)) } diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go index a1760ca2c81..706e32beae8 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go @@ -51,20 +51,20 @@ func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { assert.Contains(t, err.Error(), "cannot overwrite existing job") // Step 4: Receiving one collection doesn't complete the block, IsHeightQueued still returns true - collectionsReturned, heightReturned, complete := mcq.OnReceivedCollection(collections[0]) + collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") assert.False(t, complete, "block should not be complete with only one collection") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued after receiving one collection") // Step 5: Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) - collectionsReturned, _, complete = mcq.OnReceivedCollection(collections[1]) + collectionsReturned, _, _, complete = mcq.OnReceivedCollection(collections[1]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") assert.False(t, complete, "block should not be complete with only two collections") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") // Receive the last collection - block should now be complete - collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[2]) + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[2]) assert.NotNil(t, collectionsReturned, "should return collections when block is complete") assert.Equal(t, 3, len(collectionsReturned), "should return all 3 collections") assert.Equal(t, blockHeight, heightReturned, "should return correct block height") @@ -85,7 +85,7 @@ func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { assert.True(t, callbackInvoked, "callback should be invoked") // Step 8: OnReceivedCollection for that block returns false because the block has been removed - collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[0]) + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections for removed block") assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") assert.False(t, complete, "should not indicate completion for removed block") @@ -115,7 +115,7 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { assert.True(t, mcq.IsHeightQueued(blockHeight), "height should be queued") // Receive only one collection (block is not complete) - collectionsReturned, heightReturned, complete := mcq.OnReceivedCollection(collections[0]) + collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") assert.False(t, complete, "block should not be complete with only one collection") @@ -131,12 +131,12 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued after indexing") // Verify that remaining collections cannot be received (block has been removed) - collectionsReturned, heightReturned, complete = mcq.OnReceivedCollection(collections[1]) + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[1]) assert.Nil(t, collectionsReturned, "should not return collections for removed block") assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") assert.False(t, complete, "should not indicate completion for removed block") - collectionsReturned, _, complete = mcq.OnReceivedCollection(collections[2]) + collectionsReturned, _, _, complete = mcq.OnReceivedCollection(collections[2]) assert.Nil(t, collectionsReturned, "should not return collections for removed block") assert.False(t, complete, "should not indicate completion for removed block") From d9b4c20abebd4c0abb6a071864858147ed9c8bcb Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 2 Dec 2025 16:50:03 -0800 Subject: [PATCH 100/126] improve logging --- .../cmd/read-protocol-state/cmd/blocks.go | 4 ++ .../access/collection_sync/collection_sync.go | 3 ++ .../fetcher/block_processor.go | 4 ++ .../fetcher/missing_collection_queue.go | 48 ++++++++++++++----- 4 files changed, 46 insertions(+), 13 deletions(-) diff --git a/cmd/util/cmd/read-protocol-state/cmd/blocks.go b/cmd/util/cmd/read-protocol-state/cmd/blocks.go index 4c2ef0d13bd..d0792f9329e 100644 --- a/cmd/util/cmd/read-protocol-state/cmd/blocks.go +++ b/cmd/util/cmd/read-protocol-state/cmd/blocks.go @@ -222,6 +222,8 @@ func runE(*cobra.Command, []string) error { return fmt.Errorf("could not get root block: %w", err) } + log.Info().Msgf("searching executed block between heights %v and %v", root.Height, sealed.Height) + // find the last executed and sealed block for h := sealed.Height; h >= root.Height; h-- { block, err := reader.GetBlockByHeight(h) @@ -238,6 +240,8 @@ func runE(*cobra.Command, []string) error { common.PrettyPrintEntity(block) return nil } + + log.Info().Msgf("block at height %v is not executed yet", h) } return fmt.Errorf("could not find executed block") diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index c57c3e5c640..3c2d1bad709 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -30,6 +30,9 @@ type MissingCollectionQueue interface { // Size returns the number of missing collections currently in the queue. Size() uint + + // GetMissingCollections returns all collection IDs that are currently missing across all block heights. + GetMissingCollections() []flow.Identifier } // Requests collections by their guarantees. diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index f5ea1951923..49fe7b8da07 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -99,6 +99,10 @@ func (bp *BlockProcessor) FetchCollections( return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) } + bp.log.Debug().Uint64("block_height", blockHeight). + Int("missing_collections_count", len(missingGuarantees)). + Msg("fetched collections for finalized block") + return nil } diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index b142e9af5e0..0bdc9406af1 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -132,7 +132,6 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( collection *flow.Collection, ) ([]*flow.Collection, uint64, flow.Identifier, bool) { collectionID := collection.ID() - missingCol := flow.ZeroID mcq.mu.Lock() defer mcq.mu.Unlock() @@ -141,29 +140,21 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( height, ok := mcq.collectionToHeight[collectionID] if !ok { // No block is waiting for this collection. - return nil, 0, missingCol, false + return nil, 0, flow.ZeroID, false } jobState, exists := mcq.blockJobs[height] if !exists { // Job was already completed/removed. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, missingCol, false - } - - if len(jobState.missingCollections) > 0 { - // pick a random missing collection to return - // useful for logging/debugging purposes - // in case fetching is stuck, it's useful to know which collections are still missing - // we don't need to return all missing collections, just one is enough - missingCol = pickOne(jobState.missingCollections) + return nil, 0, flow.ZeroID, false } // Check if this collection was still missing for this block. if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { // Collection was already received or wasn't part of this block's missing set. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, missingCol, false + return nil, 0, pickOne(jobState.missingCollections), false } // Remove from missing set and add to received collections. @@ -175,7 +166,11 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( // Check if the block is now complete (all collections received). if len(jobState.missingCollections) > 0 { - return nil, 0, missingCol, false + // pick a random missing collection to return + // useful for logging/debugging purposes + // in case fetching is stuck, it's useful to know which collections are still missing + // we don't need to return all missing collections, just one is enough + return nil, 0, pickOne(jobState.missingCollections), false } // Return all received collections for this block. @@ -283,3 +278,30 @@ func (mcq *MissingCollectionQueue) Size() uint { return uint(len(mcq.blockJobs)) } + +// GetMissingCollections returns all collection IDs that are currently missing across all block heights. +// +// Returns a slice of collection identifiers that are still missing. The order is non-deterministic +// due to map iteration order. Returns an empty slice if there are no missing collections. +func (mcq *MissingCollectionQueue) GetMissingCollections() []flow.Identifier { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + // Count total missing collections to pre-allocate slice + totalMissing := 0 + for _, jobState := range mcq.blockJobs { + totalMissing += len(jobState.missingCollections) + } + + // Pre-allocate slice with capacity + missingCollections := make([]flow.Identifier, 0, totalMissing) + + // Collect all missing collection IDs from all block jobs + for _, jobState := range mcq.blockJobs { + for collectionID := range jobState.missingCollections { + missingCollections = append(missingCollections, collectionID) + } + } + + return missingCollections +} From 65245e9e76c4e07c2610df7ed7e29a10eac4e694 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 2 Dec 2025 17:10:38 -0800 Subject: [PATCH 101/126] add retry --- .../access/collection_sync/collection_sync.go | 5 ++ .../access/collection_sync/factory/config.go | 4 + .../access/collection_sync/factory/fetcher.go | 2 + .../fetcher/block_processor.go | 66 ++++++++++++++++ .../access/collection_sync/fetcher/engine.go | 76 ++++++++++++++++++- .../fetcher/missing_collection_queue.go | 26 +++++++ 6 files changed, 178 insertions(+), 1 deletion(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 3c2d1bad709..1010ea373f4 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -33,6 +33,9 @@ type MissingCollectionQueue interface { // GetMissingCollections returns all collection IDs that are currently missing across all block heights. GetMissingCollections() []flow.Identifier + + // GetMissingCollectionsByHeight returns a map of block height to collection IDs that are missing for that height. + GetMissingCollectionsByHeight() map[uint64][]flow.Identifier } // Requests collections by their guarantees. @@ -60,6 +63,8 @@ type BlockProcessor interface { MissingCollectionQueueSize() uint // PruneUpToHeight removes all tracked heights up to and including the given height. PruneUpToHeight(height uint64) + // RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. + RetryFetchingMissingCollections() error } // Fetcher is a component that consumes finalized block jobs and processes them diff --git a/engine/access/collection_sync/factory/config.go b/engine/access/collection_sync/factory/config.go index 83c231c68e1..ac51903157e 100644 --- a/engine/access/collection_sync/factory/config.go +++ b/engine/access/collection_sync/factory/config.go @@ -1,11 +1,15 @@ package factory +import "time" + // CreateFetcherConfig holds configuration parameters for creating a Fetcher. type CreateFetcherConfig struct { // MaxProcessing is the maximum number of jobs to process concurrently. MaxProcessing uint64 // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. MaxSearchAhead uint64 + // RetryInterval is the interval for retrying missing collections. If 0, uses DefaultRetryInterval. + RetryInterval time.Duration } const DefaultMaxProcessing = 10 diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index 0637dddb534..b3d74ec98e6 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -82,6 +82,7 @@ func createFetcher( mcq, indexer, collectionRequester, + blocks, ) // Register handler for received collections @@ -113,6 +114,7 @@ func createFetcher( // Update last_full_finalized_block_height metric with the max of both heights accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) }, + config.RetryInterval, ) if err != nil { return nil, nil, fmt.Errorf("could not create fetcher: %w", err) diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 49fe7b8da07..0b4cd450720 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -8,6 +8,7 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" ) // BlockProcessor implements the job lifecycle for collection indexing. @@ -17,6 +18,7 @@ type BlockProcessor struct { mcq collection_sync.MissingCollectionQueue indexer collection_sync.BlockCollectionIndexer requester collection_sync.CollectionRequester + blocks storage.Blocks } var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) @@ -28,6 +30,7 @@ var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) // - mcq: MissingCollectionQueue for tracking missing collections and callbacks // - indexer: BlockCollectionIndexer for storing and indexing collections // - requester: CollectionRequester for requesting collections from the network +// - blocks: Blocks storage for retrieving blocks to extract guarantees // // No error returns are expected during normal operation. func NewBlockProcessor( @@ -35,12 +38,14 @@ func NewBlockProcessor( mcq collection_sync.MissingCollectionQueue, indexer collection_sync.BlockCollectionIndexer, requester collection_sync.CollectionRequester, + blocks storage.Blocks, ) *BlockProcessor { return &BlockProcessor{ log: log.With().Str("coll_sync", "fetcher_processor").Logger(), mcq: mcq, indexer: indexer, requester: requester, + blocks: blocks, } } @@ -157,6 +162,67 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti return nil } +// RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. +// It retrieves blocks for the heights with missing collections and extracts the corresponding guarantees. +// +// No error returns are expected during normal operation. +func (bp *BlockProcessor) RetryFetchingMissingCollections() error { + // Get missing collections grouped by height + missingByHeight := bp.mcq.GetMissingCollectionsByHeight() + if len(missingByHeight) == 0 { + return nil + } + + // Build a set of missing collection IDs for efficient lookup + missingCollectionSet := make(map[flow.Identifier]struct{}) + for _, collectionIDs := range missingByHeight { + for _, collectionID := range collectionIDs { + // TODO: double check the collection is missing in storage + // maybe the execution data index has already indexed the collection + missingCollectionSet[collectionID] = struct{}{} + } + } + + // Collect guarantees for missing collections by retrieving blocks + var guarantees []*flow.CollectionGuarantee + for height := range missingByHeight { + // Get block for this height + block, err := bp.blocks.ByHeight(height) + if err != nil { + bp.log.Fatal(). + Uint64("block_height", height). + Err(err). + Msg("failed to retrieve block for retrying missing collections") + continue + } + + // Extract guarantees for missing collections from this block + for _, guarantee := range block.Payload.Guarantees { + if _, isMissing := missingCollectionSet[guarantee.CollectionID]; isMissing { + guarantees = append(guarantees, guarantee) + } + } + } + + if len(guarantees) == 0 { + bp.log.Debug().Msg("no guarantees found for missing collections") + return nil + } + + // Request collections using the guarantees + err := bp.requester.RequestCollectionsByGuarantees(guarantees) + if err != nil { + return fmt.Errorf("failed to retry requesting collections: %w", err) + } + + bp.log.Info(). + Int("guarantees_count", len(guarantees)). + Int("heights_count", len(missingByHeight)). + Msg("retried fetching missing collections") + + return nil +} + // MissingCollectionQueueSize returns the number of missing collections currently in the queue. func (bp *BlockProcessor) MissingCollectionQueueSize() uint { return bp.mcq.Size() diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go index 439bf89def2..7e66f81618f 100644 --- a/engine/access/collection_sync/fetcher/engine.go +++ b/engine/access/collection_sync/fetcher/engine.go @@ -2,6 +2,7 @@ package fetcher import ( "fmt" + "time" "github.com/rs/zerolog" @@ -24,12 +25,16 @@ type Fetcher struct { blockProcessor collection_sync.BlockProcessor workSignal engine.Notifier metrics module.CollectionSyncMetrics + retryInterval time.Duration } var _ collection_sync.Fetcher = (*Fetcher)(nil) var _ collection_sync.ProgressReader = (*Fetcher)(nil) var _ component.Component = (*Fetcher)(nil) +// DefaultRetryInterval is the default interval for retrying missing collections. +const DefaultRetryInterval = 30 * time.Second + // NewFetcher creates a new Fetcher component. // // Parameters: @@ -42,6 +47,7 @@ var _ component.Component = (*Fetcher)(nil) // - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit // - metrics: Optional metrics collector for reporting collection fetched height // - onHeightUpdated: Optional callback to be called when processed height is updated +// - retryInterval: Interval for retrying missing collections. If 0, uses DefaultRetryInterval // // No error returns are expected during normal operation. func NewFetcher( @@ -54,6 +60,7 @@ func NewFetcher( maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for metrics module.CollectionSyncMetrics, // optional metrics collector onHeightUpdated func(), // optional callback when processed height is updated + retryInterval time.Duration, // interval for retrying missing collections ) (*Fetcher, error) { workSignal := engine.NewNotifier() @@ -102,12 +109,16 @@ func NewFetcher( return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) } + if retryInterval == 0 { + retryInterval = DefaultRetryInterval + } + f := &Fetcher{ - Component: consumer, consumer: consumer, blockProcessor: blockProcessor, workSignal: workSignal, metrics: metrics, + retryInterval: retryInterval, } // Set up post-notifier to update metrics when a job is done @@ -127,6 +138,28 @@ func NewFetcher( onHeightUpdated() } + // Create a ComponentManager that includes both the consumer and the retry worker + componentManager := component.NewComponentManagerBuilder(). + AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + // Start the consumer component + consumer.Start(ctx) + select { + case <-ctx.Done(): + return + case <-consumer.Ready(): + ready() + } + <-consumer.Done() + }). + AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + // Retry worker that periodically retries missing collections + ready() + f.retryMissingCollectionsLoop(ctx, log) + }). + Build() + + f.Component = componentManager + return f, nil } @@ -146,3 +179,44 @@ func (s *Fetcher) ProcessedHeight() uint64 { func (s *Fetcher) Size() uint { return s.consumer.Size() } + +// retryMissingCollectionsLoop periodically retries fetching missing collections. +func (f *Fetcher) retryMissingCollectionsLoop(ctx irrecoverable.SignalerContext, log zerolog.Logger) { + ticker := time.NewTicker(f.retryInterval) + defer ticker.Stop() + + log.Info(). + Dur("retry_interval", f.retryInterval). + Msg("starting missing collections retry worker") + + // Wait for the consumer to be ready before starting retries + select { + case <-ctx.Done(): + return + case <-f.consumer.Ready(): + } + + for { + select { + case <-ctx.Done(): + log.Info().Msg("missing collections retry worker shutting down") + return + case <-ticker.C: + err := f.blockProcessor.RetryFetchingMissingCollections() + if err != nil { + log.Error(). + Err(err). + Msg("failed to retry fetching missing collections") + // Don't throw - this is a retry mechanism, failures are expected + // and we'll try again on the next interval + } else { + queueSize := f.blockProcessor.MissingCollectionQueueSize() + if queueSize > 0 { + log.Debug(). + Uint("missing_collections", queueSize). + Msg("retried fetching missing collections") + } + } + } + } +} diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 0bdc9406af1..85de6678169 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -305,3 +305,29 @@ func (mcq *MissingCollectionQueue) GetMissingCollections() []flow.Identifier { return missingCollections } + +// GetMissingCollectionsByHeight returns a map of block height to collection IDs that are missing for that height. +// +// Returns a map where keys are block heights and values are slices of collection identifiers +// that are still missing for that height. Returns an empty map if there are no missing collections. +func (mcq *MissingCollectionQueue) GetMissingCollectionsByHeight() map[uint64][]flow.Identifier { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + // Build map of height -> collection IDs + missingByHeight := make(map[uint64][]flow.Identifier) + + for height, jobState := range mcq.blockJobs { + if len(jobState.missingCollections) == 0 { + continue + } + + collectionIDs := make([]flow.Identifier, 0, len(jobState.missingCollections)) + for collectionID := range jobState.missingCollections { + collectionIDs = append(collectionIDs, collectionID) + } + missingByHeight[height] = collectionIDs + } + + return missingByHeight +} From 0728e390485eddc92ab7827cd0845079af5c5612 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 2 Dec 2025 18:16:21 -0800 Subject: [PATCH 102/126] check storage before retry --- .../fetcher/block_processor.go | 58 ++++++++++++++----- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 0b4cd450720..747dd859128 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -173,18 +173,11 @@ func (bp *BlockProcessor) RetryFetchingMissingCollections() error { return nil } - // Build a set of missing collection IDs for efficient lookup - missingCollectionSet := make(map[flow.Identifier]struct{}) - for _, collectionIDs := range missingByHeight { - for _, collectionID := range collectionIDs { - // TODO: double check the collection is missing in storage - // maybe the execution data index has already indexed the collection - missingCollectionSet[collectionID] = struct{}{} - } - } - // Collect guarantees for missing collections by retrieving blocks + // Double-check that collections are actually missing in storage (they may have been + // indexed by execution data indexer or received from another source) var guarantees []*flow.CollectionGuarantee + for height := range missingByHeight { // Get block for this height block, err := bp.blocks.ByHeight(height) @@ -192,14 +185,49 @@ func (bp *BlockProcessor) RetryFetchingMissingCollections() error { bp.log.Fatal(). Uint64("block_height", height). Err(err). - Msg("failed to retrieve block for retrying missing collections") + Msg("failed to retrieve block for retrying missing collections, skipping") continue } - // Extract guarantees for missing collections from this block - for _, guarantee := range block.Payload.Guarantees { - if _, isMissing := missingCollectionSet[guarantee.CollectionID]; isMissing { - guarantees = append(guarantees, guarantee) + // Use indexer's GetMissingCollections to filter out collections that already exist + // This handles the case where execution data indexer may have already indexed them + actuallyMissingGuarantees, err := bp.indexer.GetMissingCollections(block) + if err != nil { + bp.log.Warn(). + Uint64("block_height", height). + Err(err). + Msg("failed to check missing collections for block, skipping") + continue + } + + // Build a set of actually missing collection IDs for this block + actuallyMissingSet := make(map[flow.Identifier]struct{}) + for _, guarantee := range actuallyMissingGuarantees { + actuallyMissingSet[guarantee.CollectionID] = struct{}{} + guarantees = append(guarantees, guarantee) + } + + // Check if all collections for this block are now indexed + // If so, mark the block as done in MCQ + originalMissing := missingByHeight[height] + allIndexed := true + for _, collectionID := range originalMissing { + if _, stillMissing := actuallyMissingSet[collectionID]; stillMissing { + allIndexed = false + break + } + } + + if allIndexed { + // All collections for this block are now indexed, mark it as done + notifyJobCompletion, ok := bp.mcq.OnIndexedForBlock(height) + if ok { + // Call the callback to notify job completion + notifyJobCompletion() + bp.log.Info(). + Uint64("block_height", height). + Int("collections_count", len(originalMissing)). + Msg("all collections for block are now indexed, marked block as done") } } } From ee346e2c4f548a3d102088e2bf822c722fa89854 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 08:45:54 -0800 Subject: [PATCH 103/126] simplify RetryFetchingMissingCollections --- .../access/collection_sync/collection_sync.go | 3 -- .../fetcher/block_processor.go | 41 +++++-------------- .../fetcher/missing_collection_queue.go | 17 -------- 3 files changed, 11 insertions(+), 50 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 1010ea373f4..1562724f80d 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -22,9 +22,6 @@ type MissingCollectionQueue interface { // missingCollectionID is an arbitrary ID from the remaining missing collections, or ZeroID if none. OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, flow.Identifier, bool) - // PruneUpToHeight removes all tracked heights up to and including the given height. - PruneUpToHeight(height uint64) (callbacks []func()) - // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). IsHeightQueued(height uint64) bool diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go index 747dd859128..9f8b4cb1070 100644 --- a/engine/access/collection_sync/fetcher/block_processor.go +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -99,6 +99,10 @@ func (bp *BlockProcessor) FetchCollections( } // Request collections from collection nodes + // Note: requester does not guarantee it will keep trying to fetch the collections, + // in fact, if a collection node is down, the request may fail permanently, + // so we need to have a retry mechanism in place (see RetryFetchingMissingCollections), + // which allows requester to retry with a different collection node. err = bp.requester.RequestCollectionsByGuarantees(missingGuarantees) if err != nil { return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) @@ -164,6 +168,7 @@ func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collecti // RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. // It retrieves blocks for the heights with missing collections and extracts the corresponding guarantees. +// Note, the caller is responsible to ensure this method is called periodically, and not concurrently. // // No error returns are expected during normal operation. func (bp *BlockProcessor) RetryFetchingMissingCollections() error { @@ -200,36 +205,21 @@ func (bp *BlockProcessor) RetryFetchingMissingCollections() error { continue } - // Build a set of actually missing collection IDs for this block - actuallyMissingSet := make(map[flow.Identifier]struct{}) - for _, guarantee := range actuallyMissingGuarantees { - actuallyMissingSet[guarantee.CollectionID] = struct{}{} - guarantees = append(guarantees, guarantee) - } - - // Check if all collections for this block are now indexed - // If so, mark the block as done in MCQ - originalMissing := missingByHeight[height] - allIndexed := true - for _, collectionID := range originalMissing { - if _, stillMissing := actuallyMissingSet[collectionID]; stillMissing { - allIndexed = false - break - } - } - - if allIndexed { + // If no collections are missing, all collections for this block are now indexed + if len(actuallyMissingGuarantees) == 0 { // All collections for this block are now indexed, mark it as done notifyJobCompletion, ok := bp.mcq.OnIndexedForBlock(height) if ok { - // Call the callback to notify job completion notifyJobCompletion() bp.log.Info(). Uint64("block_height", height). - Int("collections_count", len(originalMissing)). Msg("all collections for block are now indexed, marked block as done") } + continue } + + // Add guarantees for collections that are still missing + guarantees = append(guarantees, actuallyMissingGuarantees...) } if len(guarantees) == 0 { @@ -255,12 +245,3 @@ func (bp *BlockProcessor) RetryFetchingMissingCollections() error { func (bp *BlockProcessor) MissingCollectionQueueSize() uint { return bp.mcq.Size() } - -// PruneUpToHeight removes all tracked heights up to and including the given height. -func (bp *BlockProcessor) PruneUpToHeight(height uint64) { - callbacks := bp.mcq.PruneUpToHeight(height) - // notify job completion - for _, cb := range callbacks { - cb() - } -} diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index 85de6678169..d255dc971c5 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -225,23 +225,6 @@ func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func() return jobState.callback, true } -// PruneUpToHeight removes all block jobs and their collection mappings -// for block heights less than or equal to the specified height. -func (mcq *MissingCollectionQueue) PruneUpToHeight(height uint64) []func() { - mcq.mu.Lock() - defer mcq.mu.Unlock() - - callbacks := make([]func(), 0, len(mcq.blockJobs)) - for blockHeight := range mcq.blockJobs { - if blockHeight <= height { - jobState, _ := mcq.cleanupCollectionMappingsForHeight(blockHeight) - callbacks = append(callbacks, jobState.callback) - } - } - - return callbacks -} - // cleanupCollectionMappingsForHeight removes all collection-to-height mappings for collections // belonging to the specified block height and removes the block job from tracking. // This includes both missing and received collections. From f6dacb4306f6b8b7792f7fde864bc3b0babdb02b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 10:02:24 -0800 Subject: [PATCH 104/126] Bug fix: failed Unicast calls no longer count as attempts --- engine/common/requester/engine.go | 38 ++++-- engine/common/requester/engine_test.go | 158 +++++++++++++++++++++++++ 2 files changed, 184 insertions(+), 12 deletions(-) diff --git a/engine/common/requester/engine.go b/engine/common/requester/engine.go index 1354425b1ed..8272f4bea50 100644 --- a/engine/common/requester/engine.go +++ b/engine/common/requester/engine.go @@ -364,23 +364,12 @@ func (e *Engine) dispatchRequest() (bool, error) { providers = filteredProviders } - // add item to list and set retry parameters + // add item to list (but don't update retry parameters yet - only after successful send) // NOTE: we add the retry interval to the last requested timestamp, // rather than using the current timestamp, in order to conserve a // more even distribution of timestamps over time, which should lead // to a more even distribution of entities over batch requests entityIDs = append(entityIDs, entityID) - item.NumAttempts++ - item.LastRequested = now - item.RetryAfter = e.cfg.RetryFunction(item.RetryAfter) - - // make sure the interval is within parameters - if item.RetryAfter < e.cfg.RetryInitial { - item.RetryAfter = e.cfg.RetryInitial - } - if item.RetryAfter > e.cfg.RetryMaximum { - item.RetryAfter = e.cfg.RetryMaximum - } // if we reached the maximum size for a batch, bail if uint(len(entityIDs)) >= e.cfg.BatchThreshold { @@ -417,8 +406,33 @@ func (e *Engine) dispatchRequest() (bool, error) { err = e.con.Unicast(req, providerID) if err != nil { + // Unicast failed - don't count this as an attempt, don't update retry state + // The item will be retried on the next dispatch cycle return true, fmt.Errorf("could not send request for entities %v: %w", logging.IDs(entityIDs), err) } + + // Unicast succeeded - now update retry state for all items in this batch + for _, entityID := range entityIDs { + item, exists := e.items[entityID] + if !exists { + // Item was removed (shouldn't happen, but be safe) + continue + } + + // Update retry parameters only after successful send + item.NumAttempts++ + item.LastRequested = now + item.RetryAfter = e.cfg.RetryFunction(item.RetryAfter) + + // make sure the interval is within parameters + if item.RetryAfter < e.cfg.RetryInitial { + item.RetryAfter = e.cfg.RetryInitial + } + if item.RetryAfter > e.cfg.RetryMaximum { + item.RetryAfter = e.cfg.RetryMaximum + } + } + e.requests[req.Nonce] = req // NOTE: we forget about requests after the expiry of the shortest retry time diff --git a/engine/common/requester/engine_test.go b/engine/common/requester/engine_test.go index e10555e19ba..9be46ea835b 100644 --- a/engine/common/requester/engine_test.go +++ b/engine/common/requester/engine_test.go @@ -1,7 +1,10 @@ package requester import ( + "fmt" + "math" "math/rand" + "sync" "testing" "time" @@ -486,3 +489,158 @@ func TestOriginValidation(t *testing.T) { // handler are called async, but this should be extremely quick unittest.AssertClosesBefore(t, called, time.Second) } + +// TestEntityByIDRetryWithDifferentNode tests that when a node in the selector group +// always fails (Unicast returns error), the requester will eventually retry with a different node. +func TestEntityByIDRetryWithDifferentNode(t *testing.T) { + identities := unittest.IdentityListFixture(3) + failingNodeID := identities[0].NodeID + workingNodeID := identities[1].NodeID + + final := &protocol.Snapshot{} + final.On("Identities", mock.Anything).Return( + func(selector flow.IdentityFilter[flow.Identity]) flow.IdentityList { + return identities.Filter(selector) + }, + nil, + ) + + state := &protocol.State{} + state.On("Final").Return(final) + + // Use short retry intervals for faster test execution + cfg := Config{ + BatchInterval: 50 * time.Millisecond, + BatchThreshold: 10, + RetryInitial: 50 * time.Millisecond, + RetryFunction: RetryConstant(), // Keep retry interval constant for predictable timing + RetryAttempts: math.MaxUint32, // Don't give up + RetryMaximum: 100 * time.Millisecond, + } + + // Track Unicast calls by node ID + unicastCalls := make(map[flow.Identifier][]*messages.EntityRequest) + var unicastMutex sync.Mutex + + con := &mocknetwork.Conduit{} + con.On("Unicast", mock.Anything, mock.Anything).Run( + func(args mock.Arguments) { + req := args.Get(0).(*messages.EntityRequest) + providerID := args.Get(1).(flow.Identifier) + + unicastMutex.Lock() + unicastCalls[providerID] = append(unicastCalls[providerID], req) + unicastMutex.Unlock() + }, + ).Return(func(event interface{}, targetID flow.Identifier) error { + // Failing node always returns error + if targetID == failingNodeID { + return fmt.Errorf("node %s always fails", targetID) + } + // Working node succeeds + return nil + }) + + entityID := unittest.IdentifierFixture() + + // Create engine with selector that includes both failing and working nodes + e := &Engine{ + unit: engine.NewUnit(), + metrics: metrics.NewNoopCollector(), + cfg: cfg, + state: state, + con: con, + items: make(map[flow.Identifier]*Item), + requests: make(map[uint64]*messages.EntityRequest), + selector: filter.Any, // Allow all nodes + create: func() flow.Entity { return &flow.Collection{} }, + handle: func(flow.Identifier, flow.Entity) {}, // No-op handler for this test + forcedDispatchOngoing: atomic.NewBool(false), + } + + // Request entity with selector that includes both nodes + selector := filter.Or( + filter.HasNodeID[flow.Identity](failingNodeID), + filter.HasNodeID[flow.Identity](workingNodeID), + ) + e.EntityByID(entityID, selector) + + // Force immediate dispatch - this may try either node first (random selection) + e.Force() + time.Sleep(100 * time.Millisecond) + + // Verify at least one node was tried + unicastMutex.Lock() + totalCalls := len(unicastCalls[failingNodeID]) + len(unicastCalls[workingNodeID]) + unicastMutex.Unlock() + require.Greater(t, totalCalls, 0, "at least one node should have been tried") + + // Keep retrying until working node succeeds + // Since Unicast failures don't count as attempts now, retries can happen immediately + maxRetries := 20 + for i := 0; i < maxRetries; i++ { + unicastMutex.Lock() + workingCalls := len(unicastCalls[workingNodeID]) + failingCalls := len(unicastCalls[failingNodeID]) + unicastMutex.Unlock() + + // If working node was tried and succeeded, we're done + // (successful Unicast means the request was sent, even if we don't simulate a response) + if workingCalls > 0 { + break + } + + // If only failing node was tried, force immediate retry (no need to wait since failures don't update state) + // Otherwise wait a bit for the next dispatch cycle + if failingCalls > 0 && workingCalls == 0 { + // Failing node was tried, retry immediately since state wasn't updated + time.Sleep(10 * time.Millisecond) + } else { + // Wait a bit longer if no calls yet or both were tried + time.Sleep(50 * time.Millisecond) + } + + // Force another dispatch - should retry and eventually try working node + e.Force() + time.Sleep(50 * time.Millisecond) + } + + // Verify both nodes were eventually tried + unicastMutex.Lock() + failingCalls := len(unicastCalls[failingNodeID]) + workingCalls := len(unicastCalls[workingNodeID]) + unicastMutex.Unlock() + + // At least one of the nodes should have been tried + // Due to random sampling, either node could be selected first + // But if failing node was tried, working node should eventually be tried + if failingCalls > 0 { + // Failing node was tried - verify working node was eventually tried + assert.Greater(t, workingCalls, 0, "if failing node was tried, working node should eventually be tried after failures") + } else { + // Working node was tried first and succeeded - that's fine too + assert.Greater(t, workingCalls, 0, "working node should have been tried") + } + + // Verify that requests were sent to working node (which succeeded) + // The working node should have at least one successful Unicast call + unicastMutex.Lock() + workingRequests := unicastCalls[workingNodeID] + unicastMutex.Unlock() + + assert.Greater(t, len(workingRequests), 0, "working node should have received requests") + // Verify the entity ID is in the requests sent to working node + foundEntity := false + for _, req := range workingRequests { + for _, id := range req.EntityIDs { + if id == entityID { + foundEntity = true + break + } + } + if foundEntity { + break + } + } + assert.True(t, foundEntity, "entity ID should be in requests sent to working node") +} From 49876a9915c0bb25c72a4f5eede7d2c7b82920bf Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 10:32:37 -0800 Subject: [PATCH 105/126] add retry interval config --- cmd/access/node_builder/access_node_builder.go | 8 ++++++++ engine/access/collection_sync/collection_sync.go | 2 -- engine/access/collection_sync/factory/access_builder.go | 4 ++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 10996c77153..b7c8e38d625 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -42,6 +42,7 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/factory" collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" + collection_syncfetcher "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" "github.com/onflow/flow-go/engine/access/finalized_indexer" "github.com/onflow/flow-go/engine/access/index" @@ -185,6 +186,7 @@ type AccessNodeConfig struct { registerDBPruneThreshold uint64 collectionFetcherMaxProcessing uint64 collectionFetcherMaxSearchAhead uint64 + collectionFetcherRetryInterval time.Duration collectionSync collection_syncfactory.CollectionSyncMode } @@ -295,6 +297,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { registerDBPruneThreshold: 0, collectionFetcherMaxProcessing: factory.DefaultMaxProcessing, collectionFetcherMaxSearchAhead: factory.DefaultMaxSearchAhead, + collectionFetcherRetryInterval: collection_syncfetcher.DefaultRetryInterval, collectionSync: collection_syncfactory.CollectionSyncModeExecutionAndCollection, } } @@ -1207,6 +1210,10 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "collection-fetcher-max-search-ahead", defaultConfig.collectionFetcherMaxSearchAhead, "maximum number of blocks to search ahead when fetching collections") + flags.DurationVar(&builder.collectionFetcherRetryInterval, + "collection-fetcher-retry-interval", + defaultConfig.collectionFetcherRetryInterval, + "interval for retrying missing collections. default: 30s") flags.StringVarP(&builder.ExecutionNodeAddress, "script-addr", "s", @@ -2579,6 +2586,7 @@ func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { notNil(builder.CollectionSyncMetrics), builder.collectionFetcherMaxProcessing, builder.collectionFetcherMaxSearchAhead, + builder.collectionFetcherRetryInterval, notNil(builder.lastFullBlockHeight), notNil(builder.AccessMetrics), ) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 1562724f80d..4878e51222d 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -58,8 +58,6 @@ type BlockProcessor interface { FetchCollections(ctx irrecoverable.SignalerContext, block *flow.Block, done func()) error // MissingCollectionQueueSize returns the number of missing collections currently in the queue. MissingCollectionQueueSize() uint - // PruneUpToHeight removes all tracked heights up to and including the given height. - PruneUpToHeight(height uint64) // RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. RetryFetchingMissingCollections() error } diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 864eebb2e16..0c073881b46 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -2,6 +2,7 @@ package factory import ( "fmt" + "time" "github.com/rs/zerolog" @@ -223,6 +224,7 @@ type CollectionSyncFetcherComponentResult struct { // - collectionSyncMetrics: Collection sync metrics // - maxProcessing: Maximum number of concurrent processing jobs // - maxSearchAhead: Maximum number of blocks to search ahead +// - retryInterval: Interval for retrying missing collections. If 0, uses DefaultRetryInterval // - lastFullBlockHeight: Progress reader to register the fetcher with // - accessMetrics: Access metrics for reporting last_full_finalized_block_height // @@ -245,6 +247,7 @@ func CreateCollectionSyncFetcherComponent( collectionSyncMetrics module.CollectionSyncMetrics, maxProcessing uint64, maxSearchAhead uint64, + retryInterval time.Duration, lastFullBlockHeight *ProgressReader, accessMetrics module.AccessMetrics, ) (*CollectionSyncFetcherComponentResult, error) { @@ -295,6 +298,7 @@ func CreateCollectionSyncFetcherComponent( CreateFetcherConfig{ MaxProcessing: maxProcessing, MaxSearchAhead: maxSearchAhead, + RetryInterval: retryInterval, }, ) From 75a28bb9e92656ec40b94fef65487a4a62d357fd Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 12:50:19 -0800 Subject: [PATCH 106/126] report correct height --- .../collection_sync/fetcher/missing_collection_queue.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go index d255dc971c5..12787af0063 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -147,14 +147,14 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( if !exists { // Job was already completed/removed. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, flow.ZeroID, false + return nil, height, flow.ZeroID, false } // Check if this collection was still missing for this block. if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { // Collection was already received or wasn't part of this block's missing set. // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. - return nil, 0, pickOne(jobState.missingCollections), false + return nil, height, pickOne(jobState.missingCollections), false } // Remove from missing set and add to received collections. @@ -170,7 +170,7 @@ func (mcq *MissingCollectionQueue) OnReceivedCollection( // useful for logging/debugging purposes // in case fetching is stuck, it's useful to know which collections are still missing // we don't need to return all missing collections, just one is enough - return nil, 0, pickOne(jobState.missingCollections), false + return nil, height, pickOne(jobState.missingCollections), false } // Return all received collections for this block. From 5b17ae2a946c210f77d5dc7d73770987e22a487c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 13:00:38 -0800 Subject: [PATCH 107/126] add retry interval config --- engine/access/collection_sync/factory/fetcher.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go index b3d74ec98e6..017e7b1dbf6 100644 --- a/engine/access/collection_sync/factory/fetcher.go +++ b/engine/access/collection_sync/factory/fetcher.go @@ -2,6 +2,7 @@ package factory import ( "fmt" + "time" "github.com/rs/zerolog" @@ -62,6 +63,9 @@ func createFetcher( channels.RequestCollections, filter.HasRole[flow.Identity](flow.RoleCollection), func() flow.Entity { return new(flow.Collection) }, + requester.WithBatchInterval(60*time.Second), + requester.WithValidateStaking(false), + requester.WithRetryMaximum(10*time.Second), ) if err != nil { return nil, nil, fmt.Errorf("could not create requester engine: %w", err) From c8f71b92bcd801148b70fdb8b6a1fce64582cd85 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 20:18:06 -0800 Subject: [PATCH 108/126] log indexer look up --- module/state_synchronization/indexer/indexer.go | 1 + 1 file changed, 1 insertion(+) diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index c1b374aab13..29fd8cef118 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -158,6 +158,7 @@ func (i *Indexer) onBlockIndexed() error { highestIndexedHeight := i.jobConsumer.LastProcessedIndex() if lastProcessedHeight < highestIndexedHeight { + i.log.Info().Msgf("notifying processed heights from %d to %d", lastProcessedHeight+1, highestIndexedHeight) // we need loop here because it's possible for a height to be missed here, // we should guarantee all heights are processed for height := lastProcessedHeight + 1; height <= highestIndexedHeight; height++ { From 8de563a1c347d78f833f8973b21eac22a6316f65 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 20:27:16 -0800 Subject: [PATCH 109/126] indexer remove redundant header reads --- .../state_synchronization/indexer/indexer.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index 29fd8cef118..9e6cea06586 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -162,14 +162,16 @@ func (i *Indexer) onBlockIndexed() error { // we need loop here because it's possible for a height to be missed here, // we should guarantee all heights are processed for height := lastProcessedHeight + 1; height <= highestIndexedHeight; height++ { - header, err := i.indexer.headers.ByHeight(height) - if err != nil { - // if the execution data is available, the block must be locally finalized - i.log.Error().Err(err).Msgf("could not get header for height %d:", height) - return fmt.Errorf("could not get header for height %d: %w", height, err) - } - - i.OnBlockProcessed(header.Height) + // Use BlockIDByHeight instead of ByHeight since we only need to verify the block exists + // and don't need the full header data. This avoids expensive header deserialization. + // _, err := i.indexer.headers.BlockIDByHeight(height) + // if err != nil { + // // if the execution data is available, the block must be locally finalized + // i.log.Error().Err(err).Msgf("could not get header for height %d:", height) + // return fmt.Errorf("could not get header for height %d: %w", height, err) + // } + + i.OnBlockProcessed(height) } i.lastProcessedHeight.Store(highestIndexedHeight) } From 91299c24fe261eb6b402237da01c7a18f5ce875e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 21:44:56 -0800 Subject: [PATCH 110/126] add flag to disable bitswap bloom cache --- .../node_builder/access_node_builder.go | 4 ++ cmd/execution_builder.go | 4 ++ cmd/node_builder.go | 8 ++++ cmd/observer/node_builder/observer_builder.go | 4 ++ cmd/scaffold.go | 4 ++ network/p2p/blob/blob_service.go | 40 ++++++++++++++----- 6 files changed, 55 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b7c8e38d625..cd0ef2db77f 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -659,6 +659,10 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess opts = append(opts, blob.WithReprovideInterval(-1)) } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.ExecutionDataService, builder.ExecutionDatastoreManager.Datastore(), opts...) if err != nil { diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index e79a8a4aea8..e2c8573a77a 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -413,6 +413,10 @@ func (exeNode *ExecutionNode) LoadBlobService( opts = append(opts, blob.WithReprovideInterval(-1)) } + if !node.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + if exeNode.exeConf.blobstoreRateLimit > 0 && exeNode.exeConf.blobstoreBurstLimit > 0 { opts = append(opts, blob.WithRateLimit(float64(exeNode.exeConf.blobstoreRateLimit), exeNode.exeConf.blobstoreBurstLimit)) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 2721552642c..3b690bd0b46 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -186,6 +186,13 @@ type BaseConfig struct { // This is only meaningful to Access and Execution nodes. BitswapReprovideEnabled bool + // BitswapBloomCacheEnabled configures whether the Bitswap bloom cache is enabled. + // When disabled, uses a plain blockstore instead of cached blockstore, avoiding + // the CPU cost of building the bloom filter on startup. Pebble's built-in bloom + // filters (persisted in SSTables) are still used for efficient lookups. + // This is only meaningful to Access and Execution nodes. + BitswapBloomCacheEnabled bool + TransactionFeesDisabled bool } @@ -303,6 +310,7 @@ func DefaultBaseConfig() *BaseConfig { ComplianceConfig: compliance.DefaultConfig(), DhtSystemEnabled: true, BitswapReprovideEnabled: true, + BitswapBloomCacheEnabled: true, // default: use cached blockstore } } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 0a669b58617..9d73faeafbb 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1198,6 +1198,10 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS ), } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.PublicExecutionDataService, ds, opts...) if err != nil { diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 142f2d55c6d..e5008b345f4 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -213,6 +213,10 @@ func (fnb *FlowNodeBuilder) BaseFlags() { "bitswap-reprovide-enabled", defaultConfig.BitswapReprovideEnabled, "[experimental] whether to enable bitswap reproviding. This is an experimental feature. Use with caution.") + fnb.flags.BoolVar(&fnb.BaseConfig.BitswapBloomCacheEnabled, + "bitswap-bloom-cache-enabled", + defaultConfig.BitswapBloomCacheEnabled, + "[experimental] whether to enable bitswap bloom cache. When disabled, uses a plain blockstore instead of cached blockstore, avoiding the CPU cost of building the bloom filter on startup. Pebble's built-in bloom filters (persisted in SSTables) are still used. This is an experimental feature. Use with caution.") // dynamic node startup flags fnb.flags.StringVar(&fnb.BaseConfig.DynamicStartupANPubkey, diff --git a/network/p2p/blob/blob_service.go b/network/p2p/blob/blob_service.go index b4a1257b200..74107bde431 100644 --- a/network/p2p/blob/blob_service.go +++ b/network/p2p/blob/blob_service.go @@ -55,6 +55,7 @@ var _ component.Component = (*blobService)(nil) type BlobServiceConfig struct { ReprovideInterval time.Duration // the interval at which the DHT provider entries are refreshed BitswapOptions []bitswap.Option // options to pass to the Bitswap service + SkipBloomCache bool // if true, skip the bloom cache and use plain blockstore } // WithReprovideInterval sets the interval at which DHT provider entries are refreshed @@ -98,6 +99,15 @@ func WithRateLimit(r float64, b int) network.BlobServiceOption { } } +// WithSkipBloomCache disables the bloom cache, using a plain blockstore instead. +// This avoids the CPU cost of building the bloom filter on startup by scanning all keys. +// Pebble's built-in bloom filters (persisted in SSTables) are still used for efficient lookups. +func WithSkipBloomCache(skip bool) network.BlobServiceOption { + return func(bs network.BlobService) { + bs.(*blobService).config.SkipBloomCache = skip + } +} + // NewBlobService creates a new BlobService. func NewBlobService( host host.Host, @@ -109,26 +119,38 @@ func NewBlobService( opts ...network.BlobServiceOption, ) (*blobService, error) { bsNetwork := bsnet.NewFromIpfsHost(host, r, bsnet.Prefix(protocol.ID(prefix))) - blockStore, err := blockstore.CachedBlockstore( - context.Background(), - blockstore.NewBlockstore(ds), - blockstore.DefaultCacheOpts(), - ) - if err != nil { - return nil, fmt.Errorf("failed to create cached blockstore: %w", err) - } bs := &blobService{ prefix: prefix, config: &BlobServiceConfig{ ReprovideInterval: DefaultReprovideInterval, + SkipBloomCache: false, // default: use cached blockstore }, - blockStore: blockStore, } + // Apply options before creating blockstore, as SkipBloomCache affects blockstore creation for _, opt := range opts { opt(bs) } + // Create blockstore based on config + var blockStore blockstore.Blockstore + if bs.config.SkipBloomCache { + // Use plain blockstore - Pebble's built-in bloom filters are sufficient + blockStore = blockstore.NewBlockstore(ds) + } else { + // Use cached blockstore with bloom filter (default behavior) + cachedBlockStore, err := blockstore.CachedBlockstore( + context.Background(), + blockstore.NewBlockstore(ds), + blockstore.DefaultCacheOpts(), + ) + if err != nil { + return nil, fmt.Errorf("failed to create cached blockstore: %w", err) + } + blockStore = cachedBlockStore + } + bs.blockStore = blockStore + cm := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { btswp := bitswap.New(ctx, bsNetwork, bs.blockStore, bs.config.BitswapOptions...) From 199a84aa67a177438338165c1c76f63a31606ddf Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 09:21:27 -0800 Subject: [PATCH 111/126] improve logging --- .../access/collection_sync/execution_data_index/processor.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index 39e4962dd77..a0b3ba3631f 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -81,7 +81,8 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, // to account for execution forks. collections, err := edp.provider.GetExecutionDataByHeight(ctx, height) if err != nil { - ctx.Throw(fmt.Errorf("failed to get execution data for height %d: %w", height, err)) + ctx.Throw(fmt.Errorf("collection_sync execution data processor: failed to get execution data for height %d: %w", + height, err)) return } From 9c3c5d98f062803ef85cb05e6452f72dcac485ee Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 3 Dec 2025 21:44:56 -0800 Subject: [PATCH 112/126] add flag to disable bitswap bloom cache --- .../node_builder/access_node_builder.go | 4 ++ cmd/execution_builder.go | 4 ++ cmd/node_builder.go | 8 ++++ cmd/observer/node_builder/observer_builder.go | 4 ++ cmd/scaffold.go | 4 ++ network/p2p/blob/blob_service.go | 40 ++++++++++++++----- 6 files changed, 55 insertions(+), 9 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5ac66c3d726..b6ff0e3ea34 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -659,6 +659,10 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess opts = append(opts, blob.WithReprovideInterval(-1)) } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.ExecutionDataService, builder.ExecutionDatastoreManager.Datastore(), opts...) if err != nil { diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index e79a8a4aea8..e2c8573a77a 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -413,6 +413,10 @@ func (exeNode *ExecutionNode) LoadBlobService( opts = append(opts, blob.WithReprovideInterval(-1)) } + if !node.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + if exeNode.exeConf.blobstoreRateLimit > 0 && exeNode.exeConf.blobstoreBurstLimit > 0 { opts = append(opts, blob.WithRateLimit(float64(exeNode.exeConf.blobstoreRateLimit), exeNode.exeConf.blobstoreBurstLimit)) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 2721552642c..3b690bd0b46 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -186,6 +186,13 @@ type BaseConfig struct { // This is only meaningful to Access and Execution nodes. BitswapReprovideEnabled bool + // BitswapBloomCacheEnabled configures whether the Bitswap bloom cache is enabled. + // When disabled, uses a plain blockstore instead of cached blockstore, avoiding + // the CPU cost of building the bloom filter on startup. Pebble's built-in bloom + // filters (persisted in SSTables) are still used for efficient lookups. + // This is only meaningful to Access and Execution nodes. + BitswapBloomCacheEnabled bool + TransactionFeesDisabled bool } @@ -303,6 +310,7 @@ func DefaultBaseConfig() *BaseConfig { ComplianceConfig: compliance.DefaultConfig(), DhtSystemEnabled: true, BitswapReprovideEnabled: true, + BitswapBloomCacheEnabled: true, // default: use cached blockstore } } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 6cbaafbbda7..992684459a7 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1200,6 +1200,10 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS ), } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.PublicExecutionDataService, ds, opts...) if err != nil { diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 142f2d55c6d..e5008b345f4 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -213,6 +213,10 @@ func (fnb *FlowNodeBuilder) BaseFlags() { "bitswap-reprovide-enabled", defaultConfig.BitswapReprovideEnabled, "[experimental] whether to enable bitswap reproviding. This is an experimental feature. Use with caution.") + fnb.flags.BoolVar(&fnb.BaseConfig.BitswapBloomCacheEnabled, + "bitswap-bloom-cache-enabled", + defaultConfig.BitswapBloomCacheEnabled, + "[experimental] whether to enable bitswap bloom cache. When disabled, uses a plain blockstore instead of cached blockstore, avoiding the CPU cost of building the bloom filter on startup. Pebble's built-in bloom filters (persisted in SSTables) are still used. This is an experimental feature. Use with caution.") // dynamic node startup flags fnb.flags.StringVar(&fnb.BaseConfig.DynamicStartupANPubkey, diff --git a/network/p2p/blob/blob_service.go b/network/p2p/blob/blob_service.go index b4a1257b200..74107bde431 100644 --- a/network/p2p/blob/blob_service.go +++ b/network/p2p/blob/blob_service.go @@ -55,6 +55,7 @@ var _ component.Component = (*blobService)(nil) type BlobServiceConfig struct { ReprovideInterval time.Duration // the interval at which the DHT provider entries are refreshed BitswapOptions []bitswap.Option // options to pass to the Bitswap service + SkipBloomCache bool // if true, skip the bloom cache and use plain blockstore } // WithReprovideInterval sets the interval at which DHT provider entries are refreshed @@ -98,6 +99,15 @@ func WithRateLimit(r float64, b int) network.BlobServiceOption { } } +// WithSkipBloomCache disables the bloom cache, using a plain blockstore instead. +// This avoids the CPU cost of building the bloom filter on startup by scanning all keys. +// Pebble's built-in bloom filters (persisted in SSTables) are still used for efficient lookups. +func WithSkipBloomCache(skip bool) network.BlobServiceOption { + return func(bs network.BlobService) { + bs.(*blobService).config.SkipBloomCache = skip + } +} + // NewBlobService creates a new BlobService. func NewBlobService( host host.Host, @@ -109,26 +119,38 @@ func NewBlobService( opts ...network.BlobServiceOption, ) (*blobService, error) { bsNetwork := bsnet.NewFromIpfsHost(host, r, bsnet.Prefix(protocol.ID(prefix))) - blockStore, err := blockstore.CachedBlockstore( - context.Background(), - blockstore.NewBlockstore(ds), - blockstore.DefaultCacheOpts(), - ) - if err != nil { - return nil, fmt.Errorf("failed to create cached blockstore: %w", err) - } bs := &blobService{ prefix: prefix, config: &BlobServiceConfig{ ReprovideInterval: DefaultReprovideInterval, + SkipBloomCache: false, // default: use cached blockstore }, - blockStore: blockStore, } + // Apply options before creating blockstore, as SkipBloomCache affects blockstore creation for _, opt := range opts { opt(bs) } + // Create blockstore based on config + var blockStore blockstore.Blockstore + if bs.config.SkipBloomCache { + // Use plain blockstore - Pebble's built-in bloom filters are sufficient + blockStore = blockstore.NewBlockstore(ds) + } else { + // Use cached blockstore with bloom filter (default behavior) + cachedBlockStore, err := blockstore.CachedBlockstore( + context.Background(), + blockstore.NewBlockstore(ds), + blockstore.DefaultCacheOpts(), + ) + if err != nil { + return nil, fmt.Errorf("failed to create cached blockstore: %w", err) + } + blockStore = cachedBlockStore + } + bs.blockStore = blockStore + cm := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { btswp := bitswap.New(ctx, bsNetwork, bs.blockStore, bs.config.BitswapOptions...) From 15450bfadc92ecd9c516fc73408844ff3056da82 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 11:08:26 -0800 Subject: [PATCH 113/126] fix lint --- cmd/node_builder.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 3b690bd0b46..7d9e67e7be2 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -304,12 +304,12 @@ func DefaultBaseConfig() *BaseConfig { Duration: 10 * time.Second, }, - HeroCacheMetricsEnable: false, - SyncCoreConfig: chainsync.DefaultConfig(), - CodecFactory: codecFactory, - ComplianceConfig: compliance.DefaultConfig(), - DhtSystemEnabled: true, - BitswapReprovideEnabled: true, + HeroCacheMetricsEnable: false, + SyncCoreConfig: chainsync.DefaultConfig(), + CodecFactory: codecFactory, + ComplianceConfig: compliance.DefaultConfig(), + DhtSystemEnabled: true, + BitswapReprovideEnabled: true, BitswapBloomCacheEnabled: true, // default: use cached blockstore } } From c4e822806f959a4535408b31c089a0a09df33c4c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 14:30:01 -0800 Subject: [PATCH 114/126] log block height --- .../requester/execution_data_requester.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 2fa64205f47..7077b43d244 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -326,10 +326,13 @@ func (e *executionDataRequester) processBlockJob(ctx irrecoverable.SignalerConte return } - // errors are thrown as irrecoverable errors except context cancellation, and invalid blobs - // invalid blobs are logged, and never completed, which will halt downloads after maxSearchAhead - // is reached. - e.log.Error().Err(err).Str("job_id", string(job.ID())).Msg("error encountered while processing block job") + // errors are thrown as irrecoverable errors except context cancellation, which is usually + // triggered by restart, and invalid blobs invalid blobs are logged, and never completed, + // which will halt downloads after maxSearchAhead is reached. + e.log.Error().Err(err). + Hex("block_id", logging.ID(header.ID())). + Uint64("height", header.Height). + Str("job_id", string(job.ID())).Msg("error encountered while processing block job") } // processSealedHeight downloads ExecutionData for the given block height. From d71309e37899c76db7f0d65373b2cc1d61aae9fd Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 14:28:22 -0800 Subject: [PATCH 115/126] warn about large iteration --- module/state_synchronization/indexer/indexer.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index 9e6cea06586..85f5f974864 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -158,7 +158,9 @@ func (i *Indexer) onBlockIndexed() error { highestIndexedHeight := i.jobConsumer.LastProcessedIndex() if lastProcessedHeight < highestIndexedHeight { - i.log.Info().Msgf("notifying processed heights from %d to %d", lastProcessedHeight+1, highestIndexedHeight) + if lastProcessedHeight+1000 < highestIndexedHeight { + i.log.Warn().Msgf("notifying processed heights from %d to %d", lastProcessedHeight+1, highestIndexedHeight) + } // we need loop here because it's possible for a height to be missed here, // we should guarantee all heights are processed for height := lastProcessedHeight + 1; height <= highestIndexedHeight; height++ { From 38ef51d3e71bec5b739f27ad19865913bc277958 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 14:53:26 -0800 Subject: [PATCH 116/126] requester to return the cached store --- cmd/access/node_builder/access_node_builder.go | 1 - .../execution_data_index/provider.go | 3 ++- .../collection_sync/factory/access_builder.go | 14 ++++++-------- .../factory/execution_data_processor.go | 8 +------- .../execution_data_requester.go | 1 + .../requester/execution_data_requester.go | 4 ++++ 6 files changed, 14 insertions(+), 17 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cd0ef2db77f..12fa2391ebb 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2209,7 +2209,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.Logger, builder.executionDataSyncEnabled, builder.collectionSync, - notNil(builder.ExecutionDataCache), notNil(builder.ExecutionDataRequester), notNil(collectionIndexedHeight), notNil(builder.blockCollectionIndexer), diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go index 82635006123..62a2c32e217 100644 --- a/engine/access/collection_sync/execution_data_index/provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -2,6 +2,7 @@ package execution_data_index import ( "context" + "fmt" "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" @@ -38,7 +39,7 @@ func (p *executionDataProvider) HighestIndexedHeight() uint64 { func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { blockExecutionData, err := p.cache.ByHeight(ctx, height) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to get execution data for height %d: %w", height, err) } // TODO: to support processing data from unsealed blocks, we should check here that the collections diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 0c073881b46..b6c4c8eb6c2 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -8,6 +8,7 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" @@ -116,7 +117,6 @@ func CreateProcessedLastFullBlockHeightModule( // - log: Logger for logging operations // - executionDataSyncEnabled: Whether execution data sync is enabled // - collectionSyncMode: The collection sync mode -// - executionDataCache: Execution data cache // - executionDataRequester: Execution data requester // - collectionIndexedHeight: Consumer progress for collection indexed height // - blockCollectionIndexer: Block collection indexer @@ -132,7 +132,6 @@ func CreateExecutionDataProcessorComponent( log zerolog.Logger, executionDataSyncEnabled bool, collectionSyncMode CollectionSyncMode, - executionDataCache execution_data.ExecutionDataCache, executionDataRequester state_synchronization.ExecutionDataRequester, collectionIndexedHeight storage.ConsumerProgress, blockCollectionIndexer collection_sync.BlockCollectionIndexer, @@ -164,15 +163,14 @@ func CreateExecutionDataProcessorComponent( Bool("execution_data_sync_enabled", executionDataSyncEnabled). Msg("creating execution data processor") - if executionDataCache == nil { - return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") - } - + executionDataProvider := execution_data_index.NewExecutionDataProvider( + executionDataRequester.GetCachedStore(), + executionDataRequester, + ) // Create execution data processor executionDataProcessor, err := createExecutionDataProcessor( log, - executionDataCache, - executionDataRequester, + executionDataProvider, collectionIndexedHeight, blockCollectionIndexer, func(indexedHeight uint64) { diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index 25091b14803..d15c7e53055 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -8,8 +8,6 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/module/counters" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" - "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/storage" ) @@ -30,15 +28,11 @@ import ( // No errors are expected during normal operation. func createExecutionDataProcessor( log zerolog.Logger, - cache execution_data.ExecutionDataCache, - executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, + executionDataProvider collection_sync.ExecutionDataProvider, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, onIndexedCallback func(uint64), ) (*execution_data_index.ExecutionDataProcessor, error) { - // Create execution data provider - executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataIndexedHeight) - // Convert ConsumerProgress to PersistentStrictMonotonicCounter processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) if err != nil { diff --git a/module/state_synchronization/execution_data_requester.go b/module/state_synchronization/execution_data_requester.go index d9e9725c1a3..9902c4ee0f7 100644 --- a/module/state_synchronization/execution_data_requester.go +++ b/module/state_synchronization/execution_data_requester.go @@ -19,4 +19,5 @@ type ExecutionDataIndexedHeight interface { type ExecutionDataRequester interface { component.Component ExecutionDataIndexedHeight + GetCachedStore() execution_data.ExecutionDataCache } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 7077b43d244..9dfc89e3a9e 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -274,6 +274,10 @@ func (e *executionDataRequester) HighestConsecutiveHeight() uint64 { return e.blockConsumer.LastProcessedIndex() } +func (e *executionDataRequester) GetCachedStore() execution_data.ExecutionDataCache { + return e.execDataCache +} + // runBlockConsumer runs the blockConsumer component func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { err := util.WaitClosed(ctx, e.downloader.Ready()) From d1250a3399578444d667685376e390a65f07a70b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 15:19:10 -0800 Subject: [PATCH 117/126] fix lint --- cmd/node_builder.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 3b690bd0b46..7d9e67e7be2 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -304,12 +304,12 @@ func DefaultBaseConfig() *BaseConfig { Duration: 10 * time.Second, }, - HeroCacheMetricsEnable: false, - SyncCoreConfig: chainsync.DefaultConfig(), - CodecFactory: codecFactory, - ComplianceConfig: compliance.DefaultConfig(), - DhtSystemEnabled: true, - BitswapReprovideEnabled: true, + HeroCacheMetricsEnable: false, + SyncCoreConfig: chainsync.DefaultConfig(), + CodecFactory: codecFactory, + ComplianceConfig: compliance.DefaultConfig(), + DhtSystemEnabled: true, + BitswapReprovideEnabled: true, BitswapBloomCacheEnabled: true, // default: use cached blockstore } } From 8e51aae03cddcab2f05d2053309eb054f604d271 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 15:21:42 -0800 Subject: [PATCH 118/126] fix mock --- .../mock/execution_data_requester.go | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/module/state_synchronization/mock/execution_data_requester.go b/module/state_synchronization/mock/execution_data_requester.go index 06c0bf8921c..fd1569541f9 100644 --- a/module/state_synchronization/mock/execution_data_requester.go +++ b/module/state_synchronization/mock/execution_data_requester.go @@ -3,7 +3,9 @@ package mock import ( + execution_data "github.com/onflow/flow-go/module/executiondatasync/execution_data" irrecoverable "github.com/onflow/flow-go/module/irrecoverable" + mock "github.com/stretchr/testify/mock" ) @@ -32,6 +34,26 @@ func (_m *ExecutionDataRequester) Done() <-chan struct{} { return r0 } +// GetCachedStore provides a mock function with no fields +func (_m *ExecutionDataRequester) GetCachedStore() execution_data.ExecutionDataCache { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetCachedStore") + } + + var r0 execution_data.ExecutionDataCache + if rf, ok := ret.Get(0).(func() execution_data.ExecutionDataCache); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(execution_data.ExecutionDataCache) + } + } + + return r0 +} + // HighestConsecutiveHeight provides a mock function with no fields func (_m *ExecutionDataRequester) HighestConsecutiveHeight() uint64 { ret := _m.Called() From 484ff185230b670b1d9d7e9eb6a74476498ea65c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 4 Dec 2025 16:18:45 -0800 Subject: [PATCH 119/126] fix tests --- .../collection_sync/fetcher/missing_collection_queue_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go index 706e32beae8..9ca4da7a031 100644 --- a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go +++ b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go @@ -53,7 +53,7 @@ func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { // Step 4: Receiving one collection doesn't complete the block, IsHeightQueued still returns true collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") - assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") + assert.Equal(t, blockHeight, heightReturned, "should return height even when block is not complete") assert.False(t, complete, "block should not be complete with only one collection") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued after receiving one collection") @@ -117,7 +117,7 @@ func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { // Receive only one collection (block is not complete) collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") - assert.Equal(t, uint64(0), heightReturned, "should not return height when block is not complete") + assert.Equal(t, blockHeight, heightReturned, "should return height even when block is not complete") assert.False(t, complete, "block should not be complete with only one collection") assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") From 75d04b91b531bad9af46717dbfebd780d5c080a7 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 5 Dec 2025 09:07:17 -0800 Subject: [PATCH 120/126] revert to use execution data cache --- cmd/access/node_builder/access_node_builder.go | 1 + .../execution_data_index/provider.go | 3 +-- .../collection_sync/factory/access_builder.go | 14 ++++++++------ .../factory/execution_data_processor.go | 8 +++++++- .../execution_data_requester.go | 1 - .../requester/execution_data_requester.go | 4 ---- 6 files changed, 17 insertions(+), 14 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 12fa2391ebb..cd0ef2db77f 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2209,6 +2209,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.Logger, builder.executionDataSyncEnabled, builder.collectionSync, + notNil(builder.ExecutionDataCache), notNil(builder.ExecutionDataRequester), notNil(collectionIndexedHeight), notNil(builder.blockCollectionIndexer), diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go index 62a2c32e217..82635006123 100644 --- a/engine/access/collection_sync/execution_data_index/provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -2,7 +2,6 @@ package execution_data_index import ( "context" - "fmt" "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" @@ -39,7 +38,7 @@ func (p *executionDataProvider) HighestIndexedHeight() uint64 { func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { blockExecutionData, err := p.cache.ByHeight(ctx, height) if err != nil { - return nil, fmt.Errorf("failed to get execution data for height %d: %w", height, err) + return nil, err } // TODO: to support processing data from unsealed blocks, we should check here that the collections diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index b6c4c8eb6c2..0c073881b46 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -8,7 +8,6 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/engine/access/collection_sync" - "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" @@ -117,6 +116,7 @@ func CreateProcessedLastFullBlockHeightModule( // - log: Logger for logging operations // - executionDataSyncEnabled: Whether execution data sync is enabled // - collectionSyncMode: The collection sync mode +// - executionDataCache: Execution data cache // - executionDataRequester: Execution data requester // - collectionIndexedHeight: Consumer progress for collection indexed height // - blockCollectionIndexer: Block collection indexer @@ -132,6 +132,7 @@ func CreateExecutionDataProcessorComponent( log zerolog.Logger, executionDataSyncEnabled bool, collectionSyncMode CollectionSyncMode, + executionDataCache execution_data.ExecutionDataCache, executionDataRequester state_synchronization.ExecutionDataRequester, collectionIndexedHeight storage.ConsumerProgress, blockCollectionIndexer collection_sync.BlockCollectionIndexer, @@ -163,14 +164,15 @@ func CreateExecutionDataProcessorComponent( Bool("execution_data_sync_enabled", executionDataSyncEnabled). Msg("creating execution data processor") - executionDataProvider := execution_data_index.NewExecutionDataProvider( - executionDataRequester.GetCachedStore(), - executionDataRequester, - ) + if executionDataCache == nil { + return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") + } + // Create execution data processor executionDataProcessor, err := createExecutionDataProcessor( log, - executionDataProvider, + executionDataCache, + executionDataRequester, collectionIndexedHeight, blockCollectionIndexer, func(indexedHeight uint64) { diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go index d15c7e53055..25091b14803 100644 --- a/engine/access/collection_sync/factory/execution_data_processor.go +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/storage" ) @@ -28,11 +30,15 @@ import ( // No errors are expected during normal operation. func createExecutionDataProcessor( log zerolog.Logger, - executionDataProvider collection_sync.ExecutionDataProvider, + cache execution_data.ExecutionDataCache, + executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, processedHeight storage.ConsumerProgress, indexer collection_sync.BlockCollectionIndexer, onIndexedCallback func(uint64), ) (*execution_data_index.ExecutionDataProcessor, error) { + // Create execution data provider + executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataIndexedHeight) + // Convert ConsumerProgress to PersistentStrictMonotonicCounter processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) if err != nil { diff --git a/module/state_synchronization/execution_data_requester.go b/module/state_synchronization/execution_data_requester.go index 9902c4ee0f7..d9e9725c1a3 100644 --- a/module/state_synchronization/execution_data_requester.go +++ b/module/state_synchronization/execution_data_requester.go @@ -19,5 +19,4 @@ type ExecutionDataIndexedHeight interface { type ExecutionDataRequester interface { component.Component ExecutionDataIndexedHeight - GetCachedStore() execution_data.ExecutionDataCache } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 9dfc89e3a9e..7077b43d244 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -274,10 +274,6 @@ func (e *executionDataRequester) HighestConsecutiveHeight() uint64 { return e.blockConsumer.LastProcessedIndex() } -func (e *executionDataRequester) GetCachedStore() execution_data.ExecutionDataCache { - return e.execDataCache -} - // runBlockConsumer runs the blockConsumer component func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { err := util.WaitClosed(ctx, e.downloader.Ready()) From 5b07e927c69596c5f7f7d38ea65e0658128e0402 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 5 Dec 2025 10:21:13 -0800 Subject: [PATCH 121/126] add retry --- .../access/collection_sync/collection_sync.go | 11 ++- .../execution_data_index/processor.go | 74 ++++++++++++++++++- .../execution_data_index/provider.go | 1 + 3 files changed, 83 insertions(+), 3 deletions(-) diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go index 4878e51222d..65940429c21 100644 --- a/engine/access/collection_sync/collection_sync.go +++ b/engine/access/collection_sync/collection_sync.go @@ -25,7 +25,7 @@ type MissingCollectionQueue interface { // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). IsHeightQueued(height uint64) bool - // Size returns the number of missing collections currently in the queue. + // Size returns the number of missing heights currently in the queue. Size() uint // GetMissingCollections returns all collection IDs that are currently missing across all block heights. @@ -67,23 +67,32 @@ type BlockProcessor interface { type Fetcher interface { component.Component ProgressReader + // OnFinalizedBlock notifies the fetcher that a new block has been finalized. OnFinalizedBlock() + // MissingCollectionQueueSize returns the number of missing height currently in the queue. Size() uint } // ExecutionDataProvider provides the latest height for which execution data indexer has collections. // This can be nil if execution data indexing is disabled. type ExecutionDataProvider interface { + // HighestIndexedHeight returns the highest block height for which execution data is available. + // It garautnees that all heights below it also have execution data available to be called + // with GetExecutionDataByHeight. HighestIndexedHeight() uint64 + + // It might return [execution_data.BlobNotFoundError] error, if some CID in the blob tree could not be found from the blobstore. GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) } // ExecutionDataProcessor processes execution data when new execution data is available. type ExecutionDataProcessor interface { + // OnNewExectuionData notifies the processor that new execution data is available for processing. OnNewExectuionData() } // ProgressReader provides the current progress of collection fetching/indexing. type ProgressReader interface { + // ProcessedHeight returns the highest block height for which collections have been processed. ProcessedHeight() uint64 } diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go index a0b3ba3631f..e40e45dce93 100644 --- a/engine/access/collection_sync/execution_data_index/processor.go +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -2,13 +2,16 @@ package execution_data_index import ( "fmt" + "time" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/irrecoverable" ) @@ -61,6 +64,66 @@ func (edp *ExecutionDataProcessor) OnNewExectuionData() { edp.newExecutionDataIndexed.Notify() } +// retryOnBlobNotFound executes the given function and retries it every 5 seconds +// if it returns a BlobNotFoundError. Each retry attempt is logged. +// Returns the result of the function call, or an error if retries fail or a non-BlobNotFoundError occurs. +func retryOnBlobNotFound( + ctx irrecoverable.SignalerContext, + log zerolog.Logger, + height uint64, + fn func() ([]*flow.Collection, error), +) ([]*flow.Collection, error) { + collections, err := fn() + if err == nil { + return collections, nil + } + + // If the error is not BlobNotFoundError, return immediately + if !execution_data.IsBlobNotFoundError(err) { + return nil, err + } + + retryTicker := time.NewTicker(5 * time.Second) + defer retryTicker.Stop() + + attempt := 1 + log.Error(). + Uint64("height", height). + Err(err). + Int("attempt", attempt). + Msg("execution data not found, retrying every 5 seconds") + + for { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-retryTicker.C: + attempt++ + collections, err = fn() + if err == nil { + log.Info(). + Uint64("height", height). + Int("attempt", attempt). + Msg("successfully retrieved execution data after retry") + return collections, nil + } + + // If error is still BlobNotFoundError, continue retrying + if execution_data.IsBlobNotFoundError(err) { + log.Error(). + Uint64("height", height). + Err(err). + Int("attempt", attempt). + Msg("execution data still not found, retrying") + continue + } + + // If error changed to something else, return it + return nil, err + } + } +} + func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { ready() @@ -76,10 +139,17 @@ func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, lowestMissing := edp.processedHeight.Value() + 1 for height := lowestMissing; height <= highestAvailableHeight; height++ { - // TODO: This logic supports ingesting execution data from sealed blocks. Once support is + // TODO: This logic only supports ingesting execution data from sealed blocks. Once support is // added for syncing execution data for unsealed results, this logic will need to be updated // to account for execution forks. - collections, err := edp.provider.GetExecutionDataByHeight(ctx, height) + // Fetch execution data for this height. If the blob is not found (BlobNotFoundError), + // retryOnBlobNotFound will automatically retry every 5 seconds until it's available or a different error occurs. + // retry is not needed, because the provider is supposed to guarantee that all heights below + // HighestIndexedHeight have execution data available. + // this is for debugging purpose for now. + collections, err := retryOnBlobNotFound(ctx, edp.log, height, func() ([]*flow.Collection, error) { + return edp.provider.GetExecutionDataByHeight(ctx, height) + }) if err != nil { ctx.Throw(fmt.Errorf("collection_sync execution data processor: failed to get execution data for height %d: %w", height, err)) diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go index 82635006123..68953e5ab4c 100644 --- a/engine/access/collection_sync/execution_data_index/provider.go +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -35,6 +35,7 @@ func (p *executionDataProvider) HighestIndexedHeight() uint64 { } // GetExecutionDataByHeight returns the execution data for the given block height. +// It might return [execution_data.BlobNotFoundError] error, if some CID in the blob tree could not be found from the blobstore. func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { blockExecutionData, err := p.cache.ByHeight(ctx, height) if err != nil { From f08edf4527694b9d757d85d086f4a68f33659439 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 5 Dec 2025 14:45:23 -0800 Subject: [PATCH 122/126] remove notification data --- .../collection_sync/factory/access_builder.go | 2 +- .../tracker/execution_data_tracker.go | 5 +- .../tracker/mock/execution_data_tracker.go | 8 +-- .../execution_data_requester.go | 3 +- .../state_synchronization/indexer/indexer.go | 2 +- .../indexer/indexer_test.go | 2 +- .../mock/execution_data_requester.go | 22 ------- .../requester/distributer.go | 5 +- .../requester/execution_data_requester.go | 2 +- .../execution_data_requester_test.go | 63 ++++++++++++++----- 10 files changed, 58 insertions(+), 56 deletions(-) diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go index 0c073881b46..422fc7fe2cd 100644 --- a/engine/access/collection_sync/factory/access_builder.go +++ b/engine/access/collection_sync/factory/access_builder.go @@ -193,7 +193,7 @@ func CreateExecutionDataProcessorComponent( // Initialize last_full_finalized_block_height metric accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) - distributor.AddOnExecutionDataReceivedConsumer(func(executionData *execution_data.BlockExecutionDataEntity) { + distributor.AddOnExecutionDataReceivedConsumer(func() { executionDataProcessor.OnNewExectuionData() }) diff --git a/engine/access/subscription/tracker/execution_data_tracker.go b/engine/access/subscription/tracker/execution_data_tracker.go index 7637c14e0df..ae66adba194 100644 --- a/engine/access/subscription/tracker/execution_data_tracker.go +++ b/engine/access/subscription/tracker/execution_data_tracker.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/module/state_synchronization/indexer" "github.com/onflow/flow-go/state/protocol" @@ -57,7 +56,7 @@ type ExecutionDataTracker interface { GetHighestHeight() uint64 // OnExecutionData is used to notify the tracker when a new execution data is received. - OnExecutionData(*execution_data.BlockExecutionDataEntity) + OnExecutionData() } var _ ExecutionDataTracker = (*ExecutionDataTrackerImpl)(nil) @@ -221,7 +220,7 @@ func (e *ExecutionDataTrackerImpl) GetHighestHeight() uint64 { } // OnExecutionData is used to notify the tracker when a new execution data is received. -func (e *ExecutionDataTrackerImpl) OnExecutionData(executionData *execution_data.BlockExecutionDataEntity) { +func (e *ExecutionDataTrackerImpl) OnExecutionData() { e.broadcaster.Publish() } diff --git a/engine/access/subscription/tracker/mock/execution_data_tracker.go b/engine/access/subscription/tracker/mock/execution_data_tracker.go index ccfad6bc8b4..3501457925f 100644 --- a/engine/access/subscription/tracker/mock/execution_data_tracker.go +++ b/engine/access/subscription/tracker/mock/execution_data_tracker.go @@ -6,8 +6,6 @@ import ( context "context" flow "github.com/onflow/flow-go/model/flow" - execution_data "github.com/onflow/flow-go/module/executiondatasync/execution_data" - mock "github.com/stretchr/testify/mock" ) @@ -146,9 +144,9 @@ func (_m *ExecutionDataTracker) GetStartHeightFromLatest(_a0 context.Context) (u return r0, r1 } -// OnExecutionData provides a mock function with given fields: _a0 -func (_m *ExecutionDataTracker) OnExecutionData(_a0 *execution_data.BlockExecutionDataEntity) { - _m.Called(_a0) +// OnExecutionData provides a mock function with no fields +func (_m *ExecutionDataTracker) OnExecutionData() { + _m.Called() } // NewExecutionDataTracker creates a new instance of ExecutionDataTracker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. diff --git a/module/state_synchronization/execution_data_requester.go b/module/state_synchronization/execution_data_requester.go index d9e9725c1a3..948005fbf81 100644 --- a/module/state_synchronization/execution_data_requester.go +++ b/module/state_synchronization/execution_data_requester.go @@ -2,11 +2,10 @@ package state_synchronization import ( "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) // OnExecutionDataReceivedConsumer is a callback that is called ExecutionData is received for a new block -type OnExecutionDataReceivedConsumer func(*execution_data.BlockExecutionDataEntity) +type OnExecutionDataReceivedConsumer func() type ExecutionDataIndexedHeight interface { // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index 85f5f974864..a1738db0743 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -211,7 +211,7 @@ func (i *Indexer) HighestIndexedHeight() (uint64, error) { } // OnExecutionData is used to notify when new execution data is downloaded by the execution data requester jobqueue. -func (i *Indexer) OnExecutionData(_ *execution_data.BlockExecutionDataEntity) { +func (i *Indexer) OnExecutionData() { i.exeDataNotifier.Notify() } diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index 6d8dc5a1c05..719dbc9d189 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -107,7 +107,7 @@ func (w *indexerTest) run(ctx irrecoverable.SignalerContext, reachHeight uint64, unittest.RequireComponentsReadyBefore(w.t, testTimeout, w.worker) - w.worker.OnExecutionData(nil) + w.worker.OnExecutionData() select { case <-ctx.Done(): diff --git a/module/state_synchronization/mock/execution_data_requester.go b/module/state_synchronization/mock/execution_data_requester.go index fd1569541f9..06c0bf8921c 100644 --- a/module/state_synchronization/mock/execution_data_requester.go +++ b/module/state_synchronization/mock/execution_data_requester.go @@ -3,9 +3,7 @@ package mock import ( - execution_data "github.com/onflow/flow-go/module/executiondatasync/execution_data" irrecoverable "github.com/onflow/flow-go/module/irrecoverable" - mock "github.com/stretchr/testify/mock" ) @@ -34,26 +32,6 @@ func (_m *ExecutionDataRequester) Done() <-chan struct{} { return r0 } -// GetCachedStore provides a mock function with no fields -func (_m *ExecutionDataRequester) GetCachedStore() execution_data.ExecutionDataCache { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for GetCachedStore") - } - - var r0 execution_data.ExecutionDataCache - if rf, ok := ret.Get(0).(func() execution_data.ExecutionDataCache); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(execution_data.ExecutionDataCache) - } - } - - return r0 -} - // HighestConsecutiveHeight provides a mock function with no fields func (_m *ExecutionDataRequester) HighestConsecutiveHeight() uint64 { ret := _m.Called() diff --git a/module/state_synchronization/requester/distributer.go b/module/state_synchronization/requester/distributer.go index ac97fb07982..002fb250f81 100644 --- a/module/state_synchronization/requester/distributer.go +++ b/module/state_synchronization/requester/distributer.go @@ -3,7 +3,6 @@ package requester import ( "sync" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" ) @@ -27,11 +26,11 @@ func (p *ExecutionDataDistributor) AddOnExecutionDataReceivedConsumer(consumer s } // OnExecutionDataReceived is called when new execution data is received -func (p *ExecutionDataDistributor) OnExecutionDataReceived(executionData *execution_data.BlockExecutionDataEntity) { +func (p *ExecutionDataDistributor) OnExecutionDataReceived() { p.lock.RLock() defer p.lock.RUnlock() for _, consumer := range p.consumers { - consumer(executionData) + consumer() } } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 7077b43d244..d6e289cd97f 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -440,7 +440,7 @@ func (e *executionDataRequester) processNotificationJob(ctx irrecoverable.Signal Msgf("notifying for block") // send notifications - e.distributor.OnExecutionDataReceived(entry.ExecutionData) + e.distributor.OnExecutionDataReceived() jobComplete() e.metrics.NotificationSent(entry.Height) diff --git a/module/state_synchronization/requester/execution_data_requester_test.go b/module/state_synchronization/requester/execution_data_requester_test.go index e90405a7f0b..86ac2f4b67c 100644 --- a/module/state_synchronization/requester/execution_data_requester_test.go +++ b/module/state_synchronization/requester/execution_data_requester_test.go @@ -32,6 +32,7 @@ import ( synctest "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/state/protocol" statemock "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/operation/pebbleimpl" "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" @@ -45,6 +46,8 @@ type ExecutionDataRequesterSuite struct { db *pebble.DB downloader *exedatamock.Downloader distributor *ExecutionDataDistributor + edCache *cache.ExecutionDataCache + headers storage.Headers run edTestRun @@ -410,6 +413,8 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun heroCache := herocache.NewBlockExecutionData(subscription.DefaultCacheSize, logger, metricsCollector) edCache := cache.NewExecutionDataCache(suite.downloader, headers, seals, results, heroCache) + suite.edCache = edCache + suite.headers = headers followerDistributor := pubsub.NewFollowerDistributor() processedHeight := store.NewConsumerProgress(pebbleimpl.ToDB(suite.db), module.ConsumeProgressExecutionDataRequesterBlockHeight) @@ -450,7 +455,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTestHalts(edr state_synchr fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -477,7 +482,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTestPauseResume(edr state_ fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -515,7 +520,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTest(edr state_synchroniza fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -533,24 +538,48 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTest(edr state_synchroniza return fetchedExecutionData } -func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg *fetchTestRun, done func(), fetchedExecutionData map[flow.Identifier]*execution_data.BlockExecutionData) func(ed *execution_data.BlockExecutionDataEntity) { - return func(ed *execution_data.BlockExecutionDataEntity) { - if _, has := fetchedExecutionData[ed.BlockID]; has { - suite.T().Errorf("duplicate execution data for block %s", ed.BlockID) - return - } +func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg *fetchTestRun, done func(), fetchedExecutionData map[flow.Identifier]*execution_data.BlockExecutionData, edr state_synchronization.ExecutionDataRequester, edCache *cache.ExecutionDataCache, headers storage.Headers) func() { + var lastProcessedHeight uint64 = cfg.startHeight - 1 + return func() { + // Get the highest consecutive height that has execution data + highestHeight := edr.HighestConsecutiveHeight() + + // Process all heights from lastProcessedHeight + 1 to highestHeight + for height := lastProcessedHeight + 1; height <= highestHeight; height++ { + blockID, err := headers.BlockIDByHeight(height) + if err != nil { + suite.T().Errorf("failed to get block ID for height %d: %v", height, err) + return + } - fetchedExecutionData[ed.BlockID] = ed.BlockExecutionData - if _, ok := cfg.blocksByID[ed.BlockID]; !ok { - suite.T().Errorf("unknown execution data for block %s", ed.BlockID) - return - } + if _, has := fetchedExecutionData[blockID]; has { + suite.T().Errorf("duplicate execution data for block %s", blockID) + return + } - suite.T().Logf("notified of execution data for block %v height %d (%d/%d)", ed.BlockID, cfg.blocksByID[ed.BlockID].Height, len(fetchedExecutionData), cfg.sealedCount) + // Fetch execution data from cache + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + ed, err := edCache.ByBlockID(ctx, blockID) + cancel() + if err != nil { + suite.T().Errorf("failed to get execution data for block %s: %v", blockID, err) + return + } - if cfg.IsLastSeal(ed.BlockID) { - done() + fetchedExecutionData[blockID] = ed.BlockExecutionData + if _, ok := cfg.blocksByID[blockID]; !ok { + suite.T().Errorf("unknown execution data for block %s", blockID) + return + } + + suite.T().Logf("notified of execution data for block %v height %d (%d/%d)", blockID, height, len(fetchedExecutionData), cfg.sealedCount) + + if cfg.IsLastSeal(blockID) { + done() + } } + + lastProcessedHeight = highestHeight } } From e2d29daed4ac2a82faff445cc16bd65a47ddd929 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 5 Dec 2025 15:03:18 -0800 Subject: [PATCH 123/126] remove notification consumer --- .../node_builder/access_node_builder.go | 9 -- cmd/observer/node_builder/observer_builder.go | 10 -- .../requester/execution_data_requester.go | 132 ++---------------- .../execution_data_requester_test.go | 14 +- 4 files changed, 21 insertions(+), 144 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index cd0ef2db77f..a0a2ae7dc2b 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -569,7 +569,6 @@ func (builder *FlowAccessNodeBuilder) BuildConsensusFollower() *FlowAccessNodeBu func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccessNodeBuilder { var bs network.BlobService var processedBlockHeight storage.ConsumerProgressInitializer - var processedNotifications storage.ConsumerProgressInitializer var bsDependable *module.ProxiedReadyDoneAware var execDataDistributor *edrequester.ExecutionDataDistributor var execDataCacheBackend *herocache.BlockExecutionData @@ -604,13 +603,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess processedBlockHeight = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterBlockHeight) return nil }). - Module("processed notifications consumer progress", func(node *cmd.NodeConfig) error { - // Note: progress is stored in the datastore's DB since that is where the jobqueue - // writes execution data to. - db := builder.ExecutionDatastoreManager.DB() - processedNotifications = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterNotification) - return nil - }). Module("blobservice peer manager dependencies", func(node *cmd.NodeConfig) error { bsDependable = module.NewProxiedReadyDoneAware() builder.PeerManagerDependencies.Add(bsDependable) @@ -755,7 +747,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.ExecutionDataDownloader, executionDataCache, processedBlockHeight, - processedNotifications, builder.State, builder.Storage.Headers, builder.executionDataConfig, diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 9d73faeafbb..984269eccc7 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1102,7 +1102,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS var ds datastore.Batching var bs network.BlobService var processedBlockHeight storage.ConsumerProgressInitializer - var processedNotifications storage.ConsumerProgressInitializer var publicBsDependable *module.ProxiedReadyDoneAware var execDataDistributor *edrequester.ExecutionDataDistributor var execDataCacheBackend *herocache.BlockExecutionData @@ -1150,14 +1149,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS processedBlockHeight = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterBlockHeight) return nil }). - Module("processed notifications consumer progress", func(node *cmd.NodeConfig) error { - // Note: progress is stored in the datastore's DB since that is where the jobqueue - // writes execution data to. - db := builder.ExecutionDatastoreManager.DB() - - processedNotifications = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterNotification) - return nil - }). Module("blobservice peer manager dependencies", func(node *cmd.NodeConfig) error { publicBsDependable = module.NewProxiedReadyDoneAware() builder.PeerManagerDependencies.Add(publicBsDependable) @@ -1294,7 +1285,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.ExecutionDataDownloader, executionDataCache, processedBlockHeight, - processedNotifications, builder.State, builder.Storage.Headers, builder.executionDataConfig, diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index d6e289cd97f..e47f3af9aec 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/jobqueue" "github.com/onflow/flow-go/module/state_synchronization" - "github.com/onflow/flow-go/module/state_synchronization/requester/jobs" "github.com/onflow/flow-go/module/util" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" @@ -43,7 +42,7 @@ import ( // The requester listens to block finalization event, and checks if sealed height has been changed, // if changed, it create job for each un-downloaded and sealed height. // -// The requester is made up of 3 subcomponents: +// The requester is made up of 2 subcomponents: // // * OnBlockFinalized: receives block finalized events from the finalization registrar and // forwards them to the blockConsumer. @@ -51,23 +50,16 @@ import ( // * blockConsumer: is a jobqueue that receives block finalization events. On each event, // it checks for the latest sealed block, then uses a pool of workers to // download ExecutionData for each block from the network. After each -// successful download, the blockConsumer sends a notification to the -// notificationConsumer that a new ExecutionData is available. +// successful download, the blockConsumer notifies the distributor that a new +// ExecutionData is available. // -// * notificationConsumer: is a jobqueue that receives ExecutionData fetched events. On each event, -// it checks if ExecutionData for the next consecutive block height is -// available, then uses a single worker to send notifications to registered -// consumers. -// the registered consumers are guaranteed to receive each sealed block in -// consecutive height at least once. -// -// +------------------+ +---------------+ +----------------------+ -// -->| OnBlockFinalized |----->| blockConsumer | +-->| notificationConsumer | -// +------------------+ +-------+-------+ | +-----------+----------+ -// | | | -// +------+------+ | +------+------+ -// xN | Worker Pool |----+ x1 | Worker Pool |----> Registered consumers -// +-------------+ +-------------+ +// +------------------+ +---------------+ +// -->| OnBlockFinalized |----->| blockConsumer |----> Distributor +// +------------------+ +-------+-------+ +// | +// +------+------+ +// xN | Worker Pool | +// +-------------+ const ( // DefaultFetchTimeout is the default initial timeout for fetching ExecutionData from the @@ -125,14 +117,11 @@ type executionDataRequester struct { // Local db objects headers storage.Headers - executionDataReader *jobs.ExecutionDataReader - // Notifiers for queue consumers finalizationNotifier engine.Notifier // Job queues - blockConsumer *jobqueue.ComponentConsumer - notificationConsumer *jobqueue.ComponentConsumer + blockConsumer *jobqueue.ComponentConsumer execDataCache *cache.ExecutionDataCache distributor *ExecutionDataDistributor @@ -147,7 +136,6 @@ func New( downloader execution_data.Downloader, execDataCache *cache.ExecutionDataCache, processedHeight storage.ConsumerProgressInitializer, - processedNotifications storage.ConsumerProgressInitializer, state protocol.State, headers storage.Headers, cfg ExecutionDataConfig, @@ -165,8 +153,6 @@ func New( distributor: distributor, } - executionDataNotifier := engine.NewNotifier() - // jobqueue Jobs object that tracks sealed blocks by height. This is used by the blockConsumer // to get a sequential list of sealed blocks. sealedBlockReader := jobqueue.NewSealedBlockHeaderReader(state, headers) @@ -179,8 +165,7 @@ func New( // downloaded, it updates and persists the highest consecutive downloaded height with // `processedHeight`. That way, if the node crashes, it reads the `processedHeight` and resume // from `processedHeight + 1`. If the database is empty, rootHeight will be used to init the - // last processed height. Once the execution data is fetched and stored, it notifies - // `executionDataNotifier`. + // last processed height. Once the execution data is fetched and stored, it notifies the distributor. blockConsumer, err := jobqueue.NewComponentConsumer( e.log.With().Str("module", "block_consumer").Logger(), e.finalizationNotifier.Channel(), // to listen to finalization events to find newly sealed blocks @@ -196,64 +181,12 @@ func New( } e.blockConsumer = blockConsumer - // notifies notificationConsumer when new ExecutionData blobs are available - // SetPostNotifier will notify executionDataNotifier AFTER e.blockConsumer.LastProcessedIndex is updated. - // Even though it doesn't guarantee to notify for every height at least once, the notificationConsumer is - // able to guarantee to process every height at least once, because the notificationConsumer finds new jobs - // using executionDataReader which finds new heights using e.blockConsumer.LastProcessedIndex - e.blockConsumer.SetPostNotifier(func(module.JobID) { executionDataNotifier.Notify() }) - - // jobqueue Jobs object tracks downloaded execution data by height. This is used by the - // notificationConsumer to get downloaded execution data from storage. - e.executionDataReader = jobs.NewExecutionDataReader( - e.execDataCache, - e.config.FetchTimeout, - // method to get highest consecutive height that has downloaded execution data. it is used - // here by the notification job consumer to discover new jobs. - // Note: we don't want to notify notificationConsumer for a block if it has not downloaded - // execution data yet. - func() uint64 { - return e.blockConsumer.LastProcessedIndex() - }, - ) - - // TODO (leo): we don't have to keep the notification distributor, because if we add a new notification - // consumer, we would like to consume from the beginning instead of the last consumed height - // of the existing consumer. - // without the notification distributor, each notification consumer can also be - // simplified as a component with a signal channel as notification, and a worker loop to consume - // notifications and iterate all the way to the blockConsumer.LatestProcessedIndex(). - - // notificationConsumer consumes `OnExecutionDataFetched` events, and ensures its consumer - // receives this event in consecutive block height order. - // It listens to events from `executionDataNotifier`, which is delivered when - // a block's execution data is downloaded and stored, and checks the `executionDataCache` to - // find if the next un-processed consecutive height is available. - // To know what's the height of the next un-processed consecutive height, it reads the latest - // consecutive height in `processedNotifications`. And it's persisted in storage to be crash-resistant. - // When a new consecutive height is available, it calls `processNotificationJob` to notify all the - // `e.consumers`. - // Note: the `e.consumers` will be guaranteed to receive at least one `OnExecutionDataFetched` event - // for each sealed block in consecutive block height order. - e.notificationConsumer, err = jobqueue.NewComponentConsumer( - e.log.With().Str("module", "notification_consumer").Logger(), - executionDataNotifier.Channel(), // listen for notifications from the block consumer - processedNotifications, // read and persist the notified height - e.executionDataReader, // read execution data by height - e.config.InitialBlockHeight, // initial "last processed" height for empty db - e.processNotificationJob, // process the job to send notifications for an execution data - 1, // use a single worker to ensure notification is delivered in consecutive order - 0, // search ahead limit controlled by worker count - ) - if err != nil { - return nil, fmt.Errorf("failed to create notification consumer: %w", err) - } + e.blockConsumer.SetPostNotifier(func(module.JobID) { e.distributor.OnExecutionDataReceived() }) e.metrics.ExecutionDataFetchFinished(0, true, e.blockConsumer.LastProcessedIndex()) e.Component = component.NewComponentManagerBuilder(). AddWorker(e.runBlockConsumer). - AddWorker(e.runNotificationConsumer). Build() // register callback with finalization registrar @@ -281,11 +214,6 @@ func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerCont return // context cancelled } - err = util.WaitClosed(ctx, e.notificationConsumer.Ready()) - if err != nil { - return // context cancelled - } - e.blockConsumer.Start(ctx) err = util.WaitClosed(ctx, e.blockConsumer.Ready()) @@ -296,19 +224,6 @@ func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerCont <-e.blockConsumer.Done() } -// runNotificationConsumer runs the notificationConsumer component -func (e *executionDataRequester) runNotificationConsumer(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - e.executionDataReader.AddContext(ctx) - e.notificationConsumer.Start(ctx) - - err := util.WaitClosed(ctx, e.notificationConsumer.Ready()) - if err == nil { - ready() - } - - <-e.notificationConsumer.Done() -} - // Fetch Worker Methods // processBlockJob consumes jobs from the blockConsumer and attempts to download an ExecutionData @@ -425,27 +340,6 @@ func (e *executionDataRequester) processFetchRequest(parentCtx irrecoverable.Sig return nil } -// Notification Worker Methods - -func (e *executionDataRequester) processNotificationJob(ctx irrecoverable.SignalerContext, job module.Job, jobComplete func()) { - // convert job into a block entry - entry, err := jobs.JobToBlockEntry(job) - if err != nil { - ctx.Throw(fmt.Errorf("failed to convert job to entry: %w", err)) - } - - e.log.Debug(). - Hex("block_id", logging.ID(entry.BlockID)). - Uint64("height", entry.Height). - Msgf("notifying for block") - - // send notifications - e.distributor.OnExecutionDataReceived() - jobComplete() - - e.metrics.NotificationSent(entry.Height) -} - func isInvalidBlobError(err error) bool { var malformedDataError *execution_data.MalformedDataError var blobSizeLimitExceededError *execution_data.BlobSizeLimitExceededError diff --git a/module/state_synchronization/requester/execution_data_requester_test.go b/module/state_synchronization/requester/execution_data_requester_test.go index 86ac2f4b67c..0f8f5a9b451 100644 --- a/module/state_synchronization/requester/execution_data_requester_test.go +++ b/module/state_synchronization/requester/execution_data_requester_test.go @@ -418,7 +418,6 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun followerDistributor := pubsub.NewFollowerDistributor() processedHeight := store.NewConsumerProgress(pebbleimpl.ToDB(suite.db), module.ConsumeProgressExecutionDataRequesterBlockHeight) - processedNotification := store.NewConsumerProgress(pebbleimpl.ToDB(suite.db), module.ConsumeProgressExecutionDataRequesterNotification) edr, err := New( logger, @@ -426,7 +425,6 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun suite.downloader, edCache, processedHeight, - processedNotification, state, headers, ExecutionDataConfig{ @@ -540,10 +538,14 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTest(edr state_synchroniza func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg *fetchTestRun, done func(), fetchedExecutionData map[flow.Identifier]*execution_data.BlockExecutionData, edr state_synchronization.ExecutionDataRequester, edCache *cache.ExecutionDataCache, headers storage.Headers) func() { var lastProcessedHeight uint64 = cfg.startHeight - 1 + var mu sync.Mutex return func() { + mu.Lock() + defer mu.Unlock() + // Get the highest consecutive height that has execution data highestHeight := edr.HighestConsecutiveHeight() - + // Process all heights from lastProcessedHeight + 1 to highestHeight for height := lastProcessedHeight + 1; height <= highestHeight; height++ { blockID, err := headers.BlockIDByHeight(height) @@ -553,8 +555,8 @@ func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg } if _, has := fetchedExecutionData[blockID]; has { - suite.T().Errorf("duplicate execution data for block %s", blockID) - return + // Skip if already processed + continue } // Fetch execution data from cache @@ -578,7 +580,7 @@ func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg done() } } - + lastProcessedHeight = highestHeight } } From bbc700fed0229f095baeea5713d3fd6fe94db808 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 8 Dec 2025 12:03:47 -0800 Subject: [PATCH 124/126] revert blob change --- .../cohort3/execution_state_sync_test.go | 12 ++++++ network/p2p/blob/blob_service.go | 39 ++++++++++--------- 2 files changed, 33 insertions(+), 18 deletions(-) diff --git a/integration/tests/access/cohort3/execution_state_sync_test.go b/integration/tests/access/cohort3/execution_state_sync_test.go index 08cb0d6c8dc..fc2e46d402d 100644 --- a/integration/tests/access/cohort3/execution_state_sync_test.go +++ b/integration/tests/access/cohort3/execution_state_sync_test.go @@ -196,13 +196,25 @@ func (s *ExecutionStateSyncSuite) executionDataForHeight(ctx context.Context, no BlockId: header.ID[:], EventEncodingVersion: entities.EventEncodingVersion_CCF_V0, }) + if err != nil { + s.log.Info(). + Uint64("height", height). + Hex("block_id", header.ID[:]). + Err(err). + Msg("failed to get execution data") return err } blockED, err = convert.MessageToBlockExecutionData(ed.GetBlockExecutionData(), flow.Localnet.Chain()) s.Require().NoError(err, "could not convert execution data") + s.log.Info(). + Uint64("height", height). + Hex("block_id", header.ID[:]). + Int("chunks", len(blockED.ChunkExecutionDatas)). + Msg("successfully retrieved execution data") + return err }), "could not get execution data for block %d", height) diff --git a/network/p2p/blob/blob_service.go b/network/p2p/blob/blob_service.go index 74107bde431..bdb04bafb9f 100644 --- a/network/p2p/blob/blob_service.go +++ b/network/p2p/blob/blob_service.go @@ -119,12 +119,30 @@ func NewBlobService( opts ...network.BlobServiceOption, ) (*blobService, error) { bsNetwork := bsnet.NewFromIpfsHost(host, r, bsnet.Prefix(protocol.ID(prefix))) + + // Create blockstore based on config + // var blockStore blockstore.Blockstore + // if bs.config.SkipBloomCache { + // // Use plain blockstore - Pebble's built-in bloom filters are sufficient + // blockStore = blockstore.NewBlockstore(ds) + // } else { + // Use cached blockstore with bloom filter (default behavior) + cachedBlockStore, err := blockstore.CachedBlockstore( + context.Background(), + blockstore.NewBlockstore(ds), + blockstore.DefaultCacheOpts(), + ) + if err != nil { + return nil, fmt.Errorf("failed to create cached blockstore: %w", err) + } + bs := &blobService{ prefix: prefix, config: &BlobServiceConfig{ ReprovideInterval: DefaultReprovideInterval, SkipBloomCache: false, // default: use cached blockstore }, + blockStore: cachedBlockStore, } // Apply options before creating blockstore, as SkipBloomCache affects blockstore creation @@ -132,24 +150,9 @@ func NewBlobService( opt(bs) } - // Create blockstore based on config - var blockStore blockstore.Blockstore - if bs.config.SkipBloomCache { - // Use plain blockstore - Pebble's built-in bloom filters are sufficient - blockStore = blockstore.NewBlockstore(ds) - } else { - // Use cached blockstore with bloom filter (default behavior) - cachedBlockStore, err := blockstore.CachedBlockstore( - context.Background(), - blockstore.NewBlockstore(ds), - blockstore.DefaultCacheOpts(), - ) - if err != nil { - return nil, fmt.Errorf("failed to create cached blockstore: %w", err) - } - blockStore = cachedBlockStore - } - bs.blockStore = blockStore + // blockStore = cachedBlockStore + // } + // bs.blockStore = blockStore cm := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { From 01324d6d5259f23ad28ae411899d1f31b4097c50 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 8 Dec 2025 14:41:00 -0800 Subject: [PATCH 125/126] adjustable bloom cache --- network/p2p/blob/blob_service.go | 32 +++++++++++++------------------- 1 file changed, 13 insertions(+), 19 deletions(-) diff --git a/network/p2p/blob/blob_service.go b/network/p2p/blob/blob_service.go index bdb04bafb9f..01de8460532 100644 --- a/network/p2p/blob/blob_service.go +++ b/network/p2p/blob/blob_service.go @@ -120,21 +120,7 @@ func NewBlobService( ) (*blobService, error) { bsNetwork := bsnet.NewFromIpfsHost(host, r, bsnet.Prefix(protocol.ID(prefix))) - // Create blockstore based on config - // var blockStore blockstore.Blockstore - // if bs.config.SkipBloomCache { - // // Use plain blockstore - Pebble's built-in bloom filters are sufficient - // blockStore = blockstore.NewBlockstore(ds) - // } else { - // Use cached blockstore with bloom filter (default behavior) - cachedBlockStore, err := blockstore.CachedBlockstore( - context.Background(), - blockstore.NewBlockstore(ds), - blockstore.DefaultCacheOpts(), - ) - if err != nil { - return nil, fmt.Errorf("failed to create cached blockstore: %w", err) - } + blockStore := blockstore.NewBlockstore(ds) bs := &blobService{ prefix: prefix, @@ -142,7 +128,7 @@ func NewBlobService( ReprovideInterval: DefaultReprovideInterval, SkipBloomCache: false, // default: use cached blockstore }, - blockStore: cachedBlockStore, + blockStore: blockStore, } // Apply options before creating blockstore, as SkipBloomCache affects blockstore creation @@ -150,9 +136,17 @@ func NewBlobService( opt(bs) } - // blockStore = cachedBlockStore - // } - // bs.blockStore = blockStore + if bs.config.SkipBloomCache { + cachedBlockStore, err := blockstore.CachedBlockstore( + context.Background(), + blockStore, + blockstore.DefaultCacheOpts(), + ) + if err != nil { + return nil, fmt.Errorf("failed to create cached blockstore: %w", err) + } + bs.blockStore = cachedBlockStore + } cm := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { From 27f6a599186450e2d1a2402ce0095b483fc2d511 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 8 Dec 2025 14:46:33 -0800 Subject: [PATCH 126/126] disable access node builder bloom cache --- cmd/node_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 7d9e67e7be2..7c310fc6c3b 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -310,7 +310,7 @@ func DefaultBaseConfig() *BaseConfig { ComplianceConfig: compliance.DefaultConfig(), DhtSystemEnabled: true, BitswapReprovideEnabled: true, - BitswapBloomCacheEnabled: true, // default: use cached blockstore + BitswapBloomCacheEnabled: true, // default: use cached blockstore TODO leo: change default to false } }