From 0de80c3cfab722e71bf010ad2cdc0538a422d9a7 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Sun, 2 Nov 2025 21:29:15 -0800 Subject: [PATCH 01/16] [Access] Separate collection indexing and add support for indexing from execution data --- .mockery.yaml | 1 + .../node_builder/access_node_builder.go | 100 ++- cmd/observer/node_builder/observer_builder.go | 28 + engine/access/access_test.go | 232 ++++--- engine/access/ingestion/collection_syncer.go | 411 ----------- .../access/ingestion/collections/indexer.go | 286 ++++++++ .../ingestion/collections/indexer_test.go | 432 ++++++++++++ .../collections/mock/collection_indexer.go | 90 +++ engine/access/ingestion/collections/syncer.go | 363 ++++++++++ .../collections/syncer_execution_data.go | 72 ++ .../ingestion/collections/syncer_test.go | 638 ++++++++++++++++++ engine/access/ingestion/engine.go | 40 +- engine/access/ingestion/engine_test.go | 550 +++++---------- engine/access/ingestion2/collection_syncer.go | 475 ------------- engine/access/ingestion2/engine.go | 7 +- engine/access/ingestion2/engine_test.go | 374 +--------- .../ingestion2/finalized_block_processor.go | 5 +- .../rpc/backend/backend_stream_blocks_test.go | 6 +- .../rpc/backend/script_executor_test.go | 3 +- module/counters/interfaces.go | 7 + module/counters/monotonous_counter.go | 2 + .../persistent_strict_monotonic_counter.go | 2 + module/execution/scripts_test.go | 1 + .../executiondatasync/execution_data/cache.go | 43 ++ .../execution_data/cache/cache.go | 32 +- .../mock/execution_data_cache.go | 151 +++++ .../indexer/indexer_core.go | 51 +- .../indexer/indexer_core_test.go | 61 +- 28 files changed, 2555 insertions(+), 1908 deletions(-) delete mode 100644 engine/access/ingestion/collection_syncer.go create mode 100644 engine/access/ingestion/collections/indexer.go create mode 100644 engine/access/ingestion/collections/indexer_test.go create mode 100644 engine/access/ingestion/collections/mock/collection_indexer.go create mode 100644 engine/access/ingestion/collections/syncer.go create mode 100644 engine/access/ingestion/collections/syncer_execution_data.go create mode 100644 engine/access/ingestion/collections/syncer_test.go delete mode 100644 engine/access/ingestion2/collection_syncer.go create mode 100644 module/counters/interfaces.go create mode 100644 module/executiondatasync/execution_data/cache.go create mode 100644 module/executiondatasync/execution_data/mock/execution_data_cache.go diff --git a/.mockery.yaml b/.mockery.yaml index a99b7c9e56d..dd492663dfd 100644 --- a/.mockery.yaml +++ b/.mockery.yaml @@ -22,6 +22,7 @@ packages: config: dir: "consensus/hotstuff/mocks" outpkg: "mocks" + 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/rest/common/models: github.com/onflow/flow-go/engine/access/rest/websockets: diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5c2dce02ebb..4d5ab84916e 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -41,6 +41,7 @@ import ( "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" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" @@ -332,6 +333,8 @@ type FlowAccessNodeBuilder struct { ExecutionDataCache *execdatacache.ExecutionDataCache ExecutionIndexer *indexer.Indexer ExecutionIndexerCore *indexer.IndexerCore + CollectionIndexer *collections.Indexer + CollectionSyncer *collections.Syncer ScriptExecutor *backend.ScriptExecutor RegistersAsyncStore *execution.RegistersAsyncStore Reporter *index.Reporter @@ -567,7 +570,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess var bsDependable *module.ProxiedReadyDoneAware var execDataDistributor *edrequester.ExecutionDataDistributor var execDataCacheBackend *herocache.BlockExecutionData - var executionDataStoreCache *execdatacache.ExecutionDataCache // setup dependency chain to ensure indexer starts after the requester requesterDependable := module.NewProxiedReadyDoneAware() @@ -635,13 +637,14 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess // Execution Data cache that uses a blobstore as the backend (instead of a downloader) // This ensures that it simply returns a not found error if the blob doesn't exist // instead of attempting to download it from the network. - executionDataStoreCache = execdatacache.NewExecutionDataCache( + executionDataStoreCache := execdatacache.NewExecutionDataCache( builder.ExecutionDataStore, builder.Storage.Headers, builder.Storage.Seals, builder.Storage.Results, execDataCacheBackend, ) + builder.ExecutionDataCache = executionDataStoreCache return nil }). @@ -955,6 +958,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess notNil(builder.scheduledTransactions), builder.RootChainID, indexerDerivedChainData, + notNil(builder.CollectionIndexer), notNil(builder.collectionExecutedMetric), node.StorageLockMgr, ) @@ -965,7 +969,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess registers.FirstHeight(), registers, builder.ExecutionIndexerCore, - executionDataStoreCache, + notNil(builder.ExecutionDataCache), builder.ExecutionDataRequester.HighestConsecutiveHeight, indexedBlockHeight, ) @@ -1067,7 +1071,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess node.Storage.Seals, node.Storage.Results, builder.ExecutionDataStore, - executionDataStoreCache, + notNil(builder.ExecutionDataCache), builder.RegistersAsyncStore, builder.EventsIndex, useIndex, @@ -1088,7 +1092,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess stateStreamEng, err := statestreambackend.NewEng( node.Logger, builder.stateStreamConf, - executionDataStoreCache, + notNil(builder.ExecutionDataCache), node.Storage.Headers, node.RootChainID, builder.stateStreamGrpcServer, @@ -2194,10 +2198,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return builder.RpcEng, nil }). - Component("ingestion engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - var err error - - builder.RequestEng, err = requester.New( + 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, @@ -2210,31 +2212,62 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { if err != nil { return nil, fmt.Errorf("could not create requester engine: %w", err) } + builder.RequestEng = requestEng - if builder.storeTxResultErrorMessages { - builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( + collectionIndexer, err := collections.NewIndexer( + node.Logger, + notNil(builder.collectionExecutedMetric), + node.State, + node.Storage.Blocks, + notNil(builder.collections), + notNil(builder.transactions), + 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.txResultErrorMessageProvider), - builder.transactionResultErrorMessages, - notNil(builder.ExecNodeIdentitiesProvider), - node.StorageLockMgr, + builder.ExecutionDataCache, + collectionIndexer, ) } - collectionSyncer := ingestion.NewCollectionSyncer( + collectionSyncer := collections.NewSyncer( node.Logger, - notNil(builder.collectionExecutedMetric), builder.RequestEng, node.State, - node.Storage.Blocks, notNil(builder.collections), - notNil(builder.transactions), lastFullBlockHeight, - node.StorageLockMgr, + collectionIndexer, + executionDataSyncer, ) + builder.CollectionSyncer = collectionSyncer + builder.RequestEng.WithHandle(collectionSyncer.OnCollectionDownloaded) - builder.IngestEng, err = ingestion.New( + 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, + ) + } + + ingestEng, err := ingestion.New( node.Logger, node.EngineRegistry, node.State, @@ -2243,34 +2276,31 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.Storage.Results, node.Storage.Receipts, processedFinalizedBlockHeight, - notNil(collectionSyncer), + notNil(builder.CollectionSyncer), + notNil(builder.CollectionIndexer), notNil(builder.collectionExecutedMetric), notNil(builder.TxResultErrorMessagesCore), ) if err != nil { return nil, err } + builder.IngestEng = ingestEng + ingestionDependable.Init(builder.IngestEng) builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.IngestEng.OnFinalizedBlock) return builder.IngestEng, nil - }). - Component("requester engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // We initialize the requester engine inside the ingestion engine due to the mutual dependency. However, in - // order for it to properly start and shut down, we should still return it as its own engine here, so it can - // be handled by the scaffold. - return builder.RequestEng, nil - }). - AdminCommand("backfill-tx-error-messages", func(config *cmd.NodeConfig) commands.AdminCommand { - return storageCommands.NewBackfillTxErrorMessagesCommand( - builder.Logger, - builder.State, - builder.TxResultErrorMessagesCore, - ) }) if builder.storeTxResultErrorMessages { builder. + AdminCommand("backfill-tx-error-messages", func(config *cmd.NodeConfig) commands.AdminCommand { + return storageCommands.NewBackfillTxErrorMessagesCommand( + builder.Logger, + builder.State, + builder.TxResultErrorMessagesCore, + ) + }). Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { builder.transactionResultErrorMessages = store.NewTransactionResultErrorMessages( node.Metrics.Cache, diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index e2958d9fefa..cce669f57ee 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -38,6 +38,7 @@ 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" @@ -69,6 +70,7 @@ 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" @@ -1438,7 +1440,32 @@ 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, + collectionExecutedMetric, + builder.State, + builder.Storage.Blocks, + builder.Storage.Collections, + builder.Storage.Transactions, + lastFullBlockHeight, + builder.StorageLockMgr, + ) + if err != nil { + return nil, fmt.Errorf("could not create collection indexer: %w", err) + } + builder.ExecutionIndexerCore = indexer.New( builder.Logger, metrics.NewExecutionStateIndexerCollector(), @@ -1452,6 +1479,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.scheduledTransactions, builder.RootChainID, indexerDerivedChainData, + collectionIndexer, collectionExecutedMetric, node.StorageLockMgr, ) diff --git a/engine/access/access_test.go b/engine/access/access_test.go index f5620fafe66..f32e0e5de53 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -3,7 +3,9 @@ package access_test import ( "context" "encoding/json" + "fmt" "testing" + "time" "github.com/cockroachdb/pebble/v2" "github.com/google/go-cmp/cmp" @@ -16,11 +18,15 @@ import ( "google.golang.org/protobuf/testing/protocmp" "github.com/onflow/crypto" + accessproto "github.com/onflow/flow/protobuf/go/flow/access" + entitiesproto "github.com/onflow/flow/protobuf/go/flow/entities" + execproto "github.com/onflow/flow/protobuf/go/flow/execution" "github.com/onflow/flow-go/cmd/build" hsmock "github.com/onflow/flow-go/consensus/hotstuff/mocks" "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine/access/ingestion" + ingestioncollections "github.com/onflow/flow-go/engine/access/ingestion/collections" accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc" "github.com/onflow/flow-go/engine/access/rpc/backend" @@ -51,10 +57,6 @@ import ( "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" "github.com/onflow/flow-go/utils/unittest/mocks" - - accessproto "github.com/onflow/flow/protobuf/go/flow/access" - entitiesproto "github.com/onflow/flow/protobuf/go/flow/entities" - execproto "github.com/onflow/flow/protobuf/go/flow/execution" ) type Suite struct { @@ -74,7 +76,7 @@ type Suite struct { me *mockmodule.Local rootBlock *flow.Header sealedBlock *flow.Header - finalizedBlock *flow.Header + finalizedBlock *flow.Block chainID flow.ChainID metrics *metrics.NoopCollector finalizedHeaderCache module.FinalizedHeaderCache @@ -102,7 +104,7 @@ func (suite *Suite) SetupTest() { suite.rootBlock = unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) suite.sealedBlock = suite.rootBlock - suite.finalizedBlock = unittest.BlockHeaderWithParentFixture(suite.sealedBlock) + suite.finalizedBlock = unittest.BlockWithParentFixture(suite.sealedBlock) suite.epochQuery = new(protocol.EpochQuery) suite.state.On("Sealed").Return(suite.sealedSnapshot, nil).Maybe() @@ -115,10 +117,9 @@ func (suite *Suite) SetupTest() { nil, ).Maybe() suite.finalSnapshot.On("Head").Return( - func() *flow.Header { - return suite.finalizedBlock + func() (*flow.Header, error) { + return suite.finalizedBlock.ToHeader(), nil }, - nil, ).Maybe() pstate := protocol.NewKVStoreReader(suite.T()) @@ -253,7 +254,7 @@ func (suite *Suite) TestSendExpiredTransaction() { transaction := unittest.TransactionBodyFixture(unittest.WithReferenceBlock(referenceBlock.ID())) // create latest block that is past the expiry window - latestBlock := unittest.BlockHeaderFixture() + latestBlock := unittest.BlockFixture() latestBlock.Height = referenceBlock.Height + flow.DefaultTransactionExpiry*2 refSnapshot := new(protocol.Snapshot) @@ -264,7 +265,7 @@ func (suite *Suite) TestSendExpiredTransaction() { refSnapshot. On("Head"). - Return(referenceBlock, nil). + Return(referenceBlock.ToHeader(), nil). Twice() // Advancing final state to expire ref block @@ -723,10 +724,9 @@ func (suite *Suite) TestGetSealedTransaction() { // create the ingest engine processedHeight := store.NewConsumerProgress(db, module.ConsumeProgressIngestionEngineBlockHeight) - collectionSyncer := ingestion.NewCollectionSyncer( + collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, - module.CollectionExecutedMetric(collectionExecutedMetric), - suite.request, + collectionExecutedMetric, suite.state, all.Blocks, collections, @@ -734,6 +734,17 @@ func (suite *Suite) TestGetSealedTransaction() { lastFullBlockHeight, suite.lockManager, ) + require.NoError(suite.T(), err) + + collectionSyncer := ingestioncollections.NewSyncer( + suite.log, + suite.request, + suite.state, + collections, + lastFullBlockHeight, + collectionIndexer, + nil, + ) ingestEng, err := ingestion.New( suite.log, @@ -745,67 +756,81 @@ func (suite *Suite) TestGetSealedTransaction() { all.Receipts, processedHeight, collectionSyncer, + collectionIndexer, collectionExecutedMetric, nil, ) require.NoError(suite.T(), err) // 1. Assume that follower engine updated the block storage and the protocol state. The block is reported as sealed - err = unittest.WithLock(suite.T(), suite.lockManager, storage.LockInsertBlock, func(lctx lockctx.Context) error { + err = unittest.WithLocks(suite.T(), suite.lockManager, []string{ + storage.LockInsertBlock, + storage.LockFinalizeBlock, + }, func(lctx lockctx.Context) error { return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { - return all.Blocks.BatchStore(lctx, rw, proposal) - }) - }) - require.NoError(suite.T(), err) + // store finalized block + finalized := suite.finalizedBlock + if err := all.Blocks.BatchStore(lctx, rw, unittest.ProposalFromBlock(finalized)); err != nil { + return fmt.Errorf("could not store block: %w", err) + } + if err := operation.IndexFinalizedBlockByHeight(lctx, rw, finalized.Height, finalized.ID()); err != nil { + return fmt.Errorf("could not index finalized block: %w", err) + } - err = unittest.WithLock(suite.T(), suite.lockManager, storage.LockFinalizeBlock, func(fctx lockctx.Context) error { - return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { - return operation.IndexFinalizedBlockByHeight(fctx, rw, block.Height, block.ID()) + // store new block + if err := all.Blocks.BatchStore(lctx, rw, proposal); err != nil { + return fmt.Errorf("could not store block: %w", err) + } + if err := operation.IndexFinalizedBlockByHeight(lctx, rw, block.Height, block.ID()); err != nil { + return fmt.Errorf("could not index finalized block: %w", err) + } + return nil }) }) require.NoError(suite.T(), err) suite.sealedBlock = block.ToHeader() - background, cancel := context.WithCancel(context.Background()) + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(suite.T(), context.Background()) defer cancel() - ctx := irrecoverable.NewMockSignalerContext(suite.T(), background) ingestEng.Start(ctx) - <-ingestEng.Ready() + unittest.RequireCloseBefore(suite.T(), ingestEng.Ready(), 1*time.Second, "could not start ingest engine") defer func() { cancel() - <-ingestEng.Done() + unittest.RequireCloseBefore(suite.T(), ingestEng.Done(), 1*time.Second, "could not stop ingest engine") }() // 2. Ingest engine was notified by the follower engine about a new block. // Follower engine --> Ingest engine - mb := &model.Block{ - BlockID: block.ID(), - } - ingestEng.OnFinalizedBlock(mb) + ingestEng.OnFinalizedBlock(&model.Block{BlockID: block.ID()}) // 3. Request engine is used to request missing collection suite.request.On("EntityByID", collection.ID(), mock.Anything).Return() - // 4. Indexer IndexCollection receives the requested collection and all the execution receipts - // Create a lock context for indexing - err = unittest.WithLock(suite.T(), suite.lockManager, storage.LockInsertCollection, func(indexLctx lockctx.Context) error { - return indexer.IndexCollection(indexLctx, collection, collections, suite.log, module.CollectionExecutedMetric(collectionExecutedMetric)) - }) - require.NoError(suite.T(), err) + // 4. Syncer receives the requested collection and all the ingestion engine receives the receipts + collectionSyncer.OnCollectionDownloaded(unittest.IdentifierFixture(), collection) for _, r := range executionReceipts { err = ingestEng.Process(channels.ReceiveReceipts, enNodeIDs[0], r) require.NoError(suite.T(), err) } + // block until the collection is processed by the indexer + require.Eventually(suite.T(), func() bool { + isStored, err := collectionIndexer.IsCollectionInStorage(collection.ID()) + return isStored && err == nil + }, 1*time.Second, 100*time.Millisecond, "collection not indexed") + // 5. Client requests a transaction tx := collection.Transactions[0] txID := tx.ID() getReq := &accessproto.GetTransactionRequest{ Id: txID[:], } - gResp, err := handler.GetTransactionResult(context.Background(), getReq) + + apiCtx := irrecoverable.WithSignalerContext(context.Background(), ctx) + + gResp, err := handler.GetTransactionResult(apiCtx, getReq) require.NoError(suite.T(), err) // assert that the transaction is reported as Sealed require.Equal(suite.T(), entitiesproto.TransactionStatus_SEALED, gResp.GetStatus()) @@ -831,7 +856,7 @@ func (suite *Suite) TestGetTransactionResult() { blockNegativeId := blockNegative.ID() finalSnapshot := new(protocol.Snapshot) - finalSnapshot.On("Head").Return(suite.finalizedBlock, nil) + finalSnapshot.On("Head").Return(suite.finalizedBlock.ToHeader(), nil) suite.state.On("Params").Return(suite.params) suite.state.On("Final").Return(finalSnapshot) @@ -840,16 +865,28 @@ func (suite *Suite) TestGetTransactionResult() { // specifically for this test we will consider that sealed block is far behind finalized, so we get EXECUTED status suite.sealedSnapshot.On("Head").Return(sealedBlock, nil) - err := unittest.WithLock(suite.T(), suite.lockManager, storage.LockInsertBlock, func(lctx lockctx.Context) error { + err := unittest.WithLocks(suite.T(), suite.lockManager, []string{ + storage.LockInsertBlock, + storage.LockFinalizeBlock, + }, func(lctx lockctx.Context) error { return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { - return all.Blocks.BatchStore(lctx, rw, proposal) - }) - }) - require.NoError(suite.T(), err) + // store finalized block + finalized := suite.finalizedBlock + if err := all.Blocks.BatchStore(lctx, rw, unittest.ProposalFromBlock(finalized)); err != nil { + return fmt.Errorf("could not store block: %w", err) + } + if err := operation.IndexFinalizedBlockByHeight(lctx, rw, finalized.Height, finalized.ID()); err != nil { + return fmt.Errorf("could not index finalized block: %w", err) + } - err = unittest.WithLock(suite.T(), suite.lockManager, storage.LockInsertBlock, func(lctx2 lockctx.Context) error { - return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { - return all.Blocks.BatchStore(lctx2, rw, proposalNegative) + // store new blocks + if err := all.Blocks.BatchStore(lctx, rw, proposal); err != nil { + return fmt.Errorf("could not store proposal: %w", err) + } + if err := all.Blocks.BatchStore(lctx, rw, proposalNegative); err != nil { + return fmt.Errorf("could not store negative propoal: %w", err) + } + return nil }) }) require.NoError(suite.T(), err) @@ -946,10 +983,9 @@ func (suite *Suite) TestGetTransactionResult() { lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeightProgress) require.NoError(suite.T(), err) - collectionSyncer := ingestion.NewCollectionSyncer( + collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, - module.CollectionExecutedMetric(collectionExecutedMetric), - suite.request, + collectionExecutedMetric, suite.state, all.Blocks, collections, @@ -957,6 +993,17 @@ func (suite *Suite) TestGetTransactionResult() { lastFullBlockHeight, suite.lockManager, ) + require.NoError(suite.T(), err) + + collectionSyncer := ingestioncollections.NewSyncer( + suite.log, + suite.request, + suite.state, + collections, + lastFullBlockHeight, + collectionIndexer, + nil, + ) ingestEng, err := ingestion.New( suite.log, @@ -968,17 +1015,21 @@ func (suite *Suite) TestGetTransactionResult() { all.Receipts, processedHeightInitializer, collectionSyncer, + collectionIndexer, collectionExecutedMetric, nil, ) require.NoError(suite.T(), err) - background, cancel := context.WithCancel(context.Background()) + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(suite.T(), context.Background()) defer cancel() - ctx := irrecoverable.NewMockSignalerContext(suite.T(), background) ingestEng.Start(ctx) - <-ingestEng.Ready() + unittest.RequireCloseBefore(suite.T(), ingestEng.Ready(), 1*time.Second, "could not start ingest engine") + defer func() { + cancel() + unittest.RequireCloseBefore(suite.T(), ingestEng.Done(), 1*time.Second, "could not stop ingest engine") + }() processExecutionReceipts := func( block *flow.Block, @@ -990,17 +1041,10 @@ func (suite *Suite) TestGetTransactionResult() { executionReceipts := unittest.ReceiptsForBlockFixture(block, enNodeIDs) // Ingest engine was notified by the follower engine about a new block. // Follower engine --> Ingest engine - mb := &model.Block{ - BlockID: block.ID(), - } - ingestEng.OnFinalizedBlock(mb) + ingestEng.OnFinalizedBlock(&model.Block{BlockID: block.ID()}) - // Indexer IndexCollection receives the requested collection and all the execution receipts - // Create a lock context for indexing - err = unittest.WithLock(suite.T(), suite.lockManager, storage.LockInsertCollection, func(indexLctx lockctx.Context) error { - return indexer.IndexCollection(indexLctx, collection, collections, suite.log, module.CollectionExecutedMetric(collectionExecutedMetric)) - }) - require.NoError(suite.T(), err) + // Syncer receives the requested collection and the ingestion engine processes the receipts + collectionSyncer.OnCollectionDownloaded(originID, collection) for _, r := range executionReceipts { err = ingestEng.Process(channels.ReceiveReceipts, enNodeIDs[0], r) @@ -1023,6 +1067,17 @@ func (suite *Suite) TestGetTransactionResult() { txIdNegative := collectionNegative.Transactions[0].ID() collectionIdNegative := collectionNegative.ID() + // the transactions should eventually be indexed by the collection indexer + require.Eventually(suite.T(), func() bool { + if _, err := transactions.ByID(txId); err != nil { + return false + } + if _, err := transactions.ByID(txIdNegative); err != nil { + return false + } + return true + }, 1*time.Second, 100*time.Millisecond, "transactions never indexed") + assertTransactionResult := func( resp *accessproto.TransactionResultResponse, err error, @@ -1036,13 +1091,15 @@ func (suite *Suite) TestGetTransactionResult() { require.Equal(suite.T(), collectionId, actualCollectionId) } + apiCtx := irrecoverable.WithSignalerContext(context.Background(), ctx) + // Test behaviour with transactionId provided // POSITIVE suite.Run("Get transaction result by transaction ID", func() { getReq := &accessproto.GetTransactionRequest{ Id: txId[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) assertTransactionResult(resp, err) }) @@ -1052,7 +1109,7 @@ func (suite *Suite) TestGetTransactionResult() { Id: txId[:], BlockId: blockId[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) assertTransactionResult(resp, err) }) @@ -1071,7 +1128,7 @@ func (suite *Suite) TestGetTransactionResult() { Id: txId[:], BlockId: blockNegativeId[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) require.Error(suite.T(), err) require.Contains(suite.T(), err.Error(), "failed to find: transaction not in block") require.Nil(suite.T(), resp) @@ -1083,7 +1140,7 @@ func (suite *Suite) TestGetTransactionResult() { Id: txId[:], CollectionId: collectionId[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) assertTransactionResult(resp, err) }) @@ -1092,17 +1149,7 @@ func (suite *Suite) TestGetTransactionResult() { Id: txId[:], CollectionId: collectionIdNegative[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) - require.Error(suite.T(), err) - require.Nil(suite.T(), resp) - }) - - suite.Run("Get transaction result with wrong transaction ID and correct collection ID", func() { - getReq := &accessproto.GetTransactionRequest{ - Id: txIdNegative[:], - CollectionId: collectionId[:], - } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) require.Error(suite.T(), err) require.Nil(suite.T(), resp) }) @@ -1114,7 +1161,7 @@ func (suite *Suite) TestGetTransactionResult() { BlockId: blockId[:], CollectionId: collectionId[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) assertTransactionResult(resp, err) }) @@ -1124,7 +1171,7 @@ func (suite *Suite) TestGetTransactionResult() { BlockId: blockId[:], CollectionId: collectionIdNegative[:], } - resp, err := handler.GetTransactionResult(context.Background(), getReq) + resp, err := handler.GetTransactionResult(apiCtx, getReq) require.Error(suite.T(), err) require.Nil(suite.T(), resp) }) @@ -1212,10 +1259,9 @@ func (suite *Suite) TestExecuteScript() { lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeightProgress) require.NoError(suite.T(), err) - collectionSyncer := ingestion.NewCollectionSyncer( + collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, - module.CollectionExecutedMetric(collectionExecutedMetric), - suite.request, + collectionExecutedMetric, suite.state, all.Blocks, all.Collections, @@ -1223,6 +1269,17 @@ func (suite *Suite) TestExecuteScript() { lastFullBlockHeight, suite.lockManager, ) + require.NoError(suite.T(), err) + + collectionSyncer := ingestioncollections.NewSyncer( + suite.log, + suite.request, + suite.state, + all.Collections, + lastFullBlockHeight, + collectionIndexer, + nil, + ) ingestEng, err := ingestion.New( suite.log, @@ -1234,13 +1291,14 @@ func (suite *Suite) TestExecuteScript() { all.Receipts, processedHeightInitializer, collectionSyncer, + collectionIndexer, collectionExecutedMetric, nil, ) require.NoError(suite.T(), err) // create another block as a predecessor of the block created earlier - prevBlock := unittest.BlockWithParentFixture(suite.finalizedBlock) + prevBlock := unittest.BlockWithParentFixture(suite.finalizedBlock.ToHeader()) // create a block and a seal pointing to that block lastBlock := unittest.BlockWithParentFixture(prevBlock.ToHeader()) @@ -1394,7 +1452,7 @@ func (suite *Suite) TestAPICallNodeVersionInfo() { // updated correctly when a block with a greater height is finalized. func (suite *Suite) TestLastFinalizedBlockHeightResult() { suite.RunTest(func(handler *rpc.Handler, db storage.DB, all *store.All) { - block := unittest.BlockWithParentFixture(suite.finalizedBlock) + block := unittest.BlockWithParentFixture(suite.finalizedBlock.ToHeader()) proposal := unittest.ProposalFromBlock(block) newFinalizedBlock := unittest.BlockWithParentFixture(block.ToHeader()) @@ -1428,7 +1486,7 @@ func (suite *Suite) TestLastFinalizedBlockHeightResult() { resp, err := handler.GetBlockHeaderByID(context.Background(), req) assertFinalizedBlockHeader(resp, err) - suite.finalizedBlock = newFinalizedBlock.ToHeader() + suite.finalizedBlock = newFinalizedBlock resp, err = handler.GetBlockHeaderByID(context.Background(), req) assertFinalizedBlockHeader(resp, err) @@ -1437,7 +1495,7 @@ func (suite *Suite) TestLastFinalizedBlockHeightResult() { func (suite *Suite) createChain() (*flow.Proposal, *flow.Collection) { collection := unittest.CollectionFixture(10) - refBlockID := unittest.IdentifierFixture() + refBlockID := suite.finalizedBlock.ID() // prepare cluster committee members clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)) // guarantee signers must be cluster committee members, so that access will fetch collection from @@ -1451,7 +1509,7 @@ func (suite *Suite) createChain() (*flow.Proposal, *flow.Collection) { SignerIndices: indices, } block := unittest.BlockWithParentAndPayload( - suite.finalizedBlock, + suite.finalizedBlock.ToHeader(), unittest.PayloadFixture(unittest.WithGuarantees(guarantee)), ) proposal := unittest.ProposalFromBlock(block) diff --git a/engine/access/ingestion/collection_syncer.go b/engine/access/ingestion/collection_syncer.go deleted file mode 100644 index 7e0375abcff..00000000000 --- a/engine/access/ingestion/collection_syncer.go +++ /dev/null @@ -1,411 +0,0 @@ -package ingestion - -import ( - "context" - "errors" - "fmt" - "time" - - "github.com/rs/zerolog" - - "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" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/state_synchronization/indexer" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -var ( - defaultMissingCollsForBlockThreshold = missingCollsForBlockThreshold - defaultMissingCollsForAgeThreshold uint64 = missingCollsForAgeThreshold -) - -// The CollectionSyncer type provides mechanisms for syncing and indexing data -// from the Flow blockchain into local storage. Specifically, it handles -// the retrieval and processing of collections and transactions that may -// have been missed due to network delays, restarts, or gaps in finalization. -// -// It is responsible for ensuring the local node has -// all collections associated with finalized blocks starting from the -// last fully synced height. It works by periodically scanning the finalized -// block range, identifying missing collections, and triggering requests -// to fetch them from the network. Once collections are retrieved, it -// ensures they are persisted in the local collection and transaction stores. -// -// The syncer maintains a persistent, strictly monotonic counter -// (`lastFullBlockHeight`) to track the highest finalized block for which -// all collections have been fully indexed. It uses this information to -// avoid redundant processing and to measure catch-up progress. -// -// It is meant to operate in a background goroutine as part of the -// node's ingestion pipeline. -type CollectionSyncer struct { - logger zerolog.Logger - collectionExecutedMetric module.CollectionExecutedMetric - - state protocol.State - requester module.Requester - - blocks storage.Blocks - collections storage.Collections - transactions storage.Transactions - - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter - lockManager storage.LockManager -} - -// NewCollectionSyncer creates a new CollectionSyncer responsible for requesting, -// tracking, and indexing missing collections. -func NewCollectionSyncer( - logger zerolog.Logger, - collectionExecutedMetric module.CollectionExecutedMetric, - requester module.Requester, - state protocol.State, - blocks storage.Blocks, - collections storage.Collections, - transactions storage.Transactions, - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter, - lockManager storage.LockManager, -) *CollectionSyncer { - collectionExecutedMetric.UpdateLastFullBlockHeight(lastFullBlockHeight.Value()) - - return &CollectionSyncer{ - logger: logger, - state: state, - requester: requester, - blocks: blocks, - collections: collections, - transactions: transactions, - lastFullBlockHeight: lastFullBlockHeight, - collectionExecutedMetric: collectionExecutedMetric, - lockManager: lockManager, - } -} - -// RequestCollections continuously monitors and triggers collection sync operations. -// It handles on startup collection catchup, periodic missing collection requests, and full block height updates. -func (s *CollectionSyncer) RequestCollections(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - requestCtx, cancel := context.WithTimeout(ctx, collectionCatchupTimeout) - defer cancel() - - // on start-up, AN wants to download all missing collections to serve it to end users - err := s.requestMissingCollectionsBlocking(requestCtx) - if err != nil { - s.logger.Error().Err(err).Msg("error downloading missing collections") - } - ready() - - requestCollectionsTicker := time.NewTicker(missingCollsRequestInterval) - defer requestCollectionsTicker.Stop() - - // Collections are requested concurrently in this design. - // To maintain accurate progress tracking and avoid redundant requests, - // we periodically update the `lastFullBlockHeight` to reflect the latest - // finalized block with all collections successfully indexed. - updateLastFullBlockHeightTicker := time.NewTicker(fullBlockRefreshInterval) - defer updateLastFullBlockHeightTicker.Stop() - - for { - select { - case <-ctx.Done(): - return - - case <-requestCollectionsTicker.C: - err := s.requestMissingCollections() - if err != nil { - ctx.Throw(err) - } - - case <-updateLastFullBlockHeightTicker.C: - err := s.updateLastFullBlockHeight() - if err != nil { - ctx.Throw(err) - } - } - } -} - -// requestMissingCollections checks if missing collections should be requested based on configured -// block or age thresholds and triggers requests if needed. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) requestMissingCollections() error { - lastFullBlockHeight := s.lastFullBlockHeight.Value() - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return fmt.Errorf("failed to get finalized block: %w", err) - } - - collections, incompleteBlocksCount, err := s.findMissingCollections(lastFullBlockHeight) - if err != nil { - return err - } - - blocksThresholdReached := incompleteBlocksCount >= defaultMissingCollsForBlockThreshold - ageThresholdReached := lastFinalizedBlock.Height-lastFullBlockHeight > defaultMissingCollsForAgeThreshold - shouldRequest := blocksThresholdReached || ageThresholdReached - - if shouldRequest { - // warn log since generally this should not happen - s.logger.Warn(). - Uint64("finalized_height", lastFinalizedBlock.Height). - Uint64("last_full_blk_height", lastFullBlockHeight). - Int("missing_collection_blk_count", incompleteBlocksCount). - Int("missing_collection_count", len(collections)). - Msg("re-requesting missing collections") - - s.requestCollections(collections) - } - - return nil -} - -// requestMissingCollectionsBlocking requests and waits for all missing collections to be downloaded, -// blocking until either completion or context timeout. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) requestMissingCollectionsBlocking(ctx context.Context) error { - missingCollections, _, err := s.findMissingCollections(s.lastFullBlockHeight.Value()) - if err != nil { - return err - } - if len(missingCollections) == 0 { - s.logger.Info().Msg("skipping requesting missing collections. no missing collections found") - return nil - } - - s.requestCollections(missingCollections) - - collectionsToBeDownloaded := make(map[flow.Identifier]struct{}) - for _, collection := range missingCollections { - collectionsToBeDownloaded[collection.CollectionID] = struct{}{} - } - - collectionStoragePollTicker := time.NewTicker(collectionCatchupDBPollInterval) - defer collectionStoragePollTicker.Stop() - - // we want to wait for all collections to be downloaded so we poll local storage periodically to make sure each - // collection was successfully saved in the storage. - for len(collectionsToBeDownloaded) > 0 { - select { - case <-ctx.Done(): - return fmt.Errorf("failed to complete collection retrieval: %w", ctx.Err()) - - case <-collectionStoragePollTicker.C: - s.logger.Info(). - Int("total_missing_collections", len(collectionsToBeDownloaded)). - Msg("retrieving missing collections...") - - for collectionID := range collectionsToBeDownloaded { - downloaded, err := s.isCollectionInStorage(collectionID) - if err != nil { - return err - } - - if downloaded { - delete(collectionsToBeDownloaded, collectionID) - } - } - } - } - - s.logger.Info().Msg("collection catchup done") - return nil -} - -// findMissingCollections scans block heights from last known full block up to the latest finalized -// block and returns all missing collection along with the count of incomplete blocks. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findMissingCollections(lastFullBlockHeight uint64) ([]*flow.CollectionGuarantee, int, error) { - // first block to look up collections at - firstBlockHeight := lastFullBlockHeight + 1 - - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return nil, 0, fmt.Errorf("failed to get finalized block: %w", err) - } - // last block to look up collections at - lastBlockHeight := lastFinalizedBlock.Height - - var missingCollections []*flow.CollectionGuarantee - var incompleteBlocksCount int - - for currBlockHeight := firstBlockHeight; currBlockHeight <= lastBlockHeight; currBlockHeight++ { - collections, err := s.findMissingCollectionsAtHeight(currBlockHeight) - if err != nil { - return nil, 0, err - } - - if len(collections) == 0 { - continue - } - - missingCollections = append(missingCollections, collections...) - incompleteBlocksCount += 1 - } - - return missingCollections, incompleteBlocksCount, nil -} - -// findMissingCollectionsAtHeight returns all missing collections for a specific block height. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findMissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) { - block, err := s.blocks.ByHeight(height) - if err != nil { - return nil, fmt.Errorf("failed to retrieve block by height %d: %w", height, err) - } - - var missingCollections []*flow.CollectionGuarantee - for _, guarantee := range block.Payload.Guarantees { - inStorage, err := s.isCollectionInStorage(guarantee.CollectionID) - if err != nil { - return nil, err - } - - if !inStorage { - missingCollections = append(missingCollections, guarantee) - } - } - - return missingCollections, nil -} - -// isCollectionInStorage checks whether the given collection is present in local storage. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) isCollectionInStorage(collectionID flow.Identifier) (bool, error) { - _, err := s.collections.LightByID(collectionID) - if err == nil { - return true, nil - } - - if errors.Is(err, storage.ErrNotFound) { - return false, nil - } - - return false, fmt.Errorf("failed to retrieve collection %s: %w", collectionID.String(), err) -} - -// RequestCollectionsForBlock conditionally requests missing collections for a specific block height, -// skipping requests if the block is already below the known full block height. -func (s *CollectionSyncer) RequestCollectionsForBlock(height uint64, missingCollections []*flow.CollectionGuarantee) { - // skip requesting collections, if this block is below the last full block height. - // this means that either we have already received these collections, or the block - // may contain unverifiable guarantees (in case this node has just joined the network) - if height <= s.lastFullBlockHeight.Value() { - s.logger.Debug(). - Msg("skipping requesting collections for finalized block as its collections have been already retrieved") - return - } - - s.requestCollections(missingCollections) -} - -// requestCollections registers collection download requests in the requester engine and -// causes the requester to immediately dispatch requests. -func (s *CollectionSyncer) requestCollections(collections []*flow.CollectionGuarantee) { - for _, guarantee := range collections { - guarantors, err := protocol.FindGuarantors(s.state, guarantee) - if err != nil { - // failed to find guarantors for guarantees contained in a finalized block is fatal error - s.logger.Fatal().Err(err).Msgf("could not find guarantors for collection %v", guarantee.CollectionID) - } - s.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) - } - - if len(collections) > 0 { - s.requester.Force() - } -} - -// updateLastFullBlockHeight updates the next highest block height where all previous collections have been indexed. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) updateLastFullBlockHeight() error { - lastFullBlockHeight := s.lastFullBlockHeight.Value() - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return fmt.Errorf("failed to get finalized block: %w", err) - } - - // track the latest contiguous full height - newLastFullBlockHeight, err := s.findLowestBlockHeightWithMissingCollections(lastFullBlockHeight, lastFinalizedBlock.Height) - if err != nil { - return fmt.Errorf("failed to find last full block height: %w", err) - } - - // if more contiguous blocks are now complete, update db - if newLastFullBlockHeight > lastFullBlockHeight { - err := s.lastFullBlockHeight.Set(newLastFullBlockHeight) - if err != nil { - return fmt.Errorf("failed to update last full block height: %w", err) - } - - s.collectionExecutedMetric.UpdateLastFullBlockHeight(newLastFullBlockHeight) - - s.logger.Debug(). - Uint64("last_full_block_height", newLastFullBlockHeight). - Msg("updated last full block height counter") - } - - return nil -} - -// findLowestBlockHeightWithMissingCollections finds the next block height with missing collections, -// returning the latest contiguous height where all collections are present. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findLowestBlockHeightWithMissingCollections( - lastKnownFullBlockHeight uint64, - finalizedBlockHeight uint64, -) (uint64, error) { - newLastFullBlockHeight := lastKnownFullBlockHeight - - for currBlockHeight := lastKnownFullBlockHeight + 1; currBlockHeight <= finalizedBlockHeight; currBlockHeight++ { - missingCollections, err := s.findMissingCollectionsAtHeight(currBlockHeight) - if err != nil { - return 0, err - } - - // return when we find the first block with missing collections - if len(missingCollections) > 0 { - return newLastFullBlockHeight, nil - } - - newLastFullBlockHeight = currBlockHeight - } - - return newLastFullBlockHeight, nil -} - -// OnCollectionDownloaded indexes and persists a downloaded collection. -// This is a callback intended to be used with the requester engine. -func (s *CollectionSyncer) OnCollectionDownloaded(_ flow.Identifier, entity flow.Entity) { - collection, ok := entity.(*flow.Collection) - if !ok { - s.logger.Error().Msgf("invalid entity type (%T)", entity) - return - } - - // Create a lock context for indexing - lctx := s.lockManager.NewContext() - defer lctx.Release() - err := lctx.AcquireLock(storage.LockInsertCollection) - if err != nil { - // TODO(leo): should be using irrecoverable.Context - s.logger.Fatal().Err(err).Msg("could not acquire lock for collection indexing") - return - } - - err = indexer.IndexCollection(lctx, collection, s.collections, s.logger, s.collectionExecutedMetric) - if err != nil { - s.logger.Error().Err(err).Msg("could not index collection after it has been downloaded") - return - } -} diff --git a/engine/access/ingestion/collections/indexer.go b/engine/access/ingestion/collections/indexer.go new file mode 100644 index 00000000000..966fe6a0684 --- /dev/null +++ b/engine/access/ingestion/collections/indexer.go @@ -0,0 +1,286 @@ +package collections + +import ( + "errors" + "fmt" + "time" + + "github.com/jordanschalm/lockctx" + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/common/fifoqueue" + "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" + "github.com/onflow/flow-go/utils/logging" + "github.com/rs/zerolog" +) + +type CollectionIndexer interface { + // OnCollectionReceived notifies the collection indexer that a new collection is available to be indexed. + // Calling this method multiple times with the same collection is a no-op. + // This method is non-blocking. + OnCollectionReceived(collection *flow.Collection) + + // MissingCollectionsAtHeight returns all collections that are not present in storage for a specific + // block height. + // + // Expected error returns during normal operation: + // - [storage.ErrNotFound]: if provided block height is not finalized + MissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) + + // IsCollectionInStorage checks whether the given collection is present in local storage. + // + // No error returns are expected during normal operation. + IsCollectionInStorage(collectionID flow.Identifier) (bool, error) +} + +const ( + // lastFullBlockRefreshInterval is the interval at which the last full block height is updated. + lastFullBlockRefreshInterval = 1 * time.Second + + // defaultQueueCapacity is the default capacity of the pending collections queue. + defaultQueueCapacity = 10_000 +) + +// Indexer stores and indexes collections received from the network. It is designed to be the central +// point for accumulating collection from various subsystems that my receive them from the network. +// For example, collections may be received from execution data sync, the collection syncer, or the +// execution state indexer. Depending on the node's configuration, one or more of these subsystems +// will feed the indexer with collections. +// +// The indexer also maintains the last full block height state, which is the highest block height +// for which all collections are stored and indexed. +type Indexer struct { + log zerolog.Logger + metrics module.CollectionExecutedMetric + + state protocol.State + blocks storage.Blocks + collections storage.Collections + transactions storage.Transactions + lockManager lockctx.Manager + lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + + pendingCollectionsNotifier engine.Notifier + pendingCollectionsQueue *fifoqueue.FifoQueue +} + +// NewIndexer creates a new Indexer. +// +// No error returns are expected during normal operation. +func NewIndexer( + log zerolog.Logger, + metrics module.CollectionExecutedMetric, + state protocol.State, + blocks storage.Blocks, + collections storage.Collections, + transactions storage.Transactions, + lastFullBlockHeight *counters.PersistentStrictMonotonicCounter, + lockManager lockctx.Manager, +) (*Indexer, error) { + metrics.UpdateLastFullBlockHeight(lastFullBlockHeight.Value()) + + collectionsQueue, err := fifoqueue.NewFifoQueue(defaultQueueCapacity) + if err != nil { + return nil, fmt.Errorf("could not create collections queue: %w", err) + } + + return &Indexer{ + log: log.With().Str("component", "collection-indexer").Logger(), + metrics: metrics, + state: state, + blocks: blocks, + collections: collections, + transactions: transactions, + lockManager: lockManager, + pendingCollectionsNotifier: engine.NewNotifier(), + pendingCollectionsQueue: collectionsQueue, + lastFullBlockHeight: lastFullBlockHeight, + }, nil +} + +// WorkerLoop is a [component.ComponentWorker] that continuously processes collections submitted to +// the indexer and maintains the last full block height state. +// +// There should only be a single instance of this method running at a time. +func (ci *Indexer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + updateLastFullBlockHeightTicker := time.NewTicker(lastFullBlockRefreshInterval) + defer updateLastFullBlockHeightTicker.Stop() + + for { + select { + case <-ctx.Done(): + return + + case <-updateLastFullBlockHeightTicker.C: + err := ci.updateLastFullBlockHeight() + if err != nil { + ctx.Throw(err) + return + } + + case <-ci.pendingCollectionsNotifier.Channel(): + for { + v, ok := ci.pendingCollectionsQueue.Pop() + if !ok { + break // no more pending collections + } + + collection, ok := v.(*flow.Collection) + if !ok { + ctx.Throw(fmt.Errorf("received invalid object. expected *flow.Collection, got: %T", collection)) + return + } + + if err := ci.indexCollection(collection); err != nil { + ctx.Throw(fmt.Errorf("error indexing collection: %w", err)) + return + } + } + } + } +} + +// OnCollectionReceived notifies the collection indexer that a new collection is available to be indexed. +// Calling this method multiple times with the same collection is a no-op. +// This method is non-blocking. +func (ci *Indexer) OnCollectionReceived(collection *flow.Collection) { + if !ci.pendingCollectionsQueue.Push(collection) { + ci.log.Warn(). + Hex("collection_id", logging.ID(collection.ID())). + Msg("dropping collection because queue is full") + return + } + ci.pendingCollectionsNotifier.Notify() +} + +// indexCollection indexes a collection and its transactions. +// Skips indexing and returns without an error if the collection is already indexed. +// +// No error returns are expected during normal operation. +func (ci *Indexer) indexCollection(collection *flow.Collection) error { + // skip indexing if collection is already indexed. on the common path, collections may be received + // via multiple subsystems (e.g. execution data sync, collection sync, execution state indexer). + // In this case, the indexer will be notified multiple times for the same collection. Only the + // first notification should be processed. + // + // It's OK that this check is not done atomically with the index operation since the collections + // module will perform a similar check. Also, this module should be the only system performing + // collection writes. + exists, err := ci.IsCollectionInStorage(collection.ID()) + if err != nil { + return fmt.Errorf("failed to check if collection is in storage: %w", err) + } + if exists { + return nil + } + + lctx := ci.lockManager.NewContext() + defer lctx.Release() + err = lctx.AcquireLock(storage.LockInsertCollection) + if err != nil { + return fmt.Errorf("could not acquire lock for indexing collections: %w", err) + } + + // store the collection, including constituent transactions, and index transactionID -> collectionID + light, err := ci.collections.StoreAndIndexByTransaction(lctx, collection) + if err != nil { + return fmt.Errorf("failed to store collection: %w", err) + } + + ci.metrics.CollectionFinalized(light) + ci.metrics.CollectionExecuted(light) + return nil +} + +// updateLastFullBlockHeight updates the next highest block height where all previous collections +// are indexed if it has changed. +// +// No error returns are expected during normal operation. +func (ci *Indexer) updateLastFullBlockHeight() error { + lastFullBlockHeight := ci.lastFullBlockHeight.Value() + lastFinalizedBlock, err := ci.state.Final().Head() + if err != nil { + return fmt.Errorf("failed to get finalized block: %w", err) + } + + newLastFullBlockHeight := lastFullBlockHeight + for height := lastFullBlockHeight + 1; height <= lastFinalizedBlock.Height; height++ { + missingCollections, err := ci.MissingCollectionsAtHeight(height) + if err != nil { + // no errors are expected since all blocks are finalized and must be present in storage + return fmt.Errorf("failed to retrieve missing collections for block height %d: %w", height, err) + } + + // stop when we find the first block with missing collections + if len(missingCollections) > 0 { + break + } + + newLastFullBlockHeight = height + } + + if newLastFullBlockHeight > lastFullBlockHeight { + err = ci.lastFullBlockHeight.Set(newLastFullBlockHeight) + if err != nil { + return fmt.Errorf("failed to update last full block height: %w", err) + } + + ci.metrics.UpdateLastFullBlockHeight(newLastFullBlockHeight) + + ci.log.Debug(). + Uint64("old", lastFullBlockHeight). + Uint64("new", newLastFullBlockHeight). + Msg("updated last full block height counter") + } + + return nil +} + +// MissingCollectionsAtHeight returns all collections that are not present in storage for a specific +// block height. +// +// Expected error returns during normal operation: +// - [storage.ErrNotFound]: if provided block height is not finalized +func (ci *Indexer) MissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) { + block, err := ci.blocks.ByHeight(height) + if err != nil { + return nil, fmt.Errorf("failed to retrieve block by height %d: %w", height, err) + } + + var missingCollections []*flow.CollectionGuarantee + for _, guarantee := range block.Payload.Guarantees { + inStorage, err := ci.IsCollectionInStorage(guarantee.CollectionID) + if err != nil { + return nil, err + } + + if !inStorage { + missingCollections = append(missingCollections, guarantee) + } + } + + return missingCollections, nil +} + +// IsCollectionInStorage checks whether the given collection is present in local storage. +// +// No error returns are expected during normal operation. +func (ci *Indexer) IsCollectionInStorage(collectionID flow.Identifier) (bool, error) { + _, err := ci.collections.LightByID(collectionID) + if err == nil { + return true, nil + } + + if errors.Is(err, storage.ErrNotFound) { + return false, nil + } + + return false, fmt.Errorf("failed to retrieve collection %s: %w", collectionID.String(), err) +} diff --git a/engine/access/ingestion/collections/indexer_test.go b/engine/access/ingestion/collections/indexer_test.go new file mode 100644 index 00000000000..1ada9b0ea69 --- /dev/null +++ b/engine/access/ingestion/collections/indexer_test.go @@ -0,0 +1,432 @@ +package collections + +import ( + "context" + "errors" + "fmt" + "testing" + "testing/synctest" + "time" + + "github.com/cockroachdb/pebble/v2" + "github.com/jordanschalm/lockctx" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "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/metrics" + protocolmock "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" + "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" +) + +type IndexerSuite struct { + suite.Suite + + state *protocolmock.State + blocks *storagemock.Blocks + collections *storagemock.Collections + transactions *storagemock.Transactions + lastFullBlockCP *storagemock.ConsumerProgress + lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + lockManager lockctx.Manager + + height uint64 +} + +func TestIndexer(t *testing.T) { + suite.Run(t, new(IndexerSuite)) +} + +func (s *IndexerSuite) SetupTest() { + s.state = protocolmock.NewState(s.T()) + s.blocks = storagemock.NewBlocks(s.T()) + s.collections = storagemock.NewCollections(s.T()) + s.transactions = storagemock.NewTransactions(s.T()) + s.lastFullBlockCP = storagemock.NewConsumerProgress(s.T()) + s.lockManager = storage.NewTestingLockManager() + + s.height = 100 + + var err error + s.lastFullBlockCP.On("ProcessedIndex").Return(s.height-1, nil) + s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(s.lastFullBlockCP) + require.NoError(s.T(), err) +} + +func (s *IndexerSuite) createIndexer(t *testing.T) *Indexer { + indexer, err := NewIndexer( + unittest.Logger(), + metrics.NewNoopCollector(), + s.state, + s.blocks, + s.collections, + s.transactions, + s.lastFullBlockHeight, + s.lockManager, + ) + require.NoError(t, err) + return indexer +} + +func (s *IndexerSuite) fixture(g *fixtures.GeneratorSuite) (*flow.Block, []*flow.Collection) { + parentHeader := g.Headers().Fixture(fixtures.Header.WithHeight(s.height)) + return s.fixtureWithParent(g, parentHeader) +} + +func (s *IndexerSuite) fixtureWithParent(g *fixtures.GeneratorSuite, parentHeader *flow.Header) (*flow.Block, []*flow.Collection) { + collections := g.Collections().List(3) + guarantees := make([]*flow.CollectionGuarantee, len(collections)) + for i, collection := range collections { + guarantee := g.Guarantees().Fixture(fixtures.Guarantee.WithCollectionID(collection.ID())) + guarantees[i] = guarantee + } + payload := g.Payloads().Fixture( + fixtures.Payload.WithGuarantees(guarantees...), + fixtures.Payload.WithReceiptStubs(), + fixtures.Payload.WithResults(), + fixtures.Payload.WithSeals(), + ) + block := g.Blocks().Fixture( + fixtures.Block.WithParentHeader(parentHeader), + fixtures.Block.WithPayload(payload), + ) + return block, collections +} + +func (s *IndexerSuite) TestWorkerProcessing_ComponentLifecycle() { + synctest.Test(s.T(), func(t *testing.T) { + indexer := s.createIndexer(s.T()) + + cm := component.NewComponentManagerBuilder(). + AddWorker(indexer.WorkerLoop). + Build() + + signalerCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + defer cancel() + + cm.Start(signalerCtx) + + synctest.Wait() + unittest.RequireClosed(t, cm.Ready(), "worker should be ready") + + cancel() + synctest.Wait() + unittest.RequireClosed(t, cm.Done(), "worker should be done") + }) +} + +func (s *IndexerSuite) TestMissingCollectionsAtHeight_HappyPath() { + g := fixtures.NewGeneratorSuite() + block, collectionList := s.fixture(g) + + s.blocks.On("ByHeight", block.Height).Return(block, nil) + + s.collections.On("LightByID", collectionList[0].ID()).Return(collectionList[0].Light(), nil).Once() + s.collections.On("LightByID", mock.Anything).Return(nil, storage.ErrNotFound).Twice() + // the other 2 collections are missing + + indexer := s.createIndexer(s.T()) + + missingCollections, err := indexer.MissingCollectionsAtHeight(block.Height) + s.Require().NoError(err) + + s.Equal(len(missingCollections), 2) + s.Equal(collectionList[1].ID(), missingCollections[0].CollectionID) + s.Equal(collectionList[2].ID(), missingCollections[1].CollectionID) +} + +func (s *IndexerSuite) TestMissingCollectionsAtHeight_ErrorCases() { + g := fixtures.NewGeneratorSuite() + block, _ := s.fixture(g) + + s.Run("block error", func() { + height := s.height + 10 + s.blocks.On("ByHeight", height).Return(nil, storage.ErrNotFound).Once() + + indexer := s.createIndexer(s.T()) + + missingCollections, err := indexer.MissingCollectionsAtHeight(height) + s.Require().ErrorIs(err, storage.ErrNotFound) + s.Require().Empty(missingCollections) + }) + + s.Run("collection error", func() { + expectedErr := errors.New("expected error") + + s.blocks.On("ByHeight", block.Height).Return(block, nil).Once() + s.collections.On("LightByID", block.Payload.Guarantees[0].CollectionID).Return(nil, expectedErr).Once() + + indexer := s.createIndexer(s.T()) + + missingCollections, err := indexer.MissingCollectionsAtHeight(block.Height) + s.Require().ErrorIs(err, expectedErr) + s.Require().Empty(missingCollections) + }) +} + +func (s *IndexerSuite) TestIsCollectionInStorage() { + g := fixtures.NewGeneratorSuite() + collection := g.Collections().Fixture() + + s.Run("happy path", func() { + s.collections.On("LightByID", collection.ID()).Return(collection.Light(), nil).Once() + + indexer := s.createIndexer(s.T()) + + inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + s.Require().NoError(err) + s.Require().True(inStorage) + }) + + s.Run("not in storage", func() { + s.collections.On("LightByID", collection.ID()).Return(nil, storage.ErrNotFound).Once() + + indexer := s.createIndexer(s.T()) + + inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + s.Require().NoError(err) + s.Require().False(inStorage) + }) + + s.Run("unexpected error", func() { + expectedErr := errors.New("unexpected error") + s.collections.On("LightByID", collection.ID()).Return(nil, expectedErr).Once() + + indexer := s.createIndexer(s.T()) + + inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + s.Require().ErrorIs(err, expectedErr) + s.Require().False(inStorage) + }) +} + +func (s *IndexerSuite) TestUpdateLastFullBlockHeight() { + g := fixtures.NewGeneratorSuite() + + s.Run("no new complete blocks", func() { + indexer := s.createIndexer(s.T()) + original := indexer.lastFullBlockHeight.Value() + + // last full block height is the finalized height + finalizedBlock := g.Blocks().Fixture(fixtures.Block.WithHeight(original)) + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedBlock.ToHeader(), nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + indexer.updateLastFullBlockHeight() + + updated := indexer.lastFullBlockHeight.Value() + s.Equal(original, updated) + }) + + s.Run("updates last full block height and stops updating at finalized height", func() { + indexer := s.createIndexer(s.T()) + + lastFullBlockHeight := indexer.lastFullBlockHeight.Value() + finalizedHeight := lastFullBlockHeight + 3 + + // last full block height is the finalized height + finalizedBlock := g.Blocks().Fixture(fixtures.Block.WithHeight(finalizedHeight)) + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedBlock.ToHeader(), nil) + s.state.On("Final").Return(finalSnapshot).Once() + + // Note: it should not check any heights above the finalized height + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + block := g.Blocks().Fixture(fixtures.Block.WithHeight(height)) + s.blocks.On("ByHeight", height).Return(block, nil).Once() + for _, guarantee := range block.Payload.Guarantees { + collection := g.Collections().Fixture() + s.collections.On("LightByID", guarantee.CollectionID).Return(collection.Light(), nil).Once() + } + } + s.lastFullBlockCP.On("SetProcessedIndex", finalizedBlock.Height).Return(nil).Once() + + indexer.updateLastFullBlockHeight() + + updated := indexer.lastFullBlockHeight.Value() + s.Equal(finalizedBlock.Height, updated) + }) +} + +func (s *IndexerSuite) TestOnCollectionReceived() { + g := fixtures.NewGeneratorSuite() + + collection := g.Collections().Fixture() + + synctest.Test(s.T(), func(t *testing.T) { + s.collections.On("LightByID", collection.ID()).Return(nil, storage.ErrNotFound).Once() + s.collections.On("StoreAndIndexByTransaction", mock.Anything, collection).Return(collection.Light(), nil).Once() + + indexer := s.createIndexer(s.T()) + + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + defer cancel() + + ready := make(chan struct{}) + done := make(chan struct{}) + go func() { + defer close(done) + indexer.WorkerLoop(ctx, func() { close(ready) }) + }() + + synctest.Wait() + unittest.RequireClosed(t, ready, "worker should be ready") + + // calling OnCollectionReceived stores and indexes the collection when the collection is not in storage. + indexer.OnCollectionReceived(collection) + synctest.Wait() + + // calling OnCollectionReceived does skips indexing when the collection is already in storage. + s.collections.On("LightByID", collection.ID()).Unset() + s.collections.On("LightByID", collection.ID()).Return(collection.Light(), nil).Once() + + indexer.OnCollectionReceived(collection) + synctest.Wait() + }) +} + +func (s *IndexerSuite) TestWorkerProcessing_ProcessesCollections() { + g := fixtures.NewGeneratorSuite() + rootBlock := g.Blocks().Genesis() + + RunWithBlockchain(s.T(), func(bc *blockchain) { + initializer := store.NewConsumerProgress(bc.db, module.ConsumeProgressLastFullBlockHeight) + progress, err := initializer.Initialize(rootBlock.Height) + s.Require().NoError(err) + + lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter(progress) + s.Require().NoError(err) + + indexer, err := NewIndexer( + unittest.Logger(), + metrics.NewNoopCollector(), + bc.state, + bc.all.Blocks, + bc.collections, + bc.transactions, + lastFullBlockHeight, + bc.lockManager, + ) + s.Require().NoError(err) + + cm := component.NewComponentManagerBuilder(). + AddWorker(indexer.WorkerLoop). + Build() + + signalerCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + defer cancel() + + cm.Start(signalerCtx) + unittest.RequireCloseBefore(s.T(), cm.Ready(), 100*time.Millisecond, "could not start worker") + defer func() { + cancel() + unittest.RequireCloseBefore(s.T(), cm.Done(), 100*time.Millisecond, "could not stop worker") + }() + + prevHeader := rootBlock.ToHeader() + for range 10 { + block, collectionList := s.fixtureWithParent(g, prevHeader) + bc.finalizeBlock(s.T(), block) + prevHeader = block.ToHeader() + + for _, collection := range collectionList { + indexer.OnCollectionReceived(collection) + } + + // wait until all collections are indexed + require.Eventually(s.T(), func() bool { + missingCollections, err := indexer.MissingCollectionsAtHeight(block.Height) + return err == nil && len(missingCollections) == 0 + }, time.Second, 10*time.Millisecond) + + // check that all collection indices are populated + // ByID uses all of the indices. + for _, collection := range collectionList { + actual, err := bc.collections.ByID(collection.ID()) + s.Require().NoError(err) + s.Require().Equal(collection, actual) + } + } + + // wait until the last full block height is updated + require.Eventually(s.T(), func() bool { + return lastFullBlockHeight.Value() == prevHeader.Height + }, time.Second, 100*time.Millisecond) + }) +} + +type blockchain struct { + db storage.DB + lockManager lockctx.Manager + + all *store.All + transactions storage.Transactions + collections storage.Collections + + state *protocolmock.State +} + +func newBlockchain(t *testing.T, pdb *pebble.DB) *blockchain { + metrics := metrics.NewNoopCollector() + + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + + all := store.InitAll(metrics, db) + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + return &blockchain{ + db: db, + lockManager: lockManager, + all: all, + transactions: transactions, + collections: collections, + state: protocolmock.NewState(t), + } +} + +func (bc *blockchain) finalizeBlock(t *testing.T, block *flow.Block) { + // Add the target block as finalized. + err := unittest.WithLocks(t, bc.lockManager, []string{ + storage.LockInsertBlock, + storage.LockFinalizeBlock, + }, func(lctx lockctx.Context) error { + return bc.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + proposal := unittest.ProposalFromBlock(block) + if err := bc.all.Blocks.BatchStore(lctx, rw, proposal); err != nil { + return fmt.Errorf("could not store block: %w", err) + } + if err := operation.IndexFinalizedBlockByHeight(lctx, rw, block.Height, block.ID()); err != nil { + return fmt.Errorf("could not index block by height: %w", err) + } + return nil + }) + }) + require.NoError(t, err) + + // final snapshot may not be queried for every block. + finalSnapshot := protocolmock.NewSnapshot(t) + finalSnapshot.On("Head").Return(block.ToHeader(), nil).Maybe() + bc.state.On("Final").Unset() + bc.state.On("Final").Return(finalSnapshot) +} + +func RunWithBlockchain(t *testing.T, fn func(bc *blockchain)) { + unittest.RunWithPebbleDB(t, func(pdb *pebble.DB) { + bc := newBlockchain(t, pdb) + fn(bc) + }) +} diff --git a/engine/access/ingestion/collections/mock/collection_indexer.go b/engine/access/ingestion/collections/mock/collection_indexer.go new file mode 100644 index 00000000000..9cfafd0df99 --- /dev/null +++ b/engine/access/ingestion/collections/mock/collection_indexer.go @@ -0,0 +1,90 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// CollectionIndexer is an autogenerated mock type for the CollectionIndexer type +type CollectionIndexer struct { + mock.Mock +} + +// IsCollectionInStorage provides a mock function with given fields: collectionID +func (_m *CollectionIndexer) IsCollectionInStorage(collectionID flow.Identifier) (bool, error) { + ret := _m.Called(collectionID) + + if len(ret) == 0 { + panic("no return value specified for IsCollectionInStorage") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(collectionID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(collectionID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(collectionID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MissingCollectionsAtHeight provides a mock function with given fields: height +func (_m *CollectionIndexer) MissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) { + ret := _m.Called(height) + + if len(ret) == 0 { + panic("no return value specified for MissingCollectionsAtHeight") + } + + var r0 []*flow.CollectionGuarantee + var r1 error + if rf, ok := ret.Get(0).(func(uint64) ([]*flow.CollectionGuarantee, error)); ok { + return rf(height) + } + if rf, ok := ret.Get(0).(func(uint64) []*flow.CollectionGuarantee); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*flow.CollectionGuarantee) + } + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// OnCollectionReceived provides a mock function with given fields: collection +func (_m *CollectionIndexer) OnCollectionReceived(collection *flow.Collection) { + _m.Called(collection) +} + +// NewCollectionIndexer creates a new instance of CollectionIndexer. 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 NewCollectionIndexer(t interface { + mock.TestingT + Cleanup(func()) +}) *CollectionIndexer { + mock := &CollectionIndexer{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/engine/access/ingestion/collections/syncer.go b/engine/access/ingestion/collections/syncer.go new file mode 100644 index 00000000000..0bafe69f247 --- /dev/null +++ b/engine/access/ingestion/collections/syncer.go @@ -0,0 +1,363 @@ +package collections + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/rs/zerolog" + + "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" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/util" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +const ( + // DefaultCollectionCatchupTimeout is the timeout for the initial collection catchup process + // during startup. + DefaultCollectionCatchupTimeout = 30 * time.Second + + // DefaultCollectionCatchupDBPollInterval is the interval at which the storage is polled to check + // if missing collections have been received during the initial collection catchup process. + DefaultCollectionCatchupDBPollInterval = 1 * time.Second + + // DefaultMissingCollsRequestInterval is the interval at which missing collections are requested + // from the network. + DefaultMissingCollsRequestInterval = 1 * time.Minute + + // DefaultMissingCollsForBlockThreshold is the threshold number of blocks with missing collections + // beyond which collections should be re-requested this is to prevent spamming the collection + // nodes with request + DefaultMissingCollsForBlockThreshold = 100 + + // DefaultMissingCollsForAgeThreshold is the block height threshold beyond which collections + // should be re-requested, regardless of the number of blocks for which collection are missing. + // This is to ensure that if a collection is missing for a long time (in terms of block height) + // it is eventually re-requested. + DefaultMissingCollsForAgeThreshold = 100 +) + +// The Syncer is responsible for syncing collections for finalized blocks from the network. It has +// three main responsibilities: +// 1. Handle requests for collections for finalized blocks from the ingestion engine by submitting +// the requests to Collection nodes. +// 2. Track blocks with missing collections, and periodically re-request the missing collections +// from the network. +// 3. Submit collections received to the Indexer for storage and indexing. +// +// The Syncer guarantees that all collection for finalized blocks will eventually be received as long +// as there are honest Collection nodes in each cluster, and the node is able to successfully communicate +// with them over the networking layer. +// +// It is responsible for ensuring the local node has all collections associated with finalized blocks +// starting from the last fully synced height. +// It works by periodically scanning the finalized block range from the last full block height up to +// the latest finalized block height, identifying missing collections, and triggering requests to +// fetch them from the network. Once collections are retrieved, it submits them to the Indexer for +// storage and indexing. +// +// It is meant to operate in a background goroutine as part of the node's ingestion pipeline. +type Syncer struct { + log zerolog.Logger + + requester module.Requester + indexer CollectionIndexer + state protocol.State + collections storage.Collections + lastFullBlockHeight counters.Reader + execDataSyncer *ExecutionDataSyncer + + collectionCatchupTimeout time.Duration + collectionCatchupDBPollInterval time.Duration + missingCollsForBlockThreshold int + missingCollsForAgeThreshold uint64 + missingCollsRequestInterval time.Duration +} + +// NewSyncer creates a new Syncer responsible for requesting, tracking, and indexing missing collections. +func NewSyncer( + log zerolog.Logger, + requester module.Requester, + state protocol.State, + collections storage.Collections, + lastFullBlockHeight counters.Reader, + collectionIndexer CollectionIndexer, + execDataSyncer *ExecutionDataSyncer, +) *Syncer { + return &Syncer{ + log: log.With().Str("component", "collection-syncer").Logger(), + state: state, + requester: requester, + collections: collections, + lastFullBlockHeight: lastFullBlockHeight, + indexer: collectionIndexer, + execDataSyncer: execDataSyncer, + + collectionCatchupTimeout: DefaultCollectionCatchupTimeout, + collectionCatchupDBPollInterval: DefaultCollectionCatchupDBPollInterval, + missingCollsForBlockThreshold: DefaultMissingCollsForBlockThreshold, + missingCollsForAgeThreshold: DefaultMissingCollsForAgeThreshold, + missingCollsRequestInterval: DefaultMissingCollsRequestInterval, + } +} + +// WorkerLoop is a [component.ComponentWorker] that continuously monitors for missing collections, and +// requests them from the network if needed. It also performs an initial collection catchup on startup. +func (s *Syncer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + requestCtx, cancel := context.WithTimeout(ctx, s.collectionCatchupTimeout) + defer cancel() + + // attempt to download all known missing collections on start-up. + err := s.requestMissingCollectionsBlocking(requestCtx) + if err != nil { + if ctx.Err() != nil { + s.log.Error().Err(err).Msg("engine shutdown while downloading missing collections") + return + } + + if !errors.Is(err, context.DeadlineExceeded) { + ctx.Throw(fmt.Errorf("error downloading missing collections: %w", err)) + return + } + + // timed out during catchup. continue with normal startup. + // missing collections will be requested periodically. + s.log.Error().Err(err).Msg("timed out syncing collections during startup") + } + ready() + + requestCollectionsTicker := time.NewTicker(s.missingCollsRequestInterval) + defer requestCollectionsTicker.Stop() + + for { + select { + case <-ctx.Done(): + return + + case <-requestCollectionsTicker.C: + err := s.requestMissingCollections(ctx) + if err != nil { + ctx.Throw(err) + return + } + } + } +} + +// OnCollectionDownloaded notifies the collection syncer that a collection has been downloaded. +// This callback implements [requester.HandleFunc] and is intended to be used with the requester engine. +func (s *Syncer) OnCollectionDownloaded(_ flow.Identifier, entity flow.Entity) { + s.indexer.OnCollectionReceived(entity.(*flow.Collection)) +} + +// RequestCollectionsForBlock conditionally requests missing collections for a specific block height, +// skipping requests if the block is already below the known full block height. +// +// No error returns are expected during normal operation. +func (s *Syncer) RequestCollectionsForBlock(height uint64, missingCollections []*flow.CollectionGuarantee) error { + // skip requesting collections, if this block is below the last full block height. + // this means that either we have already received these collections, or the block + // may contain unverifiable guarantees (in case this node has just joined the network) + if height <= s.lastFullBlockHeight.Value() { + s.log.Debug().Msg("collections for finalized block already retrieved. skipping request.") + return nil + } + + err := s.requestCollections(missingCollections) + if err != nil { + return fmt.Errorf("failed to request collections: %w", err) + } + + // trigger immediate dispatch of any pending collection requests. + s.requester.Force() + + return nil +} + +// requestCollections registers collection download requests in the requester engine and +// causes the requester to immediately dispatch requests. +// +// No error returns are expected during normal operation. +func (s *Syncer) requestCollections(collections []*flow.CollectionGuarantee) error { + for _, guarantee := range collections { + guarantors, err := protocol.FindGuarantors(s.state, guarantee) + if err != nil { + // failed to find guarantors for guarantees contained in a finalized block is fatal error + return fmt.Errorf("could not find guarantors for collection %v: %w", guarantee.CollectionID, err) + } + s.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) + } + + return nil +} + +// requestMissingCollections checks if missing collections should be requested based on configured +// block or age thresholds and triggers requests if needed. +// +// No error returns are expected during normal operation. +func (s *Syncer) requestMissingCollections(ctx context.Context) error { + lastFullBlockHeight := s.lastFullBlockHeight.Value() + lastFinalizedBlock, err := s.state.Final().Head() + if err != nil { + return fmt.Errorf("failed to get finalized block: %w", err) + } + + // if the node is syncing execution data, use the already downloaded data to index any available + // collections we are still missing. + lastSyncedHeight := lastFullBlockHeight + if s.execDataSyncer != nil { + lastSyncedHeight, err = s.execDataSyncer.IndexFromStartHeight(ctx, lastFullBlockHeight) + if err != nil { + return fmt.Errorf("failed to index collections from execution data: %w", err) + } + // At this point, we have already submitted all collections within blocks up to `lastSyncedHeight` + // for indexing. However, indexing is completed asynchronously, so we updated `lastSyncedHeight` + // to be the last block for which we have execution data to avoid re-requesting already + // submitted collections. + } + + // request all other missing collections from Collection nodes. + collections, incompleteBlocksCount, err := s.findMissingCollections(lastSyncedHeight, lastFinalizedBlock.Height) + if err != nil { + return err + } + + blocksThresholdReached := incompleteBlocksCount >= s.missingCollsForBlockThreshold + ageThresholdReached := lastFinalizedBlock.Height-lastFullBlockHeight > s.missingCollsForAgeThreshold + if len(collections) > 0 && (blocksThresholdReached || ageThresholdReached) { + // warn log since this should generally not happen + s.log.Warn(). + Uint64("finalized_height", lastFinalizedBlock.Height). + Uint64("last_full_blk_height", lastFullBlockHeight). + Int("missing_collection_blk_count", incompleteBlocksCount). + Int("missing_collection_count", len(collections)). + Msg("re-requesting missing collections") + + err = s.requestCollections(collections) + if err != nil { + return fmt.Errorf("failed to request collections: %w", err) + } + // since this is a re-request, do not use force. new finalized block requests will force + // dispatch. On the happy path, this will happen at least once per second. + } + + return nil +} + +// findMissingCollections scans block heights from last known full block up to the latest finalized +// block and returns all missing collection along with the count of incomplete blocks. +// +// No error returns are expected during normal operation. +func (s *Syncer) findMissingCollections(lastFullBlockHeight, finalizedBlockHeight uint64) ([]*flow.CollectionGuarantee, int, error) { + var missingCollections []*flow.CollectionGuarantee + var incompleteBlocksCount int + + for height := lastFullBlockHeight + 1; height <= finalizedBlockHeight; height++ { + collections, err := s.indexer.MissingCollectionsAtHeight(height) + if err != nil { + return nil, 0, err + } + + if len(collections) > 0 { + missingCollections = append(missingCollections, collections...) + incompleteBlocksCount += 1 + } + } + + return missingCollections, incompleteBlocksCount, nil +} + +// requestMissingCollectionsBlocking requests and waits for all missing collections to be downloaded, +// blocking until either completion or context timeout. +// +// No error returns are expected during normal operation. +func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { + lastFullBlockHeight := s.lastFullBlockHeight.Value() + lastFinalizedBlock, err := s.state.Final().Head() + if err != nil { + return fmt.Errorf("failed to get finalized block: %w", err) + } + + progress := util.LogProgress(s.log, util.DefaultLogProgressConfig("requesting missing collections", lastFinalizedBlock.Height-lastFullBlockHeight)) + + collectionsToBeDownloaded := make(map[flow.Identifier]struct{}) + for height := lastFullBlockHeight + 1; height <= lastFinalizedBlock.Height; height++ { + if ctx.Err() != nil { + return fmt.Errorf("missing collection catchup interrupted: %w", ctx.Err()) + } + + collections, err := s.indexer.MissingCollectionsAtHeight(height) + if err != nil { + return fmt.Errorf("failed to find missing collections at height %d: %w", height, err) + } + + if len(collections) > 0 { + var submitted bool + if s.execDataSyncer != nil { + submitted, err = s.execDataSyncer.IndexForHeight(ctx, height) + if err != nil { + return fmt.Errorf("failed to index collections from execution data: %w", err) + } + } + + // if the data wasn't available from execution data, request it from Collection nodes. + if !submitted { + err = s.requestCollections(collections) + if err != nil { + return fmt.Errorf("failed to request collections: %w", err) + } + for _, collection := range collections { + collectionsToBeDownloaded[collection.CollectionID] = struct{}{} + } + } + } + + progress(1) + } + + if len(collectionsToBeDownloaded) == 0 { + s.log.Info().Msg("no missing collections to download") + return nil + } + + // trigger immediate dispatch of any pending collection requests. + s.requester.Force() + + collectionStoragePollTicker := time.NewTicker(s.collectionCatchupDBPollInterval) + defer collectionStoragePollTicker.Stop() + + // we want to wait for all collections to be downloaded so we poll local storage periodically to make sure each + // collection was successfully saved in the storage. + for len(collectionsToBeDownloaded) > 0 { + select { + case <-ctx.Done(): + return fmt.Errorf("failed to complete collection retrieval: %w", ctx.Err()) + + case <-collectionStoragePollTicker.C: + s.log.Debug(). + Int("total_missing_collections", len(collectionsToBeDownloaded)). + Msg("retrieving missing collections...") + + for collectionID := range collectionsToBeDownloaded { + downloaded, err := s.indexer.IsCollectionInStorage(collectionID) + if err != nil { + return err + } + + if downloaded { + delete(collectionsToBeDownloaded, collectionID) + } + } + } + } + + s.log.Info().Msg("collection catchup done") + return nil +} diff --git a/engine/access/ingestion/collections/syncer_execution_data.go b/engine/access/ingestion/collections/syncer_execution_data.go new file mode 100644 index 00000000000..adca55b060f --- /dev/null +++ b/engine/access/ingestion/collections/syncer_execution_data.go @@ -0,0 +1,72 @@ +package collections + +import ( + "context" + "errors" + "fmt" + + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage" + "github.com/rs/zerolog" +) + +type ExecutionDataSyncer struct { + log zerolog.Logger + + executionDataCache execution_data.ExecutionDataCache + indexer CollectionIndexer +} + +func NewExecutionDataSyncer( + log zerolog.Logger, + executionDataCache execution_data.ExecutionDataCache, + indexer CollectionIndexer, +) *ExecutionDataSyncer { + return &ExecutionDataSyncer{ + log: log.With().Str("component", "execution-data-syncer").Logger(), + executionDataCache: executionDataCache, + indexer: indexer, + } +} + +// IndexForHeight indexes the collections for a given height using locally available execution data. +// Returns false and no error if execution data for the block is not available. +// +// No error returns are expected during normal operation. +func (s *ExecutionDataSyncer) IndexForHeight(ctx context.Context, height uint64) (bool, error) { + executionData, err := s.executionDataCache.ByHeight(ctx, height) + if err != nil { + if errors.Is(err, storage.ErrNotFound) || execution_data.IsBlobNotFoundError(err) { + return false, nil // data for the block is not available yet. + } + return false, fmt.Errorf("failed to get execution data for height %d: %w", height, err) + } + + // index all collections except for the system chunk. + for _, chunkData := range executionData.ChunkExecutionDatas[:len(executionData.ChunkExecutionDatas)-1] { + s.indexer.OnCollectionReceived(chunkData.Collection) + } + + return true, nil +} + +// IndexFromStartHeight indexes the collections for all blocks with available execution data starting +// from the last full block height. Returns the last indexed height. +// +// No error returns are expected during normal operation. +func (s *ExecutionDataSyncer) IndexFromStartHeight(ctx context.Context, lastFullBlockHeight uint64) (uint64, error) { + lastIndexedHeight := lastFullBlockHeight + height := lastFullBlockHeight + 1 + for { + submitted, err := s.IndexForHeight(ctx, height) + if err != nil { + return 0, err + } + if !submitted { + return lastIndexedHeight, nil + } + + lastIndexedHeight = height + height++ + } +} diff --git a/engine/access/ingestion/collections/syncer_test.go b/engine/access/ingestion/collections/syncer_test.go new file mode 100644 index 00000000000..d08f11548f9 --- /dev/null +++ b/engine/access/ingestion/collections/syncer_test.go @@ -0,0 +1,638 @@ +package collections + +import ( + "context" + "errors" + "testing" + "testing/synctest" + "time" + + "github.com/ipfs/go-cid" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + collectionsmock "github.com/onflow/flow-go/engine/access/ingestion/collections/mock" + "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" + execdatamock "github.com/onflow/flow-go/module/executiondatasync/execution_data/mock" + "github.com/onflow/flow-go/module/irrecoverable" + modulemock "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/module/signature" + protocolmock "github.com/onflow/flow-go/state/protocol/mock" + storagemock "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/fixtures" +) + +type SyncerSuite struct { + suite.Suite + + state *protocolmock.State + collections *storagemock.Collections + lastFullBlockHeightCP *storagemock.ConsumerProgress + indexer *collectionsmock.CollectionIndexer + executionDataCache *execdatamock.ExecutionDataCache + + requester *modulemock.Requester + lastFullBlockHeight *counters.PersistentStrictMonotonicCounter +} + +func TestSyncerSuite(t *testing.T) { + suite.Run(t, new(SyncerSuite)) +} + +func (s *SyncerSuite) SetupTest() { + s.state = protocolmock.NewState(s.T()) + s.collections = storagemock.NewCollections(s.T()) + s.lastFullBlockHeightCP = storagemock.NewConsumerProgress(s.T()) + s.indexer = collectionsmock.NewCollectionIndexer(s.T()) + s.requester = modulemock.NewRequester(s.T()) + s.executionDataCache = execdatamock.NewExecutionDataCache(s.T()) + + var err error + s.lastFullBlockHeightCP.On("ProcessedIndex").Return(uint64(100), nil).Once() + s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(s.lastFullBlockHeightCP) + s.Require().NoError(err) +} + +func (s *SyncerSuite) createSyncer() *Syncer { + execDataSyncer := NewExecutionDataSyncer( + unittest.Logger(), + s.executionDataCache, + s.indexer, + ) + + return NewSyncer( + unittest.Logger(), + s.requester, + s.state, + s.collections, + s.lastFullBlockHeight, + s.indexer, + execDataSyncer, + ) +} + +// TestComponentLifecycle tests starting and stopping the syncer's worker loop within a component manager. +func (s *SyncerSuite) TestComponentLifecycle() { + synctest.Test(s.T(), func(t *testing.T) { + g := fixtures.NewGeneratorSuite() + header := g.Headers().Fixture(fixtures.Header.WithHeight(s.lastFullBlockHeight.Value())) + + // finalized block is the same as last full block height, so no catchup is needed + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(header, nil).Once() + s.state.On("Final").Return(finalSnapshot, nil).Once() + + syncer := s.createSyncer() + + cm := component.NewComponentManagerBuilder(). + AddWorker(syncer.WorkerLoop). + Build() + + signalerCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + defer cancel() + + cm.Start(signalerCtx) + + synctest.Wait() + unittest.RequireClosed(t, cm.Ready(), "worker should be ready") + + cancel() + synctest.Wait() + unittest.RequireClosed(t, cm.Done(), "worker should be done") + }) +} + +// TestOnCollectionDownloaded tests that the syncer calls OnCollectionReceived on the indexer when +// it receives a collection downloaded notification. +func (s *SyncerSuite) TestOnCollectionDownloaded() { + g := fixtures.NewGeneratorSuite() + collection := g.Collections().Fixture() + + s.indexer.On("OnCollectionReceived", collection).Once() + + syncer := s.createSyncer() + + syncer.OnCollectionDownloaded(g.Identifiers().Fixture(), collection) +} + +// TestRequestCollectionsForBlock tests calls to RequestCollectionsForBlock submit requests for the +// provided collections for heights > last full block height, and ignores requests for blocks that +// are <= last full block height. +func (s *SyncerSuite) TestRequestCollectionsForBlock() { + g := fixtures.NewGeneratorSuite() + + guarantors := g.Identities().List(3, fixtures.Identity.WithRole(flow.RoleCollection)) + signerIndices, err := signature.EncodeSignersToIndices(guarantors.NodeIDs(), guarantors.NodeIDs()) + require.NoError(s.T(), err) + + guarantees := g.Guarantees().List(10, fixtures.Guarantee.WithSignerIndices(signerIndices)) + + syncer := s.createSyncer() + + s.Run("request for block <= last full block height", func() { + // EntityByID should not be called + err := syncer.RequestCollectionsForBlock(s.lastFullBlockHeight.Value(), guarantees) + s.Require().NoError(err) + }) + + s.Run("request for block above last full block height", func() { + for _, guarantee := range guarantees { + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() + } + + // called once for the batch + s.requester.On("Force").Once() + + err := syncer.RequestCollectionsForBlock(s.lastFullBlockHeight.Value()+1, guarantees) + s.Require().NoError(err) + }) + + s.Run("finding guarantors fails", func() { + expectedError := errors.New("state lookup failed") + s.mockGuarantorsForCollectionReturnsError(guarantees[0], expectedError) + + err := syncer.RequestCollectionsForBlock(s.lastFullBlockHeight.Value()+1, guarantees) + s.Require().ErrorIs(err, expectedError) + }) +} + +// TestWorkerLoop_InitialCatchup_CollectionNodesOnly tests the initial collection catchup when configured +// to only use collection nodes to retrieve missing collections. +func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_CollectionNodesOnly() { + g := fixtures.NewGeneratorSuite() + + guarantors := g.Identities().List(3, fixtures.Identity.WithRole(flow.RoleCollection)) + signerIndices, err := signature.EncodeSignersToIndices(guarantors.NodeIDs(), guarantors.NodeIDs()) + s.Require().NoError(err) + + finalizedHeight := s.lastFullBlockHeight.Value() + 10 + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)), nil).Once() + s.state.On("Final").Return(finalSnapshot, nil).Once() + + guaranteesByHeight := make(map[uint64][]*flow.CollectionGuarantee) + for height := s.lastFullBlockHeight.Value() + 1; height <= finalizedHeight; height++ { + guarantees := g.Guarantees().List(10, fixtures.Guarantee.WithSignerIndices(signerIndices)) + guaranteesByHeight[height] = guarantees + + s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() + + for _, guarantee := range guarantees { + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() + } + } + + // called once for the batch + s.requester.On("Force").Once() + + for _, guarantees := range guaranteesByHeight { + for _, guarantee := range guarantees { + // simulate the collection being missing by randomly returning false. All collections should + // eventually be found in storage. + s.indexer.On("IsCollectionInStorage", guarantee.CollectionID).Return(func(collectionID flow.Identifier) (bool, error) { + return g.Random().Bool(), nil + }) + } + } + + syncer := NewSyncer( + unittest.Logger(), + s.requester, + s.state, + s.collections, + s.lastFullBlockHeight, + s.indexer, + nil, // execution data indexing is disabled + ) + + synctest.Test(s.T(), func(t *testing.T) { + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + + done := make(chan struct{}) + ready := make(chan struct{}) + + go func() { + defer close(done) + syncer.WorkerLoop(ctx, func() { close(ready) }) + }() + + iterations := 0 + loop: + for { + // note: this sleep advances the synctest bubble's time, but does not actually block + time.Sleep(syncer.collectionCatchupDBPollInterval + 1) + + synctest.Wait() + select { + case <-ready: + break loop + default: + } + + // there are 100 collections, and each collection is marked as downloaded with a 50% + // probability. Allow up to 100 iterations in pessimistic case, but it should complete + // much faster. + iterations++ + if iterations > 100 { + t.Error("worker never completed initial catchup") + break + } + } + + cancel() + synctest.Wait() + unittest.RequireClosed(t, done, "worker should be done") + }) +} + +// TestWorkerLoop_InitialCatchup_SplitExecutionDataAndCollectionNodes tests the initial collection +// catchup when configured to use both execution data and collection nodes to retrieve missing collections. +func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_SplitExecutionDataAndCollectionNodes() { + g := fixtures.NewGeneratorSuite() + + guarantors := g.Identities().List(3, fixtures.Identity.WithRole(flow.RoleCollection)) + signerIndices, err := signature.EncodeSignersToIndices(guarantors.NodeIDs(), guarantors.NodeIDs()) + s.Require().NoError(err) + + finalizedHeight := s.lastFullBlockHeight.Value() + 10 + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)), nil).Once() + s.state.On("Final").Return(finalSnapshot, nil).Once() + + // simulate that execution data is only available for the first 5 blocks. + maxExecutionDataHeight := s.lastFullBlockHeight.Value() + 5 + + // the syncer should lookup execution data for each of these blocks, and submit the collections + // to the indexer. It should stop after the first height for which execution data is not available + // within the cache. + for height := s.lastFullBlockHeight.Value() + 1; height <= maxExecutionDataHeight; height++ { + execData, guarantees := executionDataFixture(g) + + s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() + + for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { + s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() + } + } + + // the syncer should request collections from collection nodes for all remaining heights. + guaranteesByHeight := make(map[uint64][]*flow.CollectionGuarantee) + for height := maxExecutionDataHeight + 1; height <= finalizedHeight; height++ { + guarantees := g.Guarantees().List(10, fixtures.Guarantee.WithSignerIndices(signerIndices)) + guaranteesByHeight[height] = guarantees + + s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(nil, execution_data.NewBlobNotFoundError(cid.Cid{})).Once() + + for _, guarantee := range guarantees { + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() + } + } + + // called once for the batch + s.requester.On("Force").Once() + + for _, guarantees := range guaranteesByHeight { + for _, guarantee := range guarantees { + // simulate the collection being missing by randomly returning false. All collections should + // eventually be found in storage. + s.indexer.On("IsCollectionInStorage", guarantee.CollectionID).Return(func(collectionID flow.Identifier) (bool, error) { + return g.Random().Bool(), nil + }) + } + } + + syncer := s.createSyncer() + + synctest.Test(s.T(), func(t *testing.T) { + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + + done := make(chan struct{}) + ready := make(chan struct{}) + + go func() { + defer close(done) + syncer.WorkerLoop(ctx, func() { close(ready) }) + }() + + iterations := 0 + loop: + for { + // note: this sleep advances the synctest bubble's time, but does not actually block + time.Sleep(syncer.collectionCatchupDBPollInterval + 1) + + synctest.Wait() + select { + case <-ready: + break loop + default: + } + + // there are 100 collections, and each collection is marked as downloaded with a 50% + // probability. Allow up to 100 iterations in pessimistic case, but it should complete + // much faster. + iterations++ + if iterations > 100 { + t.Error("worker never completed initial catchup") + break + } + } + + cancel() + synctest.Wait() + unittest.RequireClosed(t, done, "worker should be done") + }) +} + +// TestWorkerLoop_InitialCatchup_AllAvailableFromExecutionData tests the initial collection catchup +// when configured to use execution data and data is available for all blocks. +func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_AllAvailableFromExecutionData() { + g := fixtures.NewGeneratorSuite() + + finalizedHeight := s.lastFullBlockHeight.Value() + 10 + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)), nil).Once() + s.state.On("Final").Return(finalSnapshot, nil).Once() + + // simulate that execution data is available for all blocks. + maxExecutionDataHeight := finalizedHeight + + // the syncer should lookup execution data for each of these blocks, and submit the collections + // to the indexer. + for height := s.lastFullBlockHeight.Value() + 1; height <= maxExecutionDataHeight; height++ { + execData, guarantees := executionDataFixture(g) + + s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() + + for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { + s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() + } + } + + syncer := s.createSyncer() + + synctest.Test(s.T(), func(t *testing.T) { + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + + done := make(chan struct{}) + ready := make(chan struct{}) + + go func() { + defer close(done) + syncer.WorkerLoop(ctx, func() { close(ready) }) + }() + + synctest.Wait() + unittest.RequireClosed(t, ready, "worker should be ready") + + cancel() + synctest.Wait() + unittest.RequireClosed(t, done, "worker should be done") + }) +} + +// TestWorkerLoop_InitialCatchup_Timesout tests the initial collection catchup times out before completing, +// and gracefully continues with normal startup. +func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_Timesout() { + g := fixtures.NewGeneratorSuite() + + synctest.Test(s.T(), func(t *testing.T) { + finalizedHeight := s.lastFullBlockHeight.Value() + 10 + finalizedHeader := g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)) + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + // block the first call to MissingCollectionsAtHeight that is called during startup until after + // the timeout. This simulates the catchup logic taking too long. + unblockStartup := make(chan struct{}) + s.indexer. + On("MissingCollectionsAtHeight", s.lastFullBlockHeight.Value()+1). + Return(func(uint64) ([]*flow.CollectionGuarantee, error) { + // note: this sleep advances the synctest bubble's time, but does not actually block + time.Sleep(DefaultCollectionCatchupTimeout + time.Second) + close(unblockStartup) + return nil, nil + }).Once() + + syncer := NewSyncer( + unittest.Logger(), + s.requester, + s.state, + s.collections, + s.lastFullBlockHeight, + s.indexer, + nil, // execution data indexing is disabled + ) + + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + + done := make(chan struct{}) + ready := make(chan struct{}) + + go func() { + defer close(done) + syncer.WorkerLoop(ctx, func() { close(ready) }) + }() + + // this is only closed after the timeout, so ready should be closed + <-unblockStartup + unittest.RequireClosed(t, ready, "worker should be ready") + + cancel() + synctest.Wait() + unittest.RequireClosed(t, done, "worker should be done") + }) +} + +func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { + g := fixtures.NewGeneratorSuite() + + guarantors := g.Identities().List(3, fixtures.Identity.WithRole(flow.RoleCollection)) + signerIndices, err := signature.EncodeSignersToIndices(guarantors.NodeIDs(), guarantors.NodeIDs()) + s.Require().NoError(err) + + lastFullBlockHeight := s.lastFullBlockHeight.Value() + finalizedHeight := lastFullBlockHeight + 10 + finalizedHeader := g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)) + allGuarantees := make(map[uint64][]*flow.CollectionGuarantee, 0) + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + allGuarantees[height] = g.Guarantees().List(3, fixtures.Guarantee.WithSignerIndices(signerIndices)) + } + + s.Run("no missing collections", func() { + s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + s.indexer.On("MissingCollectionsAtHeight", height).Return(nil, nil).Once() + } + }) + }) + + s.Run("missing collections - request skipped below thresholds", func() { + s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + } + }) + }) + + s.Run("missing collections - request sent when count exceeds missingCollsForBlockThreshold", func() { + s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + syncer.missingCollsForBlockThreshold = 9 + + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + for _, guarantee := range allGuarantees[height] { + s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + } + } + }) + }) + + s.Run("missing collections - request sent when age exceeds missingCollsForAgeThreshold", func() { + s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + syncer.missingCollsForAgeThreshold = 9 + + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + for height := s.lastFullBlockHeight.Value() + 1; height <= finalizedHeight; height++ { + s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + } + + for height := s.lastFullBlockHeight.Value() + 1; height <= finalizedHeight; height++ { + for _, guarantee := range allGuarantees[height] { + s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + } + } + }) + }) + + s.Run("missing collections - processed from execution data", func() { + execDataSyncer := NewExecutionDataSyncer( + unittest.Logger(), + s.executionDataCache, + s.indexer, + ) + + s.runWorkerLoopMissingCollections(g, execDataSyncer, func(syncer *Syncer) { + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { + execData, _ := executionDataFixture(g) + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() + for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { + s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() + } + } + // syncer continues until it receives a not found error. + s.executionDataCache.On("ByHeight", mock.Anything, finalizedHeight+1).Return(nil, execution_data.NewBlobNotFoundError(cid.Cid{})).Once() + }) + }) +} + +func (s *SyncerSuite) runWorkerLoopMissingCollections(g *fixtures.GeneratorSuite, execDataSyncer *ExecutionDataSyncer, onReady func(*Syncer)) { + synctest.Test(s.T(), func(t *testing.T) { + // last full block is latest finalized block, so initial catchup is skipped + finalizedHeight := s.lastFullBlockHeight.Value() + finalizedHeader := g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)) + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + syncer := NewSyncer( + unittest.Logger(), + s.requester, + s.state, + s.collections, + s.lastFullBlockHeight, + s.indexer, + execDataSyncer, + ) + + ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + + done := make(chan struct{}) + ready := make(chan struct{}) + + go func() { + defer close(done) + syncer.WorkerLoop(ctx, func() { + onReady(syncer) + close(ready) + }) + }() + + <-ready + + time.Sleep(syncer.missingCollsRequestInterval + 1) + synctest.Wait() + + cancel() + synctest.Wait() + unittest.RequireClosed(t, done, "worker should be done") + }) +} + +func executionDataFixture(g *fixtures.GeneratorSuite) (*execution_data.BlockExecutionDataEntity, []*flow.CollectionGuarantee) { + chunkDatas := g.ChunkExecutionDatas().List(10) + execData := g.BlockExecutionDataEntities().Fixture(fixtures.BlockExecutionData.WithChunkExecutionDatas(chunkDatas...)) + guarantees := make([]*flow.CollectionGuarantee, len(chunkDatas)-1) + for i := range len(chunkDatas) - 1 { + guarantees[i] = g.Guarantees().Fixture(fixtures.Guarantee.WithCollectionID(chunkDatas[i].Collection.ID())) + } + return execData, guarantees +} + +func (s *SyncerSuite) mockGuarantorsForCollection(guarantee *flow.CollectionGuarantee, members flow.IdentitySkeletonList) { + cluster := protocolmock.NewCluster(s.T()) + cluster.On("Members").Return(members, nil).Once() + + epoch := protocolmock.NewCommittedEpoch(s.T()) + epoch.On("ClusterByChainID", guarantee.ClusterChainID).Return(cluster, nil).Once() + + query := protocolmock.NewEpochQuery(s.T()) + query.On("Current").Return(epoch, nil).Once() + + snapshot := protocolmock.NewSnapshot(s.T()) + snapshot.On("Epochs").Return(query).Once() + + s.state.On("AtBlockID", guarantee.ReferenceBlockID).Return(snapshot).Once() +} + +func (s *SyncerSuite) mockGuarantorsForCollectionReturnsError(guarantee *flow.CollectionGuarantee, expectedError error) { + query := protocolmock.NewEpochQuery(s.T()) + query.On("Current").Return(nil, expectedError).Once() + + snapshot := protocolmock.NewSnapshot(s.T()) + snapshot.On("Epochs").Return(query).Once() + + s.state.On("AtBlockID", guarantee.ReferenceBlockID).Return(snapshot).Once() +} diff --git a/engine/access/ingestion/engine.go b/engine/access/ingestion/engine.go index d544b3effd8..82e9da250e6 100644 --- a/engine/access/ingestion/engine.go +++ b/engine/access/ingestion/engine.go @@ -3,12 +3,12 @@ package ingestion import ( "context" "fmt" - "time" "github.com/rs/zerolog" "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/access/ingestion/tx_error_messages" "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/flow" @@ -24,26 +24,6 @@ import ( ) const ( - // time to wait for the all the missing collections to be received at node startup - collectionCatchupTimeout = 30 * time.Second - - // time to poll the storage to check if missing collections have been received - collectionCatchupDBPollInterval = 10 * time.Millisecond - - // time to update the FullBlockHeight index - fullBlockRefreshInterval = 1 * time.Second - - // time to request missing collections from the network - missingCollsRequestInterval = 1 * time.Minute - - // a threshold of number of blocks with missing collections beyond which collections should be re-requested - // this is to prevent spamming the collection nodes with request - missingCollsForBlockThreshold = 100 - - // a threshold of block height beyond which collections should be re-requested (regardless of the number of blocks for which collection are missing) - // this is to ensure that if a collection is missing for a long time (in terms of block height) it is eventually re-requested - missingCollsForAgeThreshold = 100 - // default queue capacity defaultQueueCapacity = 10_000 @@ -83,7 +63,9 @@ type Engine struct { maxReceiptHeight uint64 executionResults storage.ExecutionResults - collectionSyncer *CollectionSyncer + collectionSyncer *collections.Syncer + collectionIndexer *collections.Indexer + // TODO: There's still a need for this metric to be in the ingestion engine rather than collection syncer. // Maybe it is a good idea to split it up? collectionExecutedMetric module.CollectionExecutedMetric @@ -105,7 +87,8 @@ func New( executionResults storage.ExecutionResults, executionReceipts storage.ExecutionReceipts, finalizedProcessedHeight storage.ConsumerProgressInitializer, - collectionSyncer *CollectionSyncer, + collectionSyncer *collections.Syncer, + collectionIndexer *collections.Indexer, collectionExecutedMetric module.CollectionExecutedMetric, txErrorMessagesCore *tx_error_messages.TxErrorMessagesCore, ) (*Engine, error) { @@ -147,6 +130,7 @@ func New( messageHandler: messageHandler, txErrorMessagesCore: txErrorMessagesCore, collectionSyncer: collectionSyncer, + collectionIndexer: collectionIndexer, } // jobqueue Jobs object that tracks finalized blocks by height. This is used by the finalizedBlockConsumer @@ -176,12 +160,11 @@ func New( // Add workers builder := component.NewComponentManagerBuilder(). - AddWorker(e.collectionSyncer.RequestCollections). + AddWorker(e.collectionSyncer.WorkerLoop). + AddWorker(e.collectionIndexer.WorkerLoop). AddWorker(e.processExecutionReceipts). AddWorker(e.runFinalizedBlockConsumer) - //TODO: should I add a check for nil ptr for collection syncer ? (as done below) - // If txErrorMessagesCore is provided, add a worker responsible for processing // transaction result error messages by receipts. This worker listens for blocks // containing execution receipts and processes any associated transaction result @@ -385,7 +368,10 @@ func (e *Engine) processFinalizedBlock(block *flow.Block) error { } } - e.collectionSyncer.RequestCollectionsForBlock(block.Height, block.Payload.Guarantees) + err = e.collectionSyncer.RequestCollectionsForBlock(block.Height, block.Payload.Guarantees) + if err != nil { + return fmt.Errorf("could not request collections for block: %w", err) + } e.collectionExecutedMetric.BlockFinalized(block) return nil diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index ece7712446a..ba9f4eb0905 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -2,7 +2,6 @@ package ingestion import ( "context" - "math/rand" "os" "sync" "testing" @@ -15,10 +14,10 @@ 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" - "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" @@ -29,12 +28,13 @@ import ( "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" + protocolmock "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/fixtures" "github.com/onflow/flow-go/utils/unittest/mocks" ) @@ -43,9 +43,9 @@ type Suite struct { // protocol state proto struct { - state *protocol.FollowerState - snapshot *protocol.Snapshot - params *protocol.Params + state *protocolmock.FollowerState + snapshot *protocolmock.Snapshot + params *protocolmock.Params } me *modulemock.Local @@ -103,18 +103,12 @@ func (s *Suite) SetupTest() { 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.proto.state = new(protocolmock.FollowerState) + s.proto.snapshot = new(protocolmock.Snapshot) + s.proto.params = new(protocolmock.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() @@ -144,11 +138,11 @@ func (s *Suite) SetupTest() { s.rootBlock = unittest.Block.Genesis(flow.Emulator) parent := s.rootBlock.ToHeader() - for i := 0; i < blockCount; i++ { + for range blockCount { block := unittest.BlockWithParentFixture(parent) + s.blockMap[block.Height] = block // update for next iteration parent = block.ToHeader() - s.blockMap[block.Height] = block } s.finalizedBlock = parent @@ -159,14 +153,6 @@ func (s *Suite) SetupTest() { ), ).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) @@ -187,19 +173,17 @@ func (s *Suite) SetupTest() { // initEngineAndSyncer create new instance of ingestion engine and collection syncer. // It waits until the ingestion engine starts. -func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *CollectionSyncer) { +func (s *Suite) initEngineAndSyncer() (*Engine, *collections.Syncer, *collections.Indexer) { 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) - syncer := NewCollectionSyncer( + indexer, err := collections.NewIndexer( s.log, s.collectionExecutedMetric, - s.request, s.proto.state, s.blocks, s.collections, @@ -207,6 +191,17 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, 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, + ) eng, err := New( s.log, @@ -218,16 +213,13 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, s.receipts, processedHeightInitializer, syncer, + indexer, s.collectionExecutedMetric, nil, ) - require.NoError(s.T(), err) - eng.ComponentManager.Start(ctx) - <-eng.Ready() - - return eng, syncer + return eng, syncer, indexer } // mockCollectionsForBlock mocks collections for block @@ -241,7 +233,7 @@ func (s *Suite) mockCollectionsForBlock(block *flow.Block) { } // generateBlock prepares block with payload and specified guarantee.SignerIndices -func (s *Suite) generateBlock(clusterCommittee flow.IdentitySkeletonList, snap *protocol.Snapshot) *flow.Block { +func (s *Suite) generateBlock(clusterCommittee flow.IdentitySkeletonList, snap *protocolmock.Snapshot) *flow.Block { block := unittest.BlockFixture( unittest.Block.WithPayload(unittest.PayloadFixture( unittest.WithGuarantees(unittest.CollectionGuaranteesFixture(4)...), @@ -267,10 +259,14 @@ func (s *Suite) generateBlock(clusterCommittee flow.IdentitySkeletonList, snap * // TestOnFinalizedBlock checks that when a block is received, a request for each individual collection is made func (s *Suite) TestOnFinalizedBlockSingle() { - cluster := new(protocol.Cluster) - epoch := new(protocol.CommittedEpoch) - epochs := new(protocol.EpochQuery) - snap := new(protocol.Snapshot) + cluster := new(protocolmock.Cluster) + epoch := new(protocolmock.CommittedEpoch) + epochs := new(protocolmock.EpochQuery) + snap := new(protocolmock.Snapshot) + + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() + s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) epochs.On("Current").Return(epoch, nil) @@ -280,8 +276,15 @@ func (s *Suite) TestOnFinalizedBlockSingle() { 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) + eng, _, _ := s.initEngineAndSyncer() + + irrecoverableCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), s.ctx) + eng.ComponentManager.Start(irrecoverableCtx) + unittest.RequireCloseBefore(s.T(), eng.Ready(), 100*time.Millisecond, "could not start worker") + defer func() { + cancel() + unittest.RequireCloseBefore(s.T(), eng.Done(), 100*time.Millisecond, "could not stop worker") + }() block := s.generateBlock(clusterCommittee, snap) block.Height = s.finalizedBlock.Height + 1 @@ -326,10 +329,14 @@ func (s *Suite) TestOnFinalizedBlockSingle() { // TestOnFinalizedBlockSeveralBlocksAhead checks OnFinalizedBlock with a block several blocks newer than the last block processed func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { - cluster := new(protocol.Cluster) - epoch := new(protocol.CommittedEpoch) - epochs := new(protocol.EpochQuery) - snap := new(protocol.Snapshot) + cluster := new(protocolmock.Cluster) + epoch := new(protocolmock.CommittedEpoch) + epochs := new(protocolmock.EpochQuery) + snap := new(protocolmock.Snapshot) + + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() + s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) epochs.On("Current").Return(epoch, nil) @@ -339,8 +346,15 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { 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) + eng, _, _ := s.initEngineAndSyncer() + + irrecoverableCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), s.ctx) + eng.ComponentManager.Start(irrecoverableCtx) + unittest.RequireCloseBefore(s.T(), eng.Ready(), 100*time.Millisecond, "could not start worker") + defer func() { + cancel() + unittest.RequireCloseBefore(s.T(), eng.Done(), 100*time.Millisecond, "could not stop worker") + }() newBlocksCount := 3 startHeight := s.finalizedBlock.Height + 1 @@ -403,31 +417,13 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { s.results.AssertNumberOfCalls(s.T(), "Index", expectedIndexCalls) } -// TestOnCollection checks that when a Collection is received, it is persisted -func (s *Suite) TestOnCollection() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - s.initEngineAndSyncer(irrecoverableCtx) - - collection := unittest.CollectionFixture(5) - light := collection.Light() - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", mock.Anything, &collection).Return(light, nil).Once() - - // Create a lock context for indexing - err := unittest.WithLock(s.T(), s.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { - return indexer.IndexCollection(lctx, &collection, s.collections, s.log, s.collectionExecutedMetric) - }) - require.NoError(s.T(), err) - - // check that the collection was stored and indexed - s.collections.AssertExpectations(s.T()) -} - // TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed func (s *Suite) TestExecutionReceiptsAreIndexed() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() + s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() + + eng, _, _ := s.initEngineAndSyncer() originID := unittest.IdentifierFixture() collection := unittest.CollectionFixture(5) @@ -475,352 +471,118 @@ func (s *Suite) TestExecutionReceiptsAreIndexed() { s.receipts.AssertExpectations(s.T()) } -// TestOnCollectionDuplicate checks that when a duplicate collection is received, the node doesn't -// crash but just ignores its transactions. -func (s *Suite) TestOnCollectionDuplicate() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - s.initEngineAndSyncer(irrecoverableCtx) - - collection := unittest.CollectionFixture(5) - light := collection.Light() - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", mock.Anything, &collection).Return(light, storage.ErrAlreadyExists).Once() - - // Create a lock context for indexing - err := unittest.WithLock(s.T(), s.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { - return indexer.IndexCollection(lctx, &collection, s.collections, s.log, s.collectionExecutedMetric) - }) - require.ErrorIs(s.T(), err, storage.ErrAlreadyExists) - - // check that the collection was stored and indexed - s.collections.AssertExpectations(s.T()) -} - -// TestRequestMissingCollections tests that the all missing collections are requested on the call to requestMissingCollections -func (s *Suite) TestRequestMissingCollections() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - _, syncer := s.initEngineAndSyncer(irrecoverableCtx) - - blkCnt := 3 - startHeight := uint64(1000) +// TestCollectionSyncing tests the happy path of syncing collections for finalized blocks. +// It performs syncs for a single block passed via the OnFinalizedBlock callback, and verifies that +// the finalized block processing logic submits the request for each collection in the finalized block, +// the indexer indexes all collections received from the network, and the last full block height is +// updated after the collections are indexed. +func (s *Suite) TestCollectionSyncing() { + g := fixtures.NewGeneratorSuite() - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - - // generate the test blocks and collections - var collIDs []flow.Identifier - refBlockID := unittest.IdentifierFixture() - for i := 0; i < blkCnt; i++ { - block := unittest.BlockFixture( - // some blocks may not be present hence add a gap - unittest.Block.WithHeight(startHeight+uint64(i)), - unittest.Block.WithPayload(unittest.PayloadFixture( - unittest.WithGuarantees(unittest.CollectionGuaranteesFixture(4, unittest.WithCollRef(refBlockID))...)), - )) - s.blockMap[block.Height] = block - s.finalizedBlock = block.ToHeader() - - for _, c := range block.Payload.Guarantees { - collIDs = append(collIDs, c.CollectionID) - c.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) - c.SignerIndices = indices - } - } - - // consider collections are missing for all blocks - err := s.lastFullBlockHeight.Set(startHeight - 1) - s.Require().NoError(err) - - // consider the last test block as the head - - // p is the probability of not receiving the collection before the next poll and it - // helps simulate the slow trickle of the requested collections being received - var p float32 - - // rcvdColl is the map simulating the collection storage key-values - rcvdColl := make(map[flow.Identifier]struct{}) + guarantors := g.Identities().List(3, fixtures.Identity.WithRole(flow.RoleCollection)) + signerIndices, err := signature.EncodeSignersToIndices(guarantors.NodeIDs(), guarantors.NodeIDs()) + require.NoError(s.T(), err) - // for the first lookup call for each collection, it will be reported as missing from db - // for the subsequent calls, it will be reported as present with the probability p - s.collections.On("LightByID", mock.Anything).Return( - func(cID flow.Identifier) *flow.LightCollection { - return nil // the actual collection object return is never really read - }, - func(cID flow.Identifier) error { - if _, ok := rcvdColl[cID]; ok { - return nil - } - if rand.Float32() >= p { - rcvdColl[cID] = struct{}{} - } - return storage.ErrNotFound - }). - // simulate some db i/o contention - After(time.Millisecond * time.Duration(rand.Intn(5))) - - // set up the requester engine mock - // entityByID should be called once per collection - for _, c := range collIDs { - s.request.On("EntityByID", c, mock.Anything).Return() + collections := g.Collections().List(10) + guarantees := make([]*flow.CollectionGuarantee, len(collections)) + for i, collection := range collections { + guarantee := g.Guarantees().Fixture( + fixtures.Guarantee.WithCollectionID(collection.ID()), + fixtures.Guarantee.WithSignerIndices(signerIndices), + ) + guarantees[i] = guarantee } - // force should be called once - s.request.On("Force").Return() - - cluster := new(protocol.Cluster) - cluster.On("Members").Return(clusterCommittee, nil) - epoch := new(protocol.CommittedEpoch) - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs := new(protocol.EpochQuery) - epochs.On("Current").Return(epoch, nil) - snap := new(protocol.Snapshot) - snap.On("Epochs").Return(epochs) - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - assertExpectations := func() { - s.request.AssertExpectations(s.T()) - s.collections.AssertExpectations(s.T()) - s.proto.snapshot.AssertExpectations(s.T()) - s.blocks.AssertExpectations(s.T()) + payload := g.Payloads().Fixture( + fixtures.Payload.WithGuarantees(guarantees...), + fixtures.Payload.WithSeals(g.Seals().List(10)...), + ) + block := g.Blocks().Fixture( + fixtures.Block.WithPayload(payload), + fixtures.Block.WithParentHeader(s.finalizedBlock), // replace the finalized block + ) + s.blockMap[block.Height] = block + hotstuffBlock := hotmodel.Block{ + BlockID: block.ID(), } - // test 1 - collections are not received before timeout - s.Run("timeout before all missing collections are received", func() { - - // simulate that collection are never received - p = 1 - - // timeout after 3 db polls - ctx, cancel := context.WithTimeout(context.Background(), 100*collectionCatchupDBPollInterval) - defer cancel() - - err := syncer.requestMissingCollectionsBlocking(ctx) - - require.Error(s.T(), err) - require.Contains(s.T(), err.Error(), "context deadline exceeded") - - assertExpectations() - }) - // test 2 - all collections are eventually received before the deadline - s.Run("all missing collections are received", func() { - - // 90% of the time, collections are reported as not received when the collection storage is queried - p = 0.9 - - ctx, cancel := context.WithTimeout(context.Background(), collectionCatchupTimeout) - defer cancel() - - err := syncer.requestMissingCollectionsBlocking(ctx) - - require.NoError(s.T(), err) - require.Len(s.T(), rcvdColl, len(collIDs)) - - assertExpectations() - }) -} - -// TestProcessBackgroundCalls tests that updateLastFullBlockHeight and checkMissingCollections -// function calls keep the FullBlockIndex up-to-date and request collections if blocks with missing -// collections exceed the threshold. -func (s *Suite) TestProcessBackgroundCalls() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - _, syncer := s.initEngineAndSyncer(irrecoverableCtx) - - blkCnt := 3 - collPerBlk := 10 - startHeight := uint64(1000) - blocks := make([]*flow.Block, blkCnt) - collMap := make(map[flow.Identifier]*flow.LightCollection, blkCnt*collPerBlk) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() + for _, guarantee := range block.Payload.Guarantees { + // initially, all collections should be missing from storage + s.collections.On("LightByID", guarantee.CollectionID).Return(nil, storage.ErrNotFound) - refBlockID := unittest.IdentifierFixture() - // generate the test blocks, cgs and collections - for i := 0; i < blkCnt; i++ { - guarantees := make([]*flow.CollectionGuarantee, collPerBlk) - for j := 0; j < collPerBlk; j++ { - coll := unittest.CollectionFixture(2).Light() - collMap[coll.ID()] = coll - cg := unittest.CollectionGuaranteeFixture(func(cg *flow.CollectionGuarantee) { - cg.CollectionID = coll.ID() - cg.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) - cg.SignerIndices = indices - guarantees[j] = cg - } - block := unittest.BlockFixture( - unittest.Block.WithHeight(startHeight+uint64(i)), - unittest.Block.WithPayload(unittest.PayloadFixture(unittest.WithGuarantees(guarantees...))), - ) - s.blockMap[block.Height] = block - blocks[i] = block - s.finalizedBlock = block.ToHeader() + // setup requester engine requests + s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) + s.request.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() } + s.request.On("Force").Once() - finalizedHeight := s.finalizedBlock.Height - - cluster := new(protocol.Cluster) - cluster.On("Members").Return(clusterCommittee, nil) - epoch := new(protocol.CommittedEpoch) - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs := new(protocol.EpochQuery) - epochs.On("Current").Return(epoch, nil) - snap := new(protocol.Snapshot) - snap.On("Epochs").Return(epochs) - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - // blkMissingColl controls which collections are reported as missing by the collections storage mock - blkMissingColl := make([]bool, blkCnt) - for i := 0; i < blkCnt; i++ { - blkMissingColl[i] = false - for _, cg := range blocks[i].Payload.Guarantees { - j := i - s.collections.On("LightByID", cg.CollectionID).Return( - func(cID flow.Identifier) *flow.LightCollection { - return collMap[cID] - }, - func(cID flow.Identifier) error { - if blkMissingColl[j] { - return storage.ErrNotFound - } - return nil - }) - } + // setup finalized block indexer mocks + guaranteeIDs := []flow.Identifier(flow.GetIDs(block.Payload.Guarantees)) + s.blocks.On("IndexBlockContainingCollectionGuarantees", block.ID(), guaranteeIDs).Return(nil).Once() + for _, seal := range payload.Seals { + s.results.On("Index", seal.BlockID, seal.ResultID).Return(nil).Once() } - rootBlk := blocks[0] - - // root block is the last complete block - err := s.lastFullBlockHeight.Set(rootBlk.Height) - s.Require().NoError(err) - - s.Run("missing collections are requested when count exceeds defaultMissingCollsForBlockThreshold", func() { - // lower the block threshold to request missing collections - defaultMissingCollsForBlockThreshold = 2 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - // setup receive engine expectations - for _, cg := range blocks[i].Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() - } - } - // force should be called once - s.request.On("Force").Return().Once() - - err := syncer.requestMissingCollections() - s.Require().NoError(err) + // initialize the engine using the initial finalized block + initialFinalSnapshot := protocolmock.NewSnapshot(s.T()) + initialFinalSnapshot.On("Head").Return(s.finalizedBlock, nil) + s.proto.state.On("Final").Return(initialFinalSnapshot, nil) - // assert that missing collections are requested - s.request.AssertExpectations(s.T()) + eng, syncer, _ := s.initEngineAndSyncer() - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // no new call to UpdateLastFullBlockHeight should be made - }) + irrecoverableCtx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), s.ctx) + eng.ComponentManager.Start(irrecoverableCtx) + unittest.RequireCloseBefore(s.T(), eng.Ready(), 1*time.Second, "could not start worker") + defer func() { + cancel() + unittest.RequireCloseBefore(s.T(), eng.Done(), 100*time.Millisecond, "could not stop worker") + }() - s.Run("missing collections are requested when count exceeds defaultMissingCollsForAgeThreshold", func() { - // lower the height threshold to request missing collections - defaultMissingCollsForAgeThreshold = 1 + // progress the finalized block, and submit the new block to the engine + newFinalSnapshot := protocolmock.NewSnapshot(s.T()) + newFinalSnapshot.On("Head").Return(block.ToHeader(), nil) + s.proto.state.On("Final").Unset() + s.proto.state.On("Final").Return(newFinalSnapshot, nil) - // raise the block threshold to ensure it does not trigger missing collection request - defaultMissingCollsForBlockThreshold = blkCnt + 1 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - // setup receive engine expectations - for _, cg := range blocks[i].Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() - } - } - // force should be called once - s.request.On("Force").Return().Once() - - err := syncer.requestMissingCollections() - s.Require().NoError(err) - - // assert that missing collections are requested - s.request.AssertExpectations(s.T()) - - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made - }) - - s.Run("missing collections are not requested if defaultMissingCollsForBlockThreshold not reached", func() { - // raise the thresholds to avoid requesting missing collections - defaultMissingCollsForAgeThreshold = 3 - defaultMissingCollsForBlockThreshold = 3 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - } - - err := syncer.requestMissingCollections() - s.Require().NoError(err) - - // assert that missing collections are not requested even though there are collections missing - s.request.AssertExpectations(s.T()) - - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made - }) - - // create new block - height := blocks[blkCnt-1].Height + 1 - finalizedBlk := unittest.BlockFixture( - unittest.Block.WithHeight(height), - ) - s.blockMap[height] = finalizedBlk + eng.OnFinalizedBlock(&hotstuffBlock) - finalizedHeight = finalizedBlk.Height - s.finalizedBlock = finalizedBlk.ToHeader() + // wait until the finalized block jobqueue completes processing the block + require.Eventually(s.T(), func() bool { + return eng.finalizedBlockConsumer.LastProcessedIndex() == block.Height + }, 2*time.Second, 100*time.Millisecond, "finalized block processor never processed block") - blockBeforeFinalized := blocks[blkCnt-1] + // all requests should be sent after the finalized block processor completes processing the block. + // The requester engine calls the syncer's OnCollectionDownloaded callback for each response. + // simulate receiving the collection responses from the network. + for _, collection := range collections { + collectionID := collection.ID() + light := collection.Light() + s.collections.On("StoreAndIndexByTransaction", mock.Anything, collection).Return(light, nil).Once() - s.Run("full block height index is advanced if newer full blocks are discovered", func() { - // set lastFullBlockHeight to block - err = s.lastFullBlockHeight.Set(blockBeforeFinalized.Height) - s.Require().NoError(err) + // the collections are now available in storage. + s.collections.On("LightByID", collectionID).Unset() + s.collections.On("LightByID", collectionID).Return(light, nil) - err = syncer.updateLastFullBlockHeight() - s.Require().NoError(err) - s.Require().Equal(finalizedHeight, s.lastFullBlockHeight.Value()) - s.Require().NoError(err) + syncer.OnCollectionDownloaded(g.Identifiers().Fixture(), collection) + } - s.blocks.AssertExpectations(s.T()) - }) + // make sure that the collection indexer updates the last full block height + require.Eventually(s.T(), func() bool { + return s.lastFullBlockHeight.Value() == block.Height + }, 2*time.Second, 100*time.Millisecond, "last full block height never updated") +} - s.Run("full block height index is not advanced beyond finalized blocks", func() { - err = syncer.updateLastFullBlockHeight() - s.Require().NoError(err) +func (s *Suite) mockGuarantorsForCollection(guarantee *flow.CollectionGuarantee, members flow.IdentitySkeletonList) { + cluster := protocolmock.NewCluster(s.T()) + cluster.On("Members").Return(members, nil).Once() - s.Require().Equal(finalizedHeight, s.lastFullBlockHeight.Value()) - s.blocks.AssertExpectations(s.T()) - }) -} + epoch := protocolmock.NewCommittedEpoch(s.T()) + epoch.On("ClusterByChainID", guarantee.ClusterChainID).Return(cluster, nil).Once() -func (s *Suite) TestComponentShutdown() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) + query := protocolmock.NewEpochQuery(s.T()) + query.On("Current").Return(epoch, nil).Once() - // 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) + snapshot := protocolmock.NewSnapshot(s.T()) + snapshot.On("Epochs").Return(query).Once() - err := eng.Process(channels.ReceiveReceipts, unittest.IdentifierFixture(), new(flow.ExecutionReceipt)) - s.Assert().ErrorIs(err, component.ErrComponentShutdown) + s.proto.state.On("AtBlockID", guarantee.ReferenceBlockID).Return(snapshot).Once() } diff --git a/engine/access/ingestion2/collection_syncer.go b/engine/access/ingestion2/collection_syncer.go deleted file mode 100644 index 397c135b56a..00000000000 --- a/engine/access/ingestion2/collection_syncer.go +++ /dev/null @@ -1,475 +0,0 @@ -package ingestion2 - -import ( - "context" - "errors" - "fmt" - "time" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/common/fifoqueue" - "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" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/state_synchronization/indexer" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -const ( - // time to wait for the all the missing collections to be received at node startup - collectionCatchupTimeout = 30 * time.Second - - // time to poll the storage to check if missing collections have been received - collectionCatchupDBPollInterval = 10 * time.Millisecond - - // time to request missing collections from the network - missingCollsRequestInterval = 1 * time.Minute - - // a threshold of number of blocks with missing collections beyond which collections should be re-requested - // this is to prevent spamming the collection nodes with request - missingCollsForBlockThreshold = 100 - - // a threshold of block height beyond which collections should be re-requested (regardless of the number of blocks for which collection are missing) - // this is to ensure that if a collection is missing for a long time (in terms of block height) it is eventually re-requested - missingCollsForAgeThreshold = 100 - - // time to update the FullBlockHeight index - fullBlockRefreshInterval = 1 * time.Second -) - -var ( - // we change these values in tests. that's why we want to have their non-const shape - defaultMissingCollsForBlockThreshold = missingCollsForBlockThreshold - defaultMissingCollsForAgeThreshold uint64 = missingCollsForAgeThreshold -) - -// The CollectionSyncer type provides mechanisms for syncing and indexing data -// from the Flow blockchain into local storage. Specifically, it handles -// the retrieval and processing of collections and transactions that may -// have been missed due to network delays, restarts, or gaps in finalization. -// -// It is responsible for ensuring the local node has -// all collections associated with finalized blocks starting from the -// last fully synced height. It works by periodically scanning the finalized -// block range, identifying missing collections, and triggering requests -// to fetch them from the network. Once collections are retrieved, it -// ensures they are persisted in the local collection and transaction stores. -// -// The syncer maintains a persistent, strictly monotonic counter -// (`lastFullBlockHeight`) to track the highest finalized block for which -// all collections have been fully indexed. It uses this information to -// avoid redundant processing and to measure catch-up progress. -// -// It is meant to operate in a background goroutine as part of the -// node's ingestion pipeline. -type CollectionSyncer struct { - logger zerolog.Logger - collectionExecutedMetric module.CollectionExecutedMetric - - state protocol.State - requester module.Requester - - // collections to be indexed - pendingCollections *engine.FifoMessageStore - pendingCollectionsHandler *engine.MessageHandler - - blocks storage.Blocks - collections storage.Collections - transactions storage.Transactions - - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter - lockManager storage.LockManager -} - -// NewCollectionSyncer creates a new CollectionSyncer responsible for requesting, -// tracking, and indexing missing collections. -func NewCollectionSyncer( - logger zerolog.Logger, - collectionExecutedMetric module.CollectionExecutedMetric, - requester module.Requester, - state protocol.State, - blocks storage.Blocks, - collections storage.Collections, - transactions storage.Transactions, - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter, - lockManager storage.LockManager, -) (*CollectionSyncer, error) { - collectionExecutedMetric.UpdateLastFullBlockHeight(lastFullBlockHeight.Value()) - - collectionsQueue, err := fifoqueue.NewFifoQueue(defaultQueueCapacity) - if err != nil { - return nil, fmt.Errorf("could not create collections queue: %w", err) - } - - pendingCollections := &engine.FifoMessageStore{FifoQueue: collectionsQueue} - pendingCollectionsHandler := engine.NewMessageHandler( - logger, - engine.NewNotifier(), - engine.Pattern{ - Match: func(msg *engine.Message) bool { - _, ok := msg.Payload.(*flow.Collection) - return ok - }, - Store: pendingCollections, - }, - ) - - return &CollectionSyncer{ - logger: logger, - state: state, - requester: requester, - pendingCollectionsHandler: pendingCollectionsHandler, - pendingCollections: pendingCollections, - blocks: blocks, - collections: collections, - transactions: transactions, - lastFullBlockHeight: lastFullBlockHeight, - collectionExecutedMetric: collectionExecutedMetric, - lockManager: lockManager, - }, nil -} - -// StartWorkerLoop continuously monitors and triggers collection sync operations. -// It handles on startup collection catchup, periodic missing collection requests, and full block height updates. -func (s *CollectionSyncer) StartWorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - requestCtx, cancel := context.WithTimeout(ctx, collectionCatchupTimeout) - defer cancel() - - // on start-up, AN wants to download all missing collections to serve it to end users - err := s.requestMissingCollectionsBlocking(requestCtx) - if err != nil { - s.logger.Error().Err(err).Msg("error downloading missing collections") - } - ready() - - requestCollectionsTicker := time.NewTicker(missingCollsRequestInterval) - defer requestCollectionsTicker.Stop() - - // Collections are requested concurrently in this design. - // To maintain accurate progress tracking and avoid redundant requests, - // we periodically update the `lastFullBlockHeight` to reflect the latest - // finalized block with all collections successfully indexed. - updateLastFullBlockHeightTicker := time.NewTicker(fullBlockRefreshInterval) - defer updateLastFullBlockHeightTicker.Stop() - - for { - select { - case <-ctx.Done(): - return - - case <-requestCollectionsTicker.C: - err := s.requestMissingCollections() - if err != nil { - ctx.Throw(err) - } - - case <-updateLastFullBlockHeightTicker.C: - err := s.updateLastFullBlockHeight() - if err != nil { - ctx.Throw(err) - } - - case <-s.pendingCollectionsHandler.GetNotifier(): - msg, ok := s.pendingCollections.Get() - if !ok { - ctx.Throw(fmt.Errorf("could not get pending collection")) - } - - collection, ok := msg.Payload.(*flow.Collection) - if !ok { - ctx.Throw(fmt.Errorf("could not cast pending collection to *flow.Collection. got: %T", msg.Payload)) - return - } - - // Create a lock context for indexing - lctx := s.lockManager.NewContext() - err := lctx.AcquireLock(storage.LockInsertCollection) - if err != nil { - ctx.Throw(fmt.Errorf("could not acquire lock for collection indexing: %w", err)) - return - } - defer lctx.Release() - - err = indexer.IndexCollection(lctx, collection, s.collections, s.logger, s.collectionExecutedMetric) - if err != nil { - ctx.Throw(fmt.Errorf("error indexing collection: %w", err)) - return - } - } - } -} - -// requestMissingCollections checks if missing collections should be requested based on configured -// block or age thresholds and triggers requests if needed. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) requestMissingCollections() error { - lastFullBlockHeight := s.lastFullBlockHeight.Value() - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return fmt.Errorf("failed to get finalized block: %w", err) - } - - collections, incompleteBlocksCount, err := s.findMissingCollections(lastFullBlockHeight) - if err != nil { - return err - } - - blocksThresholdReached := incompleteBlocksCount >= defaultMissingCollsForBlockThreshold - ageThresholdReached := lastFinalizedBlock.Height-lastFullBlockHeight > defaultMissingCollsForAgeThreshold - shouldRequest := blocksThresholdReached || ageThresholdReached - - if shouldRequest { - // warn log since generally this should not happen - s.logger.Warn(). - Uint64("finalized_height", lastFinalizedBlock.Height). - Uint64("last_full_blk_height", lastFullBlockHeight). - Int("missing_collection_blk_count", incompleteBlocksCount). - Int("missing_collection_count", len(collections)). - Msg("re-requesting missing collections") - - s.requestCollections(collections, false) - } - - return nil -} - -// requestMissingCollectionsBlocking requests and waits for all missing collections to be downloaded, -// blocking until either completion or context timeout. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) requestMissingCollectionsBlocking(ctx context.Context) error { - missingCollections, _, err := s.findMissingCollections(s.lastFullBlockHeight.Value()) - if err != nil { - return err - } - if len(missingCollections) == 0 { - s.logger.Info().Msg("skipping requesting missing collections. no missing collections found") - return nil - } - - s.requestCollections(missingCollections, true) - - collectionsToBeDownloaded := make(map[flow.Identifier]struct{}) - for _, collection := range missingCollections { - collectionsToBeDownloaded[collection.CollectionID] = struct{}{} - } - - collectionStoragePollTicker := time.NewTicker(collectionCatchupDBPollInterval) - defer collectionStoragePollTicker.Stop() - - // we want to wait for all collections to be downloaded so we poll local storage periodically to make sure each - // collection was successfully saved in the storage. - for len(collectionsToBeDownloaded) > 0 { - select { - case <-ctx.Done(): - return fmt.Errorf("failed to complete collection retrieval: %w", ctx.Err()) - - case <-collectionStoragePollTicker.C: - s.logger.Info(). - Int("total_missing_collections", len(collectionsToBeDownloaded)). - Msg("retrieving missing collections...") - - for collectionID := range collectionsToBeDownloaded { - downloaded, err := s.isCollectionInStorage(collectionID) - if err != nil { - return err - } - - if downloaded { - delete(collectionsToBeDownloaded, collectionID) - } - } - } - } - - s.logger.Info().Msg("collection catchup done") - return nil -} - -// findMissingCollections scans block heights from last known full block up to the latest finalized -// block and returns all missing collection along with the count of incomplete blocks. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findMissingCollections(lastFullBlockHeight uint64) ([]*flow.CollectionGuarantee, int, error) { - // first block to look up collections at - firstBlockHeight := lastFullBlockHeight + 1 - - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return nil, 0, fmt.Errorf("failed to get finalized block: %w", err) - } - // last block to look up collections at - lastBlockHeight := lastFinalizedBlock.Height - - var missingCollections []*flow.CollectionGuarantee - var incompleteBlocksCount int - - for currBlockHeight := firstBlockHeight; currBlockHeight <= lastBlockHeight; currBlockHeight++ { - collections, err := s.findMissingCollectionsAtHeight(currBlockHeight) - if err != nil { - return nil, 0, err - } - - if len(collections) == 0 { - continue - } - - missingCollections = append(missingCollections, collections...) - incompleteBlocksCount += 1 - } - - return missingCollections, incompleteBlocksCount, nil -} - -// findMissingCollectionsAtHeight returns all missing collections for a specific block height. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findMissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) { - block, err := s.blocks.ByHeight(height) - if err != nil { - return nil, fmt.Errorf("failed to retrieve block by height %d: %w", height, err) - } - - var missingCollections []*flow.CollectionGuarantee - for _, guarantee := range block.Payload.Guarantees { - inStorage, err := s.isCollectionInStorage(guarantee.CollectionID) - if err != nil { - return nil, err - } - - if !inStorage { - missingCollections = append(missingCollections, guarantee) - } - } - - return missingCollections, nil -} - -// isCollectionInStorage checks whether the given collection is present in local storage. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) isCollectionInStorage(collectionID flow.Identifier) (bool, error) { - _, err := s.collections.LightByID(collectionID) - if err == nil { - return true, nil - } - - if errors.Is(err, storage.ErrNotFound) { - return false, nil - } - - return false, fmt.Errorf("failed to retrieve collection %s: %w", collectionID.String(), err) -} - -// RequestCollectionsForBlock conditionally requests missing collections for a specific block height, -// skipping requests if the block is already below the known full block height. -func (s *CollectionSyncer) RequestCollectionsForBlock(height uint64, missingCollections []*flow.CollectionGuarantee) { - // skip requesting collections, if this block is below the last full block height. - // this means that either we have already received these collections, or the block - // may contain unverifiable guarantees (in case this node has just joined the network) - if height <= s.lastFullBlockHeight.Value() { - s.logger.Debug(). - Msg("skipping requesting collections for finalized block as its collections have been already retrieved") - return - } - - s.requestCollections(missingCollections, false) -} - -// requestCollections registers collection download requests in the requester engine, -// optionally forcing immediate dispatch. -func (s *CollectionSyncer) requestCollections(collections []*flow.CollectionGuarantee, immediately bool) { - for _, guarantee := range collections { - guarantors, err := protocol.FindGuarantors(s.state, guarantee) - if err != nil { - // failed to find guarantors for guarantees contained in a finalized block is fatal error - s.logger.Fatal().Err(err).Msgf("could not find guarantors for collection %v", guarantee.CollectionID) - } - s.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) - } - - if immediately { - s.requester.Force() - } -} - -// updateLastFullBlockHeight updates the next highest block height where all previous collections have been indexed. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) updateLastFullBlockHeight() error { - lastFullBlockHeight := s.lastFullBlockHeight.Value() - lastFinalizedBlock, err := s.state.Final().Head() - if err != nil { - return fmt.Errorf("failed to get finalized block: %w", err) - } - - // track the latest contiguous full height - newLastFullBlockHeight, err := s.findLowestBlockHeightWithMissingCollections(lastFullBlockHeight, lastFinalizedBlock.Height) - if err != nil { - return fmt.Errorf("failed to find last full block height: %w", err) - } - - // if more contiguous blocks are now complete, update db - if newLastFullBlockHeight > lastFullBlockHeight { - err := s.lastFullBlockHeight.Set(newLastFullBlockHeight) - if err != nil { - return fmt.Errorf("failed to update last full block height: %w", err) - } - - s.collectionExecutedMetric.UpdateLastFullBlockHeight(newLastFullBlockHeight) - - s.logger.Debug(). - Uint64("last_full_block_height", newLastFullBlockHeight). - Msg("updated last full block height counter") - } - - return nil -} - -// findLowestBlockHeightWithMissingCollections finds the next block height with missing collections, -// returning the latest contiguous height where all collections are present. -// -// No errors are expected during normal operations. -func (s *CollectionSyncer) findLowestBlockHeightWithMissingCollections( - lastKnownFullBlockHeight uint64, - finalizedBlockHeight uint64, -) (uint64, error) { - newLastFullBlockHeight := lastKnownFullBlockHeight - - for currBlockHeight := lastKnownFullBlockHeight + 1; currBlockHeight <= finalizedBlockHeight; currBlockHeight++ { - missingCollections, err := s.findMissingCollectionsAtHeight(currBlockHeight) - if err != nil { - return 0, err - } - - // return when we find the first block with missing collections - if len(missingCollections) > 0 { - return newLastFullBlockHeight, nil - } - - newLastFullBlockHeight = currBlockHeight - } - - return newLastFullBlockHeight, nil -} - -// OnCollectionDownloaded indexes and persists a downloaded collection. -// This function is a callback intended to be used by the requester engine. -func (s *CollectionSyncer) OnCollectionDownloaded(id flow.Identifier, entity flow.Entity) { - err := s.pendingCollectionsHandler.Process(id, entity) - if err != nil { - // this is an unexpected error condition. The only expected error returned from Process - // is for an unexpected type. since OnCollectionDownloaded is called from the requester engine, - // which is configured to only process collections, any error returned here indicates - // a bug or state corruption. - s.logger.Fatal().Err(err).Msg("failed to process pending collections") - return - } -} diff --git a/engine/access/ingestion2/engine.go b/engine/access/ingestion2/engine.go index 3c8e42e2fde..ad2fbc449d4 100644 --- a/engine/access/ingestion2/engine.go +++ b/engine/access/ingestion2/engine.go @@ -16,6 +16,7 @@ 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" @@ -35,7 +36,7 @@ type Engine struct { log zerolog.Logger finalizedBlockProcessor *FinalizedBlockProcessor - collectionSyncer *CollectionSyncer + collectionSyncer *collections.Syncer messageHandler *engine.MessageHandler executionReceiptsQueue *engine.FifoMessageStore @@ -49,7 +50,7 @@ func New( log zerolog.Logger, net network.EngineRegistry, finalizedBlockProcessor *FinalizedBlockProcessor, - collectionSyncer *CollectionSyncer, + collectionSyncer *collections.Syncer, receipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, ) (*Engine, error) { @@ -85,7 +86,7 @@ func New( builder := component.NewComponentManagerBuilder(). AddWorker(e.messageHandlerLoop). AddWorker(e.finalizedBlockProcessor.StartWorkerLoop). - AddWorker(e.collectionSyncer.StartWorkerLoop) + AddWorker(e.collectionSyncer.WorkerLoop) e.ComponentManager = builder.Build() // engine gets execution receipts from channels.ReceiveReceipts channel diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index c7813d7b028..a74080d2a53 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -2,7 +2,6 @@ package ingestion2 import ( "context" - "math/rand" "os" "sync" "testing" @@ -15,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/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 +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, *CollectionSyncer) { +func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *collections.Syncer) { processedHeightInitializer := store.NewConsumerProgress(s.db, module.ConsumeProgressIngestionEngineBlockHeight) lastFullBlockHeight, err := store.NewConsumerProgress(s.db, module.ConsumeProgressLastFullBlockHeight).Initialize(s.finalizedBlock.Height) @@ -209,10 +209,9 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeight) require.NoError(s.T(), err) - syncer, err := NewCollectionSyncer( + indexer, err := collections.NewIndexer( s.log, s.collectionExecutedMetric, - module.Requester(s.request), s.proto.state, s.blocks, s.collections, @@ -222,6 +221,17 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, ) require.NoError(s.T(), err) + syncer := collections.NewSyncer( + s.log, + s.request, + s.proto.state, + s.collections, + s.lastFullBlockHeight, + indexer, + nil, + ) + require.NoError(s.T(), err) + blockProcessor, err := NewFinalizedBlockProcessor( s.log, s.proto.state, @@ -329,6 +339,7 @@ func (s *Suite) TestOnFinalizedBlockSingle() { wg.Done() }).Once() } + s.request.On("Force").Return().Once() // process the block through the finalized callback eng.OnFinalizedBlock(&hotstuffBlock) @@ -395,6 +406,8 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { 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() } @@ -417,27 +430,6 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { s.results.AssertNumberOfCalls(s.T(), "Index", expectedIndexCalls) } -// TestOnCollection checks that when a Collection is received, it is persisted -func (s *Suite) TestOnCollection() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - s.initEngineAndSyncer(irrecoverableCtx) - - collection := unittest.CollectionFixture(5) - light := collection.Light() - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", mock.Anything, &collection).Return(light, nil).Once() - - // Create a lock context for indexing - err := unittest.WithLock(s.T(), s.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { - return indexer.IndexCollection(lctx, &collection, s.collections, s.log, s.collectionExecutedMetric) - }) - require.NoError(s.T(), err) - - // check that the collection was stored and indexed - s.collections.AssertExpectations(s.T()) -} - // TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed func (s *Suite) TestExecutionReceiptsAreIndexed() { irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) @@ -487,335 +479,3 @@ func (s *Suite) TestExecutionReceiptsAreIndexed() { s.results.AssertExpectations(s.T()) s.receipts.AssertExpectations(s.T()) } - -// TestOnCollectionDuplicate checks that when a duplicate collection is received, the node doesn't -// crash but just ignores its transactions. -func (s *Suite) TestOnCollectionDuplicate() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - s.initEngineAndSyncer(irrecoverableCtx) - collection := unittest.CollectionFixture(5) - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", mock.Anything, &collection).Return(nil, storage.ErrAlreadyExists).Once() - - // Create a lock context for indexing - err := unittest.WithLock(s.T(), s.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { - return indexer.IndexCollection(lctx, &collection, s.collections, s.log, s.collectionExecutedMetric) - }) - require.Error(s.T(), err) - require.ErrorIs(s.T(), err, storage.ErrAlreadyExists) - - // check that the collection was stored and indexed - s.collections.AssertExpectations(s.T()) -} - -// TestRequestMissingCollections tests that the all missing collections are requested on the call to requestMissingCollections -func (s *Suite) TestRequestMissingCollections() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - _, syncer := s.initEngineAndSyncer(irrecoverableCtx) - - blkCnt := 3 - startHeight := uint64(1000) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - - // generate the test blocks and collections - var collIDs []flow.Identifier - refBlockID := unittest.IdentifierFixture() - for i := 0; i < blkCnt; i++ { - block := unittest.BlockFixture( - // some blocks may not be present hence add a gap - unittest.Block.WithHeight(startHeight+uint64(i)), - unittest.Block.WithPayload(unittest.PayloadFixture( - unittest.WithGuarantees(unittest.CollectionGuaranteesFixture(4, unittest.WithCollRef(refBlockID))...)), - )) - s.blockMap[block.Height] = block - s.finalizedBlock = block.ToHeader() - - for _, c := range block.Payload.Guarantees { - collIDs = append(collIDs, c.CollectionID) - c.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) - c.SignerIndices = indices - } - } - - // consider collections are missing for all blocks - err := s.lastFullBlockHeight.Set(startHeight - 1) - s.Require().NoError(err) - - // consider the last test block as the head - - // p is the probability of not receiving the collection before the next poll and it - // helps simulate the slow trickle of the requested collections being received - var p float32 - - // rcvdColl is the map simulating the collection storage key-values - rcvdColl := make(map[flow.Identifier]struct{}) - - // for the first lookup call for each collection, it will be reported as missing from db - // for the subsequent calls, it will be reported as present with the probability p - s.collections.On("LightByID", mock.Anything).Return( - func(cID flow.Identifier) *flow.LightCollection { - return nil // the actual collection object return is never really read - }, - func(cID flow.Identifier) error { - if _, ok := rcvdColl[cID]; ok { - return nil - } - if rand.Float32() >= p { - rcvdColl[cID] = struct{}{} - } - return storage.ErrNotFound - }). - // simulate some db i/o contention - After(time.Millisecond * time.Duration(rand.Intn(5))) - - // setup the requester engine mock - // entityByID should be called once per collection - for _, c := range collIDs { - s.request.On("EntityByID", c, mock.Anything).Return() - } - // force should be called once - s.request.On("Force").Return() - - cluster := protocol.NewCluster(s.T()) - cluster.On("Members").Return(clusterCommittee, nil) - epoch := protocol.NewCommittedEpoch(s.T()) - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs := protocol.NewEpochQuery(s.T()) - epochs.On("Current").Return(epoch, nil) - snap := protocol.NewSnapshot(s.T()) - snap.On("Epochs").Return(epochs) - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - assertExpectations := func() { - s.request.AssertExpectations(s.T()) - s.collections.AssertExpectations(s.T()) - s.proto.snapshot.AssertExpectations(s.T()) - s.blocks.AssertExpectations(s.T()) - } - - // test 1 - collections are not received before timeout - s.Run("timeout before all missing collections are received", func() { - - // simulate that collection are never received - p = 1 - - // timeout after 3 db polls - ctx, cancel := context.WithTimeout(context.Background(), 100*collectionCatchupDBPollInterval) - defer cancel() - - err := syncer.requestMissingCollectionsBlocking(ctx) - - require.Error(s.T(), err) - require.Contains(s.T(), err.Error(), "context deadline exceeded") - - assertExpectations() - }) - // test 2 - all collections are eventually received before the deadline - s.Run("all missing collections are received", func() { - - // 90% of the time, collections are reported as not received when the collection storage is queried - p = 0.9 - - ctx, cancel := context.WithTimeout(context.Background(), collectionCatchupTimeout) - defer cancel() - - err := syncer.requestMissingCollectionsBlocking(ctx) - - require.NoError(s.T(), err) - require.Len(s.T(), rcvdColl, len(collIDs)) - - assertExpectations() - }) -} - -// TestProcessBackgroundCalls tests that updateLastFullBlockHeight and checkMissingCollections -// function calls keep the FullBlockIndex up-to-date and request collections if blocks with missing -// collections exceed the threshold. -func (s *Suite) TestProcessBackgroundCalls() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - _, syncer := s.initEngineAndSyncer(irrecoverableCtx) - - blkCnt := 3 - collPerBlk := 10 - startHeight := uint64(1000) - blocks := make([]*flow.Block, blkCnt) - collMap := make(map[flow.Identifier]*flow.LightCollection, blkCnt*collPerBlk) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - - refBlockID := unittest.IdentifierFixture() - // generate the test blocks, cgs and collections - for i := 0; i < blkCnt; i++ { - guarantees := make([]*flow.CollectionGuarantee, collPerBlk) - for j := 0; j < collPerBlk; j++ { - coll := unittest.CollectionFixture(2).Light() - collMap[coll.ID()] = coll - cg := unittest.CollectionGuaranteeFixture(func(cg *flow.CollectionGuarantee) { - cg.CollectionID = coll.ID() - cg.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) - cg.SignerIndices = indices - guarantees[j] = cg - } - block := unittest.BlockFixture( - unittest.Block.WithHeight(startHeight+uint64(i)), - unittest.Block.WithPayload(unittest.PayloadFixture(unittest.WithGuarantees(guarantees...))), - ) - s.blockMap[block.Height] = block - blocks[i] = block - s.finalizedBlock = block.ToHeader() - } - - finalizedHeight := s.finalizedBlock.Height - - cluster := protocol.NewCluster(s.T()) - cluster.On("Members").Return(clusterCommittee, nil) - epoch := protocol.NewCommittedEpoch(s.T()) - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs := protocol.NewEpochQuery(s.T()) - epochs.On("Current").Return(epoch, nil) - snap := protocol.NewSnapshot(s.T()) - snap.On("Epochs").Return(epochs) - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - // blkMissingColl controls which collections are reported as missing by the collections storage mock - blkMissingColl := make([]bool, blkCnt) - for i := 0; i < blkCnt; i++ { - blkMissingColl[i] = false - for _, cg := range blocks[i].Payload.Guarantees { - j := i - s.collections.On("LightByID", cg.CollectionID).Return( - func(cID flow.Identifier) *flow.LightCollection { - return collMap[cID] - }, - func(cID flow.Identifier) error { - if blkMissingColl[j] { - return storage.ErrNotFound - } - return nil - }) - } - } - - rootBlk := blocks[0] - - // root block is the last complete block - err := s.lastFullBlockHeight.Set(rootBlk.Height) - s.Require().NoError(err) - - s.Run("missing collections are requested when count exceeds defaultMissingCollsForBlockThreshold", func() { - // lower the block threshold to request missing collections - defaultMissingCollsForBlockThreshold = 2 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - // setup receive engine expectations - for _, cg := range blocks[i].Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() - } - } - - err := syncer.requestMissingCollections() - s.Require().NoError(err) - - // assert that missing collections are requested - s.request.AssertExpectations(s.T()) - - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // no new call to UpdateLastFullBlockHeight should be made - }) - - s.Run("missing collections are requested when count exceeds defaultMissingCollsForAgeThreshold", func() { - // lower the height threshold to request missing collections - defaultMissingCollsForAgeThreshold = 1 - - // raise the block threshold to ensure it does not trigger missing collection request - defaultMissingCollsForBlockThreshold = blkCnt + 1 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - // setup receive engine expectations - for _, cg := range blocks[i].Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() - } - } - - err := syncer.requestMissingCollections() - s.Require().NoError(err) - - // assert that missing collections are requested - s.request.AssertExpectations(s.T()) - - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made - }) - - s.Run("missing collections are not requested if defaultMissingCollsForBlockThreshold not reached", func() { - // raise the thresholds to avoid requesting missing collections - defaultMissingCollsForAgeThreshold = 3 - defaultMissingCollsForBlockThreshold = 3 - - // mark all blocks beyond the root block as incomplete - for i := 1; i < blkCnt; i++ { - blkMissingColl[i] = true - } - - err := syncer.requestMissingCollections() - s.Require().NoError(err) - - // assert that missing collections are not requested even though there are collections missing - s.request.AssertExpectations(s.T()) - - // last full blk index is not advanced - s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made - }) - - // create new block - height := blocks[blkCnt-1].Height + 1 - finalizedBlk := unittest.BlockFixture( - unittest.Block.WithHeight(height), - ) - s.blockMap[height] = finalizedBlk - - finalizedHeight = finalizedBlk.Height - s.finalizedBlock = finalizedBlk.ToHeader() - - blockBeforeFinalized := blocks[blkCnt-1] - - s.Run("full block height index is advanced if newer full blocks are discovered", func() { - // set lastFullBlockHeight to block - err = s.lastFullBlockHeight.Set(blockBeforeFinalized.Height) - s.Require().NoError(err) - - err = syncer.updateLastFullBlockHeight() - s.Require().NoError(err) - s.Require().Equal(finalizedHeight, s.lastFullBlockHeight.Value()) - s.Require().NoError(err) - - s.blocks.AssertExpectations(s.T()) - }) - - s.Run("full block height index is not advanced beyond finalized blocks", func() { - err = syncer.updateLastFullBlockHeight() - s.Require().NoError(err) - - s.Require().Equal(finalizedHeight, s.lastFullBlockHeight.Value()) - s.blocks.AssertExpectations(s.T()) - }) -} diff --git a/engine/access/ingestion2/finalized_block_processor.go b/engine/access/ingestion2/finalized_block_processor.go index 78511dd9803..1bb9da66cd7 100644 --- a/engine/access/ingestion2/finalized_block_processor.go +++ b/engine/access/ingestion2/finalized_block_processor.go @@ -6,6 +6,7 @@ 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" @@ -49,7 +50,7 @@ type FinalizedBlockProcessor struct { executionResults storage.ExecutionResults - collectionSyncer *CollectionSyncer + collectionSyncer *collections.Syncer collectionExecutedMetric module.CollectionExecutedMetric } @@ -63,7 +64,7 @@ func NewFinalizedBlockProcessor( blocks storage.Blocks, executionResults storage.ExecutionResults, finalizedProcessedHeight storage.ConsumerProgressInitializer, - syncer *CollectionSyncer, + syncer *collections.Syncer, collectionExecutedMetric module.CollectionExecutedMetric, ) (*FinalizedBlockProcessor, error) { reader := jobqueue.NewFinalizedBlockReader(state, blocks) diff --git a/engine/access/rpc/backend/backend_stream_blocks_test.go b/engine/access/rpc/backend/backend_stream_blocks_test.go index 3bf259dc7fa..ff574e6fbda 100644 --- a/engine/access/rpc/backend/backend_stream_blocks_test.go +++ b/engine/access/rpc/backend/backend_stream_blocks_test.go @@ -300,7 +300,7 @@ func (s *BackendBlocksSuite) setupBlockStatusesForTestCases(baseTests []testType // Parameters: // - blockStatus: The status of the blocks being tracked (Sealed or Finalized). // - highestHeader: The highest header that the block tracker should report. -func (s *BackendBlocksSuite) setupBlockTrackerMock(blockStatus flow.BlockStatus, highestHeader *flow.Header) { +func (s *BackendBlocksSuite) setupBlockTrackerMock(highestHeader *flow.Header) { s.snapshot.On("Head").Unset() s.snapshot.On("Head").Return(highestHeader, nil) err := s.blockTracker.ProcessOnFinalizedBlock() @@ -377,7 +377,7 @@ func (s *BackendBlocksSuite) subscribe( // add "backfill" block - blocks that are already in the database before the test starts // this simulates a subscription on a past block if test.highestBackfill > 0 { - s.setupBlockTrackerMock(test.blockStatus, s.blocksArray[test.highestBackfill].ToHeader()) + s.setupBlockTrackerMock(s.blocksArray[test.highestBackfill].ToHeader()) } subCtx, subCancel := context.WithCancel(context.Background()) @@ -395,7 +395,7 @@ func (s *BackendBlocksSuite) subscribe( // simulate new block received. // all blocks with index <= highestBackfill were already received if i > test.highestBackfill { - s.setupBlockTrackerMock(test.blockStatus, b.ToHeader()) + s.setupBlockTrackerMock(b.ToHeader()) broadcaster.Publish() } diff --git a/engine/access/rpc/backend/script_executor_test.go b/engine/access/rpc/backend/script_executor_test.go index 5f56a13a79e..9d127da6810 100644 --- a/engine/access/rpc/backend/script_executor_test.go +++ b/engine/access/rpc/backend/script_executor_test.go @@ -142,7 +142,8 @@ func (s *ScriptExecutorSuite) SetupTest() { nil, s.chain.ChainID(), derivedChainData, - module.CollectionExecutedMetric(metrics.NewNoopCollector()), + nil, + metrics.NewNoopCollector(), lockManager, ) diff --git a/module/counters/interfaces.go b/module/counters/interfaces.go new file mode 100644 index 00000000000..4b4870f86bf --- /dev/null +++ b/module/counters/interfaces.go @@ -0,0 +1,7 @@ +package counters + +// Reader is an interface for reading the value of a counter. +type Reader interface { + // Value returns the current value of the counter. + Value() uint64 +} diff --git a/module/counters/monotonous_counter.go b/module/counters/monotonous_counter.go index a96498d8327..21ae37c56a6 100644 --- a/module/counters/monotonous_counter.go +++ b/module/counters/monotonous_counter.go @@ -10,6 +10,8 @@ type StrictMonotonicCounter struct { atomicCounter uint64 } +var _ Reader = (*StrictMonotonicCounter)(nil) + // NewMonotonicCounter creates new counter with initial value func NewMonotonicCounter(initialValue uint64) StrictMonotonicCounter { return StrictMonotonicCounter{ diff --git a/module/counters/persistent_strict_monotonic_counter.go b/module/counters/persistent_strict_monotonic_counter.go index 95c2e3e8aae..1afed41b892 100644 --- a/module/counters/persistent_strict_monotonic_counter.go +++ b/module/counters/persistent_strict_monotonic_counter.go @@ -20,6 +20,8 @@ type PersistentStrictMonotonicCounter struct { counter StrictMonotonicCounter } +var _ Reader = (*PersistentStrictMonotonicCounter)(nil) + // NewPersistentStrictMonotonicCounter creates a new PersistentStrictMonotonicCounter. // The consumer progress and associated db entry must not be accessed outside of calls to the returned object, // otherwise the state may become inconsistent. diff --git a/module/execution/scripts_test.go b/module/execution/scripts_test.go index db8c1bd84c3..af0b3749642 100644 --- a/module/execution/scripts_test.go +++ b/module/execution/scripts_test.go @@ -192,6 +192,7 @@ func (s *scriptTestSuite) SetupTest() { flow.Testnet, derivedChainData, nil, + nil, lockManager, ) diff --git a/module/executiondatasync/execution_data/cache.go b/module/executiondatasync/execution_data/cache.go new file mode 100644 index 00000000000..15488d66f45 --- /dev/null +++ b/module/executiondatasync/execution_data/cache.go @@ -0,0 +1,43 @@ +package execution_data + +import ( + "context" + + "github.com/onflow/flow-go/model/flow" +) + +// ExecutionDataCache provides a read-through cache for execution data. +// All methods are safe for concurrent use. +type ExecutionDataCache interface { + // ByID returns the execution data for the given ExecutionDataID. + // + // Expected error returns during normal operation: + // - [BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore + // - [MalformedDataError]: if some level of the blob tree cannot be properly deserialized + // - [BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size + ByID(ctx context.Context, executionDataID flow.Identifier) (*BlockExecutionDataEntity, error) + + // ByBlockID returns the execution data for the given block ID. + // + // Expected error returns during normal operation: + // - [storage.ErrNotFound]: if a seal or execution result is not available for the block + // - [BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore + // - [MalformedDataError]: if some level of the blob tree cannot be properly deserialized + // - [BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size + ByBlockID(ctx context.Context, blockID flow.Identifier) (*BlockExecutionDataEntity, error) + + // ByHeight returns the execution data for the given block height. + // + // Expected error returns during normal operation: + // - [storage.ErrNotFound]: if a seal or execution result is not available for the block + // - [BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore + // - [MalformedDataError]: if some level of the blob tree cannot be properly deserialized + // - [BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size + ByHeight(ctx context.Context, height uint64) (*BlockExecutionDataEntity, error) + + // LookupID returns the ExecutionDataID for the given block ID. + // + // Expected error returns during normal operation: + // - [storage.ErrNotFound]: if a seal or execution result is not available for the block + LookupID(blockID flow.Identifier) (flow.Identifier, error) +} diff --git a/module/executiondatasync/execution_data/cache/cache.go b/module/executiondatasync/execution_data/cache/cache.go index 5b541855583..90887ad7a6c 100644 --- a/module/executiondatasync/execution_data/cache/cache.go +++ b/module/executiondatasync/execution_data/cache/cache.go @@ -39,10 +39,10 @@ func NewExecutionDataCache( // ByID returns the execution data for the given ExecutionDataID. // -// Expected errors during normal operations: -// - BlobNotFoundError if some CID in the blob tree could not be found from the blobstore -// - MalformedDataError if some level of the blob tree cannot be properly deserialized -// - BlobSizeLimitExceededError if some blob in the blob tree exceeds the maximum allowed size +// Expected error returns during normal operation: +// - [execution_data.BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore +// - [execution_data.MalformedDataError]: if some level of the blob tree cannot be properly deserialized +// - [execution_data.BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size func (c *ExecutionDataCache) ByID(ctx context.Context, executionDataID flow.Identifier) (*execution_data.BlockExecutionDataEntity, error) { execData, err := c.backend.Get(ctx, executionDataID) if err != nil { @@ -54,11 +54,11 @@ func (c *ExecutionDataCache) ByID(ctx context.Context, executionDataID flow.Iden // ByBlockID returns the execution data for the given block ID. // -// Expected errors during normal operations: -// - storage.ErrNotFound if a seal or execution result is not available for the block -// - BlobNotFoundError if some CID in the blob tree could not be found from the blobstore -// - MalformedDataError if some level of the blob tree cannot be properly deserialized -// - BlobSizeLimitExceededError if some blob in the blob tree exceeds the maximum allowed size +// Expected error returns during normal operation: +// - [storage.ErrNotFound]: if a seal or execution result is not available for the block +// - [execution_data.BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore +// - [execution_data.MalformedDataError]: if some level of the blob tree cannot be properly deserialized +// - [execution_data.BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size func (c *ExecutionDataCache) ByBlockID(ctx context.Context, blockID flow.Identifier) (*execution_data.BlockExecutionDataEntity, error) { if execData, ok := c.cache.Get(blockID); ok { return execData, nil @@ -83,11 +83,11 @@ func (c *ExecutionDataCache) ByBlockID(ctx context.Context, blockID flow.Identif // ByHeight returns the execution data for the given block height. // -// Expected errors during normal operations: -// - storage.ErrNotFound if a seal or execution result is not available for the block -// - BlobNotFoundError if some CID in the blob tree could not be found from the blobstore -// - MalformedDataError if some level of the blob tree cannot be properly deserialized -// - BlobSizeLimitExceededError if some blob in the blob tree exceeds the maximum allowed size +// Expected error returns during normal operation: +// - [storage.ErrNotFound]: if the block, seal, or execution result is not available for the height. +// - [execution_data.BlobNotFoundError]: if some CID in the blob tree could not be found from the blobstore +// - [execution_data.MalformedDataError]: if some level of the blob tree cannot be properly deserialized +// - [execution_data.BlobSizeLimitExceededError]: if some blob in the blob tree exceeds the maximum allowed size func (c *ExecutionDataCache) ByHeight(ctx context.Context, height uint64) (*execution_data.BlockExecutionDataEntity, error) { blockID, err := c.headers.BlockIDByHeight(height) if err != nil { @@ -99,8 +99,8 @@ func (c *ExecutionDataCache) ByHeight(ctx context.Context, height uint64) (*exec // LookupID returns the ExecutionDataID for the given block ID. // -// Expected errors during normal operations: -// - storage.ErrNotFound if a seal or execution result is not available for the block +// Expected error returns during normal operation: +// - [storage.ErrNotFound]: if a seal or execution result is not available for the block func (c *ExecutionDataCache) LookupID(blockID flow.Identifier) (flow.Identifier, error) { seal, err := c.seals.FinalizedSealForBlock(blockID) if err != nil { diff --git a/module/executiondatasync/execution_data/mock/execution_data_cache.go b/module/executiondatasync/execution_data/mock/execution_data_cache.go new file mode 100644 index 00000000000..3475518f986 --- /dev/null +++ b/module/executiondatasync/execution_data/mock/execution_data_cache.go @@ -0,0 +1,151 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +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" +) + +// ExecutionDataCache is an autogenerated mock type for the ExecutionDataCache type +type ExecutionDataCache struct { + mock.Mock +} + +// ByBlockID provides a mock function with given fields: ctx, blockID +func (_m *ExecutionDataCache) ByBlockID(ctx context.Context, blockID flow.Identifier) (*execution_data.BlockExecutionDataEntity, error) { + ret := _m.Called(ctx, blockID) + + if len(ret) == 0 { + panic("no return value specified for ByBlockID") + } + + var r0 *execution_data.BlockExecutionDataEntity + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, flow.Identifier) (*execution_data.BlockExecutionDataEntity, error)); ok { + return rf(ctx, blockID) + } + if rf, ok := ret.Get(0).(func(context.Context, flow.Identifier) *execution_data.BlockExecutionDataEntity); ok { + r0 = rf(ctx, blockID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*execution_data.BlockExecutionDataEntity) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, flow.Identifier) error); ok { + r1 = rf(ctx, blockID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// ByHeight provides a mock function with given fields: ctx, height +func (_m *ExecutionDataCache) ByHeight(ctx context.Context, height uint64) (*execution_data.BlockExecutionDataEntity, error) { + ret := _m.Called(ctx, height) + + if len(ret) == 0 { + panic("no return value specified for ByHeight") + } + + var r0 *execution_data.BlockExecutionDataEntity + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, uint64) (*execution_data.BlockExecutionDataEntity, error)); ok { + return rf(ctx, height) + } + if rf, ok := ret.Get(0).(func(context.Context, uint64) *execution_data.BlockExecutionDataEntity); ok { + r0 = rf(ctx, height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*execution_data.BlockExecutionDataEntity) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, uint64) error); ok { + r1 = rf(ctx, height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// ByID provides a mock function with given fields: ctx, executionDataID +func (_m *ExecutionDataCache) ByID(ctx context.Context, executionDataID flow.Identifier) (*execution_data.BlockExecutionDataEntity, error) { + ret := _m.Called(ctx, executionDataID) + + if len(ret) == 0 { + panic("no return value specified for ByID") + } + + var r0 *execution_data.BlockExecutionDataEntity + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, flow.Identifier) (*execution_data.BlockExecutionDataEntity, error)); ok { + return rf(ctx, executionDataID) + } + if rf, ok := ret.Get(0).(func(context.Context, flow.Identifier) *execution_data.BlockExecutionDataEntity); ok { + r0 = rf(ctx, executionDataID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*execution_data.BlockExecutionDataEntity) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, flow.Identifier) error); ok { + r1 = rf(ctx, executionDataID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// LookupID provides a mock function with given fields: blockID +func (_m *ExecutionDataCache) LookupID(blockID flow.Identifier) (flow.Identifier, error) { + ret := _m.Called(blockID) + + if len(ret) == 0 { + panic("no return value specified for LookupID") + } + + var r0 flow.Identifier + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (flow.Identifier, error)); ok { + return rf(blockID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) flow.Identifier); ok { + r0 = rf(blockID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.Identifier) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(blockID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewExecutionDataCache creates a new instance of ExecutionDataCache. 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 NewExecutionDataCache(t interface { + mock.TestingT + Cleanup(func()) +}) *ExecutionDataCache { + mock := &ExecutionDataCache{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index 60dc2d5bdf3..6d3704a3176 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -11,6 +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/fvm/blueprints" "github.com/onflow/flow-go/fvm/storage/derived" "github.com/onflow/flow-go/fvm/systemcontracts" @@ -30,6 +31,7 @@ type IndexerCore struct { fvmEnv templates.Environment metrics module.ExecutionStateIndexerMetrics collectionExecutedMetric module.CollectionExecutedMetric + collectionIndexer collections.CollectionIndexer registers storage.RegisterIndex headers storage.Headers @@ -61,6 +63,7 @@ func New( scheduledTransactions storage.ScheduledTransactions, chainID flow.ChainID, derivedChainData *derived.DerivedChainData, + collectionIndexer collections.CollectionIndexer, collectionExecutedMetric module.CollectionExecutedMetric, lockManager lockctx.Manager, ) *IndexerCore { @@ -90,6 +93,7 @@ func New( serviceAddress: chainID.Chain().ServiceAddress(), derivedChainData: derivedChainData, + collectionIndexer: collectionIndexer, collectionExecutedMetric: collectionExecutedMetric, lockManager: lockManager, } @@ -225,12 +229,9 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti // than the latest indexed block. Calling the collection handler with a collection that // has already been indexed is a noop. indexedCount := 0 - if len(data.ChunkExecutionDatas) > 0 { + if len(data.ChunkExecutionDatas) > 1 { for _, chunk := range data.ChunkExecutionDatas[0 : len(data.ChunkExecutionDatas)-1] { - err := c.indexCollection(chunk.Collection) - if err != nil { - return err - } + c.collectionIndexer.OnCollectionReceived(chunk.Collection) indexedCount++ } } @@ -450,43 +451,3 @@ func (c *IndexerCore) indexRegisters(registers map[ledger.Path]*ledger.Payload, return c.registers.Store(regEntries, height) } - -func (c *IndexerCore) indexCollection(collection *flow.Collection) error { - lctx := c.lockManager.NewContext() - defer lctx.Release() - err := lctx.AcquireLock(storage.LockInsertCollection) - if err != nil { - return fmt.Errorf("could not acquire lock for indexing collections: %w", err) - } - - err = IndexCollection(lctx, collection, c.collections, c.log, c.collectionExecutedMetric) - if err != nil { - return fmt.Errorf("could not handle collection") - } - return nil -} - -// IndexCollection handles the response of the collection request made earlier when a block was received. -// -// No error returns are expected during normal operations. -func IndexCollection( - lctx lockctx.Proof, - collection *flow.Collection, - collections storage.Collections, - logger zerolog.Logger, - collectionExecutedMetric module.CollectionExecutedMetric, -) error { - - // FIX: we can't index guarantees here, as we might have more than one block - // with the same collection as long as it is not finalized - - // store the collection, including constituent transactions, and index transactionID -> collectionID - light, err := collections.StoreAndIndexByTransaction(lctx, collection) - if err != nil { - return err - } - - collectionExecutedMetric.CollectionFinalized(light) - collectionExecutedMetric.CollectionExecuted(light) - return nil -} diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index 24ae7424c48..7d7fb0b8f63 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -13,11 +13,11 @@ import ( mocks "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + collectionsmock "github.com/onflow/flow-go/engine/access/ingestion/collections/mock" rpcconvert "github.com/onflow/flow-go/engine/common/rpc/convert" "github.com/onflow/flow-go/fvm/storage/derived" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/ledger" - "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/executiondatasync/execution_data" @@ -44,6 +44,7 @@ type indexCoreTest struct { results *storagemock.LightTransactionResults headers *storagemock.Headers scheduledTransactions *storagemock.ScheduledTransactions + collectionIndexer *collectionsmock.CollectionIndexer ctx context.Context blocks []*flow.Block data *execution_data.BlockExecutionDataEntity @@ -69,6 +70,7 @@ func newIndexCoreTest( collections: storagemock.NewCollections(t), transactions: storagemock.NewTransactions(t), scheduledTransactions: storagemock.NewScheduledTransactions(t), + collectionIndexer: collectionsmock.NewCollectionIndexer(t), blocks: blocks, ctx: context.Background(), data: exeData, @@ -125,36 +127,6 @@ func (i *indexCoreTest) useDefaultHeights() *indexCoreTest { return i } -func (i *indexCoreTest) setStoreRegisters(f func(t *testing.T, entries flow.RegisterEntries, height uint64) error) *indexCoreTest { - i.registers. - On("Store", mock.AnythingOfType("flow.RegisterEntries"), mock.AnythingOfType("uint64")). - Return(func(entries flow.RegisterEntries, height uint64) error { - return f(i.t, entries, height) - }).Once() - return i -} - -func (i *indexCoreTest) setStoreEvents(f func(*testing.T, flow.Identifier, []flow.EventsList) error) *indexCoreTest { - i.events. - On("BatchStore", mock.AnythingOfType("flow.Identifier"), mock.AnythingOfType("[]flow.EventsList"), mock.Anything). - Return(func(blockID flow.Identifier, events []flow.EventsList, batch storage.ReaderBatchWriter) error { - require.NotNil(i.t, batch) - return f(i.t, blockID, events) - }) - return i -} - -func (i *indexCoreTest) setStoreTransactionResults(f func(*testing.T, flow.Identifier, []flow.LightTransactionResult) error) *indexCoreTest { - i.results. - On("BatchStore", mock.Anything, mock.Anything, mock.AnythingOfType("flow.Identifier"), mock.AnythingOfType("[]flow.LightTransactionResult")). - Return(func(lctx lockctx.Proof, batch storage.ReaderBatchWriter, blockID flow.Identifier, results []flow.LightTransactionResult) error { - require.True(i.t, lctx.HoldsLock(storage.LockInsertLightTransactionResult)) - require.NotNil(i.t, batch) - return f(i.t, blockID, results) - }) - return i -} - func (i *indexCoreTest) setGetRegisters(f func(t *testing.T, ID flow.RegisterID, height uint64) (flow.RegisterValue, error)) *indexCoreTest { i.registers. On("Get", mock.AnythingOfType("flow.RegisterID"), mock.AnythingOfType("uint64")). @@ -229,6 +201,7 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.scheduledTransactions, i.g.ChainID(), derivedChainData, + i.collectionIndexer, collectionExecutedMetric, lockManager, ) @@ -278,7 +251,7 @@ func TestExecutionState_IndexBlockData(t *testing.T) { }). Return(nil) for _, collection := range tf.ExpectedCollections { - test.collections.On("StoreAndIndexByTransaction", mock.Anything, collection).Return(&flow.LightCollection{}, nil) + test.collectionIndexer.On("OnCollectionReceived", collection).Once() } for txID, scheduledTxID := range tf.ExpectedScheduledTransactions { test.scheduledTransactions.On("BatchIndex", mock.Anything, blockID, txID, scheduledTxID, mock.Anything). @@ -362,26 +335,6 @@ func newBlockHeadersStorage(blocks []*flow.Block) storage.Headers { return synctest.MockBlockHeaderStorage(synctest.WithByID(blocksByID)) } -// trieRegistersPayloadComparer checks that trie payloads and register payloads are same, used for testing. -func trieRegistersPayloadComparer(t *testing.T, triePayloads []*ledger.Payload, registerPayloads flow.RegisterEntries) { - assert.Equal(t, len(triePayloads), len(registerPayloads.Values()), "registers length should equal") - - // crate a lookup map that matches flow register ID to index in the payloads slice - payloadRegID := make(map[flow.RegisterID]int) - for i, p := range triePayloads { - k, _ := p.Key() - regKey, _ := convert.LedgerKeyToRegisterID(k) - payloadRegID[regKey] = i - } - - for _, entry := range registerPayloads { - index, ok := payloadRegID[entry.Key] - assert.True(t, ok, fmt.Sprintf("register entry not found for key %s", entry.Key.String())) - val := triePayloads[index].Value() - assert.True(t, val.Equals(entry.Value), fmt.Sprintf("payload values not same %s - %s", val, entry.Value)) - } -} - func TestIndexerIntegration_StoreAndGet(t *testing.T) { lockManager := storage.NewTestingLockManager() regOwnerAddress := unittest.RandomAddressFixture() @@ -416,6 +369,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, + collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -451,6 +405,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, + collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -479,6 +434,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, + collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -524,6 +480,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, + collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) From 8498ec6f111be9c3b3eb138f12253937a5764f25 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 09:18:35 -0800 Subject: [PATCH 02/16] improve docs and cleanup --- .../node_builder/access_node_builder.go | 7 +-- cmd/observer/node_builder/observer_builder.go | 1 - engine/access/access_test.go | 3 - .../access/ingestion/collections/indexer.go | 39 ++++++------ .../ingestion/collections/indexer_test.go | 2 - engine/access/ingestion/collections/syncer.go | 59 ++++++++++--------- .../collections/syncer_execution_data.go | 8 +-- .../ingestion/collections/syncer_test.go | 2 - engine/access/ingestion/engine_test.go | 1 - engine/access/ingestion2/engine_test.go | 1 - 10 files changed, 53 insertions(+), 70 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 4d5ab84916e..fabc67bd7f4 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -637,14 +637,13 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess // Execution Data cache that uses a blobstore as the backend (instead of a downloader) // This ensures that it simply returns a not found error if the blob doesn't exist // instead of attempting to download it from the network. - executionDataStoreCache := execdatacache.NewExecutionDataCache( + builder.ExecutionDataCache = execdatacache.NewExecutionDataCache( builder.ExecutionDataStore, builder.Storage.Headers, builder.Storage.Seals, builder.Storage.Results, execDataCacheBackend, ) - builder.ExecutionDataCache = executionDataStoreCache return nil }). @@ -2220,7 +2219,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.State, node.Storage.Blocks, notNil(builder.collections), - notNil(builder.transactions), lastFullBlockHeight, node.StorageLockMgr, ) @@ -2235,8 +2233,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { var executionDataSyncer *collections.ExecutionDataSyncer if builder.executionDataSyncEnabled && !builder.executionDataIndexingEnabled { executionDataSyncer = collections.NewExecutionDataSyncer( - node.Logger, - builder.ExecutionDataCache, + notNil(builder.ExecutionDataCache), collectionIndexer, ) } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index cce669f57ee..4b7965b16ba 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1458,7 +1458,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.State, builder.Storage.Blocks, builder.Storage.Collections, - builder.Storage.Transactions, lastFullBlockHeight, builder.StorageLockMgr, ) diff --git a/engine/access/access_test.go b/engine/access/access_test.go index f32e0e5de53..8cf3f8de86c 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -730,7 +730,6 @@ func (suite *Suite) TestGetSealedTransaction() { suite.state, all.Blocks, collections, - transactions, lastFullBlockHeight, suite.lockManager, ) @@ -989,7 +988,6 @@ func (suite *Suite) TestGetTransactionResult() { suite.state, all.Blocks, collections, - transactions, lastFullBlockHeight, suite.lockManager, ) @@ -1265,7 +1263,6 @@ func (suite *Suite) TestExecuteScript() { suite.state, all.Blocks, all.Collections, - all.Transactions, lastFullBlockHeight, suite.lockManager, ) diff --git a/engine/access/ingestion/collections/indexer.go b/engine/access/ingestion/collections/indexer.go index 966fe6a0684..f56d674d6c3 100644 --- a/engine/access/ingestion/collections/indexer.go +++ b/engine/access/ingestion/collections/indexer.go @@ -19,6 +19,14 @@ import ( "github.com/rs/zerolog" ) +const ( + // lastFullBlockRefreshInterval is the interval at which the last full block height is updated. + lastFullBlockRefreshInterval = 1 * time.Second + + // defaultQueueCapacity is the default capacity of the pending collections queue. + defaultQueueCapacity = 10_000 +) + type CollectionIndexer interface { // OnCollectionReceived notifies the collection indexer that a new collection is available to be indexed. // Calling this method multiple times with the same collection is a no-op. @@ -26,7 +34,7 @@ type CollectionIndexer interface { OnCollectionReceived(collection *flow.Collection) // MissingCollectionsAtHeight returns all collections that are not present in storage for a specific - // block height. + // finalized block height. // // Expected error returns during normal operation: // - [storage.ErrNotFound]: if provided block height is not finalized @@ -38,16 +46,8 @@ type CollectionIndexer interface { IsCollectionInStorage(collectionID flow.Identifier) (bool, error) } -const ( - // lastFullBlockRefreshInterval is the interval at which the last full block height is updated. - lastFullBlockRefreshInterval = 1 * time.Second - - // defaultQueueCapacity is the default capacity of the pending collections queue. - defaultQueueCapacity = 10_000 -) - // Indexer stores and indexes collections received from the network. It is designed to be the central -// point for accumulating collection from various subsystems that my receive them from the network. +// point for accumulating collections from various subsystems that my receive them from the network. // For example, collections may be received from execution data sync, the collection syncer, or the // execution state indexer. Depending on the node's configuration, one or more of these subsystems // will feed the indexer with collections. @@ -55,14 +55,13 @@ const ( // The indexer also maintains the last full block height state, which is the highest block height // for which all collections are stored and indexed. type Indexer struct { - log zerolog.Logger - metrics module.CollectionExecutedMetric + log zerolog.Logger + metrics module.CollectionExecutedMetric + lockManager lockctx.Manager state protocol.State blocks storage.Blocks collections storage.Collections - transactions storage.Transactions - lockManager lockctx.Manager lastFullBlockHeight *counters.PersistentStrictMonotonicCounter pendingCollectionsNotifier engine.Notifier @@ -78,7 +77,6 @@ func NewIndexer( state protocol.State, blocks storage.Blocks, collections storage.Collections, - transactions storage.Transactions, lastFullBlockHeight *counters.PersistentStrictMonotonicCounter, lockManager lockctx.Manager, ) (*Indexer, error) { @@ -92,14 +90,13 @@ func NewIndexer( return &Indexer{ log: log.With().Str("component", "collection-indexer").Logger(), metrics: metrics, + lockManager: lockManager, state: state, blocks: blocks, collections: collections, - transactions: transactions, - lockManager: lockManager, + lastFullBlockHeight: lastFullBlockHeight, pendingCollectionsNotifier: engine.NewNotifier(), pendingCollectionsQueue: collectionsQueue, - lastFullBlockHeight: lastFullBlockHeight, }, nil } @@ -121,7 +118,7 @@ func (ci *Indexer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component case <-updateLastFullBlockHeightTicker.C: err := ci.updateLastFullBlockHeight() if err != nil { - ctx.Throw(err) + ctx.Throw(fmt.Errorf("failed to update last full block height: %w", err)) return } @@ -134,7 +131,7 @@ func (ci *Indexer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component collection, ok := v.(*flow.Collection) if !ok { - ctx.Throw(fmt.Errorf("received invalid object. expected *flow.Collection, got: %T", collection)) + ctx.Throw(fmt.Errorf("collection indexer received invalid object. expected *flow.Collection, got: %T", collection)) return } @@ -244,7 +241,7 @@ func (ci *Indexer) updateLastFullBlockHeight() error { } // MissingCollectionsAtHeight returns all collections that are not present in storage for a specific -// block height. +// finalized block height. // // Expected error returns during normal operation: // - [storage.ErrNotFound]: if provided block height is not finalized diff --git a/engine/access/ingestion/collections/indexer_test.go b/engine/access/ingestion/collections/indexer_test.go index 1ada9b0ea69..8df0bf28820 100644 --- a/engine/access/ingestion/collections/indexer_test.go +++ b/engine/access/ingestion/collections/indexer_test.go @@ -71,7 +71,6 @@ func (s *IndexerSuite) createIndexer(t *testing.T) *Indexer { s.state, s.blocks, s.collections, - s.transactions, s.lastFullBlockHeight, s.lockManager, ) @@ -315,7 +314,6 @@ func (s *IndexerSuite) TestWorkerProcessing_ProcessesCollections() { bc.state, bc.all.Blocks, bc.collections, - bc.transactions, lastFullBlockHeight, bc.lockManager, ) diff --git a/engine/access/ingestion/collections/syncer.go b/engine/access/ingestion/collections/syncer.go index 0bafe69f247..57ef0e5fd78 100644 --- a/engine/access/ingestion/collections/syncer.go +++ b/engine/access/ingestion/collections/syncer.go @@ -28,16 +28,16 @@ const ( // if missing collections have been received during the initial collection catchup process. DefaultCollectionCatchupDBPollInterval = 1 * time.Second - // DefaultMissingCollsRequestInterval is the interval at which missing collections are requested - // from the network. + // DefaultMissingCollsRequestInterval is the interval at which the syncer checks missing collections + // and re-requests them from the network if needed. DefaultMissingCollsRequestInterval = 1 * time.Minute // DefaultMissingCollsForBlockThreshold is the threshold number of blocks with missing collections - // beyond which collections should be re-requested this is to prevent spamming the collection - // nodes with request + // beyond which collections should be re-requested. This prevents spamming the collection nodes + // with requests for recent data. DefaultMissingCollsForBlockThreshold = 100 - // DefaultMissingCollsForAgeThreshold is the block height threshold beyond which collections + // DefaultMissingCollsForAgeThreshold is the block height threshold below which collections // should be re-requested, regardless of the number of blocks for which collection are missing. // This is to ensure that if a collection is missing for a long time (in terms of block height) // it is eventually re-requested. @@ -56,12 +56,11 @@ const ( // as there are honest Collection nodes in each cluster, and the node is able to successfully communicate // with them over the networking layer. // -// It is responsible for ensuring the local node has all collections associated with finalized blocks -// starting from the last fully synced height. -// It works by periodically scanning the finalized block range from the last full block height up to -// the latest finalized block height, identifying missing collections, and triggering requests to -// fetch them from the network. Once collections are retrieved, it submits them to the Indexer for -// storage and indexing. +// It is responsible for ensuring the local node has all collections contained within finalized blocks +// starting from the last fully synced height. It works by periodically scanning the finalized block +// range from the last full block height up to the latest finalized block height, identifying missing +// collections, and triggering requests to fetch them from the network. Once collections are retrieved, +// it submits them to the Indexer for storage and indexing. // // It is meant to operate in a background goroutine as part of the node's ingestion pipeline. type Syncer struct { @@ -74,6 +73,7 @@ type Syncer struct { lastFullBlockHeight counters.Reader execDataSyncer *ExecutionDataSyncer + // these are held as members to allow configuring their values during testing. collectionCatchupTimeout time.Duration collectionCatchupDBPollInterval time.Duration missingCollsForBlockThreshold int @@ -144,7 +144,7 @@ func (s *Syncer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.R case <-requestCollectionsTicker.C: err := s.requestMissingCollections(ctx) if err != nil { - ctx.Throw(err) + ctx.Throw(fmt.Errorf("failed to request missing collections: %w", err)) return } } @@ -153,12 +153,13 @@ func (s *Syncer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.R // OnCollectionDownloaded notifies the collection syncer that a collection has been downloaded. // This callback implements [requester.HandleFunc] and is intended to be used with the requester engine. +// Panics if the provided entity is not a [flow.Collection]. func (s *Syncer) OnCollectionDownloaded(_ flow.Identifier, entity flow.Entity) { s.indexer.OnCollectionReceived(entity.(*flow.Collection)) } // RequestCollectionsForBlock conditionally requests missing collections for a specific block height, -// skipping requests if the block is already below the known full block height. +// skipping requests if the block is already below the known last full block height. // // No error returns are expected during normal operation. func (s *Syncer) RequestCollectionsForBlock(height uint64, missingCollections []*flow.CollectionGuarantee) error { @@ -181,8 +182,7 @@ func (s *Syncer) RequestCollectionsForBlock(height uint64, missingCollections [] return nil } -// requestCollections registers collection download requests in the requester engine and -// causes the requester to immediately dispatch requests. +// requestCollections registers collection download requests in the requester engine // // No error returns are expected during normal operation. func (s *Syncer) requestCollections(collections []*flow.CollectionGuarantee) error { @@ -194,7 +194,6 @@ func (s *Syncer) requestCollections(collections []*flow.CollectionGuarantee) err } s.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) } - return nil } @@ -217,18 +216,20 @@ func (s *Syncer) requestMissingCollections(ctx context.Context) error { if err != nil { return fmt.Errorf("failed to index collections from execution data: %w", err) } - // At this point, we have already submitted all collections within blocks up to `lastSyncedHeight` - // for indexing. However, indexing is completed asynchronously, so we updated `lastSyncedHeight` - // to be the last block for which we have execution data to avoid re-requesting already + // At this point, all collections within blocks up to and including `lastSyncedHeight` were + // submitted for indexing. However, indexing is completed asynchronously, so `lastSyncedHeight` + // was set to the last block for which we have execution data to avoid re-requesting already // submitted collections. } // request all other missing collections from Collection nodes. collections, incompleteBlocksCount, err := s.findMissingCollections(lastSyncedHeight, lastFinalizedBlock.Height) if err != nil { - return err + return fmt.Errorf("failed to find missing collections: %w", err) } + // only send requests if we are sufficiently behind the latest finalized block to avoid spamming + // collection nodes with requests. blocksThresholdReached := incompleteBlocksCount >= s.missingCollsForBlockThreshold ageThresholdReached := lastFinalizedBlock.Height-lastFullBlockHeight > s.missingCollsForAgeThreshold if len(collections) > 0 && (blocksThresholdReached || ageThresholdReached) { @@ -287,7 +288,7 @@ func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { progress := util.LogProgress(s.log, util.DefaultLogProgressConfig("requesting missing collections", lastFinalizedBlock.Height-lastFullBlockHeight)) - collectionsToBeDownloaded := make(map[flow.Identifier]struct{}) + pendingCollections := make(map[flow.Identifier]struct{}) for height := lastFullBlockHeight + 1; height <= lastFinalizedBlock.Height; height++ { if ctx.Err() != nil { return fmt.Errorf("missing collection catchup interrupted: %w", ctx.Err()) @@ -314,7 +315,7 @@ func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { return fmt.Errorf("failed to request collections: %w", err) } for _, collection := range collections { - collectionsToBeDownloaded[collection.CollectionID] = struct{}{} + pendingCollections[collection.CollectionID] = struct{}{} } } } @@ -322,7 +323,7 @@ func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { progress(1) } - if len(collectionsToBeDownloaded) == 0 { + if len(pendingCollections) == 0 { s.log.Info().Msg("no missing collections to download") return nil } @@ -333,26 +334,26 @@ func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { collectionStoragePollTicker := time.NewTicker(s.collectionCatchupDBPollInterval) defer collectionStoragePollTicker.Stop() - // we want to wait for all collections to be downloaded so we poll local storage periodically to make sure each - // collection was successfully saved in the storage. - for len(collectionsToBeDownloaded) > 0 { + // we want to wait for all collections to be downloaded so we poll local storage periodically to + // make sure each collection was successfully stored and indexed. + for len(pendingCollections) > 0 { select { case <-ctx.Done(): return fmt.Errorf("failed to complete collection retrieval: %w", ctx.Err()) case <-collectionStoragePollTicker.C: s.log.Debug(). - Int("total_missing_collections", len(collectionsToBeDownloaded)). + Int("total_missing_collections", len(pendingCollections)). Msg("retrieving missing collections...") - for collectionID := range collectionsToBeDownloaded { + for collectionID := range pendingCollections { downloaded, err := s.indexer.IsCollectionInStorage(collectionID) if err != nil { return err } if downloaded { - delete(collectionsToBeDownloaded, collectionID) + delete(pendingCollections, collectionID) } } } diff --git a/engine/access/ingestion/collections/syncer_execution_data.go b/engine/access/ingestion/collections/syncer_execution_data.go index adca55b060f..1be0ac55fda 100644 --- a/engine/access/ingestion/collections/syncer_execution_data.go +++ b/engine/access/ingestion/collections/syncer_execution_data.go @@ -7,23 +7,21 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" - "github.com/rs/zerolog" ) +// ExecutionDataSyncer submits collections from execution data to the collections indexer. It is +// designed to be used within the collection syncer to optimize indexing when collection data is +// already available on the node. type ExecutionDataSyncer struct { - log zerolog.Logger - executionDataCache execution_data.ExecutionDataCache indexer CollectionIndexer } func NewExecutionDataSyncer( - log zerolog.Logger, executionDataCache execution_data.ExecutionDataCache, indexer CollectionIndexer, ) *ExecutionDataSyncer { return &ExecutionDataSyncer{ - log: log.With().Str("component", "execution-data-syncer").Logger(), executionDataCache: executionDataCache, indexer: indexer, } diff --git a/engine/access/ingestion/collections/syncer_test.go b/engine/access/ingestion/collections/syncer_test.go index d08f11548f9..d13b7dd7712 100644 --- a/engine/access/ingestion/collections/syncer_test.go +++ b/engine/access/ingestion/collections/syncer_test.go @@ -60,7 +60,6 @@ func (s *SyncerSuite) SetupTest() { func (s *SyncerSuite) createSyncer() *Syncer { execDataSyncer := NewExecutionDataSyncer( - unittest.Logger(), s.executionDataCache, s.indexer, ) @@ -535,7 +534,6 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { s.Run("missing collections - processed from execution data", func() { execDataSyncer := NewExecutionDataSyncer( - unittest.Logger(), s.executionDataCache, s.indexer, ) diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index ba9f4eb0905..c30a7de96e9 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -187,7 +187,6 @@ func (s *Suite) initEngineAndSyncer() (*Engine, *collections.Syncer, *collection s.proto.state, s.blocks, s.collections, - s.transactions, s.lastFullBlockHeight, s.lockManager, ) diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index a74080d2a53..8067ac3c4cd 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -215,7 +215,6 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, s.proto.state, s.blocks, s.collections, - s.transactions, s.lastFullBlockHeight, s.lockManager, ) From e552e1157ca32d9777b0e4bbaa9d7fa269d84a04 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 09:23:22 -0800 Subject: [PATCH 03/16] fix lint errors --- engine/access/ingestion/collections/indexer.go | 3 ++- engine/access/ingestion/collections/indexer_test.go | 6 ++++-- engine/access/ingestion2/finalized_block_processor.go | 6 +++++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/engine/access/ingestion/collections/indexer.go b/engine/access/ingestion/collections/indexer.go index f56d674d6c3..9011053b180 100644 --- a/engine/access/ingestion/collections/indexer.go +++ b/engine/access/ingestion/collections/indexer.go @@ -6,6 +6,8 @@ import ( "time" "github.com/jordanschalm/lockctx" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/flow" @@ -16,7 +18,6 @@ import ( "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" - "github.com/rs/zerolog" ) const ( diff --git a/engine/access/ingestion/collections/indexer_test.go b/engine/access/ingestion/collections/indexer_test.go index 8df0bf28820..0a1260a6cc5 100644 --- a/engine/access/ingestion/collections/indexer_test.go +++ b/engine/access/ingestion/collections/indexer_test.go @@ -223,7 +223,8 @@ func (s *IndexerSuite) TestUpdateLastFullBlockHeight() { finalSnapshot.On("Head").Return(finalizedBlock.ToHeader(), nil).Once() s.state.On("Final").Return(finalSnapshot).Once() - indexer.updateLastFullBlockHeight() + err := indexer.updateLastFullBlockHeight() + s.Require().NoError(err) updated := indexer.lastFullBlockHeight.Value() s.Equal(original, updated) @@ -252,7 +253,8 @@ func (s *IndexerSuite) TestUpdateLastFullBlockHeight() { } s.lastFullBlockCP.On("SetProcessedIndex", finalizedBlock.Height).Return(nil).Once() - indexer.updateLastFullBlockHeight() + err := indexer.updateLastFullBlockHeight() + s.Require().NoError(err) updated := indexer.lastFullBlockHeight.Value() s.Equal(finalizedBlock.Height, updated) diff --git a/engine/access/ingestion2/finalized_block_processor.go b/engine/access/ingestion2/finalized_block_processor.go index 1bb9da66cd7..0c6700450bb 100644 --- a/engine/access/ingestion2/finalized_block_processor.go +++ b/engine/access/ingestion2/finalized_block_processor.go @@ -159,7 +159,11 @@ func (p *FinalizedBlockProcessor) indexFinalizedBlock(block *flow.Block) error { } } - p.collectionSyncer.RequestCollectionsForBlock(block.Height, block.Payload.Guarantees) + 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 From dc240088c9c3fc9672e6058ccc23e5d905d00c70 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 10:35:32 -0800 Subject: [PATCH 04/16] remove maybe from tests --- engine/access/ingestion/engine_test.go | 8 ++++---- module/executiondatasync/execution_data/cache/cache.go | 2 ++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index c30a7de96e9..808b90b0690 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -264,7 +264,7 @@ func (s *Suite) TestOnFinalizedBlockSingle() { snap := new(protocolmock.Snapshot) finalSnapshot := protocolmock.NewSnapshot(s.T()) - finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Twice() s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) @@ -334,7 +334,7 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { snap := new(protocolmock.Snapshot) finalSnapshot := protocolmock.NewSnapshot(s.T()) - finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Twice() s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) @@ -419,8 +419,8 @@ func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { // TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed func (s *Suite) TestExecutionReceiptsAreIndexed() { finalSnapshot := protocolmock.NewSnapshot(s.T()) - finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Maybe() - s.proto.state.On("Final").Return(finalSnapshot, nil).Twice() + finalSnapshot.On("Head").Return(s.finalizedBlock, nil).Once() + s.proto.state.On("Final").Return(finalSnapshot, nil).Once() eng, _, _ := s.initEngineAndSyncer() diff --git a/module/executiondatasync/execution_data/cache/cache.go b/module/executiondatasync/execution_data/cache/cache.go index 90887ad7a6c..a2918dd0aae 100644 --- a/module/executiondatasync/execution_data/cache/cache.go +++ b/module/executiondatasync/execution_data/cache/cache.go @@ -20,6 +20,8 @@ type ExecutionDataCache struct { cache mempool.ExecutionData } +var _ execution_data.ExecutionDataCache = (*ExecutionDataCache)(nil) + // NewExecutionDataCache returns a new ExecutionDataCache. func NewExecutionDataCache( backend execution_data.ExecutionDataGetter, From b8225502702eeffc5924a7b09eee086ab2303b66 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 15:26:34 -0800 Subject: [PATCH 05/16] Apply suggestions from code review Co-authored-by: Jordan Schalm --- engine/access/access_test.go | 4 ++-- engine/access/ingestion/collections/indexer.go | 10 +++++----- .../access/ingestion/collections/indexer_test.go | 14 +++++++------- engine/access/ingestion/collections/syncer.go | 4 ++-- engine/access/ingestion/collections/syncer_test.go | 8 ++++---- 5 files changed, 20 insertions(+), 20 deletions(-) diff --git a/engine/access/access_test.go b/engine/access/access_test.go index 8cf3f8de86c..15beb15953e 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -818,7 +818,7 @@ func (suite *Suite) TestGetSealedTransaction() { require.Eventually(suite.T(), func() bool { isStored, err := collectionIndexer.IsCollectionInStorage(collection.ID()) return isStored && err == nil - }, 1*time.Second, 100*time.Millisecond, "collection not indexed") + }, 1*time.Second, 10*time.Millisecond, "collection not indexed") // 5. Client requests a transaction tx := collection.Transactions[0] @@ -1074,7 +1074,7 @@ func (suite *Suite) TestGetTransactionResult() { return false } return true - }, 1*time.Second, 100*time.Millisecond, "transactions never indexed") + }, 1*time.Second, 10*time.Millisecond, "transactions never indexed") assertTransactionResult := func( resp *accessproto.TransactionResultResponse, diff --git a/engine/access/ingestion/collections/indexer.go b/engine/access/ingestion/collections/indexer.go index 9011053b180..d2c5151930b 100644 --- a/engine/access/ingestion/collections/indexer.go +++ b/engine/access/ingestion/collections/indexer.go @@ -38,7 +38,7 @@ type CollectionIndexer interface { // finalized block height. // // Expected error returns during normal operation: - // - [storage.ErrNotFound]: if provided block height is not finalized + // - [storage.ErrNotFound]: if provided block height is not finalized or below this node's root block MissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) // IsCollectionInStorage checks whether the given collection is present in local storage. @@ -169,8 +169,7 @@ func (ci *Indexer) indexCollection(collection *flow.Collection) error { // first notification should be processed. // // It's OK that this check is not done atomically with the index operation since the collections - // module will perform a similar check. Also, this module should be the only system performing - // collection writes. + // storage module is solely responsible for enforcing consistency (even if this is a stale read). exists, err := ci.IsCollectionInStorage(collection.ID()) if err != nil { return fmt.Errorf("failed to check if collection is in storage: %w", err) @@ -197,8 +196,9 @@ func (ci *Indexer) indexCollection(collection *flow.Collection) error { return nil } -// updateLastFullBlockHeight updates the next highest block height where all previous collections -// are indexed if it has changed. +// updateLastFullBlockHeight updates the LastFullBlockHeight index (if it has changed). +// The LastFullBlockHeight index tracks the height of the highest block B, such that all collections +// in B and in all B's ancestors have been indexed by this node. // // No error returns are expected during normal operation. func (ci *Indexer) updateLastFullBlockHeight() error { diff --git a/engine/access/ingestion/collections/indexer_test.go b/engine/access/ingestion/collections/indexer_test.go index 0a1260a6cc5..d882bdb1fb5 100644 --- a/engine/access/ingestion/collections/indexer_test.go +++ b/engine/access/ingestion/collections/indexer_test.go @@ -161,15 +161,15 @@ func (s *IndexerSuite) TestMissingCollectionsAtHeight_ErrorCases() { }) s.Run("collection error", func() { - expectedErr := errors.New("expected error") + exception := errors.New("exception") s.blocks.On("ByHeight", block.Height).Return(block, nil).Once() - s.collections.On("LightByID", block.Payload.Guarantees[0].CollectionID).Return(nil, expectedErr).Once() + s.collections.On("LightByID", block.Payload.Guarantees[0].CollectionID).Return(nil, exception).Once() indexer := s.createIndexer(s.T()) missingCollections, err := indexer.MissingCollectionsAtHeight(block.Height) - s.Require().ErrorIs(err, expectedErr) + s.Require().ErrorIs(err, exception) s.Require().Empty(missingCollections) }) } @@ -199,13 +199,13 @@ func (s *IndexerSuite) TestIsCollectionInStorage() { }) s.Run("unexpected error", func() { - expectedErr := errors.New("unexpected error") - s.collections.On("LightByID", collection.ID()).Return(nil, expectedErr).Once() + exception := errors.New("unexpected error") + s.collections.On("LightByID", collection.ID()).Return(nil, exception).Once() indexer := s.createIndexer(s.T()) inStorage, err := indexer.IsCollectionInStorage(collection.ID()) - s.Require().ErrorIs(err, expectedErr) + s.Require().ErrorIs(err, exception) s.Require().False(inStorage) }) } @@ -363,7 +363,7 @@ func (s *IndexerSuite) TestWorkerProcessing_ProcessesCollections() { // wait until the last full block height is updated require.Eventually(s.T(), func() bool { return lastFullBlockHeight.Value() == prevHeader.Height - }, time.Second, 100*time.Millisecond) + }, time.Second, 10*time.Millisecond) }) } diff --git a/engine/access/ingestion/collections/syncer.go b/engine/access/ingestion/collections/syncer.go index 57ef0e5fd78..78cf887608a 100644 --- a/engine/access/ingestion/collections/syncer.go +++ b/engine/access/ingestion/collections/syncer.go @@ -52,7 +52,7 @@ const ( // from the network. // 3. Submit collections received to the Indexer for storage and indexing. // -// The Syncer guarantees that all collection for finalized blocks will eventually be received as long +// The Syncer guarantees that all collections for finalized blocks will eventually be received as long // as there are honest Collection nodes in each cluster, and the node is able to successfully communicate // with them over the networking layer. // @@ -71,7 +71,7 @@ type Syncer struct { state protocol.State collections storage.Collections lastFullBlockHeight counters.Reader - execDataSyncer *ExecutionDataSyncer + execDataSyncer *ExecutionDataSyncer // may be nil // these are held as members to allow configuring their values during testing. collectionCatchupTimeout time.Duration diff --git a/engine/access/ingestion/collections/syncer_test.go b/engine/access/ingestion/collections/syncer_test.go index d13b7dd7712..be744da5df0 100644 --- a/engine/access/ingestion/collections/syncer_test.go +++ b/engine/access/ingestion/collections/syncer_test.go @@ -134,7 +134,7 @@ func (s *SyncerSuite) TestRequestCollectionsForBlock() { syncer := s.createSyncer() s.Run("request for block <= last full block height", func() { - // EntityByID should not be called + s.requester.AssertNotCalled(s.T(), "EntityByID") err := syncer.RequestCollectionsForBlock(s.lastFullBlockHeight.Value(), guarantees) s.Require().NoError(err) }) @@ -153,11 +153,11 @@ func (s *SyncerSuite) TestRequestCollectionsForBlock() { }) s.Run("finding guarantors fails", func() { - expectedError := errors.New("state lookup failed") - s.mockGuarantorsForCollectionReturnsError(guarantees[0], expectedError) + exception := errors.New("state lookup failed") + s.mockGuarantorsForCollectionReturnsError(guarantees[0], exception) err := syncer.RequestCollectionsForBlock(s.lastFullBlockHeight.Value()+1, guarantees) - s.Require().ErrorIs(err, expectedError) + s.Require().ErrorIs(err, exception) }) } From 7015fff4889adcbf32d0442ea0ad42f5dcfad02f Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 15:27:07 -0800 Subject: [PATCH 06/16] fix flaky pipeline tests --- .../pipeline_functional_test.go | 72 ++++++++++++++----- .../optimistic_sync/pipeline_test_utils.go | 26 +++++++ .../indexer/indexer_core.go | 4 +- 3 files changed, 83 insertions(+), 19 deletions(-) diff --git a/module/executiondatasync/optimistic_sync/pipeline_functional_test.go b/module/executiondatasync/optimistic_sync/pipeline_functional_test.go index 84364b0e7bd..61ee16145b3 100644 --- a/module/executiondatasync/optimistic_sync/pipeline_functional_test.go +++ b/module/executiondatasync/optimistic_sync/pipeline_functional_test.go @@ -5,6 +5,7 @@ import ( "fmt" "os" "testing" + "testing/synctest" "time" "github.com/jordanschalm/lockctx" @@ -436,9 +437,14 @@ func (p *PipelineFunctionalSuite) TestMainCtxCancellationDuringWaitingPersist() // TestPipelineShutdownOnParentAbandon verifies that the pipeline transitions correctly to a shutdown state when the parent is abandoned. func (p *PipelineFunctionalSuite) TestPipelineShutdownOnParentAbandon() { + assertNoError := func(err error) { + p.Require().NoError(err) + } + tests := []struct { name string config PipelineConfig + checkError func(err error) customSetup func(pipeline Pipeline, updateChan chan State, errChan chan error) }{ { @@ -449,43 +455,68 @@ func (p *PipelineFunctionalSuite) TestPipelineShutdownOnParentAbandon() { }, parentState: StateAbandoned, }, + checkError: assertNoError, + customSetup: func(pipeline Pipeline, updateChan chan State, errChan chan error) {}, }, { name: "from StateProcessing", + config: PipelineConfig{ + beforePipelineRun: func(pipeline *PipelineImpl) { + p.execDataRequester.On("RequestExecutionData", mock.Anything).Return(func(ctx context.Context) (*execution_data.BlockExecutionData, error) { + pipeline.OnParentStateUpdated(StateAbandoned) // abandon during processing step + return p.expectedExecutionData, nil + }).Once() + // this method may not be called depending on how quickly the RequestExecutionData + // mock returns. + p.txResultErrMsgsRequester.On("Request", mock.Anything).Return(p.expectedTxResultErrMsgs, nil).Maybe() + }, + parentState: StateWaitingPersist, + }, + checkError: func(err error) { + // depending on the timing, the error may be during or after the indexing step. + if err != nil { + p.Require().ErrorContains(err, "could not perform indexing") + } else { + p.Require().NoError(err) + } + }, customSetup: func(pipeline Pipeline, updateChan chan State, errChan chan error) { - waitForStateUpdates(p.T(), updateChan, errChan, StateProcessing) - - pipeline.OnParentStateUpdated(StateAbandoned) + synctestWaitForStateUpdates(p.T(), updateChan, StateProcessing) }, - config: p.config, }, { name: "from StateWaitingPersist", + config: PipelineConfig{ + beforePipelineRun: func(pipeline *PipelineImpl) { + p.execDataRequester.On("RequestExecutionData", mock.Anything).Return(p.expectedExecutionData, nil).Once() + p.txResultErrMsgsRequester.On("Request", mock.Anything).Return(p.expectedTxResultErrMsgs, nil).Once() + }, + parentState: StateWaitingPersist, + }, + checkError: assertNoError, customSetup: func(pipeline Pipeline, updateChan chan State, errChan chan error) { - waitForStateUpdates(p.T(), updateChan, errChan, StateProcessing, StateWaitingPersist) - + synctestWaitForStateUpdates(p.T(), updateChan, StateProcessing, StateWaitingPersist) pipeline.OnParentStateUpdated(StateAbandoned) }, - config: p.config, }, } for _, test := range tests { p.T().Run(test.name, func(t *testing.T) { - p.WithRunningPipeline(func(pipeline Pipeline, updateChan chan State, errChan chan error, cancel context.CancelFunc) { - p.execDataRequester.On("RequestExecutionData", mock.Anything).Return(p.expectedExecutionData, nil).Maybe() - p.txResultErrMsgsRequester.On("Request", mock.Anything).Return(p.expectedTxResultErrMsgs, nil).Maybe() + p.execDataRequester.On("RequestExecutionData", mock.Anything).Unset() + p.txResultErrMsgsRequester.On("Request", mock.Anything).Unset() - if test.customSetup != nil { + synctest.Test(p.T(), func(t *testing.T) { + p.WithRunningPipeline(func(pipeline Pipeline, updateChan chan State, errChan chan error, cancel context.CancelFunc) { test.customSetup(pipeline, updateChan, errChan) - } - waitForStateUpdates(p.T(), updateChan, errChan, StateAbandoned) - waitForError(p.T(), errChan, nil) + synctestWaitForStateUpdates(p.T(), updateChan, StateAbandoned) + test.checkError(<-errChan) - p.Assert().Equal(StateAbandoned, pipeline.GetState()) - p.Assert().Nil(p.core.workingData) - }, test.config) + p.Assert().Equal(StateAbandoned, pipeline.GetState()) + p.Assert().Nil(p.core.workingData) + }, test.config) + }) }) } } @@ -534,13 +565,18 @@ func (p *PipelineFunctionalSuite) WithRunningPipeline( pipelineIsReady := make(chan struct{}) go func() { + defer close(errChan) + if pipelineConfig.beforePipelineRun != nil { pipelineConfig.beforePipelineRun(pipeline) } close(pipelineIsReady) - errChan <- pipeline.Run(ctx, p.core, pipelineConfig.parentState) + err := pipeline.Run(ctx, p.core, pipelineConfig.parentState) + if err != nil { + errChan <- err + } }() <-pipelineIsReady diff --git a/module/executiondatasync/optimistic_sync/pipeline_test_utils.go b/module/executiondatasync/optimistic_sync/pipeline_test_utils.go index 98e4e35a072..e0d9c7a98a7 100644 --- a/module/executiondatasync/optimistic_sync/pipeline_test_utils.go +++ b/module/executiondatasync/optimistic_sync/pipeline_test_utils.go @@ -2,6 +2,7 @@ package optimistic_sync import ( "testing" + "testing/synctest" "time" "github.com/rs/zerolog" @@ -116,3 +117,28 @@ func createPipeline(t *testing.T) (*PipelineImpl, *osmock.Core, <-chan State, *m return pipeline, mockCore, stateReceiver.updateChan, parent } + +// synctestWaitForStateUpdates waits for a sequence of state updates to occur using synctest.Wait. +// updates must be received in the correct order or the test will fail. +// TODO: refactor all tests to use the synctest approach. +func synctestWaitForStateUpdates(t *testing.T, updateChan <-chan State, expectedStates ...State) { + for _, expected := range expectedStates { + synctest.Wait() + update, ok := <-updateChan + require.True(t, ok, "update channel closed unexpectedly") + assert.Equalf(t, expected, update, "expected pipeline to transition to %s, but got %s", expected, update) + } +} + +// synctestWaitForError waits for an error from the errChan using synctest.Wait and asserts it matches the expected error. +// TODO: refactor all tests to use the synctest approach. +func synctestWaitForError(t *testing.T, errChan <-chan error, expectedErr error) { + synctest.Wait() + err := <-errChan + if expectedErr == nil { + assert.NoError(t, err, "Pipeline should complete without errors") + } else { + assert.ErrorIs(t, err, expectedErr) + } + +} diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index 6d3704a3176..b0e0d2fdbaa 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -222,12 +222,14 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti g.Go(func() error { start := time.Now() - // index all collections except the system chunk // 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 collections except the system chunk. if there is only a single chunk, it is the + // system chunk and can be skipped. indexedCount := 0 if len(data.ChunkExecutionDatas) > 1 { for _, chunk := range data.ChunkExecutionDatas[0 : len(data.ChunkExecutionDatas)-1] { From a1baef22525e4ae31fd2a816ca09932cb8f61eee Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 15:43:04 -0800 Subject: [PATCH 07/16] fix lint --- .../optimistic_sync/pipeline_test_utils.go | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/module/executiondatasync/optimistic_sync/pipeline_test_utils.go b/module/executiondatasync/optimistic_sync/pipeline_test_utils.go index e0d9c7a98a7..31084099361 100644 --- a/module/executiondatasync/optimistic_sync/pipeline_test_utils.go +++ b/module/executiondatasync/optimistic_sync/pipeline_test_utils.go @@ -129,16 +129,3 @@ func synctestWaitForStateUpdates(t *testing.T, updateChan <-chan State, expected assert.Equalf(t, expected, update, "expected pipeline to transition to %s, but got %s", expected, update) } } - -// synctestWaitForError waits for an error from the errChan using synctest.Wait and asserts it matches the expected error. -// TODO: refactor all tests to use the synctest approach. -func synctestWaitForError(t *testing.T, errChan <-chan error, expectedErr error) { - synctest.Wait() - err := <-errChan - if expectedErr == nil { - assert.NoError(t, err, "Pipeline should complete without errors") - } else { - assert.ErrorIs(t, err, expectedErr) - } - -} From 24143427802d7998d0135e702a1d36b9475ec917 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 16:40:58 -0800 Subject: [PATCH 08/16] fix indexer tests --- module/state_synchronization/indexer/indexer_test.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index 83f647244d2..a17e45a3bf2 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -6,7 +6,6 @@ import ( "testing" "time" - mocks "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -202,7 +201,7 @@ func TestIndexer_Success(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collections.On("StoreAndIndexByTransaction", mocks.Anything, collection).Return(collection.Light(), nil).Once() + test.indexTest.collectionIndexer.On("OnCollectionReceived", collection).Return(nil).Once() test.indexTest.registers.On("Store", flow.RegisterEntries{}, block.Height).Return(nil).Once() } @@ -244,7 +243,7 @@ func TestIndexer_Failure(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collections.On("StoreAndIndexByTransaction", mocks.Anything, collection).Return(collection.Light(), nil).Once() + test.indexTest.collectionIndexer.On("OnCollectionReceived", collection).Return(nil).Once() // return an error on the last block to trigger the error path if block.Height == lastHeight { From 36f28762598d98a9d55c71ae558f4f2f93baf200 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Mon, 3 Nov 2025 18:21:18 -0800 Subject: [PATCH 09/16] fix lint --- utils/unittest/fixtures/payload.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/unittest/fixtures/payload.go b/utils/unittest/fixtures/payload.go index 3fa7ca617ce..a8934feedc5 100644 --- a/utils/unittest/fixtures/payload.go +++ b/utils/unittest/fixtures/payload.go @@ -51,7 +51,7 @@ func (f payloadFactory) WithReceiptStubs(receipts ...*flow.ExecutionReceiptStub) } // WithResults is an option that sets the `Results` of the payload. -func (f payloadFactory) WithResults(results flow.ExecutionResultList) PayloadOption { +func (f payloadFactory) WithResults(results ...*flow.ExecutionResult) PayloadOption { return func(g *PayloadGenerator, payload *flow.Payload) { payload.Results = results } From 10ea9e0d279408a729e27dfbe7c0b8d199a53127 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 15:05:30 -0800 Subject: [PATCH 10/16] [Access] Refactor the collection indexing and syncing --- .../node_builder/access_node_builder.go | 2 + cmd/observer/node_builder/observer_builder.go | 1 + .../access/ingestion/collections/indexer.go | 78 ++--- .../ingestion/collections/indexer_test.go | 16 +- .../collections/mock/collection_indexer.go | 28 +- engine/access/ingestion/collections/syncer.go | 282 +++++++----------- .../collections/syncer_execution_data.go | 46 ++- .../ingestion/collections/syncer_test.go | 156 ++++++---- engine/access/ingestion/engine_test.go | 1 + engine/access/ingestion2/engine_test.go | 1 + .../execution_data/execution_data.go | 25 ++ .../indexer/indexer_core.go | 15 +- 12 files changed, 319 insertions(+), 332 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index fabc67bd7f4..6a79fd918bb 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -2215,6 +2215,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { collectionIndexer, err := collections.NewIndexer( node.Logger, + builder.ProtocolDB, notNil(builder.collectionExecutedMetric), node.State, node.Storage.Blocks, @@ -2233,6 +2234,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { var executionDataSyncer *collections.ExecutionDataSyncer if builder.executionDataSyncEnabled && !builder.executionDataIndexingEnabled { executionDataSyncer = collections.NewExecutionDataSyncer( + node.Logger, notNil(builder.ExecutionDataCache), collectionIndexer, ) diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 4b7965b16ba..7e4ae5ce6ed 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1454,6 +1454,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS var collectionExecutedMetric module.CollectionExecutedMetric = metrics.NewNoopCollector() collectionIndexer, err := collections.NewIndexer( builder.Logger, + builder.ProtocolDB, collectionExecutedMetric, builder.State, builder.Storage.Blocks, diff --git a/engine/access/ingestion/collections/indexer.go b/engine/access/ingestion/collections/indexer.go index d2c5151930b..3fc6969a586 100644 --- a/engine/access/ingestion/collections/indexer.go +++ b/engine/access/ingestion/collections/indexer.go @@ -34,17 +34,18 @@ type CollectionIndexer interface { // This method is non-blocking. OnCollectionReceived(collection *flow.Collection) + // IndexCollections indexes a set of collections, skipping any collections which already exist in storage. + // Calling this method multiple times with the same collections is a no-op. + // + // No error returns are expected during normal operation. + IndexCollections(collections []*flow.Collection) error + // MissingCollectionsAtHeight returns all collections that are not present in storage for a specific // finalized block height. // // Expected error returns during normal operation: // - [storage.ErrNotFound]: if provided block height is not finalized or below this node's root block MissingCollectionsAtHeight(height uint64) ([]*flow.CollectionGuarantee, error) - - // IsCollectionInStorage checks whether the given collection is present in local storage. - // - // No error returns are expected during normal operation. - IsCollectionInStorage(collectionID flow.Identifier) (bool, error) } // Indexer stores and indexes collections received from the network. It is designed to be the central @@ -56,10 +57,11 @@ type CollectionIndexer interface { // The indexer also maintains the last full block height state, which is the highest block height // for which all collections are stored and indexed. type Indexer struct { - log zerolog.Logger - metrics module.CollectionExecutedMetric - lockManager lockctx.Manager + log zerolog.Logger + metrics module.CollectionExecutedMetric + db storage.DB + lockManager lockctx.Manager state protocol.State blocks storage.Blocks collections storage.Collections @@ -74,6 +76,7 @@ type Indexer struct { // No error returns are expected during normal operation. func NewIndexer( log zerolog.Logger, + db storage.DB, metrics module.CollectionExecutedMetric, state protocol.State, blocks storage.Blocks, @@ -91,6 +94,7 @@ func NewIndexer( return &Indexer{ log: log.With().Str("component", "collection-indexer").Logger(), metrics: metrics, + db: db, lockManager: lockManager, state: state, blocks: blocks, @@ -136,7 +140,7 @@ func (ci *Indexer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component return } - if err := ci.indexCollection(collection); err != nil { + if err := ci.IndexCollections([]*flow.Collection{collection}); err != nil { ctx.Throw(fmt.Errorf("error indexing collection: %w", err)) return } @@ -158,11 +162,11 @@ func (ci *Indexer) OnCollectionReceived(collection *flow.Collection) { ci.pendingCollectionsNotifier.Notify() } -// indexCollection indexes a collection and its transactions. -// Skips indexing and returns without an error if the collection is already indexed. +// IndexCollections indexes a set of collections, skipping any collections which already exist in storage. +// Calling this method multiple times with the same collections is a no-op. // // No error returns are expected during normal operation. -func (ci *Indexer) indexCollection(collection *flow.Collection) error { +func (ci *Indexer) IndexCollections(collections []*flow.Collection) error { // skip indexing if collection is already indexed. on the common path, collections may be received // via multiple subsystems (e.g. execution data sync, collection sync, execution state indexer). // In this case, the indexer will be notified multiple times for the same collection. Only the @@ -170,30 +174,36 @@ func (ci *Indexer) indexCollection(collection *flow.Collection) error { // // It's OK that this check is not done atomically with the index operation since the collections // storage module is solely responsible for enforcing consistency (even if this is a stale read). - exists, err := ci.IsCollectionInStorage(collection.ID()) - if err != nil { - return fmt.Errorf("failed to check if collection is in storage: %w", err) - } - if exists { - return nil + newCollections := make([]*flow.Collection, 0) + for _, collection := range collections { + exists, err := ci.isCollectionInStorage(collection.ID()) + if err != nil { + return fmt.Errorf("failed to check if collection is in storage: %w", err) + } + if !exists { + newCollections = append(newCollections, collection) + } } - lctx := ci.lockManager.NewContext() - defer lctx.Release() - err = lctx.AcquireLock(storage.LockInsertCollection) - if err != nil { - return fmt.Errorf("could not acquire lock for indexing collections: %w", err) + if len(newCollections) == 0 { + return nil } - // store the collection, including constituent transactions, and index transactionID -> collectionID - light, err := ci.collections.StoreAndIndexByTransaction(lctx, collection) - if err != nil { - return fmt.Errorf("failed to store collection: %w", err) - } + return storage.WithLock(ci.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return ci.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + for _, collection := range newCollections { + // store the collection, including constituent transactions, and index transactionID -> collectionID + light, err := ci.collections.BatchStoreAndIndexByTransaction(lctx, collection, rw) + if err != nil { + return fmt.Errorf("failed to store collection: %w", err) + } - ci.metrics.CollectionFinalized(light) - ci.metrics.CollectionExecuted(light) - return nil + ci.metrics.CollectionFinalized(light) + ci.metrics.CollectionExecuted(light) + } + return nil + }) + }) } // updateLastFullBlockHeight updates the LastFullBlockHeight index (if it has changed). @@ -254,7 +264,7 @@ func (ci *Indexer) MissingCollectionsAtHeight(height uint64) ([]*flow.Collection var missingCollections []*flow.CollectionGuarantee for _, guarantee := range block.Payload.Guarantees { - inStorage, err := ci.IsCollectionInStorage(guarantee.CollectionID) + inStorage, err := ci.isCollectionInStorage(guarantee.CollectionID) if err != nil { return nil, err } @@ -267,10 +277,10 @@ func (ci *Indexer) MissingCollectionsAtHeight(height uint64) ([]*flow.Collection return missingCollections, nil } -// IsCollectionInStorage checks whether the given collection is present in local storage. +// isCollectionInStorage checks whether the given collection is present in local storage. // // No error returns are expected during normal operation. -func (ci *Indexer) IsCollectionInStorage(collectionID flow.Identifier) (bool, error) { +func (ci *Indexer) isCollectionInStorage(collectionID flow.Identifier) (bool, error) { _, err := ci.collections.LightByID(collectionID) if err == nil { return true, nil diff --git a/engine/access/ingestion/collections/indexer_test.go b/engine/access/ingestion/collections/indexer_test.go index d882bdb1fb5..294ac3c034a 100644 --- a/engine/access/ingestion/collections/indexer_test.go +++ b/engine/access/ingestion/collections/indexer_test.go @@ -33,6 +33,7 @@ import ( type IndexerSuite struct { suite.Suite + db *storagemock.DB state *protocolmock.State blocks *storagemock.Blocks collections *storagemock.Collections @@ -49,6 +50,7 @@ func TestIndexer(t *testing.T) { } func (s *IndexerSuite) SetupTest() { + s.db = storagemock.NewDB(s.T()) s.state = protocolmock.NewState(s.T()) s.blocks = storagemock.NewBlocks(s.T()) s.collections = storagemock.NewCollections(s.T()) @@ -67,6 +69,7 @@ func (s *IndexerSuite) SetupTest() { func (s *IndexerSuite) createIndexer(t *testing.T) *Indexer { indexer, err := NewIndexer( unittest.Logger(), + s.db, metrics.NewNoopCollector(), s.state, s.blocks, @@ -183,7 +186,7 @@ func (s *IndexerSuite) TestIsCollectionInStorage() { indexer := s.createIndexer(s.T()) - inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + inStorage, err := indexer.isCollectionInStorage(collection.ID()) s.Require().NoError(err) s.Require().True(inStorage) }) @@ -193,7 +196,7 @@ func (s *IndexerSuite) TestIsCollectionInStorage() { indexer := s.createIndexer(s.T()) - inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + inStorage, err := indexer.isCollectionInStorage(collection.ID()) s.Require().NoError(err) s.Require().False(inStorage) }) @@ -204,7 +207,7 @@ func (s *IndexerSuite) TestIsCollectionInStorage() { indexer := s.createIndexer(s.T()) - inStorage, err := indexer.IsCollectionInStorage(collection.ID()) + inStorage, err := indexer.isCollectionInStorage(collection.ID()) s.Require().ErrorIs(err, exception) s.Require().False(inStorage) }) @@ -268,7 +271,11 @@ func (s *IndexerSuite) TestOnCollectionReceived() { synctest.Test(s.T(), func(t *testing.T) { s.collections.On("LightByID", collection.ID()).Return(nil, storage.ErrNotFound).Once() - s.collections.On("StoreAndIndexByTransaction", mock.Anything, collection).Return(collection.Light(), nil).Once() + s.collections.On("BatchStoreAndIndexByTransaction", mock.Anything, collection, mock.Anything).Return(collection.Light(), nil).Once() + + s.db.On("WithReaderBatchWriter", mock.Anything).Return(func(fn func(storage.ReaderBatchWriter) error) error { + return fn(nil) + }).Once() indexer := s.createIndexer(s.T()) @@ -312,6 +319,7 @@ func (s *IndexerSuite) TestWorkerProcessing_ProcessesCollections() { indexer, err := NewIndexer( unittest.Logger(), + bc.db, metrics.NewNoopCollector(), bc.state, bc.all.Blocks, diff --git a/engine/access/ingestion/collections/mock/collection_indexer.go b/engine/access/ingestion/collections/mock/collection_indexer.go index 9cfafd0df99..9d2f174cfee 100644 --- a/engine/access/ingestion/collections/mock/collection_indexer.go +++ b/engine/access/ingestion/collections/mock/collection_indexer.go @@ -12,32 +12,22 @@ type CollectionIndexer struct { mock.Mock } -// IsCollectionInStorage provides a mock function with given fields: collectionID -func (_m *CollectionIndexer) IsCollectionInStorage(collectionID flow.Identifier) (bool, error) { - ret := _m.Called(collectionID) +// IndexCollections provides a mock function with given fields: _a0 +func (_m *CollectionIndexer) IndexCollections(_a0 []*flow.Collection) error { + ret := _m.Called(_a0) if len(ret) == 0 { - panic("no return value specified for IsCollectionInStorage") + panic("no return value specified for IndexCollections") } - var r0 bool - var r1 error - if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { - return rf(collectionID) - } - if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { - r0 = rf(collectionID) + var r0 error + if rf, ok := ret.Get(0).(func([]*flow.Collection) error); ok { + r0 = rf(_a0) } else { - r0 = ret.Get(0).(bool) + r0 = ret.Error(0) } - if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { - r1 = rf(collectionID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // MissingCollectionsAtHeight provides a mock function with given fields: height diff --git a/engine/access/ingestion/collections/syncer.go b/engine/access/ingestion/collections/syncer.go index 78cf887608a..996d5c8b509 100644 --- a/engine/access/ingestion/collections/syncer.go +++ b/engine/access/ingestion/collections/syncer.go @@ -28,20 +28,15 @@ const ( // if missing collections have been received during the initial collection catchup process. DefaultCollectionCatchupDBPollInterval = 1 * time.Second - // DefaultMissingCollsRequestInterval is the interval at which the syncer checks missing collections + // DefaultMissingCollectionRequestInterval is the interval at which the syncer checks missing collections // and re-requests them from the network if needed. - DefaultMissingCollsRequestInterval = 1 * time.Minute + DefaultMissingCollectionRequestInterval = 1 * time.Minute - // DefaultMissingCollsForBlockThreshold is the threshold number of blocks with missing collections - // beyond which collections should be re-requested. This prevents spamming the collection nodes - // with requests for recent data. - DefaultMissingCollsForBlockThreshold = 100 - - // DefaultMissingCollsForAgeThreshold is the block height threshold below which collections + // DefaultMissingCollectionRequestThreshold is the block height threshold below which collections // should be re-requested, regardless of the number of blocks for which collection are missing. // This is to ensure that if a collection is missing for a long time (in terms of block height) // it is eventually re-requested. - DefaultMissingCollsForAgeThreshold = 100 + DefaultMissingCollectionRequestThreshold = 100 ) // The Syncer is responsible for syncing collections for finalized blocks from the network. It has @@ -74,11 +69,10 @@ type Syncer struct { execDataSyncer *ExecutionDataSyncer // may be nil // these are held as members to allow configuring their values during testing. - collectionCatchupTimeout time.Duration - collectionCatchupDBPollInterval time.Duration - missingCollsForBlockThreshold int - missingCollsForAgeThreshold uint64 - missingCollsRequestInterval time.Duration + collectionCatchupTimeout time.Duration + collectionCatchupDBPollInterval time.Duration + missingCollectionRequestThreshold uint64 + missingCollectionRequestInterval time.Duration } // NewSyncer creates a new Syncer responsible for requesting, tracking, and indexing missing collections. @@ -100,53 +94,61 @@ func NewSyncer( indexer: collectionIndexer, execDataSyncer: execDataSyncer, - collectionCatchupTimeout: DefaultCollectionCatchupTimeout, - collectionCatchupDBPollInterval: DefaultCollectionCatchupDBPollInterval, - missingCollsForBlockThreshold: DefaultMissingCollsForBlockThreshold, - missingCollsForAgeThreshold: DefaultMissingCollsForAgeThreshold, - missingCollsRequestInterval: DefaultMissingCollsRequestInterval, + collectionCatchupTimeout: DefaultCollectionCatchupTimeout, + collectionCatchupDBPollInterval: DefaultCollectionCatchupDBPollInterval, + missingCollectionRequestThreshold: DefaultMissingCollectionRequestThreshold, + missingCollectionRequestInterval: DefaultMissingCollectionRequestInterval, } } // WorkerLoop is a [component.ComponentWorker] that continuously monitors for missing collections, and // requests them from the network if needed. It also performs an initial collection catchup on startup. func (s *Syncer) WorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - requestCtx, cancel := context.WithTimeout(ctx, s.collectionCatchupTimeout) - defer cancel() - - // attempt to download all known missing collections on start-up. - err := s.requestMissingCollectionsBlocking(requestCtx) - if err != nil { - if ctx.Err() != nil { - s.log.Error().Err(err).Msg("engine shutdown while downloading missing collections") + // Block marking the component ready until either the first run of the missing collections catchup + // completes, or `collectionCatchupTimeout` expires. This improves the user experience by preventing + // the Access API from starting while the initial catchup completes. The intention is to avoid + // returning NotFound errors immediately after startup for data that should be available, but has + // not yet been indexed. + // TODO (peter): consider removing this. I'm not convinced that this provides much value since in + // the common case (node restart), the block finalization would also pause so the node should not + // become farther behind in terms of collections. + readyChan := make(chan struct{}) + go func() { + select { + case <-ctx.Done(): return + case <-readyChan: + case <-time.After(s.collectionCatchupTimeout): } + ready() + }() + + requestCollectionsTicker := time.NewTicker(s.missingCollectionRequestInterval) + defer requestCollectionsTicker.Stop() + + initialCatchupComplete := false + for { + err := s.requestMissingCollections(ctx, !initialCatchupComplete) + if err != nil { + if errors.Is(err, context.Canceled) { + return + } - if !errors.Is(err, context.DeadlineExceeded) { - ctx.Throw(fmt.Errorf("error downloading missing collections: %w", err)) + ctx.Throw(fmt.Errorf("failed to request missing collections: %w", err)) return } - // timed out during catchup. continue with normal startup. - // missing collections will be requested periodically. - s.log.Error().Err(err).Msg("timed out syncing collections during startup") - } - ready() - - requestCollectionsTicker := time.NewTicker(s.missingCollsRequestInterval) - defer requestCollectionsTicker.Stop() + // after the first successful run, mark the catchup as complete. This will cause the worker + // to call the ready function if it was not already called. + if !initialCatchupComplete { + initialCatchupComplete = true + close(readyChan) + } - for { select { + case <-requestCollectionsTicker.C: case <-ctx.Done(): return - - case <-requestCollectionsTicker.C: - err := s.requestMissingCollections(ctx) - if err != nil { - ctx.Throw(fmt.Errorf("failed to request missing collections: %w", err)) - return - } } } } @@ -197,168 +199,98 @@ func (s *Syncer) requestCollections(collections []*flow.CollectionGuarantee) err return nil } -// requestMissingCollections checks if missing collections should be requested based on configured -// block or age thresholds and triggers requests if needed. +// requestMissingCollections requests all missing collections using local execution data if available, +// otherwise requests them from the network. Only sends requests to the network if we are more than +// `missingCollectionRequestThreshold` blocks behind the latest finalized block. // -// No error returns are expected during normal operation. -func (s *Syncer) requestMissingCollections(ctx context.Context) error { +// Expected error returns during normal operation: +// - [context.Canceled]: if the context is canceled before all collections are requested +func (s *Syncer) requestMissingCollections(ctx context.Context, forceSendRequests bool) error { lastFullBlockHeight := s.lastFullBlockHeight.Value() lastFinalizedBlock, err := s.state.Final().Head() if err != nil { return fmt.Errorf("failed to get finalized block: %w", err) } - // if the node is syncing execution data, use the already downloaded data to index any available - // collections we are still missing. - lastSyncedHeight := lastFullBlockHeight - if s.execDataSyncer != nil { - lastSyncedHeight, err = s.execDataSyncer.IndexFromStartHeight(ctx, lastFullBlockHeight) - if err != nil { - return fmt.Errorf("failed to index collections from execution data: %w", err) - } - // At this point, all collections within blocks up to and including `lastSyncedHeight` were - // submitted for indexing. However, indexing is completed asynchronously, so `lastSyncedHeight` - // was set to the last block for which we have execution data to avoid re-requesting already - // submitted collections. - } - - // request all other missing collections from Collection nodes. - collections, incompleteBlocksCount, err := s.findMissingCollections(lastSyncedHeight, lastFinalizedBlock.Height) - if err != nil { - return fmt.Errorf("failed to find missing collections: %w", err) - } - // only send requests if we are sufficiently behind the latest finalized block to avoid spamming // collection nodes with requests. - blocksThresholdReached := incompleteBlocksCount >= s.missingCollsForBlockThreshold - ageThresholdReached := lastFinalizedBlock.Height-lastFullBlockHeight > s.missingCollsForAgeThreshold - if len(collections) > 0 && (blocksThresholdReached || ageThresholdReached) { - // warn log since this should generally not happen - s.log.Warn(). - Uint64("finalized_height", lastFinalizedBlock.Height). - Uint64("last_full_blk_height", lastFullBlockHeight). - Int("missing_collection_blk_count", incompleteBlocksCount). - Int("missing_collection_count", len(collections)). - Msg("re-requesting missing collections") - - err = s.requestCollections(collections) - if err != nil { - return fmt.Errorf("failed to request collections: %w", err) - } - // since this is a re-request, do not use force. new finalized block requests will force - // dispatch. On the happy path, this will happen at least once per second. + shouldSendRequestsToNetwork := forceSendRequests + if lastFullBlockHeight < lastFinalizedBlock.Height && (lastFinalizedBlock.Height-lastFullBlockHeight >= s.missingCollectionRequestThreshold) { + shouldSendRequestsToNetwork = true } - return nil -} + progress := util.LogProgress(s.log, util.DefaultLogProgressConfig("requesting missing collections", lastFinalizedBlock.Height-lastFullBlockHeight)) -// findMissingCollections scans block heights from last known full block up to the latest finalized -// block and returns all missing collection along with the count of incomplete blocks. -// -// No error returns are expected during normal operation. -func (s *Syncer) findMissingCollections(lastFullBlockHeight, finalizedBlockHeight uint64) ([]*flow.CollectionGuarantee, int, error) { - var missingCollections []*flow.CollectionGuarantee - var incompleteBlocksCount int + requestedBlocks := 0 + requestedCollections := 0 + for height := lastFullBlockHeight + 1; height <= lastFinalizedBlock.Height; height++ { + if ctx.Err() != nil { + return fmt.Errorf("missing collection catchup interrupted: %w", ctx.Err()) + } - for height := lastFullBlockHeight + 1; height <= finalizedBlockHeight; height++ { - collections, err := s.indexer.MissingCollectionsAtHeight(height) + collections, requested, err := s.requestForHeight(ctx, height, shouldSendRequestsToNetwork) if err != nil { - return nil, 0, err + return fmt.Errorf("failed to request collections for height %d: %w", height, err) } - - if len(collections) > 0 { - missingCollections = append(missingCollections, collections...) - incompleteBlocksCount += 1 + if requested { + requestedBlocks++ + requestedCollections += len(collections) } + + progress(1) } - return missingCollections, incompleteBlocksCount, nil + if requestedBlocks > 0 { + s.log.Warn(). + Uint64("finalized_height", lastFinalizedBlock.Height). + Uint64("last_full_block_height", lastFullBlockHeight). + Int("requested_block_count", requestedBlocks). + Int("requested_collection_count", requestedCollections). + Msg("re-requesting missing collections") + } + + return nil } -// requestMissingCollectionsBlocking requests and waits for all missing collections to be downloaded, -// blocking until either completion or context timeout. +// requestForHeight requests all missing collections for the given height. +// If collections are available from execution data, they are indexed first. All other collections +// are requested from the network if `requestFromNetwork` is true. +// Returns true if the collections were missing and requested. +// Returns the list of collections that were requested from the network. // // No error returns are expected during normal operation. -func (s *Syncer) requestMissingCollectionsBlocking(ctx context.Context) error { - lastFullBlockHeight := s.lastFullBlockHeight.Value() - lastFinalizedBlock, err := s.state.Final().Head() +func (s *Syncer) requestForHeight(ctx context.Context, height uint64, requestFromNetwork bool) ([]*flow.CollectionGuarantee, bool, error) { + collections, err := s.indexer.MissingCollectionsAtHeight(height) if err != nil { - return fmt.Errorf("failed to get finalized block: %w", err) + return nil, false, fmt.Errorf("failed to find missing collections at height %d: %w", height, err) + } + if len(collections) == 0 { + return nil, false, nil } - progress := util.LogProgress(s.log, util.DefaultLogProgressConfig("requesting missing collections", lastFinalizedBlock.Height-lastFullBlockHeight)) - - pendingCollections := make(map[flow.Identifier]struct{}) - for height := lastFullBlockHeight + 1; height <= lastFinalizedBlock.Height; height++ { - if ctx.Err() != nil { - return fmt.Errorf("missing collection catchup interrupted: %w", ctx.Err()) - } - - collections, err := s.indexer.MissingCollectionsAtHeight(height) + // always index available collections from execution data. + if s.execDataSyncer != nil { + submitted, err := s.execDataSyncer.IndexForHeight(ctx, height) if err != nil { - return fmt.Errorf("failed to find missing collections at height %d: %w", height, err) + return nil, false, fmt.Errorf("failed to index collections from execution data: %w", err) } - - if len(collections) > 0 { - var submitted bool - if s.execDataSyncer != nil { - submitted, err = s.execDataSyncer.IndexForHeight(ctx, height) - if err != nil { - return fmt.Errorf("failed to index collections from execution data: %w", err) - } - } - - // if the data wasn't available from execution data, request it from Collection nodes. - if !submitted { - err = s.requestCollections(collections) - if err != nil { - return fmt.Errorf("failed to request collections: %w", err) - } - for _, collection := range collections { - pendingCollections[collection.CollectionID] = struct{}{} - } - } + if submitted { + return nil, true, nil } - - progress(1) } - if len(pendingCollections) == 0 { - s.log.Info().Msg("no missing collections to download") - return nil + // only request collections from the network if asked to do so. + if !requestFromNetwork { + return nil, false, nil } - // trigger immediate dispatch of any pending collection requests. - s.requester.Force() - - collectionStoragePollTicker := time.NewTicker(s.collectionCatchupDBPollInterval) - defer collectionStoragePollTicker.Stop() - - // we want to wait for all collections to be downloaded so we poll local storage periodically to - // make sure each collection was successfully stored and indexed. - for len(pendingCollections) > 0 { - select { - case <-ctx.Done(): - return fmt.Errorf("failed to complete collection retrieval: %w", ctx.Err()) - - case <-collectionStoragePollTicker.C: - s.log.Debug(). - Int("total_missing_collections", len(pendingCollections)). - Msg("retrieving missing collections...") - - for collectionID := range pendingCollections { - downloaded, err := s.indexer.IsCollectionInStorage(collectionID) - if err != nil { - return err - } - - if downloaded { - delete(pendingCollections, collectionID) - } - } - } + err = s.requestCollections(collections) + if err != nil { + return nil, false, fmt.Errorf("failed to request collections: %w", err) } + // request made during catchup do not use Force() so they can be batched together for efficiency. + // In practice, Force() is called once for each finalized block, so requests are dispatched at + // least as often as the block rate. - s.log.Info().Msg("collection catchup done") - return nil + return collections, true, nil } diff --git a/engine/access/ingestion/collections/syncer_execution_data.go b/engine/access/ingestion/collections/syncer_execution_data.go index 1be0ac55fda..c8a2086f8e9 100644 --- a/engine/access/ingestion/collections/syncer_execution_data.go +++ b/engine/access/ingestion/collections/syncer_execution_data.go @@ -7,30 +7,36 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" + "github.com/rs/zerolog" ) // ExecutionDataSyncer submits collections from execution data to the collections indexer. It is // designed to be used within the collection syncer to optimize indexing when collection data is // already available on the node. type ExecutionDataSyncer struct { + log zerolog.Logger executionDataCache execution_data.ExecutionDataCache indexer CollectionIndexer } func NewExecutionDataSyncer( + log zerolog.Logger, executionDataCache execution_data.ExecutionDataCache, indexer CollectionIndexer, ) *ExecutionDataSyncer { return &ExecutionDataSyncer{ + log: log.With().Str("component", "execution-data-syncer").Logger(), executionDataCache: executionDataCache, indexer: indexer, } } -// IndexForHeight indexes the collections for a given height using locally available execution data. +// IndexForHeight indexes the collections for a given finalized block height using locally available +// execution data. // Returns false and no error if execution data for the block is not available. // -// No error returns are expected during normal operation. +// Expected error returns during normal operation: +// - [context.Canceled]: if the context is canceled before the collections are indexed. func (s *ExecutionDataSyncer) IndexForHeight(ctx context.Context, height uint64) (bool, error) { executionData, err := s.executionDataCache.ByHeight(ctx, height) if err != nil { @@ -40,31 +46,19 @@ func (s *ExecutionDataSyncer) IndexForHeight(ctx context.Context, height uint64) return false, fmt.Errorf("failed to get execution data for height %d: %w", height, err) } - // index all collections except for the system chunk. - for _, chunkData := range executionData.ChunkExecutionDatas[:len(executionData.ChunkExecutionDatas)-1] { - s.indexer.OnCollectionReceived(chunkData.Collection) - } - - return true, nil -} - -// IndexFromStartHeight indexes the collections for all blocks with available execution data starting -// from the last full block height. Returns the last indexed height. -// -// No error returns are expected during normal operation. -func (s *ExecutionDataSyncer) IndexFromStartHeight(ctx context.Context, lastFullBlockHeight uint64) (uint64, error) { - lastIndexedHeight := lastFullBlockHeight - height := lastFullBlockHeight + 1 - for { - submitted, err := s.IndexForHeight(ctx, height) + // index all standard (non-system) collections. + standardCollections := executionData.StandardCollections() + if len(standardCollections) > 0 { + err = s.indexer.IndexCollections(standardCollections) if err != nil { - return 0, err + return false, fmt.Errorf("failed to index collections from execution data for height %d: %w", height, err) } - if !submitted { - return lastIndexedHeight, nil - } - - lastIndexedHeight = height - height++ } + + s.log.Debug(). + Uint64("height", height). + Int("collection_count", len(standardCollections)). + Msg("indexed collections from execution data") + + return true, nil } diff --git a/engine/access/ingestion/collections/syncer_test.go b/engine/access/ingestion/collections/syncer_test.go index be744da5df0..4ee0759a681 100644 --- a/engine/access/ingestion/collections/syncer_test.go +++ b/engine/access/ingestion/collections/syncer_test.go @@ -3,6 +3,7 @@ package collections import ( "context" "errors" + "fmt" "testing" "testing/synctest" "time" @@ -22,6 +23,7 @@ import ( modulemock "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/module/signature" protocolmock "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/utils/unittest" "github.com/onflow/flow-go/utils/unittest/fixtures" @@ -60,6 +62,7 @@ func (s *SyncerSuite) SetupTest() { func (s *SyncerSuite) createSyncer() *Syncer { execDataSyncer := NewExecutionDataSyncer( + unittest.Logger(), s.executionDataCache, s.indexer, ) @@ -188,19 +191,6 @@ func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_CollectionNodesOnly() { } } - // called once for the batch - s.requester.On("Force").Once() - - for _, guarantees := range guaranteesByHeight { - for _, guarantee := range guarantees { - // simulate the collection being missing by randomly returning false. All collections should - // eventually be found in storage. - s.indexer.On("IsCollectionInStorage", guarantee.CollectionID).Return(func(collectionID flow.Identifier) (bool, error) { - return g.Random().Bool(), nil - }) - } - } - syncer := NewSyncer( unittest.Logger(), s.requester, @@ -276,10 +266,7 @@ func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_SplitExecutionDataAndCollect s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() - - for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { - s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() - } + s.indexer.On("IndexCollections", execData.StandardCollections()).Return(nil).Once() } // the syncer should request collections from collection nodes for all remaining heights. @@ -297,19 +284,6 @@ func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_SplitExecutionDataAndCollect } } - // called once for the batch - s.requester.On("Force").Once() - - for _, guarantees := range guaranteesByHeight { - for _, guarantee := range guarantees { - // simulate the collection being missing by randomly returning false. All collections should - // eventually be found in storage. - s.indexer.On("IsCollectionInStorage", guarantee.CollectionID).Return(func(collectionID flow.Identifier) (bool, error) { - return g.Random().Bool(), nil - }) - } - } - syncer := s.createSyncer() synctest.Test(s.T(), func(t *testing.T) { @@ -373,9 +347,7 @@ func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_AllAvailableFromExecutionDat s.indexer.On("MissingCollectionsAtHeight", height).Return(guarantees, nil).Once() s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() - for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { - s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() - } + s.indexer.On("IndexCollections", execData.StandardCollections()).Return(nil).Once() } syncer := s.createSyncer() @@ -406,14 +378,14 @@ func (s *SyncerSuite) TestWorkerLoop_InitialCatchup_Timesout() { g := fixtures.NewGeneratorSuite() synctest.Test(s.T(), func(t *testing.T) { - finalizedHeight := s.lastFullBlockHeight.Value() + 10 + finalizedHeight := s.lastFullBlockHeight.Value() + 1 finalizedHeader := g.Headers().Fixture(fixtures.Header.WithHeight(finalizedHeight)) finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() s.state.On("Final").Return(finalSnapshot).Once() - // block the first call to MissingCollectionsAtHeight that is called during startup until after - // the timeout. This simulates the catchup logic taking too long. + // block the first call to MissingCollectionsAtHeight during startup until after the timeout. + // This simulates the catchup logic taking too long. unblockStartup := make(chan struct{}) s.indexer. On("MissingCollectionsAtHeight", s.lastFullBlockHeight.Value()+1). @@ -470,7 +442,7 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { } s.Run("no missing collections", func() { - s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + s.runWorkerLoopMissingCollections(g, nil, nil, func(syncer *Syncer) { finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() s.state.On("Final").Return(finalSnapshot).Once() @@ -482,7 +454,7 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { }) s.Run("missing collections - request skipped below thresholds", func() { - s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { + s.runWorkerLoopMissingCollections(g, nil, nil, func(syncer *Syncer) { finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() s.state.On("Final").Return(finalSnapshot).Once() @@ -493,9 +465,9 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { }) }) - s.Run("missing collections - request sent when count exceeds missingCollsForBlockThreshold", func() { - s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { - syncer.missingCollsForBlockThreshold = 9 + s.Run("missing collections - request sent when missing blocks exceeds missingCollectionRequestThreshold", func() { + s.runWorkerLoopMissingCollections(g, nil, nil, func(syncer *Syncer) { + syncer.missingCollectionRequestThreshold = 9 finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() @@ -503,6 +475,7 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + for _, guarantee := range allGuarantees[height] { s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) @@ -511,52 +484,94 @@ func (s *SyncerSuite) TestWorkerLoop_RequestsMissingCollections() { }) }) - s.Run("missing collections - request sent when age exceeds missingCollsForAgeThreshold", func() { - s.runWorkerLoopMissingCollections(g, nil, func(syncer *Syncer) { - syncer.missingCollsForAgeThreshold = 9 + s.Run("missing collections - processed from execution data", func() { + execDataSyncer := NewExecutionDataSyncer( + unittest.Logger(), + s.executionDataCache, + s.indexer, + ) + s.runWorkerLoopMissingCollections(g, execDataSyncer, nil, func(syncer *Syncer) { finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() s.state.On("Final").Return(finalSnapshot).Once() - for height := s.lastFullBlockHeight.Value() + 1; height <= finalizedHeight; height++ { + for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() - } - for height := s.lastFullBlockHeight.Value() + 1; height <= finalizedHeight; height++ { - for _, guarantee := range allGuarantees[height] { - s.requester.On("EntityByID", guarantee.CollectionID, mock.Anything).Once() - s.mockGuarantorsForCollection(guarantee, guarantors.ToSkeleton()) - } + execData, _ := executionDataFixture(g) + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() + s.indexer.On("IndexCollections", execData.StandardCollections()).Return(nil).Once() } }) }) - s.Run("missing collections - processed from execution data", func() { + s.Run("handles context cancelation gracefully", func() { execDataSyncer := NewExecutionDataSyncer( + unittest.Logger(), s.executionDataCache, s.indexer, ) - s.runWorkerLoopMissingCollections(g, execDataSyncer, func(syncer *Syncer) { + s.runWorkerLoopMissingCollections(g, execDataSyncer, nil, func(syncer *Syncer) { finalSnapshot := protocolmock.NewSnapshot(s.T()) finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() s.state.On("Final").Return(finalSnapshot).Once() - for height := lastFullBlockHeight + 1; height <= finalizedHeight; height++ { - execData, _ := executionDataFixture(g) - s.executionDataCache.On("ByHeight", mock.Anything, height).Return(execData, nil).Once() - for _, chunkData := range execData.ChunkExecutionDatas[:len(execData.ChunkExecutionDatas)-1] { - s.indexer.On("OnCollectionReceived", chunkData.Collection).Once() - } - } - // syncer continues until it receives a not found error. - s.executionDataCache.On("ByHeight", mock.Anything, finalizedHeight+1).Return(nil, execution_data.NewBlobNotFoundError(cid.Cid{})).Once() + height := lastFullBlockHeight + 1 + s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + s.executionDataCache.On("ByHeight", mock.Anything, height).Return(nil, context.Canceled).Once() + + // should exit without error + }) + }) + + s.Run("throws exception if finalized block is not found", func() { + exception := fmt.Errorf("finalized block not found: %w", storage.ErrNotFound) + s.runWorkerLoopMissingCollections(g, nil, exception, func(syncer *Syncer) { + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(nil, exception).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + // should throw exception + }) + }) + + s.Run("throws exception if MissingCollectionsAtHeight returns an error", func() { + exception := fmt.Errorf("missing collections at height: %w", storage.ErrNotFound) + s.runWorkerLoopMissingCollections(g, nil, exception, func(syncer *Syncer) { + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + height := s.lastFullBlockHeight.Value() + 1 + s.indexer.On("MissingCollectionsAtHeight", height).Return(nil, exception).Once() + + // should throw exception + }) + }) + + s.Run("throws exception if requestCollections returns an error", func() { + exception := fmt.Errorf("request collections: %w", storage.ErrNotFound) + s.runWorkerLoopMissingCollections(g, nil, exception, func(syncer *Syncer) { + syncer.missingCollectionRequestThreshold = 9 // make sure the request is sent + + finalSnapshot := protocolmock.NewSnapshot(s.T()) + finalSnapshot.On("Head").Return(finalizedHeader, nil).Once() + s.state.On("Final").Return(finalSnapshot).Once() + + height := s.lastFullBlockHeight.Value() + 1 + guarantee := allGuarantees[height][0] + s.indexer.On("MissingCollectionsAtHeight", height).Return(allGuarantees[height], nil).Once() + + s.mockGuarantorsForCollectionReturnsError(guarantee, exception) + + // should throw exception }) }) } -func (s *SyncerSuite) runWorkerLoopMissingCollections(g *fixtures.GeneratorSuite, execDataSyncer *ExecutionDataSyncer, onReady func(*Syncer)) { +func (s *SyncerSuite) runWorkerLoopMissingCollections(g *fixtures.GeneratorSuite, execDataSyncer *ExecutionDataSyncer, expectedError error, onReady func(*Syncer)) { synctest.Test(s.T(), func(t *testing.T) { // last full block is latest finalized block, so initial catchup is skipped finalizedHeight := s.lastFullBlockHeight.Value() @@ -575,14 +590,23 @@ func (s *SyncerSuite) runWorkerLoopMissingCollections(g *fixtures.GeneratorSuite execDataSyncer, ) - ctx, cancel := irrecoverable.NewMockSignalerContextWithCancel(s.T(), context.Background()) + ctx, cancel := context.WithCancel(context.Background()) + + var signalerCtx irrecoverable.SignalerContext + if expectedError == nil { + signalerCtx = irrecoverable.NewMockSignalerContext(s.T(), ctx) + } else { + signalerCtx = irrecoverable.NewMockSignalerContextWithCallback(s.T(), ctx, func(err error) { + s.Require().ErrorIs(err, expectedError) + }) + } done := make(chan struct{}) ready := make(chan struct{}) go func() { defer close(done) - syncer.WorkerLoop(ctx, func() { + syncer.WorkerLoop(signalerCtx, func() { onReady(syncer) close(ready) }) @@ -590,7 +614,7 @@ func (s *SyncerSuite) runWorkerLoopMissingCollections(g *fixtures.GeneratorSuite <-ready - time.Sleep(syncer.missingCollsRequestInterval + 1) + time.Sleep(syncer.missingCollectionRequestInterval + 1) synctest.Wait() cancel() diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index 808b90b0690..d3e0e62fce1 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -183,6 +183,7 @@ func (s *Suite) initEngineAndSyncer() (*Engine, *collections.Syncer, *collection indexer, err := collections.NewIndexer( s.log, + s.db, s.collectionExecutedMetric, s.proto.state, s.blocks, diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go index 8067ac3c4cd..7e22207b5ac 100644 --- a/engine/access/ingestion2/engine_test.go +++ b/engine/access/ingestion2/engine_test.go @@ -211,6 +211,7 @@ func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, indexer, err := collections.NewIndexer( s.log, + s.db, s.collectionExecutedMetric, s.proto.state, s.blocks, diff --git a/module/executiondatasync/execution_data/execution_data.go b/module/executiondatasync/execution_data/execution_data.go index 46f6e741738..d0645329d3d 100644 --- a/module/executiondatasync/execution_data/execution_data.go +++ b/module/executiondatasync/execution_data/execution_data.go @@ -68,6 +68,31 @@ type BlockExecutionData struct { ChunkExecutionDatas []*ChunkExecutionData } +// StandardChunks returns all standard (non-system) chunks for the block. +func (bd *BlockExecutionData) StandardChunks() []*ChunkExecutionData { + return bd.ChunkExecutionDatas[:len(bd.ChunkExecutionDatas)-1] +} + +// StandardCollections returns all standard (non-system) collections for the block. +func (bd *BlockExecutionData) StandardCollections() []*flow.Collection { + standardChunks := bd.StandardChunks() + collections := make([]*flow.Collection, len(standardChunks)) + for i, chunk := range standardChunks { + collections[i] = chunk.Collection + } + return collections +} + +// SystemChunk returns the system chunk for the block. +func (bd *BlockExecutionData) SystemChunk() *ChunkExecutionData { + return bd.ChunkExecutionDatas[len(bd.ChunkExecutionDatas)-1] +} + +// SystemCollection returns the system collection for the block. +func (bd *BlockExecutionData) SystemCollection() *flow.Collection { + return bd.SystemChunk().Collection +} + // ConvertTransactionResults converts a list of flow.TransactionResults into a list of // flow.LightTransactionResults to be included in a ChunkExecutionData. func ConvertTransactionResults(results flow.TransactionResults) []flow.LightTransactionResult { diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index b0e0d2fdbaa..b2d92e65e69 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -228,18 +228,17 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti // than the latest indexed block. Calling the collection handler with a collection that // has already been indexed is a noop. - // index all collections except the system chunk. if there is only a single chunk, it is the - // system chunk and can be skipped. - indexedCount := 0 - if len(data.ChunkExecutionDatas) > 1 { - for _, chunk := range data.ChunkExecutionDatas[0 : len(data.ChunkExecutionDatas)-1] { - c.collectionIndexer.OnCollectionReceived(chunk.Collection) - indexedCount++ + // index all standard (non-system) collections + standardCollections := data.StandardCollections() + if len(standardCollections) > 0 { + err := c.collectionIndexer.IndexCollections(standardCollections) + if err != nil { + return fmt.Errorf("could not index collections: %w", err) } } lg.Debug(). - Int("collection_count", indexedCount). + Int("collection_count", len(standardCollections)). Dur("duration_ms", time.Since(start)). Msg("indexed collections") From dc48833b8a4d9022f9d048c7bd2efcedb6946701 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 15:25:04 -0800 Subject: [PATCH 11/16] fix access_test --- engine/access/access_test.go | 7 +++++-- engine/access/ingestion/collections/syncer.go | 6 +++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/engine/access/access_test.go b/engine/access/access_test.go index 15beb15953e..21fcf1107ff 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -726,6 +726,7 @@ func (suite *Suite) TestGetSealedTransaction() { collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, + db, collectionExecutedMetric, suite.state, all.Blocks, @@ -816,8 +817,8 @@ func (suite *Suite) TestGetSealedTransaction() { // block until the collection is processed by the indexer require.Eventually(suite.T(), func() bool { - isStored, err := collectionIndexer.IsCollectionInStorage(collection.ID()) - return isStored && err == nil + _, err := collections.LightByID(collection.ID()) + return err == nil }, 1*time.Second, 10*time.Millisecond, "collection not indexed") // 5. Client requests a transaction @@ -984,6 +985,7 @@ func (suite *Suite) TestGetTransactionResult() { collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, + db, collectionExecutedMetric, suite.state, all.Blocks, @@ -1259,6 +1261,7 @@ func (suite *Suite) TestExecuteScript() { collectionIndexer, err := ingestioncollections.NewIndexer( suite.log, + db, collectionExecutedMetric, suite.state, all.Blocks, diff --git a/engine/access/ingestion/collections/syncer.go b/engine/access/ingestion/collections/syncer.go index 996d5c8b509..53007147fe7 100644 --- a/engine/access/ingestion/collections/syncer.go +++ b/engine/access/ingestion/collections/syncer.go @@ -212,10 +212,14 @@ func (s *Syncer) requestMissingCollections(ctx context.Context, forceSendRequest return fmt.Errorf("failed to get finalized block: %w", err) } + if lastFullBlockHeight >= lastFinalizedBlock.Height { + return nil + } + // only send requests if we are sufficiently behind the latest finalized block to avoid spamming // collection nodes with requests. shouldSendRequestsToNetwork := forceSendRequests - if lastFullBlockHeight < lastFinalizedBlock.Height && (lastFinalizedBlock.Height-lastFullBlockHeight >= s.missingCollectionRequestThreshold) { + if lastFinalizedBlock.Height-lastFullBlockHeight >= s.missingCollectionRequestThreshold { shouldSendRequestsToNetwork = true } From e4f04fe9d2657fc1294c1c658c169595acbd6053 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 15:26:43 -0800 Subject: [PATCH 12/16] fix lint --- engine/access/ingestion/collections/syncer_execution_data.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/engine/access/ingestion/collections/syncer_execution_data.go b/engine/access/ingestion/collections/syncer_execution_data.go index c8a2086f8e9..92fc55f375c 100644 --- a/engine/access/ingestion/collections/syncer_execution_data.go +++ b/engine/access/ingestion/collections/syncer_execution_data.go @@ -5,9 +5,10 @@ import ( "errors" "fmt" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" - "github.com/rs/zerolog" ) // ExecutionDataSyncer submits collections from execution data to the collections indexer. It is From 4dde3b62ce5c5e9b9b22e81a02dd5910f071067b Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 15:42:38 -0800 Subject: [PATCH 13/16] add blockexecutiondata tests, fix indexer tests --- .../execution_data/execution_data_test.go | 34 +++++++++++++++++++ .../indexer/indexer_core_test.go | 4 +-- .../indexer/indexer_test.go | 4 +-- 3 files changed, 37 insertions(+), 5 deletions(-) diff --git a/module/executiondatasync/execution_data/execution_data_test.go b/module/executiondatasync/execution_data/execution_data_test.go index 37cd18a18c6..c3595abe614 100644 --- a/module/executiondatasync/execution_data/execution_data_test.go +++ b/module/executiondatasync/execution_data/execution_data_test.go @@ -7,8 +7,42 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/utils/unittest/fixtures" ) +func TestBlockExecutionDataHelpers(t *testing.T) { + g := fixtures.NewGeneratorSuite() + chunkDatas := g.ChunkExecutionDatas().List(5) + blockData := g.BlockExecutionDatas().Fixture( + fixtures.BlockExecutionData.WithChunkExecutionDatas(chunkDatas...), + ) + + expectedStardardChunks := chunkDatas[:len(chunkDatas)-1] + expectedStandardCollections := make([]*flow.Collection, len(expectedStardardChunks)) + for i, chunk := range expectedStardardChunks { + expectedStandardCollections[i] = chunk.Collection + } + + expectedSystemChunk := chunkDatas[len(chunkDatas)-1] + expectedSystemCollection := expectedSystemChunk.Collection + + t.Run("StandardChunks", func(t *testing.T) { + assert.Equal(t, blockData.StandardChunks(), expectedStardardChunks) + }) + + t.Run("StandardCollections", func(t *testing.T) { + assert.Equal(t, blockData.StandardCollections(), expectedStandardCollections) + }) + + t.Run("SystemChunk", func(t *testing.T) { + assert.Equal(t, blockData.SystemChunk(), expectedSystemChunk) + }) + + t.Run("SystemCollection", func(t *testing.T) { + assert.Equal(t, blockData.SystemCollection(), expectedSystemCollection) + }) +} + func TestConvertTransactionResults(t *testing.T) { t.Parallel() diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index 7d7fb0b8f63..45e97f69de1 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -250,9 +250,7 @@ func TestExecutionState_IndexBlockData(t *testing.T) { assert.ElementsMatch(t, tf.ExpectedRegisterEntries, entries) }). Return(nil) - for _, collection := range tf.ExpectedCollections { - test.collectionIndexer.On("OnCollectionReceived", collection).Once() - } + 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 { diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index a17e45a3bf2..7efd98f680f 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -201,7 +201,7 @@ func TestIndexer_Success(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("OnCollectionReceived", collection).Return(nil).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 +243,7 @@ func TestIndexer_Failure(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("OnCollectionReceived", collection).Return(nil).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 142556f4864bf9e88785b5b27721512bb2fb29ed Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 17:56:21 -0800 Subject: [PATCH 14/16] fix integration test --- .../access/cohort3/collection_indexing_test.go | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/integration/tests/access/cohort3/collection_indexing_test.go b/integration/tests/access/cohort3/collection_indexing_test.go index 97eee746be5..230155efadf 100644 --- a/integration/tests/access/cohort3/collection_indexing_test.go +++ b/integration/tests/access/cohort3/collection_indexing_test.go @@ -3,6 +3,7 @@ package cohort3 import ( "context" "fmt" + "net/http" "testing" "time" @@ -82,7 +83,7 @@ func (s *CollectionIndexingSuite) Test() { // start the network with access_2 disconnected. // this simulates it falling behind on syncing collections access2 := s.net.ContainerByName("access_2") - s.Require().NoError(access2.Disconnect()) + s.Require().NoError(access2.Pause()) // wait for access_1 to sync collections targetBlockCount := uint64(50) @@ -98,7 +99,10 @@ func (s *CollectionIndexingSuite) Test() { s.Require().NoError(s.net.ContainerByName("collection_2").Pause()) // now start access_2, and wait for it to catch up with collections - s.Require().NoError(access2.Connect()) + s.Require().NoError(access2.Start()) + + // wait for access_2 to complete startup + s.waitForMetricsAPI("access_2") s.Eventually(func() bool { value, err := s.getLastFullHeight("access_2") @@ -118,3 +122,13 @@ func (s *CollectionIndexingSuite) getLastFullHeight(containerName string) (uint6 return uint64(values[0].GetGauge().GetValue()), nil } + +func (s *CollectionIndexingSuite) waitForMetricsAPI(containerName string) { + node := s.net.ContainerByName(containerName) + metricsURL := fmt.Sprintf("http://0.0.0.0:%s/metrics", node.Port(testnet.MetricsPort)) + + s.Require().Eventually(func() bool { + _, err := http.Get(metricsURL) + return err == nil + }, 60*time.Second, 1*time.Second) +} From 4fc5a59b82918848f1955c5e33cbe1c298534d5c Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 17:57:02 -0800 Subject: [PATCH 15/16] fix lint --- engine/access/ingestion/collections/syncer_execution_data.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/engine/access/ingestion/collections/syncer_execution_data.go b/engine/access/ingestion/collections/syncer_execution_data.go index c8a2086f8e9..92fc55f375c 100644 --- a/engine/access/ingestion/collections/syncer_execution_data.go +++ b/engine/access/ingestion/collections/syncer_execution_data.go @@ -5,9 +5,10 @@ import ( "errors" "fmt" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" - "github.com/rs/zerolog" ) // ExecutionDataSyncer submits collections from execution data to the collections indexer. It is From e8670013ea151109c8235a13ef72bd785209f186 Mon Sep 17 00:00:00 2001 From: Peter Argue <89119817+peterargue@users.noreply.github.com> Date: Tue, 4 Nov 2025 18:20:31 -0800 Subject: [PATCH 16/16] backport change to reduce log level of exec data sync --- .../state_synchronization/requester/execution_data_requester.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index c6c04823f01..35bb21a08a9 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -410,7 +410,7 @@ func (e *executionDataRequester) processFetchRequest(parentCtx irrecoverable.Sig parentCtx.Throw(err) } - logger.Info(). + logger.Debug(). Hex("execution_data_id", logging.ID(execData.ExecutionDataID)). Msg("execution data fetched")