diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 1da04c85888..c5c415f1357 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -45,6 +45,7 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/index" "github.com/onflow/flow-go/engine/access/ingestion" + "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" pingeng "github.com/onflow/flow-go/engine/access/ping" "github.com/onflow/flow-go/engine/access/rest" "github.com/onflow/flow-go/engine/access/rest/routes" @@ -56,6 +57,7 @@ import ( "github.com/onflow/flow-go/engine/access/subscription" 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" "github.com/onflow/flow-go/engine/common/version" @@ -163,7 +165,6 @@ type AccessNodeConfig struct { executionDataConfig edrequester.ExecutionDataConfig PublicNetworkConfig PublicNetworkConfig TxResultCacheSize uint - TxErrorMessagesCacheSize uint executionDataIndexingEnabled bool registersDBPath string checkpointFile string @@ -175,6 +176,7 @@ type AccessNodeConfig struct { programCacheSize uint checkPayerBalanceMode string versionControlEnabled bool + storeTxResultErrorMessages bool stopControlEnabled bool registerDBPruneThreshold uint64 } @@ -248,7 +250,6 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { apiRatelimits: nil, apiBurstlimits: nil, TxResultCacheSize: 0, - TxErrorMessagesCacheSize: 1000, PublicNetworkConfig: PublicNetworkConfig{ BindAddress: cmd.NotSet, Metrics: metrics.NewNoopCollector(), @@ -280,6 +281,7 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { programCacheSize: 0, checkPayerBalanceMode: accessNode.Disabled.String(), versionControlEnabled: true, + storeTxResultErrorMessages: false, stopControlEnabled: false, registerDBPruneThreshold: pruner.DefaultThreshold, } @@ -351,6 +353,9 @@ type FlowAccessNodeBuilder struct { stateStreamGrpcServer *grpcserver.GrpcServer stateStreamBackend *statestreambackend.StateStreamBackend + nodeBackend *backend.Backend + + TxResultErrorMessagesCore *tx_error_messages.TxErrorMessagesCore } func (builder *FlowAccessNodeBuilder) buildFollowerState() *FlowAccessNodeBuilder { @@ -1241,7 +1246,6 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.BoolVar(&builder.retryEnabled, "retry-enabled", defaultConfig.retryEnabled, "whether to enable the retry mechanism at the access node level") flags.BoolVar(&builder.rpcMetricsEnabled, "rpc-metrics-enabled", defaultConfig.rpcMetricsEnabled, "whether to enable the rpc metrics") flags.UintVar(&builder.TxResultCacheSize, "transaction-result-cache-size", defaultConfig.TxResultCacheSize, "transaction result cache size.(Disabled by default i.e 0)") - flags.UintVar(&builder.TxErrorMessagesCacheSize, "transaction-error-messages-cache-size", defaultConfig.TxErrorMessagesCacheSize, "transaction error messages cache size.(By default 1000)") flags.StringVarP(&builder.nodeInfoFile, "node-info-file", "", @@ -1375,7 +1379,10 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "tx-result-query-mode", defaultConfig.rpcConf.BackendConfig.TxResultQueryMode, "mode to use when querying transaction results. one of [local-only, execution-nodes-only(default), failover]") - + flags.BoolVar(&builder.storeTxResultErrorMessages, + "store-tx-result-error-messages", + defaultConfig.storeTxResultErrorMessages, + "whether to enable storing transaction error messages into the db") // Script Execution flags.StringVar(&builder.rpcConf.BackendConfig.ScriptExecutionMode, "script-execution-mode", @@ -1488,9 +1495,6 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { return errors.New("circuit-breaker-restore-timeout must be greater than 0") } } - if builder.TxErrorMessagesCacheSize == 0 { - return errors.New("transaction-error-messages-cache-size must be greater than 0") - } if builder.checkPayerBalanceMode != accessNode.Disabled.String() && !builder.executionDataIndexingEnabled { return errors.New("execution-data-indexing-enabled must be set if check-payer-balance is enabled") @@ -1603,7 +1607,8 @@ func (builder *FlowAccessNodeBuilder) enqueueRelayNetwork() { } func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { - var processedBlockHeight storage.ConsumerProgress + var processedFinalizedBlockHeight storage.ConsumerProgress + var processedTxErrorMessagesBlockHeight storage.ConsumerProgress if builder.executionDataSyncEnabled { builder.BuildExecutionSyncComponents() @@ -1799,8 +1804,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.Storage.LightTransactionResults) return nil }). - Module("processed block height consumer progress", func(node *cmd.NodeConfig) error { - processedBlockHeight = bstorage.NewConsumerProgress(builder.DB, module.ConsumeProgressIngestionEngineBlockHeight) + Module("processed finalized block height consumer progress", func(node *cmd.NodeConfig) error { + processedFinalizedBlockHeight = bstorage.NewConsumerProgress(builder.DB, module.ConsumeProgressIngestionEngineBlockHeight) return nil }). Module("processed last full block height monotonic consumer progress", func(node *cmd.NodeConfig) error { @@ -1817,6 +1822,13 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil }). + Module("transaction result error messages storage", func(node *cmd.NodeConfig) error { + if builder.storeTxResultErrorMessages { + builder.Storage.TransactionResultErrorMessages = bstorage.NewTransactionResultErrorMessages(node.Metrics.Cache, node.DB, bstorage.DefaultCacheSize) + } + + return nil + }). Component("version control", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { if !builder.versionControlEnabled { noop := &module.NoopReadyDoneAware{} @@ -1944,7 +1956,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { } - nodeBackend, err := backend.New(backend.Params{ + builder.nodeBackend, err = backend.New(backend.Params{ State: node.State, CollectionRPC: builder.CollectionRPC, HistoricalAccessNodes: builder.HistoricalAccessRPCs, @@ -1954,6 +1966,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { Transactions: node.Storage.Transactions, ExecutionReceipts: node.Storage.Receipts, ExecutionResults: node.Storage.Results, + TxResultErrorMessages: node.Storage.TransactionResultErrorMessages, ChainID: node.RootChainID, AccessMetrics: builder.AccessMetrics, ConnFactory: connFactory, @@ -1965,7 +1978,6 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, Communicator: backend.NewNodeCommunicator(backendConfig.CircuitBreakerConfig.Enabled), TxResultCacheSize: builder.TxResultCacheSize, - TxErrorMessagesCacheSize: builder.TxErrorMessagesCacheSize, ScriptExecutor: builder.ScriptExecutor, ScriptExecutionMode: scriptExecMode, CheckPayerBalanceMode: checkPayerBalanceMode, @@ -1997,8 +2009,8 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.AccessMetrics, builder.rpcMetricsEnabled, builder.Me, - nodeBackend, - nodeBackend, + builder.nodeBackend, + builder.nodeBackend, builder.secureGrpcServer, builder.unsecureGrpcServer, builder.stateStreamBackend, @@ -2037,6 +2049,28 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil, fmt.Errorf("could not create requester engine: %w", err) } + preferredENIdentifiers, err := commonrpc.IdentifierList(builder.rpcConf.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 := commonrpc.IdentifierList(builder.rpcConf.BackendConfig.FixedExecutionNodeIDs) + if err != nil { + return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) + } + + if builder.storeTxResultErrorMessages { + builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( + node.Logger, + node.State, + builder.nodeBackend, + node.Storage.Receipts, + node.Storage.TransactionResultErrorMessages, + preferredENIdentifiers, + fixedENIdentifiers, + ) + } + builder.IngestEng, err = ingestion.New( node.Logger, node.EngineRegistry, @@ -2050,8 +2084,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { node.Storage.Results, node.Storage.Receipts, builder.collectionExecutedMetric, - processedBlockHeight, + processedFinalizedBlockHeight, lastFullBlockHeight, + builder.TxResultErrorMessagesCore, ) if err != nil { return nil, err @@ -2069,6 +2104,31 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return builder.RequestEng, nil }) + if builder.storeTxResultErrorMessages { + builder.Module("processed error messages block height consumer progress", func(node *cmd.NodeConfig) error { + processedTxErrorMessagesBlockHeight = bstorage.NewConsumerProgress( + builder.DB, + module.ConsumeProgressEngineTxErrorMessagesBlockHeight, + ) + return nil + }) + builder.Component("transaction result error messages engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + engine, err := tx_error_messages.New( + node.Logger, + node.State, + node.Storage.Headers, + processedTxErrorMessagesBlockHeight, + builder.TxResultErrorMessagesCore, + ) + if err != nil { + return nil, err + } + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(engine.OnFinalizedBlock) + + return engine, nil + }) + } + if builder.supportsObserver { builder.Component("public sync request handler", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { syncRequestHandler, err := synceng.NewRequestHandlerEngine( diff --git a/engine/access/access_test.go b/engine/access/access_test.go index d7ea12e613c..0ee61785dce 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -329,16 +329,15 @@ func (suite *Suite) TestSendTransactionToRandomCollectionNode() { connFactory.On("GetAccessAPIClient", collNode2.Address, nil).Return(col2ApiClient, &mockCloser{}, nil) bnd, err := backend.New(backend.Params{State: suite.state, - Collections: collections, - Transactions: transactions, - ChainID: suite.chainID, - AccessMetrics: metrics, - ConnFactory: connFactory, - MaxHeightRange: backend.DefaultMaxHeightRange, - Log: suite.log, - SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, - Communicator: backend.NewNodeCommunicator(false), - TxErrorMessagesCacheSize: 1000, + Collections: collections, + Transactions: transactions, + ChainID: suite.chainID, + AccessMetrics: metrics, + ConnFactory: connFactory, + MaxHeightRange: backend.DefaultMaxHeightRange, + Log: suite.log, + SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, + Communicator: backend.NewNodeCommunicator(false), }) require.NoError(suite.T(), err) @@ -659,7 +658,6 @@ func (suite *Suite) TestGetSealedTransaction() { Log: suite.log, SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, Communicator: backend.NewNodeCommunicator(false), - TxErrorMessagesCacheSize: 1000, TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, }) require.NoError(suite.T(), err) @@ -686,7 +684,23 @@ func (suite *Suite) TestGetSealedTransaction() { // create the ingest engine processedHeight := bstorage.NewConsumerProgress(db, module.ConsumeProgressIngestionEngineBlockHeight) - ingestEng, err := ingestion.New(suite.log, suite.net, suite.state, suite.me, suite.request, all.Blocks, all.Headers, collections, transactions, results, receipts, collectionExecutedMetric, processedHeight, lastFullBlockHeight) + ingestEng, err := ingestion.New( + suite.log, + suite.net, + suite.state, + suite.me, + suite.request, + all.Blocks, + all.Headers, + collections, + transactions, + results, + receipts, + collectionExecutedMetric, + processedHeight, + lastFullBlockHeight, + nil, + ) require.NoError(suite.T(), err) // 1. Assume that follower engine updated the block storage and the protocol state. The block is reported as sealed @@ -743,7 +757,6 @@ func (suite *Suite) TestGetTransactionResult() { all := util.StorageLayer(suite.T(), db) results := bstorage.NewExecutionResults(suite.metrics, db) receipts := bstorage.NewExecutionReceipts(suite.metrics, db, results, bstorage.DefaultCacheSize) - originID := unittest.IdentifierFixture() *suite.state = protocol.State{} @@ -778,6 +791,9 @@ func (suite *Suite) TestGetTransactionResult() { allIdentities := append(colIdentities, enIdentities...) finalSnapshot.On("Identities", mock.Anything).Return(allIdentities, nil) + suite.state.On("AtBlockID", blockNegativeId).Return(suite.sealedSnapshot) + suite.sealedSnapshot.On("Identities", mock.Anything).Return(allIdentities, nil) + // assume execution node returns an empty list of events suite.execClient.On("GetTransactionResult", mock.Anything, mock.Anything).Return(&execproto.GetTransactionResultResponse{ Events: nil, @@ -821,7 +837,6 @@ func (suite *Suite) TestGetTransactionResult() { Log: suite.log, SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, Communicator: backend.NewNodeCommunicator(false), - TxErrorMessagesCacheSize: 1000, TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, }) require.NoError(suite.T(), err) @@ -840,6 +855,7 @@ func (suite *Suite) TestGetTransactionResult() { require.NoError(suite.T(), err) processedHeight := bstorage.NewConsumerProgress(db, module.ConsumeProgressIngestionEngineBlockHeight) + lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter( bstorage.NewConsumerProgress(db, module.ConsumeProgressLastFullBlockHeight), suite.rootBlock.Height, @@ -847,7 +863,23 @@ func (suite *Suite) TestGetTransactionResult() { require.NoError(suite.T(), err) // create the ingest engine - ingestEng, err := ingestion.New(suite.log, suite.net, suite.state, suite.me, suite.request, all.Blocks, all.Headers, collections, transactions, results, receipts, collectionExecutedMetric, processedHeight, lastFullBlockHeight) + ingestEng, err := ingestion.New( + suite.log, + suite.net, + suite.state, + suite.me, + suite.request, + all.Blocks, + all.Headers, + collections, + transactions, + results, + receipts, + collectionExecutedMetric, + processedHeight, + lastFullBlockHeight, + nil, + ) require.NoError(suite.T(), err) background, cancel := context.WithCancel(context.Background()) @@ -943,6 +975,7 @@ func (suite *Suite) TestGetTransactionResult() { } resp, err := handler.GetTransactionResult(context.Background(), getReq) require.Error(suite.T(), err) + require.Contains(suite.T(), err.Error(), "failed to find: transaction not in block") require.Nil(suite.T(), resp) }) @@ -1009,7 +1042,6 @@ func (suite *Suite) TestExecuteScript() { collections := bstorage.NewCollections(db, transactions) results := bstorage.NewExecutionResults(suite.metrics, db) receipts := bstorage.NewExecutionReceipts(suite.metrics, db, results, bstorage.DefaultCacheSize) - identities := unittest.IdentityListFixture(2, unittest.WithRole(flow.RoleExecution)) suite.sealedSnapshot.On("Identities", mock.Anything).Return(identities, nil) suite.finalSnapshot.On("Identities", mock.Anything).Return(identities, nil) @@ -1020,25 +1052,24 @@ func (suite *Suite) TestExecuteScript() { var err error suite.backend, err = backend.New(backend.Params{ - State: suite.state, - CollectionRPC: suite.collClient, - Blocks: all.Blocks, - Headers: all.Headers, - Collections: collections, - Transactions: transactions, - ExecutionReceipts: receipts, - ExecutionResults: results, - ChainID: suite.chainID, - AccessMetrics: suite.metrics, - ConnFactory: connFactory, - MaxHeightRange: backend.DefaultMaxHeightRange, - FixedExecutionNodeIDs: (identities.NodeIDs()).Strings(), - Log: suite.log, - SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, - Communicator: backend.NewNodeCommunicator(false), - ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly, - TxErrorMessagesCacheSize: 1000, - TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, + State: suite.state, + CollectionRPC: suite.collClient, + Blocks: all.Blocks, + Headers: all.Headers, + Collections: collections, + Transactions: transactions, + ExecutionReceipts: receipts, + ExecutionResults: results, + ChainID: suite.chainID, + AccessMetrics: suite.metrics, + ConnFactory: connFactory, + MaxHeightRange: backend.DefaultMaxHeightRange, + FixedExecutionNodeIDs: (identities.NodeIDs()).Strings(), + Log: suite.log, + SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, + Communicator: backend.NewNodeCommunicator(false), + ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly, + TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, }) require.NoError(suite.T(), err) @@ -1069,6 +1100,7 @@ func (suite *Suite) TestExecuteScript() { Once() processedHeight := bstorage.NewConsumerProgress(db, module.ConsumeProgressIngestionEngineBlockHeight) + lastFullBlockHeight, err := counters.NewPersistentStrictMonotonicCounter( bstorage.NewConsumerProgress(db, module.ConsumeProgressLastFullBlockHeight), suite.rootBlock.Height, @@ -1076,7 +1108,23 @@ func (suite *Suite) TestExecuteScript() { require.NoError(suite.T(), err) // create the ingest engine - ingestEng, err := ingestion.New(suite.log, suite.net, suite.state, suite.me, suite.request, all.Blocks, all.Headers, collections, transactions, results, receipts, collectionExecutedMetric, processedHeight, lastFullBlockHeight) + ingestEng, err := ingestion.New( + suite.log, + suite.net, + suite.state, + suite.me, + suite.request, + all.Blocks, + all.Headers, + collections, + transactions, + results, + receipts, + collectionExecutedMetric, + processedHeight, + lastFullBlockHeight, + nil, + ) require.NoError(suite.T(), err) // create another block as a predecessor of the block created earlier diff --git a/engine/access/ingestion/engine.go b/engine/access/ingestion/engine.go index b36e5598c59..0e4ac42367e 100644 --- a/engine/access/ingestion/engine.go +++ b/engine/access/ingestion/engine.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/access/ingestion/tx_error_messages" "github.com/onflow/flow-go/engine/common/fifoqueue" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" @@ -50,8 +51,9 @@ const ( // default queue capacity defaultQueueCapacity = 10_000 - // how many workers will concurrently process the tasks in the jobqueue - workersCount = 1 + // processFinalizedBlocksWorkersCount defines the number of workers that + // concurrently process finalized blocks in the job queue. + processFinalizedBlocksWorkersCount = 1 // ensure blocks are processed sequentially by jobqueue searchAhead = 1 @@ -77,10 +79,12 @@ type Engine struct { executionReceiptsQueue engine.MessageStore // Job queue finalizedBlockConsumer *jobqueue.ComponentConsumer - // Notifier for queue consumer finalizedBlockNotifier engine.Notifier + // txResultErrorMessagesChan is used to fetch and store transaction result error messages for blocks + txResultErrorMessagesChan chan flow.Identifier + log zerolog.Logger // used to log relevant actions with context state protocol.State // used to access the protocol state me module.Local // used to access local node information @@ -99,6 +103,8 @@ type Engine struct { lastFullBlockHeight *counters.PersistentStrictMonotonicCounter // metrics collectionExecutedMetric module.CollectionExecutedMetric + + txErrorMessagesCore *tx_error_messages.TxErrorMessagesCore } var _ network.MessageProcessor = (*Engine)(nil) @@ -119,8 +125,9 @@ func New( executionResults storage.ExecutionResults, executionReceipts storage.ExecutionReceipts, collectionExecutedMetric module.CollectionExecutedMetric, - processedHeight storage.ConsumerProgress, + finalizedProcessedHeight storage.ConsumerProgress, lastFullBlockHeight *counters.PersistentStrictMonotonicCounter, + txErrorMessagesCore *tx_error_messages.TxErrorMessagesCore, ) (*Engine, error) { executionReceiptsRawQueue, err := fifoqueue.NewFifoQueue(defaultQueueCapacity) if err != nil { @@ -162,8 +169,10 @@ func New( // queue / notifier for execution receipts executionReceiptsNotifier: engine.NewNotifier(), + txResultErrorMessagesChan: make(chan flow.Identifier, 1), executionReceiptsQueue: executionReceiptsQueue, messageHandler: messageHandler, + txErrorMessagesCore: txErrorMessagesCore, } // jobqueue Jobs object that tracks finalized blocks by height. This is used by the finalizedBlockConsumer @@ -172,7 +181,7 @@ func New( defaultIndex, err := e.defaultProcessedIndex() if err != nil { - return nil, fmt.Errorf("could not read default processed index: %w", err) + return nil, fmt.Errorf("could not read default finalized processed index: %w", err) } // create a jobqueue that will process new available finalized block. The `finalizedBlockNotifier` is used to @@ -180,11 +189,11 @@ func New( e.finalizedBlockConsumer, err = jobqueue.NewComponentConsumer( e.log.With().Str("module", "ingestion_block_consumer").Logger(), e.finalizedBlockNotifier.Channel(), - processedHeight, + finalizedProcessedHeight, finalizedBlockReader, defaultIndex, e.processFinalizedBlockJob, - workersCount, + processFinalizedBlocksWorkersCount, searchAhead, ) if err != nil { @@ -192,11 +201,20 @@ func New( } // Add workers - e.ComponentManager = component.NewComponentManagerBuilder(). + builder := component.NewComponentManagerBuilder(). AddWorker(e.processBackground). AddWorker(e.processExecutionReceipts). - AddWorker(e.runFinalizedBlockConsumer). - Build() + AddWorker(e.runFinalizedBlockConsumer) + + // If txErrorMessagesCore is provided, add a worker responsible for processing + // transaction result error messages by receipts. This worker listens for blocks + // containing execution receipts and processes any associated transaction result + // error messages. The worker is added only when error message processing is enabled. + if txErrorMessagesCore != nil { + builder.AddWorker(e.processTransactionResultErrorMessagesByReceipts) + } + + e.ComponentManager = builder.Build() // register engine with the execution receipt provider _, err = net.Register(channels.ReceiveReceipts, e) @@ -209,7 +227,7 @@ func New( // defaultProcessedIndex returns the last finalized block height from the protocol state. // -// The BlockConsumer utilizes this return height to fetch and consume block jobs from +// The finalizedBlockConsumer utilizes this return height to fetch and consume block jobs from // jobs queue the first time it initializes. // // No errors are expected during normal operation. @@ -337,6 +355,42 @@ func (e *Engine) processAvailableExecutionReceipts(ctx context.Context) error { if err := e.handleExecutionReceipt(msg.OriginID, receipt); err != nil { return err } + + // Notify to fetch and store transaction result error messages for the block. + // If txErrorMessagesCore is enabled, the receipt's BlockID is sent to trigger + // transaction error message processing. This step is skipped if error message + // storage is not enabled. + if e.txErrorMessagesCore != nil { + e.txResultErrorMessagesChan <- receipt.BlockID + } + } +} + +// processTransactionResultErrorMessagesByReceipts handles error messages related to transaction +// results by reading from the error messages channel and processing them accordingly. +// +// This function listens for messages on the txResultErrorMessagesChan channel and +// processes each transaction result error message as it arrives. +// +// No errors are expected during normal operation. +func (e *Engine) processTransactionResultErrorMessagesByReceipts(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + + for { + select { + case <-ctx.Done(): + return + case blockID := <-e.txResultErrorMessagesChan: + err := e.txErrorMessagesCore.HandleTransactionResultErrorMessages(ctx, blockID) + if err != nil { + // TODO: we should revisit error handling here. + // Errors that come from querying the EN and possibly ExecutionNodesForBlockID should be logged and + // retried later, while others should cause an exception. + e.log.Error(). + Err(err). + Msg("error encountered while processing transaction result error messages by receipts") + } + } } } diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index d93056bd80c..dd6404390fc 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -47,18 +47,19 @@ type Suite struct { params *protocol.Params } - me *modulemock.Local - net *mocknetwork.Network - request *modulemock.Requester - obsIdentity *flow.Identity - provider *mocknetwork.Engine - blocks *storage.Blocks - headers *storage.Headers - collections *storage.Collections - transactions *storage.Transactions - receipts *storage.ExecutionReceipts - results *storage.ExecutionResults - seals *storage.Seals + me *modulemock.Local + net *mocknetwork.Network + request *modulemock.Requester + obsIdentity *flow.Identity + provider *mocknetwork.Engine + blocks *storage.Blocks + headers *storage.Headers + collections *storage.Collections + transactions *storage.Transactions + receipts *storage.ExecutionReceipts + results *storage.ExecutionResults + seals *storage.Seals + conduit *mocknetwork.Conduit downloader *downloadermock.Downloader sealedBlock *flow.Header @@ -102,7 +103,6 @@ func (s *Suite) SetupTest() { 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("Final").Return(s.proto.snapshot, nil) s.proto.state.On("Params").Return(s.proto.params) s.proto.snapshot.On("Head").Return( func() *flow.Header { @@ -119,7 +119,6 @@ func (s *Suite) SetupTest() { Return(conduit, nil). Once() s.request = modulemock.NewRequester(s.T()) - s.provider = mocknetwork.NewEngine(s.T()) s.blocks = storage.NewBlocks(s.T()) s.headers = storage.NewHeaders(s.T()) @@ -166,6 +165,10 @@ func (s *Suite) SetupTest() { ).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) + s.collectionExecutedMetric, err = indexer.NewCollectionExecutedMetricImpl( s.log, metrics.NewNoopCollector(), @@ -189,7 +192,24 @@ func (s *Suite) initIngestionEngine(ctx irrecoverable.SignalerContext) *Engine { ) require.NoError(s.T(), err) - eng, err := New(s.log, s.net, s.proto.state, s.me, s.request, s.blocks, s.headers, s.collections, s.transactions, s.results, s.receipts, s.collectionExecutedMetric, processedHeight, s.lastFullBlockHeight) + eng, err := New( + s.log, + s.net, + s.proto.state, + s.me, + s.request, + s.blocks, + s.headers, + s.collections, + s.transactions, + s.results, + s.receipts, + s.collectionExecutedMetric, + processedHeight, + s.lastFullBlockHeight, + nil, + ) + require.NoError(s.T(), err) eng.ComponentManager.Start(ctx) diff --git a/engine/access/ingestion/tx_error_messages/tx_error_messages_core.go b/engine/access/ingestion/tx_error_messages/tx_error_messages_core.go new file mode 100644 index 00000000000..88f3b93ec5d --- /dev/null +++ b/engine/access/ingestion/tx_error_messages/tx_error_messages_core.go @@ -0,0 +1,144 @@ +package tx_error_messages + +import ( + "context" + "fmt" + + execproto "github.com/onflow/flow/protobuf/go/flow/execution" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine/access/rpc/backend" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" + "github.com/onflow/flow-go/engine/common/rpc/convert" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// TxErrorMessagesCore is responsible for managing transaction result error messages +// It handles both storage and retrieval of error messages +// from execution nodes. +type TxErrorMessagesCore struct { + log zerolog.Logger // used to log relevant actions with context + state protocol.State // used to access the protocol state + + backend *backend.Backend + + executionReceipts storage.ExecutionReceipts + transactionResultErrorMessages storage.TransactionResultErrorMessages + + preferredExecutionNodeIDs flow.IdentifierList + fixedExecutionNodeIDs flow.IdentifierList +} + +// NewTxErrorMessagesCore creates a new instance of TxErrorMessagesCore. +func NewTxErrorMessagesCore( + log zerolog.Logger, + state protocol.State, + backend *backend.Backend, + executionReceipts storage.ExecutionReceipts, + transactionResultErrorMessages storage.TransactionResultErrorMessages, + preferredExecutionNodeIDs flow.IdentifierList, + fixedExecutionNodeIDs flow.IdentifierList, +) *TxErrorMessagesCore { + return &TxErrorMessagesCore{ + log: log.With().Str("module", "tx_error_messages_core").Logger(), + state: state, + backend: backend, + executionReceipts: executionReceipts, + transactionResultErrorMessages: transactionResultErrorMessages, + preferredExecutionNodeIDs: preferredExecutionNodeIDs, + fixedExecutionNodeIDs: fixedExecutionNodeIDs, + } +} + +// HandleTransactionResultErrorMessages processes transaction result error messages for a given block ID. +// It retrieves error messages from the backend if they do not already exist in storage. +// +// The function first checks if error messages for the given block ID are already present in storage. +// If they are not, it fetches the messages from execution nodes and stores them. +// +// Parameters: +// - ctx: The context for managing cancellation and deadlines during the operation. +// - blockID: The identifier of the block for which transaction result error messages need to be processed. +// +// No errors are expected during normal operation. +func (c *TxErrorMessagesCore) HandleTransactionResultErrorMessages(ctx context.Context, blockID flow.Identifier) error { + exists, err := c.transactionResultErrorMessages.Exists(blockID) + if err != nil { + return fmt.Errorf("could not check existance of transaction result error messages: %w", err) + } + + if exists { + return nil + } + + // retrieves error messages from the backend if they do not already exist in storage + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + c.executionReceipts, + c.state, + c.log, + c.preferredExecutionNodeIDs, + c.fixedExecutionNodeIDs, + ) + if err != nil { + c.log.Error().Err(err).Msg(fmt.Sprintf("failed to find execution nodes for block id: %s", blockID)) + return fmt.Errorf("could not find execution nodes for block: %w", err) + } + + req := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: convert.IdentifierToMessage(blockID), + } + + c.log.Debug(). + Msgf("transaction error messages for block %s are being downloaded", blockID) + + resp, execNode, err := c.backend.GetTransactionErrorMessagesFromAnyEN(ctx, execNodes, req) + if err != nil { + c.log.Error().Err(err).Msg("failed to get transaction error messages from execution nodes") + return err + } + + if len(resp) > 0 { + err = c.storeTransactionResultErrorMessages(blockID, resp, execNode) + if err != nil { + return fmt.Errorf("could not store error messages (block: %s): %w", blockID, err) + } + } + + return nil +} + +// storeTransactionResultErrorMessages stores the transaction result error messages for a given block ID. +// +// Parameters: +// - blockID: The identifier of the block for which the error messages are to be stored. +// - errorMessagesResponses: A slice of responses containing the error messages to be stored. +// - execNode: The execution node associated with the error messages. +// +// No errors are expected during normal operation. +func (c *TxErrorMessagesCore) storeTransactionResultErrorMessages( + blockID flow.Identifier, + errorMessagesResponses []*execproto.GetTransactionErrorMessagesResponse_Result, + execNode *flow.IdentitySkeleton, +) error { + errorMessages := make([]flow.TransactionResultErrorMessage, 0, len(errorMessagesResponses)) + for _, value := range errorMessagesResponses { + errorMessage := flow.TransactionResultErrorMessage{ + ErrorMessage: value.ErrorMessage, + TransactionID: convert.MessageToIdentifier(value.TransactionId), + Index: value.Index, + ExecutorID: execNode.NodeID, + } + errorMessages = append(errorMessages, errorMessage) + } + + err := c.transactionResultErrorMessages.Store(blockID, errorMessages) + if err != nil { + return fmt.Errorf("failed to store transaction error messages: %w", err) + } + + return nil +} diff --git a/engine/access/ingestion/tx_error_messages/tx_error_messages_core_test.go b/engine/access/ingestion/tx_error_messages/tx_error_messages_core_test.go new file mode 100644 index 00000000000..9d5d6466d97 --- /dev/null +++ b/engine/access/ingestion/tx_error_messages/tx_error_messages_core_test.go @@ -0,0 +1,332 @@ +package tx_error_messages + +import ( + "context" + "fmt" + "os" + "testing" + + execproto "github.com/onflow/flow/protobuf/go/flow/execution" + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + accessmock "github.com/onflow/flow-go/engine/access/mock" + "github.com/onflow/flow-go/engine/access/rpc/backend" + connectionmock "github.com/onflow/flow-go/engine/access/rpc/connection/mock" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" + protocol "github.com/onflow/flow-go/state/protocol/mock" + storage "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/utils/unittest" +) + +const expectedErrorMsg = "expected test error" + +type TxErrorMessagesCoreSuite struct { + suite.Suite + + log zerolog.Logger + proto struct { + state *protocol.FollowerState + snapshot *protocol.Snapshot + params *protocol.Params + } + + receipts *storage.ExecutionReceipts + txErrorMessages *storage.TransactionResultErrorMessages + + enNodeIDs flow.IdentityList + execClient *accessmock.ExecutionAPIClient + connFactory *connectionmock.ConnectionFactory + + blockMap map[uint64]*flow.Block + rootBlock flow.Block + finalizedBlock *flow.Header + + ctx context.Context + cancel context.CancelFunc +} + +func TestTxErrorMessagesCore(t *testing.T) { + suite.Run(t, new(TxErrorMessagesCoreSuite)) +} + +// TearDownTest stops the engine and cleans up the db +func (s *TxErrorMessagesCoreSuite) TearDownTest() { + s.cancel() +} + +type mockCloser struct{} + +func (mc *mockCloser) Close() error { return nil } + +func (s *TxErrorMessagesCoreSuite) SetupTest() { + s.log = zerolog.New(os.Stderr) + s.ctx, s.cancel = context.WithCancel(context.Background()) + // mock out protocol state + s.proto.state = protocol.NewFollowerState(s.T()) + s.proto.snapshot = protocol.NewSnapshot(s.T()) + s.proto.params = protocol.NewParams(s.T()) + s.execClient = accessmock.NewExecutionAPIClient(s.T()) + s.connFactory = connectionmock.NewConnectionFactory(s.T()) + s.receipts = storage.NewExecutionReceipts(s.T()) + s.txErrorMessages = storage.NewTransactionResultErrorMessages(s.T()) + + s.rootBlock = unittest.BlockFixture() + s.rootBlock.Header.Height = 0 + s.finalizedBlock = unittest.BlockWithParentFixture(s.rootBlock.Header).Header + + s.proto.state.On("Params").Return(s.proto.params) + + // Mock the finalized root block header with height 0. + s.proto.params.On("FinalizedRoot").Return(s.rootBlock.Header, nil) + + s.proto.snapshot.On("Head").Return( + func() *flow.Header { + return s.finalizedBlock + }, + nil, + ).Maybe() + s.proto.state.On("Final").Return(s.proto.snapshot, nil) + + // Create identities for 1 execution nodes. + s.enNodeIDs = unittest.IdentityListFixture(1, unittest.WithRole(flow.RoleExecution)) +} + +// TestHandleTransactionResultErrorMessages checks that transaction result error messages +// are properly fetched from the execution nodes, processed, and stored in the protocol database. +func (s *TxErrorMessagesCoreSuite) TestHandleTransactionResultErrorMessages() { + irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) + + block := unittest.BlockWithParentFixture(s.finalizedBlock) + blockId := block.ID() + + s.connFactory.On("GetExecutionAPIClient", mock.Anything).Return(s.execClient, &mockCloser{}, nil) + + // Mock the protocol snapshot to return fixed execution node IDs. + setupReceiptsForBlock(s.receipts, block, s.enNodeIDs.NodeIDs()[0]) + s.proto.snapshot.On("Identities", mock.Anything).Return(s.enNodeIDs, nil) + s.proto.state.On("AtBlockID", blockId).Return(s.proto.snapshot).Once() + + // Create mock transaction results with a mix of failed and non-failed transactions. + resultsByBlockID := mockTransactionResultsByBlock(5) + + // Prepare a request to fetch transaction error messages by block ID from execution nodes. + exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: blockId[:], + } + + s.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). + Return(createTransactionErrorMessagesResponse(resultsByBlockID), nil). + Once() + + // Mock the txErrorMessages storage to confirm that error messages do not exist yet. + s.txErrorMessages.On("Exists", blockId). + Return(false, nil).Once() + + // Prepare the expected transaction error messages that should be stored. + expectedStoreTxErrorMessages := createExpectedTxErrorMessages(resultsByBlockID, s.enNodeIDs.NodeIDs()[0]) + + // Mock the storage of the fetched error messages into the protocol database. + s.txErrorMessages.On("Store", blockId, expectedStoreTxErrorMessages). + Return(nil).Once() + + core := s.initCore() + err := core.HandleTransactionResultErrorMessages(irrecoverableCtx, blockId) + require.NoError(s.T(), err) + + // Verify that the mock expectations for storing the error messages were met. + s.txErrorMessages.AssertExpectations(s.T()) + + // Now simulate the second try when the error messages already exist in storage. + // Mock the txErrorMessages storage to confirm that error messages exist. + s.txErrorMessages.On("Exists", blockId). + Return(true, nil).Once() + err = core.HandleTransactionResultErrorMessages(irrecoverableCtx, blockId) + require.NoError(s.T(), err) + + // Verify that the mock expectations for storing the error messages were not met. + s.txErrorMessages.AssertExpectations(s.T()) + s.execClient.AssertExpectations(s.T()) +} + +// TestHandleTransactionResultErrorMessages_ErrorCases tests the error handling of +// the HandleTransactionResultErrorMessages function in the following cases: +// +// 1. Execution node fetch error: When fetching transaction error messages from the execution node fails, +// the function should return an appropriate error and no further actions should be taken. +// 2. Storage store error after fetching results: When fetching transaction error messages succeeds, +// but storing them in the storage fails, the function should return an error and no further actions should be taken. +func (s *TxErrorMessagesCoreSuite) TestHandleTransactionResultErrorMessages_ErrorCases() { + irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) + + block := unittest.BlockWithParentFixture(s.finalizedBlock) + blockId := block.ID() + + s.connFactory.On("GetExecutionAPIClient", mock.Anything).Return(s.execClient, &mockCloser{}, nil) + + // Mock the protocol snapshot to return fixed execution node IDs. + setupReceiptsForBlock(s.receipts, block, s.enNodeIDs.NodeIDs()[0]) + s.proto.snapshot.On("Identities", mock.Anything).Return(s.enNodeIDs, nil) + s.proto.state.On("AtBlockID", blockId).Return(s.proto.snapshot) + + s.Run("Execution node fetch error", func() { + // Mock the txErrorMessages storage to confirm that error messages do not exist yet. + s.txErrorMessages.On("Exists", blockId).Return(false, nil).Once() + + // Simulate an error when fetching transaction error messages from the execution node. + exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: blockId[:], + } + s.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). + Return(nil, fmt.Errorf("execution node fetch error")).Once() + + core := s.initCore() + err := core.HandleTransactionResultErrorMessages(irrecoverableCtx, blockId) + + // Assert that the function returns an error due to the client fetch error. + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), "execution node fetch error") + + // Ensure that no further steps are taken after the client fetch error. + s.txErrorMessages.AssertNotCalled(s.T(), "Store", mock.Anything, mock.Anything) + }) + + s.Run("Storage error after fetching results", func() { + // Simulate successful fetching of transaction error messages but error in storing them. + + // Mock the txErrorMessages storage to confirm that error messages do not exist yet. + s.txErrorMessages.On("Exists", blockId).Return(false, nil).Once() + + // Create mock transaction results with a mix of failed and non-failed transactions. + resultsByBlockID := mockTransactionResultsByBlock(5) + + // Prepare a request to fetch transaction error messages by block ID from execution nodes. + exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: blockId[:], + } + s.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). + Return(createTransactionErrorMessagesResponse(resultsByBlockID), nil).Once() + + // Simulate an error when attempting to store the fetched transaction error messages in storage. + expectedStoreTxErrorMessages := createExpectedTxErrorMessages(resultsByBlockID, s.enNodeIDs.NodeIDs()[0]) + s.txErrorMessages.On("Store", blockId, expectedStoreTxErrorMessages). + Return(fmt.Errorf("storage error")).Once() + + core := s.initCore() + err := core.HandleTransactionResultErrorMessages(irrecoverableCtx, blockId) + + // Assert that the function returns an error due to the store error. + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), "storage error") + + // Ensure that storage existence check and transaction fetch were called before the store error. + s.txErrorMessages.AssertCalled(s.T(), "Exists", blockId) + s.execClient.AssertCalled(s.T(), "GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq) + }) +} + +// initCore create new instance of transaction error messages core. +func (s *TxErrorMessagesCoreSuite) initCore() *TxErrorMessagesCore { + // Initialize the backend + backend, err := backend.New(backend.Params{ + State: s.proto.state, + ExecutionReceipts: s.receipts, + ConnFactory: s.connFactory, + MaxHeightRange: backend.DefaultMaxHeightRange, + FixedExecutionNodeIDs: s.enNodeIDs.NodeIDs().Strings(), + Log: s.log, + SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, + Communicator: backend.NewNodeCommunicator(false), + ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly, + TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, + ChainID: flow.Testnet, + }) + require.NoError(s.T(), err) + + core := NewTxErrorMessagesCore( + s.log, + s.proto.state, + backend, + s.receipts, + s.txErrorMessages, + s.enNodeIDs.NodeIDs(), + nil, + ) + return core +} + +// createExpectedTxErrorMessages creates a list of expected transaction error messages based on transaction results +func createExpectedTxErrorMessages(resultsByBlockID []flow.LightTransactionResult, executionNode flow.Identifier) []flow.TransactionResultErrorMessage { + // Prepare the expected transaction error messages that should be stored. + var expectedStoreTxErrorMessages []flow.TransactionResultErrorMessage + + for i, result := range resultsByBlockID { + if result.Failed { + errMsg := fmt.Sprintf("%s.%s", expectedErrorMsg, result.TransactionID) + + expectedStoreTxErrorMessages = append(expectedStoreTxErrorMessages, + flow.TransactionResultErrorMessage{ + TransactionID: result.TransactionID, + ErrorMessage: errMsg, + Index: uint32(i), + ExecutorID: executionNode, + }) + } + } + + return expectedStoreTxErrorMessages +} + +// mockTransactionResultsByBlock create mock transaction results with a mix of failed and non-failed transactions. +func mockTransactionResultsByBlock(count int) []flow.LightTransactionResult { + // Create mock transaction results with a mix of failed and non-failed transactions. + resultsByBlockID := make([]flow.LightTransactionResult, 0) + for i := 0; i < count; i++ { + resultsByBlockID = append(resultsByBlockID, flow.LightTransactionResult{ + TransactionID: unittest.IdentifierFixture(), + Failed: i%2 == 0, // create a mix of failed and non-failed transactions + ComputationUsed: 0, + }) + } + + return resultsByBlockID +} + +// setupReceiptsForBlock sets up mock execution receipts for a block and returns the receipts along +// with the identities of the execution nodes that processed them. +func setupReceiptsForBlock(receipts *storage.ExecutionReceipts, block *flow.Block, eNodeID flow.Identifier) { + receipt1 := unittest.ReceiptForBlockFixture(block) + receipt1.ExecutorID = eNodeID + receipt2 := unittest.ReceiptForBlockFixture(block) + receipt2.ExecutorID = eNodeID + receipt1.ExecutionResult = receipt2.ExecutionResult + + receiptsList := flow.ExecutionReceiptList{receipt1, receipt2} + + receipts. + On("ByBlockID", block.ID()). + Return(func(flow.Identifier) flow.ExecutionReceiptList { + return receiptsList + }, nil) +} + +// createTransactionErrorMessagesResponse create TransactionErrorMessagesResponse from execution node based on results. +func createTransactionErrorMessagesResponse(resultsByBlockID []flow.LightTransactionResult) *execproto.GetTransactionErrorMessagesResponse { + exeErrMessagesResp := &execproto.GetTransactionErrorMessagesResponse{} + + for i, result := range resultsByBlockID { + if result.Failed { + errMsg := fmt.Sprintf("%s.%s", expectedErrorMsg, result.TransactionID) + exeErrMessagesResp.Results = append(exeErrMessagesResp.Results, &execproto.GetTransactionErrorMessagesResponse_Result{ + TransactionId: result.TransactionID[:], + ErrorMessage: errMsg, + Index: uint32(i), + }) + } + } + + return exeErrMessagesResp +} diff --git a/engine/access/ingestion/tx_error_messages/tx_error_messages_engine.go b/engine/access/ingestion/tx_error_messages/tx_error_messages_engine.go new file mode 100644 index 00000000000..cdd65bdc0b3 --- /dev/null +++ b/engine/access/ingestion/tx_error_messages/tx_error_messages_engine.go @@ -0,0 +1,181 @@ +package tx_error_messages + +import ( + "context" + "fmt" + "time" + + "github.com/rs/zerolog" + "github.com/sethvargo/go-retry" + + "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/module/jobqueue" + "github.com/onflow/flow-go/module/util" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +const ( + // processTxErrorMessagesWorkersCount defines the number of workers that + // concurrently process transaction error messages in the job queue. + processTxErrorMessagesWorkersCount = 3 + + // defaultRetryDelay specifies the initial delay for the exponential backoff + // when the process of fetching transaction error messages fails. + // + // This delay increases with each retry attempt, up to the maximum defined by + // defaultMaxRetryDelay. + defaultRetryDelay = 1 * time.Second + + // defaultMaxRetryDelay specifies the maximum delay for the exponential backoff + // when the process of fetching transaction error messages fails. + // + // Once this delay is reached, the backoff will no longer increase with each retry. + defaultMaxRetryDelay = 5 * time.Minute +) + +// Engine represents the component responsible for managing and processing +// transaction result error messages. It retrieves, stores, +// and retries fetching of error messages from execution nodes, ensuring +// that they are processed and stored for sealed blocks. +// +// No errors are expected during normal operation. +type Engine struct { + *component.ComponentManager + + log zerolog.Logger + state protocol.State + headers storage.Headers + + // Job queue + txErrorMessagesConsumer *jobqueue.ComponentConsumer + // Notifiers for queue consumer + txErrorMessagesNotifier engine.Notifier + + txErrorMessagesCore *TxErrorMessagesCore // core logic for handling tx error messages +} + +// New creates a new Engine instance, initializing all necessary components +// for processing transaction result error messages. This includes setting +// up the job queue and the notifier for handling finalized blocks. +// +// No errors are expected during normal operation. +func New( + log zerolog.Logger, + state protocol.State, + headers storage.Headers, + txErrorMessagesProcessedHeight storage.ConsumerProgress, + txErrorMessagesCore *TxErrorMessagesCore, +) (*Engine, error) { + e := &Engine{ + log: log.With().Str("engine", "tx_error_messages_engine").Logger(), + state: state, + headers: headers, + txErrorMessagesCore: txErrorMessagesCore, + txErrorMessagesNotifier: engine.NewNotifier(), + } + + // jobqueue Jobs object that tracks sealed blocks by height. This is used by the txErrorMessagesConsumer + // to get a sequential list of sealed blocks. + sealedBlockReader := jobqueue.NewSealedBlockHeaderReader(state, headers) + + var err error + // Create a job queue that will process error messages for new sealed blocks. + // It listens to block finalization events from `txErrorMessagesNotifier`, then checks if there + // are new sealed blocks with `sealedBlockReader`. If there are, it starts workers to process + // them with `processTxResultErrorMessagesJob`, which fetches transaction error messages. At most + // `processTxErrorMessagesWorkersCount` workers will be created for concurrent processing. + // When a sealed block's error messages has been processed, it updates and persists the highest consecutive + // processed height with `txErrorMessagesProcessedHeight`. That way, if the node crashes, + // it reads the `txErrorMessagesProcessedHeight` and resume from `txErrorMessagesProcessedHeight + 1`. + // If the database is empty, rootHeight will be used to init the last processed height. + e.txErrorMessagesConsumer, err = jobqueue.NewComponentConsumer( + e.log.With().Str("engine", "tx_error_messages").Logger(), + e.txErrorMessagesNotifier.Channel(), + txErrorMessagesProcessedHeight, + sealedBlockReader, + e.state.Params().SealedRoot().Height, + e.processTxResultErrorMessagesJob, + processTxErrorMessagesWorkersCount, + 0, + ) + if err != nil { + return nil, fmt.Errorf("error creating transaction result error messages jobqueue: %w", err) + } + + // Add workers + e.ComponentManager = component.NewComponentManagerBuilder(). + AddWorker(e.runTxResultErrorMessagesConsumer). + Build() + + return e, nil +} + +// processTxResultErrorMessagesJob processes a job for transaction error messages by +// converting the job to a block and processing error messages. If processing +// fails for all attempts, it logs the error. +func (e *Engine) processTxResultErrorMessagesJob(ctx irrecoverable.SignalerContext, job module.Job, done func()) { + header, err := jobqueue.JobToBlockHeader(job) + if err != nil { + ctx.Throw(fmt.Errorf("failed to convert job to block: %w", err)) + } + + err = e.processErrorMessagesForBlock(ctx, header.ID()) + if err == nil { + done() + return + } + + e.log.Error(). + Err(err). + Str("job_id", string(job.ID())). + Msg("error encountered while processing transaction result error messages job") +} + +// runTxResultErrorMessagesConsumer runs the txErrorMessagesConsumer component +func (e *Engine) runTxResultErrorMessagesConsumer(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + e.txErrorMessagesConsumer.Start(ctx) + + err := util.WaitClosed(ctx, e.txErrorMessagesConsumer.Ready()) + if err == nil { + ready() + } + + <-e.txErrorMessagesConsumer.Done() +} + +// OnFinalizedBlock is called by the follower engine after a block has been finalized and the state has been updated. +// Receives block finalized events from the finalization distributor and forwards them to the txErrorMessagesConsumer. +func (e *Engine) OnFinalizedBlock(*model.Block) { + e.txErrorMessagesNotifier.Notify() +} + +// processErrorMessagesForBlock processes transaction result error messages for block. +// If the process fails, it will retry, using exponential backoff. +// +// No errors are expected during normal operation. +func (e *Engine) processErrorMessagesForBlock(ctx context.Context, blockID flow.Identifier) error { + backoff := retry.NewExponential(defaultRetryDelay) + backoff = retry.WithCappedDuration(defaultMaxRetryDelay, backoff) + backoff = retry.WithJitterPercent(15, backoff) + + attempt := 0 + return retry.Do(ctx, backoff, func(context.Context) error { + if attempt > 0 { + e.log.Debug(). + Str("block_id", blockID.String()). + Uint64("attempt", uint64(attempt)). + Msgf("retrying process transaction result error messages") + + } + attempt++ + err := e.txErrorMessagesCore.HandleTransactionResultErrorMessages(ctx, blockID) + + return retry.RetryableError(err) + }) +} diff --git a/engine/access/ingestion/tx_error_messages/tx_error_messages_engine_test.go b/engine/access/ingestion/tx_error_messages/tx_error_messages_engine_test.go new file mode 100644 index 00000000000..44980d7c79f --- /dev/null +++ b/engine/access/ingestion/tx_error_messages/tx_error_messages_engine_test.go @@ -0,0 +1,249 @@ +package tx_error_messages + +import ( + "context" + "os" + "sync" + "testing" + "time" + + "github.com/dgraph-io/badger/v2" + execproto "github.com/onflow/flow/protobuf/go/flow/execution" + "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" + accessmock "github.com/onflow/flow-go/engine/access/mock" + "github.com/onflow/flow-go/engine/access/rpc/backend" + connectionmock "github.com/onflow/flow-go/engine/access/rpc/connection/mock" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/irrecoverable" + protocol "github.com/onflow/flow-go/state/protocol/mock" + bstorage "github.com/onflow/flow-go/storage/badger" + storage "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/mocks" +) + +// TxErrorMessagesEngineSuite is a test suite for the transaction error messages engine. +// It sets up the necessary mocks and dependencies to test the functionality of +// handling transaction error messages. +type TxErrorMessagesEngineSuite struct { + suite.Suite + + log zerolog.Logger + proto struct { + state *protocol.FollowerState + snapshot *protocol.Snapshot + params *protocol.Params + } + headers *storage.Headers + receipts *storage.ExecutionReceipts + txErrorMessages *storage.TransactionResultErrorMessages + + enNodeIDs flow.IdentityList + execClient *accessmock.ExecutionAPIClient + connFactory *connectionmock.ConnectionFactory + + blockMap map[uint64]*flow.Block + rootBlock flow.Block + sealedBlock *flow.Header + + db *badger.DB + dbDir string + + ctx context.Context + cancel context.CancelFunc +} + +// TestTxErrorMessagesEngine runs the test suite for the transaction error messages engine. +func TestTxErrorMessagesEngine(t *testing.T) { + suite.Run(t, new(TxErrorMessagesEngineSuite)) +} + +// TearDownTest stops the engine and cleans up the db +func (s *TxErrorMessagesEngineSuite) TearDownTest() { + s.cancel() + err := os.RemoveAll(s.dbDir) + s.Require().NoError(err) +} + +func (s *TxErrorMessagesEngineSuite) SetupTest() { + s.log = zerolog.New(os.Stderr) + s.ctx, s.cancel = context.WithCancel(context.Background()) + s.db, s.dbDir = unittest.TempBadgerDB(s.T()) + // mock out protocol state + s.proto.state = protocol.NewFollowerState(s.T()) + s.proto.snapshot = protocol.NewSnapshot(s.T()) + s.proto.params = protocol.NewParams(s.T()) + s.execClient = accessmock.NewExecutionAPIClient(s.T()) + s.connFactory = connectionmock.NewConnectionFactory(s.T()) + s.headers = storage.NewHeaders(s.T()) + s.receipts = storage.NewExecutionReceipts(s.T()) + s.txErrorMessages = storage.NewTransactionResultErrorMessages(s.T()) + + blockCount := 5 + s.blockMap = make(map[uint64]*flow.Block, blockCount) + s.rootBlock = unittest.BlockFixture() + s.rootBlock.Header.Height = 0 + parent := s.rootBlock.Header + + for i := 0; i < blockCount; i++ { + block := unittest.BlockWithParentFixture(parent) + // update for next iteration + parent = block.Header + s.blockMap[block.Header.Height] = block + } + + s.sealedBlock = parent + + s.headers.On("ByHeight", mock.AnythingOfType("uint64")).Return( + mocks.ConvertStorageOutput( + mocks.StorageMapGetter(s.blockMap), + func(block *flow.Block) *flow.Header { return block.Header }, + ), + ).Maybe() + + s.proto.state.On("Params").Return(s.proto.params) + + // Mock the finalized and sealed root block header with height 0. + s.proto.params.On("FinalizedRoot").Return(s.rootBlock.Header, nil) + s.proto.params.On("SealedRoot").Return(s.rootBlock.Header, nil) + + s.proto.snapshot.On("Head").Return( + func() *flow.Header { + return s.sealedBlock + }, + nil, + ).Maybe() + + s.proto.state.On("Sealed").Return(s.proto.snapshot, nil) + s.proto.state.On("Final").Return(s.proto.snapshot, nil) + + // Create identities for 1 execution nodes. + s.enNodeIDs = unittest.IdentityListFixture(1, unittest.WithRole(flow.RoleExecution)) +} + +// initEngine creates a new instance of the transaction error messages engine +// and waits for it to start. It initializes the engine with mocked components and state. +func (s *TxErrorMessagesEngineSuite) initEngine(ctx irrecoverable.SignalerContext) *Engine { + processedTxErrorMessagesBlockHeight := bstorage.NewConsumerProgress( + s.db, + module.ConsumeProgressEngineTxErrorMessagesBlockHeight, + ) + + // Initialize the backend with the mocked state, blocks, headers, transactions, etc. + backend, err := backend.New(backend.Params{ + State: s.proto.state, + Headers: s.headers, + ExecutionReceipts: s.receipts, + ConnFactory: s.connFactory, + MaxHeightRange: backend.DefaultMaxHeightRange, + FixedExecutionNodeIDs: s.enNodeIDs.NodeIDs().Strings(), + Log: s.log, + SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, + Communicator: backend.NewNodeCommunicator(false), + ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly, + TxResultQueryMode: backend.IndexQueryModeExecutionNodesOnly, + ChainID: flow.Testnet, + }) + require.NoError(s.T(), err) + + txResultErrorMessagesCore := NewTxErrorMessagesCore( + s.log, + s.proto.state, + backend, + s.receipts, + s.txErrorMessages, + s.enNodeIDs.NodeIDs(), + nil, + ) + + eng, err := New( + s.log, + s.proto.state, + s.headers, + processedTxErrorMessagesBlockHeight, + txResultErrorMessagesCore, + ) + require.NoError(s.T(), err) + + eng.ComponentManager.Start(ctx) + <-eng.Ready() + + return eng +} + +// TestOnFinalizedBlockHandleTxErrorMessages tests the handling of transaction error messages +// when a new finalized block is processed. It verifies that the engine fetches transaction +// error messages from execution nodes and stores them in the database. +func (s *TxErrorMessagesEngineSuite) TestOnFinalizedBlockHandleTxErrorMessages() { + irrecoverableCtx := irrecoverable.NewMockSignalerContext(s.T(), s.ctx) + + block := unittest.BlockWithParentFixture(s.sealedBlock) + + s.blockMap[block.Header.Height] = block + s.sealedBlock = block.Header + + hotstuffBlock := hotmodel.Block{ + BlockID: block.ID(), + } + + // mock the connection factory + s.connFactory.On("GetExecutionAPIClient", mock.Anything).Return(s.execClient, &mockCloser{}, nil) + + s.proto.snapshot.On("Identities", mock.Anything).Return(s.enNodeIDs, nil) + s.proto.state.On("AtBlockID", mock.Anything).Return(s.proto.snapshot) + + count := 6 + wg := sync.WaitGroup{} + wg.Add(count) + + for _, b := range s.blockMap { + blockID := b.ID() + + // Mock the protocol snapshot to return fixed execution node IDs. + setupReceiptsForBlock(s.receipts, b, s.enNodeIDs.NodeIDs()[0]) + + // Mock the txErrorMessages storage to confirm that error messages do not exist yet. + s.txErrorMessages.On("Exists", blockID). + Return(false, nil).Once() + + // Create mock transaction results with a mix of failed and non-failed transactions. + resultsByBlockID := mockTransactionResultsByBlock(5) + + // Prepare a request to fetch transaction error messages by block ID from execution nodes. + exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: blockID[:], + } + + s.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). + Return(createTransactionErrorMessagesResponse(resultsByBlockID), nil).Once() + + // Prepare the expected transaction error messages that should be stored. + expectedStoreTxErrorMessages := createExpectedTxErrorMessages(resultsByBlockID, s.enNodeIDs.NodeIDs()[0]) + + // Mock the storage of the fetched error messages into the protocol database. + s.txErrorMessages.On("Store", blockID, expectedStoreTxErrorMessages).Return(nil). + Run(func(args mock.Arguments) { + // Ensure the test does not complete its work faster than necessary + wg.Done() + }).Once() + } + + eng := s.initEngine(irrecoverableCtx) + // process the block through the finalized callback + eng.OnFinalizedBlock(&hotstuffBlock) + + // Verify that all transaction error messages were processed within the timeout. + unittest.RequireReturnsBefore(s.T(), wg.Wait, 2*time.Second, "expect to process new block before timeout") + + // Ensure all expectations were met. + s.txErrorMessages.AssertExpectations(s.T()) + s.headers.AssertExpectations(s.T()) + s.proto.state.AssertExpectations(s.T()) + s.execClient.AssertExpectations(s.T()) +} diff --git a/engine/access/rpc/backend/backend.go b/engine/access/rpc/backend/backend.go index 59254e617bd..a48f335670d 100644 --- a/engine/access/rpc/backend/backend.go +++ b/engine/access/rpc/backend/backend.go @@ -17,10 +17,10 @@ import ( "github.com/onflow/flow-go/engine/access/rpc/connection" "github.com/onflow/flow-go/engine/access/subscription" "github.com/onflow/flow-go/engine/common/rpc" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/version" "github.com/onflow/flow-go/fvm/blueprints" "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/counters" "github.com/onflow/flow-go/module/execution" @@ -29,12 +29,6 @@ import ( "github.com/onflow/flow-go/storage" ) -// minExecutionNodesCnt is the minimum number of execution nodes expected to have sent the execution receipt for a block -const minExecutionNodesCnt = 2 - -// maxAttemptsForExecutionReceipt is the maximum number of attempts to find execution receipts for a given block ID -const maxAttemptsForExecutionReceipt = 3 - // DefaultMaxHeightRange is the default maximum size of range requests. const DefaultMaxHeightRange = 250 @@ -99,6 +93,7 @@ type Params struct { Transactions storage.Transactions ExecutionReceipts storage.ExecutionReceipts ExecutionResults storage.ExecutionResults + TxResultErrorMessages storage.TransactionResultErrorMessages ChainID flow.ChainID AccessMetrics module.AccessMetrics ConnFactory connection.ConnectionFactory @@ -110,7 +105,6 @@ type Params struct { SnapshotHistoryLimit int Communicator Communicator TxResultCacheSize uint - TxErrorMessagesCacheSize uint ScriptExecutor execution.ScriptExecutor ScriptExecutionMode IndexQueryMode CheckPayerBalanceMode access.PayerBalanceMode @@ -148,18 +142,6 @@ func New(params Params) (*Backend, error) { } } - // NOTE: The transaction error message cache is currently only used by the access node and not by the observer node. - // To avoid introducing unnecessary command line arguments in the observer, one case could be that the error - // message cache is nil for the observer node. - var txErrorMessagesCache *lru.Cache[flow.Identifier, string] - - if params.TxErrorMessagesCacheSize > 0 { - txErrorMessagesCache, err = lru.New[flow.Identifier, string](int(params.TxErrorMessagesCacheSize)) - if err != nil { - return nil, fmt.Errorf("failed to init cache for transaction error messages: %w", err) - } - } - // the system tx is hardcoded and never changes during runtime systemTx, err := blueprints.SystemChunkTransaction(params.ChainID.Chain()) if err != nil { @@ -261,6 +243,7 @@ func New(params Params) (*Backend, error) { chainID: params.ChainID, transactions: params.Transactions, executionReceipts: params.ExecutionReceipts, + txResultErrorMessages: params.TxResultErrorMessages, transactionValidator: txValidator, transactionMetrics: params.AccessMetrics, retry: retry, @@ -268,7 +251,6 @@ func New(params Params) (*Backend, error) { previousAccessNodes: params.HistoricalAccessNodes, nodeCommunicator: params.Communicator, txResultCache: txResCache, - txErrorMessagesCache: txErrorMessagesCache, txResultQueryMode: params.TxResultQueryMode, systemTx: systemTx, systemTxID: systemTxID, @@ -288,12 +270,12 @@ func New(params Params) (*Backend, error) { retry.SetBackend(b) - preferredENIdentifiers, err = identifierList(params.PreferredExecutionNodeIDs) + preferredENIdentifiers, err = commonrpc.IdentifierList(params.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 = identifierList(params.FixedExecutionNodeIDs) + fixedENIdentifiers, err = commonrpc.IdentifierList(params.FixedExecutionNodeIDs) if err != nil { return nil, fmt.Errorf("failed to convert node id string to Flow Identifier for fixed EN map: %w", err) } @@ -301,18 +283,6 @@ func New(params Params) (*Backend, error) { return b, nil } -func identifierList(ids []string) (flow.IdentifierList, error) { - idList := make(flow.IdentifierList, len(ids)) - for i, idStr := range ids { - id, err := flow.HexStringToIdentifier(idStr) - if err != nil { - return nil, fmt.Errorf("failed to convert node id string %s to Flow Identifier: %w", id, err) - } - idList[i] = id - } - return idList, nil -} - func configureTransactionValidator( state protocol.State, chainID flow.ChainID, @@ -420,229 +390,6 @@ func (b *Backend) GetNetworkParameters(_ context.Context) access.NetworkParamete } } -// executionNodesForBlockID returns upto maxNodesCnt number of randomly chosen execution node identities -// which have executed the given block ID. -// If no such execution node is found, an InsufficientExecutionReceipts error is returned. -func executionNodesForBlockID( - ctx context.Context, - blockID flow.Identifier, - executionReceipts storage.ExecutionReceipts, - state protocol.State, - log zerolog.Logger, -) (flow.IdentitySkeletonList, error) { - var ( - executorIDs flow.IdentifierList - err error - ) - - // check if the block ID is of the root block. If it is then don't look for execution receipts since they - // will not be present for the root block. - rootBlock := state.Params().FinalizedRoot() - - if rootBlock.ID() == blockID { - executorIdentities, err := state.Final().Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) - if err != nil { - return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) - } - executorIDs = executorIdentities.NodeIDs() - } else { - // try to find at least minExecutionNodesCnt execution node ids from the execution receipts for the given blockID - for attempt := 0; attempt < maxAttemptsForExecutionReceipt; attempt++ { - executorIDs, err = findAllExecutionNodes(blockID, executionReceipts, log) - if err != nil { - return nil, err - } - - if len(executorIDs) >= minExecutionNodesCnt { - break - } - - // log the attempt - log.Debug().Int("attempt", attempt).Int("max_attempt", maxAttemptsForExecutionReceipt). - Int("execution_receipts_found", len(executorIDs)). - Str("block_id", blockID.String()). - Msg("insufficient execution receipts") - - // if one or less execution receipts may have been received then re-query - // in the hope that more might have been received by now - - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-time.After(100 * time.Millisecond << time.Duration(attempt)): - // retry after an exponential backoff - } - } - - receiptCnt := len(executorIDs) - // if less than minExecutionNodesCnt execution receipts have been received so far, then return random ENs - if receiptCnt < minExecutionNodesCnt { - newExecutorIDs, err := state.AtBlockID(blockID).Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) - if err != nil { - return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) - } - executorIDs = newExecutorIDs.NodeIDs() - } - } - - // choose from the preferred or fixed execution nodes - subsetENs, err := chooseExecutionNodes(state, executorIDs) - if err != nil { - return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) - } - - if len(subsetENs) == 0 { - return nil, fmt.Errorf("no matching execution node found for block ID %v", blockID) - } - - return subsetENs, nil -} - -// findAllExecutionNodes find all the execution nodes ids from the execution receipts that have been received for the -// given blockID -func findAllExecutionNodes( - blockID flow.Identifier, - executionReceipts storage.ExecutionReceipts, - log zerolog.Logger, -) (flow.IdentifierList, error) { - // lookup the receipt's storage with the block ID - allReceipts, err := executionReceipts.ByBlockID(blockID) - if err != nil { - return nil, fmt.Errorf("failed to retreive execution receipts for block ID %v: %w", blockID, err) - } - - executionResultMetaList := make(flow.ExecutionReceiptMetaList, 0, len(allReceipts)) - for _, r := range allReceipts { - executionResultMetaList = append(executionResultMetaList, r.Meta()) - } - executionResultGroupedMetaList := executionResultMetaList.GroupByResultID() - - // maximum number of matching receipts found so far for any execution result id - maxMatchedReceiptCnt := 0 - // execution result id key for the highest number of matching receipts in the identicalReceipts map - var maxMatchedReceiptResultID flow.Identifier - - // find the largest list of receipts which have the same result ID - for resultID, executionReceiptList := range executionResultGroupedMetaList { - currentMatchedReceiptCnt := executionReceiptList.Size() - if currentMatchedReceiptCnt > maxMatchedReceiptCnt { - maxMatchedReceiptCnt = currentMatchedReceiptCnt - maxMatchedReceiptResultID = resultID - } - } - - // if there are more than one execution result for the same block ID, log as error - if executionResultGroupedMetaList.NumberGroups() > 1 { - identicalReceiptsStr := fmt.Sprintf("%v", flow.GetIDs(allReceipts)) - log.Error(). - Str("block_id", blockID.String()). - Str("execution_receipts", identicalReceiptsStr). - Msg("execution receipt mismatch") - } - - // pick the largest list of matching receipts - matchingReceiptMetaList := executionResultGroupedMetaList.GetGroup(maxMatchedReceiptResultID) - - metaReceiptGroupedByExecutorID := matchingReceiptMetaList.GroupByExecutorID() - - // collect all unique execution node ids from the receipts - var executorIDs flow.IdentifierList - for executorID := range metaReceiptGroupedByExecutorID { - executorIDs = append(executorIDs, executorID) - } - - return executorIDs, nil -} - -// chooseExecutionNodes finds the subset of execution nodes defined in the identity table by first -// choosing the preferred execution nodes which have executed the transaction. If no such preferred -// execution nodes are found, then the fixed execution nodes defined in the identity table are returned -// If neither preferred nor fixed nodes are defined, then all execution node matching the executor IDs are returned. -// e.g. If execution nodes in identity table are {1,2,3,4}, preferred ENs are defined as {2,3,4} -// and the executor IDs is {1,2,3}, then {2, 3} is returned as the chosen subset of ENs -func chooseExecutionNodes(state protocol.State, executorIDs flow.IdentifierList) (flow.IdentitySkeletonList, error) { - allENs, err := state.Final().Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) - if err != nil { - return nil, fmt.Errorf("failed to retrieve all execution IDs: %w", err) - } - - // choose from preferred EN IDs - if len(preferredENIdentifiers) > 0 { - chosenIDs := chooseFromPreferredENIDs(allENs, executorIDs) - return chosenIDs.ToSkeleton(), nil - } - - // if no preferred EN ID is found, then choose from the fixed EN IDs - if len(fixedENIdentifiers) > 0 { - // choose fixed ENs which have executed the transaction - chosenIDs := allENs.Filter(filter.And( - filter.HasNodeID[flow.Identity](fixedENIdentifiers...), - filter.HasNodeID[flow.Identity](executorIDs...), - )) - if len(chosenIDs) > 0 { - return chosenIDs.ToSkeleton(), nil - } - // if no such ENs are found, then just choose all fixed ENs - chosenIDs = allENs.Filter(filter.HasNodeID[flow.Identity](fixedENIdentifiers...)) - return chosenIDs.ToSkeleton(), nil - } - - // if no preferred or fixed ENs have been specified, then return all executor IDs i.e., no preference at all - return allENs.Filter(filter.HasNodeID[flow.Identity](executorIDs...)).ToSkeleton(), nil -} - -// chooseFromPreferredENIDs finds the subset of execution nodes if preferred execution nodes are defined. -// If preferredENIdentifiers is set and there are less than maxNodesCnt nodes selected, than the list is padded up to -// maxNodesCnt nodes using the following order: -// 1. Use any EN with a receipt. -// 2. Use any preferred node not already selected. -// 3. Use any EN not already selected. -func chooseFromPreferredENIDs(allENs flow.IdentityList, executorIDs flow.IdentifierList) flow.IdentityList { - var chosenIDs flow.IdentityList - - // filter for both preferred and executor IDs - chosenIDs = allENs.Filter(filter.And( - filter.HasNodeID[flow.Identity](preferredENIdentifiers...), - filter.HasNodeID[flow.Identity](executorIDs...), - )) - - if len(chosenIDs) >= maxNodesCnt { - return chosenIDs - } - - // function to add nodes to chosenIDs if they are not already included - addIfNotExists := func(candidates flow.IdentityList) { - for _, en := range candidates { - _, exists := chosenIDs.ByNodeID(en.NodeID) - if !exists { - chosenIDs = append(chosenIDs, en) - if len(chosenIDs) >= maxNodesCnt { - return - } - } - } - } - - // add any EN with a receipt - receiptENs := allENs.Filter(filter.HasNodeID[flow.Identity](executorIDs...)) - addIfNotExists(receiptENs) - if len(chosenIDs) >= maxNodesCnt { - return chosenIDs - } - - // add any preferred node not already selected - preferredENs := allENs.Filter(filter.HasNodeID[flow.Identity](preferredENIdentifiers...)) - addIfNotExists(preferredENs) - if len(chosenIDs) >= maxNodesCnt { - return chosenIDs - } - - // add any EN not already selected - addIfNotExists(allENs) - - return chosenIDs -} - // resolveHeightError processes errors returned during height-based queries. // If the error is due to a block not being found, this function determines whether the queried // height falls outside the node's accessible range and provides context-sensitive error messages diff --git a/engine/access/rpc/backend/backend_accounts.go b/engine/access/rpc/backend/backend_accounts.go index 252a834364a..f3a38219a31 100644 --- a/engine/access/rpc/backend/backend_accounts.go +++ b/engine/access/rpc/backend/backend_accounts.go @@ -14,6 +14,7 @@ import ( "github.com/onflow/flow-go/engine/access/rpc/connection" "github.com/onflow/flow-go/engine/common/rpc" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/rpc/convert" fvmerrors "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/model/flow" @@ -419,7 +420,15 @@ func (b *backendAccounts) getAccountFromAnyExeNode( BlockId: blockID[:], } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { return nil, rpc.ConvertError(err, "failed to find execution node to query", codes.Internal) } diff --git a/engine/access/rpc/backend/backend_events.go b/engine/access/rpc/backend/backend_events.go index b808e20c176..9f762672996 100644 --- a/engine/access/rpc/backend/backend_events.go +++ b/engine/access/rpc/backend/backend_events.go @@ -303,7 +303,13 @@ func (b *backendEvents) getBlockEventsFromExecutionNode( // choose the last block ID to find the list of execution nodes lastBlockID := blockIDs[len(blockIDs)-1] - execNodes, err := executionNodesForBlockID(ctx, lastBlockID, b.executionReceipts, b.state, b.log) + execNodes, err := rpc.ExecutionNodesForBlockID(ctx, + lastBlockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers) if err != nil { return nil, rpc.ConvertError(err, "failed to retrieve events from execution node", codes.Internal) } diff --git a/engine/access/rpc/backend/backend_scripts.go b/engine/access/rpc/backend/backend_scripts.go index 0efb21839b5..7d22f117912 100644 --- a/engine/access/rpc/backend/backend_scripts.go +++ b/engine/access/rpc/backend/backend_scripts.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/engine/access/rpc/connection" "github.com/onflow/flow-go/engine/common/rpc" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" fvmerrors "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -224,7 +225,7 @@ func (b *backendScripts) executeScriptOnAvailableExecutionNodes( r *scriptExecutionRequest, ) ([]byte, time.Duration, error) { // find few execution nodes which have executed the block earlier and provided an execution receipt for it - executors, err := executionNodesForBlockID(ctx, r.blockID, b.executionReceipts, b.state, b.log) + executors, err := commonrpc.ExecutionNodesForBlockID(ctx, r.blockID, b.executionReceipts, b.state, b.log, preferredENIdentifiers, fixedENIdentifiers) if err != nil { return nil, 0, status.Errorf(codes.Internal, "failed to find script executors at blockId %v: %v", r.blockID.String(), err) } diff --git a/engine/access/rpc/backend/backend_stream_blocks_test.go b/engine/access/rpc/backend/backend_stream_blocks_test.go index 69aa9e67823..6e655c8afda 100644 --- a/engine/access/rpc/backend/backend_stream_blocks_test.go +++ b/engine/access/rpc/backend/backend_stream_blocks_test.go @@ -148,15 +148,14 @@ func (s *BackendBlocksSuite) SetupTest() { // backendParams returns the Params configuration for the backend. func (s *BackendBlocksSuite) backendParams() Params { return Params{ - State: s.state, - Blocks: s.blocks, - Headers: s.headers, - ChainID: s.chainID, - MaxHeightRange: DefaultMaxHeightRange, - SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, - AccessMetrics: metrics.NewNoopCollector(), - Log: s.log, - TxErrorMessagesCacheSize: 1000, + State: s.state, + Blocks: s.blocks, + Headers: s.headers, + ChainID: s.chainID, + MaxHeightRange: DefaultMaxHeightRange, + SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, + AccessMetrics: metrics.NewNoopCollector(), + Log: s.log, SubscriptionHandler: subscription.NewSubscriptionHandler( s.log, s.broadcaster, diff --git a/engine/access/rpc/backend/backend_stream_transactions_test.go b/engine/access/rpc/backend/backend_stream_transactions_test.go index ec8ff353bf3..24cdf601f17 100644 --- a/engine/access/rpc/backend/backend_stream_transactions_test.go +++ b/engine/access/rpc/backend/backend_stream_transactions_test.go @@ -192,22 +192,21 @@ func (s *TransactionStatusSuite) TearDownTest() { // backendParams returns the Params configuration for the backend. func (s *TransactionStatusSuite) backendParams() Params { return Params{ - State: s.state, - Blocks: s.blocks, - Headers: s.headers, - Collections: s.collections, - Transactions: s.transactions, - ExecutionReceipts: s.receipts, - ExecutionResults: s.results, - ChainID: s.chainID, - CollectionRPC: s.colClient, - MaxHeightRange: DefaultMaxHeightRange, - SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, - Communicator: NewNodeCommunicator(false), - AccessMetrics: metrics.NewNoopCollector(), - Log: s.log, - TxErrorMessagesCacheSize: 1000, - BlockTracker: s.blockTracker, + State: s.state, + Blocks: s.blocks, + Headers: s.headers, + Collections: s.collections, + Transactions: s.transactions, + ExecutionReceipts: s.receipts, + ExecutionResults: s.results, + ChainID: s.chainID, + CollectionRPC: s.colClient, + MaxHeightRange: DefaultMaxHeightRange, + SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, + Communicator: NewNodeCommunicator(false), + AccessMetrics: metrics.NewNoopCollector(), + Log: s.log, + BlockTracker: s.blockTracker, SubscriptionHandler: subscription.NewSubscriptionHandler( s.log, s.broadcaster, diff --git a/engine/access/rpc/backend/backend_test.go b/engine/access/rpc/backend/backend_test.go index 73edb75791e..617135ad1aa 100644 --- a/engine/access/rpc/backend/backend_test.go +++ b/engine/access/rpc/backend/backend_test.go @@ -29,6 +29,7 @@ import ( backendmock "github.com/onflow/flow-go/engine/access/rpc/backend/mock" "github.com/onflow/flow-go/engine/access/rpc/connection" connectionmock "github.com/onflow/flow-go/engine/access/rpc/connection/mock" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/rpc/convert" "github.com/onflow/flow-go/engine/common/version" "github.com/onflow/flow-go/fvm/blueprints" @@ -72,6 +73,7 @@ type Suite struct { results *storagemock.ExecutionResults transactionResults *storagemock.LightTransactionResults events *storagemock.Events + txErrorMessages *storagemock.TransactionResultErrorMessages db *badger.DB dbDir string @@ -112,6 +114,7 @@ func (suite *Suite) SetupTest() { suite.collections = new(storagemock.Collections) suite.receipts = new(storagemock.ExecutionReceipts) suite.results = new(storagemock.ExecutionResults) + suite.txErrorMessages = storagemock.NewTransactionResultErrorMessages(suite.T()) suite.colClient = new(accessmock.AccessAPIClient) suite.execClient = new(accessmock.ExecutionAPIClient) suite.transactionResults = storagemock.NewLightTransactionResults(suite.T()) @@ -909,10 +912,6 @@ func (suite *Suite) TestGetTransactionResultByIndex() { suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - exeEventReq := &execproto.GetTransactionByIndexRequest{ BlockId: blockId[:], Index: index, @@ -924,7 +923,7 @@ func (suite *Suite) TestGetTransactionResultByIndex() { params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() backend, err := New(params) @@ -979,10 +978,6 @@ func (suite *Suite) TestGetTransactionResultsByBlockID() { suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - exeEventReq := &execproto.GetTransactionsByBlockIDRequest{ BlockId: blockId[:], } @@ -992,7 +987,7 @@ func (suite *Suite) TestGetTransactionResultsByBlockID() { } // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() backend, err := New(params) @@ -1072,10 +1067,6 @@ func (suite *Suite) TestTransactionStatusTransition() { suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - exeEventReq := &execproto.GetTransactionResultRequest{ BlockId: blockID[:], TransactionId: txID[:], @@ -1087,7 +1078,7 @@ func (suite *Suite) TestTransactionStatusTransition() { params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() backend, err := New(params) @@ -1730,7 +1721,7 @@ func (suite *Suite) TestGetNetworkParameters() { suite.Require().Equal(expectedChainID, actual.ChainID) } -// TestExecutionNodesForBlockID tests the common method backend.executionNodesForBlockID used for serving all API calls +// TestExecutionNodesForBlockID tests the common method backend.ExecutionNodesForBlockID used for serving all API calls // that need to talk to an execution node. func (suite *Suite) TestExecutionNodesForBlockID() { @@ -1796,7 +1787,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { expectedENs = flow.IdentityList{} } - allExecNodes, err := executionNodesForBlockID(context.Background(), block.ID(), suite.receipts, suite.state, suite.log) + allExecNodes, err := commonrpc.ExecutionNodesForBlockID(context.Background(), block.ID(), suite.receipts, suite.state, suite.log, preferredENIdentifiers, fixedENIdentifiers) require.NoError(suite.T(), err) execNodeSelectorFactory := NodeSelectorFactory{circuitBreakerEnabled: false} @@ -1810,7 +1801,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { { expectedENs := expectedENs.ToSkeleton() - if len(expectedENs) > maxNodesCnt { + if len(expectedENs) > commonrpc.MaxNodesCnt { for _, actual := range actualList { require.Contains(suite.T(), expectedENs, actual) } @@ -1819,7 +1810,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { } } } - // if we don't find sufficient receipts, executionNodesForBlockID should return a list of random ENs + // if we don't find sufficient receipts, ExecutionNodesForBlockID should return a list of random ENs suite.Run("insufficient receipts return random ENs in State", func() { // return no receipts at all attempts attempt1Receipts = flow.ExecutionReceiptList{} @@ -1827,7 +1818,8 @@ func (suite *Suite) TestExecutionNodesForBlockID() { attempt3Receipts = flow.ExecutionReceiptList{} suite.state.On("AtBlockID", mock.Anything).Return(suite.snapshot) - allExecNodes, err := executionNodesForBlockID(context.Background(), block.ID(), suite.receipts, suite.state, suite.log) + allExecNodes, err := commonrpc.ExecutionNodesForBlockID(context.Background(), block.ID(), suite.receipts, suite.state, suite.log, preferredENIdentifiers, + fixedENIdentifiers) require.NoError(suite.T(), err) execNodeSelectorFactory := NodeSelectorFactory{circuitBreakerEnabled: false} @@ -1839,7 +1831,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { actualList = append(actualList, actual) } - require.Equal(suite.T(), len(actualList), maxNodesCnt) + require.Equal(suite.T(), len(actualList), commonrpc.MaxNodesCnt) }) // if no preferred or fixed ENs are specified, the ExecutionNodesForBlockID function should @@ -1860,7 +1852,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { suite.Run("two preferred ENs with zero fixed EN", func() { // mark the first two ENs as preferred preferredENs := allExecutionNodes[0:2] - expectedList := allExecutionNodes[0:maxNodesCnt] + expectedList := allExecutionNodes[0:commonrpc.MaxNodesCnt] testExecutionNodesForBlockID(preferredENs, nil, expectedList) }) // if both are specified, the ExecutionNodesForBlockID function should @@ -1870,7 +1862,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { fixedENs := allExecutionNodes[0:5] // mark the first two of the fixed ENs as preferred ENs preferredENs := fixedENs[0:2] - expectedList := fixedENs[0:maxNodesCnt] + expectedList := fixedENs[0:commonrpc.MaxNodesCnt] testExecutionNodesForBlockID(preferredENs, fixedENs, expectedList) }) // if both are specified, but the preferred ENs don't match the ExecutorIDs in the ER, @@ -1895,7 +1887,7 @@ func (suite *Suite) TestExecutionNodesForBlockID() { currentAttempt = 0 // mark the first two ENs as preferred preferredENs := allExecutionNodes[0:2] - expectedList := allExecutionNodes[0:maxNodesCnt] + expectedList := allExecutionNodes[0:commonrpc.MaxNodesCnt] testExecutionNodesForBlockID(preferredENs, nil, expectedList) }) // if preferredENIdentifiers was set and there are less than maxNodesCnt nodes selected than check the order @@ -1916,10 +1908,10 @@ func (suite *Suite) TestExecutionNodesForBlockID() { additionalNode[0], } - chosenIDs := chooseFromPreferredENIDs(allExecutionNodes, executorIDs) + chosenIDs := commonrpc.ChooseFromPreferredENIDs(allExecutionNodes, executorIDs, preferredENIdentifiers) require.ElementsMatch(suite.T(), chosenIDs, expectedOrder) - require.Equal(suite.T(), len(chosenIDs), maxNodesCnt) + require.Equal(suite.T(), len(chosenIDs), commonrpc.MaxNodesCnt) }) } @@ -1965,13 +1957,9 @@ func (suite *Suite) TestGetTransactionResultEventEncodingVersion() { suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() backend, err := New(params) @@ -2031,13 +2019,9 @@ func (suite *Suite) TestGetTransactionResultByIndexAndBlockIdEventEncodingVersio suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() backend, err := New(params) @@ -2131,17 +2115,13 @@ func (suite *Suite) TestNodeCommunicator() { suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - exeEventReq := &execproto.GetTransactionsByBlockIDRequest{ BlockId: blockId[:], } params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = (fixedENIDs.NodeIDs()).Strings() // Left only one preferred execution node params.PreferredExecutionNodeIDs = []string{fixedENIDs[0].NodeID.String()} @@ -2219,25 +2199,24 @@ func generateEncodedEvents(t *testing.T, n int) ([]flow.Event, []flow.Event) { func (suite *Suite) defaultBackendParams() Params { return Params{ - State: suite.state, - Blocks: suite.blocks, - Headers: suite.headers, - Collections: suite.collections, - Transactions: suite.transactions, - ExecutionReceipts: suite.receipts, - ExecutionResults: suite.results, - ChainID: suite.chainID, - CollectionRPC: suite.colClient, - MaxHeightRange: DefaultMaxHeightRange, - SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, - Communicator: NewNodeCommunicator(false), - AccessMetrics: metrics.NewNoopCollector(), - Log: suite.log, - TxErrorMessagesCacheSize: 1000, - BlockTracker: nil, - TxResultQueryMode: IndexQueryModeExecutionNodesOnly, - LastFullBlockHeight: suite.lastFullBlockHeight, - VersionControl: suite.versionControl, + State: suite.state, + Blocks: suite.blocks, + Headers: suite.headers, + Collections: suite.collections, + Transactions: suite.transactions, + ExecutionReceipts: suite.receipts, + ExecutionResults: suite.results, + ChainID: suite.chainID, + CollectionRPC: suite.colClient, + MaxHeightRange: DefaultMaxHeightRange, + SnapshotHistoryLimit: DefaultSnapshotHistoryLimit, + Communicator: NewNodeCommunicator(false), + AccessMetrics: metrics.NewNoopCollector(), + Log: suite.log, + BlockTracker: nil, + TxResultQueryMode: IndexQueryModeExecutionNodesOnly, + LastFullBlockHeight: suite.lastFullBlockHeight, + VersionControl: suite.versionControl, } } diff --git a/engine/access/rpc/backend/backend_transactions.go b/engine/access/rpc/backend/backend_transactions.go index 7436863f9af..e7596fd2f65 100644 --- a/engine/access/rpc/backend/backend_transactions.go +++ b/engine/access/rpc/backend/backend_transactions.go @@ -17,6 +17,7 @@ import ( "github.com/onflow/flow-go/access" "github.com/onflow/flow-go/engine/access/rpc/connection" "github.com/onflow/flow-go/engine/common/rpc" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/rpc/convert" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -25,23 +26,28 @@ import ( "github.com/onflow/flow-go/storage" ) +const DefaultFailedErrorMessage = "failed" + type backendTransactions struct { *TransactionsLocalDataProvider - staticCollectionRPC accessproto.AccessAPIClient // rpc client tied to a fixed collection node - transactions storage.Transactions - executionReceipts storage.ExecutionReceipts - chainID flow.ChainID - transactionMetrics module.TransactionMetrics - transactionValidator *access.TransactionValidator - retry *Retry - connFactory connection.ConnectionFactory - - previousAccessNodes []accessproto.AccessAPIClient - log zerolog.Logger - nodeCommunicator Communicator - txResultCache *lru.Cache[flow.Identifier, *access.TransactionResult] - txErrorMessagesCache *lru.Cache[flow.Identifier, string] // cache for transactions error messages, indexed by hash(block_id, tx_id). - txResultQueryMode IndexQueryMode + staticCollectionRPC accessproto.AccessAPIClient // rpc client tied to a fixed collection node + transactions storage.Transactions + executionReceipts storage.ExecutionReceipts + // NOTE: The transaction error message is currently only used by the access node and not by the observer node. + // To avoid introducing unnecessary command line arguments in the observer, one case could be that the error + // message cache is nil for the observer node. + txResultErrorMessages storage.TransactionResultErrorMessages + chainID flow.ChainID + transactionMetrics module.TransactionMetrics + transactionValidator *access.TransactionValidator + retry *Retry + connFactory connection.ConnectionFactory + + previousAccessNodes []accessproto.AccessAPIClient + log zerolog.Logger + nodeCommunicator Communicator + txResultCache *lru.Cache[flow.Identifier, *access.TransactionResult] + txResultQueryMode IndexQueryMode systemTxID flow.Identifier systemTx *flow.TransactionBody @@ -405,7 +411,15 @@ func (b *backendTransactions) getTransactionResultsByBlockIDFromExecutionNode( BlockId: blockID[:], } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { if IsInsufficientExecutionReceipts(err) { return nil, status.Errorf(codes.NotFound, err.Error()) @@ -559,7 +573,15 @@ func (b *backendTransactions) getTransactionResultByIndexFromExecutionNode( Index: index, } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { if IsInsufficientExecutionReceipts(err) { return nil, status.Errorf(codes.NotFound, err.Error()) @@ -743,7 +765,15 @@ func (b *backendTransactions) getTransactionResultFromExecutionNode( TransactionId: transactionID[:], } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { // if no execution receipt were found, return a NotFound GRPC error if IsInsufficientExecutionReceipts(err) { @@ -962,28 +992,34 @@ func (b *backendTransactions) tryGetTransactionResultByIndex( } // LookupErrorMessageByTransactionID returns transaction error message for specified transaction. -// If an error message for transaction can be found in the cache then it will be used to serve the request, otherwise -// an RPC call will be made to the EN to fetch that error message, fetched value will be cached in the LRU cache. +// If transaction error messages are stored locally, they will be checked first in local storage. +// If error messages are not stored locally, an RPC call will be made to the EN to fetch message. +// // Expected errors during normal operation: -// - InsufficientExecutionReceipts - found insufficient receipts for given block ID. +// - InsufficientExecutionReceipts - found insufficient receipts for the given block ID. // - status.Error - remote GRPC call to EN has failed. func (b *backendTransactions) LookupErrorMessageByTransactionID( ctx context.Context, blockID flow.Identifier, + height uint64, transactionID flow.Identifier, ) (string, error) { - var cacheKey flow.Identifier - var value string - - if b.txErrorMessagesCache != nil { - cacheKey = flow.MakeIDFromFingerPrint(append(blockID[:], transactionID[:]...)) - value, cached := b.txErrorMessagesCache.Get(cacheKey) - if cached { - return value, nil + if b.txResultErrorMessages != nil { + res, err := b.txResultErrorMessages.ByBlockIDTransactionID(blockID, transactionID) + if err == nil { + return res.ErrorMessage, nil } } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { if IsInsufficientExecutionReceipts(err) { return "", status.Errorf(codes.NotFound, err.Error()) @@ -997,23 +1033,30 @@ func (b *backendTransactions) LookupErrorMessageByTransactionID( resp, err := b.getTransactionErrorMessageFromAnyEN(ctx, execNodes, req) if err != nil { - return "", fmt.Errorf("could not fetch error message from ENs: %w", err) - } - value = resp.ErrorMessage + // If no execution nodes return a valid response, + // return a static message "failed". + txResult, err := b.txResultsIndex.ByBlockIDTransactionID(blockID, height, transactionID) + if err != nil { + return "", rpc.ConvertStorageError(err) + } - if b.txErrorMessagesCache != nil { - b.txErrorMessagesCache.Add(cacheKey, value) + if txResult.Failed { + return DefaultFailedErrorMessage, nil + } + + // in case tx result is not failed + return "", nil } - return value, nil + return resp.ErrorMessage, nil } -// LookupErrorMessageByIndex returns transaction error message for specified transaction using its index. -// If an error message for transaction can be found in cache then it will be used to serve the request, otherwise -// an RPC call will be made to the EN to fetch that error message, fetched value will be cached in the LRU cache. +// LookupErrorMessageByIndex returns the transaction error message for a specified transaction using its index. +// If transaction error messages are stored locally, they will be checked first in local storage. +// If error messages are not stored locally, an RPC call will be made to the EN to fetch message. +// // Expected errors during normal operation: -// - status.Error[codes.NotFound] - transaction result for given block ID and tx index is not available. -// - InsufficientExecutionReceipts - found insufficient receipts for given block ID. +// - InsufficientExecutionReceipts - found insufficient receipts for the given block ID. // - status.Error - remote GRPC call to EN has failed. func (b *backendTransactions) LookupErrorMessageByIndex( ctx context.Context, @@ -1021,23 +1064,22 @@ func (b *backendTransactions) LookupErrorMessageByIndex( height uint64, index uint32, ) (string, error) { - txResult, err := b.txResultsIndex.ByBlockIDTransactionIndex(blockID, height, index) - if err != nil { - return "", rpc.ConvertStorageError(err) - } - - var cacheKey flow.Identifier - var value string - - if b.txErrorMessagesCache != nil { - cacheKey = flow.MakeIDFromFingerPrint(append(blockID[:], txResult.TransactionID[:]...)) - value, cached := b.txErrorMessagesCache.Get(cacheKey) - if cached { - return value, nil + if b.txResultErrorMessages != nil { + res, err := b.txResultErrorMessages.ByBlockIDTransactionIndex(blockID, index) + if err == nil { + return res.ErrorMessage, nil } } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID( + ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { if IsInsufficientExecutionReceipts(err) { return "", status.Errorf(codes.NotFound, err.Error()) @@ -1051,55 +1093,57 @@ func (b *backendTransactions) LookupErrorMessageByIndex( resp, err := b.getTransactionErrorMessageByIndexFromAnyEN(ctx, execNodes, req) if err != nil { - return "", fmt.Errorf("could not fetch error message from ENs: %w", err) - } - value = resp.ErrorMessage + // If no execution nodes return a valid response, + // return a static message "failed" + txResult, err := b.txResultsIndex.ByBlockIDTransactionIndex(blockID, height, index) + if err != nil { + return "", rpc.ConvertStorageError(err) + } + + if txResult.Failed { + return DefaultFailedErrorMessage, nil + } - if b.txErrorMessagesCache != nil { - b.txErrorMessagesCache.Add(cacheKey, value) + // in case tx result is not failed + return "", nil } - return value, nil + return resp.ErrorMessage, nil } // LookupErrorMessagesByBlockID returns all error messages for failed transactions by blockID. -// An RPC call will be made to the EN to fetch missing errors messages, fetched value will be cached in the LRU cache. +// If transaction error messages are stored locally, they will be checked first in local storage. +// If error messages are not stored locally, an RPC call will be made to the EN to fetch messages. +// // Expected errors during normal operation: -// - status.Error[codes.NotFound] - transaction results for given block ID are not available. -// - InsufficientExecutionReceipts - found insufficient receipts for given block ID. +// - InsufficientExecutionReceipts - found insufficient receipts for the given block ID. // - status.Error - remote GRPC call to EN has failed. func (b *backendTransactions) LookupErrorMessagesByBlockID( ctx context.Context, blockID flow.Identifier, height uint64, ) (map[flow.Identifier]string, error) { - txResults, err := b.txResultsIndex.ByBlockID(blockID, height) - if err != nil { - return nil, rpc.ConvertStorageError(err) - } + result := make(map[flow.Identifier]string) - results := make(map[flow.Identifier]string) - - if b.txErrorMessagesCache != nil { - needToFetch := false - for _, txResult := range txResults { - if txResult.Failed { - cacheKey := flow.MakeIDFromFingerPrint(append(blockID[:], txResult.TransactionID[:]...)) - if value, ok := b.txErrorMessagesCache.Get(cacheKey); ok { - results[txResult.TransactionID] = value - } else { - needToFetch = true - } + if b.txResultErrorMessages != nil { + res, err := b.txResultErrorMessages.ByBlockID(blockID) + if err == nil { + for _, value := range res { + result[value.TransactionID] = value.ErrorMessage } - } - // all transactions were served from cache or there were no failed transactions - if !needToFetch { - return results, nil + return result, nil } } - execNodes, err := executionNodesForBlockID(ctx, blockID, b.executionReceipts, b.state, b.log) + execNodes, err := commonrpc.ExecutionNodesForBlockID(ctx, + blockID, + b.executionReceipts, + b.state, + b.log, + preferredENIdentifiers, + fixedENIdentifiers, + ) if err != nil { if IsInsufficientExecutionReceipts(err) { return nil, status.Errorf(codes.NotFound, err.Error()) @@ -1110,18 +1154,28 @@ func (b *backendTransactions) LookupErrorMessagesByBlockID( BlockId: convert.IdentifierToMessage(blockID), } - resp, err := b.getTransactionErrorMessagesFromAnyEN(ctx, execNodes, req) + resp, _, err := b.GetTransactionErrorMessagesFromAnyEN(ctx, execNodes, req) if err != nil { - return nil, fmt.Errorf("could not fetch error message from ENs: %w", err) + // If no execution nodes return a valid response, + // return a static message "failed" + txResults, err := b.txResultsIndex.ByBlockID(blockID, height) + if err != nil { + return nil, rpc.ConvertStorageError(err) + } + + for _, txResult := range txResults { + if txResult.Failed { + result[txResult.TransactionID] = DefaultFailedErrorMessage + } + } + + return result, nil } - result := make(map[flow.Identifier]string, len(resp)) + for _, value := range resp { - if b.txErrorMessagesCache != nil { - cacheKey := flow.MakeIDFromFingerPrint(append(req.BlockId, value.TransactionId...)) - b.txErrorMessagesCache.Add(cacheKey, value.ErrorMessage) - } result[convert.MessageToIdentifier(value.TransactionId)] = value.ErrorMessage } + return result, nil } @@ -1209,26 +1263,29 @@ func (b *backendTransactions) getTransactionErrorMessageByIndexFromAnyEN( return resp, nil } -// getTransactionErrorMessagesFromAnyEN performs an RPC call using available nodes passed as argument. List of nodes must be non-empty otherwise an error will be returned. +// GetTransactionErrorMessagesFromAnyEN performs an RPC call using available nodes passed as argument. List of nodes must be non-empty otherwise an error will be returned. // Expected errors during normal operation: // - status.Error - GRPC call failed, some of possible codes are: // - codes.NotFound - request cannot be served by EN because of absence of data. // - codes.Unavailable - remote node is not unavailable. -func (b *backendTransactions) getTransactionErrorMessagesFromAnyEN( +func (b *backendTransactions) GetTransactionErrorMessagesFromAnyEN( ctx context.Context, execNodes flow.IdentitySkeletonList, req *execproto.GetTransactionErrorMessagesByBlockIDRequest, -) ([]*execproto.GetTransactionErrorMessagesResponse_Result, error) { +) ([]*execproto.GetTransactionErrorMessagesResponse_Result, *flow.IdentitySkeleton, error) { // if we were passed 0 execution nodes add a specific error if len(execNodes) == 0 { - return nil, errors.New("zero execution nodes") + return nil, nil, errors.New("zero execution nodes") } var resp *execproto.GetTransactionErrorMessagesResponse + var execNode *flow.IdentitySkeleton + errToReturn := b.nodeCommunicator.CallAvailableNode( execNodes, func(node *flow.IdentitySkeleton) error { var err error + execNode = node resp, err = b.tryGetTransactionErrorMessagesByBlockIDFromEN(ctx, node, req) if err == nil { b.log.Debug(). @@ -1245,10 +1302,10 @@ func (b *backendTransactions) getTransactionErrorMessagesFromAnyEN( // log the errors if errToReturn != nil { b.log.Err(errToReturn).Msg("failed to get transaction error messages from execution nodes") - return nil, errToReturn + return nil, nil, errToReturn } - return resp.GetResults(), nil + return resp.GetResults(), execNode, nil } // Expected errors during normal operation: diff --git a/engine/access/rpc/backend/backend_transactions_test.go b/engine/access/rpc/backend/backend_transactions_test.go index 5d1c513cef8..dabdd33bbbf 100644 --- a/engine/access/rpc/backend/backend_transactions_test.go +++ b/engine/access/rpc/backend/backend_transactions_test.go @@ -338,96 +338,191 @@ func (suite *Suite) TestGetTransactionResultUnknownFromCache() { }) } -// TestLookupTransactionErrorMessage_HappyPath tests lookup of a transaction error message. In a happy path if it wasn't found in the cache, it -// has to be fetched from the execution node, otherwise served from the cache. -// If the transaction has not failed, the error message must be empty. -func (suite *Suite) TestLookupTransactionErrorMessage_HappyPath() { +// TestLookupTransactionErrorMessageByTransactionID_HappyPath verifies the lookup of a transaction error message +// by block id and transaction id. +// It tests two cases: +// 1. Happy path where the error message is fetched from the EN if it's not found in the cache. +// 2. Happy path where the error message is served from the storage database if it exists. +func (suite *Suite) TestLookupTransactionErrorMessageByTransactionID_HappyPath() { block := unittest.BlockFixture() blockId := block.ID() failedTx := unittest.TransactionFixture() failedTxId := failedTx.ID() + failedTxIndex := rand.Uint32() + // Setup mock receipts and execution node identities. _, fixedENIDs := suite.setupReceipts(&block) suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - params := suite.defaultBackendParams() - // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() + params.TxResultErrorMessages = suite.txErrorMessages - backend, err := New(params) - suite.Require().NoError(err) + // Test case: transaction error message is fetched from the EN. + suite.Run("happy path from EN", func() { + // the connection factory should be used to get the execution node client + params.ConnFactory = suite.setupConnectionFactory() + params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - expectedErrorMsg := "some error" + // Mock the cache lookup for the transaction error message, returning "not found". + suite.txErrorMessages.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(nil, storage.ErrNotFound).Once() - exeEventReq := &execproto.GetTransactionErrorMessageRequest{ - BlockId: blockId[:], - TransactionId: failedTxId[:], - } + backend, err := New(params) + suite.Require().NoError(err) - exeEventResp := &execproto.GetTransactionErrorMessageResponse{ - TransactionId: failedTxId[:], - ErrorMessage: expectedErrorMsg, - } + // Mock the execution node API call to fetch the error message. + exeEventReq := &execproto.GetTransactionErrorMessageRequest{ + BlockId: blockId[:], + TransactionId: failedTxId[:], + } + exeEventResp := &execproto.GetTransactionErrorMessageResponse{ + TransactionId: failedTxId[:], + ErrorMessage: expectedErrorMsg, + } + suite.execClient.On("GetTransactionErrorMessage", mock.Anything, exeEventReq).Return(exeEventResp, nil).Once() - suite.execClient.On("GetTransactionErrorMessage", mock.Anything, exeEventReq).Return(exeEventResp, nil).Once() + // Perform the lookup and assert that the error message is retrieved correctly. + errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, block.Header.Height, failedTxId) + suite.Require().NoError(err) + suite.Require().Equal(expectedErrorMsg, errMsg) + suite.assertAllExpectations() + }) - errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, failedTxId) - suite.Require().NoError(err) - suite.Require().Equal(expectedErrorMsg, errMsg) + // Test case: transaction error message is fetched from the storage database. + suite.Run("happy path from storage db", func() { + backend, err := New(params) + suite.Require().NoError(err) - // ensure the transaction error message is cached after retrieval; we do this by mocking the grpc call - // only once - errMsg, err = backend.LookupErrorMessageByTransactionID(context.Background(), blockId, failedTxId) - suite.Require().NoError(err) - suite.Require().Equal(expectedErrorMsg, errMsg) - suite.assertAllExpectations() + // Mock the cache lookup for the transaction error message, returning a stored result. + suite.txErrorMessages.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(&flow.TransactionResultErrorMessage{ + TransactionID: failedTxId, + ErrorMessage: expectedErrorMsg, + Index: failedTxIndex, + ExecutorID: unittest.IdentifierFixture(), + }, nil).Once() + + // Perform the lookup and assert that the error message is retrieved correctly from storage. + errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, block.Header.Height, failedTxId) + suite.Require().NoError(err) + suite.Require().Equal(expectedErrorMsg, errMsg) + suite.assertAllExpectations() + }) } -// TestLookupTransactionErrorMessage_FailedToFetch tests lookup of a transaction error message, when a transaction result -// is not in the cache and needs to be fetched from EN, but the EN fails to return it. -func (suite *Suite) TestLookupTransactionErrorMessage_FailedToFetch() { +// TestLookupTransactionErrorMessageByTransactionID_FailedToFetch tests the case when a transaction error message +// is not in the cache and needs to be fetched from the EN, but the EN fails to return it. +// It tests three cases: +// 1. The transaction is not found in the transaction results, leading to a "NotFound" error. +// 2. The transaction result is not failed, and the error message is empty. +// 3. The transaction result is failed, and the error message "failed" are returned. +func (suite *Suite) TestLookupTransactionErrorMessageByTransactionID_FailedToFetch() { block := unittest.BlockFixture() blockId := block.ID() failedTx := unittest.TransactionFixture() failedTxId := failedTx.ID() + // Setup mock receipts and execution node identities. _, fixedENIDs := suite.setupReceipts(&block) suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) + // Create a mock index reporter + reporter := syncmock.NewIndexReporter(suite.T()) + reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) + reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) params := suite.defaultBackendParams() - // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + // The connection factory should be used to get the execution node client + params.ConnFactory = suite.setupConnectionFactory() + // Initialize the transaction results index with the mock reporter. params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() + params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) + err := params.TxResultsIndex.Initialize(reporter) + suite.Require().NoError(err) + + params.TxResultErrorMessages = suite.txErrorMessages backend, err := New(params) suite.Require().NoError(err) - // lookup should try each of the 2 ENs in fixedENIDs - suite.execClient.On("GetTransactionErrorMessage", mock.Anything, mock.Anything).Return(nil, - status.Error(codes.Unavailable, "")).Twice() + // Test case: failed to fetch from EN, transaction is unknown. + suite.Run("failed to fetch from EN, unknown tx", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessage", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction and tx error message is not found in the storage. + suite.txErrorMessages.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(nil, storage.ErrNotFound).Once() + suite.transactionResults.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(nil, storage.ErrNotFound).Once() - errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, failedTxId) - suite.Require().Error(err) - suite.Require().Equal(codes.Unavailable, status.Code(err)) - suite.Require().Empty(errMsg) + // Perform the lookup and expect a "NotFound" error with an empty error message. + errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, block.Header.Height, failedTxId) + suite.Require().Error(err) + suite.Require().Equal(codes.NotFound, status.Code(err)) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) - suite.assertAllExpectations() + // Test case: failed to fetch from EN, but the transaction result is not failed. + suite.Run("failed to fetch from EN, tx result is not failed", func() { + // Lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessage", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction error message is not found in storage. + suite.txErrorMessages.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(nil, storage.ErrNotFound).Once() + + // Setup mock that the transaction result exists and is not failed. + suite.transactionResults.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(&flow.LightTransactionResult{ + TransactionID: failedTxId, + Failed: false, + ComputationUsed: 0, + }, nil).Once() + + // Perform the lookup and expect no error and an empty error message. + errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, block.Header.Height, failedTxId) + suite.Require().NoError(err) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) + + // Test case: failed to fetch from EN, but the transaction result is failed. + suite.Run("failed to fetch from EN, tx result is failed", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessage", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction error message is not found in storage. + suite.txErrorMessages.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(nil, storage.ErrNotFound).Once() + + // Setup mock that the transaction result exists and is failed. + suite.transactionResults.On("ByBlockIDTransactionID", blockId, failedTxId). + Return(&flow.LightTransactionResult{ + TransactionID: failedTxId, + Failed: true, + ComputationUsed: 0, + }, nil).Once() + + // Perform the lookup and expect the failed error message to be returned. + errMsg, err := backend.LookupErrorMessageByTransactionID(context.Background(), blockId, block.Header.Height, failedTxId) + suite.Require().NoError(err) + suite.Require().Equal(errMsg, DefaultFailedErrorMessage) + suite.assertAllExpectations() + }) } -// TestLookupTransactionErrorMessageByIndex_HappyPath tests lookup of a transaction error message by index. -// In a happy path if it wasn't found in the cache, it has to be fetched from the execution node, otherwise served from the cache. -// If the transaction has not failed, the error message must be empty. +// TestLookupTransactionErrorMessageByIndex_HappyPath verifies the lookup of a transaction error message +// by block ID and transaction index. +// It tests two cases: +// 1. Happy path where the error message is fetched from the EN if it is not found in the cache. +// 2. Happy path where the error message is served from the storage database if it exists. func (suite *Suite) TestLookupTransactionErrorMessageByIndex_HappyPath() { block := unittest.BlockFixture() blockId := block.ID() @@ -435,122 +530,90 @@ func (suite *Suite) TestLookupTransactionErrorMessageByIndex_HappyPath() { failedTxId := failedTx.ID() failedTxIndex := rand.Uint32() - suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). - Return(&flow.LightTransactionResult{ - TransactionID: failedTxId, - Failed: true, - ComputationUsed: 0, - }, nil).Twice() - + // Setup mock receipts and execution node identities. _, fixedENIDs := suite.setupReceipts(&block) suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - - // create a mock index reporter - reporter := syncmock.NewIndexReporter(suite.T()) - reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) - reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) - params := suite.defaultBackendParams() + params.TxResultErrorMessages = suite.txErrorMessages - // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - - params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) - err := params.TxResultsIndex.Initialize(reporter) - suite.Require().NoError(err) - - backend, err := New(params) - suite.Require().NoError(err) - - expectedErrorMsg := "some error" - - exeEventReq := &execproto.GetTransactionErrorMessageByIndexRequest{ - BlockId: blockId[:], - Index: failedTxIndex, - } - - exeEventResp := &execproto.GetTransactionErrorMessageResponse{ - TransactionId: failedTxId[:], - ErrorMessage: expectedErrorMsg, - } - - suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, exeEventReq).Return(exeEventResp, nil).Once() - - errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) - suite.Require().NoError(err) - suite.Require().Equal(expectedErrorMsg, errMsg) - - // ensure the transaction error message is cached after retrieval; we do this by mocking the grpc call - // only once - errMsg, err = backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) - suite.Require().NoError(err) - suite.Require().Equal(expectedErrorMsg, errMsg) - suite.assertAllExpectations() -} - -// TestLookupTransactionErrorMessageByIndex_UnknownTransaction tests lookup of a transaction error message by index, -// when a transaction result has not been synced yet, in this case nothing we can do but return an error. -func (suite *Suite) TestLookupTransactionErrorMessageByIndex_UnknownTransaction() { - block := unittest.BlockFixture() - blockId := block.ID() - failedTxIndex := rand.Uint32() + // Test case: transaction error message is fetched from the EN. + suite.Run("happy path from EN", func() { + // the connection factory should be used to get the execution node client + params.ConnFactory = suite.setupConnectionFactory() + params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). - Return(nil, storage.ErrNotFound).Once() + // Mock the cache lookup for the transaction error message, returning "not found". + suite.txErrorMessages.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(nil, storage.ErrNotFound).Once() - // create a mock index reporter - reporter := syncmock.NewIndexReporter(suite.T()) - reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) - reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) - - params := suite.defaultBackendParams() + backend, err := New(params) + suite.Require().NoError(err) - params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) - err := params.TxResultsIndex.Initialize(reporter) - suite.Require().NoError(err) + // Mock the execution node API call to fetch the error message. + exeEventReq := &execproto.GetTransactionErrorMessageByIndexRequest{ + BlockId: blockId[:], + Index: failedTxIndex, + } + exeEventResp := &execproto.GetTransactionErrorMessageResponse{ + TransactionId: failedTxId[:], + ErrorMessage: expectedErrorMsg, + } + suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, exeEventReq).Return(exeEventResp, nil).Once() - backend, err := New(params) - suite.Require().NoError(err) + // Perform the lookup and assert that the error message is retrieved correctly. + errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) + suite.Require().NoError(err) + suite.Require().Equal(expectedErrorMsg, errMsg) + suite.assertAllExpectations() + }) - errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) - suite.Require().Error(err) - suite.Require().Equal(codes.NotFound, status.Code(err)) - suite.Require().Empty(errMsg) + // Test case: transaction error message is fetched from the storage database. + suite.Run("happy path from storage db", func() { + backend, err := New(params) + suite.Require().NoError(err) - suite.assertAllExpectations() + // Mock the cache lookup for the transaction error message, returning a stored result. + suite.txErrorMessages.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(&flow.TransactionResultErrorMessage{ + TransactionID: failedTxId, + ErrorMessage: expectedErrorMsg, + Index: failedTxIndex, + ExecutorID: unittest.IdentifierFixture(), + }, nil).Once() + + // Perform the lookup and assert that the error message is retrieved correctly from storage. + errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) + suite.Require().NoError(err) + suite.Require().Equal(expectedErrorMsg, errMsg) + suite.assertAllExpectations() + }) } -// TestLookupTransactionErrorMessageByIndex_FailedToFetch tests lookup of a transaction error message by index, -// when a transaction result is not in the cache and needs to be fetched from EN, but the EN fails to return it. +// TestLookupTransactionErrorMessageByIndex_FailedToFetch verifies the behavior of looking up a transaction error message by index +// when the error message is not in the cache, and fetching it from the EN fails. +// It tests three cases: +// 1. The transaction is not found in the transaction results, leading to a "NotFound" error. +// 2. The transaction result is not failed, and the error message is empty. +// 3. The transaction result is failed, and the error message "failed" are returned. func (suite *Suite) TestLookupTransactionErrorMessageByIndex_FailedToFetch() { block := unittest.BlockFixture() blockId := block.ID() + failedTxIndex := rand.Uint32() failedTx := unittest.TransactionFixture() failedTxId := failedTx.ID() - failedTxIndex := rand.Uint32() - - suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). - Return(&flow.LightTransactionResult{ - TransactionID: failedTxId, - Failed: true, - ComputationUsed: 0, - }, nil).Once() + // Setup mock receipts and execution node identities. _, fixedENIDs := suite.setupReceipts(&block) suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory + // Create a mock connection factory connFactory := connectionmock.NewConnectionFactory(suite.T()) connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // create a mock index reporter + // Create a mock index reporter reporter := syncmock.NewIndexReporter(suite.T()) reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) @@ -559,30 +622,92 @@ func (suite *Suite) TestLookupTransactionErrorMessageByIndex_FailedToFetch() { // the connection factory should be used to get the execution node client params.ConnFactory = connFactory params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - + // Initialize the transaction results index with the mock reporter. params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) err := params.TxResultsIndex.Initialize(reporter) suite.Require().NoError(err) + params.TxResultErrorMessages = suite.txErrorMessages + backend, err := New(params) suite.Require().NoError(err) - // lookup should try each of the 2 ENs in fixedENIDs - suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, mock.Anything).Return(nil, - status.Error(codes.Unavailable, "")).Twice() + // Test case: failed to fetch from EN, transaction is unknown. + suite.Run("failed to fetch from EN, unknown tx", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() - errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) - suite.Require().Error(err) - suite.Require().Equal(codes.Unavailable, status.Code(err)) - suite.Require().Empty(errMsg) + // Setup mock that the transaction and tx error message is not found in the storage. + suite.txErrorMessages.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(nil, storage.ErrNotFound).Once() + suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(nil, storage.ErrNotFound).Once() - suite.assertAllExpectations() + // Perform the lookup and expect a "NotFound" error with an empty error message. + errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) + suite.Require().Error(err) + suite.Require().Equal(codes.NotFound, status.Code(err)) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) + + // Test case: failed to fetch from EN, but the transaction result is not failed. + suite.Run("failed to fetch from EN, tx result is not failed", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction error message is not found in storage. + suite.txErrorMessages.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(nil, storage.ErrNotFound).Once() + + // Setup mock that the transaction result exists and is not failed. + suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(&flow.LightTransactionResult{ + TransactionID: failedTxId, + Failed: false, + ComputationUsed: 0, + }, nil).Once() + + // Perform the lookup and expect no error and an empty error message. + errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) + suite.Require().NoError(err) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) + + // Test case: failed to fetch from EN, but the transaction result is failed. + suite.Run("failed to fetch from EN, tx result is failed", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessageByIndex", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction error message is not found in storage. + suite.txErrorMessages.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(nil, storage.ErrNotFound).Once() + + // Setup mock that the transaction result exists and is failed. + suite.transactionResults.On("ByBlockIDTransactionIndex", blockId, failedTxIndex). + Return(&flow.LightTransactionResult{ + TransactionID: failedTxId, + Failed: true, + ComputationUsed: 0, + }, nil).Once() + + // Perform the lookup and expect the failed error message to be returned. + errMsg, err := backend.LookupErrorMessageByIndex(context.Background(), blockId, block.Header.Height, failedTxIndex) + suite.Require().NoError(err) + suite.Require().Equal(errMsg, DefaultFailedErrorMessage) + suite.assertAllExpectations() + }) } -// TestLookupTransactionErrorMessages_HappyPath tests lookup of a transaction error messages by block ID. -// In a happy path, it has to be fetched from the execution node if there are no cached results. -// All fetched transactions have to be cached for future calls. -func (suite *Suite) TestLookupTransactionErrorMessages_HappyPath() { +// TestLookupTransactionErrorMessagesByBlockID_HappyPath verifies the lookup of transaction error messages by block ID. +// It tests two cases: +// 1. Happy path where the error messages are fetched from the EN if they are not found in the cache. +// 2. Happy path where the error messages are served from the storage database if they exist. +func (suite *Suite) TestLookupTransactionErrorMessagesByBlockID_HappyPath() { block := unittest.BlockFixture() blockId := block.ID() @@ -595,211 +720,225 @@ func (suite *Suite) TestLookupTransactionErrorMessages_HappyPath() { }) } - suite.transactionResults.On("ByBlockID", blockId). - Return(resultsByBlockID, nil).Twice() - _, fixedENIDs := suite.setupReceipts(&block) suite.state.On("Final").Return(suite.snapshot, nil).Maybe() suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - - // create a mock index reporter - reporter := syncmock.NewIndexReporter(suite.T()) - reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) - reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) - params := suite.defaultBackendParams() + params.TxResultErrorMessages = suite.txErrorMessages - // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - - params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) - err := params.TxResultsIndex.Initialize(reporter) - suite.Require().NoError(err) + // Test case: transaction error messages is fetched from the EN. + suite.Run("happy path from EN", func() { + // the connection factory should be used to get the execution node client + params.ConnFactory = suite.setupConnectionFactory() + params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() - backend, err := New(params) - suite.Require().NoError(err) + // Mock the cache lookup for the transaction error messages, returning "not found". + suite.txErrorMessages.On("ByBlockID", blockId). + Return(nil, storage.ErrNotFound).Once() - expectedErrorMsg := "some error" + backend, err := New(params) + suite.Require().NoError(err) - exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ - BlockId: blockId[:], - } + // Mock the execution node API call to fetch the error messages. + exeEventReq := &execproto.GetTransactionErrorMessagesByBlockIDRequest{ + BlockId: blockId[:], + } + exeErrMessagesResp := &execproto.GetTransactionErrorMessagesResponse{} + for _, result := range resultsByBlockID { + r := result + if r.Failed { + errMsg := fmt.Sprintf("%s.%s", expectedErrorMsg, r.TransactionID) + exeErrMessagesResp.Results = append(exeErrMessagesResp.Results, &execproto.GetTransactionErrorMessagesResponse_Result{ + TransactionId: r.TransactionID[:], + ErrorMessage: errMsg, + }) + } + } + suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). + Return(exeErrMessagesResp, nil). + Once() - exeEventResp := &execproto.GetTransactionErrorMessagesResponse{} - for _, result := range resultsByBlockID { - r := result - if r.Failed { - errMsg := fmt.Sprintf("%s.%s", expectedErrorMsg, r.TransactionID) - exeEventResp.Results = append(exeEventResp.Results, &execproto.GetTransactionErrorMessagesResponse_Result{ - TransactionId: r.TransactionID[:], - ErrorMessage: errMsg, - }) + // Perform the lookup and assert that the error message is retrieved correctly. + errMessages, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) + suite.Require().NoError(err) + suite.Require().Len(errMessages, len(exeErrMessagesResp.Results)) + for _, expectedResult := range exeErrMessagesResp.Results { + errMsg, ok := errMessages[convert.MessageToIdentifier(expectedResult.TransactionId)] + suite.Require().True(ok) + suite.Assert().Equal(expectedResult.ErrorMessage, errMsg) } - } + suite.assertAllExpectations() + }) - suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, exeEventReq). - Return(exeEventResp, nil). - Once() + // Test case: transaction error messages is fetched from the storage database. + suite.Run("happy path from storage db", func() { + backend, err := New(params) + suite.Require().NoError(err) - errMessages, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) - suite.Require().NoError(err) - suite.Require().Len(errMessages, len(exeEventResp.Results)) - for _, expectedResult := range exeEventResp.Results { - errMsg, ok := errMessages[convert.MessageToIdentifier(expectedResult.TransactionId)] - suite.Require().True(ok) - suite.Assert().Equal(expectedResult.ErrorMessage, errMsg) - } + // Mock the cache lookup for the transaction error messages, returning a stored result. + var txErrorMessages []flow.TransactionResultErrorMessage + for i, result := range resultsByBlockID { + if result.Failed { + errMsg := fmt.Sprintf("%s.%s", expectedErrorMsg, result.TransactionID) + + txErrorMessages = append(txErrorMessages, + flow.TransactionResultErrorMessage{ + TransactionID: result.TransactionID, + ErrorMessage: errMsg, + Index: uint32(i), + ExecutorID: unittest.IdentifierFixture(), + }) + } + } + suite.txErrorMessages.On("ByBlockID", blockId). + Return(txErrorMessages, nil).Once() - // ensure the transaction error message is cached after retrieval; we do this by mocking the grpc call - // only once - errMessages, err = backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) - suite.Require().NoError(err) - suite.Require().Len(errMessages, len(exeEventResp.Results)) - for _, expectedResult := range exeEventResp.Results { - errMsg, ok := errMessages[convert.MessageToIdentifier(expectedResult.TransactionId)] - suite.Require().True(ok) - suite.Assert().Equal(expectedResult.ErrorMessage, errMsg) - } - suite.assertAllExpectations() + // Perform the lookup and assert that the error message is retrieved correctly from storage. + errMessages, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) + suite.Require().NoError(err) + suite.Require().Len(errMessages, len(txErrorMessages)) + for _, expected := range txErrorMessages { + errMsg, ok := errMessages[expected.TransactionID] + suite.Require().True(ok) + suite.Assert().Equal(expected.ErrorMessage, errMsg) + } + suite.assertAllExpectations() + }) } -// TestLookupTransactionErrorMessages_HappyPath_NoFailedTxns tests lookup of a transaction error messages by block ID. -// In a happy path where a block with no failed txns is requested. We don't want to perform an RPC call in this case. -func (suite *Suite) TestLookupTransactionErrorMessages_HappyPath_NoFailedTxns() { +// TestLookupTransactionErrorMessagesByBlockID_FailedToFetch tests lookup of a transaction error messages by block ID, +// when a transaction result is not in the cache and needs to be fetched from EN, but the EN fails to return it. +// It tests three cases: +// 1. The transaction is not found in the transaction results, leading to a "NotFound" error. +// 2. The transaction result is not failed, and the error message is empty. +// 3. The transaction result is failed, and the error message "failed" are returned. +func (suite *Suite) TestLookupTransactionErrorMessagesByBlockID_FailedToFetch() { block := unittest.BlockFixture() blockId := block.ID() - resultsByBlockID := []flow.LightTransactionResult{ - { - TransactionID: unittest.IdentifierFixture(), - Failed: false, - ComputationUsed: 0, - }, - { - TransactionID: unittest.IdentifierFixture(), - Failed: false, - ComputationUsed: 0, - }, - } - - suite.transactionResults.On("ByBlockID", blockId). - Return(resultsByBlockID, nil).Once() + // Setup mock receipts and execution node identities. + _, fixedENIDs := suite.setupReceipts(&block) + suite.state.On("Final").Return(suite.snapshot, nil).Maybe() + suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - // create a mock index reporter + // Create a mock index reporter reporter := syncmock.NewIndexReporter(suite.T()) reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) params := suite.defaultBackendParams() - + // the connection factory should be used to get the execution node client + params.ConnFactory = suite.setupConnectionFactory() + params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() + // Initialize the transaction results index with the mock reporter. params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) err := params.TxResultsIndex.Initialize(reporter) suite.Require().NoError(err) - backend, err := New(params) - suite.Require().NoError(err) - - errMessages, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) - suite.Require().NoError(err) - suite.Require().Empty(errMessages) - suite.assertAllExpectations() -} - -// TestLookupTransactionErrorMessages_UnknownTransaction tests lookup of a transaction error messages by block ID, -// when a transaction results for block has not been synced yet, in this case nothing we can do but return an error. -func (suite *Suite) TestLookupTransactionErrorMessages_UnknownTransaction() { - block := unittest.BlockFixture() - blockId := block.ID() - - suite.transactionResults.On("ByBlockID", blockId). - Return(nil, storage.ErrNotFound).Once() - - // create a mock index reporter - reporter := syncmock.NewIndexReporter(suite.T()) - reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) - reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) - - params := suite.defaultBackendParams() - - params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) - err := params.TxResultsIndex.Initialize(reporter) - suite.Require().NoError(err) + params.TxResultErrorMessages = suite.txErrorMessages backend, err := New(params) suite.Require().NoError(err) - errMsg, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) - suite.Require().Error(err) - suite.Require().Equal(codes.NotFound, status.Code(err)) - suite.Require().Empty(errMsg) - - suite.assertAllExpectations() -} - -// TestLookupTransactionErrorMessages_FailedToFetch tests lookup of a transaction error messages by block ID, -// when a transaction result is not in the cache and needs to be fetched from EN, but the EN fails to return it. -func (suite *Suite) TestLookupTransactionErrorMessages_FailedToFetch() { - block := unittest.BlockFixture() - blockId := block.ID() - - resultsByBlockID := []flow.LightTransactionResult{ - { - TransactionID: unittest.IdentifierFixture(), - Failed: true, - ComputationUsed: 0, - }, - { - TransactionID: unittest.IdentifierFixture(), - Failed: true, - ComputationUsed: 0, - }, - } - - suite.transactionResults.On("ByBlockID", blockId). - Return(resultsByBlockID, nil).Once() + // Test case: failed to fetch from EN, transaction is unknown. + suite.Run("failed to fetch from EN, unknown tx", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() - _, fixedENIDs := suite.setupReceipts(&block) - suite.state.On("Final").Return(suite.snapshot, nil).Maybe() - suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) - - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) + // Setup mock that the transaction and tx error messages is not found in the storage. + suite.txErrorMessages.On("ByBlockID", blockId). + Return(nil, storage.ErrNotFound).Once() + suite.transactionResults.On("ByBlockID", blockId). + Return(nil, storage.ErrNotFound).Once() - // create a mock index reporter - reporter := syncmock.NewIndexReporter(suite.T()) - reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) - reporter.On("HighestIndexedHeight").Return(block.Header.Height+10, nil) + // Perform the lookup and expect a "NotFound" error with an empty error message. + errMsg, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) + suite.Require().Error(err) + suite.Require().Equal(codes.NotFound, status.Code(err)) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) - params := suite.defaultBackendParams() - // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() + // Test case: failed to fetch from EN, but the transaction result is not failed. + suite.Run("failed to fetch from EN, tx result is not failed", func() { + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() + + // Setup mock that the transaction error message is not found in storage. + suite.txErrorMessages.On("ByBlockID", blockId). + Return(nil, storage.ErrNotFound).Once() + + // Setup mock that the transaction results exists and is not failed. + suite.transactionResults.On("ByBlockID", blockId). + Return([]flow.LightTransactionResult{ + { + TransactionID: unittest.IdentifierFixture(), + Failed: false, + ComputationUsed: 0, + }, + { + TransactionID: unittest.IdentifierFixture(), + Failed: false, + ComputationUsed: 0, + }, + }, nil).Once() + + // Perform the lookup and expect no error and an empty error messages. + errMsg, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) + suite.Require().NoError(err) + suite.Require().Empty(errMsg) + suite.assertAllExpectations() + }) - params.TxResultsIndex = index.NewTransactionResultsIndex(index.NewReporter(), suite.transactionResults) - err := params.TxResultsIndex.Initialize(reporter) - suite.Require().NoError(err) + // Test case: failed to fetch from EN, but the transaction result is failed. + suite.Run("failed to fetch from EN, tx result is failed", func() { + failedResultsByBlockID := []flow.LightTransactionResult{ + { + TransactionID: unittest.IdentifierFixture(), + Failed: true, + ComputationUsed: 0, + }, + { + TransactionID: unittest.IdentifierFixture(), + Failed: true, + ComputationUsed: 0, + }, + } - backend, err := New(params) - suite.Require().NoError(err) + // lookup should try each of the 2 ENs in fixedENIDs + suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, mock.Anything).Return(nil, + status.Error(codes.Unavailable, "")).Twice() - // pretend the first transaction has been cached, but there are multiple failed txns so still a request has to be made. - backend.txErrorMessagesCache.Add(resultsByBlockID[0].TransactionID, "some error") + // Setup mock that the transaction error messages is not found in storage. + suite.txErrorMessages.On("ByBlockID", blockId). + Return(nil, storage.ErrNotFound).Once() - suite.execClient.On("GetTransactionErrorMessagesByBlockID", mock.Anything, mock.Anything).Return(nil, - status.Error(codes.Unavailable, "")).Twice() + // Setup mock that the transaction results exists and is failed. + suite.transactionResults.On("ByBlockID", blockId). + Return(failedResultsByBlockID, nil).Once() - errMsg, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) - suite.Require().Error(err) - suite.Require().Equal(codes.Unavailable, status.Code(err)) - suite.Require().Empty(errMsg) + // Setup mock expected the transaction error messages after retrieving the failed result. + expectedTxErrorMessages := make(map[flow.Identifier]string) + for _, result := range failedResultsByBlockID { + if result.Failed { + expectedTxErrorMessages[result.TransactionID] = DefaultFailedErrorMessage + } + } - suite.assertAllExpectations() + // Perform the lookup and expect the failed error messages to be returned. + errMsg, err := backend.LookupErrorMessagesByBlockID(context.Background(), blockId, block.Header.Height) + suite.Require().NoError(err) + suite.Require().Len(errMsg, len(expectedTxErrorMessages)) + for txID, expectedMessage := range expectedTxErrorMessages { + actualMessage, ok := errMsg[txID] + suite.Require().True(ok) + suite.Assert().Equal(expectedMessage, actualMessage) + } + suite.assertAllExpectations() + }) } // TestGetSystemTransaction_HappyPath tests that GetSystemTransaction call returns system chunk transaction. @@ -851,13 +990,9 @@ func (suite *Suite) TestGetSystemTransactionResult_HappyPath() { On("ByBlockID", block.ID()). Return(flow.ExecutionReceiptList{receipt1}, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // the connection factory should be used to get the execution node client params := suite.defaultBackendParams() - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() exeEventReq := &execproto.GetTransactionsByBlockIDRequest{ BlockId: blockID[:], @@ -1046,13 +1181,9 @@ func (suite *Suite) TestGetSystemTransactionResult_FailedEncodingConversion() { On("ByBlockID", block.ID()). Return(flow.ExecutionReceiptList{receipt1}, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // the connection factory should be used to get the execution node client params := suite.defaultBackendParams() - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() exeEventReq := &execproto.GetTransactionsByBlockIDRequest{ BlockId: blockID[:], @@ -1175,10 +1306,6 @@ func (suite *Suite) TestTransactionResultFromStorage() { suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) suite.snapshot.On("Head", mock.Anything).Return(block.Header, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // create a mock index reporter reporter := syncmock.NewIndexReporter(suite.T()) reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) @@ -1191,7 +1318,7 @@ func (suite *Suite) TestTransactionResultFromStorage() { // Set up the backend parameters and the backend instance params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() params.TxResultQueryMode = IndexQueryModeLocalOnly @@ -1266,10 +1393,6 @@ func (suite *Suite) TestTransactionByIndexFromStorage() { suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) suite.snapshot.On("Head", mock.Anything).Return(block.Header, nil) - // Create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // create a mock index reporter reporter := syncmock.NewIndexReporter(suite.T()) reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) @@ -1282,7 +1405,7 @@ func (suite *Suite) TestTransactionByIndexFromStorage() { // Set up the backend parameters and the backend instance params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() params.TxResultQueryMode = IndexQueryModeLocalOnly @@ -1363,10 +1486,6 @@ func (suite *Suite) TestTransactionResultsByBlockIDFromStorage() { suite.snapshot.On("Identities", mock.Anything).Return(fixedENIDs, nil) suite.snapshot.On("Head", mock.Anything).Return(block.Header, nil) - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - // create a mock index reporter reporter := syncmock.NewIndexReporter(suite.T()) reporter.On("LowestIndexedHeight").Return(block.Header.Height, nil) @@ -1379,7 +1498,7 @@ func (suite *Suite) TestTransactionResultsByBlockIDFromStorage() { // Set up the state and snapshot mocks and the backend instance params := suite.defaultBackendParams() // the connection factory should be used to get the execution node client - params.ConnFactory = connFactory + params.ConnFactory = suite.setupConnectionFactory() params.FixedExecutionNodeIDs = fixedENIDs.NodeIDs().Strings() params.EventsIndex = index.NewEventsIndex(indexReporter, suite.events) diff --git a/engine/access/rpc/backend/node_selector.go b/engine/access/rpc/backend/node_selector.go index c7d2ada5fb4..7d9ee12f64c 100644 --- a/engine/access/rpc/backend/node_selector.go +++ b/engine/access/rpc/backend/node_selector.go @@ -3,12 +3,10 @@ package backend import ( "fmt" + commonrpc "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/model/flow" ) -// maxNodesCnt is the maximum number of nodes that will be contacted to complete an API request. -const maxNodesCnt = 3 - // NodeSelector is an interface that represents the ability to select node identities that the access node is trying to reach. // It encapsulates the internal logic of node selection and provides a way to change implementations for different types // of nodes. Implementations of this interface should define the Next method, which returns the next node identity to be @@ -32,7 +30,7 @@ func (n *NodeSelectorFactory) SelectNodes(nodes flow.IdentitySkeletonList) (Node var err error // If the circuit breaker is disabled, the legacy logic should be used, which selects only a specified number of nodes. if !n.circuitBreakerEnabled { - nodes, err = nodes.Sample(maxNodesCnt) + nodes, err = nodes.Sample(commonrpc.MaxNodesCnt) if err != nil { return nil, fmt.Errorf("sampling failed: %w", err) } diff --git a/engine/access/rpc/backend/transactions_local_data_provider.go b/engine/access/rpc/backend/transactions_local_data_provider.go index f68fb35ed4b..921580452ec 100644 --- a/engine/access/rpc/backend/transactions_local_data_provider.go +++ b/engine/access/rpc/backend/transactions_local_data_provider.go @@ -31,18 +31,16 @@ type TransactionErrorMessage interface { // Expected errors during normal operation: // - InsufficientExecutionReceipts - found insufficient receipts for given block ID. // - status.Error - remote GRPC call to EN has failed. - LookupErrorMessageByTransactionID(ctx context.Context, blockID flow.Identifier, transactionID flow.Identifier) (string, error) + LookupErrorMessageByTransactionID(ctx context.Context, blockID flow.Identifier, height uint64, transactionID flow.Identifier) (string, error) // LookupErrorMessageByIndex is a function type for getting transaction error message by index. // Expected errors during normal operation: - // - status.Error[codes.NotFound] - transaction result for given block ID and tx index is not available. // - InsufficientExecutionReceipts - found insufficient receipts for given block ID. // - status.Error - remote GRPC call to EN has failed. LookupErrorMessageByIndex(ctx context.Context, blockID flow.Identifier, height uint64, index uint32) (string, error) // LookupErrorMessagesByBlockID is a function type for getting transaction error messages by block ID. // Expected errors during normal operation: - // - status.Error[codes.NotFound] - transaction results for given block ID are not available. // - InsufficientExecutionReceipts - found insufficient receipts for given block ID. // - status.Error - remote GRPC call to EN has failed. LookupErrorMessagesByBlockID(ctx context.Context, blockID flow.Identifier, height uint64) (map[flow.Identifier]string, error) @@ -84,7 +82,7 @@ func (t *TransactionsLocalDataProvider) GetTransactionResultFromStorage( var txErrorMessage string var txStatusCode uint = 0 if txResult.Failed { - txErrorMessage, err = t.txErrorMessages.LookupErrorMessageByTransactionID(ctx, blockID, transactionID) + txErrorMessage, err = t.txErrorMessages.LookupErrorMessageByTransactionID(ctx, blockID, block.Header.Height, transactionID) if err != nil { return nil, err } diff --git a/engine/common/rpc/utils.go b/engine/common/rpc/utils.go new file mode 100644 index 00000000000..60eceff5bb3 --- /dev/null +++ b/engine/common/rpc/utils.go @@ -0,0 +1,268 @@ +package rpc + +import ( + "context" + "fmt" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// minExecutionNodesCnt is the minimum number of execution nodes expected to have sent the execution receipt for a block +const minExecutionNodesCnt = 2 + +// maxAttemptsForExecutionReceipt is the maximum number of attempts to find execution receipts for a given block ID +const maxAttemptsForExecutionReceipt = 3 + +// MaxNodesCnt is the maximum number of nodes that will be contacted to complete an API request. +const MaxNodesCnt = 3 + +func IdentifierList(ids []string) (flow.IdentifierList, error) { + idList := make(flow.IdentifierList, len(ids)) + for i, idStr := range ids { + id, err := flow.HexStringToIdentifier(idStr) + if err != nil { + return nil, fmt.Errorf("failed to convert node id string %s to Flow Identifier: %w", id, err) + } + idList[i] = id + } + return idList, nil +} + +// ExecutionNodesForBlockID returns upto maxNodesCnt number of randomly chosen execution node identities +// which have executed the given block ID. +// If no such execution node is found, an InsufficientExecutionReceipts error is returned. +func ExecutionNodesForBlockID( + ctx context.Context, + blockID flow.Identifier, + executionReceipts storage.ExecutionReceipts, + state protocol.State, + log zerolog.Logger, + preferredENIdentifiers flow.IdentifierList, + fixedENIdentifiers flow.IdentifierList, +) (flow.IdentitySkeletonList, error) { + var ( + executorIDs flow.IdentifierList + err error + ) + + // check if the block ID is of the root block. If it is then don't look for execution receipts since they + // will not be present for the root block. + rootBlock := state.Params().FinalizedRoot() + + if rootBlock.ID() == blockID { + executorIdentities, err := state.Final().Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) + if err != nil { + return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) + } + executorIDs = executorIdentities.NodeIDs() + } else { + // try to find at least minExecutionNodesCnt execution node ids from the execution receipts for the given blockID + for attempt := 0; attempt < maxAttemptsForExecutionReceipt; attempt++ { + executorIDs, err = findAllExecutionNodes(blockID, executionReceipts, log) + if err != nil { + return nil, err + } + + if len(executorIDs) >= minExecutionNodesCnt { + break + } + + // log the attempt + log.Debug().Int("attempt", attempt).Int("max_attempt", maxAttemptsForExecutionReceipt). + Int("execution_receipts_found", len(executorIDs)). + Str("block_id", blockID.String()). + Msg("insufficient execution receipts") + + // if one or less execution receipts may have been received then re-query + // in the hope that more might have been received by now + + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-time.After(100 * time.Millisecond << time.Duration(attempt)): + // retry after an exponential backoff + } + } + + receiptCnt := len(executorIDs) + // if less than minExecutionNodesCnt execution receipts have been received so far, then return random ENs + if receiptCnt < minExecutionNodesCnt { + newExecutorIDs, err := state.AtBlockID(blockID).Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) + if err != nil { + return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) + } + executorIDs = newExecutorIDs.NodeIDs() + } + } + + // choose from the preferred or fixed execution nodes + subsetENs, err := chooseExecutionNodes(state, executorIDs, preferredENIdentifiers, fixedENIdentifiers) + if err != nil { + return nil, fmt.Errorf("failed to retreive execution IDs for block ID %v: %w", blockID, err) + } + + if len(subsetENs) == 0 { + return nil, fmt.Errorf("no matching execution node found for block ID %v", blockID) + } + + return subsetENs, nil +} + +// findAllExecutionNodes find all the execution nodes ids from the execution receipts that have been received for the +// given blockID +func findAllExecutionNodes( + blockID flow.Identifier, + executionReceipts storage.ExecutionReceipts, + log zerolog.Logger, +) (flow.IdentifierList, error) { + // lookup the receipt's storage with the block ID + allReceipts, err := executionReceipts.ByBlockID(blockID) + if err != nil { + return nil, fmt.Errorf("failed to retreive execution receipts for block ID %v: %w", blockID, err) + } + + executionResultMetaList := make(flow.ExecutionReceiptMetaList, 0, len(allReceipts)) + for _, r := range allReceipts { + executionResultMetaList = append(executionResultMetaList, r.Meta()) + } + executionResultGroupedMetaList := executionResultMetaList.GroupByResultID() + + // maximum number of matching receipts found so far for any execution result id + maxMatchedReceiptCnt := 0 + // execution result id key for the highest number of matching receipts in the identicalReceipts map + var maxMatchedReceiptResultID flow.Identifier + + // find the largest list of receipts which have the same result ID + for resultID, executionReceiptList := range executionResultGroupedMetaList { + currentMatchedReceiptCnt := executionReceiptList.Size() + if currentMatchedReceiptCnt > maxMatchedReceiptCnt { + maxMatchedReceiptCnt = currentMatchedReceiptCnt + maxMatchedReceiptResultID = resultID + } + } + + // if there are more than one execution result for the same block ID, log as error + if executionResultGroupedMetaList.NumberGroups() > 1 { + identicalReceiptsStr := fmt.Sprintf("%v", flow.GetIDs(allReceipts)) + log.Error(). + Str("block_id", blockID.String()). + Str("execution_receipts", identicalReceiptsStr). + Msg("execution receipt mismatch") + } + + // pick the largest list of matching receipts + matchingReceiptMetaList := executionResultGroupedMetaList.GetGroup(maxMatchedReceiptResultID) + + metaReceiptGroupedByExecutorID := matchingReceiptMetaList.GroupByExecutorID() + + // collect all unique execution node ids from the receipts + var executorIDs flow.IdentifierList + for executorID := range metaReceiptGroupedByExecutorID { + executorIDs = append(executorIDs, executorID) + } + + return executorIDs, nil +} + +// chooseExecutionNodes finds the subset of execution nodes defined in the identity table by first +// choosing the preferred execution nodes which have executed the transaction. If no such preferred +// execution nodes are found, then the fixed execution nodes defined in the identity table are returned +// If neither preferred nor fixed nodes are defined, then all execution node matching the executor IDs are returned. +// e.g. If execution nodes in identity table are {1,2,3,4}, preferred ENs are defined as {2,3,4} +// and the executor IDs is {1,2,3}, then {2, 3} is returned as the chosen subset of ENs +func chooseExecutionNodes( + state protocol.State, + executorIDs flow.IdentifierList, + preferredENIdentifiers flow.IdentifierList, + fixedENIdentifiers flow.IdentifierList, +) (flow.IdentitySkeletonList, error) { + allENs, err := state.Final().Identities(filter.HasRole[flow.Identity](flow.RoleExecution)) + if err != nil { + return nil, fmt.Errorf("failed to retrieve all execution IDs: %w", err) + } + + // choose from preferred EN IDs + if len(preferredENIdentifiers) > 0 { + chosenIDs := ChooseFromPreferredENIDs(allENs, executorIDs, preferredENIdentifiers) + return chosenIDs.ToSkeleton(), nil + } + + // if no preferred EN ID is found, then choose from the fixed EN IDs + if len(fixedENIdentifiers) > 0 { + // choose fixed ENs which have executed the transaction + chosenIDs := allENs.Filter(filter.And( + filter.HasNodeID[flow.Identity](fixedENIdentifiers...), + filter.HasNodeID[flow.Identity](executorIDs...), + )) + if len(chosenIDs) > 0 { + return chosenIDs.ToSkeleton(), nil + } + // if no such ENs are found, then just choose all fixed ENs + chosenIDs = allENs.Filter(filter.HasNodeID[flow.Identity](fixedENIdentifiers...)) + return chosenIDs.ToSkeleton(), nil + } + + // if no preferred or fixed ENs have been specified, then return all executor IDs i.e., no preference at all + return allENs.Filter(filter.HasNodeID[flow.Identity](executorIDs...)).ToSkeleton(), nil +} + +// ChooseFromPreferredENIDs finds the subset of execution nodes if preferred execution nodes are defined. +// If preferredENIdentifiers is set and there are less than maxNodesCnt nodes selected, than the list is padded up to +// maxNodesCnt nodes using the following order: +// 1. Use any EN with a receipt. +// 2. Use any preferred node not already selected. +// 3. Use any EN not already selected. +func ChooseFromPreferredENIDs(allENs flow.IdentityList, + executorIDs flow.IdentifierList, + preferredENIdentifiers flow.IdentifierList, +) flow.IdentityList { + var chosenIDs flow.IdentityList + + // filter for both preferred and executor IDs + chosenIDs = allENs.Filter(filter.And( + filter.HasNodeID[flow.Identity](preferredENIdentifiers...), + filter.HasNodeID[flow.Identity](executorIDs...), + )) + + if len(chosenIDs) >= MaxNodesCnt { + return chosenIDs + } + + // function to add nodes to chosenIDs if they are not already included + addIfNotExists := func(candidates flow.IdentityList) { + for _, en := range candidates { + _, exists := chosenIDs.ByNodeID(en.NodeID) + if !exists { + chosenIDs = append(chosenIDs, en) + if len(chosenIDs) >= MaxNodesCnt { + return + } + } + } + } + + // add any EN with a receipt + receiptENs := allENs.Filter(filter.HasNodeID[flow.Identity](executorIDs...)) + addIfNotExists(receiptENs) + if len(chosenIDs) >= MaxNodesCnt { + return chosenIDs + } + + // add any preferred node not already selected + preferredENs := allENs.Filter(filter.HasNodeID[flow.Identity](preferredENIdentifiers...)) + addIfNotExists(preferredENs) + if len(chosenIDs) >= MaxNodesCnt { + return chosenIDs + } + + // add any EN not already selected + addIfNotExists(allENs) + + return chosenIDs +} diff --git a/integration/tests/access/cohort3/access_store_tx_error_messages_test.go b/integration/tests/access/cohort3/access_store_tx_error_messages_test.go new file mode 100644 index 00000000000..23a0b37f551 --- /dev/null +++ b/integration/tests/access/cohort3/access_store_tx_error_messages_test.go @@ -0,0 +1,199 @@ +package cohort3 + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + sdk "github.com/onflow/flow-go-sdk" + sdkcrypto "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go-sdk/templates" + + "github.com/onflow/flow-go/integration/convert" + "github.com/onflow/flow-go/integration/testnet" + "github.com/onflow/flow-go/integration/tests/lib" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage/badger" +) + +const maxReceiptHeightMetric = "access_ingestion_max_receipt_height" + +func TestAccessStoreTxErrorMessages(t *testing.T) { + suite.Run(t, new(AccessStoreTxErrorMessagesSuite)) +} + +// AccessStoreTxErrorMessagesSuite tests the access for storing transaction error messages. +type AccessStoreTxErrorMessagesSuite struct { + suite.Suite + + log zerolog.Logger + + // root context for the current test + ctx context.Context + cancel context.CancelFunc + + net *testnet.FlowNetwork +} + +func (s *AccessStoreTxErrorMessagesSuite) TearDownTest() { + s.log.Info().Msg("================> Start TearDownTest") + s.net.Remove() + s.cancel() + s.log.Info().Msg("================> Finish TearDownTest") +} + +// SetupTest sets up the test suite by starting the network. +// The access are started with correct parameters and store transaction error messages. +func (s *AccessStoreTxErrorMessagesSuite) SetupTest() { + defaultAccess := testnet.NewNodeConfig( + flow.RoleAccess, + testnet.WithLogLevel(zerolog.FatalLevel), + ) + + storeTxAccess := testnet.NewNodeConfig( + flow.RoleAccess, + testnet.WithLogLevel(zerolog.InfoLevel), + testnet.WithAdditionalFlagf("--store-tx-result-error-messages=true"), + testnet.WithMetricsServer(), + ) + + consensusConfigs := []func(config *testnet.NodeConfig){ + // `cruise-ctl-fallback-proposal-duration` is set to 250ms instead to of 100ms + // to purposely slow down the block rate. This is needed since the crypto module + // update providing faster BLS operations. + testnet.WithAdditionalFlag("--cruise-ctl-fallback-proposal-duration=250ms"), + testnet.WithAdditionalFlagf("--required-verification-seal-approvals=%d", 1), + testnet.WithAdditionalFlagf("--required-construction-seal-approvals=%d", 1), + testnet.WithLogLevel(zerolog.FatalLevel), + } + + nodeConfigs := []testnet.NodeConfig{ + testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleConsensus, consensusConfigs...), + testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel)), + + defaultAccess, + storeTxAccess, + } + + // prepare the network + conf := testnet.NewNetworkConfig("access_store_tx_error_messages_test", nodeConfigs) + s.net = testnet.PrepareFlowNetwork(s.T(), conf, flow.Localnet) + + // start the network + s.ctx, s.cancel = context.WithCancel(context.Background()) + + s.net.Start(s.ctx) +} + +// TestAccessStoreTxErrorMessages verifies that transaction result error messages +// are stored correctly in the database by sending a transaction, generating an error, +// and checking if the error message is properly stored and retrieved from the database. +func (s *AccessStoreTxErrorMessagesSuite) TestAccessStoreTxErrorMessages() { + // Create and send a transaction that will result in an error. + txResult := s.createAndSendTxWithTxError() + + txBlockID := convert.IDFromSDK(txResult.BlockID) + txID := convert.IDFromSDK(txResult.TransactionID) + expectedTxResultErrorMessage := txResult.Error.Error() + + accessContainerName := "access_2" + + // Wait until execution receipts are handled, transaction error messages are stored. + s.Eventually(func() bool { + value, err := s.getMaxReceiptHeight(accessContainerName) + return err == nil && value > txResult.BlockHeight + }, 60*time.Second, 1*time.Second) + + // Stop the network containers before checking the results. + s.net.StopContainers() + + // Get the access node and open the protocol DB. + accessNode := s.net.ContainerByName(accessContainerName) + // setup storage objects needed to get the execution data id + anDB, err := accessNode.DB() + require.NoError(s.T(), err, "could not open db") + + metrics := metrics.NewNoopCollector() + anTxErrorMessages := badger.NewTransactionResultErrorMessages(metrics, anDB, badger.DefaultCacheSize) + + // Fetch the stored error message by block ID and transaction ID. + errMsgResult, err := anTxErrorMessages.ByBlockIDTransactionID(txBlockID, txID) + s.Require().NoError(err) + + // Verify that the error message retrieved matches the expected values. + s.Require().Equal(txID, errMsgResult.TransactionID) + s.Require().Equal(expectedTxResultErrorMessage, errMsgResult.ErrorMessage) +} + +// createAndSendTxWithTxError creates and sends a transaction that will result in an error. +// This function creates a new account, causing an error during execution. +func (s *AccessStoreTxErrorMessagesSuite) createAndSendTxWithTxError() *sdk.TransactionResult { + // prepare environment to create a new account + serviceAccountClient, err := s.net.ContainerByName(testnet.PrimaryAN).TestnetClient() + s.Require().NoError(err) + + latestBlockID, err := serviceAccountClient.GetLatestBlockID(s.ctx) + s.Require().NoError(err) + + // create new account to deploy Counter to + accountPrivateKey := lib.RandomPrivateKey() + + accountKey := sdk.NewAccountKey(). + FromPrivateKey(accountPrivateKey). + SetHashAlgo(sdkcrypto.SHA3_256). + SetWeight(sdk.AccountKeyWeightThreshold) + + serviceAddress := sdk.Address(serviceAccountClient.Chain.ServiceAddress()) + + // Generate the account creation transaction + createAccountTx, err := templates.CreateAccount( + []*sdk.AccountKey{accountKey}, + []templates.Contract{}, serviceAddress) + s.Require().NoError(err) + + // Generate the account creation transaction + createAccountTx. + SetReferenceBlockID(sdk.Identifier(latestBlockID)). + SetProposalKey(serviceAddress, 1, serviceAccountClient.GetAndIncrementSeqNumber()). + SetPayer(serviceAddress). + SetComputeLimit(9999) + + // Sign and send the transaction. + err = serviceAccountClient.SignAndSendTransaction(s.ctx, createAccountTx) + s.Require().NoError(err) + + // Wait for the transaction to be sealed and return the transaction result. + accountCreationTxRes, err := serviceAccountClient.WaitForSealed(s.ctx, createAccountTx.ID()) + s.Require().NoError(err) + + return accountCreationTxRes +} + +// getMaxReceiptHeight retrieves the maximum receipt height for a given container by +// querying the metrics endpoint. This is used to confirm that the transaction receipts +// have been processed. +func (s *AccessStoreTxErrorMessagesSuite) getMaxReceiptHeight(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, maxReceiptHeightMetric) + + // If no values are found in the metrics, return an error. + if len(values) == 0 { + return 0, fmt.Errorf("no values found") + } + + // Return the first value found as the max receipt height. + return uint64(values[0].GetGauge().GetValue()), nil +} diff --git a/model/flow/transaction_result.go b/model/flow/transaction_result.go index 3a327118165..f4506c47039 100644 --- a/model/flow/transaction_result.go +++ b/model/flow/transaction_result.go @@ -52,3 +52,17 @@ type LightTransactionResult struct { // ComputationUsed is amount of computation used while executing the transaction. ComputationUsed uint64 } + +// TransactionResultErrorMessage represents an error message resulting from a transaction's execution. +// This struct holds the transaction's ID, its index, any error message generated during execution, +// and the identifier of the execution node that provided the error message. +type TransactionResultErrorMessage struct { + // TransactionID is the ID of the transaction this result error was emitted from. + TransactionID Identifier + // Index is the index of the transaction this result error was emitted from. + Index uint32 + // ErrorMessage contains the error message of any error that may have occurred when the transaction was executed. + ErrorMessage string + // Executor node ID of the execution node that the message was received from. + ExecutorID Identifier +} diff --git a/module/jobqueue.go b/module/jobqueue.go index a87208a94f4..9150956504b 100644 --- a/module/jobqueue.go +++ b/module/jobqueue.go @@ -13,8 +13,9 @@ const ( ConsumeProgressExecutionDataIndexerBlockHeight = "ConsumeProgressExecutionDataIndexerBlockHeight" - ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" - ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" + ConsumeProgressIngestionEngineBlockHeight = "ConsumeProgressIngestionEngineBlockHeight" + ConsumeProgressEngineTxErrorMessagesBlockHeight = "ConsumeProgressEngineTxErrorMessagesBlockHeight" + ConsumeProgressLastFullBlockHeight = "ConsumeProgressLastFullBlockHeight" ) // JobID is a unique ID of the job. diff --git a/module/metrics/labels.go b/module/metrics/labels.go index 82260ca3c5d..20b66ad7d68 100644 --- a/module/metrics/labels.go +++ b/module/metrics/labels.go @@ -109,28 +109,30 @@ const ( ResourceNetworkingUnicastDialConfigCache = "unicast_dial_config_cache" ResourceNetworkingGossipsubDuplicateMessagesTrackerCache = "gossipsub_duplicate_messages_tracker_cache" - ResourceFollowerPendingBlocksCache = "follower_pending_block_cache" // follower engine - ResourceFollowerLoopCertifiedBlocksChannel = "follower_loop_certified_blocks_channel" // follower loop, certified blocks buffered channel - ResourceClusterBlockProposalQueue = "cluster_compliance_proposal_queue" // collection node, compliance engine - ResourceTransactionIngestQueue = "ingest_transaction_queue" // collection node, ingest engine - ResourceBeaconKey = "beacon-key" // consensus node, DKG engine - ResourceDKGMessage = "dkg_private_message" // consensus, DKG messaging engine - ResourceApprovalQueue = "sealing_approval_queue" // consensus node, sealing engine - ResourceReceiptQueue = "sealing_receipt_queue" // consensus node, sealing engine - ResourceApprovalResponseQueue = "sealing_approval_response_queue" // consensus node, sealing engine - ResourceBlockResponseQueue = "compliance_block_response_queue" // consensus node, compliance engine - ResourceBlockProposalQueue = "compliance_proposal_queue" // consensus node, compliance engine - ResourceBlockVoteQueue = "vote_aggregator_queue" // consensus/collection node, vote aggregator - ResourceTimeoutObjectQueue = "timeout_aggregator_queue" // consensus/collection node, timeout aggregator - ResourceCollectionGuaranteesQueue = "ingestion_col_guarantee_queue" // consensus node, ingestion engine - ResourceChunkDataPack = "chunk_data_pack" // execution node - ResourceChunkDataPackRequests = "chunk_data_pack_request" // execution node - ResourceEvents = "events" // execution node - ResourceServiceEvents = "service_events" // execution node - ResourceTransactionResults = "transaction_results" // execution node - ResourceTransactionResultIndices = "transaction_result_indices" // execution node - ResourceTransactionResultByBlock = "transaction_result_by_block" // execution node - ResourceExecutionDataCache = "execution_data_cache" // access node + ResourceFollowerPendingBlocksCache = "follower_pending_block_cache" // follower engine + ResourceFollowerLoopCertifiedBlocksChannel = "follower_loop_certified_blocks_channel" // follower loop, certified blocks buffered channel + ResourceClusterBlockProposalQueue = "cluster_compliance_proposal_queue" // collection node, compliance engine + ResourceTransactionIngestQueue = "ingest_transaction_queue" // collection node, ingest engine + ResourceBeaconKey = "beacon-key" // consensus node, DKG engine + ResourceDKGMessage = "dkg_private_message" // consensus, DKG messaging engine + ResourceApprovalQueue = "sealing_approval_queue" // consensus node, sealing engine + ResourceReceiptQueue = "sealing_receipt_queue" // consensus node, sealing engine + ResourceApprovalResponseQueue = "sealing_approval_response_queue" // consensus node, sealing engine + ResourceBlockResponseQueue = "compliance_block_response_queue" // consensus node, compliance engine + ResourceBlockProposalQueue = "compliance_proposal_queue" // consensus node, compliance engine + ResourceBlockVoteQueue = "vote_aggregator_queue" // consensus/collection node, vote aggregator + ResourceTimeoutObjectQueue = "timeout_aggregator_queue" // consensus/collection node, timeout aggregator + ResourceCollectionGuaranteesQueue = "ingestion_col_guarantee_queue" // consensus node, ingestion engine + ResourceChunkDataPack = "chunk_data_pack" // execution node + ResourceChunkDataPackRequests = "chunk_data_pack_request" // execution node + ResourceEvents = "events" // execution node + ResourceServiceEvents = "service_events" // execution node + ResourceTransactionResults = "transaction_results" // execution node + ResourceTransactionResultIndices = "transaction_result_indices" // execution node + ResourceTransactionResultErrorMessages = "transaction_result_error_messages" // execution node + ResourceTransactionResultErrorMessagesIndices = "transaction_result_error_messages_indices" // execution node + ResourceTransactionResultByBlock = "transaction_result_by_block" // execution node + ResourceExecutionDataCache = "execution_data_cache" // access node ) const ( diff --git a/storage/all.go b/storage/all.go index 2d0075aa8be..26bb89bd454 100644 --- a/storage/all.go +++ b/storage/all.go @@ -2,26 +2,27 @@ package storage // All includes all the storage modules type All struct { - Headers Headers - Guarantees Guarantees - Seals Seals - Index Index - Payloads Payloads - Blocks Blocks - QuorumCertificates QuorumCertificates - Setups EpochSetups - EpochCommits EpochCommits - Results ExecutionResults - Receipts ExecutionReceipts - ChunkDataPacks ChunkDataPacks - Commits Commits - Transactions Transactions - LightTransactionResults LightTransactionResults - TransactionResults TransactionResults - Collections Collections - Events Events - EpochProtocolStateEntries EpochProtocolStateEntries - ProtocolKVStore ProtocolKVStore - VersionBeacons VersionBeacons - RegisterIndex RegisterIndex + Headers Headers + Guarantees Guarantees + Seals Seals + Index Index + Payloads Payloads + Blocks Blocks + QuorumCertificates QuorumCertificates + Setups EpochSetups + EpochCommits EpochCommits + Results ExecutionResults + Receipts ExecutionReceipts + ChunkDataPacks ChunkDataPacks + Commits Commits + Transactions Transactions + LightTransactionResults LightTransactionResults + TransactionResults TransactionResults + TransactionResultErrorMessages TransactionResultErrorMessages + Collections Collections + Events Events + EpochProtocolStateEntries EpochProtocolStateEntries + ProtocolKVStore ProtocolKVStore + VersionBeacons VersionBeacons + RegisterIndex RegisterIndex } diff --git a/storage/badger/operation/prefix.go b/storage/badger/operation/prefix.go index ad909faf394..4113e1fcd3f 100644 --- a/storage/badger/operation/prefix.go +++ b/storage/badger/operation/prefix.go @@ -89,20 +89,22 @@ const ( codeJobQueuePointer = 72 // legacy codes (should be cleaned up) - codeChunkDataPack = 100 - codeCommit = 101 - codeEvent = 102 - codeExecutionStateInteractions = 103 - codeTransactionResult = 104 - codeFinalizedCluster = 105 - codeServiceEvent = 106 - codeTransactionResultIndex = 107 - codeLightTransactionResult = 108 - codeLightTransactionResultIndex = 109 - codeIndexCollection = 200 - codeIndexExecutionResultByBlock = 202 - codeIndexCollectionByTransaction = 203 - codeIndexResultApprovalByChunk = 204 + codeChunkDataPack = 100 + codeCommit = 101 + codeEvent = 102 + codeExecutionStateInteractions = 103 + codeTransactionResult = 104 + codeFinalizedCluster = 105 + codeServiceEvent = 106 + codeTransactionResultIndex = 107 + codeLightTransactionResult = 108 + codeLightTransactionResultIndex = 109 + codeTransactionResultErrorMessage = 110 + codeTransactionResultErrorMessageIndex = 111 + codeIndexCollection = 200 + codeIndexExecutionResultByBlock = 202 + codeIndexCollectionByTransaction = 203 + codeIndexResultApprovalByChunk = 204 // TEMPORARY codes blockedNodeIDs = 205 // manual override for adding node IDs to list of ejected nodes, applies to networking layer only diff --git a/storage/badger/operation/transaction_results.go b/storage/badger/operation/transaction_results.go index 7d5fcf47086..c4264640364 100644 --- a/storage/badger/operation/transaction_results.go +++ b/storage/badger/operation/transaction_results.go @@ -120,3 +120,51 @@ func LookupLightTransactionResultsByBlockIDUsingIndex(blockID flow.Identifier, t return traverse(makePrefix(codeLightTransactionResultIndex, blockID), txErrIterFunc) } + +// BatchInsertTransactionResultErrorMessage inserts a transaction result error message by block ID and transaction ID +// into the database using a batch write. +func BatchInsertTransactionResultErrorMessage(blockID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) func(batch *badger.WriteBatch) error { + return batchWrite(makePrefix(codeTransactionResultErrorMessage, blockID, transactionResultErrorMessage.TransactionID), transactionResultErrorMessage) +} + +// BatchIndexTransactionResultErrorMessage indexes a transaction result error message by index within the block using a +// batch write. +func BatchIndexTransactionResultErrorMessage(blockID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) func(batch *badger.WriteBatch) error { + return batchWrite(makePrefix(codeTransactionResultErrorMessageIndex, blockID, transactionResultErrorMessage.Index), transactionResultErrorMessage) +} + +// RetrieveTransactionResultErrorMessage retrieves a transaction result error message by block ID and transaction ID. +func RetrieveTransactionResultErrorMessage(blockID flow.Identifier, transactionID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) func(*badger.Txn) error { + return retrieve(makePrefix(codeTransactionResultErrorMessage, blockID, transactionID), transactionResultErrorMessage) +} + +// RetrieveTransactionResultErrorMessageByIndex retrieves a transaction result error message by block ID and index. +func RetrieveTransactionResultErrorMessageByIndex(blockID flow.Identifier, txIndex uint32, transactionResultErrorMessage *flow.TransactionResultErrorMessage) func(*badger.Txn) error { + return retrieve(makePrefix(codeTransactionResultErrorMessageIndex, blockID, txIndex), transactionResultErrorMessage) +} + +// TransactionResultErrorMessagesExists checks whether tx result error messages exist in the database. +func TransactionResultErrorMessagesExists(blockID flow.Identifier, blockExists *bool) func(*badger.Txn) error { + return exists(makePrefix(codeTransactionResultErrorMessageIndex, blockID), blockExists) +} + +// LookupTransactionResultErrorMessagesByBlockIDUsingIndex retrieves all tx result error messages for a block, by using +// tx_index index. This correctly handles cases of duplicate transactions within block. +func LookupTransactionResultErrorMessagesByBlockIDUsingIndex(blockID flow.Identifier, txResultErrorMessages *[]flow.TransactionResultErrorMessage) func(*badger.Txn) error { + txErrIterFunc := func() (checkFunc, createFunc, handleFunc) { + check := func(_ []byte) bool { + return true + } + var val flow.TransactionResultErrorMessage + create := func() interface{} { + return &val + } + handle := func() error { + *txResultErrorMessages = append(*txResultErrorMessages, val) + return nil + } + return check, create, handle + } + + return traverse(makePrefix(codeTransactionResultErrorMessageIndex, blockID), txErrIterFunc) +} diff --git a/storage/badger/transaction_result_error_messages.go b/storage/badger/transaction_result_error_messages.go new file mode 100644 index 00000000000..e2abf659d5e --- /dev/null +++ b/storage/badger/transaction_result_error_messages.go @@ -0,0 +1,212 @@ +package badger + +import ( + "fmt" + + "github.com/dgraph-io/badger/v2" + + "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/storage" + "github.com/onflow/flow-go/storage/badger/operation" +) + +var _ storage.TransactionResultErrorMessages = (*TransactionResultErrorMessages)(nil) + +type TransactionResultErrorMessages struct { + db *badger.DB + cache *Cache[string, flow.TransactionResultErrorMessage] + indexCache *Cache[string, flow.TransactionResultErrorMessage] + blockCache *Cache[string, []flow.TransactionResultErrorMessage] +} + +func NewTransactionResultErrorMessages(collector module.CacheMetrics, db *badger.DB, transactionResultsCacheSize uint) *TransactionResultErrorMessages { + retrieve := func(key string) func(tx *badger.Txn) (flow.TransactionResultErrorMessage, error) { + var txResultErrMsg flow.TransactionResultErrorMessage + return func(tx *badger.Txn) (flow.TransactionResultErrorMessage, error) { + + blockID, txID, err := KeyToBlockIDTransactionID(key) + if err != nil { + return flow.TransactionResultErrorMessage{}, fmt.Errorf("could not convert key: %w", err) + } + + err = operation.RetrieveTransactionResultErrorMessage(blockID, txID, &txResultErrMsg)(tx) + if err != nil { + return flow.TransactionResultErrorMessage{}, handleError(err, flow.TransactionResultErrorMessage{}) + } + return txResultErrMsg, nil + } + } + retrieveIndex := func(key string) func(tx *badger.Txn) (flow.TransactionResultErrorMessage, error) { + var txResultErrMsg flow.TransactionResultErrorMessage + return func(tx *badger.Txn) (flow.TransactionResultErrorMessage, error) { + + blockID, txIndex, err := KeyToBlockIDIndex(key) + if err != nil { + return flow.TransactionResultErrorMessage{}, fmt.Errorf("could not convert index key: %w", err) + } + + err = operation.RetrieveTransactionResultErrorMessageByIndex(blockID, txIndex, &txResultErrMsg)(tx) + if err != nil { + return flow.TransactionResultErrorMessage{}, handleError(err, flow.TransactionResultErrorMessage{}) + } + return txResultErrMsg, nil + } + } + retrieveForBlock := func(key string) func(tx *badger.Txn) ([]flow.TransactionResultErrorMessage, error) { + var txResultErrMsg []flow.TransactionResultErrorMessage + return func(tx *badger.Txn) ([]flow.TransactionResultErrorMessage, error) { + + blockID, err := KeyToBlockID(key) + if err != nil { + return nil, fmt.Errorf("could not convert index key: %w", err) + } + + err = operation.LookupTransactionResultErrorMessagesByBlockIDUsingIndex(blockID, &txResultErrMsg)(tx) + if err != nil { + return nil, handleError(err, flow.TransactionResultErrorMessage{}) + } + return txResultErrMsg, nil + } + } + + return &TransactionResultErrorMessages{ + db: db, + cache: newCache[string, flow.TransactionResultErrorMessage](collector, metrics.ResourceTransactionResultErrorMessages, + withLimit[string, flow.TransactionResultErrorMessage](transactionResultsCacheSize), + withStore(noopStore[string, flow.TransactionResultErrorMessage]), + withRetrieve(retrieve), + ), + indexCache: newCache[string, flow.TransactionResultErrorMessage](collector, metrics.ResourceTransactionResultErrorMessagesIndices, + withLimit[string, flow.TransactionResultErrorMessage](transactionResultsCacheSize), + withStore(noopStore[string, flow.TransactionResultErrorMessage]), + withRetrieve(retrieveIndex), + ), + blockCache: newCache[string, []flow.TransactionResultErrorMessage](collector, metrics.ResourceTransactionResultErrorMessagesIndices, + withLimit[string, []flow.TransactionResultErrorMessage](transactionResultsCacheSize), + withStore(noopStore[string, []flow.TransactionResultErrorMessage]), + withRetrieve(retrieveForBlock), + ), + } +} + +// Store will store transaction result error messages for the given block ID. +// +// No errors are expected during normal operation. +func (t *TransactionResultErrorMessages) Store(blockID flow.Identifier, transactionResultErrorMessages []flow.TransactionResultErrorMessage) error { + batch := NewBatch(t.db) + + err := t.batchStore(blockID, transactionResultErrorMessages, batch) + if err != nil { + return err + } + + err = batch.Flush() + if err != nil { + return fmt.Errorf("cannot flush batch: %w", err) + } + + return nil +} + +// Exists returns true if transaction result error messages for the given ID have been stored. +// +// No errors are expected during normal operation. +func (t *TransactionResultErrorMessages) Exists(blockID flow.Identifier) (bool, error) { + // if the block is in the cache, return true + key := KeyFromBlockID(blockID) + if ok := t.blockCache.IsCached(key); ok { + return ok, nil + } + // otherwise, check badger store + var exists bool + err := t.db.View(operation.TransactionResultErrorMessagesExists(blockID, &exists)) + if err != nil { + return false, fmt.Errorf("could not check existence: %w", err) + } + return exists, nil +} + +// BatchStore inserts a batch of transaction result error messages into a batch +// +// No errors are expected during normal operation. +func (t *TransactionResultErrorMessages) batchStore( + blockID flow.Identifier, + transactionResultErrorMessages []flow.TransactionResultErrorMessage, + batch storage.BatchStorage, +) error { + writeBatch := batch.GetWriter() + + for _, result := range transactionResultErrorMessages { + err := operation.BatchInsertTransactionResultErrorMessage(blockID, &result)(writeBatch) + if err != nil { + return fmt.Errorf("cannot batch insert tx result error message: %w", err) + } + + err = operation.BatchIndexTransactionResultErrorMessage(blockID, &result)(writeBatch) + if err != nil { + return fmt.Errorf("cannot batch index tx result error message: %w", err) + } + } + + batch.OnSucceed(func() { + for _, result := range transactionResultErrorMessages { + key := KeyFromBlockIDTransactionID(blockID, result.TransactionID) + // cache for each transaction, so that it's faster to retrieve + t.cache.Insert(key, result) + + keyIndex := KeyFromBlockIDIndex(blockID, result.Index) + t.indexCache.Insert(keyIndex, result) + } + + key := KeyFromBlockID(blockID) + t.blockCache.Insert(key, transactionResultErrorMessages) + }) + return nil +} + +// ByBlockIDTransactionID returns the transaction result error message for the given block ID and transaction ID +// +// Expected errors during normal operation: +// - `storage.ErrNotFound` if no transaction error message is known at given block and transaction id. +func (t *TransactionResultErrorMessages) ByBlockIDTransactionID(blockID flow.Identifier, transactionID flow.Identifier) (*flow.TransactionResultErrorMessage, error) { + tx := t.db.NewTransaction(false) + defer tx.Discard() + key := KeyFromBlockIDTransactionID(blockID, transactionID) + transactionResultErrorMessage, err := t.cache.Get(key)(tx) + if err != nil { + return nil, err + } + return &transactionResultErrorMessage, nil +} + +// ByBlockIDTransactionIndex returns the transaction result error message for the given blockID and transaction index +// +// Expected errors during normal operation: +// - `storage.ErrNotFound` if no transaction error message is known at given block and transaction index. +func (t *TransactionResultErrorMessages) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) (*flow.TransactionResultErrorMessage, error) { + tx := t.db.NewTransaction(false) + defer tx.Discard() + key := KeyFromBlockIDIndex(blockID, txIndex) + transactionResultErrorMessage, err := t.indexCache.Get(key)(tx) + if err != nil { + return nil, err + } + return &transactionResultErrorMessage, nil +} + +// ByBlockID gets all transaction result error messages for a block, ordered by transaction index. +// Note: This method will return an empty slice both if the block is not indexed yet and if the block does not have any errors. +// +// No errors are expected during normal operation. +func (t *TransactionResultErrorMessages) ByBlockID(blockID flow.Identifier) ([]flow.TransactionResultErrorMessage, error) { + tx := t.db.NewTransaction(false) + defer tx.Discard() + key := KeyFromBlockID(blockID) + transactionResultErrorMessages, err := t.blockCache.Get(key)(tx) + if err != nil { + return nil, err + } + return transactionResultErrorMessages, nil +} diff --git a/storage/badger/transaction_result_error_messages_test.go b/storage/badger/transaction_result_error_messages_test.go new file mode 100644 index 00000000000..e21e8aaf348 --- /dev/null +++ b/storage/badger/transaction_result_error_messages_test.go @@ -0,0 +1,110 @@ +package badger_test + +import ( + "fmt" + "testing" + + "github.com/dgraph-io/badger/v2" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "golang.org/x/exp/rand" + + "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/utils/unittest" + + bstorage "github.com/onflow/flow-go/storage/badger" +) + +func TestStoringTransactionResultErrorMessages(t *testing.T) { + unittest.RunWithBadgerDB(t, func(db *badger.DB) { + metrics := metrics.NewNoopCollector() + store := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + + blockID := unittest.IdentifierFixture() + + // test db Exists by block id + exists, err := store.Exists(blockID) + require.NoError(t, err) + require.False(t, exists) + + // check retrieving by ByBlockID + messages, err := store.ByBlockID(blockID) + require.NoError(t, err) + require.Nil(t, messages) + + txErrorMessages := make([]flow.TransactionResultErrorMessage, 0) + for i := 0; i < 10; i++ { + expected := flow.TransactionResultErrorMessage{ + TransactionID: unittest.IdentifierFixture(), + ErrorMessage: fmt.Sprintf("a runtime error %d", i), + ExecutorID: unittest.IdentifierFixture(), + Index: rand.Uint32(), + } + txErrorMessages = append(txErrorMessages, expected) + } + err = store.Store(blockID, txErrorMessages) + require.NoError(t, err) + + // test db Exists by block id + exists, err = store.Exists(blockID) + require.NoError(t, err) + require.True(t, exists) + + // check retrieving by ByBlockIDTransactionID + for _, txErrorMessage := range txErrorMessages { + actual, err := store.ByBlockIDTransactionID(blockID, txErrorMessage.TransactionID) + require.NoError(t, err) + assert.Equal(t, txErrorMessage, *actual) + } + + // check retrieving by ByBlockIDTransactionIndex + for _, txErrorMessage := range txErrorMessages { + actual, err := store.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) + require.NoError(t, err) + assert.Equal(t, txErrorMessage, *actual) + } + + // check retrieving by ByBlockID + actual, err := store.ByBlockID(blockID) + require.NoError(t, err) + assert.Equal(t, txErrorMessages, actual) + + // test loading from database + newStore := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + for _, txErrorMessage := range txErrorMessages { + actual, err := newStore.ByBlockIDTransactionID(blockID, txErrorMessage.TransactionID) + require.NoError(t, err) + assert.Equal(t, txErrorMessage, *actual) + } + + // check retrieving by index from both cache and db + for i, txErrorMessage := range txErrorMessages { + actual, err := store.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) + require.NoError(t, err) + assert.Equal(t, txErrorMessages[i], *actual) + + actual, err = newStore.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) + require.NoError(t, err) + assert.Equal(t, txErrorMessages[i], *actual) + } + }) +} + +func TestReadingNotStoreTransactionResultErrorMessage(t *testing.T) { + unittest.RunWithBadgerDB(t, func(db *badger.DB) { + metrics := metrics.NewNoopCollector() + store := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + + blockID := unittest.IdentifierFixture() + txID := unittest.IdentifierFixture() + txIndex := rand.Uint32() + + _, err := store.ByBlockIDTransactionID(blockID, txID) + assert.ErrorIs(t, err, storage.ErrNotFound) + + _, err = store.ByBlockIDTransactionIndex(blockID, txIndex) + assert.ErrorIs(t, err, storage.ErrNotFound) + }) +} diff --git a/storage/mock/transaction_result_error_messages.go b/storage/mock/transaction_result_error_messages.go new file mode 100644 index 00000000000..c1bebf7f326 --- /dev/null +++ b/storage/mock/transaction_result_error_messages.go @@ -0,0 +1,163 @@ +// Code generated by mockery v2.43.2. DO NOT EDIT. + +package mock + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// TransactionResultErrorMessages is an autogenerated mock type for the TransactionResultErrorMessages type +type TransactionResultErrorMessages struct { + mock.Mock +} + +// ByBlockID provides a mock function with given fields: id +func (_m *TransactionResultErrorMessages) ByBlockID(id flow.Identifier) ([]flow.TransactionResultErrorMessage, error) { + ret := _m.Called(id) + + if len(ret) == 0 { + panic("no return value specified for ByBlockID") + } + + var r0 []flow.TransactionResultErrorMessage + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) ([]flow.TransactionResultErrorMessage, error)); ok { + return rf(id) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) []flow.TransactionResultErrorMessage); ok { + r0 = rf(id) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]flow.TransactionResultErrorMessage) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(id) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// ByBlockIDTransactionID provides a mock function with given fields: blockID, transactionID +func (_m *TransactionResultErrorMessages) ByBlockIDTransactionID(blockID flow.Identifier, transactionID flow.Identifier) (*flow.TransactionResultErrorMessage, error) { + ret := _m.Called(blockID, transactionID) + + if len(ret) == 0 { + panic("no return value specified for ByBlockIDTransactionID") + } + + var r0 *flow.TransactionResultErrorMessage + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier, flow.Identifier) (*flow.TransactionResultErrorMessage, error)); ok { + return rf(blockID, transactionID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier, flow.Identifier) *flow.TransactionResultErrorMessage); ok { + r0 = rf(blockID, transactionID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*flow.TransactionResultErrorMessage) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier, flow.Identifier) error); ok { + r1 = rf(blockID, transactionID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// ByBlockIDTransactionIndex provides a mock function with given fields: blockID, txIndex +func (_m *TransactionResultErrorMessages) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) (*flow.TransactionResultErrorMessage, error) { + ret := _m.Called(blockID, txIndex) + + if len(ret) == 0 { + panic("no return value specified for ByBlockIDTransactionIndex") + } + + var r0 *flow.TransactionResultErrorMessage + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier, uint32) (*flow.TransactionResultErrorMessage, error)); ok { + return rf(blockID, txIndex) + } + if rf, ok := ret.Get(0).(func(flow.Identifier, uint32) *flow.TransactionResultErrorMessage); ok { + r0 = rf(blockID, txIndex) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*flow.TransactionResultErrorMessage) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier, uint32) error); ok { + r1 = rf(blockID, txIndex) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Exists provides a mock function with given fields: blockID +func (_m *TransactionResultErrorMessages) Exists(blockID flow.Identifier) (bool, error) { + ret := _m.Called(blockID) + + if len(ret) == 0 { + panic("no return value specified for Exists") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { + return rf(blockID) + } + if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { + r0 = rf(blockID) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { + r1 = rf(blockID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Store provides a mock function with given fields: blockID, transactionResultErrorMessages +func (_m *TransactionResultErrorMessages) Store(blockID flow.Identifier, transactionResultErrorMessages []flow.TransactionResultErrorMessage) error { + ret := _m.Called(blockID, transactionResultErrorMessages) + + if len(ret) == 0 { + panic("no return value specified for Store") + } + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.TransactionResultErrorMessage) error); ok { + r0 = rf(blockID, transactionResultErrorMessages) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// NewTransactionResultErrorMessages creates a new instance of TransactionResultErrorMessages. 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 NewTransactionResultErrorMessages(t interface { + mock.TestingT + Cleanup(func()) +}) *TransactionResultErrorMessages { + mock := &TransactionResultErrorMessages{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/storage/transaction_results.go b/storage/transaction_results.go index bb66023bc83..6aaacc3a880 100644 --- a/storage/transaction_results.go +++ b/storage/transaction_results.go @@ -33,3 +33,35 @@ type LightTransactionResults interface { // ByBlockID gets all transaction results for a block, ordered by transaction index ByBlockID(id flow.Identifier) ([]flow.LightTransactionResult, error) } + +// TransactionResultErrorMessages represents persistent storage for transaction result error messages +type TransactionResultErrorMessages interface { + + // Store will store transaction result error messages for the given block ID. + // + // No errors are expected during normal operation. + Store(blockID flow.Identifier, transactionResultErrorMessages []flow.TransactionResultErrorMessage) error + + // Exists returns true if transaction result error messages for the given ID have been stored. + // + // No errors are expected during normal operation. + Exists(blockID flow.Identifier) (bool, error) + + // ByBlockIDTransactionID returns the transaction result error message for the given block ID and transaction ID. + // + // Expected errors during normal operation: + // - `storage.ErrNotFound` if no transaction error message is known at given block and transaction id. + ByBlockIDTransactionID(blockID flow.Identifier, transactionID flow.Identifier) (*flow.TransactionResultErrorMessage, error) + + // ByBlockIDTransactionIndex returns the transaction result error message for the given blockID and transaction index. + // + // Expected errors during normal operation: + // - `storage.ErrNotFound` if no transaction error message is known at given block and transaction index. + ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) (*flow.TransactionResultErrorMessage, error) + + // ByBlockID gets all transaction result error messages for a block, ordered by transaction index. + // Note: This method will return an empty slice both if the block is not indexed yet and if the block does not have any errors. + // + // No errors are expected during normal operation. + ByBlockID(id flow.Identifier) ([]flow.TransactionResultErrorMessage, error) +}