diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5ac66c3d726..a0a2ae7dc2b 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -39,9 +39,13 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/verification" recovery "github.com/onflow/flow-go/consensus/recovery/protocol" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/factory" + collection_syncfactory "github.com/onflow/flow-go/engine/access/collection_sync/factory" + collection_syncfetcher "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" + collsyncindexer "github.com/onflow/flow-go/engine/access/collection_sync/indexer" + "github.com/onflow/flow-go/engine/access/finalized_indexer" "github.com/onflow/flow-go/engine/access/index" - "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" @@ -59,7 +63,6 @@ import ( "github.com/onflow/flow-go/engine/access/subscription" subscriptiontracker "github.com/onflow/flow-go/engine/access/subscription/tracker" followereng "github.com/onflow/flow-go/engine/common/follower" - "github.com/onflow/flow-go/engine/common/requester" commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/stop" synceng "github.com/onflow/flow-go/engine/common/synchronization" @@ -75,7 +78,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/chainsync" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/executiondatasync/execution_data" execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" @@ -182,6 +184,10 @@ type AccessNodeConfig struct { storeTxResultErrorMessages bool stopControlEnabled bool registerDBPruneThreshold uint64 + collectionFetcherMaxProcessing uint64 + collectionFetcherMaxSearchAhead uint64 + collectionFetcherRetryInterval time.Duration + collectionSync collection_syncfactory.CollectionSyncMode } type PublicNetworkConfig struct { @@ -289,6 +295,10 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { storeTxResultErrorMessages: false, stopControlEnabled: false, registerDBPruneThreshold: 0, + collectionFetcherMaxProcessing: factory.DefaultMaxProcessing, + collectionFetcherMaxSearchAhead: factory.DefaultMaxSearchAhead, + collectionFetcherRetryInterval: collection_syncfetcher.DefaultRetryInterval, + collectionSync: collection_syncfactory.CollectionSyncModeExecutionAndCollection, } } @@ -315,6 +325,7 @@ type FlowAccessNodeBuilder struct { RestMetrics *metrics.RestCollector AccessMetrics module.AccessMetrics PingMetrics module.PingMetrics + CollectionSyncMetrics module.CollectionSyncMetrics Committee hotstuff.DynamicCommittee Finalized *flow.Header // latest finalized block that the node knows of at startup time Pending []*flow.ProposalHeader @@ -323,13 +334,13 @@ type FlowAccessNodeBuilder struct { ExecutionDataDownloader execution_data.Downloader PublicBlobService network.BlobService ExecutionDataRequester state_synchronization.ExecutionDataRequester + ExecutionDataDistributor *edrequester.ExecutionDataDistributor ExecutionDataStore execution_data.ExecutionDataStore ExecutionDataBlobstore blobs.Blobstore ExecutionDataCache *execdatacache.ExecutionDataCache ExecutionIndexer *indexer.Indexer ExecutionIndexerCore *indexer.IndexerCore - CollectionIndexer *collections.Indexer - CollectionSyncer *collections.Syncer + blockCollectionIndexer collection_sync.BlockCollectionIndexer ScriptExecutor *backend.ScriptExecutor RegistersAsyncStore *execution.RegistersAsyncStore Reporter *index.Reporter @@ -347,8 +358,6 @@ type FlowAccessNodeBuilder struct { events storage.Events lightTransactionResults storage.LightTransactionResults transactionResultErrorMessages storage.TransactionResultErrorMessages - transactions storage.Transactions - collections storage.Collections scheduledTransactions storage.ScheduledTransactions // The sync engine participants provider is the libp2p peer store for the access node @@ -357,12 +366,13 @@ type FlowAccessNodeBuilder struct { SyncEngineParticipantsProviderFactory func() module.IdentifierProvider // engines - IngestEng *ingestion.Engine - RequestEng *requester.Engine FollowerEng *followereng.ComplianceEngine - SyncEng *synceng.Engine StateStreamEng *statestreambackend.Engine + // for tx status deriver to know about the highest full block (a block with all collections synced) + // backed by either collection fetcher to execution data syncing + lastFullBlockHeight *collection_syncfactory.ProgressReader + // grpc servers secureGrpcServer *grpcserver.GrpcServer unsecureGrpcServer *grpcserver.GrpcServer @@ -372,7 +382,6 @@ type FlowAccessNodeBuilder struct { nodeBackend *backend.Backend ExecNodeIdentitiesProvider *commonrpc.ExecutionNodeIdentitiesProvider - TxResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore txResultErrorMessageProvider error_messages.Provider } @@ -537,9 +546,8 @@ func (builder *FlowAccessNodeBuilder) buildSyncEngine() *FlowAccessNodeBuilder { if err != nil { return nil, fmt.Errorf("could not create synchronization engine: %w", err) } - builder.SyncEng = sync - return builder.SyncEng, nil + return sync, nil }) return builder @@ -561,7 +569,6 @@ func (builder *FlowAccessNodeBuilder) BuildConsensusFollower() *FlowAccessNodeBu func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccessNodeBuilder { var bs network.BlobService var processedBlockHeight storage.ConsumerProgressInitializer - var processedNotifications storage.ConsumerProgressInitializer var bsDependable *module.ProxiedReadyDoneAware var execDataDistributor *edrequester.ExecutionDataDistributor var execDataCacheBackend *herocache.BlockExecutionData @@ -576,14 +583,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess AdminCommand("read-execution-data", func(config *cmd.NodeConfig) commands.AdminCommand { return stateSyncCommands.NewReadExecutionDataCommand(builder.ExecutionDataStore) }). - Module("transactions and collections storage", func(node *cmd.NodeConfig) error { - transactions := store.NewTransactions(node.Metrics.Cache, node.ProtocolDB) - collections := store.NewCollections(node.ProtocolDB, transactions) - builder.transactions = transactions - builder.collections = collections - - return nil - }). Module("execution data datastore and blobstore", func(node *cmd.NodeConfig) error { var err error builder.ExecutionDatastoreManager, err = edstorage.CreateDatastoreManager( @@ -604,13 +603,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess processedBlockHeight = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterBlockHeight) return nil }). - Module("processed notifications consumer progress", func(node *cmd.NodeConfig) error { - // Note: progress is stored in the datastore's DB since that is where the jobqueue - // writes execution data to. - db := builder.ExecutionDatastoreManager.DB() - processedNotifications = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterNotification) - return nil - }). Module("blobservice peer manager dependencies", func(node *cmd.NodeConfig) error { bsDependable = module.NewProxiedReadyDoneAware() builder.PeerManagerDependencies.Add(bsDependable) @@ -659,6 +651,10 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess opts = append(opts, blob.WithReprovideInterval(-1)) } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.ExecutionDataService, builder.ExecutionDatastoreManager.Datastore(), opts...) if err != nil { @@ -732,6 +728,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } execDataDistributor = edrequester.NewExecutionDataDistributor() + builder.ExecutionDataDistributor = execDataDistributor // Execution Data cache with a downloader as the backend. This is used by the requester // to download and cache execution data for each block. It shares a cache backend instance @@ -750,7 +747,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.ExecutionDataDownloader, executionDataCache, processedBlockHeight, - processedNotifications, builder.State, builder.Storage.Headers, builder.executionDataConfig, @@ -945,13 +941,12 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess notNil(builder.Storage.RegisterIndex), notNil(builder.Storage.Headers), notNil(builder.events), - notNil(builder.collections), - notNil(builder.transactions), + notNil(builder.Storage.Collections), + notNil(builder.Storage.Transactions), notNil(builder.lightTransactionResults), notNil(builder.scheduledTransactions), builder.RootChainID, indexerDerivedChainData, - notNil(builder.CollectionIndexer), notNil(builder.collectionExecutedMetric), node.StorageLockMgr, ) @@ -1030,10 +1025,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } builder.stateStreamConf.RpcMetricsEnabled = builder.rpcMetricsEnabled - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } broadcaster := engine.NewBroadcaster() eventQueryMode, err := query_mode.ParseIndexQueryMode(builder.rpcConf.BackendConfig.EventQueryMode) @@ -1052,7 +1043,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.executionDataConfig.InitialBlockHeight, node.Storage.Headers, broadcaster, - highestAvailableHeight, + notNil(builder.ExecutionDataRequester), builder.EventsIndex, useIndex, ) @@ -1154,6 +1145,7 @@ func (builder *FlowAccessNodeBuilder) ParseFlags() error { } func (builder *FlowAccessNodeBuilder) extraFlags() { + var collectionSyncStr string builder.ExtraFlags(func(flags *pflag.FlagSet) { defaultConfig := DefaultAccessNodeConfig() @@ -1205,6 +1197,18 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "", defaultConfig.rpcConf.CollectionAddr, "the address (of the collection node) to send transactions to") + flags.Uint64Var(&builder.collectionFetcherMaxProcessing, + "collection-fetcher-max-processing", + defaultConfig.collectionFetcherMaxProcessing, + "maximum number of collection fetcher requests to process concurrently") + flags.Uint64Var(&builder.collectionFetcherMaxSearchAhead, + "collection-fetcher-max-search-ahead", + defaultConfig.collectionFetcherMaxSearchAhead, + "maximum number of blocks to search ahead when fetching collections") + flags.DurationVar(&builder.collectionFetcherRetryInterval, + "collection-fetcher-retry-interval", + defaultConfig.collectionFetcherRetryInterval, + "interval for retrying missing collections. default: 30s") flags.StringVarP(&builder.ExecutionNodeAddress, "script-addr", "s", @@ -1329,6 +1333,12 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "public-network-execution-data-sync-enabled", defaultConfig.publicNetworkExecutionDataEnabled, "[experimental] whether to enable the execution data sync protocol on public network") + flags.StringVar(&collectionSyncStr, + "collection-sync", + defaultConfig.collectionSync.String(), + "collection sync mode: 'execution_first' (only fetch from execution nodes if execution data syncing is on, "+ + "otherwise fetch from collection nodes), 'execution_and_collection' (fetch from both collection nodes and execution nodes), "+ + "'collection_only' (only fetch from collection nodes)") flags.StringVar(&builder.executionDataDir, "execution-data-dir", defaultConfig.executionDataDir, "directory to use for Execution Data database") flags.Uint64Var(&builder.executionDataStartHeight, "execution-data-start-height", @@ -1517,6 +1527,12 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { if builder.supportsObserver && (builder.PublicNetworkConfig.BindAddress == cmd.NotSet || builder.PublicNetworkConfig.BindAddress == "") { return errors.New("public-network-address must be set if supports-observer is true") } + // Parse collection sync mode from string flag + collectionSyncMode, err := collection_syncfactory.ParseCollectionSyncMode(collectionSyncStr) + if err != nil { + return err + } + builder.collectionSync = collectionSyncMode if builder.executionDataSyncEnabled { if builder.executionDataConfig.FetchTimeout <= 0 { return errors.New("execution-data-fetch-timeout must be greater than 0") @@ -1678,7 +1694,8 @@ func (builder *FlowAccessNodeBuilder) Initialize() error { builder.EnqueueNetworkInit() builder.AdminCommand("get-transactions", func(conf *cmd.NodeConfig) commands.AdminCommand { - return storageCommands.NewGetTransactionsCommand(conf.State, conf.Storage.Payloads, notNil(builder.collections)) + return storageCommands.NewGetTransactionsCommand(conf.State, conf.Storage.Payloads, + notNil(builder.Storage.Collections)) }) // if this is an access node that supports public followers, enqueue the public network @@ -1718,20 +1735,15 @@ func (builder *FlowAccessNodeBuilder) enqueueRelayNetwork() { } func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { - var processedFinalizedBlockHeight storage.ConsumerProgressInitializer - var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer - if builder.executionDataSyncEnabled { builder.BuildExecutionSyncComponents() } - ingestionDependable := module.NewProxiedReadyDoneAware() - builder.IndexerDependencies.Add(ingestionDependable) versionControlDependable := module.NewProxiedReadyDoneAware() builder.IndexerDependencies.Add(versionControlDependable) stopControlDependable := module.NewProxiedReadyDoneAware() builder.IndexerDependencies.Add(stopControlDependable) - var lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + var collectionIndexedHeight storage.ConsumerProgress builder. BuildConsensusFollower(). @@ -1860,7 +1872,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.CollectionsToMarkFinalized, builder.CollectionsToMarkExecuted, builder.BlocksToMarkExecuted, - builder.collections, + builder.Storage.Collections, builder.Storage.Blocks, builder.BlockTransactions, ) @@ -1874,6 +1886,10 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.PingMetrics = metrics.NewPingCollector() return nil }). + Module("collection sync metrics", func(node *cmd.NodeConfig) error { + builder.CollectionSyncMetrics = metrics.NewCollectionSyncCollector() + return nil + }). Module("server certificate", func(node *cmd.NodeConfig) error { // generate the server certificate that will be served by the GRPC server x509Certificate, err := grpcutils.X509Certificate(node.NetworkKey) @@ -1943,23 +1959,26 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.lightTransactionResults) return nil }). - Module("processed finalized block height consumer progress", func(node *cmd.NodeConfig) error { - processedFinalizedBlockHeight = store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) + Module("block collection indexer", func(node *cmd.NodeConfig) error { + builder.blockCollectionIndexer = collsyncindexer.NewBlockCollectionIndexer( + notNil(builder.collectionExecutedMetric), + node.StorageLockMgr, + builder.ProtocolDB, + notNil(builder.Storage.Collections), + ) return nil }). Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { - rootBlockHeight := node.State.Params().FinalizedRoot().Height - - progress, err := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) + result, err := collection_syncfactory.CreateProcessedLastFullBlockHeightModule( + node.Logger, + node.State, + builder.ProtocolDB, + ) if err != nil { return err } - - lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(progress) - if err != nil { - return fmt.Errorf("failed to initialize monotonic consumer progress: %w", err) - } - + builder.lastFullBlockHeight = result.LastFullBlockHeight + collectionIndexedHeight = result.CollectionIndexedHeight return nil }). Component("version control", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -2087,33 +2106,15 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } - preferredENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.PreferredExecutionNodeIDs) - if err != nil { - return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for preferred EN map: %w", err) - } - - fixedENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.FixedExecutionNodeIDs) - if err != nil { - return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) - } - - builder.ExecNodeIdentitiesProvider = commonrpc.NewExecutionNodeIdentitiesProvider( - node.Logger, - node.State, - node.Storage.Receipts, - preferredENIdentifiers, - fixedENIdentifiers, - ) + // ExecNodeIdentitiesProvider is initialized in the "execution node identities provider" module + // to ensure it's available before modules that depend on it. Verify it's initialized here. + _ = notNil(builder.ExecNodeIdentitiesProvider) nodeCommunicator := node_communicator.NewNodeCommunicator(backendConfig.CircuitBreakerConfig.Enabled) - builder.txResultErrorMessageProvider = error_messages.NewTxErrorMessageProvider( - node.Logger, - builder.transactionResultErrorMessages, // might be nil - notNil(builder.TxResultsIndex), - connFactory, - nodeCommunicator, - notNil(builder.ExecNodeIdentitiesProvider), - ) + + // txResultErrorMessageProvider is initialized in the "transaction result error message provider" module + // to ensure it's available before modules that depend on it. Verify it's initialized here. + _ = notNil(builder.txResultErrorMessageProvider) builder.nodeBackend, err = backend.New(backend.Params{ State: node.State, @@ -2121,8 +2122,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { HistoricalAccessNodes: notNil(builder.HistoricalAccessRPCs), Blocks: node.Storage.Blocks, Headers: node.Storage.Headers, - Collections: notNil(builder.collections), - Transactions: notNil(builder.transactions), + Collections: node.Storage.Collections, + Transactions: node.Storage.Transactions, ExecutionReceipts: node.Storage.Receipts, ExecutionResults: node.Storage.Results, Seals: node.Storage.Seals, @@ -2151,7 +2152,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { EventsIndex: notNil(builder.EventsIndex), TxResultQueryMode: txResultQueryMode, TxResultsIndex: notNil(builder.TxResultsIndex), - LastFullBlockHeight: lastFullBlockHeight, + LastFullBlockHeight: notNil(builder.lastFullBlockHeight), IndexReporter: indexReporter, VersionControl: notNil(builder.VersionControl), ExecNodeIdentitiesProvider: notNil(builder.ExecNodeIdentitiesProvider), @@ -2192,112 +2193,109 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } return builder.RpcEng, nil - }). - Component("requester engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - requestEng, err := requester.New( - node.Logger.With().Str("entity", "collection").Logger(), - node.Metrics.Engine, - node.EngineRegistry, - node.Me, - node.State, - channels.RequestCollections, - filter.HasRole[flow.Identity](flow.RoleCollection), - func() flow.Entity { return new(flow.Collection) }, - ) - if err != nil { - return nil, fmt.Errorf("could not create requester engine: %w", err) - } - builder.RequestEng = requestEng + }) - collectionIndexer, err := collections.NewIndexer( - node.Logger, - builder.ProtocolDB, - notNil(builder.collectionExecutedMetric), - node.State, - node.Storage.Blocks, - notNil(builder.collections), - lastFullBlockHeight, - node.StorageLockMgr, - ) - if err != nil { - return nil, fmt.Errorf("could not create collection indexer: %w", err) - } - builder.CollectionIndexer = collectionIndexer + builder.Component("execution data processor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + processor, err := collection_syncfactory.CreateExecutionDataProcessorComponent( + builder.Logger, + builder.executionDataSyncEnabled, + builder.collectionSync, + notNil(builder.ExecutionDataCache), + notNil(builder.ExecutionDataRequester), + notNil(collectionIndexedHeight), + notNil(builder.blockCollectionIndexer), + notNil(builder.CollectionSyncMetrics), + notNil(builder.lastFullBlockHeight), + notNil(builder.AccessMetrics), + notNil(builder.ExecutionDataDistributor), + ) + if err != nil { + return nil, err + } + return processor, nil + }) - // the collection syncer has support for indexing collections from execution data if the - // syncer falls behind. This is only needed if the execution state indexing is disabled, - // since it will also index collections. - var executionDataSyncer *collections.ExecutionDataSyncer - if builder.executionDataSyncEnabled && !builder.executionDataIndexingEnabled { - executionDataSyncer = collections.NewExecutionDataSyncer( - node.Logger, - notNil(builder.ExecutionDataCache), - collectionIndexer, - ) - } + builder.Component("finalized block indexer", createFinalizedBlockIndexer(builder)) + // leo: ingest receipt allows AN to store receipts sent from the EN, + // but this is unnecessary, because access node can get the receipts from sealed block + // builder.Component("ingest receipt", createIngestReceiptEngine(builder)) + createCollectionSyncFetcher(builder) - collectionSyncer := collections.NewSyncer( - node.Logger, - builder.RequestEng, - node.State, - notNil(builder.collections), - lastFullBlockHeight, - collectionIndexer, - executionDataSyncer, - ) - builder.CollectionSyncer = collectionSyncer + // Initialize ExecNodeIdentitiesProvider as a module so it's available before components run. + // This is needed because the "transaction result error messages storage" module depends on it. + builder.Module("execution node identities provider", func(node *cmd.NodeConfig) error { + backendConfig := builder.rpcConf.BackendConfig - builder.RequestEng.WithHandle(collectionSyncer.OnCollectionDownloaded) + preferredENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.PreferredExecutionNodeIDs) + if err != nil { + return fmt.Errorf("failed to convert node id string to Flow Identifier for preferred EN map: %w", err) + } - 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, - ) - } + fixedENIdentifiers, err := flow.IdentifierListFromHex(backendConfig.FixedExecutionNodeIDs) + if err != nil { + return fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) + } - ingestEng, err := ingestion.New( - node.Logger, - node.EngineRegistry, - node.State, - node.Me, - node.StorageLockMgr, - node.ProtocolDB, - node.Storage.Blocks, - node.Storage.Results, - node.Storage.Receipts, - processedFinalizedBlockHeight, - notNil(builder.CollectionSyncer), - notNil(builder.CollectionIndexer), - notNil(builder.collectionExecutedMetric), - notNil(builder.TxResultErrorMessagesCore), - builder.FollowerDistributor, - ) + builder.ExecNodeIdentitiesProvider = commonrpc.NewExecutionNodeIdentitiesProvider( + node.Logger, + node.State, + node.Storage.Receipts, + preferredENIdentifiers, + fixedENIdentifiers, + ) + + return nil + }) + + // Initialize txResultErrorMessageProvider as a module so it's available before components run. + // This is needed because the "transaction result error messages storage" module depends on it. + builder.Module("transaction result error message provider", func(node *cmd.NodeConfig) error { + backendConfig := builder.rpcConf.BackendConfig + accessMetrics := builder.AccessMetrics + cacheSize := int(backendConfig.ConnectionPoolSize) + + var connBackendCache *rpcConnection.Cache + var err error + if cacheSize > 0 { + connBackendCache, err = rpcConnection.NewCache(node.Logger, accessMetrics, cacheSize) if err != nil { - return nil, err + return fmt.Errorf("could not initialize connection cache: %w", err) } - builder.IngestEng = ingestEng + } - ingestionDependable.Init(builder.IngestEng) + connFactory := &rpcConnection.ConnectionFactoryImpl{ + AccessConfig: backendConfig.AccessConfig, + CollectionConfig: backendConfig.CollectionConfig, + ExecutionConfig: backendConfig.ExecutionConfig, + AccessMetrics: accessMetrics, + Log: node.Logger, + Manager: rpcConnection.NewManager( + node.Logger, + accessMetrics, + connBackendCache, + backendConfig.CircuitBreakerConfig, + builder.rpcConf.CompressorName, + ), + } - return builder.IngestEng, nil - }) + nodeCommunicator := node_communicator.NewNodeCommunicator(backendConfig.CircuitBreakerConfig.Enabled) + builder.txResultErrorMessageProvider = error_messages.NewTxErrorMessageProvider( + node.Logger, + builder.transactionResultErrorMessages, // might be nil + notNil(builder.TxResultsIndex), + connFactory, + nodeCommunicator, + notNil(builder.ExecNodeIdentitiesProvider), + ) + + return nil + }) if builder.storeTxResultErrorMessages { + var txResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore + var processedTxErrorMessagesBlockHeight storage.ConsumerProgressInitializer builder. - AdminCommand("backfill-tx-error-messages", func(config *cmd.NodeConfig) commands.AdminCommand { - return storageCommands.NewBackfillTxErrorMessagesCommand( - builder.Logger, - builder.State, - builder.TxResultErrorMessagesCore, - ) - }). + // Initialize transactionResultErrorMessages storage first, as it's needed by the core Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { builder.transactionResultErrorMessages = store.NewTransactionResultErrorMessages( node.Metrics.Cache, @@ -2306,6 +2304,24 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { ) return nil }). + // Create the core after storage is initialized + Module("transaction result error messages core", func(node *cmd.NodeConfig) error { + txResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( + node.Logger, + notNil(builder.txResultErrorMessageProvider), + notNil(builder.transactionResultErrorMessages), + notNil(builder.ExecNodeIdentitiesProvider), + node.StorageLockMgr, + ) + return nil + }). + AdminCommand("backfill-tx-error-messages", func(config *cmd.NodeConfig) commands.AdminCommand { + return storageCommands.NewBackfillTxErrorMessagesCommand( + builder.Logger, + builder.State, + txResultErrorMessagesCore, + ) + }). Module("processed error messages block height consumer progress", func(node *cmd.NodeConfig) error { processedTxErrorMessagesBlockHeight = store.NewConsumerProgress( builder.ProtocolDB, @@ -2320,7 +2336,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.State, node.Storage.Headers, processedTxErrorMessagesBlockHeight, - builder.TxResultErrorMessagesCore, + txResultErrorMessagesCore, builder.FollowerDistributor, ) if err != nil { @@ -2523,6 +2539,74 @@ func (builder *FlowAccessNodeBuilder) initPublicLibp2pNode(networkKey crypto.Pri return libp2pNode, nil } +func createFinalizedBlockIndexer(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + processedFinalizedBlockHeight := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressIngestionEngineBlockHeight) + + finalizedBlockProcessor, err := finalized_indexer.NewFinalizedBlockProcessor( + node.Logger, + node.State, + node.StorageLockMgr, + node.ProtocolDB, + node.Storage.Blocks, + processedFinalizedBlockHeight, + builder.FollowerDistributor.FinalizationDistributor, + notNil(builder.collectionExecutedMetric), + ) + if err != nil { + return nil, fmt.Errorf("could not create finalized block processor: %w", err) + } + + return finalizedBlockProcessor, nil + } +} + +func createCollectionSyncFetcher(builder *FlowAccessNodeBuilder) { + var requesterEng module.ReadyDoneAware + builder. + Component("collection_sync fetcher", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + result, err := collection_syncfactory.CreateCollectionSyncFetcherComponent( + node.Logger, + builder.executionDataSyncEnabled, + builder.collectionSync, + node.Metrics.Engine, + node.EngineRegistry, + node.State, + node.Me, + node.Storage.Blocks, + builder.ProtocolDB, + notNil(builder.blockCollectionIndexer), + builder.FollowerDistributor, + notNil(builder.collectionExecutedMetric), + notNil(builder.CollectionSyncMetrics), + builder.collectionFetcherMaxProcessing, + builder.collectionFetcherMaxSearchAhead, + builder.collectionFetcherRetryInterval, + notNil(builder.lastFullBlockHeight), + notNil(builder.AccessMetrics), + ) + if err != nil { + return nil, err + } + requesterEng = result.Requester + return result.Fetcher, nil + }). + Component("collection_sync collection requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + return requesterEng, nil + }) +} + +// func createIngestReceiptEngine(builder *FlowAccessNodeBuilder) func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +// return func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { +// return ingest_receipt.New( +// node.Logger, +// node.EngineRegistry, +// node.Storage.Receipts, +// notNil(builder.collectionExecutedMetric), +// ) +// } +// } + // notNil ensures that the input is not nil and returns it // the usage is to ensure the dependencies are initialized before initializing a module. // for instance, the IngestionEngine depends on storage.Collections, which is initialized in a diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index e79a8a4aea8..e2c8573a77a 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -413,6 +413,10 @@ func (exeNode *ExecutionNode) LoadBlobService( opts = append(opts, blob.WithReprovideInterval(-1)) } + if !node.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + if exeNode.exeConf.blobstoreRateLimit > 0 && exeNode.exeConf.blobstoreBurstLimit > 0 { opts = append(opts, blob.WithRateLimit(float64(exeNode.exeConf.blobstoreRateLimit), exeNode.exeConf.blobstoreBurstLimit)) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 2721552642c..7c310fc6c3b 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -186,6 +186,13 @@ type BaseConfig struct { // This is only meaningful to Access and Execution nodes. BitswapReprovideEnabled bool + // BitswapBloomCacheEnabled configures whether the Bitswap bloom cache is enabled. + // When disabled, uses a plain blockstore instead of cached blockstore, avoiding + // the CPU cost of building the bloom filter on startup. Pebble's built-in bloom + // filters (persisted in SSTables) are still used for efficient lookups. + // This is only meaningful to Access and Execution nodes. + BitswapBloomCacheEnabled bool + TransactionFeesDisabled bool } @@ -297,12 +304,13 @@ func DefaultBaseConfig() *BaseConfig { Duration: 10 * time.Second, }, - HeroCacheMetricsEnable: false, - SyncCoreConfig: chainsync.DefaultConfig(), - CodecFactory: codecFactory, - ComplianceConfig: compliance.DefaultConfig(), - DhtSystemEnabled: true, - BitswapReprovideEnabled: true, + HeroCacheMetricsEnable: false, + SyncCoreConfig: chainsync.DefaultConfig(), + CodecFactory: codecFactory, + ComplianceConfig: compliance.DefaultConfig(), + DhtSystemEnabled: true, + BitswapReprovideEnabled: true, + BitswapBloomCacheEnabled: true, // default: use cached blockstore TODO leo: change default to false } } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 6cbaafbbda7..984269eccc7 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -38,7 +38,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/apiproxy" "github.com/onflow/flow-go/engine/access/index" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/engine/access/rest" restapiproxy "github.com/onflow/flow-go/engine/access/rest/apiproxy" "github.com/onflow/flow-go/engine/access/rest/router" @@ -70,7 +69,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/chainsync" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/executiondatasync/execution_data" execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" @@ -1104,7 +1102,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS var ds datastore.Batching var bs network.BlobService var processedBlockHeight storage.ConsumerProgressInitializer - var processedNotifications storage.ConsumerProgressInitializer var publicBsDependable *module.ProxiedReadyDoneAware var execDataDistributor *edrequester.ExecutionDataDistributor var execDataCacheBackend *herocache.BlockExecutionData @@ -1152,14 +1149,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS processedBlockHeight = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterBlockHeight) return nil }). - Module("processed notifications consumer progress", func(node *cmd.NodeConfig) error { - // Note: progress is stored in the datastore's DB since that is where the jobqueue - // writes execution data to. - db := builder.ExecutionDatastoreManager.DB() - - processedNotifications = store.NewConsumerProgress(db, module.ConsumeProgressExecutionDataRequesterNotification) - return nil - }). Module("blobservice peer manager dependencies", func(node *cmd.NodeConfig) error { publicBsDependable = module.NewProxiedReadyDoneAware() builder.PeerManagerDependencies.Add(publicBsDependable) @@ -1200,6 +1189,10 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS ), } + if !builder.BitswapBloomCacheEnabled { + opts = append(opts, blob.WithSkipBloomCache(true)) + } + var err error bs, err = node.EngineRegistry.RegisterBlobService(channels.PublicExecutionDataService, ds, opts...) if err != nil { @@ -1292,7 +1285,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.ExecutionDataDownloader, executionDataCache, processedBlockHeight, - processedNotifications, builder.State, builder.Storage.Headers, builder.executionDataConfig, @@ -1438,31 +1430,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS return nil, fmt.Errorf("could not create derived chain data: %w", err) } - rootBlockHeight := node.State.Params().FinalizedRoot().Height - progress, err := store.NewConsumerProgress(builder.ProtocolDB, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) - if err != nil { - return nil, fmt.Errorf("could not create last full block height consumer progress: %w", err) - } - - lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter(progress) - if err != nil { - return nil, fmt.Errorf("could not create last full block height counter: %w", err) - } - var collectionExecutedMetric module.CollectionExecutedMetric = metrics.NewNoopCollector() - collectionIndexer, err := collections.NewIndexer( - builder.Logger, - builder.ProtocolDB, - collectionExecutedMetric, - builder.State, - builder.Storage.Blocks, - builder.Storage.Collections, - lastFullBlockHeight, - builder.StorageLockMgr, - ) - if err != nil { - return nil, fmt.Errorf("could not create collection indexer: %w", err) - } builder.ExecutionIndexerCore = indexer.New( builder.Logger, @@ -1477,7 +1445,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.scheduledTransactions, builder.RootChainID, indexerDerivedChainData, - collectionIndexer, collectionExecutedMetric, node.StorageLockMgr, ) @@ -1556,10 +1523,6 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS } builder.stateStreamConf.RpcMetricsEnabled = builder.rpcMetricsEnabled - highestAvailableHeight, err := builder.ExecutionDataRequester.HighestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("could not get highest consecutive height: %w", err) - } broadcaster := engine.NewBroadcaster() eventQueryMode, err := query_mode.ParseIndexQueryMode(builder.rpcConf.BackendConfig.EventQueryMode) @@ -1578,7 +1541,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS builder.executionDataConfig.InitialBlockHeight, node.Storage.Headers, broadcaster, - highestAvailableHeight, + builder.ExecutionDataRequester, builder.EventsIndex, useIndex, ) diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 142f2d55c6d..e5008b345f4 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -213,6 +213,10 @@ func (fnb *FlowNodeBuilder) BaseFlags() { "bitswap-reprovide-enabled", defaultConfig.BitswapReprovideEnabled, "[experimental] whether to enable bitswap reproviding. This is an experimental feature. Use with caution.") + fnb.flags.BoolVar(&fnb.BaseConfig.BitswapBloomCacheEnabled, + "bitswap-bloom-cache-enabled", + defaultConfig.BitswapBloomCacheEnabled, + "[experimental] whether to enable bitswap bloom cache. When disabled, uses a plain blockstore instead of cached blockstore, avoiding the CPU cost of building the bloom filter on startup. Pebble's built-in bloom filters (persisted in SSTables) are still used. This is an experimental feature. Use with caution.") // dynamic node startup flags fnb.flags.StringVar(&fnb.BaseConfig.DynamicStartupANPubkey, diff --git a/cmd/util/cmd/read-protocol-state/cmd/blocks.go b/cmd/util/cmd/read-protocol-state/cmd/blocks.go index 4c2ef0d13bd..d0792f9329e 100644 --- a/cmd/util/cmd/read-protocol-state/cmd/blocks.go +++ b/cmd/util/cmd/read-protocol-state/cmd/blocks.go @@ -222,6 +222,8 @@ func runE(*cobra.Command, []string) error { return fmt.Errorf("could not get root block: %w", err) } + log.Info().Msgf("searching executed block between heights %v and %v", root.Height, sealed.Height) + // find the last executed and sealed block for h := sealed.Height; h >= root.Height; h-- { block, err := reader.GetBlockByHeight(h) @@ -238,6 +240,8 @@ func runE(*cobra.Command, []string) error { common.PrettyPrintEntity(block) return nil } + + log.Info().Msgf("block at height %v is not executed yet", h) } return fmt.Errorf("could not find executed block") diff --git a/consensus/hotstuff/distributor.go b/consensus/hotstuff/distributor.go new file mode 100644 index 00000000000..34544a54697 --- /dev/null +++ b/consensus/hotstuff/distributor.go @@ -0,0 +1,8 @@ +package hotstuff + +import "github.com/onflow/flow-go/consensus/hotstuff/model" + +type Distributor interface { + AddOnBlockFinalizedConsumer(consumer func(block *model.Block)) + AddOnBlockIncorporatedConsumer(consumer func(block *model.Block)) +} diff --git a/consensus/hotstuff/mocks/distributor.go b/consensus/hotstuff/mocks/distributor.go new file mode 100644 index 00000000000..b914f899c9f --- /dev/null +++ b/consensus/hotstuff/mocks/distributor.go @@ -0,0 +1,37 @@ +// Code generated by mockery. DO NOT EDIT. + +package mocks + +import ( + model "github.com/onflow/flow-go/consensus/hotstuff/model" + mock "github.com/stretchr/testify/mock" +) + +// Distributor is an autogenerated mock type for the Distributor type +type Distributor struct { + mock.Mock +} + +// AddOnBlockFinalizedConsumer provides a mock function with given fields: consumer +func (_m *Distributor) AddOnBlockFinalizedConsumer(consumer func(*model.Block)) { + _m.Called(consumer) +} + +// AddOnBlockIncorporatedConsumer provides a mock function with given fields: consumer +func (_m *Distributor) AddOnBlockIncorporatedConsumer(consumer func(*model.Block)) { + _m.Called(consumer) +} + +// NewDistributor creates a new instance of Distributor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDistributor(t interface { + mock.TestingT + Cleanup(func()) +}) *Distributor { + mock := &Distributor{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/engine/access/collection_sync/collection_sync.go b/engine/access/collection_sync/collection_sync.go new file mode 100644 index 00000000000..65940429c21 --- /dev/null +++ b/engine/access/collection_sync/collection_sync.go @@ -0,0 +1,98 @@ +package collection_sync + +import ( + "context" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" +) + +// Tracks missing collections per height and invokes job callbacks when complete. +type MissingCollectionQueue interface { + // EnqueueMissingCollections tracks the given missing collection IDs for the given block height. + EnqueueMissingCollections(blockHeight uint64, ids []flow.Identifier, callback func()) error + + // OnIndexedForBlock returns the callback function for the given block height + OnIndexedForBlock(blockHeight uint64) (func(), bool) + + // On receipt of a collection, MCQ updates internal state and, if a block + // just became complete, returns: (collections, height, missingCollectionID, true). + // Otherwise, returns (nil, height, missingCollectionID, false). + // missingCollectionID is an arbitrary ID from the remaining missing collections, or ZeroID if none. + OnReceivedCollection(collection *flow.Collection) ([]*flow.Collection, uint64, flow.Identifier, bool) + + // IsHeightQueued returns true if the given height is still being tracked (has not been indexed yet). + IsHeightQueued(height uint64) bool + + // Size returns the number of missing heights currently in the queue. + Size() uint + + // GetMissingCollections returns all collection IDs that are currently missing across all block heights. + GetMissingCollections() []flow.Identifier + + // GetMissingCollectionsByHeight returns a map of block height to collection IDs that are missing for that height. + GetMissingCollectionsByHeight() map[uint64][]flow.Identifier +} + +// Requests collections by their guarantees. +type CollectionRequester interface { + RequestCollectionsByGuarantees(guarantees []*flow.CollectionGuarantee) error +} + +// BlockCollectionIndexer stores and indexes collections for a given block height. +type BlockCollectionIndexer interface { + // IndexCollectionsForBlock stores and indexes collections for a given block height. + // No error is exepcted during normal operation. + IndexCollectionsForBlock(blockHeight uint64, cols []*flow.Collection) error + + // GetMissingCollections retrieves the block and returns collection guarantees that whose collections + // are missing in storage. + // Only garantees whose collections that are not already in storage are returned. + GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) +} + +// BlockProcessor processes blocks to fetch and index their collections. +type BlockProcessor interface { + // RequestCollectionsForBlock requests all missing collections for the given block. + FetchCollections(ctx irrecoverable.SignalerContext, block *flow.Block, done func()) error + // MissingCollectionQueueSize returns the number of missing collections currently in the queue. + MissingCollectionQueueSize() uint + // RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. + RetryFetchingMissingCollections() error +} + +// Fetcher is a component that consumes finalized block jobs and processes them +// to index collections. It uses a job consumer with windowed throttling to prevent node overload. +type Fetcher interface { + component.Component + ProgressReader + // OnFinalizedBlock notifies the fetcher that a new block has been finalized. + OnFinalizedBlock() + // MissingCollectionQueueSize returns the number of missing height currently in the queue. + Size() uint +} + +// ExecutionDataProvider provides the latest height for which execution data indexer has collections. +// This can be nil if execution data indexing is disabled. +type ExecutionDataProvider interface { + // HighestIndexedHeight returns the highest block height for which execution data is available. + // It garautnees that all heights below it also have execution data available to be called + // with GetExecutionDataByHeight. + HighestIndexedHeight() uint64 + + // It might return [execution_data.BlobNotFoundError] error, if some CID in the blob tree could not be found from the blobstore. + GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) +} + +// ExecutionDataProcessor processes execution data when new execution data is available. +type ExecutionDataProcessor interface { + // OnNewExectuionData notifies the processor that new execution data is available for processing. + OnNewExectuionData() +} + +// ProgressReader provides the current progress of collection fetching/indexing. +type ProgressReader interface { + // ProcessedHeight returns the highest block height for which collections have been processed. + ProcessedHeight() uint64 +} diff --git a/engine/access/collection_sync/execution_data_index/processor.go b/engine/access/collection_sync/execution_data_index/processor.go new file mode 100644 index 00000000000..e40e45dce93 --- /dev/null +++ b/engine/access/collection_sync/execution_data_index/processor.go @@ -0,0 +1,195 @@ +package execution_data_index + +import ( + "fmt" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/irrecoverable" +) + +type ExecutionDataProcessor struct { + component.Component + log zerolog.Logger + newExecutionDataIndexed engine.Notifier + provider collection_sync.ExecutionDataProvider + indexer collection_sync.BlockCollectionIndexer + // state + processedHeight *counters.PersistentStrictMonotonicCounter + onIndexedCallback func(uint64) +} + +var _ collection_sync.ExecutionDataProcessor = (*ExecutionDataProcessor)(nil) +var _ collection_sync.ProgressReader = (*ExecutionDataProcessor)(nil) +var _ component.Component = (*ExecutionDataProcessor)(nil) + +func NewExecutionDataProcessor( + log zerolog.Logger, + provider collection_sync.ExecutionDataProvider, + indexer collection_sync.BlockCollectionIndexer, + processedHeight *counters.PersistentStrictMonotonicCounter, + onIndexedCallback func(uint64), +) *ExecutionDataProcessor { + edp := &ExecutionDataProcessor{ + log: log.With().Str("coll_sync", "data_processor").Logger(), + newExecutionDataIndexed: engine.NewNotifier(), + provider: provider, + indexer: indexer, + processedHeight: processedHeight, + onIndexedCallback: onIndexedCallback, + } + + // Initialize the notifier so that even if no new execution data comes in, + // the worker loop can still be triggered to process any existing data. + edp.newExecutionDataIndexed.Notify() + + // Build component manager with worker loop + cm := component.NewComponentManagerBuilder(). + AddWorker(edp.workerLoop). + Build() + + edp.Component = cm + + return edp +} + +func (edp *ExecutionDataProcessor) OnNewExectuionData() { + edp.newExecutionDataIndexed.Notify() +} + +// retryOnBlobNotFound executes the given function and retries it every 5 seconds +// if it returns a BlobNotFoundError. Each retry attempt is logged. +// Returns the result of the function call, or an error if retries fail or a non-BlobNotFoundError occurs. +func retryOnBlobNotFound( + ctx irrecoverable.SignalerContext, + log zerolog.Logger, + height uint64, + fn func() ([]*flow.Collection, error), +) ([]*flow.Collection, error) { + collections, err := fn() + if err == nil { + return collections, nil + } + + // If the error is not BlobNotFoundError, return immediately + if !execution_data.IsBlobNotFoundError(err) { + return nil, err + } + + retryTicker := time.NewTicker(5 * time.Second) + defer retryTicker.Stop() + + attempt := 1 + log.Error(). + Uint64("height", height). + Err(err). + Int("attempt", attempt). + Msg("execution data not found, retrying every 5 seconds") + + for { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-retryTicker.C: + attempt++ + collections, err = fn() + if err == nil { + log.Info(). + Uint64("height", height). + Int("attempt", attempt). + Msg("successfully retrieved execution data after retry") + return collections, nil + } + + // If error is still BlobNotFoundError, continue retrying + if execution_data.IsBlobNotFoundError(err) { + log.Error(). + Uint64("height", height). + Err(err). + Int("attempt", attempt). + Msg("execution data still not found, retrying") + continue + } + + // If error changed to something else, return it + return nil, err + } + } +} + +func (edp *ExecutionDataProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + // using a single threaded loop to index each execution for height + // since indexing collections is blocking anyway, and reading the execution data + // is quick, because we cache for 100 heights. + for { + select { + case <-ctx.Done(): + return + case <-edp.newExecutionDataIndexed.Channel(): + highestAvailableHeight := edp.provider.HighestIndexedHeight() + lowestMissing := edp.processedHeight.Value() + 1 + + for height := lowestMissing; height <= highestAvailableHeight; height++ { + // TODO: This logic only supports ingesting execution data from sealed blocks. Once support is + // added for syncing execution data for unsealed results, this logic will need to be updated + // to account for execution forks. + // Fetch execution data for this height. If the blob is not found (BlobNotFoundError), + // retryOnBlobNotFound will automatically retry every 5 seconds until it's available or a different error occurs. + // retry is not needed, because the provider is supposed to guarantee that all heights below + // HighestIndexedHeight have execution data available. + // this is for debugging purpose for now. + collections, err := retryOnBlobNotFound(ctx, edp.log, height, func() ([]*flow.Collection, error) { + return edp.provider.GetExecutionDataByHeight(ctx, height) + }) + if err != nil { + ctx.Throw(fmt.Errorf("collection_sync execution data processor: failed to get execution data for height %d: %w", + height, err)) + return + } + + // Note: the collections might have been indexed by fetcher engine already, + // but IndexCollectionsForBlock will handle deduplication by first check if the collections already exist, + // if so, it will skip indexing them again. + err = edp.indexer.IndexCollectionsForBlock(height, collections) + if err != nil { + ctx.Throw(fmt.Errorf("failed to index collections for block height %d: %w", height, err)) + return + } + + // Update processed height after successful indexing + err = edp.processedHeight.Set(height) + if err != nil { + ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) + return + } + + // Log progress for each height with all relevant information + edp.log.Debug(). + Uint64("indexed", height). + Uint64("lowest_missing", lowestMissing). + Uint64("highest_available", highestAvailableHeight). + Uint64("processed_count", height-lowestMissing+1). + Uint64("remaining_count", highestAvailableHeight-height). + Uint64("total_to_process", highestAvailableHeight-lowestMissing+1). + Msg("indexed execution data progress") + + edp.onIndexedCallback(height) + } + } + } +} + +// ProcessedHeight returns the highest consecutive height for which execution data has been processed, +// meaning the collections for that height have been indexed. +func (edp *ExecutionDataProcessor) ProcessedHeight() uint64 { + return edp.processedHeight.Value() +} diff --git a/engine/access/collection_sync/execution_data_index/provider.go b/engine/access/collection_sync/execution_data_index/provider.go new file mode 100644 index 00000000000..68953e5ab4c --- /dev/null +++ b/engine/access/collection_sync/execution_data_index/provider.go @@ -0,0 +1,49 @@ +package execution_data_index + +import ( + "context" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" +) + +var _ collection_sync.ExecutionDataProvider = (*executionDataProvider)(nil) + +// executionDataProvider implements ExecutionDataProvider by querying ExecutionDataCache. +type executionDataProvider struct { + cache execution_data.ExecutionDataCache + highestExectuionDataHeight state_synchronization.ExecutionDataIndexedHeight +} + +// NewExecutionDataProvider creates a new ExecutionDataProvider that reads from the given ExecutionDataCache. +// The headers storage is used to determine the search range for finding available heights. +func NewExecutionDataProvider( + cache execution_data.ExecutionDataCache, + highestExectuionDataHeight state_synchronization.ExecutionDataIndexedHeight, +) *executionDataProvider { + return &executionDataProvider{ + cache: cache, + highestExectuionDataHeight: highestExectuionDataHeight, + } +} + +// HighestIndexedHeight returns the highest block height for which execution data is available. +func (p *executionDataProvider) HighestIndexedHeight() uint64 { + return p.highestExectuionDataHeight.HighestConsecutiveHeight() +} + +// GetExecutionDataByHeight returns the execution data for the given block height. +// It might return [execution_data.BlobNotFoundError] error, if some CID in the blob tree could not be found from the blobstore. +func (p *executionDataProvider) GetExecutionDataByHeight(ctx context.Context, height uint64) ([]*flow.Collection, error) { + blockExecutionData, err := p.cache.ByHeight(ctx, height) + if err != nil { + return nil, err + } + + // TODO: to support processing data from unsealed blocks, we should check here that the collections + // returned match the guarantees in the block. + + return blockExecutionData.StandardCollections(), nil +} diff --git a/engine/access/collection_sync/factory/access_builder.go b/engine/access/collection_sync/factory/access_builder.go new file mode 100644 index 00000000000..422fc7fe2cd --- /dev/null +++ b/engine/access/collection_sync/factory/access_builder.go @@ -0,0 +1,316 @@ +package factory + +import ( + "fmt" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/consensus/hotstuff" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" + edrequester "github.com/onflow/flow-go/module/state_synchronization/requester" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/store" +) + +// ProcessedLastFullBlockHeightModuleResult contains the results from creating the processed last full block height module. +type ProcessedLastFullBlockHeightModuleResult struct { + LastFullBlockHeight *ProgressReader + CollectionIndexedHeight storage.ConsumerProgress +} + +// CreateProcessedLastFullBlockHeightModule initializes and syncs the progress trackers for collection sync. +// +// Parameters: +// - log: Logger for logging operations +// - state: Protocol state to get root block height +// - db: Database for storing progress +// +// Returns: +// - The result containing the ProgressReader and collection indexed height +// - An error if the initialization fails +func CreateProcessedLastFullBlockHeightModule( + log zerolog.Logger, + state protocol.State, + db storage.DB, +) (*ProcessedLastFullBlockHeightModuleResult, error) { + rootBlockHeight := state.Params().SealedRoot().Height + + // Initialize ConsumeProgressLastFullBlockHeight + progress, err := store.NewConsumerProgress(db, module.ConsumeProgressLastFullBlockHeight).Initialize(rootBlockHeight) + if err != nil { + return nil, err + } + + lastProgress, err := progress.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get last processed index for last full block height: %w", err) + } + + // Sync ConsumeProgressLastFullBlockHeight and ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // by taking the max value of each and updating both + fetchAndIndexedTracker := store.NewConsumerProgress(db, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + fetchAndIndexed, err := fetchAndIndexedTracker.Initialize(rootBlockHeight) + if err != nil { + return nil, fmt.Errorf("failed to initialize fetch and indexed collections block height tracker: %w", err) + } + fetchAndIndexedValue, err := fetchAndIndexed.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get fetch and indexed collections block height: %w", err) + } + + // Take the max of both values + maxValue := max(lastProgress, fetchAndIndexedValue) + + // Update both trackers if needed + if lastProgress < maxValue { + if err := progress.SetProcessedIndex(maxValue); err != nil { + return nil, fmt.Errorf("failed to update last full block height: %w", err) + } + log.Info(). + Uint64("old_value", lastProgress). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressLastFullBlockHeight). + Msg("synced collection sync progress tracker") + } + + if fetchAndIndexedValue < maxValue { + if err := fetchAndIndexed.SetProcessedIndex(maxValue); err != nil { + return nil, fmt.Errorf("failed to update fetch and indexed collections block height: %w", err) + } + log.Info(). + Uint64("old_value", fetchAndIndexedValue). + Uint64("new_value", maxValue). + Str("tracker", module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight). + Msg("synced collection sync progress tracker") + } + + if lastProgress == maxValue && fetchAndIndexedValue == maxValue { + log.Info(). + Uint64("value", maxValue). + Msg("collection sync progress trackers already in sync") + } + + // Get the final synced value for ProgressReader + finalProgress, err := progress.ProcessedIndex() + if err != nil { + return nil, fmt.Errorf("failed to get final synced progress: %w", err) + } + + // Create ProgressReader that aggregates progress from executionDataProcessor and collectionFetcher + return &ProcessedLastFullBlockHeightModuleResult{ + LastFullBlockHeight: NewProgressReader(finalProgress), + CollectionIndexedHeight: progress, + }, nil +} + +// CreateExecutionDataProcessorComponent creates an execution data processor component. +// It creates an execution data processor if execution data sync is enabled and collection sync mode is not "collection_only". +// +// Parameters: +// - log: Logger for logging operations +// - executionDataSyncEnabled: Whether execution data sync is enabled +// - collectionSyncMode: The collection sync mode +// - executionDataCache: Execution data cache +// - executionDataRequester: Execution data requester +// - collectionIndexedHeight: Consumer progress for collection indexed height +// - blockCollectionIndexer: Block collection indexer +// - collectionSyncMetrics: Collection sync metrics +// - lastFullBlockHeight: Progress reader to register the processor with +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height +// - distributor: Execution data distributor to notify on new execution data +// +// Returns: +// - The processor component (or NoopReadyDoneAware if not created) +// - An error if the processor creation fails +func CreateExecutionDataProcessorComponent( + log zerolog.Logger, + executionDataSyncEnabled bool, + collectionSyncMode CollectionSyncMode, + executionDataCache execution_data.ExecutionDataCache, + executionDataRequester state_synchronization.ExecutionDataRequester, + collectionIndexedHeight storage.ConsumerProgress, + blockCollectionIndexer collection_sync.BlockCollectionIndexer, + collectionSyncMetrics module.CollectionSyncMetrics, + lastFullBlockHeight *ProgressReader, + accessMetrics module.AccessMetrics, + distributor *edrequester.ExecutionDataDistributor, +) (module.ReadyDoneAware, error) { + shouldCreate := collectionSyncMode.ShouldCreateExecutionDataProcessor(executionDataSyncEnabled) + if !shouldCreate { + // Log when execution data processor is not created + if !executionDataSyncEnabled { + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("execution data processor not created: execution data sync is disabled") + } else if collectionSyncMode == CollectionSyncModeCollectionOnly { + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("execution data processor not created: collection sync mode is 'collection_only'") + } + + return &module.NoopReadyDoneAware{}, nil + } + + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("creating execution data processor") + + if executionDataCache == nil { + return nil, fmt.Errorf("ExecutionDataCache must be created before execution data processor") + } + + // Create execution data processor + executionDataProcessor, err := createExecutionDataProcessor( + log, + executionDataCache, + executionDataRequester, + collectionIndexedHeight, + blockCollectionIndexer, + func(indexedHeight uint64) { + collectionSyncMetrics.CollectionSyncedHeight(indexedHeight) + // Also update last_full_finalized_block_height metric with the max of both heights + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + }, + ) + if err != nil { + return nil, fmt.Errorf("could not create execution data processor: %w", err) + } + + // Register with ProgressReader + lastFullBlockHeight.SetExecutionDataProcessor(executionDataProcessor) + + // Initialize collection synced height metric to avoid spikes in dashboard + collectionSyncMetrics.CollectionSyncedHeight(executionDataProcessor.ProcessedHeight()) + // Initialize last_full_finalized_block_height metric + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + + distributor.AddOnExecutionDataReceivedConsumer(func() { + executionDataProcessor.OnNewExectuionData() + }) + + return executionDataProcessor, nil +} + +// CollectionSyncFetcherComponentResult contains the results from creating the collection sync fetcher component. +type CollectionSyncFetcherComponentResult struct { + Fetcher module.ReadyDoneAware + Requester module.ReadyDoneAware +} + +// CreateCollectionSyncFetcherComponent creates a collection fetcher and requester engine based on the collection sync mode. +// +// Parameters: +// - log: Logger for logging operations +// - executionDataSyncEnabled: Whether execution data sync is enabled +// - collectionSyncMode: The collection sync mode +// - engineMetrics: Engine metrics +// - engineRegistry: Engine registry +// - state: Protocol state +// - me: Local node identity +// - blocks: Blocks storage +// - db: Database for storage operations +// - blockCollectionIndexer: Block collection indexer +// - followerDistributor: Follower distributor +// - collectionExecutedMetric: Collection executed metric +// - collectionSyncMetrics: Collection sync metrics +// - maxProcessing: Maximum number of concurrent processing jobs +// - maxSearchAhead: Maximum number of blocks to search ahead +// - retryInterval: Interval for retrying missing collections. If 0, uses DefaultRetryInterval +// - lastFullBlockHeight: Progress reader to register the fetcher with +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height +// +// Returns: +// - The result containing the fetcher component, requester component, and requester engine +// - An error if the fetcher creation fails +func CreateCollectionSyncFetcherComponent( + log zerolog.Logger, + executionDataSyncEnabled bool, + collectionSyncMode CollectionSyncMode, + engineMetrics module.EngineMetrics, + engineRegistry network.EngineRegistry, + state protocol.State, + me module.Local, + blocks storage.Blocks, + db storage.DB, + blockCollectionIndexer collection_sync.BlockCollectionIndexer, + followerDistributor hotstuff.Distributor, + collectionExecutedMetric module.CollectionExecutedMetric, + collectionSyncMetrics module.CollectionSyncMetrics, + maxProcessing uint64, + maxSearchAhead uint64, + retryInterval time.Duration, + lastFullBlockHeight *ProgressReader, + accessMetrics module.AccessMetrics, +) (*CollectionSyncFetcherComponentResult, error) { + // Create fetcher if: + // 1. collectionSync is "execution_and_collection" (always create, even with execution data sync) + // 2. collectionSync is "collection_only" (always create) + // 3. collectionSync is "execution_first" and execution data sync is disabled + shouldCreateFetcher := collectionSyncMode.ShouldCreateFetcher(executionDataSyncEnabled) + + if !shouldCreateFetcher { + // skip if execution data sync is enabled and not in execution_and_collection or collection_only mode + // because the execution data contains the collections, so no need to fetch them separately. + // otherwise, if both fetching and syncing are enabled, they might slow down each other, + // because the database operation requires locking. + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("collection sync fetcher not created: execution data sync is enabled and collection sync mode is not 'execution_and_collection' or 'collection_only'") + return &CollectionSyncFetcherComponentResult{ + Fetcher: &module.NoopReadyDoneAware{}, + Requester: &module.NoopReadyDoneAware{}, + }, nil + } + + log.Info(). + Str("collection_sync_mode", collectionSyncMode.String()). + Bool("execution_data_sync_enabled", executionDataSyncEnabled). + Msg("creating collection sync fetcher") + + // Fetcher always uses ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight + // to avoid contention with execution data processor which uses ConsumeProgressLastFullBlockHeight + fetchAndIndexedCollectionsBlockHeight := store.NewConsumerProgress(db, module.ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight) + + // Create fetcher and requesterEng + requesterEng, fetcher, err := createFetcher( + log, + engineMetrics, + engineRegistry, + state, + me, + blocks, + blockCollectionIndexer, + fetchAndIndexedCollectionsBlockHeight, + followerDistributor, + collectionSyncMetrics, + lastFullBlockHeight, + accessMetrics, + CreateFetcherConfig{ + MaxProcessing: maxProcessing, + MaxSearchAhead: maxSearchAhead, + RetryInterval: retryInterval, + }, + ) + + if err != nil { + return nil, fmt.Errorf("could not create collection fetcher: %w", err) + } + + // Register with ProgressReader + lastFullBlockHeight.SetCollectionFetcher(fetcher) + + return &CollectionSyncFetcherComponentResult{ + Fetcher: fetcher, + Requester: requesterEng, + }, nil +} diff --git a/engine/access/collection_sync/factory/collection_sync_mode.go b/engine/access/collection_sync/factory/collection_sync_mode.go new file mode 100644 index 00000000000..53042319bca --- /dev/null +++ b/engine/access/collection_sync/factory/collection_sync_mode.go @@ -0,0 +1,58 @@ +package factory + +import "fmt" + +// CollectionSyncMode represents the mode for collection synchronization. +type CollectionSyncMode string + +const ( + // CollectionSyncModeExecutionFirst fetches from execution nodes first if execution data syncing is enabled, + // otherwise fetches from collection nodes. + CollectionSyncModeExecutionFirst CollectionSyncMode = "execution_first" + // CollectionSyncModeExecutionAndCollection fetches from both collection nodes and execution nodes. + CollectionSyncModeExecutionAndCollection CollectionSyncMode = "execution_and_collection" + // CollectionSyncModeCollectionOnly only fetches from collection nodes. + CollectionSyncModeCollectionOnly CollectionSyncMode = "collection_only" +) + +// String returns the string representation of the CollectionSyncMode. +func (m CollectionSyncMode) String() string { + return string(m) +} + +// ParseCollectionSyncMode parses a string into a CollectionSyncMode. +func ParseCollectionSyncMode(s string) (CollectionSyncMode, error) { + switch s { + case string(CollectionSyncModeExecutionFirst): + return CollectionSyncModeExecutionFirst, nil + case string(CollectionSyncModeExecutionAndCollection): + return CollectionSyncModeExecutionAndCollection, nil + case string(CollectionSyncModeCollectionOnly): + return CollectionSyncModeCollectionOnly, nil + default: + return "", fmt.Errorf("invalid collection sync mode: %s, must be one of [execution_first, execution_and_collection, collection_only]", s) + } +} + +// ShouldCreateFetcher returns whether a collection fetcher should be created based on the sync mode +// and whether execution data sync is enabled. +// +// A fetcher should be created if: +// - The mode is ExecutionAndCollection (always create, even with execution data sync) +// - The mode is CollectionOnly (always create) +// - The mode is ExecutionFirst and execution data sync is disabled +func (m CollectionSyncMode) ShouldCreateFetcher(executionDataSyncEnabled bool) bool { + return m == CollectionSyncModeExecutionAndCollection || + m == CollectionSyncModeCollectionOnly || + (m == CollectionSyncModeExecutionFirst && !executionDataSyncEnabled) +} + +// ShouldCreateExecutionDataProcessor returns whether an execution data processor should be created +// based on the sync mode and whether execution data sync is enabled. +// +// An execution data processor should be created if: +// - Execution data sync is enabled AND +// - The mode is NOT CollectionOnly (since CollectionOnly mode only fetches from collection nodes) +func (m CollectionSyncMode) ShouldCreateExecutionDataProcessor(executionDataSyncEnabled bool) bool { + return executionDataSyncEnabled && m != CollectionSyncModeCollectionOnly +} diff --git a/engine/access/collection_sync/factory/collection_sync_mode_test.go b/engine/access/collection_sync/factory/collection_sync_mode_test.go new file mode 100644 index 00000000000..2938a40b655 --- /dev/null +++ b/engine/access/collection_sync/factory/collection_sync_mode_test.go @@ -0,0 +1,204 @@ +package factory + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// TestCollectionSyncMode_String tests the String method for all CollectionSyncMode constants. +func TestCollectionSyncMode_String(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + expected string + }{ + { + name: "execution_first", + mode: CollectionSyncModeExecutionFirst, + expected: "execution_first", + }, + { + name: "execution_and_collection", + mode: CollectionSyncModeExecutionAndCollection, + expected: "execution_and_collection", + }, + { + name: "collection_only", + mode: CollectionSyncModeCollectionOnly, + expected: "collection_only", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + assert.Equal(t, test.expected, test.mode.String()) + }) + } +} + +// TestParseCollectionSyncMode_Valid tests the ParseCollectionSyncMode function with valid inputs. +func TestParseCollectionSyncMode_Valid(t *testing.T) { + tests := map[string]CollectionSyncMode{ + "execution_first": CollectionSyncModeExecutionFirst, + "execution_and_collection": CollectionSyncModeExecutionAndCollection, + "collection_only": CollectionSyncModeCollectionOnly, + } + + for input, expectedMode := range tests { + t.Run(input, func(t *testing.T) { + mode, err := ParseCollectionSyncMode(input) + require.NoError(t, err) + assert.Equal(t, expectedMode, mode) + }) + } +} + +// TestParseCollectionSyncMode_Invalid tests the ParseCollectionSyncMode function with invalid inputs. +func TestParseCollectionSyncMode_Invalid(t *testing.T) { + invalidInputs := []string{ + "", + "unknown", + "invalid", + "execution-first", // wrong separator + "executionFirst", // wrong format + "EXECUTION_FIRST", // wrong case + "execution_and_collection_extra", // extra suffix + } + + for _, input := range invalidInputs { + t.Run(input, func(t *testing.T) { + mode, err := ParseCollectionSyncMode(input) + assert.Error(t, err) + assert.Empty(t, mode) + expectedErr := fmt.Errorf("invalid collection sync mode: %s, must be one of [execution_first, execution_and_collection, collection_only]", input) + assert.EqualError(t, err, expectedErr.Error()) + }) + } +} + +// TestCollectionSyncMode_ShouldCreateFetcher tests the ShouldCreateFetcher method with all combinations +// of modes and execution data sync enabled/disabled. +func TestCollectionSyncMode_ShouldCreateFetcher(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string + }{ + { + name: "execution_first_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: true, + expectedShouldCreate: false, + description: "should not create fetcher when execution data sync is enabled", + }, + { + name: "execution_first_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should create fetcher when execution data sync is disabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should always create fetcher for execution_and_collection mode", + }, + { + name: "execution_and_collection_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should always create fetcher for execution_and_collection mode", + }, + { + name: "collection_only_with_execution_data_sync_enabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should always create fetcher for collection_only mode", + }, + { + name: "collection_only_with_execution_data_sync_disabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: false, + expectedShouldCreate: true, + description: "should always create fetcher for collection_only mode", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + result := test.mode.ShouldCreateFetcher(test.executionDataSyncEnabled) + assert.Equal(t, test.expectedShouldCreate, result, test.description) + }) + } +} + +// TestCollectionSyncMode_ShouldCreateExecutionDataProcessor tests the ShouldCreateExecutionDataProcessor method +// with all combinations of modes and execution data sync enabled/disabled. +func TestCollectionSyncMode_ShouldCreateExecutionDataProcessor(t *testing.T) { + tests := []struct { + name string + mode CollectionSyncMode + executionDataSyncEnabled bool + expectedShouldCreate bool + description string + }{ + { + name: "execution_first_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should create processor when execution data sync is enabled", + }, + { + name: "execution_first_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionFirst, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor when execution data sync is disabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_enabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: true, + expectedShouldCreate: true, + description: "should create processor when execution data sync is enabled", + }, + { + name: "execution_and_collection_with_execution_data_sync_disabled", + mode: CollectionSyncModeExecutionAndCollection, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor when execution data sync is disabled", + }, + { + name: "collection_only_with_execution_data_sync_enabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: true, + expectedShouldCreate: false, + description: "should not create processor for collection_only mode even with execution data sync enabled", + }, + { + name: "collection_only_with_execution_data_sync_disabled", + mode: CollectionSyncModeCollectionOnly, + executionDataSyncEnabled: false, + expectedShouldCreate: false, + description: "should not create processor for collection_only mode", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + result := test.mode.ShouldCreateExecutionDataProcessor(test.executionDataSyncEnabled) + assert.Equal(t, test.expectedShouldCreate, result, test.description) + }) + } +} diff --git a/engine/access/collection_sync/factory/config.go b/engine/access/collection_sync/factory/config.go new file mode 100644 index 00000000000..ac51903157e --- /dev/null +++ b/engine/access/collection_sync/factory/config.go @@ -0,0 +1,16 @@ +package factory + +import "time" + +// CreateFetcherConfig holds configuration parameters for creating a Fetcher. +type CreateFetcherConfig struct { + // MaxProcessing is the maximum number of jobs to process concurrently. + MaxProcessing uint64 + // MaxSearchAhead is the maximum number of jobs beyond processedIndex to process. 0 means no limit. + MaxSearchAhead uint64 + // RetryInterval is the interval for retrying missing collections. If 0, uses DefaultRetryInterval. + RetryInterval time.Duration +} + +const DefaultMaxProcessing = 10 +const DefaultMaxSearchAhead = 20 diff --git a/engine/access/collection_sync/factory/execution_data_processor.go b/engine/access/collection_sync/factory/execution_data_processor.go new file mode 100644 index 00000000000..25091b14803 --- /dev/null +++ b/engine/access/collection_sync/factory/execution_data_processor.go @@ -0,0 +1,52 @@ +package factory + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/execution_data_index" + "github.com/onflow/flow-go/module/counters" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/module/state_synchronization" + "github.com/onflow/flow-go/storage" +) + +// createExecutionDataProcessor creates a new ExecutionDataProcessor with the provided dependencies. +// +// Parameters: +// - log: Logger for the component +// - cache: Execution data cache for retrieving execution data by height +// - executionDataTracker: Tracker for execution data that provides the highest available height +// - processedHeight: Consumer progress for tracking processed heights +// - indexer: Block collection indexer for indexing collections +// - onIndexedCallback: Callback function to be called when a block's execution data has been indexed +// +// Returns: +// - *ExecutionDataProcessor: A new ExecutionDataProcessor instance +// - error: An error if the processor could not be created +// +// No errors are expected during normal operation. +func createExecutionDataProcessor( + log zerolog.Logger, + cache execution_data.ExecutionDataCache, + executionDataIndexedHeight state_synchronization.ExecutionDataIndexedHeight, + processedHeight storage.ConsumerProgress, + indexer collection_sync.BlockCollectionIndexer, + onIndexedCallback func(uint64), +) (*execution_data_index.ExecutionDataProcessor, error) { + // Create execution data provider + executionDataProvider := execution_data_index.NewExecutionDataProvider(cache, executionDataIndexedHeight) + + // Convert ConsumerProgress to PersistentStrictMonotonicCounter + processedHeightCounter, err := counters.NewPersistentStrictMonotonicCounter(processedHeight) + if err != nil { + return nil, fmt.Errorf("failed to create persistent strict monotonic counter: %w", err) + } + + // Create the execution data processor + processor := execution_data_index.NewExecutionDataProcessor(log, executionDataProvider, indexer, processedHeightCounter, onIndexedCallback) + + return processor, nil +} diff --git a/engine/access/collection_sync/factory/fetcher.go b/engine/access/collection_sync/factory/fetcher.go new file mode 100644 index 00000000000..017e7b1dbf6 --- /dev/null +++ b/engine/access/collection_sync/factory/fetcher.go @@ -0,0 +1,134 @@ +package factory + +import ( + "fmt" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/consensus/hotstuff" + "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/engine/access/collection_sync/fetcher" + "github.com/onflow/flow-go/engine/common/requester" + "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/network" + "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// createFetcher creates a new Fetcher component with all its dependencies. +// +// Parameters: +// - log: Logger for the component +// - engineMetrics: Metrics collector for the requester engine +// - engineRegistry: Engine registry for creating the requester engine +// - state: Protocol state +// - me: Local node identity +// - blocks: Blocks storage +// - processedFinalizedBlockHeight: Initializer for tracking processed block heights +// - collectionSyncMetrics: Optional metrics collector for tracking collection sync progress +// - lastFullBlockHeight: Progress reader to get the max of fetcher and syncer heights +// - accessMetrics: Access metrics for reporting last_full_finalized_block_height +// - config: Configuration for the fetcher +// +// Returns both the Fetcher and BlockProcessor so they can be reused in other components. +// +// No error returns are expected during normal operation. +func createFetcher( + log zerolog.Logger, + engineMetrics module.EngineMetrics, + engineRegistry network.EngineRegistry, + state protocol.State, + me module.Local, + blocks storage.Blocks, + indexer collection_sync.BlockCollectionIndexer, + processedFinalizedBlockHeight storage.ConsumerProgressInitializer, + distributor hotstuff.Distributor, + collectionSyncMetrics module.CollectionSyncMetrics, + lastFullBlockHeight collection_sync.ProgressReader, + accessMetrics module.AccessMetrics, + config CreateFetcherConfig, +) (*requester.Engine, collection_sync.Fetcher, error) { + // Create requester engine for requesting collections + requestEng, err := requester.New( + log.With().Str("entity", "collection").Logger(), + engineMetrics, + engineRegistry, + me, + state, + channels.RequestCollections, + filter.HasRole[flow.Identity](flow.RoleCollection), + func() flow.Entity { return new(flow.Collection) }, + requester.WithBatchInterval(60*time.Second), + requester.WithValidateStaking(false), + requester.WithRetryMaximum(10*time.Second), + ) + if err != nil { + return nil, nil, fmt.Errorf("could not create requester engine: %w", err) + } + + // Create MissingCollectionQueue + mcq := fetcher.NewMissingCollectionQueue() + + // Create CollectionRequester + collectionRequester := fetcher.NewCollectionRequester( + requestEng, + state, + ) + + // Create BlockProcessor + blockProcessor := fetcher.NewBlockProcessor( + log, + mcq, + indexer, + collectionRequester, + blocks, + ) + + // Register handler for received collections + requestEng.WithHandle(func(originID flow.Identifier, entity flow.Entity) { + collection, ok := entity.(*flow.Collection) + if !ok { + return + } + + // Forward collection to BlockProcessor, which handles MCQ, indexing, and completion + err := blockProcessor.OnReceiveCollection(originID, collection) + if err != nil { + log.Fatal().Err(err).Msg("failed to process received collection") + return + } + }) + + // Create Fetcher with callback to update last_full_finalized_block_height metric + collectionFetcher, err := fetcher.NewFetcher( + log, + blockProcessor, + processedFinalizedBlockHeight, + state, + blocks, + config.MaxProcessing, + config.MaxSearchAhead, + collectionSyncMetrics, + func() { + // Update last_full_finalized_block_height metric with the max of both heights + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + }, + config.RetryInterval, + ) + if err != nil { + return nil, nil, fmt.Errorf("could not create fetcher: %w", err) + } + + accessMetrics.UpdateLastFullBlockHeight(lastFullBlockHeight.ProcessedHeight()) + + distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { + collectionFetcher.OnFinalizedBlock() + }) + + return requestEng, collectionFetcher, nil +} diff --git a/engine/access/collection_sync/factory/progress_reader.go b/engine/access/collection_sync/factory/progress_reader.go new file mode 100644 index 00000000000..1756b2b23c8 --- /dev/null +++ b/engine/access/collection_sync/factory/progress_reader.go @@ -0,0 +1,69 @@ +package factory + +import "github.com/onflow/flow-go/engine/access/collection_sync" + +// ProgressReader aggregates progress from multiple backends and returns the maximum +// processed height. It can be initialized with an readonly lastProgress value and +// two optional backends: executionDataProcessor and collectionFetcher. +type ProgressReader struct { + // this is a readonly value set at initialization so that if ProcessedHeight was + // called before any backend was set, it can still return a meaningful value + lastProgress uint64 + executionDataProcessor collection_sync.ProgressReader + collectionFetcher collection_sync.ProgressReader +} + +var _ collection_sync.ProgressReader = (*ProgressReader)(nil) + +// NewProgressReader creates a new ProgressReader initialized with lastProgress. +// Backends can be added using SetExecutionDataProcessor and SetCollectionFetcher. +func NewProgressReader(lastProgress uint64) *ProgressReader { + return &ProgressReader{ + lastProgress: lastProgress, + executionDataProcessor: nil, + collectionFetcher: nil, + } +} + +// SetExecutionDataProcessor sets the execution data processor backend. +func (pr *ProgressReader) SetExecutionDataProcessor(backend collection_sync.ProgressReader) { + pr.executionDataProcessor = backend +} + +// SetCollectionFetcher sets the collection fetcher backend. +func (pr *ProgressReader) SetCollectionFetcher(backend collection_sync.ProgressReader) { + pr.collectionFetcher = backend +} + +// ProcessedHeight returns the maximum processed height from the available backends. +// If both backends are available, it returns the maximum of their progress and lastProgress. +// If only one backend is available, it returns the maximum of that backend's progress and lastProgress. +// If neither backend is available, it returns lastProgress. +func (pr *ProgressReader) ProcessedHeight() uint64 { + hasExecutionData := pr.executionDataProcessor != nil + hasCollectionFetcher := pr.collectionFetcher != nil + + if hasExecutionData && hasCollectionFetcher { + execHeight := pr.executionDataProcessor.ProcessedHeight() + collectionHeight := pr.collectionFetcher.ProcessedHeight() + return max(execHeight, collectionHeight, pr.lastProgress) + } + + if hasExecutionData { + execHeight := pr.executionDataProcessor.ProcessedHeight() + if pr.lastProgress > execHeight { + return pr.lastProgress + } + return execHeight + } + + if hasCollectionFetcher { + collectionHeight := pr.collectionFetcher.ProcessedHeight() + if pr.lastProgress > collectionHeight { + return pr.lastProgress + } + return collectionHeight + } + + return pr.lastProgress +} diff --git a/engine/access/collection_sync/fetcher/block_processor.go b/engine/access/collection_sync/fetcher/block_processor.go new file mode 100644 index 00000000000..9f8b4cb1070 --- /dev/null +++ b/engine/access/collection_sync/fetcher/block_processor.go @@ -0,0 +1,247 @@ +package fetcher + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" +) + +// BlockProcessor implements the job lifecycle for collection indexing. +// It orchestrates the flow: request → receive → index → complete. +type BlockProcessor struct { + log zerolog.Logger + mcq collection_sync.MissingCollectionQueue + indexer collection_sync.BlockCollectionIndexer + requester collection_sync.CollectionRequester + blocks storage.Blocks +} + +var _ collection_sync.BlockProcessor = (*BlockProcessor)(nil) + +// NewBlockProcessor creates a new BlockProcessor. +// +// Parameters: +// - log: Logger for the component +// - mcq: MissingCollectionQueue for tracking missing collections and callbacks +// - indexer: BlockCollectionIndexer for storing and indexing collections +// - requester: CollectionRequester for requesting collections from the network +// - blocks: Blocks storage for retrieving blocks to extract guarantees +// +// No error returns are expected during normal operation. +func NewBlockProcessor( + log zerolog.Logger, + mcq collection_sync.MissingCollectionQueue, + indexer collection_sync.BlockCollectionIndexer, + requester collection_sync.CollectionRequester, + blocks storage.Blocks, +) *BlockProcessor { + return &BlockProcessor{ + log: log.With().Str("coll_sync", "fetcher_processor").Logger(), + mcq: mcq, + indexer: indexer, + requester: requester, + blocks: blocks, + } +} + +// FetchCollections processes a block for collection fetching. +// It checks if the block is already indexed, and if not, enqueues missing collections +// and optionally requests them based on EDI lag. +// +// No error returns are expected during normal operation. +func (bp *BlockProcessor) FetchCollections( + ctx irrecoverable.SignalerContext, + block *flow.Block, + done func(), +) error { + blockHeight := block.Height + bp.log.Debug().Uint64("block_height", blockHeight). + Msg("processing collection fetching job for finalized block") + + // Get missing collections for this block + missingGuarantees, err := bp.indexer.GetMissingCollections(block) + if err != nil { + return fmt.Errorf("failed to get missing collections for block height %d: %w", blockHeight, err) + } + + // If there are no missing collections, this block is considered complete. + // Caution: This relies on the assumption that: + // whenever a collection exists in storage, all of its transactions must have already been indexed. + // This assumption currently holds because transaction indexing by collection is always performed + // in the same batch that stores the collection (via collections.BatchStoreAndIndexByTransaction). + // Note: when we receives a collection, we need it in memory, and don't index until all collections + // of the block are received. + if len(missingGuarantees) == 0 { + done() + return nil + } + + // Extract collection IDs for enqueueing + collectionIDs := make([]flow.Identifier, len(missingGuarantees)) + for i, guarantee := range missingGuarantees { + collectionIDs[i] = guarantee.CollectionID + } + + // Enqueue missing collections with notifyJobCompletion + // When all collections are received and indexed, the notifyJobCompletion callback + // will be returned in the OnIndexedForBlock method, calling it will notify the job + // queue that the job is complete. + notifyJobCompletion := done + + err = bp.mcq.EnqueueMissingCollections(blockHeight, collectionIDs, notifyJobCompletion) + if err != nil { + return fmt.Errorf("failed to enqueue missing collections for block height %d: %w", blockHeight, err) + } + + // Request collections from collection nodes + // Note: requester does not guarantee it will keep trying to fetch the collections, + // in fact, if a collection node is down, the request may fail permanently, + // so we need to have a retry mechanism in place (see RetryFetchingMissingCollections), + // which allows requester to retry with a different collection node. + err = bp.requester.RequestCollectionsByGuarantees(missingGuarantees) + if err != nil { + return fmt.Errorf("failed to request collections for block height %d: %w", blockHeight, err) + } + + bp.log.Debug().Uint64("block_height", blockHeight). + Int("missing_collections_count", len(missingGuarantees)). + Msg("fetched collections for finalized block") + + return nil +} + +// OnReceiveCollection is called when a collection is received from the requester. +// It passes the collection to MCQ, and if it completes a block, indexes it and marks it as done. +// +// No error returns are expected during normal operation. +func (bp *BlockProcessor) OnReceiveCollection(originID flow.Identifier, collection *flow.Collection) error { + collectionID := collection.ID() + + // Pass collection to MCQ + collections, height, missingCol, complete := bp.mcq.OnReceivedCollection(collection) + + // Log collection receipt and whether it completes a block + if complete { + bp.log.Info(). + Hex("collection_id", collectionID[:]). + Hex("origin_id", originID[:]). + Uint64("block_height", height). + Int("collections_count", len(collections)). + Msg("received collection completing block to be indexed") + } else { + bp.log.Debug(). + Hex("collection_id", collectionID[:]). + Hex("origin_id", originID[:]). + // there might be multiple missing collections, just log one of them + Hex("missing_col", missingCol[:]). + Uint64("block_height", height). + Msg("received collection (block not yet complete)") + } + + // OnReceiveCollection might be called multiple times for the same collection + // but complete will only be true once per block height. + if !complete { + // Block is not complete yet, nothing more to do + // or block is complete, but another thread is indexing it + return nil + } + + // Block became complete, index it + err := bp.indexer.IndexCollectionsForBlock(height, collections) + if err != nil { + return fmt.Errorf("failed to index collections for block height %d: %w", height, err) + } + + // Mark the block as indexed + notifyJobCompletion, ok := bp.mcq.OnIndexedForBlock(height) + if ok { + notifyJobCompletion() + } + + return nil +} + +// RetryFetchingMissingCollections retries fetching all missing collections currently in the queue. +// It retrieves blocks for the heights with missing collections and extracts the corresponding guarantees. +// Note, the caller is responsible to ensure this method is called periodically, and not concurrently. +// +// No error returns are expected during normal operation. +func (bp *BlockProcessor) RetryFetchingMissingCollections() error { + // Get missing collections grouped by height + missingByHeight := bp.mcq.GetMissingCollectionsByHeight() + if len(missingByHeight) == 0 { + return nil + } + + // Collect guarantees for missing collections by retrieving blocks + // Double-check that collections are actually missing in storage (they may have been + // indexed by execution data indexer or received from another source) + var guarantees []*flow.CollectionGuarantee + + for height := range missingByHeight { + // Get block for this height + block, err := bp.blocks.ByHeight(height) + if err != nil { + bp.log.Fatal(). + Uint64("block_height", height). + Err(err). + Msg("failed to retrieve block for retrying missing collections, skipping") + continue + } + + // Use indexer's GetMissingCollections to filter out collections that already exist + // This handles the case where execution data indexer may have already indexed them + actuallyMissingGuarantees, err := bp.indexer.GetMissingCollections(block) + if err != nil { + bp.log.Warn(). + Uint64("block_height", height). + Err(err). + Msg("failed to check missing collections for block, skipping") + continue + } + + // If no collections are missing, all collections for this block are now indexed + if len(actuallyMissingGuarantees) == 0 { + // All collections for this block are now indexed, mark it as done + notifyJobCompletion, ok := bp.mcq.OnIndexedForBlock(height) + if ok { + notifyJobCompletion() + bp.log.Info(). + Uint64("block_height", height). + Msg("all collections for block are now indexed, marked block as done") + } + continue + } + + // Add guarantees for collections that are still missing + guarantees = append(guarantees, actuallyMissingGuarantees...) + } + + if len(guarantees) == 0 { + bp.log.Debug().Msg("no guarantees found for missing collections") + return nil + } + + // Request collections using the guarantees + err := bp.requester.RequestCollectionsByGuarantees(guarantees) + if err != nil { + return fmt.Errorf("failed to retry requesting collections: %w", err) + } + + bp.log.Info(). + Int("guarantees_count", len(guarantees)). + Int("heights_count", len(missingByHeight)). + Msg("retried fetching missing collections") + + return nil +} + +// MissingCollectionQueueSize returns the number of missing collections currently in the queue. +func (bp *BlockProcessor) MissingCollectionQueueSize() uint { + return bp.mcq.Size() +} diff --git a/engine/access/collection_sync/fetcher/engine.go b/engine/access/collection_sync/fetcher/engine.go new file mode 100644 index 00000000000..7e66f81618f --- /dev/null +++ b/engine/access/collection_sync/fetcher/engine.go @@ -0,0 +1,222 @@ +package fetcher + +import ( + "fmt" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/jobqueue" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// Fetcher is a component that consumes finalized block jobs and processes them +// to index collections. It uses a job consumer with windowed throttling to prevent node overload. +type Fetcher struct { + component.Component + + consumer *jobqueue.ComponentConsumer + blockProcessor collection_sync.BlockProcessor + workSignal engine.Notifier + metrics module.CollectionSyncMetrics + retryInterval time.Duration +} + +var _ collection_sync.Fetcher = (*Fetcher)(nil) +var _ collection_sync.ProgressReader = (*Fetcher)(nil) +var _ component.Component = (*Fetcher)(nil) + +// DefaultRetryInterval is the default interval for retrying missing collections. +const DefaultRetryInterval = 30 * time.Second + +// NewFetcher creates a new Fetcher component. +// +// Parameters: +// - log: Logger for the component +// - blockProcessor: BlockProcessor implementation for processing collection indexing jobs +// - progressInitializer: Initializer for tracking processed block heights +// - state: Protocol state for reading finalized block information +// - blocks: Blocks storage for reading blocks by height +// - maxProcessing: Maximum number of jobs to process concurrently +// - maxSearchAhead: Maximum number of jobs beyond processedIndex to process. 0 means no limit +// - metrics: Optional metrics collector for reporting collection fetched height +// - onHeightUpdated: Optional callback to be called when processed height is updated +// - retryInterval: Interval for retrying missing collections. If 0, uses DefaultRetryInterval +// +// No error returns are expected during normal operation. +func NewFetcher( + log zerolog.Logger, + blockProcessor collection_sync.BlockProcessor, + progressInitializer storage.ConsumerProgressInitializer, + state protocol.State, + blocks storage.Blocks, + maxProcessing uint64, // max number of blocks to fetch collections concurrently + maxSearchAhead uint64, // max number of blocks beyond the next unfullfilled height to fetch collections for + metrics module.CollectionSyncMetrics, // optional metrics collector + onHeightUpdated func(), // optional callback when processed height is updated + retryInterval time.Duration, // interval for retrying missing collections +) (*Fetcher, error) { + workSignal := engine.NewNotifier() + + // Read the default index from the sealed root height + defaultIndex := state.Params().SealedRoot().Height + + // Create a Jobs instance that reads finalized blocks by height + // each job is a finalized block + jobs := jobqueue.NewFinalizedBlockReader(state, blocks) + + // Create an adapter function that wraps the BlockProcessor interface + // For each finalized block, a worker is responsible for fetching all collections for that block. + // Since a block may contain multiple collections and fetching is asynchronous, tracking which + // collections have been received requires stateful management. The blockProcessor handles this + // by fetching collections and maintaining state about which collections are still missing. + // It also invokes the done callback to notify the job consumer when the worker has completed + // processing all collections for the block. + // + // The processor function translates a job (finalized block) and passes it to the blockProcessor + // to fetch and index the associated collections. + processorFunc := func(ctx irrecoverable.SignalerContext, job module.Job, done func()) { + // Convert job to block + block, err := jobs.ConvertJobToBlock(job) + if err != nil { + ctx.Throw(fmt.Errorf("could not convert job to block: %w", err)) + return + } + + err = blockProcessor.FetchCollections(ctx, block, done) + if err != nil { + ctx.Throw(fmt.Errorf("failed to process collection indexing job: %w", err)) + } + } + + consumer, err := jobqueue.NewComponentConsumer( + log.With().Str("coll_sync", "fetcher").Logger(), + workSignal.Channel(), + progressInitializer, + jobs, + defaultIndex, + processorFunc, + maxProcessing, + maxSearchAhead, + ) + if err != nil { + return nil, fmt.Errorf("failed to create collection syncing consumer: %w", err) + } + + if retryInterval == 0 { + retryInterval = DefaultRetryInterval + } + + f := &Fetcher{ + consumer: consumer, + blockProcessor: blockProcessor, + workSignal: workSignal, + metrics: metrics, + retryInterval: retryInterval, + } + + // Set up post-notifier to update metrics when a job is done + // Only update metrics when the processed height actually changes, since processedIndex + // only advances when consecutive jobs complete, not on every individual job completion. + consumer.SetPostNotifier(func(jobID module.JobID) { + metrics.CollectionFetchedHeight(f.ProcessedHeight()) + metrics.MissingCollectionQueueSize(f.blockProcessor.MissingCollectionQueueSize()) + if onHeightUpdated != nil { + onHeightUpdated() + } + }) + + // report the initial metrics, otherwise it creates spikes in dashboard + metrics.CollectionFetchedHeight(f.ProcessedHeight()) + if onHeightUpdated != nil { + onHeightUpdated() + } + + // Create a ComponentManager that includes both the consumer and the retry worker + componentManager := component.NewComponentManagerBuilder(). + AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + // Start the consumer component + consumer.Start(ctx) + select { + case <-ctx.Done(): + return + case <-consumer.Ready(): + ready() + } + <-consumer.Done() + }). + AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + // Retry worker that periodically retries missing collections + ready() + f.retryMissingCollectionsLoop(ctx, log) + }). + Build() + + f.Component = componentManager + + return f, nil +} + +// OnFinalizedBlock is called when a new block is finalized. It notifies the job consumer +// that new work is available. +func (s *Fetcher) OnFinalizedBlock() { + s.workSignal.Notify() +} + +// LastProcessedIndex returns the last processed job index. +func (s *Fetcher) ProcessedHeight() uint64 { + return s.consumer.LastProcessedIndex() +} + +// Size returns the number of in-memory jobs that the consumer is processing. +// Optional methods, not required for operation but useful for monitoring. +func (s *Fetcher) Size() uint { + return s.consumer.Size() +} + +// retryMissingCollectionsLoop periodically retries fetching missing collections. +func (f *Fetcher) retryMissingCollectionsLoop(ctx irrecoverable.SignalerContext, log zerolog.Logger) { + ticker := time.NewTicker(f.retryInterval) + defer ticker.Stop() + + log.Info(). + Dur("retry_interval", f.retryInterval). + Msg("starting missing collections retry worker") + + // Wait for the consumer to be ready before starting retries + select { + case <-ctx.Done(): + return + case <-f.consumer.Ready(): + } + + for { + select { + case <-ctx.Done(): + log.Info().Msg("missing collections retry worker shutting down") + return + case <-ticker.C: + err := f.blockProcessor.RetryFetchingMissingCollections() + if err != nil { + log.Error(). + Err(err). + Msg("failed to retry fetching missing collections") + // Don't throw - this is a retry mechanism, failures are expected + // and we'll try again on the next interval + } else { + queueSize := f.blockProcessor.MissingCollectionQueueSize() + if queueSize > 0 { + log.Debug(). + Uint("missing_collections", queueSize). + Msg("retried fetching missing collections") + } + } + } + } +} diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue.go b/engine/access/collection_sync/fetcher/missing_collection_queue.go new file mode 100644 index 00000000000..12787af0063 --- /dev/null +++ b/engine/access/collection_sync/fetcher/missing_collection_queue.go @@ -0,0 +1,316 @@ +package fetcher + +import ( + "fmt" + "sync" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" +) + +// MissingCollectionQueue helps the job processor to keep track of the jobs and their callbacks. +// Note, it DOES NOT index collections directly, instead, it only keeps track of which collections are missing +// for each block height, and when all collections for a block height have been received, it returns the +// collections to the caller for processing (storing and indexing). And let the caller to notify the completion +// of the processing, so that it can mark the job as done by calling the callback. +// This allows the MissingCollectionQueue to be decoupled from the actual processing of the collections, keep +// all states in memory and allow the different callers to hold the lock less time and reduce contention. +// +// The caller is responsible for checking if collections are already in storage before enqueueing them. +// Only collections that are actually missing should be passed to EnqueueMissingCollections. +// +// MissingCollectionQueue is safe for concurrent use. +type MissingCollectionQueue struct { + // mu protects all the maps below. + mu sync.RWMutex + + // blockJobs maps block height to the job state for that height. + blockJobs map[uint64]*blockJobState + + // collectionToHeight maps collection ID to the single block height waiting for that collection. + // This enforces a 1:1 relationship: each collection belongs to exactly one block. + // This allows efficient lookup when a collection is received. + collectionToHeight map[flow.Identifier]uint64 +} + +// blockJobState tracks the state of a job for a specific block height. +type blockJobState struct { + // missingCollections is a set of collection IDs that are still missing for this block height. + missingCollections map[flow.Identifier]struct{} + // receivedCollections stores the collections that have been received so far, keyed by collection ID. + // This allows us to return all collections when the block becomes complete. + receivedCollections map[flow.Identifier]*flow.Collection + // callback is invoked when all collections for this block height have been received and indexed. + callback func() +} + +var _ collection_sync.MissingCollectionQueue = (*MissingCollectionQueue)(nil) + +// NewMissingCollectionQueue creates a new MissingCollectionQueue. +// +// No error returns are expected during normal operation. +func NewMissingCollectionQueue() *MissingCollectionQueue { + return &MissingCollectionQueue{ + blockJobs: make(map[uint64]*blockJobState), + collectionToHeight: make(map[flow.Identifier]uint64), + } +} + +// pickOne returns an arbitrary key from the given map. +// The map must be non-empty. The selection is non-deterministic due to Go's map iteration order. +func pickOne(m map[flow.Identifier]struct{}) flow.Identifier { + for id := range m { + return id + } + return flow.ZeroID +} + +// EnqueueMissingCollections registers missing collections for a block height along with a callback +// that will be invoked when all collections for that height have been received and indexed. +// +// The caller is responsible for checking if collections are already in storage before calling this method. +// Only collections that are actually missing should be passed in collectionIDs. +// +// Returns an error if the block height is already enqueued to prevent overwriting existing jobs. +func (mcq *MissingCollectionQueue) EnqueueMissingCollections( + blockHeight uint64, + collectionIDs []flow.Identifier, + callback func(), +) error { + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Check if block height is already enqueued to prevent overwriting. + if _, exists := mcq.blockJobs[blockHeight]; exists { + return fmt.Errorf( + "block height %d is already enqueued, cannot overwrite existing job", + blockHeight, + ) + } + + // Create the job state with all collections marked as missing. + // The caller has already verified these collections are not in storage. + missingSet := make(map[flow.Identifier]struct{}, len(collectionIDs)) + for _, id := range collectionIDs { + missingSet[id] = struct{}{} + } + + jobState := &blockJobState{ + missingCollections: missingSet, + receivedCollections: make(map[flow.Identifier]*flow.Collection), + callback: callback, + } + + mcq.blockJobs[blockHeight] = jobState + + // Update the collection-to-height mapping, enforcing 1:1 relationship. + for _, collectionID := range collectionIDs { + existingHeight, exists := mcq.collectionToHeight[collectionID] + if exists && existingHeight != blockHeight { + // Collection is already assigned to a different block - this violates the 1:1 constraint. + return fmt.Errorf( + "fatal: collection %v is already assigned to block height %d, cannot assign to height %d", + collectionID, existingHeight, blockHeight, + ) + } + mcq.collectionToHeight[collectionID] = blockHeight + } + + return nil +} + +// OnReceivedCollection notifies the queue that a collection has been received. +// It checks if the block height is now complete and returns the collections and height. +// +// The collection parameter should be the actual collection object received from the requester. +// +// Returns: +// - (collections, height, missingCollectionID, true) if the block height became complete +// - (nil, 0, missingCollectionID, false) if no block height became complete +// missingCollectionID is an arbitrary ID from the remaining missing collections, or ZeroID if none. +func (mcq *MissingCollectionQueue) OnReceivedCollection( + collection *flow.Collection, +) ([]*flow.Collection, uint64, flow.Identifier, bool) { + collectionID := collection.ID() + + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Find the block height waiting for this collection (1:1 relationship). + height, ok := mcq.collectionToHeight[collectionID] + if !ok { + // No block is waiting for this collection. + return nil, 0, flow.ZeroID, false + } + + jobState, exists := mcq.blockJobs[height] + if !exists { + // Job was already completed/removed. + // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. + return nil, height, flow.ZeroID, false + } + + // Check if this collection was still missing for this block. + if _, wasMissing := jobState.missingCollections[collectionID]; !wasMissing { + // Collection was already received or wasn't part of this block's missing set. + // Don't delete from collectionToHeight - cleanup happens in OnIndexedForBlock. + return nil, height, pickOne(jobState.missingCollections), false + } + + // Remove from missing set and add to received collections. + delete(jobState.missingCollections, collectionID) + // Store the collection so it can be returned when the block is complete. + jobState.receivedCollections[collectionID] = collection + + // Don't delete from collectionToHeight - the mapping is kept until OnIndexedForBlock cleans it up. + + // Check if the block is now complete (all collections received). + if len(jobState.missingCollections) > 0 { + // pick a random missing collection to return + // useful for logging/debugging purposes + // in case fetching is stuck, it's useful to know which collections are still missing + // we don't need to return all missing collections, just one is enough + return nil, height, pickOne(jobState.missingCollections), false + } + + // Return all received collections for this block. + collections := make([]*flow.Collection, 0, len(jobState.receivedCollections)) + for _, col := range jobState.receivedCollections { + collections = append(collections, col) + } + return collections, height, flow.ZeroID, true +} + +// IsHeightQueued returns true if the given height has queued collections +// Returns false if the height is not tracked +func (mcq *MissingCollectionQueue) IsHeightQueued(height uint64) bool { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + _, exists := mcq.blockJobs[height] + return exists +} + +// OnIndexedForBlock notifies the queue that a block height has been indexed, +// removes that block height from tracking, and return the callback for caller to +// invoke. +// +// Returns: +// (callback, true) if the height existed and was processed; +// (nil, false) if the height was not tracked. +// +// Note, caller should invoke the returned callback if not nil. +// +// Behavior: OnIndexedForBlock can return the callback even before the block is +// complete (i.e., before all collections have been received). This allows the caller +// to index a block with partial collections if needed. After indexing: +// - The block is removed from tracking (IsHeightQueued returns false) +// - All collection-to-height mappings for this block are cleaned up +// - Any remaining missing collections are removed from tracking +// - Subsequent OnReceivedCollection calls for collections belonging to this block +// will return (nil, 0, false) because the block has been removed +func (mcq *MissingCollectionQueue) OnIndexedForBlock(blockHeight uint64) (func(), bool) { + mcq.mu.Lock() + defer mcq.mu.Unlock() + + // Clean up all collection-to-height mappings and remove the block job. + // This ensures the height is removed from tracking once the callback is invoked. + jobState, exists := mcq.cleanupCollectionMappingsForHeight(blockHeight) + if !exists { + // Block was not tracked or already completed + return nil, false + } + + // Get the callback from the job state. + return jobState.callback, true +} + +// cleanupCollectionMappingsForHeight removes all collection-to-height mappings for collections +// belonging to the specified block height and removes the block job from tracking. +// This includes both missing and received collections. +// +// Returns the job state and a bool indicating if the block height existed. +// +// This method must be called while holding the write lock (mcq.mu.Lock()). +func (mcq *MissingCollectionQueue) cleanupCollectionMappingsForHeight( + blockHeight uint64, +) (*blockJobState, bool) { + jobState, exists := mcq.blockJobs[blockHeight] + if !exists { + return nil, false + } + + // Clean up from missing collections. + for collectionID := range jobState.missingCollections { + delete(mcq.collectionToHeight, collectionID) + } + // Clean up from received collections. + for collectionID := range jobState.receivedCollections { + delete(mcq.collectionToHeight, collectionID) + } + + delete(mcq.blockJobs, blockHeight) + + return jobState, true +} + +// Size returns the number of incomplete jobs currently in the queue. +func (mcq *MissingCollectionQueue) Size() uint { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + return uint(len(mcq.blockJobs)) +} + +// GetMissingCollections returns all collection IDs that are currently missing across all block heights. +// +// Returns a slice of collection identifiers that are still missing. The order is non-deterministic +// due to map iteration order. Returns an empty slice if there are no missing collections. +func (mcq *MissingCollectionQueue) GetMissingCollections() []flow.Identifier { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + // Count total missing collections to pre-allocate slice + totalMissing := 0 + for _, jobState := range mcq.blockJobs { + totalMissing += len(jobState.missingCollections) + } + + // Pre-allocate slice with capacity + missingCollections := make([]flow.Identifier, 0, totalMissing) + + // Collect all missing collection IDs from all block jobs + for _, jobState := range mcq.blockJobs { + for collectionID := range jobState.missingCollections { + missingCollections = append(missingCollections, collectionID) + } + } + + return missingCollections +} + +// GetMissingCollectionsByHeight returns a map of block height to collection IDs that are missing for that height. +// +// Returns a map where keys are block heights and values are slices of collection identifiers +// that are still missing for that height. Returns an empty map if there are no missing collections. +func (mcq *MissingCollectionQueue) GetMissingCollectionsByHeight() map[uint64][]flow.Identifier { + mcq.mu.RLock() + defer mcq.mu.RUnlock() + + // Build map of height -> collection IDs + missingByHeight := make(map[uint64][]flow.Identifier) + + for height, jobState := range mcq.blockJobs { + if len(jobState.missingCollections) == 0 { + continue + } + + collectionIDs := make([]flow.Identifier, 0, len(jobState.missingCollections)) + for collectionID := range jobState.missingCollections { + collectionIDs = append(collectionIDs, collectionID) + } + missingByHeight[height] = collectionIDs + } + + return missingByHeight +} diff --git a/engine/access/collection_sync/fetcher/missing_collection_queue_test.go b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go new file mode 100644 index 00000000000..9ca4da7a031 --- /dev/null +++ b/engine/access/collection_sync/fetcher/missing_collection_queue_test.go @@ -0,0 +1,146 @@ +package fetcher + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +// TestMissingCollectionQueue_CompleteBlockLifecycle tests the complete lifecycle of a block: +// 1. Initially IsHeightQueued returns false +// 2. After enqueuing, IsHeightQueued returns true +// 3. Enqueuing twice should error (not idempotent - prevents overwriting) +// 4. Receiving one collection doesn't complete the block, IsHeightQueued still returns true +// 5. Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) +// 6. OnIndexedForBlock returns the callback +// 7. After indexing, IsHeightQueued returns false +// 8. OnReceivedCollection for that block returns false because the block has been removed +func TestMissingCollectionQueue_CompleteBlockLifecycle(t *testing.T) { + mcq := NewMissingCollectionQueue() + blockHeight := uint64(100) + + // Create test collections + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackInvoked := false + callback := func() { + callbackInvoked = true + } + + // Step 1: Initially IsHeightQueued returns false + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued initially") + + // Step 2: After enqueuing, IsHeightQueued returns true + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should be queued after enqueuing") + + // Step 3: Enqueuing twice should error (prevents overwriting) + anotherCallback := func() {} + err = mcq.EnqueueMissingCollections(blockHeight, collectionIDs, anotherCallback) + require.Error(t, err) + assert.Contains(t, err.Error(), "already enqueued") + assert.Contains(t, err.Error(), "cannot overwrite existing job") + + // Step 4: Receiving one collection doesn't complete the block, IsHeightQueued still returns true + collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.Equal(t, blockHeight, heightReturned, "should return height even when block is not complete") + assert.False(t, complete, "block should not be complete with only one collection") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued after receiving one collection") + + // Step 5: Receiving all collections completes the block, IsHeightQueued still returns true (not yet indexed) + collectionsReturned, _, _, complete = mcq.OnReceivedCollection(collections[1]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.False(t, complete, "block should not be complete with only two collections") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") + + // Receive the last collection - block should now be complete + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[2]) + assert.NotNil(t, collectionsReturned, "should return collections when block is complete") + assert.Equal(t, 3, len(collectionsReturned), "should return all 3 collections") + assert.Equal(t, blockHeight, heightReturned, "should return correct block height") + assert.True(t, complete, "block should be complete after receiving all collections") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued (not yet indexed)") + + // Step 6: OnIndexedForBlock returns the callback + returnedCallback, exists := mcq.OnIndexedForBlock(blockHeight) + assert.NotNil(t, returnedCallback, "should return callback") + assert.True(t, exists, "block should exist") + assert.False(t, callbackInvoked, "callback should not be invoked yet") + + // Step 7: After indexing, IsHeightQueued returns false + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued after indexing") + + // Invoke the callback + returnedCallback() + assert.True(t, callbackInvoked, "callback should be invoked") + + // Step 8: OnReceivedCollection for that block returns false because the block has been removed + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") + assert.False(t, complete, "should not indicate completion for removed block") +} + +// TestMissingCollectionQueue_IndexBeforeBlockCompletion tests that OnIndexedForBlock +// can return the callback even before the block is complete (i.e., before all collections have been received). +func TestMissingCollectionQueue_IndexBeforeBlockCompletion(t *testing.T) { + mcq := NewMissingCollectionQueue() + blockHeight := uint64(200) + + // Create test collections + collections := unittest.CollectionListFixture(3) + collectionIDs := make([]flow.Identifier, len(collections)) + for i, col := range collections { + collectionIDs[i] = col.ID() + } + + callbackInvoked := false + callback := func() { + callbackInvoked = true + } + + // Enqueue block with 3 collections + err := mcq.EnqueueMissingCollections(blockHeight, collectionIDs, callback) + require.NoError(t, err) + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should be queued") + + // Receive only one collection (block is not complete) + collectionsReturned, heightReturned, _, complete := mcq.OnReceivedCollection(collections[0]) + assert.Nil(t, collectionsReturned, "should not return collections when block is not complete") + assert.Equal(t, blockHeight, heightReturned, "should return height even when block is not complete") + assert.False(t, complete, "block should not be complete with only one collection") + assert.True(t, mcq.IsHeightQueued(blockHeight), "height should still be queued") + + // OnIndexedForBlock can return the callback even before the block is complete + returnedCallback, exists := mcq.OnIndexedForBlock(blockHeight) + assert.NotNil(t, returnedCallback, "should return callback even when block is not complete") + assert.True(t, exists, "block should exist") + assert.False(t, callbackInvoked, "callback should not be invoked yet") + + // After indexing, the block is removed from tracking + assert.False(t, mcq.IsHeightQueued(blockHeight), "height should not be queued after indexing") + + // Verify that remaining collections cannot be received (block has been removed) + collectionsReturned, heightReturned, _, complete = mcq.OnReceivedCollection(collections[1]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.Equal(t, uint64(0), heightReturned, "should not return height for removed block") + assert.False(t, complete, "should not indicate completion for removed block") + + collectionsReturned, _, _, complete = mcq.OnReceivedCollection(collections[2]) + assert.Nil(t, collectionsReturned, "should not return collections for removed block") + assert.False(t, complete, "should not indicate completion for removed block") + + // Invoke the callback + returnedCallback() + assert.True(t, callbackInvoked, "callback should be invoked") +} diff --git a/engine/access/collection_sync/fetcher/requester.go b/engine/access/collection_sync/fetcher/requester.go new file mode 100644 index 00000000000..bf7b2613c2a --- /dev/null +++ b/engine/access/collection_sync/fetcher/requester.go @@ -0,0 +1,59 @@ +package fetcher + +import ( + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/state/protocol" +) + +var _ collection_sync.CollectionRequester = (*CollectionRequester)(nil) + +// CollectionRequester requests collections from collection nodes on the network. +// It implements the collection_sync.CollectionRequester interface. +type CollectionRequester struct { + requester module.Requester + state protocol.State +} + +// NewCollectionRequester creates a new CollectionRequester. +// +// Parameters: +// - requester: The requester engine for requesting entities from the network +// - state: Protocol state for finding guarantors +// +// No error returns are expected during normal operation. +func NewCollectionRequester( + requester module.Requester, + state protocol.State, +) *CollectionRequester { + return &CollectionRequester{ + requester: requester, + state: state, + } +} + +// RequestCollectionsByGuarantees requests collections by their guarantees from collection nodes on the network. +// For each guarantee, it finds the guarantors and requests the collection from them. +// +// No error returns are expected during normal operation. +func (cr *CollectionRequester) RequestCollectionsByGuarantees(guarantees []*flow.CollectionGuarantee) error { + for _, guarantee := range guarantees { + // Find guarantors for this guarantee + guarantors, err := protocol.FindGuarantors(cr.state, guarantee) + if err != nil { + // Failed to find guarantors - this could happen if the reference block is unknown + // or if the cluster is not found. Skip this collection rather than failing entirely. + continue + } + + // Request the collection from the guarantors + cr.requester.EntityByID(guarantee.CollectionID, filter.HasNodeID[flow.Identity](guarantors...)) + } + + // Force immediate dispatch of all pending requests + cr.requester.Force() + + return nil +} diff --git a/engine/access/collection_sync/indexer/indexer.go b/engine/access/collection_sync/indexer/indexer.go new file mode 100644 index 00000000000..ae7dc6ce55b --- /dev/null +++ b/engine/access/collection_sync/indexer/indexer.go @@ -0,0 +1,95 @@ +package indexer + +import ( + "fmt" + + "github.com/jordanschalm/lockctx" + + "github.com/onflow/flow-go/engine/access/collection_sync" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/storage" +) + +// blockCollectionIndexerImpl implements BlockCollectionIndexer. +// It stores and indexes fetcher for a given block height. +type blockCollectionIndexerImpl struct { + metrics module.CollectionExecutedMetric + lockManager lockctx.Manager + db storage.DB + collections storage.Collections +} + +var _ collection_sync.BlockCollectionIndexer = (*blockCollectionIndexerImpl)(nil) + +// NewBlockCollectionIndexer creates a new BlockCollectionIndexer implementation. +// +// Parameters: +// - metrics: Metrics collector for tracking collection indexing +// - lockManager: Lock manager for coordinating database access +// - db: Database for storage operations +// - collections: collections storage for storing and indexing collections +// +// No error returns are expected during normal operation. +func NewBlockCollectionIndexer( + metrics module.CollectionExecutedMetric, + lockManager lockctx.Manager, + db storage.DB, + collections storage.Collections, +) *blockCollectionIndexerImpl { + return &blockCollectionIndexerImpl{ + metrics: metrics, + lockManager: lockManager, + db: db, + collections: collections, + } +} + +// IndexCollectionsForBlock stores and indexes collections for a given block height. +// +// No error returns are expected during normal operation. +func (bci *blockCollectionIndexerImpl) IndexCollectionsForBlock( + _ uint64, + cols []*flow.Collection, +) error { + // Store and index collections + return storage.WithLock(bci.lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return bci.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + for _, collection := range cols { + // Store the collection, including constituent transactions, and index transactionID -> collectionID + light, err := bci.collections.BatchStoreAndIndexByTransaction(lctx, collection, rw) + if err != nil { + return fmt.Errorf("failed to store collection: %w", err) + } + + bci.metrics.CollectionFinalized(light) + bci.metrics.CollectionExecuted(light) + } + return nil + }) + }) +} + +// GetMissingCollections retrieves the block and returns collection guarantees that whose collections +// are missing in storage. +// Only garantees whose collections that are not already in storage are returned. +// No error returns are expected during normal operation. +func (bci *blockCollectionIndexerImpl) GetMissingCollections(block *flow.Block) ([]*flow.CollectionGuarantee, error) { + var missingGuarantees []*flow.CollectionGuarantee + for _, guarantee := range block.Payload.Guarantees { + // Check if collection already exists in storage + exists, err := bci.collections.ExistByID(guarantee.CollectionID) + if err != nil { + // Unexpected error + return nil, fmt.Errorf("failed to check if collection %v exists: %w", guarantee.CollectionID, err) + } + + if !exists { + // Collection is missing + missingGuarantees = append(missingGuarantees, guarantee) + } + // If collection exists, skip it + } + + return missingGuarantees, nil +} diff --git a/engine/access/collection_sync/indexer/indexer_benchmark_test.go b/engine/access/collection_sync/indexer/indexer_benchmark_test.go new file mode 100644 index 00000000000..463e443a1c9 --- /dev/null +++ b/engine/access/collection_sync/indexer/indexer_benchmark_test.go @@ -0,0 +1,144 @@ +package indexer + +import ( + "flag" + "sync" + "testing" + "time" + + "github.com/cockroachdb/pebble/v2" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/pebbleimpl" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" +) + +var ( + collectionsPerHeight = flag.Int("collections-per-height", 5, "Number of collections per height") + transactionsPerCollection = flag.Int("transactions-per-collection", 2, "Number of transactions per collection") +) + +// BenchmarkIndexCollectionsForBlock_Sequential benchmarks IndexCollectionsForBlock +// when called sequentially for multiple heights. +// It measures how many heights per second can be indexed. +// +// Benchmark results for 30s (Apple M1 Pro, Pebble storage, Nov 20, 2025): +// - Default config (5 collections/height, 2 transactions/collection): ~197.9 heights/sec +// - Higher load (10 collections/height, 5 transactions/collection): ~168.0 heights/sec +func BenchmarkIndexCollectionsForBlock_Sequential(b *testing.B) { + unittest.RunWithPebbleDB(b, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + indexer := NewBlockCollectionIndexer( + metrics, + lockManager, + db, + collections, + ) + + // Pre-generate collections for all heights + collectionsByHeight := make([][]*flow.Collection, b.N) + for height := 0; height < b.N; height++ { + cols := make([]*flow.Collection, *collectionsPerHeight) + for i := 0; i < *collectionsPerHeight; i++ { + col := unittest.CollectionFixture(*transactionsPerCollection) + cols[i] = &col + } + collectionsByHeight[height] = cols + } + + b.ResetTimer() + b.ReportAllocs() + + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + + // Report heights per second + b.ReportMetric(float64(b.N)/b.Elapsed().Seconds(), "heights/sec") + }) +} + +// BenchmarkIndexCollectionsForBlock_Concurrent benchmarks IndexCollectionsForBlock +// with 2 concurrent threads, each independently calling IndexCollectionsForBlock +// sequentially for the same height range. Both threads use the same data for each height. +// It measures how many heights per second each thread can process. +// +// Benchmark results for 30s (Apple M1 Pro, Pebble storage, Nov 20, 2025): +// - Default config (5 collections/height, 2 transactions/collection): +// Thread 1: ~185.9 heights/sec, Thread 2: ~185.9 heights/sec +func BenchmarkIndexCollectionsForBlock_Concurrent(b *testing.B) { + unittest.RunWithPebbleDB(b, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + // Create separate indexer instances for each thread + indexer := NewBlockCollectionIndexer( + metrics, + lockManager, + db, + collections, + ) + + // Pre-generate collections for all heights (shared between threads) + collectionsByHeight := make([][]*flow.Collection, b.N) + for height := 0; height < b.N; height++ { + cols := make([]*flow.Collection, *collectionsPerHeight) + for i := 0; i < *collectionsPerHeight; i++ { + col := unittest.CollectionFixture(*transactionsPerCollection) + cols[i] = &col + } + collectionsByHeight[height] = cols + } + + b.ResetTimer() + b.ReportAllocs() + + var wg sync.WaitGroup + var thread1Elapsed, thread2Elapsed time.Duration + + // Thread 1: process heights sequentially + wg.Add(1) + go func() { + defer wg.Done() + start := time.Now() + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + thread1Elapsed = time.Since(start) + }() + + // Thread 2: process the same heights sequentially + wg.Add(1) + go func() { + defer wg.Done() + start := time.Now() + for height := 0; height < b.N; height++ { + err := indexer.IndexCollectionsForBlock(uint64(height), collectionsByHeight[height]) + require.NoError(b, err) + } + thread2Elapsed = time.Since(start) + }() + + wg.Wait() + + // Report heights per second for each thread + thread1HeightsPerSec := float64(b.N) / thread1Elapsed.Seconds() + thread2HeightsPerSec := float64(b.N) / thread2Elapsed.Seconds() + b.ReportMetric(thread1HeightsPerSec, "heights/sec-thread1") + b.ReportMetric(thread2HeightsPerSec, "heights/sec-thread2") + }) +} diff --git a/engine/access/collection_sync/indexer/indexer_test.go b/engine/access/collection_sync/indexer/indexer_test.go new file mode 100644 index 00000000000..2fd8b972b70 --- /dev/null +++ b/engine/access/collection_sync/indexer/indexer_test.go @@ -0,0 +1,143 @@ +package indexer + +import ( + "testing" + + "github.com/cockroachdb/pebble/v2" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + modulemock "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/pebbleimpl" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/fixtures" +) + +// TestIndexCollectionsForBlock verifies the complete indexing workflow: +// +// Verification Process: +// 1. Initial State Check: +// - Before indexing, GetMissingCollections returns all collections as missing +// - This confirms the indexer correctly identifies unindexed collections +// +// 2. First Indexing: +// - IndexCollectionsForBlock successfully stores and indexes all collections +// - Collections are persisted to storage and can be retrieved by ID +// - Transactions are stored and can be queried by transactions.ByID +// - Transaction-to-collection index exists and can be queried by collections.LightByTransactionID +// - Metrics are called for each collection (CollectionFinalized, CollectionExecuted) +// +// 3. Post-Indexing Verification: +// - GetMissingCollections returns empty (no missing collections) +// - This confirms indexing was successful and collections are now available +// +// 4. Idempotency Check: +// - Indexing the same block again succeeds without error +// - GetMissingCollections still returns empty after second indexing +// - This confirms the operation is idempotent and safe to retry +func TestIndexCollectionsForBlock(t *testing.T) { + unittest.RunWithPebbleDB(t, func(pdb *pebble.DB) { + db := pebbleimpl.ToDB(pdb) + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + collectionMetrics := modulemock.NewCollectionExecutedMetric(t) + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + indexer := NewBlockCollectionIndexer( + collectionMetrics, + lockManager, + db, + collections, + ) + + g := fixtures.NewGeneratorSuite() + height := uint64(100) + collectionList := g.Collections().List(3) + + // Create guarantees and block before indexing + guarantees := make([]*flow.CollectionGuarantee, len(collectionList)) + for i, collection := range collectionList { + guarantee := g.Guarantees().Fixture(fixtures.Guarantee.WithCollectionID(collection.ID())) + guarantees[i] = guarantee + } + + payload := g.Payloads().Fixture( + fixtures.Payload.WithGuarantees(guarantees...), + ) + block := g.Blocks().Fixture( + fixtures.Block.WithPayload(payload), + ) + + // Step 1: Initial State Check - Before indexing, GetMissingCollections should return all collections as missing + missing, err := indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 3) + require.Equal(t, guarantees[0].CollectionID, missing[0].CollectionID) + require.Equal(t, guarantees[1].CollectionID, missing[1].CollectionID) + require.Equal(t, guarantees[2].CollectionID, missing[2].CollectionID) + + // Step 2: First Indexing - Expect metrics to be called for each collection + for _, collection := range collectionList { + light := collection.Light() + collectionMetrics.On("CollectionFinalized", light).Once() + collectionMetrics.On("CollectionExecuted", light).Once() + } + + err = indexer.IndexCollectionsForBlock(height, collectionList) + require.NoError(t, err) + + // Step 2 (continued): Verify all collections are stored and can be retrieved + for _, collection := range collectionList { + stored, err := collections.ByID(collection.ID()) + require.NoError(t, err) + require.Equal(t, collection.ID(), stored.ID()) + } + + // Step 2 (continued): Verify all transactions are stored and can be queried by ID + for _, collection := range collectionList { + for _, tx := range collection.Transactions { + storedTx, err := transactions.ByID(tx.ID()) + require.NoError(t, err) + require.Equal(t, tx.ID(), storedTx.ID()) + } + } + + // Step 2 (continued): Verify transaction-to-collection index exists and can be queried + for _, collection := range collectionList { + for _, tx := range collection.Transactions { + lightCollection, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + require.Equal(t, collection.ID(), lightCollection.ID()) + } + } + + // Step 3: Post-Indexing Verification - After indexing, GetMissingCollections should return empty + missing, err = indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 0) + + collectionMetrics.AssertExpectations(t) + + // Step 4: Idempotency Check - Indexing again is idempotent and should not error + // Set up expectations for the second call (metrics are still called even if collections already exist) + for _, collection := range collectionList { + light := collection.Light() + collectionMetrics.On("CollectionFinalized", light).Once() + collectionMetrics.On("CollectionExecuted", light).Once() + } + + err = indexer.IndexCollectionsForBlock(height, collectionList) + require.NoError(t, err) + + collectionMetrics.AssertExpectations(t) + + // Step 4 (continued): After second indexing, GetMissingCollections should still return empty + missing, err = indexer.GetMissingCollections(block) + require.NoError(t, err) + require.Len(t, missing, 0) + }) +} diff --git a/engine/access/finalized_indexer/processor.go b/engine/access/finalized_indexer/processor.go new file mode 100644 index 00000000000..b7ec6e384b4 --- /dev/null +++ b/engine/access/finalized_indexer/processor.go @@ -0,0 +1,187 @@ +package finalized_indexer + +import ( + "fmt" + + "github.com/jordanschalm/lockctx" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/consensus/hotstuff" + "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// FinalizedBlockProcessor processes finalized blocks and builds a collection-to-finalized-block index. +// +// The processor iterates through each finalized block sequentially and indexes which finalized block +// contains each collection. This index is necessary because while a collection can belong to multiple +// unfinalized blocks, each collection belongs to exactly one finalized block. This uniqueness property +// enables efficient transaction result lookups. +// +// The collection-to-block index is used by the GetTransactionResult API to locate the block containing +// a transaction. The lookup process is: transaction -> collection (via collection sync indexer) -> +// finalized block (via this indexer). The transaction-to-collection index is built by the collection +// sync indexer, which indexes either sealed or finalized blocks. When indexing finalized blocks, the +// transaction-to-collection mapping is also unique. +// +// The processor uses a worker loop that processes blocks sequentially by height, triggered by the +// distributor when new blocks are finalized. This design enables high-throughput block finalization +// handling without blocking other system components. +type FinalizedBlockProcessor struct { + log zerolog.Logger + component.Component + + newBlockFinalized engine.Notifier + state protocol.State + blocks storage.Blocks + db storage.DB + lockManager storage.LockManager + processedProgress storage.ConsumerProgress + + collectionExecutedMetric module.CollectionExecutedMetric +} + +// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor, +// setting up worker loop infrastructure to handle finalized block processing. +// +// No errors are expected during normal operations. +func NewFinalizedBlockProcessor( + log zerolog.Logger, + state protocol.State, + lockManager storage.LockManager, + db storage.DB, + blocks storage.Blocks, + finalizedProcessedHeight storage.ConsumerProgressInitializer, + distributor hotstuff.Distributor, + collectionExecutedMetric module.CollectionExecutedMetric, +) (*FinalizedBlockProcessor, error) { + finalizedBlock, err := state.Final().Head() + if err != nil { + return nil, fmt.Errorf("could not get finalized block header: %w", err) + } + + // Initialize the progress tracker + processedProgress, err := finalizedProcessedHeight.Initialize(finalizedBlock.Height) + if err != nil { + return nil, fmt.Errorf("could not initialize processed height: %w", err) + } + + processor := &FinalizedBlockProcessor{ + log: log.With().Str("component", "finalized_block_processor").Logger(), + newBlockFinalized: engine.NewNotifier(), + state: state, + db: db, + lockManager: lockManager, + blocks: blocks, + processedProgress: processedProgress, + collectionExecutedMetric: collectionExecutedMetric, + } + + // Initialize the notifier so that even if no new blocks are finalized, + // the worker loop can still be triggered to process any existing blocks. + processor.newBlockFinalized.Notify() + + distributor.AddOnBlockFinalizedConsumer(func(_ *model.Block) { + processor.newBlockFinalized.Notify() + }) + + // Build component manager with worker loop + cm := component.NewComponentManagerBuilder(). + AddWorker(processor.workerLoop). + Build() + + processor.Component = cm + + return processor, nil +} + +// workerLoop processes finalized blocks sequentially using a for loop to iterate through heights. +func (p *FinalizedBlockProcessor) workerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + // using a single threaded loop to process each finalized block by height + // since indexing collections is blocking anyway, and reading the blocks + // is quick. + for { + select { + case <-ctx.Done(): + return + case <-p.newBlockFinalized.Channel(): + finalizedHeader, err := p.state.Final().Head() + if err != nil { + ctx.Throw(fmt.Errorf("failed to get finalized block header: %w", err)) + return + } + highestAvailableHeight := finalizedHeader.Height + + processedHeight, err := p.processedProgress.ProcessedIndex() + if err != nil { + ctx.Throw(fmt.Errorf("failed to get processed height: %w", err)) + return + } + lowestMissing := processedHeight + 1 + + for height := lowestMissing; height <= highestAvailableHeight; height++ { + block, err := p.blocks.ByHeight(height) + if err != nil { + ctx.Throw(fmt.Errorf("failed to get block by height %d: %w", height, err)) + return + } + + err = p.indexForFinalizedBlock(block) + if err != nil { + ctx.Throw(fmt.Errorf("failed to index finalized block at height %d: %w", height, err)) + return + } + + // Update processed height after successful indexing + err = p.processedProgress.SetProcessedIndex(height) + if err != nil { + ctx.Throw(fmt.Errorf("failed to update processed height to %d: %w", height, err)) + return + } + + // Log progress for each height with all relevant information + p.log.Debug(). + Uint64("indexed", height). + Uint64("lowest_missing", lowestMissing). + Uint64("highest_available", highestAvailableHeight). + Uint64("processed_count", height-lowestMissing+1). + Uint64("remaining_count", highestAvailableHeight-height). + Uint64("total_to_process", highestAvailableHeight-lowestMissing+1). + Msg("indexed finalized block progress") + } + } + } +} + +// indexForFinalizedBlock indexes the given finalized block’s collection guarantees +// +// No errors are expected during normal operations. +func (p *FinalizedBlockProcessor) indexForFinalizedBlock(block *flow.Block) error { + err := storage.WithLock(p.lockManager, storage.LockIndexBlockByPayloadGuarantees, + func(lctx lockctx.Context) error { + return p.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + // require storage.LockIndexBlockByPayloadGuarantees + err := p.blocks.BatchIndexBlockContainingCollectionGuarantees(lctx, rw, block.ID(), flow.GetIDs(block.Payload.Guarantees)) + if err != nil { + return fmt.Errorf("could not index block for collections: %w", err) + } + + return nil + }) + }) + if err != nil { + return fmt.Errorf("could not index execution results: %w", err) + } + + p.collectionExecutedMetric.BlockFinalized(block) + + return nil +} diff --git a/engine/access/ingestion2/engine.go b/engine/access/ingest_receipt/engine.go similarity index 83% rename from engine/access/ingestion2/engine.go rename to engine/access/ingest_receipt/engine.go index 3b81b776156..c1508457b04 100644 --- a/engine/access/ingestion2/engine.go +++ b/engine/access/ingest_receipt/engine.go @@ -1,4 +1,4 @@ -// Package ingestion2 implements a modular ingestion engine responsible for +// Package finalized_indexer implements a modular ingestion engine responsible for // orchestrating the processing of finalized blockchain data and receiving // execution receipts from the network. // @@ -6,7 +6,7 @@ // - Receiving and persisting execution receipts from the network. // - Subscribing to finalized block events. // - Synchronizing collections associated with finalized blocks. -package ingestion2 +package ingest_receipt import ( "context" @@ -14,9 +14,7 @@ import ( "github.com/rs/zerolog" - "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/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,9 +33,6 @@ type Engine struct { log zerolog.Logger - finalizedBlockProcessor *FinalizedBlockProcessor - collectionSyncer *collections.Syncer - messageHandler *engine.MessageHandler executionReceiptsQueue *engine.FifoMessageStore receipts storage.ExecutionReceipts @@ -49,8 +44,6 @@ var _ network.MessageProcessor = (*Engine)(nil) func New( log zerolog.Logger, net network.EngineRegistry, - finalizedBlockProcessor *FinalizedBlockProcessor, - collectionSyncer *collections.Syncer, receipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, ) (*Engine, error) { @@ -72,9 +65,7 @@ func New( ) e := &Engine{ - log: log.With().Str("engine", "ingestion2").Logger(), - finalizedBlockProcessor: finalizedBlockProcessor, - collectionSyncer: collectionSyncer, + log: log.With().Str("engine", "ingest_receipt").Logger(), messageHandler: messageHandler, executionReceiptsQueue: executionReceiptsQueue, receipts: receipts, @@ -84,9 +75,7 @@ func New( // register our workers which are basically consumers of different kinds of data. // engine notifies workers when new data is available so that they can start processing them. builder := component.NewComponentManagerBuilder(). - AddWorker(e.messageHandlerLoop). - AddWorker(e.finalizedBlockProcessor.StartWorkerLoop). - AddWorker(e.collectionSyncer.WorkerLoop) + AddWorker(e.messageHandlerLoop) e.ComponentManager = builder.Build() // engine gets execution receipts from channels.ReceiveReceipts channel @@ -175,9 +164,3 @@ func (e *Engine) persistExecutionReceipt(receipt *flow.ExecutionReceipt) error { e.collectionExecutedMetric.ExecutionReceiptReceived(receipt) return nil } - -// OnFinalizedBlock is called by the follower engine after a block has been finalized and the state has been updated. -// Receives block finalized events from the finalization registrar and forwards them to the consumer. -func (e *Engine) OnFinalizedBlock(_ *model.Block) { - e.finalizedBlockProcessor.Notify() -} diff --git a/engine/access/ingestion2/engine_test.go b/engine/access/ingestion2/engine_test.go deleted file mode 100644 index 169f8476012..00000000000 --- a/engine/access/ingestion2/engine_test.go +++ /dev/null @@ -1,484 +0,0 @@ -package ingestion2 - -import ( - "context" - "os" - "sync" - "testing" - "time" - - "github.com/jordanschalm/lockctx" - "github.com/rs/zerolog" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - hotmodel "github.com/onflow/flow-go/consensus/hotstuff/model" - "github.com/onflow/flow-go/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" - "github.com/onflow/flow-go/module/mempool/stdmap" - "github.com/onflow/flow-go/module/metrics" - modulemock "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/module/signature" - "github.com/onflow/flow-go/module/state_synchronization/indexer" - "github.com/onflow/flow-go/network/channels" - mocknetwork "github.com/onflow/flow-go/network/mock" - protocol "github.com/onflow/flow-go/state/protocol/mock" - "github.com/onflow/flow-go/storage" - storagemock "github.com/onflow/flow-go/storage/mock" - "github.com/onflow/flow-go/storage/operation/pebbleimpl" - "github.com/onflow/flow-go/storage/store" - "github.com/onflow/flow-go/utils/unittest" - "github.com/onflow/flow-go/utils/unittest/mocks" -) - -type Suite struct { - suite.Suite - - // protocol state - proto struct { - state *protocol.FollowerState - snapshot *protocol.Snapshot - params *protocol.Params - } - - me *modulemock.Local - net *mocknetwork.EngineRegistry - request *modulemock.Requester - obsIdentity *flow.Identity - provider *mocknetwork.Engine - blocks *storagemock.Blocks - headers *storagemock.Headers - collections *storagemock.Collections - transactions *storagemock.Transactions - receipts *storagemock.ExecutionReceipts - results *storagemock.ExecutionResults - seals *storagemock.Seals - - conduit *mocknetwork.Conduit - downloader *downloadermock.Downloader - sealedBlock *flow.Header - finalizedBlock *flow.Header - log zerolog.Logger - blockMap map[uint64]*flow.Block - rootBlock *flow.Block - - collectionExecutedMetric *indexer.CollectionExecutedMetricImpl - - ctx context.Context - cancel context.CancelFunc - - db storage.DB - dbDir string - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter - lockManager lockctx.Manager -} - -func TestIngestEngine(t *testing.T) { - suite.Run(t, new(Suite)) -} - -// TearDownTest stops the engine and cleans up the db -func (s *Suite) TearDownTest() { - s.cancel() - err := os.RemoveAll(s.dbDir) - s.Require().NoError(err) -} - -func (s *Suite) SetupTest() { - s.log = unittest.Logger() - s.ctx, s.cancel = context.WithCancel(context.Background()) - pdb, dbDir := unittest.TempPebbleDB(s.T()) - s.db = pebbleimpl.ToDB(pdb) - s.dbDir = dbDir - s.lockManager = storage.NewTestingLockManager() - - s.obsIdentity = unittest.IdentityFixture(unittest.WithRole(flow.RoleAccess)) - - s.blocks = storagemock.NewBlocks(s.T()) - // mock out protocol state - s.proto.state = new(protocol.FollowerState) - s.proto.snapshot = new(protocol.Snapshot) - s.proto.params = new(protocol.Params) - s.finalizedBlock = unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) - s.proto.state.On("Identity").Return(s.obsIdentity, nil) - s.proto.state.On("Params").Return(s.proto.params) - s.proto.snapshot.On("Head").Return( - func() *flow.Header { - return s.finalizedBlock - }, - nil, - ).Maybe() - - s.me = modulemock.NewLocal(s.T()) - s.me.On("NodeID").Return(s.obsIdentity.NodeID).Maybe() - s.net = mocknetwork.NewEngineRegistry(s.T()) - conduit := mocknetwork.NewConduit(s.T()) - s.net.On("Register", channels.ReceiveReceipts, mock.Anything). - Return(conduit, nil). - Once() - s.request = modulemock.NewRequester(s.T()) - s.provider = mocknetwork.NewEngine(s.T()) - s.blocks = storagemock.NewBlocks(s.T()) - s.headers = storagemock.NewHeaders(s.T()) - s.collections = new(storagemock.Collections) - s.receipts = new(storagemock.ExecutionReceipts) - s.transactions = new(storagemock.Transactions) - s.results = new(storagemock.ExecutionResults) - s.results.On("BatchIndex", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() - collectionsToMarkFinalized := stdmap.NewTimes(100) - collectionsToMarkExecuted := stdmap.NewTimes(100) - blocksToMarkExecuted := stdmap.NewTimes(100) - blockTransactions := stdmap.NewIdentifierMap(100) - - s.proto.state.On("Identity").Return(s.obsIdentity, nil) - s.proto.state.On("Params").Return(s.proto.params) - - blockCount := 5 - s.blockMap = make(map[uint64]*flow.Block, blockCount) - s.rootBlock = unittest.Block.Genesis(flow.Emulator) - parent := s.rootBlock.ToHeader() - - for i := 0; i < blockCount; i++ { - block := unittest.BlockWithParentFixture(parent) - // update for next iteration - parent = block.ToHeader() - s.blockMap[block.Height] = block - } - s.finalizedBlock = parent - - s.blocks.On("ByHeight", mock.AnythingOfType("uint64")).Return( - mocks.ConvertStorageOutput( - mocks.StorageMapGetter(s.blockMap), - func(block *flow.Block) *flow.Block { return block }, - ), - ).Maybe() - - s.proto.snapshot.On("Head").Return( - func() *flow.Header { - return s.finalizedBlock - }, - nil, - ).Maybe() - s.proto.state.On("Final").Return(s.proto.snapshot, nil) - - // Mock the finalized root block header with height 0. - header := unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) - s.proto.params.On("FinalizedRoot").Return(header, nil) - - var err error - s.collectionExecutedMetric, err = indexer.NewCollectionExecutedMetricImpl( - s.log, - metrics.NewNoopCollector(), - collectionsToMarkFinalized, - collectionsToMarkExecuted, - blocksToMarkExecuted, - s.collections, - s.blocks, - blockTransactions, - ) - require.NoError(s.T(), err) -} - -func (s *Suite) TestComponentShutdown() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - // start then shut down the engine - unittest.AssertClosesBefore(s.T(), eng.Ready(), 10*time.Millisecond) - s.cancel() - unittest.AssertClosesBefore(s.T(), eng.Done(), 10*time.Millisecond) - - err := eng.Process(channels.ReceiveReceipts, unittest.IdentifierFixture(), &flow.ExecutionReceipt{}) - s.Assert().ErrorIs(err, component.ErrComponentShutdown) -} - -// initEngineAndSyncer create new instance of ingestion engine and collection collectionSyncer. -// It waits until the ingestion engine starts. -func (s *Suite) initEngineAndSyncer(ctx irrecoverable.SignalerContext) (*Engine, *collections.Syncer) { - processedHeightInitializer := store.NewConsumerProgress(s.db, module.ConsumeProgressIngestionEngineBlockHeight) - - lastFullBlockHeight, err := store.NewConsumerProgress(s.db, module.ConsumeProgressLastFullBlockHeight).Initialize(s.finalizedBlock.Height) - require.NoError(s.T(), err) - - s.lastFullBlockHeight, err = counters.NewPersistentStrictMonotonicCounter(lastFullBlockHeight) - require.NoError(s.T(), err) - - indexer, err := collections.NewIndexer( - s.log, - s.db, - s.collectionExecutedMetric, - s.proto.state, - s.blocks, - s.collections, - s.lastFullBlockHeight, - s.lockManager, - ) - require.NoError(s.T(), err) - - syncer := collections.NewSyncer( - s.log, - s.request, - s.proto.state, - s.collections, - s.lastFullBlockHeight, - indexer, - nil, - ) - require.NoError(s.T(), err) - - blockProcessor, err := NewFinalizedBlockProcessor( - s.log, - s.proto.state, - s.lockManager, - s.db, - s.blocks, - s.results, - processedHeightInitializer, - syncer, - s.collectionExecutedMetric, - ) - require.NoError(s.T(), err) - - eng, err := New( - s.log, - s.net, - blockProcessor, - syncer, - s.receipts, - s.collectionExecutedMetric, - ) - - require.NoError(s.T(), err) - - eng.ComponentManager.Start(ctx) - <-eng.Ready() - - return eng, syncer -} - -// mockCollectionsForBlock mocks collections for block -func (s *Suite) mockCollectionsForBlock(block *flow.Block) { - // we should query the block once and index the guarantee payload once - for _, g := range block.Payload.Guarantees { - collection := unittest.CollectionFixture(1) - light := collection.Light() - s.collections.On("LightByID", g.CollectionID).Return(light, nil).Twice() - } -} - -// generateBlock prepares block with payload and specified guarantee.SignerIndices -func (s *Suite) generateBlock(clusterCommittee flow.IdentitySkeletonList, snap *protocol.Snapshot) *flow.Block { - block := unittest.BlockFixture( - unittest.Block.WithPayload(unittest.PayloadFixture( - unittest.WithGuarantees(unittest.CollectionGuaranteesFixture(4)...), - unittest.WithExecutionResults(unittest.ExecutionResultFixture()), - unittest.WithSeals(unittest.Seal.Fixture()), - )), - ) - - refBlockID := unittest.IdentifierFixture() - for _, guarantee := range block.Payload.Guarantees { - guarantee.ReferenceBlockID = refBlockID - // guarantee signers must be cluster committee members, so that access will fetch collection from - // the signers that are specified by guarantee.SignerIndices - indices, err := signature.EncodeSignersToIndices(clusterCommittee.NodeIDs(), clusterCommittee.NodeIDs()) - require.NoError(s.T(), err) - guarantee.SignerIndices = indices - } - - s.proto.state.On("AtBlockID", refBlockID).Return(snap) - - return block -} - -// TestOnFinalizedBlock checks that when a block is received, a request for each individual collection is made -func (s *Suite) TestOnFinalizedBlockSingle() { - cluster := protocol.NewCluster(s.T()) - epoch := protocol.NewCommittedEpoch(s.T()) - epochs := protocol.NewEpochQuery(s.T()) - snap := protocol.NewSnapshot(s.T()) - - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs.On("Current").Return(epoch, nil) - snap.On("Epochs").Return(epochs) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - cluster.On("Members").Return(clusterCommittee, nil) - - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - block := s.generateBlock(clusterCommittee, snap) - block.Height = s.finalizedBlock.Height + 1 - s.blockMap[block.Height] = block - s.mockCollectionsForBlock(block) - s.finalizedBlock = block.ToHeader() - - hotstuffBlock := hotmodel.Block{ - BlockID: block.ID(), - } - - // expect that the block storage is indexed with each of the collection guarantee - s.blocks.On("BatchIndexBlockContainingCollectionGuarantees", mock.Anything, mock.Anything, block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() - for _, seal := range block.Payload.Seals { - s.results.On("Index", seal.BlockID, seal.ResultID).Return(nil).Once() - } - - missingCollectionCount := 4 - wg := sync.WaitGroup{} - wg.Add(missingCollectionCount) - - for _, cg := range block.Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Run(func(args mock.Arguments) { - // Ensure the test does not complete its work faster than necessary - wg.Done() - }).Once() - } - s.request.On("Force").Return().Once() - - // process the block through the finalized callback - eng.OnFinalizedBlock(&hotstuffBlock) - - unittest.RequireReturnsBefore(s.T(), wg.Wait, 100*time.Millisecond, "expect to process new block before timeout") - - // assert that the block was retrieved and all collections were requested - s.headers.AssertExpectations(s.T()) - s.request.AssertNumberOfCalls(s.T(), "EntityByID", len(block.Payload.Guarantees)) - s.results.AssertNumberOfCalls(s.T(), "BatchIndex", len(block.Payload.Seals)) -} - -// TestOnFinalizedBlockSeveralBlocksAhead checks OnFinalizedBlock with a block several blocks newer than the last block processed -func (s *Suite) TestOnFinalizedBlockSeveralBlocksAhead() { - cluster := protocol.NewCluster(s.T()) - epoch := protocol.NewCommittedEpoch(s.T()) - epochs := protocol.NewEpochQuery(s.T()) - snap := protocol.NewSnapshot(s.T()) - - epoch.On("ClusterByChainID", mock.Anything).Return(cluster, nil) - epochs.On("Current").Return(epoch, nil) - snap.On("Epochs").Return(epochs) - - // prepare cluster committee members - clusterCommittee := unittest.IdentityListFixture(32 * 4).Filter(filter.HasRole[flow.Identity](flow.RoleCollection)).ToSkeleton() - cluster.On("Members").Return(clusterCommittee, nil) - - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - newBlocksCount := 3 - startHeight := s.finalizedBlock.Height + 1 - blocks := make([]*flow.Block, newBlocksCount) - - // generate the test blocks, cgs and collections - for i := 0; i < newBlocksCount; i++ { - block := s.generateBlock(clusterCommittee, snap) - block.Height = startHeight + uint64(i) - s.blockMap[block.Height] = block - blocks[i] = block - s.mockCollectionsForBlock(block) - s.finalizedBlock = block.ToHeader() - } - - // latest of all the new blocks which are newer than the last block processed - latestBlock := blocks[2] - - // block several blocks newer than the last block processed - hotstuffBlock := hotmodel.Block{ - BlockID: latestBlock.ID(), - } - - missingCollectionCountPerBlock := 4 - wg := sync.WaitGroup{} - wg.Add(missingCollectionCountPerBlock * newBlocksCount) - - // expected all new blocks after last block processed - for _, block := range blocks { - s.blocks.On("BatchIndexBlockContainingCollectionGuarantees", mock.Anything, mock.Anything, block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() - - for _, cg := range block.Payload.Guarantees { - s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Run(func(args mock.Arguments) { - // Ensure the test does not complete its work faster than necessary, so we can check all expected results - wg.Done() - }).Once() - } - s.request.On("Force").Return().Once() - - for _, seal := range block.Payload.Seals { - s.results.On("Index", seal.BlockID, seal.ResultID).Return(nil).Once() - } - } - - eng.OnFinalizedBlock(&hotstuffBlock) - - unittest.RequireReturnsBefore(s.T(), wg.Wait, 100*time.Millisecond, "expect to process all blocks before timeout") - - expectedEntityByIDCalls := 0 - expectedIndexCalls := 0 - for _, block := range blocks { - expectedEntityByIDCalls += len(block.Payload.Guarantees) - expectedIndexCalls += len(block.Payload.Seals) - } - - s.headers.AssertExpectations(s.T()) - s.blocks.AssertNumberOfCalls(s.T(), "BatchIndexBlockContainingCollectionGuarantees", newBlocksCount) - s.request.AssertNumberOfCalls(s.T(), "EntityByID", expectedEntityByIDCalls) - s.results.AssertNumberOfCalls(s.T(), "BatchIndex", expectedIndexCalls) -} - -// TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed -func (s *Suite) TestExecutionReceiptsAreIndexed() { - irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) - eng, _ := s.initEngineAndSyncer(irrecoverableCtx) - - collection := unittest.CollectionFixture(5) - light := collection.Light() - - // we should store the collection and index its transactions - s.collections.On("StoreAndIndexByTransaction", &collection).Return(light, nil).Once() - block := unittest.BlockFixture( - unittest.Block.WithHeight(0), - unittest.Block.WithPayload( - unittest.PayloadFixture(unittest.WithGuarantees([]*flow.CollectionGuarantee{}...)), - ), - ) - s.blocks.On("ByID", mock.Anything).Return(block, nil) - - // for each transaction in the collection, we should store it - needed := make(map[flow.Identifier]struct{}) - for _, txID := range light.Transactions { - needed[txID] = struct{}{} - } - s.transactions.On("Store", mock.Anything).Return(nil).Run( - func(args mock.Arguments) { - tx := args.Get(0).(*flow.TransactionBody) - _, pending := needed[tx.ID()] - s.Assert().True(pending, "tx not pending (%x)", tx.ID()) - }, - ) - er1 := unittest.ExecutionReceiptFixture() - er2 := unittest.ExecutionReceiptFixture() - - s.receipts.On("Store", mock.Anything).Return(nil) - s.blocks.On("ByID", er1.ExecutionResult.BlockID).Return(nil, storage.ErrNotFound) - - s.receipts.On("Store", mock.Anything).Return(nil) - s.blocks.On("ByID", er2.ExecutionResult.BlockID).Return(nil, storage.ErrNotFound) - - err := eng.persistExecutionReceipt(er1) - require.NoError(s.T(), err) - - err = eng.persistExecutionReceipt(er2) - require.NoError(s.T(), err) - - s.receipts.AssertExpectations(s.T()) - s.results.AssertExpectations(s.T()) - s.receipts.AssertExpectations(s.T()) -} diff --git a/engine/access/ingestion2/finalized_block_processor.go b/engine/access/ingestion2/finalized_block_processor.go deleted file mode 100644 index 6ae7e3da9a1..00000000000 --- a/engine/access/ingestion2/finalized_block_processor.go +++ /dev/null @@ -1,188 +0,0 @@ -package ingestion2 - -import ( - "fmt" - - "github.com/jordanschalm/lockctx" - "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" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/jobqueue" - "github.com/onflow/flow-go/module/util" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -const ( - // finalizedBlockProcessorWorkerCount defines the number of workers that - // concurrently process finalized blocks in the job queue. - // MUST be 1 to ensure sequential processing - finalizedBlockProcessorWorkerCount = 1 - - // searchAhead is a number of blocks that should be processed ahead by jobqueue - // MUST be 1 to ensure sequential processing - searchAhead = 1 -) - -// FinalizedBlockProcessor handles processing of finalized blocks, -// including indexing and syncing of related collections and execution results. -// -// FinalizedBlockProcessor is designed to handle the ingestion of finalized Flow blocks -// in a scalable and decoupled manner. It uses a jobqueue.ComponentConsumer to consume -// and process finalized block jobs asynchronously. This design enables the processor -// to handle high-throughput block finalization events without blocking other parts -// of the system. -// -// The processor relies on a notifier (engine.Notifier) to signal when a new finalized -// block is available, which triggers the job consumer to process it. The actual -// processing involves indexing block-to-collection and block-to-execution-result -// mappings, as well as requesting the associated collections. -type FinalizedBlockProcessor struct { - log zerolog.Logger - - consumer *jobqueue.ComponentConsumer - consumerNotifier engine.Notifier - lockManager storage.LockManager - db storage.DB - - blocks storage.Blocks - executionResults storage.ExecutionResults - - collectionSyncer *collections.Syncer - collectionExecutedMetric module.CollectionExecutedMetric -} - -// NewFinalizedBlockProcessor creates and initializes a new FinalizedBlockProcessor, -// setting up job consumer infrastructure to handle finalized block processing. -// -// No errors are expected during normal operations. -func NewFinalizedBlockProcessor( - log zerolog.Logger, - state protocol.State, - lockManager storage.LockManager, - db storage.DB, - blocks storage.Blocks, - executionResults storage.ExecutionResults, - finalizedProcessedHeight storage.ConsumerProgressInitializer, - syncer *collections.Syncer, - collectionExecutedMetric module.CollectionExecutedMetric, -) (*FinalizedBlockProcessor, error) { - reader := jobqueue.NewFinalizedBlockReader(state, blocks) - finalizedBlock, err := state.Final().Head() - if err != nil { - return nil, fmt.Errorf("could not get finalized block header: %w", err) - } - - consumerNotifier := engine.NewNotifier() - processor := &FinalizedBlockProcessor{ - log: log, - db: db, - lockManager: lockManager, - blocks: blocks, - executionResults: executionResults, - consumerNotifier: consumerNotifier, - collectionSyncer: syncer, - collectionExecutedMetric: collectionExecutedMetric, - } - - processor.consumer, err = jobqueue.NewComponentConsumer( - log.With().Str("module", "ingestion_block_consumer").Logger(), - consumerNotifier.Channel(), - finalizedProcessedHeight, - reader, - finalizedBlock.Height, - processor.processFinalizedBlockJobCallback, - finalizedBlockProcessorWorkerCount, - searchAhead, - ) - if err != nil { - return nil, fmt.Errorf("error creating finalized block jobqueue: %w", err) - } - - return processor, nil -} - -// Notify notifies the processor that a new finalized block is available for processing. -func (p *FinalizedBlockProcessor) Notify() { - p.consumerNotifier.Notify() -} - -// StartWorkerLoop begins processing of finalized blocks and signals readiness when initialization is complete. -func (p *FinalizedBlockProcessor) StartWorkerLoop(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - p.consumer.Start(ctx) - - err := util.WaitClosed(ctx, p.consumer.Ready()) - if err == nil { - ready() - } - - <-p.consumer.Done() -} - -// processFinalizedBlockJobCallback is a jobqueue callback that processes a finalized block job. -func (p *FinalizedBlockProcessor) processFinalizedBlockJobCallback( - ctx irrecoverable.SignalerContext, - job module.Job, - done func(), -) { - block, err := jobqueue.JobToBlock(job) - if err != nil { - ctx.Throw(fmt.Errorf("failed to convert job to block: %w", err)) - return - } - - err = p.indexFinalizedBlock(block) - if err != nil { - p.log.Error().Err(err). - Str("job_id", string(job.ID())). - Msg("unexpected error during finalized block processing job") - ctx.Throw(fmt.Errorf("failed to index finalized block: %w", err)) - return - } - - done() -} - -// indexFinalizedBlock indexes the given finalized block’s collection guarantees and execution results, -// and requests related collections from the syncer. -// -// No errors are expected during normal operations. -func (p *FinalizedBlockProcessor) indexFinalizedBlock(block *flow.Block) error { - err := storage.WithLocks(p.lockManager, storage.LockGroupAccessFinalizingBlock, - func(lctx lockctx.Context) error { - return p.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { - // require storage.LockIndexBlockByPayloadGuarantees - err := p.blocks.BatchIndexBlockContainingCollectionGuarantees(lctx, rw, block.ID(), flow.GetIDs(block.Payload.Guarantees)) - if err != nil { - return fmt.Errorf("could not index block for collections: %w", err) - } - - // loop through seals and index ID -> result ID - for _, seal := range block.Payload.Seals { - // require storage.LockIndexExecutionResult - err := p.executionResults.BatchIndex(lctx, rw, seal.BlockID, seal.ResultID) - if err != nil { - return fmt.Errorf("could not index block for execution result: %w", err) - } - } - return nil - }) - }) - if err != nil { - return fmt.Errorf("could not index execution results: %w", err) - } - - err = p.collectionSyncer.RequestCollectionsForBlock(block.Height, block.Payload.Guarantees) - if err != nil { - return fmt.Errorf("could not request collections for block: %w", err) - } - - p.collectionExecutedMetric.BlockFinalized(block) - - return nil -} diff --git a/engine/access/integration_unsecure_grpc_server_test.go b/engine/access/integration_unsecure_grpc_server_test.go index edd0c9f2f9b..1ed816ded3d 100644 --- a/engine/access/integration_unsecure_grpc_server_test.go +++ b/engine/access/integration_unsecure_grpc_server_test.go @@ -41,6 +41,7 @@ import ( "github.com/onflow/flow-go/module/mempool/herocache" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/network" protocol "github.com/onflow/flow-go/state/protocol/mock" "github.com/onflow/flow-go/storage" @@ -49,6 +50,17 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +// mockExecutionDataIndexedHeight is a simple implementation of ExecutionDataIndexedHeight for testing +type mockExecutionDataIndexedHeight struct { + height uint64 +} + +func (m *mockExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + return m.height +} + +var _ state_synchronization.ExecutionDataIndexedHeight = (*mockExecutionDataIndexedHeight)(nil) + // SameGRPCPortTestSuite verifies both AccessAPI and ExecutionDataAPI client continue to work when configured // on the same port type SameGRPCPortTestSuite struct { @@ -264,13 +276,14 @@ func (suite *SameGRPCPortTestSuite) SetupTest() { eventIndexer := index.NewEventsIndex(index.NewReporter(), suite.events) + highestHeight := &mockExecutionDataIndexedHeight{height: rootBlock.Height} suite.executionDataTracker = tracker.NewExecutionDataTracker( suite.log, suite.state, rootBlock.Height, suite.headers, nil, - rootBlock.Height, + highestHeight, eventIndexer, false, ) diff --git a/engine/access/rpc/backend/backend.go b/engine/access/rpc/backend/backend.go index a5c6d3ab649..8019549a415 100644 --- a/engine/access/rpc/backend/backend.go +++ b/engine/access/rpc/backend/backend.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/access" "github.com/onflow/flow-go/access/validator" "github.com/onflow/flow-go/cmd/build" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" "github.com/onflow/flow-go/engine/access/rpc/backend/accounts" "github.com/onflow/flow-go/engine/access/rpc/backend/common" @@ -34,7 +35,6 @@ import ( "github.com/onflow/flow-go/model/access/systemcollection" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/execution" "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/state/protocol" @@ -114,7 +114,8 @@ type Params struct { EventsIndex *index.EventsIndex TxResultQueryMode query_mode.IndexQueryMode TxResultsIndex *index.TransactionResultsIndex - LastFullBlockHeight *counters.PersistentStrictMonotonicCounter + CollectionIndexedHeight storage.ConsumerProgress + LastFullBlockHeight collection_sync.ProgressReader IndexReporter state_synchronization.IndexReporter VersionControl *version.VersionControl ExecNodeIdentitiesProvider *rpc.ExecutionNodeIdentitiesProvider diff --git a/engine/access/rpc/backend/backend_test.go b/engine/access/rpc/backend/backend_test.go index 170f20ce7db..422d89c3f80 100644 --- a/engine/access/rpc/backend/backend_test.go +++ b/engine/access/rpc/backend/backend_test.go @@ -22,6 +22,7 @@ import ( "google.golang.org/grpc/status" "github.com/onflow/flow-go/cmd/build" + "github.com/onflow/flow-go/engine/access/collection_sync" accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/common" "github.com/onflow/flow-go/engine/access/rpc/backend/events" @@ -62,6 +63,17 @@ var eventEncodingVersions = []entitiesproto.EventEncodingVersion{ entitiesproto.EventEncodingVersion_CCF_V0, } +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + type Suite struct { suite.Suite @@ -2096,7 +2108,7 @@ func (suite *Suite) defaultBackendParams() Params { TxResultQueryMode: query_mode.IndexQueryModeExecutionNodesOnly, EventQueryMode: query_mode.IndexQueryModeExecutionNodesOnly, ScriptExecutionMode: query_mode.IndexQueryModeExecutionNodesOnly, - LastFullBlockHeight: suite.lastFullBlockHeight, + LastFullBlockHeight: &progressReaderAdapter{counter: suite.lastFullBlockHeight}, VersionControl: suite.versionControl, ExecNodeIdentitiesProvider: commonrpc.NewExecutionNodeIdentitiesProvider( suite.log, diff --git a/engine/access/rpc/backend/script_executor_test.go b/engine/access/rpc/backend/script_executor_test.go index 9d127da6810..20e17c959a7 100644 --- a/engine/access/rpc/backend/script_executor_test.go +++ b/engine/access/rpc/backend/script_executor_test.go @@ -142,7 +142,6 @@ func (s *ScriptExecutorSuite) SetupTest() { nil, s.chain.ChainID(), derivedChainData, - nil, metrics.NewNoopCollector(), lockManager, ) diff --git a/engine/access/rpc/backend/transactions/status/deriver.go b/engine/access/rpc/backend/transactions/status/deriver.go index d32b9d26357..af43d882d28 100644 --- a/engine/access/rpc/backend/transactions/status/deriver.go +++ b/engine/access/rpc/backend/transactions/status/deriver.go @@ -1,18 +1,18 @@ package status import ( + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/counters" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/state/protocol" ) type TxStatusDeriver struct { state protocol.State - lastFullBlockHeight *counters.PersistentStrictMonotonicCounter + lastFullBlockHeight collection_sync.ProgressReader } -func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight *counters.PersistentStrictMonotonicCounter) *TxStatusDeriver { +func NewTxStatusDeriver(state protocol.State, lastFullBlockHeight collection_sync.ProgressReader) *TxStatusDeriver { return &TxStatusDeriver{ state: state, lastFullBlockHeight: lastFullBlockHeight, @@ -48,7 +48,7 @@ func (t *TxStatusDeriver) DeriveUnknownTransactionStatus(refBlockID flow.Identif // the last full height is the height where we have received all // collections for all blocks with a lower height - fullHeight := t.lastFullBlockHeight.Value() + fullHeight := t.lastFullBlockHeight.ProcessedHeight() // if we have received collections for all blocks up to the expiry block, the transaction is expired if isExpired(refHeight, fullHeight) { diff --git a/engine/access/rpc/backend/transactions/stream/stream_backend_test.go b/engine/access/rpc/backend/transactions/stream/stream_backend_test.go index 0dc3a05b1ff..aa7f2eadcc0 100644 --- a/engine/access/rpc/backend/transactions/stream/stream_backend_test.go +++ b/engine/access/rpc/backend/transactions/stream/stream_backend_test.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/access/validator" validatormock "github.com/onflow/flow-go/access/validator/mock" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" access "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" @@ -54,6 +55,17 @@ import ( "github.com/onflow/flow-go/utils/unittest/mocks" ) +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + // TransactionStreamSuite represents a suite for testing transaction status-related functionality in the Flow blockchain. type TransactionStreamSuite struct { suite.Suite @@ -212,7 +224,7 @@ func (s *TransactionStreamSuite) initializeBackend() { txStatusDeriver := txstatus.NewTxStatusDeriver( s.state, - s.lastFullBlockHeight, + &progressReaderAdapter{counter: s.lastFullBlockHeight}, ) nodeCommunicator := node_communicator.NewNodeCommunicator(false) diff --git a/engine/access/rpc/backend/transactions/transactions_functional_test.go b/engine/access/rpc/backend/transactions/transactions_functional_test.go index aa3c1061778..aa99e81de1e 100644 --- a/engine/access/rpc/backend/transactions/transactions_functional_test.go +++ b/engine/access/rpc/backend/transactions/transactions_functional_test.go @@ -252,7 +252,7 @@ func (s *TransactionsFunctionalSuite) SetupTest() { s.Require().NoError(err) // Instantiate intermediate components - s.txStatusDeriver = txstatus.NewTxStatusDeriver(s.state, s.lastFullBlockHeight) + s.txStatusDeriver = txstatus.NewTxStatusDeriver(s.state, &progressReaderAdapter{counter: s.lastFullBlockHeight}) s.mockState = protocolmock.NewState(s.T()) s.nodeProvider = commonrpc.NewExecutionNodeIdentitiesProvider(s.log, s.mockState, s.receipts, nil, nil) diff --git a/engine/access/rpc/backend/transactions/transactions_test.go b/engine/access/rpc/backend/transactions/transactions_test.go index 0d3edc7d3ed..abafb00e84f 100644 --- a/engine/access/rpc/backend/transactions/transactions_test.go +++ b/engine/access/rpc/backend/transactions/transactions_test.go @@ -17,6 +17,7 @@ import ( "github.com/onflow/flow/protobuf/go/flow/entities" "github.com/onflow/flow-go/access/validator" + "github.com/onflow/flow-go/engine/access/collection_sync" "github.com/onflow/flow-go/engine/access/index" accessmock "github.com/onflow/flow-go/engine/access/mock" "github.com/onflow/flow-go/engine/access/rpc/backend/node_communicator" @@ -46,6 +47,17 @@ func TestTransactionsBackend(t *testing.T) { suite.Run(t, new(Suite)) } +// progressReaderAdapter adapts a PersistentStrictMonotonicCounter to implement collection_sync.ProgressReader +type progressReaderAdapter struct { + counter *counters.PersistentStrictMonotonicCounter +} + +var _ collection_sync.ProgressReader = (*progressReaderAdapter)(nil) + +func (p *progressReaderAdapter) ProcessedHeight() uint64 { + return p.counter.Value() +} + type Suite struct { suite.Suite @@ -189,7 +201,7 @@ func (suite *Suite) defaultTransactionsParams() Params { txStatusDeriver := txstatus.NewTxStatusDeriver( suite.state, - suite.lastFullBlockHeight, + &progressReaderAdapter{counter: suite.lastFullBlockHeight}, ) validatorBlocks := validator.NewProtocolStateBlocks(suite.state, suite.indexReporter) diff --git a/engine/access/state_stream/backend/backend_executiondata_test.go b/engine/access/state_stream/backend/backend_executiondata_test.go index f7587affa6d..e026615ff0c 100644 --- a/engine/access/state_stream/backend/backend_executiondata_test.go +++ b/engine/access/state_stream/backend/backend_executiondata_test.go @@ -29,6 +29,7 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" "github.com/onflow/flow-go/module/mempool/herocache" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/state_synchronization" protocolmock "github.com/onflow/flow-go/state/protocol/mock" "github.com/onflow/flow-go/storage" storagemock "github.com/onflow/flow-go/storage/mock" @@ -87,6 +88,17 @@ type executionDataTestType struct { startHeight uint64 } +// mockExecutionDataIndexedHeight is a simple implementation of ExecutionDataIndexedHeight for testing +type mockExecutionDataIndexedHeight struct { + height uint64 +} + +func (m *mockExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + return m.height +} + +var _ state_synchronization.ExecutionDataIndexedHeight = (*mockExecutionDataIndexedHeight)(nil) + func TestBackendExecutionDataSuite(t *testing.T) { suite.Run(t, new(BackendExecutionDataSuite)) } @@ -265,13 +277,14 @@ func (s *BackendExecutionDataSuite) SetupBackend(useEventsIndex bool) { require.NoError(s.T(), err) // create real execution data tracker to use GetStartHeight from it, instead of mocking + highestHeight := &mockExecutionDataIndexedHeight{height: s.rootBlock.Height} s.executionDataTrackerReal = tracker.NewExecutionDataTracker( s.logger, s.state, s.rootBlock.Height, s.headers, s.broadcaster, - s.rootBlock.Height, + highestHeight, s.eventsIndex, useEventsIndex, ) diff --git a/engine/access/subscription/tracker/execution_data_tracker.go b/engine/access/subscription/tracker/execution_data_tracker.go index f797e775652..ae66adba194 100644 --- a/engine/access/subscription/tracker/execution_data_tracker.go +++ b/engine/access/subscription/tracker/execution_data_tracker.go @@ -12,13 +12,10 @@ import ( "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/counters" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" "github.com/onflow/flow-go/module/state_synchronization/indexer" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/utils/logging" ) const ( @@ -59,7 +56,7 @@ type ExecutionDataTracker interface { GetHighestHeight() uint64 // OnExecutionData is used to notify the tracker when a new execution data is received. - OnExecutionData(*execution_data.BlockExecutionDataEntity) + OnExecutionData() } var _ ExecutionDataTracker = (*ExecutionDataTrackerImpl)(nil) @@ -68,13 +65,12 @@ var _ ExecutionDataTracker = (*ExecutionDataTrackerImpl)(nil) type ExecutionDataTrackerImpl struct { BaseTracker log zerolog.Logger - headers storage.Headers broadcaster *engine.Broadcaster indexReporter state_synchronization.IndexReporter useIndex bool // highestHeight contains the highest consecutive block height that we have consecutive execution data for - highestHeight counters.StrictMonotonicCounter + highestHeight state_synchronization.ExecutionDataIndexedHeight } // NewExecutionDataTracker creates a new ExecutionDataTrackerImpl instance. @@ -97,16 +93,15 @@ func NewExecutionDataTracker( rootHeight uint64, headers storage.Headers, broadcaster *engine.Broadcaster, - highestAvailableFinalizedHeight uint64, + highestHeight state_synchronization.ExecutionDataIndexedHeight, indexReporter state_synchronization.IndexReporter, useIndex bool, ) *ExecutionDataTrackerImpl { return &ExecutionDataTrackerImpl{ BaseTracker: NewBaseTrackerImpl(rootHeight, state, headers), log: log, - headers: headers, broadcaster: broadcaster, - highestHeight: counters.NewMonotonicCounter(highestAvailableFinalizedHeight), + highestHeight: highestHeight, indexReporter: indexReporter, useIndex: useIndex, } @@ -221,25 +216,11 @@ func (e *ExecutionDataTrackerImpl) GetStartHeightFromLatest(ctx context.Context) // GetHighestHeight returns the highest height that we have consecutive execution data for. func (e *ExecutionDataTrackerImpl) GetHighestHeight() uint64 { - return e.highestHeight.Value() + return e.highestHeight.HighestConsecutiveHeight() } // OnExecutionData is used to notify the tracker when a new execution data is received. -func (e *ExecutionDataTrackerImpl) OnExecutionData(executionData *execution_data.BlockExecutionDataEntity) { - log := e.log.With().Hex("block_id", logging.ID(executionData.BlockID)).Logger() - - log.Trace().Msg("received execution data") - - header, err := e.headers.ByBlockID(executionData.BlockID) - if err != nil { - // if the execution data is available, the block must be locally finalized - log.Fatal().Err(err).Msg("failed to notify of new execution data") - return - } - - // sets the highest height for which execution data is available. - _ = e.highestHeight.Set(header.Height) - +func (e *ExecutionDataTrackerImpl) OnExecutionData() { e.broadcaster.Publish() } diff --git a/engine/access/subscription/tracker/mock/execution_data_tracker.go b/engine/access/subscription/tracker/mock/execution_data_tracker.go index ccfad6bc8b4..3501457925f 100644 --- a/engine/access/subscription/tracker/mock/execution_data_tracker.go +++ b/engine/access/subscription/tracker/mock/execution_data_tracker.go @@ -6,8 +6,6 @@ import ( context "context" flow "github.com/onflow/flow-go/model/flow" - execution_data "github.com/onflow/flow-go/module/executiondatasync/execution_data" - mock "github.com/stretchr/testify/mock" ) @@ -146,9 +144,9 @@ func (_m *ExecutionDataTracker) GetStartHeightFromLatest(_a0 context.Context) (u return r0, r1 } -// OnExecutionData provides a mock function with given fields: _a0 -func (_m *ExecutionDataTracker) OnExecutionData(_a0 *execution_data.BlockExecutionDataEntity) { - _m.Called(_a0) +// OnExecutionData provides a mock function with no fields +func (_m *ExecutionDataTracker) OnExecutionData() { + _m.Called() } // NewExecutionDataTracker creates a new instance of ExecutionDataTracker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. diff --git a/engine/common/requester/engine.go b/engine/common/requester/engine.go index 1354425b1ed..8272f4bea50 100644 --- a/engine/common/requester/engine.go +++ b/engine/common/requester/engine.go @@ -364,23 +364,12 @@ func (e *Engine) dispatchRequest() (bool, error) { providers = filteredProviders } - // add item to list and set retry parameters + // add item to list (but don't update retry parameters yet - only after successful send) // NOTE: we add the retry interval to the last requested timestamp, // rather than using the current timestamp, in order to conserve a // more even distribution of timestamps over time, which should lead // to a more even distribution of entities over batch requests entityIDs = append(entityIDs, entityID) - item.NumAttempts++ - item.LastRequested = now - item.RetryAfter = e.cfg.RetryFunction(item.RetryAfter) - - // make sure the interval is within parameters - if item.RetryAfter < e.cfg.RetryInitial { - item.RetryAfter = e.cfg.RetryInitial - } - if item.RetryAfter > e.cfg.RetryMaximum { - item.RetryAfter = e.cfg.RetryMaximum - } // if we reached the maximum size for a batch, bail if uint(len(entityIDs)) >= e.cfg.BatchThreshold { @@ -417,8 +406,33 @@ func (e *Engine) dispatchRequest() (bool, error) { err = e.con.Unicast(req, providerID) if err != nil { + // Unicast failed - don't count this as an attempt, don't update retry state + // The item will be retried on the next dispatch cycle return true, fmt.Errorf("could not send request for entities %v: %w", logging.IDs(entityIDs), err) } + + // Unicast succeeded - now update retry state for all items in this batch + for _, entityID := range entityIDs { + item, exists := e.items[entityID] + if !exists { + // Item was removed (shouldn't happen, but be safe) + continue + } + + // Update retry parameters only after successful send + item.NumAttempts++ + item.LastRequested = now + item.RetryAfter = e.cfg.RetryFunction(item.RetryAfter) + + // make sure the interval is within parameters + if item.RetryAfter < e.cfg.RetryInitial { + item.RetryAfter = e.cfg.RetryInitial + } + if item.RetryAfter > e.cfg.RetryMaximum { + item.RetryAfter = e.cfg.RetryMaximum + } + } + e.requests[req.Nonce] = req // NOTE: we forget about requests after the expiry of the shortest retry time diff --git a/engine/common/requester/engine_test.go b/engine/common/requester/engine_test.go index e10555e19ba..9be46ea835b 100644 --- a/engine/common/requester/engine_test.go +++ b/engine/common/requester/engine_test.go @@ -1,7 +1,10 @@ package requester import ( + "fmt" + "math" "math/rand" + "sync" "testing" "time" @@ -486,3 +489,158 @@ func TestOriginValidation(t *testing.T) { // handler are called async, but this should be extremely quick unittest.AssertClosesBefore(t, called, time.Second) } + +// TestEntityByIDRetryWithDifferentNode tests that when a node in the selector group +// always fails (Unicast returns error), the requester will eventually retry with a different node. +func TestEntityByIDRetryWithDifferentNode(t *testing.T) { + identities := unittest.IdentityListFixture(3) + failingNodeID := identities[0].NodeID + workingNodeID := identities[1].NodeID + + final := &protocol.Snapshot{} + final.On("Identities", mock.Anything).Return( + func(selector flow.IdentityFilter[flow.Identity]) flow.IdentityList { + return identities.Filter(selector) + }, + nil, + ) + + state := &protocol.State{} + state.On("Final").Return(final) + + // Use short retry intervals for faster test execution + cfg := Config{ + BatchInterval: 50 * time.Millisecond, + BatchThreshold: 10, + RetryInitial: 50 * time.Millisecond, + RetryFunction: RetryConstant(), // Keep retry interval constant for predictable timing + RetryAttempts: math.MaxUint32, // Don't give up + RetryMaximum: 100 * time.Millisecond, + } + + // Track Unicast calls by node ID + unicastCalls := make(map[flow.Identifier][]*messages.EntityRequest) + var unicastMutex sync.Mutex + + con := &mocknetwork.Conduit{} + con.On("Unicast", mock.Anything, mock.Anything).Run( + func(args mock.Arguments) { + req := args.Get(0).(*messages.EntityRequest) + providerID := args.Get(1).(flow.Identifier) + + unicastMutex.Lock() + unicastCalls[providerID] = append(unicastCalls[providerID], req) + unicastMutex.Unlock() + }, + ).Return(func(event interface{}, targetID flow.Identifier) error { + // Failing node always returns error + if targetID == failingNodeID { + return fmt.Errorf("node %s always fails", targetID) + } + // Working node succeeds + return nil + }) + + entityID := unittest.IdentifierFixture() + + // Create engine with selector that includes both failing and working nodes + e := &Engine{ + unit: engine.NewUnit(), + metrics: metrics.NewNoopCollector(), + cfg: cfg, + state: state, + con: con, + items: make(map[flow.Identifier]*Item), + requests: make(map[uint64]*messages.EntityRequest), + selector: filter.Any, // Allow all nodes + create: func() flow.Entity { return &flow.Collection{} }, + handle: func(flow.Identifier, flow.Entity) {}, // No-op handler for this test + forcedDispatchOngoing: atomic.NewBool(false), + } + + // Request entity with selector that includes both nodes + selector := filter.Or( + filter.HasNodeID[flow.Identity](failingNodeID), + filter.HasNodeID[flow.Identity](workingNodeID), + ) + e.EntityByID(entityID, selector) + + // Force immediate dispatch - this may try either node first (random selection) + e.Force() + time.Sleep(100 * time.Millisecond) + + // Verify at least one node was tried + unicastMutex.Lock() + totalCalls := len(unicastCalls[failingNodeID]) + len(unicastCalls[workingNodeID]) + unicastMutex.Unlock() + require.Greater(t, totalCalls, 0, "at least one node should have been tried") + + // Keep retrying until working node succeeds + // Since Unicast failures don't count as attempts now, retries can happen immediately + maxRetries := 20 + for i := 0; i < maxRetries; i++ { + unicastMutex.Lock() + workingCalls := len(unicastCalls[workingNodeID]) + failingCalls := len(unicastCalls[failingNodeID]) + unicastMutex.Unlock() + + // If working node was tried and succeeded, we're done + // (successful Unicast means the request was sent, even if we don't simulate a response) + if workingCalls > 0 { + break + } + + // If only failing node was tried, force immediate retry (no need to wait since failures don't update state) + // Otherwise wait a bit for the next dispatch cycle + if failingCalls > 0 && workingCalls == 0 { + // Failing node was tried, retry immediately since state wasn't updated + time.Sleep(10 * time.Millisecond) + } else { + // Wait a bit longer if no calls yet or both were tried + time.Sleep(50 * time.Millisecond) + } + + // Force another dispatch - should retry and eventually try working node + e.Force() + time.Sleep(50 * time.Millisecond) + } + + // Verify both nodes were eventually tried + unicastMutex.Lock() + failingCalls := len(unicastCalls[failingNodeID]) + workingCalls := len(unicastCalls[workingNodeID]) + unicastMutex.Unlock() + + // At least one of the nodes should have been tried + // Due to random sampling, either node could be selected first + // But if failing node was tried, working node should eventually be tried + if failingCalls > 0 { + // Failing node was tried - verify working node was eventually tried + assert.Greater(t, workingCalls, 0, "if failing node was tried, working node should eventually be tried after failures") + } else { + // Working node was tried first and succeeded - that's fine too + assert.Greater(t, workingCalls, 0, "working node should have been tried") + } + + // Verify that requests were sent to working node (which succeeded) + // The working node should have at least one successful Unicast call + unicastMutex.Lock() + workingRequests := unicastCalls[workingNodeID] + unicastMutex.Unlock() + + assert.Greater(t, len(workingRequests), 0, "working node should have received requests") + // Verify the entity ID is in the requests sent to working node + foundEntity := false + for _, req := range workingRequests { + for _, id := range req.EntityIDs { + if id == entityID { + foundEntity = true + break + } + } + if foundEntity { + break + } + } + assert.True(t, foundEntity, "entity ID should be in requests sent to working node") +} diff --git a/engine/execution/ingestion/mocks/collection_store.go b/engine/execution/ingestion/mocks/collection_store.go index 1404d5baf58..6585be844d4 100644 --- a/engine/execution/ingestion/mocks/collection_store.go +++ b/engine/execution/ingestion/mocks/collection_store.go @@ -29,6 +29,11 @@ func (m *MockCollectionStore) ByID(id flow.Identifier) (*flow.Collection, error) return c, nil } +func (m *MockCollectionStore) ExistByID(id flow.Identifier) (bool, error) { + _, exists := m.byID[id] + return exists, nil +} + func (m *MockCollectionStore) Store(c *flow.Collection) (*flow.LightCollection, error) { m.byID[c.ID()] = c return c.Light(), nil diff --git a/integration/testnet/client.go b/integration/testnet/client.go index f12330dfc6a..56c4124e019 100644 --- a/integration/testnet/client.go +++ b/integration/testnet/client.go @@ -304,9 +304,9 @@ func (c *Client) WaitForSealed(ctx context.Context, id sdk.Identifier) (*sdk.Tra return c.waitForStatus(ctx, id, sdk.TransactionStatusSealed) } -// WaitForExecuted waits for the transaction to be executed, then returns the result. +// WaitForExecuted waits for the transaction to be executed or sealed, then returns the result. func (c *Client) WaitForExecuted(ctx context.Context, id sdk.Identifier) (*sdk.TransactionResult, error) { - return c.waitForStatus(ctx, id, sdk.TransactionStatusExecuted) + return c.waitForStatuses(ctx, id, sdk.TransactionStatusExecuted, sdk.TransactionStatusSealed) } // WaitUntilIndexed blocks until the node has indexed the given height. @@ -331,11 +331,32 @@ func (c *Client) waitForStatus( id sdk.Identifier, targetStatus sdk.TransactionStatus, ) (*sdk.TransactionResult, error) { - fmt.Printf("Waiting for transaction %s to be %v...\n", id, targetStatus) + return c.waitForStatuses(ctx, id, targetStatus) +} + +// waitForStatuses waits for the transaction to be in one of the target statuses, then returns the result. +func (c *Client) waitForStatuses( + ctx context.Context, + id sdk.Identifier, + targetStatuses ...sdk.TransactionStatus, +) (*sdk.TransactionResult, error) { + statusSet := make(map[sdk.TransactionStatus]bool, len(targetStatuses)) + for _, status := range targetStatuses { + statusSet[status] = true + } + + var statusStr string + if len(targetStatuses) == 1 { + statusStr = targetStatuses[0].String() + } else { + statusStr = fmt.Sprintf("%v", targetStatuses) + } + fmt.Printf("Waiting for transaction %s to be %s...\n", id, statusStr) + errCount := 0 var result *sdk.TransactionResult var err error - for result == nil || (result.Status != targetStatus) { + for result == nil || !statusSet[result.Status] { childCtx, cancel := context.WithTimeout(ctx, time.Second*30) result, err = c.client.GetTransactionResult(childCtx, id) cancel() @@ -354,7 +375,7 @@ func (c *Client) waitForStatus( } fmt.Println() - fmt.Printf("(Wait for Seal) Transaction %s %s\n", id, targetStatus) + fmt.Printf("Transaction %s is %s\n", id, result.Status) return result, err } diff --git a/integration/tests/access/cohort3/access_circuit_breaker_test.go b/integration/tests/access/cohort3/access_circuit_breaker_test.go index e172be9515d..e69dc6c81c2 100644 --- a/integration/tests/access/cohort3/access_circuit_breaker_test.go +++ b/integration/tests/access/cohort3/access_circuit_breaker_test.go @@ -73,8 +73,7 @@ func (s *AccessCircuitBreakerSuite) SetupTest() { nodeConfigs = append(nodeConfigs, exeConfig) // need one dummy verification node (unused ghost) - verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel), testnet.AsGhost()) - nodeConfigs = append(nodeConfigs, verConfig) + nodeConfigs = append(nodeConfigs, testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel))) // need one controllable collection node collConfig := testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel), testnet.WithAdditionalFlag("--hotstuff-proposal-duration=100ms")) diff --git a/integration/tests/access/cohort3/collection_indexing_test.go b/integration/tests/access/cohort3/collection_indexing_test.go index 230155efadf..e53ac9eac0c 100644 --- a/integration/tests/access/cohort3/collection_indexing_test.go +++ b/integration/tests/access/cohort3/collection_indexing_test.go @@ -12,12 +12,11 @@ import ( "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" ) // This suite tests collection syncing using the ingestion engine and the indexer. -const lastFullBlockMetric = "access_ingestion_last_full_finalized_block_height" - func TestCollectionIndexing(t *testing.T) { suite.Run(t, new(CollectionIndexingSuite)) } @@ -114,7 +113,7 @@ func (s *CollectionIndexingSuite) Test() { func (s *CollectionIndexingSuite) getLastFullHeight(containerName string) (uint64, error) { node := s.net.ContainerByName(containerName) metricsURL := fmt.Sprintf("http://0.0.0.0:%s/metrics", node.Port(testnet.MetricsPort)) - values := s.net.GetMetricFromContainer(s.T(), containerName, metricsURL, lastFullBlockMetric) + values := s.net.GetMetricFromContainer(s.T(), containerName, metricsURL, metrics.MetricsCollectionSyncedHeight) if len(values) == 0 { return 0, fmt.Errorf("no values found") diff --git a/integration/tests/access/cohort3/execution_state_sync_test.go b/integration/tests/access/cohort3/execution_state_sync_test.go index 08cb0d6c8dc..fc2e46d402d 100644 --- a/integration/tests/access/cohort3/execution_state_sync_test.go +++ b/integration/tests/access/cohort3/execution_state_sync_test.go @@ -196,13 +196,25 @@ func (s *ExecutionStateSyncSuite) executionDataForHeight(ctx context.Context, no BlockId: header.ID[:], EventEncodingVersion: entities.EventEncodingVersion_CCF_V0, }) + if err != nil { + s.log.Info(). + Uint64("height", height). + Hex("block_id", header.ID[:]). + Err(err). + Msg("failed to get execution data") return err } blockED, err = convert.MessageToBlockExecutionData(ed.GetBlockExecutionData(), flow.Localnet.Chain()) s.Require().NoError(err, "could not convert execution data") + s.log.Info(). + Uint64("height", height). + Hex("block_id", header.ID[:]). + Int("chunks", len(blockED.ChunkExecutionDatas)). + Msg("successfully retrieved execution data") + return err }), "could not get execution data for block %d", height) diff --git a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go index e5278418382..1b62371ebfd 100644 --- a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go +++ b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go @@ -23,8 +23,6 @@ import ( "github.com/onflow/flow-go/storage/store" ) -const maxReceiptHeightMetric = "access_ingestion_max_receipt_height" - func TestAccessStoreTxErrorMessages(t *testing.T) { suite.Run(t, new(AccessStoreTxErrorMessagesSuite)) } @@ -112,20 +110,25 @@ func (s *AccessStoreTxErrorMessagesSuite) TestAccessStoreTxErrorMessages() { defer cancel() // Create and send a transaction that will result in an error. + s.T().Log("Creating and sending transaction that will result in an error...") txResult := s.createAndSendTxWithTxError() + s.T().Logf("Transaction sent with ID: %s", txResult.TransactionID.String()) client, err := s.net.ContainerByName(s.accessContainerName).TestnetClient() s.Require().NoError(err) // wait until the node has indexed a few blocks past the transaction block height + s.T().Logf("Waiting until access node has indexed up to block height: %d", txResult.BlockHeight+10) err = client.WaitUntilIndexed(ctx, txResult.BlockHeight+10) s.Require().NoError(err) + s.T().Logf("Access node has indexed up to block height: %d", txResult.BlockHeight+10) // Stop the network containers before checking the results. s.net.StopContainers() txResults := []*sdk.TransactionResult{txResult} txErrorMessages := s.fetchTxErrorMessages(txResults, s.accessContainerName) + s.T().Logf("Fetched transaction error messages from the database.") s.verifyTxErrorMessage(txResults, txErrorMessages) } diff --git a/integration/tests/access/cohort4/grpc_compression_test.go b/integration/tests/access/cohort4/grpc_compression_test.go index 807262350dd..fcb585fe768 100644 --- a/integration/tests/access/cohort4/grpc_compression_test.go +++ b/integration/tests/access/cohort4/grpc_compression_test.go @@ -54,8 +54,7 @@ func (s *AccessGRPCSuite) SetupTest() { exeConfig := testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)) nodeConfigs = append(nodeConfigs, exeConfig) - // need one dummy verification node (unused ghost) - verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel), testnet.AsGhost()) + verConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel)) nodeConfigs = append(nodeConfigs, verConfig) // need one controllable collection node diff --git a/integration/tests/execution/suite.go b/integration/tests/execution/suite.go index 5d69a79b82b..3a2c3e2062a 100644 --- a/integration/tests/execution/suite.go +++ b/integration/tests/execution/suite.go @@ -144,7 +144,8 @@ func (s *Suite) SetupTest() { ) s.nodeConfigs = append(s.nodeConfigs, coll1Config, coll2Config) - // add the ghost (verification) node config + // add the verification node config as ghost + // note: no block will be sealed without verification node s.ghostID = unittest.IdentifierFixture() ghostConfig := testnet.NewNodeConfig(flow.RoleVerification, testnet.WithID(s.ghostID), diff --git a/module/execution/scripts_test.go b/module/execution/scripts_test.go index 757534366c9..ccfbccfd9e1 100644 --- a/module/execution/scripts_test.go +++ b/module/execution/scripts_test.go @@ -22,6 +22,7 @@ import ( "github.com/onflow/flow-go/fvm/storage/snapshot" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/state_synchronization/indexer" synctest "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" @@ -180,7 +181,7 @@ func (s *scriptTestSuite) SetupTest() { index := indexer.New( logger, - metrics.NewNoopCollector(), + module.ExecutionStateIndexerMetrics(metrics.NewNoopCollector()), nil, s.registerIndex, headers, @@ -191,8 +192,7 @@ func (s *scriptTestSuite) SetupTest() { nil, flow.Testnet, derivedChainData, - nil, - nil, + metrics.NewNoopCollector(), lockManager, ) diff --git a/module/jobqueue.go b/module/jobqueue.go index 9150956504b..4528a75206b 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -8,14 +8,25 @@ const ( ConsumeProgressVerificationBlockHeight = "ConsumeProgressVerificationBlockHeight" ConsumeProgressVerificationChunkIndex = "ConsumeProgressVerificationChunkIndex" - ConsumeProgressExecutionDataRequesterBlockHeight = "ConsumeProgressExecutionDataRequesterBlockHeight" + // for each sealed block height, request and store the execution data + ConsumeProgressExecutionDataRequesterBlockHeight = "ConsumeProgressExecutionDataRequesterBlockHeight" + + // for OnExecutionDataFetched notification distributor to distribute the events ConsumeProgressExecutionDataRequesterNotification = "ConsumeProgressExecutionDataRequesterNotification" ConsumeProgressExecutionDataIndexerBlockHeight = "ConsumeProgressExecutionDataIndexerBlockHeight" - ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" + // for AN ingestion engine to index guarantee by block id + ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" + + // (new)the highest block height that has indexed collections from fetching missing collections from LN + ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight = "ConsumeProgressAccessFetchAndIndexedCollectionsBlockHeight" + + // on finalized block, fetch tx error message from EN ConsumeProgressEngineTxErrorMessagesBlockHeight = "ConsumeProgressEngineTxErrorMessagesBlockHeight" - ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" + + // fetch collections + ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" ) // JobID is a unique ID of the job. diff --git a/module/jobqueue/consumer.go b/module/jobqueue/consumer.go index 035f625dfaf..a09e617a2fe 100644 --- a/module/jobqueue/consumer.go +++ b/module/jobqueue/consumer.go @@ -18,7 +18,7 @@ type Worker interface { } type Consumer struct { - mu sync.Mutex + mu sync.RWMutex log zerolog.Logger // Storage @@ -124,16 +124,16 @@ func (c *Consumer) Stop() { // Size returns number of in-memory jobs that consumer is processing. func (c *Consumer) Size() uint { - c.mu.Lock() - defer c.mu.Unlock() + c.mu.RLock() + defer c.mu.RUnlock() return uint(len(c.processings)) } // LastProcessedIndex returns the last processed job index func (c *Consumer) LastProcessedIndex() uint64 { - c.mu.Lock() - defer c.mu.Unlock() + c.mu.RLock() + defer c.mu.RUnlock() return c.processedIndex } diff --git a/module/jobqueue/finalized_block_reader.go b/module/jobqueue/finalized_block_reader.go index 952fef7555c..e84dd31ed0e 100644 --- a/module/jobqueue/finalized_block_reader.go +++ b/module/jobqueue/finalized_block_reader.go @@ -27,7 +27,7 @@ func NewFinalizedBlockReader(state protocol.State, blocks storage.Blocks) *Final // AtIndex returns the block job at the given index. // The block job at an index is just the finalized block at that index (i.e., height). -func (r FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { +func (r *FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { block, err := r.blockByHeight(index) if err != nil { return nil, fmt.Errorf("could not get block by index %v: %w", index, err) @@ -35,8 +35,13 @@ func (r FinalizedBlockReader) AtIndex(index uint64) (module.Job, error) { return BlockToJob(block), nil } +// ConvertJobToBlock converts a job to a block. +func (r *FinalizedBlockReader) ConvertJobToBlock(job module.Job) (*flow.Block, error) { + return JobToBlock(job) +} + // blockByHeight returns the block at the given height. -func (r FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) { +func (r *FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) { block, err := r.blocks.ByHeight(height) if err != nil { return nil, fmt.Errorf("could not get block by height %d: %w", height, err) @@ -46,7 +51,7 @@ func (r FinalizedBlockReader) blockByHeight(height uint64) (*flow.Block, error) } // Head returns the last finalized height as job index. -func (r FinalizedBlockReader) Head() (uint64, error) { +func (r *FinalizedBlockReader) Head() (uint64, error) { header, err := r.state.Final().Head() if err != nil { return 0, fmt.Errorf("could not get header of last finalized block: %w", err) diff --git a/module/jobqueue/sealed_header_reader.go b/module/jobqueue/sealed_header_reader.go index 4a12f76f0cc..e534e657f96 100644 --- a/module/jobqueue/sealed_header_reader.go +++ b/module/jobqueue/sealed_header_reader.go @@ -3,6 +3,7 @@ package jobqueue import ( "fmt" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" @@ -28,7 +29,7 @@ func NewSealedBlockHeaderReader(state protocol.State, headers storage.Headers) * // The block header job at an index is just the finalized block header at that index (i.e., height). // Error returns: // - storage.ErrNotFound if the provided index is not sealed -func (r SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { +func (r *SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { sealed, err := r.Head() if err != nil { return nil, fmt.Errorf("could not get last sealed block height: %w", err) @@ -48,8 +49,13 @@ func (r SealedBlockHeaderReader) AtIndex(index uint64) (module.Job, error) { return BlockHeaderToJob(header), nil } +// ConvertJobToBlockHeader converts a job to a block header. +func (r *SealedBlockHeaderReader) ConvertJobToBlockHeader(job module.Job) (*flow.Header, error) { + return JobToBlockHeader(job) +} + // Head returns the last sealed height as job index. -func (r SealedBlockHeaderReader) Head() (uint64, error) { +func (r *SealedBlockHeaderReader) Head() (uint64, error) { header, err := r.state.Sealed().Head() if err != nil { return 0, fmt.Errorf("could not get header of last sealed block: %w", err) diff --git a/module/metrics.go b/module/metrics.go index 6c6385019d3..1bbd96735df 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -1214,6 +1214,19 @@ type CollectionExecutedMetric interface { UpdateLastFullBlockHeight(height uint64) } +type CollectionSyncMetrics interface { + // CollectionFetchedHeight reports the highest consecutive block height for + // which collections have been fetched from collection nodes + CollectionFetchedHeight(height uint64) + + // CollectionSyncedHeight reports the highest consecutive block height for + // which collections have been synced from execution data + CollectionSyncedHeight(height uint64) + + // MissingCollectionQueueSize reports the number of missing collections currently in the queue + MissingCollectionQueueSize(size uint) +} + type MachineAccountMetrics interface { // AccountBalance reports the current balance of the machine account. AccountBalance(bal float64) diff --git a/module/metrics/collection_sync.go b/module/metrics/collection_sync.go new file mode 100644 index 00000000000..f112e371266 --- /dev/null +++ b/module/metrics/collection_sync.go @@ -0,0 +1,54 @@ +package metrics + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/onflow/flow-go/module" +) + +type CollectionSyncCollector struct { + collectionFetchedHeight prometheus.Gauge + collectionSyncedHeight prometheus.Gauge + missingCollectionQueueSize prometheus.Gauge +} + +var _ module.CollectionSyncMetrics = (*CollectionSyncCollector)(nil) + +// to be used by test case, must be consistent with collectionSyncedHeight +const MetricsCollectionSyncedHeight = "access_collection_sync_collection_synced_height" + +func NewCollectionSyncCollector() *CollectionSyncCollector { + return &CollectionSyncCollector{ + collectionFetchedHeight: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "collection_fetched_height", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the highest block height for which collections have been fetched", + }), + collectionSyncedHeight: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "collection_synced_height", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the highest block height for which collections have been synced from execution data", + }), + missingCollectionQueueSize: promauto.NewGauge(prometheus.GaugeOpts{ + Name: "missing_collection_queue_size", + Namespace: namespaceAccess, + Subsystem: "collection_sync", + Help: "the number of missing collections currently in the queue", + }), + } +} + +func (c *CollectionSyncCollector) CollectionFetchedHeight(height uint64) { + c.collectionFetchedHeight.Set(float64(height)) +} + +func (c *CollectionSyncCollector) CollectionSyncedHeight(height uint64) { + c.collectionSyncedHeight.Set(float64(height)) +} + +func (c *CollectionSyncCollector) MissingCollectionQueueSize(size uint) { + c.missingCollectionQueueSize.Set(float64(size)) +} diff --git a/module/metrics/noop.go b/module/metrics/noop.go index b1565826f40..0594c496a84 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -397,6 +397,12 @@ func (nc *NoopCollector) CollectionExecuted(light *flow.LightCollection) {} func (nc *NoopCollector) ExecutionReceiptReceived(r *flow.ExecutionReceipt) { } +var _ module.CollectionSyncMetrics = (*NoopCollector)(nil) + +func (nc *NoopCollector) CollectionFetchedHeight(height uint64) {} +func (nc *NoopCollector) CollectionSyncedHeight(height uint64) {} +func (nc *NoopCollector) MissingCollectionQueueSize(size uint) {} + func (nc *NoopCollector) AccountBalance(bal float64) {} func (nc *NoopCollector) RecommendedMinBalance(bal float64) {} func (nc *NoopCollector) IsMisconfigured(misconfigured bool) {} diff --git a/module/mock/collection_sync_metrics.go b/module/mock/collection_sync_metrics.go new file mode 100644 index 00000000000..42270a81944 --- /dev/null +++ b/module/mock/collection_sync_metrics.go @@ -0,0 +1,39 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +import mock "github.com/stretchr/testify/mock" + +// CollectionSyncMetrics is an autogenerated mock type for the CollectionSyncMetrics type +type CollectionSyncMetrics struct { + mock.Mock +} + +// CollectionFetchedHeight provides a mock function with given fields: height +func (_m *CollectionSyncMetrics) CollectionFetchedHeight(height uint64) { + _m.Called(height) +} + +// CollectionSyncedHeight provides a mock function with given fields: height +func (_m *CollectionSyncMetrics) CollectionSyncedHeight(height uint64) { + _m.Called(height) +} + +// MissingCollectionQueueSize provides a mock function with given fields: size +func (_m *CollectionSyncMetrics) MissingCollectionQueueSize(size uint) { + _m.Called(size) +} + +// NewCollectionSyncMetrics creates a new instance of CollectionSyncMetrics. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewCollectionSyncMetrics(t interface { + mock.TestingT + Cleanup(func()) +}) *CollectionSyncMetrics { + mock := &CollectionSyncMetrics{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/module/state_synchronization/execution_data_requester.go b/module/state_synchronization/execution_data_requester.go index dde19f6da52..948005fbf81 100644 --- a/module/state_synchronization/execution_data_requester.go +++ b/module/state_synchronization/execution_data_requester.go @@ -2,19 +2,20 @@ package state_synchronization import ( "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" ) // OnExecutionDataReceivedConsumer is a callback that is called ExecutionData is received for a new block -type OnExecutionDataReceivedConsumer func(*execution_data.BlockExecutionDataEntity) +type OnExecutionDataReceivedConsumer func() + +type ExecutionDataIndexedHeight interface { + // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData + // has been received. + HighestConsecutiveHeight() uint64 +} // ExecutionDataRequester is a component that syncs ExecutionData from the network, and exposes // a callback that is called when a new ExecutionData is received type ExecutionDataRequester interface { component.Component - - // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData - // has been received. - // This method must only be called after the component is Ready. If it is called early, an error is returned. - HighestConsecutiveHeight() (uint64, error) + ExecutionDataIndexedHeight } diff --git a/module/state_synchronization/indexer/indexer.go b/module/state_synchronization/indexer/indexer.go index db164a2c12d..a1738db0743 100644 --- a/module/state_synchronization/indexer/indexer.go +++ b/module/state_synchronization/indexer/indexer.go @@ -69,7 +69,7 @@ func NewIndexer( registers storage.RegisterIndex, indexer *IndexerCore, executionCache *cache.ExecutionDataCache, - executionDataLatestHeight func() (uint64, error), + executionDataLatestHeight func() uint64, processedHeightInitializer storage.ConsumerProgressInitializer, ) (*Indexer, error) { r := &Indexer{ @@ -158,17 +158,22 @@ func (i *Indexer) onBlockIndexed() error { highestIndexedHeight := i.jobConsumer.LastProcessedIndex() if lastProcessedHeight < highestIndexedHeight { + if lastProcessedHeight+1000 < highestIndexedHeight { + i.log.Warn().Msgf("notifying processed heights from %d to %d", lastProcessedHeight+1, highestIndexedHeight) + } // we need loop here because it's possible for a height to be missed here, // we should guarantee all heights are processed for height := lastProcessedHeight + 1; height <= highestIndexedHeight; height++ { - header, err := i.indexer.headers.ByHeight(height) - if err != nil { - // if the execution data is available, the block must be locally finalized - i.log.Error().Err(err).Msgf("could not get header for height %d:", height) - return fmt.Errorf("could not get header for height %d: %w", height, err) - } - - i.OnBlockProcessed(header.Height) + // Use BlockIDByHeight instead of ByHeight since we only need to verify the block exists + // and don't need the full header data. This avoids expensive header deserialization. + // _, err := i.indexer.headers.BlockIDByHeight(height) + // if err != nil { + // // if the execution data is available, the block must be locally finalized + // i.log.Error().Err(err).Msgf("could not get header for height %d:", height) + // return fmt.Errorf("could not get header for height %d: %w", height, err) + // } + + i.OnBlockProcessed(height) } i.lastProcessedHeight.Store(highestIndexedHeight) } @@ -206,7 +211,7 @@ func (i *Indexer) HighestIndexedHeight() (uint64, error) { } // OnExecutionData is used to notify when new execution data is downloaded by the execution data requester jobqueue. -func (i *Indexer) OnExecutionData(_ *execution_data.BlockExecutionDataEntity) { +func (i *Indexer) OnExecutionData() { i.exeDataNotifier.Notify() } diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index eceec2492ad..de892153086 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-core-contracts/lib/go/templates" - "github.com/onflow/flow-go/engine/access/ingestion/collections" "github.com/onflow/flow-go/fvm/blueprints" "github.com/onflow/flow-go/fvm/storage/derived" "github.com/onflow/flow-go/fvm/systemcontracts" @@ -32,7 +31,6 @@ type IndexerCore struct { fvmEnv templates.Environment metrics module.ExecutionStateIndexerMetrics collectionExecutedMetric module.CollectionExecutedMetric - collectionIndexer collections.CollectionIndexer registers storage.RegisterIndex headers storage.Headers @@ -64,7 +62,6 @@ func New( scheduledTransactions storage.ScheduledTransactions, chainID flow.ChainID, derivedChainData *derived.DerivedChainData, - collectionIndexer collections.CollectionIndexer, collectionExecutedMetric module.CollectionExecutedMetric, lockManager lockctx.Manager, ) *IndexerCore { @@ -94,7 +91,6 @@ func New( serviceAddress: chainID.Chain().ServiceAddress(), derivedChainData: derivedChainData, - collectionIndexer: collectionIndexer, collectionExecutedMetric: collectionExecutedMetric, lockManager: lockManager, } @@ -221,32 +217,6 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti return nil }) - g.Go(func() error { - start := time.Now() - - // Note: the access ingestion engine also indexes collections, starting when the block is - // finalized. This process can fall behind due to the node being offline, resource issues - // or network congestion. This indexer ensures that collections are never farther behind - // than the latest indexed block. Calling the collection handler with a collection that - // has already been indexed is a noop. - - // index all standard (non-system) collections - standardCollections := data.StandardCollections() - if len(standardCollections) > 0 { - err := c.collectionIndexer.IndexCollections(standardCollections) - if err != nil { - return fmt.Errorf("could not index collections: %w", err) - } - } - - lg.Debug(). - Int("collection_count", len(standardCollections)). - Dur("duration_ms", time.Since(start)). - Msg("indexed collections") - - return nil - }) - g.Go(func() error { start := time.Now() diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index 5317c791e5f..814c8d8c653 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -228,7 +228,7 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.indexer = New( log, - metrics.NewNoopCollector(), + module.ExecutionStateIndexerMetrics(metrics.NewNoopCollector()), db, i.registers, i.headers, @@ -239,7 +239,6 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.scheduledTransactions, i.g.ChainID(), derivedChainData, - i.collectionIndexer, collectionExecutedMetric, lockManager, ) @@ -290,7 +289,6 @@ func TestExecutionState_IndexBlockData(t *testing.T) { assert.ElementsMatch(t, tf.ExpectedRegisterEntries, entries) }). Return(nil) - test.collectionIndexer.On("IndexCollections", tf.ExpectedCollections).Return(nil).Once() for txID, scheduledTxID := range tf.ExpectedScheduledTransactions { test.scheduledTransactions. On("BatchIndex", mocks.MatchLock(storage.LockIndexScheduledTransaction), blockID, txID, scheduledTxID, mock.Anything). @@ -353,18 +351,14 @@ func TestExecutionState_IndexBlockData(t *testing.T) { test.results.On("BatchStore", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Unset() test.scheduledTransactions.On("BatchIndex", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Unset() test.registers.On("Store", mock.Anything, mock.Anything).Unset() - test.collectionIndexer.On("IndexCollections", mock.Anything).Unset() // setup mocks to behave as they would if the block was already indexed. // tx results and scheduled transactions will not be called since events returned an error. + // The second goroutine that processes registers will still run and call Store. test.events. On("BatchStore", mocks.MatchLock(storage.LockInsertEvent), blockID, []flow.EventsList{tf.ExpectedEvents}, mock.Anything). Return(storage.ErrAlreadyExists). Once() - test.collectionIndexer. - On("IndexCollections", tf.ExpectedCollections). - Return(nil). - Once() test.registers. On("Store", mock.Anything, tf.Block.Height). Return(nil). @@ -441,7 +435,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -477,7 +470,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -506,7 +498,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) @@ -552,7 +543,6 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { nil, flow.Testnet, derivedChainData, - collectionsmock.NewCollectionIndexer(t), nil, lockManager, ) diff --git a/module/state_synchronization/indexer/indexer_test.go b/module/state_synchronization/indexer/indexer_test.go index 202ae0fe099..719dbc9d189 100644 --- a/module/state_synchronization/indexer/indexer_test.go +++ b/module/state_synchronization/indexer/indexer_test.go @@ -90,8 +90,8 @@ func newIndexerTest(t *testing.T, g *fixtures.GeneratorSuite, blocks []*flow.Blo return test } -func (w *indexerTest) latestHeight() (uint64, error) { - return w.last().Height, nil +func (w *indexerTest) latestHeight() uint64 { + return w.last().Height } func (w *indexerTest) last() *flow.Block { @@ -107,7 +107,7 @@ func (w *indexerTest) run(ctx irrecoverable.SignalerContext, reachHeight uint64, unittest.RequireComponentsReadyBefore(w.t, testTimeout, w.worker) - w.worker.OnExecutionData(nil) + w.worker.OnExecutionData() select { case <-ctx.Done(): @@ -208,7 +208,6 @@ func TestIndexer_Success(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("IndexCollections", ed.StandardCollections()).Return(nil).Once() test.indexTest.registers.On("Store", flow.RegisterEntries{}, block.Height).Return(nil).Once() } @@ -256,7 +255,6 @@ func TestIndexer_Failure(t *testing.T) { }) test.executionData.On("Get", blockID).Return(ed, true).Once() - test.indexTest.collectionIndexer.On("IndexCollections", ed.StandardCollections()).Return(nil).Once() // return an error on the last block to trigger the error path if block.Height == lastHeight { diff --git a/module/state_synchronization/mock/execution_data_indexed_height.go b/module/state_synchronization/mock/execution_data_indexed_height.go new file mode 100644 index 00000000000..1cb8b94492a --- /dev/null +++ b/module/state_synchronization/mock/execution_data_indexed_height.go @@ -0,0 +1,42 @@ +// Code generated by mockery. DO NOT EDIT. + +package mock + +import mock "github.com/stretchr/testify/mock" + +// ExecutionDataIndexedHeight is an autogenerated mock type for the ExecutionDataIndexedHeight type +type ExecutionDataIndexedHeight struct { + mock.Mock +} + +// HighestConsecutiveHeight provides a mock function with no fields +func (_m *ExecutionDataIndexedHeight) HighestConsecutiveHeight() uint64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for HighestConsecutiveHeight") + } + + var r0 uint64 + if rf, ok := ret.Get(0).(func() uint64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(uint64) + } + + return r0 +} + +// NewExecutionDataIndexedHeight creates a new instance of ExecutionDataIndexedHeight. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewExecutionDataIndexedHeight(t interface { + mock.TestingT + Cleanup(func()) +}) *ExecutionDataIndexedHeight { + mock := &ExecutionDataIndexedHeight{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/module/state_synchronization/mock/execution_data_requester.go b/module/state_synchronization/mock/execution_data_requester.go index 40afda9ff30..06c0bf8921c 100644 --- a/module/state_synchronization/mock/execution_data_requester.go +++ b/module/state_synchronization/mock/execution_data_requester.go @@ -33,7 +33,7 @@ func (_m *ExecutionDataRequester) Done() <-chan struct{} { } // HighestConsecutiveHeight provides a mock function with no fields -func (_m *ExecutionDataRequester) HighestConsecutiveHeight() (uint64, error) { +func (_m *ExecutionDataRequester) HighestConsecutiveHeight() uint64 { ret := _m.Called() if len(ret) == 0 { @@ -41,23 +41,13 @@ func (_m *ExecutionDataRequester) HighestConsecutiveHeight() (uint64, error) { } var r0 uint64 - var r1 error - if rf, ok := ret.Get(0).(func() (uint64, error)); ok { - return rf() - } if rf, ok := ret.Get(0).(func() uint64); ok { r0 = rf() } else { r0 = ret.Get(0).(uint64) } - if rf, ok := ret.Get(1).(func() error); ok { - r1 = rf() - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } // Ready provides a mock function with no fields diff --git a/module/state_synchronization/requester/distributer.go b/module/state_synchronization/requester/distributer.go index ac97fb07982..002fb250f81 100644 --- a/module/state_synchronization/requester/distributer.go +++ b/module/state_synchronization/requester/distributer.go @@ -3,7 +3,6 @@ package requester import ( "sync" - "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/state_synchronization" ) @@ -27,11 +26,11 @@ func (p *ExecutionDataDistributor) AddOnExecutionDataReceivedConsumer(consumer s } // OnExecutionDataReceived is called when new execution data is received -func (p *ExecutionDataDistributor) OnExecutionDataReceived(executionData *execution_data.BlockExecutionDataEntity) { +func (p *ExecutionDataDistributor) OnExecutionDataReceived() { p.lock.RLock() defer p.lock.RUnlock() for _, consumer := range p.consumers { - consumer(executionData) + consumer() } } diff --git a/module/state_synchronization/requester/execution_data_requester.go b/module/state_synchronization/requester/execution_data_requester.go index 6a7bb409728..e47f3af9aec 100644 --- a/module/state_synchronization/requester/execution_data_requester.go +++ b/module/state_synchronization/requester/execution_data_requester.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/jobqueue" "github.com/onflow/flow-go/module/state_synchronization" - "github.com/onflow/flow-go/module/state_synchronization/requester/jobs" "github.com/onflow/flow-go/module/util" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" @@ -43,7 +42,7 @@ import ( // The requester listens to block finalization event, and checks if sealed height has been changed, // if changed, it create job for each un-downloaded and sealed height. // -// The requester is made up of 3 subcomponents: +// The requester is made up of 2 subcomponents: // // * OnBlockFinalized: receives block finalized events from the finalization registrar and // forwards them to the blockConsumer. @@ -51,23 +50,16 @@ import ( // * blockConsumer: is a jobqueue that receives block finalization events. On each event, // it checks for the latest sealed block, then uses a pool of workers to // download ExecutionData for each block from the network. After each -// successful download, the blockConsumer sends a notification to the -// notificationConsumer that a new ExecutionData is available. +// successful download, the blockConsumer notifies the distributor that a new +// ExecutionData is available. // -// * notificationConsumer: is a jobqueue that receives ExecutionData fetched events. On each event, -// it checks if ExecutionData for the next consecutive block height is -// available, then uses a single worker to send notifications to registered -// consumers. -// the registered consumers are guaranteed to receive each sealed block in -// consecutive height at least once. -// -// +------------------+ +---------------+ +----------------------+ -// -->| OnBlockFinalized |----->| blockConsumer | +-->| notificationConsumer | -// +------------------+ +-------+-------+ | +-----------+----------+ -// | | | -// +------+------+ | +------+------+ -// xN | Worker Pool |----+ x1 | Worker Pool |----> Registered consumers -// +-------------+ +-------------+ +// +------------------+ +---------------+ +// -->| OnBlockFinalized |----->| blockConsumer |----> Distributor +// +------------------+ +-------+-------+ +// | +// +------+------+ +// xN | Worker Pool | +// +-------------+ const ( // DefaultFetchTimeout is the default initial timeout for fetching ExecutionData from the @@ -125,14 +117,11 @@ type executionDataRequester struct { // Local db objects headers storage.Headers - executionDataReader *jobs.ExecutionDataReader - // Notifiers for queue consumers finalizationNotifier engine.Notifier // Job queues - blockConsumer *jobqueue.ComponentConsumer - notificationConsumer *jobqueue.ComponentConsumer + blockConsumer *jobqueue.ComponentConsumer execDataCache *cache.ExecutionDataCache distributor *ExecutionDataDistributor @@ -147,7 +136,6 @@ func New( downloader execution_data.Downloader, execDataCache *cache.ExecutionDataCache, processedHeight storage.ConsumerProgressInitializer, - processedNotifications storage.ConsumerProgressInitializer, state protocol.State, headers storage.Headers, cfg ExecutionDataConfig, @@ -165,8 +153,6 @@ func New( distributor: distributor, } - executionDataNotifier := engine.NewNotifier() - // jobqueue Jobs object that tracks sealed blocks by height. This is used by the blockConsumer // to get a sequential list of sealed blocks. sealedBlockReader := jobqueue.NewSealedBlockHeaderReader(state, headers) @@ -179,8 +165,7 @@ func New( // downloaded, it updates and persists the highest consecutive downloaded height with // `processedHeight`. That way, if the node crashes, it reads the `processedHeight` and resume // from `processedHeight + 1`. If the database is empty, rootHeight will be used to init the - // last processed height. Once the execution data is fetched and stored, it notifies - // `executionDataNotifier`. + // last processed height. Once the execution data is fetched and stored, it notifies the distributor. blockConsumer, err := jobqueue.NewComponentConsumer( e.log.With().Str("module", "block_consumer").Logger(), e.finalizationNotifier.Channel(), // to listen to finalization events to find newly sealed blocks @@ -196,57 +181,12 @@ func New( } e.blockConsumer = blockConsumer - // notifies notificationConsumer when new ExecutionData blobs are available - // SetPostNotifier will notify executionDataNotifier AFTER e.blockConsumer.LastProcessedIndex is updated. - // Even though it doesn't guarantee to notify for every height at least once, the notificationConsumer is - // able to guarantee to process every height at least once, because the notificationConsumer finds new jobs - // using executionDataReader which finds new heights using e.blockConsumer.LastProcessedIndex - e.blockConsumer.SetPostNotifier(func(module.JobID) { executionDataNotifier.Notify() }) - - // jobqueue Jobs object tracks downloaded execution data by height. This is used by the - // notificationConsumer to get downloaded execution data from storage. - e.executionDataReader = jobs.NewExecutionDataReader( - e.execDataCache, - e.config.FetchTimeout, - // method to get highest consecutive height that has downloaded execution data. it is used - // here by the notification job consumer to discover new jobs. - // Note: we don't want to notify notificationConsumer for a block if it has not downloaded - // execution data yet. - func() (uint64, error) { - return e.blockConsumer.LastProcessedIndex(), nil - }, - ) - - // notificationConsumer consumes `OnExecutionDataFetched` events, and ensures its consumer - // receives this event in consecutive block height order. - // It listens to events from `executionDataNotifier`, which is delivered when - // a block's execution data is downloaded and stored, and checks the `executionDataCache` to - // find if the next un-processed consecutive height is available. - // To know what's the height of the next un-processed consecutive height, it reads the latest - // consecutive height in `processedNotifications`. And it's persisted in storage to be crash-resistant. - // When a new consecutive height is available, it calls `processNotificationJob` to notify all the - // `e.consumers`. - // Note: the `e.consumers` will be guaranteed to receive at least one `OnExecutionDataFetched` event - // for each sealed block in consecutive block height order. - e.notificationConsumer, err = jobqueue.NewComponentConsumer( - e.log.With().Str("module", "notification_consumer").Logger(), - executionDataNotifier.Channel(), // listen for notifications from the block consumer - processedNotifications, // read and persist the notified height - e.executionDataReader, // read execution data by height - e.config.InitialBlockHeight, // initial "last processed" height for empty db - e.processNotificationJob, // process the job to send notifications for an execution data - 1, // use a single worker to ensure notification is delivered in consecutive order - 0, // search ahead limit controlled by worker count - ) - if err != nil { - return nil, fmt.Errorf("failed to create notification consumer: %w", err) - } + e.blockConsumer.SetPostNotifier(func(module.JobID) { e.distributor.OnExecutionDataReceived() }) e.metrics.ExecutionDataFetchFinished(0, true, e.blockConsumer.LastProcessedIndex()) e.Component = component.NewComponentManagerBuilder(). AddWorker(e.runBlockConsumer). - AddWorker(e.runNotificationConsumer). Build() // register callback with finalization registrar @@ -263,15 +203,8 @@ func (e *executionDataRequester) onBlockFinalized(*model.Block) { // HighestConsecutiveHeight returns the highest consecutive block height for which ExecutionData // has been received. // This method must only be called after the component is Ready. If it is called early, an error is returned. -func (e *executionDataRequester) HighestConsecutiveHeight() (uint64, error) { - select { - case <-e.blockConsumer.Ready(): - default: - // LastProcessedIndex is not meaningful until the component has completed startup - return 0, fmt.Errorf("HighestConsecutiveHeight must not be called before the component is ready") - } - - return e.blockConsumer.LastProcessedIndex(), nil +func (e *executionDataRequester) HighestConsecutiveHeight() uint64 { + return e.blockConsumer.LastProcessedIndex() } // runBlockConsumer runs the blockConsumer component @@ -281,11 +214,6 @@ func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerCont return // context cancelled } - err = util.WaitClosed(ctx, e.notificationConsumer.Ready()) - if err != nil { - return // context cancelled - } - e.blockConsumer.Start(ctx) err = util.WaitClosed(ctx, e.blockConsumer.Ready()) @@ -296,19 +224,6 @@ func (e *executionDataRequester) runBlockConsumer(ctx irrecoverable.SignalerCont <-e.blockConsumer.Done() } -// runNotificationConsumer runs the notificationConsumer component -func (e *executionDataRequester) runNotificationConsumer(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - e.executionDataReader.AddContext(ctx) - e.notificationConsumer.Start(ctx) - - err := util.WaitClosed(ctx, e.notificationConsumer.Ready()) - if err == nil { - ready() - } - - <-e.notificationConsumer.Done() -} - // Fetch Worker Methods // processBlockJob consumes jobs from the blockConsumer and attempts to download an ExecutionData @@ -326,10 +241,13 @@ func (e *executionDataRequester) processBlockJob(ctx irrecoverable.SignalerConte return } - // errors are thrown as irrecoverable errors except context cancellation, and invalid blobs - // invalid blobs are logged, and never completed, which will halt downloads after maxSearchAhead - // is reached. - e.log.Error().Err(err).Str("job_id", string(job.ID())).Msg("error encountered while processing block job") + // errors are thrown as irrecoverable errors except context cancellation, which is usually + // triggered by restart, and invalid blobs invalid blobs are logged, and never completed, + // which will halt downloads after maxSearchAhead is reached. + e.log.Error().Err(err). + Hex("block_id", logging.ID(header.ID())). + Uint64("height", header.Height). + Str("job_id", string(job.ID())).Msg("error encountered while processing block job") } // processSealedHeight downloads ExecutionData for the given block height. @@ -422,27 +340,6 @@ func (e *executionDataRequester) processFetchRequest(parentCtx irrecoverable.Sig return nil } -// Notification Worker Methods - -func (e *executionDataRequester) processNotificationJob(ctx irrecoverable.SignalerContext, job module.Job, jobComplete func()) { - // convert job into a block entry - entry, err := jobs.JobToBlockEntry(job) - if err != nil { - ctx.Throw(fmt.Errorf("failed to convert job to entry: %w", err)) - } - - e.log.Debug(). - Hex("block_id", logging.ID(entry.BlockID)). - Uint64("height", entry.Height). - Msgf("notifying for block") - - // send notifications - e.distributor.OnExecutionDataReceived(entry.ExecutionData) - jobComplete() - - e.metrics.NotificationSent(entry.Height) -} - func isInvalidBlobError(err error) bool { var malformedDataError *execution_data.MalformedDataError var blobSizeLimitExceededError *execution_data.BlobSizeLimitExceededError diff --git a/module/state_synchronization/requester/execution_data_requester_test.go b/module/state_synchronization/requester/execution_data_requester_test.go index e90405a7f0b..0f8f5a9b451 100644 --- a/module/state_synchronization/requester/execution_data_requester_test.go +++ b/module/state_synchronization/requester/execution_data_requester_test.go @@ -32,6 +32,7 @@ import ( synctest "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/state/protocol" statemock "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/operation/pebbleimpl" "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" @@ -45,6 +46,8 @@ type ExecutionDataRequesterSuite struct { db *pebble.DB downloader *exedatamock.Downloader distributor *ExecutionDataDistributor + edCache *cache.ExecutionDataCache + headers storage.Headers run edTestRun @@ -410,10 +413,11 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun heroCache := herocache.NewBlockExecutionData(subscription.DefaultCacheSize, logger, metricsCollector) edCache := cache.NewExecutionDataCache(suite.downloader, headers, seals, results, heroCache) + suite.edCache = edCache + suite.headers = headers followerDistributor := pubsub.NewFollowerDistributor() processedHeight := store.NewConsumerProgress(pebbleimpl.ToDB(suite.db), module.ConsumeProgressExecutionDataRequesterBlockHeight) - processedNotification := store.NewConsumerProgress(pebbleimpl.ToDB(suite.db), module.ConsumeProgressExecutionDataRequesterNotification) edr, err := New( logger, @@ -421,7 +425,6 @@ func (suite *ExecutionDataRequesterSuite) prepareRequesterTest(cfg *fetchTestRun suite.downloader, edCache, processedHeight, - processedNotification, state, headers, ExecutionDataConfig{ @@ -450,7 +453,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTestHalts(edr state_synchr fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -477,7 +480,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTestPauseResume(edr state_ fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -515,7 +518,7 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTest(edr state_synchroniza fetchedExecutionData := cfg.FetchedExecutionData() // collect all execution data notifications - suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData)) + suite.distributor.AddOnExecutionDataReceivedConsumer(suite.consumeExecutionDataNotifications(cfg, func() { close(testDone) }, fetchedExecutionData, edr, suite.edCache, suite.headers)) edr.Start(signalerCtx) unittest.RequireCloseBefore(suite.T(), edr.Ready(), cfg.waitTimeout, "timed out waiting for requester to be ready") @@ -533,24 +536,52 @@ func (suite *ExecutionDataRequesterSuite) runRequesterTest(edr state_synchroniza return fetchedExecutionData } -func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg *fetchTestRun, done func(), fetchedExecutionData map[flow.Identifier]*execution_data.BlockExecutionData) func(ed *execution_data.BlockExecutionDataEntity) { - return func(ed *execution_data.BlockExecutionDataEntity) { - if _, has := fetchedExecutionData[ed.BlockID]; has { - suite.T().Errorf("duplicate execution data for block %s", ed.BlockID) - return - } +func (suite *ExecutionDataRequesterSuite) consumeExecutionDataNotifications(cfg *fetchTestRun, done func(), fetchedExecutionData map[flow.Identifier]*execution_data.BlockExecutionData, edr state_synchronization.ExecutionDataRequester, edCache *cache.ExecutionDataCache, headers storage.Headers) func() { + var lastProcessedHeight uint64 = cfg.startHeight - 1 + var mu sync.Mutex + return func() { + mu.Lock() + defer mu.Unlock() - fetchedExecutionData[ed.BlockID] = ed.BlockExecutionData - if _, ok := cfg.blocksByID[ed.BlockID]; !ok { - suite.T().Errorf("unknown execution data for block %s", ed.BlockID) - return - } + // Get the highest consecutive height that has execution data + highestHeight := edr.HighestConsecutiveHeight() + + // Process all heights from lastProcessedHeight + 1 to highestHeight + for height := lastProcessedHeight + 1; height <= highestHeight; height++ { + blockID, err := headers.BlockIDByHeight(height) + if err != nil { + suite.T().Errorf("failed to get block ID for height %d: %v", height, err) + return + } + + if _, has := fetchedExecutionData[blockID]; has { + // Skip if already processed + continue + } - suite.T().Logf("notified of execution data for block %v height %d (%d/%d)", ed.BlockID, cfg.blocksByID[ed.BlockID].Height, len(fetchedExecutionData), cfg.sealedCount) + // Fetch execution data from cache + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + ed, err := edCache.ByBlockID(ctx, blockID) + cancel() + if err != nil { + suite.T().Errorf("failed to get execution data for block %s: %v", blockID, err) + return + } + + fetchedExecutionData[blockID] = ed.BlockExecutionData + if _, ok := cfg.blocksByID[blockID]; !ok { + suite.T().Errorf("unknown execution data for block %s", blockID) + return + } - if cfg.IsLastSeal(ed.BlockID) { - done() + suite.T().Logf("notified of execution data for block %v height %d (%d/%d)", blockID, height, len(fetchedExecutionData), cfg.sealedCount) + + if cfg.IsLastSeal(blockID) { + done() + } } + + lastProcessedHeight = highestHeight } } diff --git a/module/state_synchronization/requester/jobs/execution_data_reader.go b/module/state_synchronization/requester/jobs/execution_data_reader.go index 721356ac7df..b63171f15ae 100644 --- a/module/state_synchronization/requester/jobs/execution_data_reader.go +++ b/module/state_synchronization/requester/jobs/execution_data_reader.go @@ -27,7 +27,7 @@ type ExecutionDataReader struct { store *cache.ExecutionDataCache fetchTimeout time.Duration - highestConsecutiveHeight func() (uint64, error) + highestConsecutiveHeight func() uint64 // TODO: refactor this to accept a context in AtIndex instead of storing it on the struct. // This requires also refactoring jobqueue.Consumer @@ -38,7 +38,7 @@ type ExecutionDataReader struct { func NewExecutionDataReader( store *cache.ExecutionDataCache, fetchTimeout time.Duration, - highestConsecutiveHeight func() (uint64, error), + highestConsecutiveHeight func() uint64, ) *ExecutionDataReader { return &ExecutionDataReader{ store: store, @@ -62,10 +62,7 @@ func (r *ExecutionDataReader) AtIndex(height uint64) (module.Job, error) { } // data for the requested height or a lower height, has not been downloaded yet. - highestHeight, err := r.highestConsecutiveHeight() - if err != nil { - return nil, fmt.Errorf("failed to get highest height: %w", err) - } + highestHeight := r.highestConsecutiveHeight() if height > highestHeight { return nil, storage.ErrNotFound @@ -88,5 +85,5 @@ func (r *ExecutionDataReader) AtIndex(height uint64) (module.Job, error) { // Head returns the highest consecutive block height with downloaded execution data func (r *ExecutionDataReader) Head() (uint64, error) { - return r.highestConsecutiveHeight() + return r.highestConsecutiveHeight(), nil } diff --git a/module/state_synchronization/requester/jobs/execution_data_reader_test.go b/module/state_synchronization/requester/jobs/execution_data_reader_test.go index 74d3ee8e655..49fa56a923c 100644 --- a/module/state_synchronization/requester/jobs/execution_data_reader_test.go +++ b/module/state_synchronization/requester/jobs/execution_data_reader_test.go @@ -99,9 +99,7 @@ func (suite *ExecutionDataReaderSuite) reset() { suite.reader = NewExecutionDataReader( cache, suite.fetchTimeout, - func() (uint64, error) { - return suite.highestAvailableHeight(), nil - }, + suite.highestAvailableHeight, ) } @@ -189,11 +187,12 @@ func (suite *ExecutionDataReaderSuite) TestAtIndex() { } func (suite *ExecutionDataReaderSuite) TestHead() { + expectedIndex := uint64(15) + suite.highestAvailableHeight = func() uint64 { + return expectedIndex + } + suite.reset() suite.runTest(func() { - expectedIndex := uint64(15) - suite.highestAvailableHeight = func() uint64 { - return expectedIndex - } index, err := suite.reader.Head() assert.NoError(suite.T(), err) assert.Equal(suite.T(), expectedIndex, index) diff --git a/network/p2p/blob/blob_service.go b/network/p2p/blob/blob_service.go index b4a1257b200..ed6e4c0ed42 100644 --- a/network/p2p/blob/blob_service.go +++ b/network/p2p/blob/blob_service.go @@ -55,6 +55,7 @@ var _ component.Component = (*blobService)(nil) type BlobServiceConfig struct { ReprovideInterval time.Duration // the interval at which the DHT provider entries are refreshed BitswapOptions []bitswap.Option // options to pass to the Bitswap service + SkipBloomCache bool // if true, skip the bloom cache and use plain blockstore } // WithReprovideInterval sets the interval at which DHT provider entries are refreshed @@ -98,6 +99,15 @@ func WithRateLimit(r float64, b int) network.BlobServiceOption { } } +// WithSkipBloomCache disables the bloom cache, using a plain blockstore instead. +// This avoids the CPU cost of building the bloom filter on startup by scanning all keys. +// Pebble's built-in bloom filters (persisted in SSTables) are still used for efficient lookups. +func WithSkipBloomCache(skip bool) network.BlobServiceOption { + return func(bs network.BlobService) { + bs.(*blobService).config.SkipBloomCache = skip + } +} + // NewBlobService creates a new BlobService. func NewBlobService( host host.Host, @@ -109,26 +119,36 @@ func NewBlobService( opts ...network.BlobServiceOption, ) (*blobService, error) { bsNetwork := bsnet.NewFromIpfsHost(host, r, bsnet.Prefix(protocol.ID(prefix))) - blockStore, err := blockstore.CachedBlockstore( - context.Background(), - blockstore.NewBlockstore(ds), - blockstore.DefaultCacheOpts(), - ) - if err != nil { - return nil, fmt.Errorf("failed to create cached blockstore: %w", err) - } + + blockStore := blockstore.NewBlockstore(ds) + bs := &blobService{ prefix: prefix, config: &BlobServiceConfig{ ReprovideInterval: DefaultReprovideInterval, + SkipBloomCache: false, // default: use cached blockstore }, - blockStore: blockStore, } + // Apply options before creating blockstore, as SkipBloomCache affects blockstore creation for _, opt := range opts { opt(bs) } + if bs.config.SkipBloomCache { + cachedBlockStore, err := blockstore.CachedBlockstore( + context.Background(), + blockStore, + blockstore.DefaultCacheOpts(), + ) + if err != nil { + return nil, fmt.Errorf("failed to create cached blockstore: %w", err) + } + blockStore = cachedBlockStore + } + + bs.blockStore = blockStore + cm := component.NewComponentManagerBuilder(). AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { btswp := bitswap.New(ctx, bsNetwork, bs.blockStore, bs.config.BitswapOptions...) diff --git a/scripts/update-cadence.sh b/scripts/update-cadence.sh deleted file mode 100755 index 509f29ba582..00000000000 --- a/scripts/update-cadence.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/sh -# -# This script updates all cadence dependencies to a new version. -# Specify the desired version as the only argument when running the script: -# ./scripts/update-cadence.sh v1.2.3 - -go get github.com/onflow/cadence@$1 -cd integration -go get github.com/onflow/cadence@$1 -cd ../insecure/ -go get github.com/onflow/cadence@$1 -cd .. diff --git a/scripts/update-core-contracts.sh b/scripts/update-core-contracts.sh deleted file mode 100755 index a2a294d0270..00000000000 --- a/scripts/update-core-contracts.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/sh -# -# This script updates all flow-core-contracts dependencies to a new version. -# Specify the desired version as the only argument when running the script: -# ./scripts/update-core-contracts.sh v1.2.3 - -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd integration -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd ../insecure/ -go get github.com/onflow/flow-core-contracts/lib/go/contracts@$1 -go get github.com/onflow/flow-core-contracts/lib/go/templates@$1 -cd .. diff --git a/storage/collections.go b/storage/collections.go index b8b9e026e30..044964a33ca 100644 --- a/storage/collections.go +++ b/storage/collections.go @@ -22,6 +22,12 @@ type CollectionsReader interface { // - `storage.ErrNotFound` if no light collection was found. LightByID(collID flow.Identifier) (*flow.LightCollection, error) + // ExistByID checks whether a collection with the given ID exists in storage. + // Returns (true, nil) if it exists, + // Returns (false, nil) if it does not exist. + // No errors are expected during normal operation. + ExistByID(collID flow.Identifier) (bool, error) + // LightByTransactionID returns a reduced representation of the collection // holding the given transaction ID. The reduced collection references the // constituent transactions by their hashes. @@ -48,6 +54,7 @@ type Collections interface { // StoreAndIndexByTransaction stores the collection and indexes it by transaction. // This is used by access node storing collections for finalized blocks. // + // deprecated, only used by AN ingestion engine, which is deprecated // CAUTION: current approach is NOT BFT and needs to be revised in the future. // Honest clusters ensure a transaction can only belong to one collection. However, in rare // cases, the collector clusters can exceed byzantine thresholds -- making it possible to diff --git a/storage/inmemory/collections_reader.go b/storage/inmemory/collections_reader.go index 1efc5bc16d7..dfce61b7818 100644 --- a/storage/inmemory/collections_reader.go +++ b/storage/inmemory/collections_reader.go @@ -61,6 +61,15 @@ func (c *CollectionsReader) LightByID(collID flow.Identifier) (*flow.LightCollec return val, nil } +// ExistByID checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func (c *CollectionsReader) ExistByID(collID flow.Identifier) (bool, error) { + _, exists := c.collections[collID] + return exists, nil +} + // LightByTransactionID returns the collection for the given transaction ID. Only retrieves transaction hashes. // // Expected error returns during normal operation: diff --git a/storage/locks.go b/storage/locks.go index 1981438b13b..88a7c3313e7 100644 --- a/storage/locks.go +++ b/storage/locks.go @@ -92,6 +92,12 @@ var LockGroupExecutionBootstrap = []string{ LockIndexStateCommitment, } +// deprecated (leo), only used by engine/access/ingestion/engine, which is deprecated +var LockGroupAccessFinalizingBlock = []string{ + LockIndexBlockByPayloadGuarantees, + LockIndexExecutionResult, +} + var LockGroupExecutionSaveExecutionResult = []string{ LockIndexChunkDataPackByChunkID, LockInsertEvent, @@ -102,11 +108,6 @@ var LockGroupExecutionSaveExecutionResult = []string{ LockIndexStateCommitment, } -var LockGroupAccessFinalizingBlock = []string{ - LockIndexBlockByPayloadGuarantees, - LockIndexExecutionResult, -} - var LockGroupAccessOptimisticSyncBlockPersist = []string{ LockInsertCollection, LockInsertEvent, @@ -154,7 +155,7 @@ func addLocks(builder lockctx.DAGPolicyBuilder, locks []string) { func makeLockPolicy() lockctx.Policy { builder := lockctx.NewDAGPolicyBuilder() - addLocks(builder, LockGroupAccessFinalizingBlock) + addLocks(builder, LockGroupAccessFinalizingBlock) // deprecated (leo) addLocks(builder, LockGroupAccessStateSyncIndexBlockData) addLocks(builder, LockGroupAccessOptimisticSyncBlockPersist) addLocks(builder, LockGroupExecutionBootstrap) diff --git a/storage/mock/collections.go b/storage/mock/collections.go index 592ba5fba26..579f21b3dff 100644 --- a/storage/mock/collections.go +++ b/storage/mock/collections.go @@ -76,6 +76,34 @@ func (_m *Collections) ByID(collID flow.Identifier) (*flow.Collection, error) { return r0, r1 } +// ExistByID provides a mock function with given fields: collID +func (_m *Collections) ExistByID(collID flow.Identifier) (bool, error) { + ret := _m.Called(collID) + + if len(ret) == 0 { + panic("no return value specified for ExistByID") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(collID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(collID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(collID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // LightByID provides a mock function with given fields: collID func (_m *Collections) LightByID(collID flow.Identifier) (*flow.LightCollection, error) { ret := _m.Called(collID) diff --git a/storage/mock/collections_reader.go b/storage/mock/collections_reader.go index 3a8e071d6bc..f8d337ea5cf 100644 --- a/storage/mock/collections_reader.go +++ b/storage/mock/collections_reader.go @@ -42,6 +42,34 @@ func (_m *CollectionsReader) ByID(collID flow.Identifier) (*flow.Collection, err return r0, r1 } +// ExistByID provides a mock function with given fields: collID +func (_m *CollectionsReader) ExistByID(collID flow.Identifier) (bool, error) { + ret := _m.Called(collID) + + if len(ret) == 0 { + panic("no return value specified for ExistByID") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(collID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(collID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(collID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // LightByID provides a mock function with given fields: collID func (_m *CollectionsReader) LightByID(collID flow.Identifier) (*flow.LightCollection, error) { ret := _m.Called(collID) diff --git a/storage/operation/collections.go b/storage/operation/collections.go index b2d20c9f953..01942382be3 100644 --- a/storage/operation/collections.go +++ b/storage/operation/collections.go @@ -32,6 +32,14 @@ func RetrieveCollection(r storage.Reader, collID flow.Identifier, collection *fl return RetrieveByKey(r, MakePrefix(codeCollection, collID), collection) } +// CollectionExists checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func CollectionExists(r storage.Reader, collID flow.Identifier) (bool, error) { + return KeyExists(r, MakePrefix(codeCollection, collID)) +} + // RemoveCollection removes a collection from the storage. // CAUTION: this is for recovery purposes only, and should not be used during normal operations! // It returns nil if the collection does not exist. diff --git a/storage/pebble/config.go b/storage/pebble/config.go index 06e8497d50d..be0d4f5dcba 100644 --- a/storage/pebble/config.go +++ b/storage/pebble/config.go @@ -11,6 +11,7 @@ import ( // DefaultPebbleOptions returns an optimized set of pebble options. // This is mostly copied form pebble's nightly performance benchmark. func DefaultPebbleOptions(logger zerolog.Logger, cache *pebble.Cache, comparer *pebble.Comparer) *pebble.Options { + eventListener := NewCompactionEventListener(logger) opts := &pebble.Options{ Cache: cache, Comparer: comparer, @@ -32,6 +33,10 @@ func DefaultPebbleOptions(logger zerolog.Logger, cache *pebble.Cache, comparer * // The default is 1. MaxConcurrentCompactions: func() int { return 4 }, Logger: util.NewLogger(logger), + EventListener: &pebble.EventListener{ + CompactionBegin: eventListener.CompactionBegin, + CompactionEnd: eventListener.CompactionEnd, + }, } for i := 0; i < len(opts.Levels); i++ { diff --git a/storage/pebble/event_listener.go b/storage/pebble/event_listener.go new file mode 100644 index 00000000000..1cbf45d3aa5 --- /dev/null +++ b/storage/pebble/event_listener.go @@ -0,0 +1,145 @@ +package pebble + +import ( + "time" + + "github.com/cockroachdb/pebble/v2" + "github.com/rs/zerolog" +) + +const ( + // DefaultMinCompactionSizeBytes is the default minimum input size for logging compactions. + // Small compactions below this threshold are ignored. + DefaultMinCompactionSizeBytes int64 = 200 << 20 // 200 MB + + // DefaultMinCompactionDuration is the default minimum duration for logging compactions. + // Fast compactions below this threshold are ignored. + DefaultMinCompactionDuration = 5 * time.Second + + // DefaultMinCompactionFiles is the default minimum number of input files for logging compactions. + // Compactions with fewer files are ignored. + DefaultMinCompactionFiles = 5 +) + +// CompactionEventListener implements pebble.EventListener to log compaction events. +// Only significant compactions (large size, long duration, or many files) are logged. +type CompactionEventListener struct { + logger zerolog.Logger + minSizeBytes int64 + minDuration time.Duration + minFiles int +} + +// NewCompactionEventListener creates a new CompactionEventListener with default thresholds. +func NewCompactionEventListener(logger zerolog.Logger) *CompactionEventListener { + return NewCompactionEventListenerWithThresholds( + logger, + DefaultMinCompactionSizeBytes, + DefaultMinCompactionDuration, + DefaultMinCompactionFiles, + ) +} + +// NewCompactionEventListenerWithThresholds creates a new CompactionEventListener with custom thresholds. +func NewCompactionEventListenerWithThresholds( + logger zerolog.Logger, + minSizeBytes int64, + minDuration time.Duration, + minFiles int, +) *CompactionEventListener { + return &CompactionEventListener{ + logger: logger.With().Str("module", "pebble_compaction").Logger(), + minSizeBytes: minSizeBytes, + minDuration: minDuration, + minFiles: minFiles, + } +} + +// calculateTotalSize calculates the total size of tables in a level. +func calculateTotalSize(level pebble.LevelInfo) int64 { + var totalSize int64 + for _, table := range level.Tables { + totalSize += int64(table.Size) + } + return totalSize +} + +// isSignificantCompaction checks if a compaction is significant enough to log +// based on size, file count, or duration (for end events). +func (e *CompactionEventListener) isSignificantCompaction( + totalInputSize int64, + totalInputFiles int, + duration time.Duration, +) bool { + // Log if size exceeds threshold + if totalInputSize >= e.minSizeBytes { + return true + } + // Log if file count exceeds threshold + if totalInputFiles >= e.minFiles { + return true + } + // Log if duration exceeds threshold (only available for end events) + if duration > 0 && duration >= e.minDuration { + return true + } + return false +} + +// CompactionBegin logs when a significant compaction starts. +func (e *CompactionEventListener) CompactionBegin(info pebble.CompactionInfo) { + var totalInputFiles int + var totalInputSize int64 + var inputLevels []int + + for _, level := range info.Input { + totalInputFiles += len(level.Tables) + totalInputSize += calculateTotalSize(level) + inputLevels = append(inputLevels, level.Level) + } + + // Only log if compaction is significant (size or file count threshold) + if !e.isSignificantCompaction(totalInputSize, totalInputFiles, 0) { + return + } + + e.logger.Debug(). + Int("job", info.JobID). + Str("reason", info.Reason). + Ints("input_levels", inputLevels). + Int("output_level", info.Output.Level). + Int("num_input_files", totalInputFiles). + Int64("input_size_bytes", totalInputSize). + Msg("compaction started") +} + +// CompactionEnd logs when a significant compaction ends. +func (e *CompactionEventListener) CompactionEnd(info pebble.CompactionInfo) { + var totalInputFiles int + var totalInputSize int64 + var inputLevels []int + + for _, level := range info.Input { + totalInputFiles += len(level.Tables) + totalInputSize += calculateTotalSize(level) + inputLevels = append(inputLevels, level.Level) + } + + // Only log if compaction is significant (size, file count, or duration threshold) + if !e.isSignificantCompaction(totalInputSize, totalInputFiles, info.Duration) { + return + } + + outputSize := calculateTotalSize(info.Output) + + e.logger.Info(). + Int("job", info.JobID). + Ints("input_levels", inputLevels). + Int("output_level", info.Output.Level). + Int("num_input_files", totalInputFiles). + Int("num_output_files", len(info.Output.Tables)). + Int64("input_size_bytes", totalInputSize). + Int64("output_size_bytes", outputSize). + Float64("duration_seconds", info.Duration.Seconds()). + Msg("compaction finished") +} diff --git a/storage/store/collections.go b/storage/store/collections.go index c07b2901d8e..47cb44270e6 100644 --- a/storage/store/collections.go +++ b/storage/store/collections.go @@ -109,6 +109,14 @@ func (c *Collections) LightByID(colID flow.Identifier) (*flow.LightCollection, e return &collection, nil } +// ExistByID checks whether a collection with the given ID exists in storage. +// Returns (true, nil) if it exists, +// Returns (false, nil) if it does not exist. +// No errors are expected during normal operation. +func (c *Collections) ExistByID(colID flow.Identifier) (bool, error) { + return operation.CollectionExists(c.db.Reader(), colID) +} + // Remove removes a collection from the database, including all constituent transactions and // indices inserted by Store. // Remove does not error if the collection does not exist @@ -179,11 +187,8 @@ func (c *Collections) BatchStoreAndIndexByTransaction(lctx lockctx.Proof, collec light := collection.Light() collectionID := light.ID() - err := operation.UpsertCollection(rw.Writer(), light) - if err != nil { - return nil, fmt.Errorf("could not insert collection: %w", err) - } - + // First, check if all transactions are already indexed and consistent + someTransactionIndexed := false for _, txID := range light.Transactions { var differentColTxIsIn flow.Identifier // The following is not BFT, because we can't handle the case where a transaction is included @@ -209,6 +214,20 @@ func (c *Collections) BatchStoreAndIndexByTransaction(lctx lockctx.Proof, collec if err != nil { return nil, fmt.Errorf("could not insert transaction ID: %w", err) } + someTransactionIndexed = true + } + + if !someTransactionIndexed { + // All transactions are already indexed and point to this collection. + // Since the index is always added along with the collection and transactions, + // this means the collection and its transactions have already been stored. + // Abort early to avoid redundant database writes. + return light, nil + } + + err := operation.UpsertCollection(rw.Writer(), light) + if err != nil { + return nil, fmt.Errorf("could not insert collection: %w", err) } // Store individual transactions diff --git a/storage/store/collections_test.go b/storage/store/collections_test.go index 8ab091c67b4..45576fe3199 100644 --- a/storage/store/collections_test.go +++ b/storage/store/collections_test.go @@ -187,3 +187,64 @@ func TestCollections_ConcurrentIndexByTx(t *testing.T) { assert.True(t, indexedCollection.ID() == col1.ID() || indexedCollection.ID() == col2.ID(), "Expected one of the collections to be indexed") }) } + +// TestCollections_BatchStoreAndIndexByTransaction_EarlyAbort verifies that +// BatchStoreAndIndexByTransaction aborts early when all transactions are already +// indexed and point to the same collection, avoiding redundant database writes. +func TestCollections_BatchStoreAndIndexByTransaction_EarlyAbort(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + lockManager := storage.NewTestingLockManager() + metrics := metrics.NewNoopCollector() + transactions := store.NewTransactions(metrics, db) + collections := store.NewCollections(db, transactions) + + // Create a collection with multiple transactions + collection := unittest.CollectionFixture(3) + expectedLight := collection.Light() + + // First, store the collection and index it by transaction + err := unittest.WithLock(t, lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + _, err := collections.BatchStoreAndIndexByTransaction(lctx, &collection, rw) + return err + }) + }) + require.NoError(t, err) + + // Verify the collection was stored + actualLight, err := collections.LightByID(collection.ID()) + require.NoError(t, err) + assert.Equal(t, expectedLight, actualLight) + + // Verify all transactions are indexed + for _, tx := range collection.Transactions { + collLight, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + assert.Equal(t, collection.ID(), collLight.ID()) + } + + // Try to store the same collection again - should abort early + err = unittest.WithLock(t, lockManager, storage.LockInsertCollection, func(lctx lockctx.Context) error { + return db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + light, err := collections.BatchStoreAndIndexByTransaction(lctx, &collection, rw) + require.NoError(t, err) + // Should return the light collection without error + assert.Equal(t, expectedLight, light) + return err + }) + }) + require.NoError(t, err) + + // Verify the collection still exists and is unchanged + actualLight, err = collections.LightByID(collection.ID()) + require.NoError(t, err) + assert.Equal(t, expectedLight, actualLight) + + // Verify all transactions are still indexed correctly + for _, tx := range collection.Transactions { + collLight, err := collections.LightByTransactionID(tx.ID()) + require.NoError(t, err) + assert.Equal(t, collection.ID(), collLight.ID()) + } + }) +}