diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index b4b6397739c..b50f641115e 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -346,6 +346,11 @@ type FlowAccessNodeBuilder struct { VersionControl *version.VersionControl StopControl *stop.StopControl + // storage + events storage.Events + lightTransactionResults storage.LightTransactionResults + transactionResultErrorMessages storage.TransactionResultErrorMessages + // The sync engine participants provider is the libp2p peer store for the access node // which is not available until after the network has started. // Hence, a factory function that needs to be called just before creating the sync engine @@ -872,7 +877,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess return nil }). Module("transaction results storage", func(node *cmd.NodeConfig) error { - builder.Storage.LightTransactionResults = bstorage.NewLightTransactionResults(node.Metrics.Cache, node.DB, bstorage.DefaultCacheSize) + builder.lightTransactionResults = store.NewLightTransactionResults(node.Metrics.Cache, node.ProtocolDB, bstorage.DefaultCacheSize) return nil }). DependableComponent("execution data indexer", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { @@ -961,13 +966,13 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess indexerCore, err := indexer.New( builder.Logger, metrics.NewExecutionStateIndexerCollector(), - builder.DB, + builder.ProtocolDB, builder.Storage.RegisterIndex, builder.Storage.Headers, - builder.Storage.Events, + builder.events, builder.Storage.Collections, builder.Storage.Transactions, - builder.Storage.LightTransactionResults, + builder.lightTransactionResults, builder.RootChainID.Chain(), indexerDerivedChainData, builder.collectionExecutedMetric, @@ -1834,7 +1839,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil }). Module("events storage", func(node *cmd.NodeConfig) error { - builder.Storage.Events = bstorage.NewEvents(node.Metrics.Cache, node.DB) + builder.events = store.NewEvents(node.Metrics.Cache, node.ProtocolDB) return nil }). Module("reporter", func(node *cmd.NodeConfig) error { @@ -1842,11 +1847,11 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { return nil }). Module("events index", func(node *cmd.NodeConfig) error { - builder.EventsIndex = index.NewEventsIndex(builder.Reporter, builder.Storage.Events) + builder.EventsIndex = index.NewEventsIndex(builder.Reporter, builder.events) return nil }). Module("transaction result index", func(node *cmd.NodeConfig) error { - builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.Storage.LightTransactionResults) + builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.lightTransactionResults) return nil }). Module("processed finalized block height consumer progress", func(node *cmd.NodeConfig) error { @@ -1870,7 +1875,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { }). 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) + builder.transactionResultErrorMessages = store.NewTransactionResultErrorMessages(node.Metrics.Cache, node.ProtocolDB, bstorage.DefaultCacheSize) } return nil @@ -2030,7 +2035,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { Transactions: node.Storage.Transactions, ExecutionReceipts: node.Storage.Receipts, ExecutionResults: node.Storage.Results, - TxResultErrorMessages: node.Storage.TransactionResultErrorMessages, + TxResultErrorMessages: builder.transactionResultErrorMessages, ChainID: node.RootChainID, AccessMetrics: builder.AccessMetrics, ConnFactory: connFactory, @@ -2116,7 +2121,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.TxResultErrorMessagesCore = tx_error_messages.NewTxErrorMessagesCore( node.Logger, builder.nodeBackend, - node.Storage.TransactionResultErrorMessages, + builder.transactionResultErrorMessages, builder.ExecNodeIdentitiesProvider, ) } diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index 32ca366f5ea..4cc6b7b4f2d 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -91,7 +91,7 @@ import ( "github.com/onflow/flow-go/state/protocol/blocktimer" storageerr "github.com/onflow/flow-go/storage" storage "github.com/onflow/flow-go/storage/badger" - "github.com/onflow/flow-go/storage/badger/procedure" + "github.com/onflow/flow-go/storage/operation" "github.com/onflow/flow-go/storage/operation/pebbleimpl" storagepebble "github.com/onflow/flow-go/storage/pebble" "github.com/onflow/flow-go/storage/store" @@ -128,17 +128,22 @@ type ExecutionNode struct { ingestionUnit *engine.Unit - collector *metrics.ExecutionCollector - executionState state.ExecutionState - followerState protocol.FollowerState - committee hotstuff.DynamicCommittee - ledgerStorage *ledger.Ledger - registerStore *storehouse.RegisterStore - events *storage.Events - serviceEvents *storage.ServiceEvents - txResults *storage.TransactionResults - results *storage.ExecutionResults - myReceipts *storage.MyExecutionReceipts + collector *metrics.ExecutionCollector + executionState state.ExecutionState + followerState protocol.FollowerState + committee hotstuff.DynamicCommittee + ledgerStorage *ledger.Ledger + registerStore *storehouse.RegisterStore + + // storage + events storageerr.Events + serviceEvents storageerr.ServiceEvents + txResults storageerr.TransactionResults + results storageerr.ExecutionResults + receipts storageerr.ExecutionReceipts + myReceipts storageerr.MyExecutionReceipts + commits storageerr.Commits + providerEngine exeprovider.ProviderEngine checkerEng *checker.Engine syncCore *chainsync.Core @@ -202,7 +207,7 @@ func (builder *ExecutionNodeBuilder) LoadComponentsAndModules() { Module("system specs", exeNode.LoadSystemSpecs). Module("execution metrics", exeNode.LoadExecutionMetrics). Module("sync core", exeNode.LoadSyncCore). - Module("execution receipts storage", exeNode.LoadExecutionReceiptsStorage). + Module("execution storage", exeNode.LoadExecutionStorage). Module("follower distributor", exeNode.LoadFollowerDistributor). Module("authorization checking function", exeNode.LoadAuthorizationCheckingFunction). Module("execution data datastore", exeNode.LoadExecutionDataDatastore). @@ -279,9 +284,9 @@ func (exeNode *ExecutionNode) LoadExecutionMetrics(node *NodeConfig) error { // report the highest executed block height as soon as possible // this is guaranteed to exist because LoadBootstrapper has inserted // the root block as executed block - var height uint64 var blockID flow.Identifier - err := node.DB.View(procedure.GetLastExecutedBlock(&height, &blockID)) + reader := node.ProtocolDB.Reader() + err := operation.RetrieveExecutedBlock(reader, &blockID) if err != nil { // database has not been bootstrapped yet if errors.Is(err, storageerr.ErrNotFound) { @@ -290,7 +295,12 @@ func (exeNode *ExecutionNode) LoadExecutionMetrics(node *NodeConfig) error { return fmt.Errorf("could not get highest executed block: %w", err) } - exeNode.collector.ExecutionLastExecutedBlockHeight(height) + executed, err := node.Storage.Headers.ByBlockID(blockID) + if err != nil { + return fmt.Errorf("could not get header by id: %v: %w", blockID, err) + } + + exeNode.collector.ExecutionLastExecutedBlockHeight(executed.Height) return nil } @@ -300,11 +310,19 @@ func (exeNode *ExecutionNode) LoadSyncCore(node *NodeConfig) error { return err } -func (exeNode *ExecutionNode) LoadExecutionReceiptsStorage( +func (exeNode *ExecutionNode) LoadExecutionStorage( node *NodeConfig, ) error { - exeNode.results = storage.NewExecutionResults(node.Metrics.Cache, node.DB) - exeNode.myReceipts = storage.NewMyExecutionReceipts(node.Metrics.Cache, node.DB, node.Storage.Receipts.(*storage.ExecutionReceipts)) + db := node.ProtocolDB + exeNode.commits = store.NewCommits(node.Metrics.Cache, db) + exeNode.results = store.NewExecutionResults(node.Metrics.Cache, db) + exeNode.receipts = store.NewExecutionReceipts(node.Metrics.Cache, db, exeNode.results, storage.DefaultCacheSize) + exeNode.myReceipts = store.NewMyExecutionReceipts(node.Metrics.Cache, db, exeNode.receipts) + + // Needed for gRPC server, make sure to assign to main scoped vars + exeNode.events = store.NewEvents(node.Metrics.Cache, db) + exeNode.serviceEvents = store.NewServiceEvents(node.Metrics.Cache, db) + exeNode.txResults = store.NewTransactionResults(node.Metrics.Cache, db, exeNode.exeConf.transactionResultsCacheSize) return nil } @@ -438,7 +456,7 @@ func (exeNode *ExecutionNode) LoadGCPBlockDataUploader( retryableUploader := uploader.NewBadgerRetryableUploaderWrapper( asyncUploader, node.Storage.Blocks, - node.Storage.Commits, + exeNode.commits, node.Storage.Collections, exeNode.events, exeNode.results, @@ -743,14 +761,18 @@ func (exeNode *ExecutionNode) LoadExecutionState( chunkDataPacks := store.NewChunkDataPacks(node.Metrics.Cache, pebbleimpl.ToDB(chunkDataPackDB), node.Storage.Collections, exeNode.exeConf.chunkDataPackCacheSize) - // Needed for gRPC server, make sure to assign to main scoped vars - exeNode.events = storage.NewEvents(node.Metrics.Cache, node.DB) - exeNode.serviceEvents = storage.NewServiceEvents(node.Metrics.Cache, node.DB) - exeNode.txResults = storage.NewTransactionResults(node.Metrics.Cache, node.DB, exeNode.exeConf.transactionResultsCacheSize) + getLatestFinalized := func() (uint64, error) { + final, err := node.State.Final().Head() + if err != nil { + return 0, err + } + + return final.Height, nil + } exeNode.executionState = state.NewExecutionState( exeNode.ledgerStorage, - node.Storage.Commits, + exeNode.commits, node.Storage.Blocks, node.Storage.Headers, node.Storage.Collections, @@ -760,7 +782,8 @@ func (exeNode *ExecutionNode) LoadExecutionState( exeNode.events, exeNode.serviceEvents, exeNode.txResults, - node.DB, + node.ProtocolDB, + getLatestFinalized, node.Tracer, exeNode.registerStore, exeNode.exeConf.enableStorehouse, @@ -1316,7 +1339,7 @@ func (exeNode *ExecutionNode) LoadGrpcServer( exeNode.events, exeNode.results, exeNode.txResults, - node.Storage.Commits, + exeNode.commits, exeNode.metricsProvider, node.RootChainID, signature.NewBlockSignerDecoder(exeNode.committee), @@ -1330,7 +1353,7 @@ func (exeNode *ExecutionNode) LoadBootstrapper(node *NodeConfig) error { // check if the execution database already exists bootstrapper := bootstrap.NewBootstrapper(node.Logger) - commit, bootstrapped, err := bootstrapper.IsBootstrapped(node.DB) + commit, bootstrapped, err := bootstrapper.IsBootstrapped(node.ProtocolDB) if err != nil { return fmt.Errorf("could not query database to know whether database has been bootstrapped: %w", err) } @@ -1355,7 +1378,7 @@ func (exeNode *ExecutionNode) LoadBootstrapper(node *NodeConfig) error { return fmt.Errorf("could not load bootstrap state from checkpoint file: %w", err) } - err = bootstrapper.BootstrapExecutionDatabase(node.DB, node.RootSeal) + err = bootstrapper.BootstrapExecutionDatabase(node.ProtocolDB, node.RootSeal) if err != nil { return fmt.Errorf("could not bootstrap execution database: %w", err) } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 90b7f2b03e7..e47e9164924 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -27,6 +27,7 @@ import ( "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/events" + "github.com/onflow/flow-go/storage" bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/utils/grpcutils" ) @@ -202,6 +203,7 @@ type NodeConfig struct { Metrics Metrics DB *badger.DB PebbleDB *pebble.DB + ProtocolDB storage.DB SecretsDB *badger.DB Storage Storage ProtocolEvents *events.Distributor diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 0ced9001d4b..c38806ffba2 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -299,6 +299,10 @@ type ObserverServiceBuilder struct { EventsIndex *index.EventsIndex ScriptExecutor *backend.ScriptExecutor + // storage + events storage.Events + lightTransactionResults storage.LightTransactionResults + // available until after the network has started. Hence, a factory function that needs to be called just before // creating the sync engine SyncEngineParticipantsProviderFactory func() module.IdentifierProvider @@ -1326,7 +1330,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS indexedBlockHeight = store.NewConsumerProgress(badgerimpl.ToDB(builder.DB), module.ConsumeProgressExecutionDataIndexerBlockHeight) return nil }).Module("transaction results storage", func(node *cmd.NodeConfig) error { - builder.Storage.LightTransactionResults = bstorage.NewLightTransactionResults(node.Metrics.Cache, node.DB, bstorage.DefaultCacheSize) + builder.lightTransactionResults = store.NewLightTransactionResults(node.Metrics.Cache, node.ProtocolDB, bstorage.DefaultCacheSize) return nil }).DependableComponent("execution data indexer", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { // Note: using a DependableComponent here to ensure that the indexer does not block @@ -1415,13 +1419,13 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS indexerCore, err := indexer.New( builder.Logger, metrics.NewExecutionStateIndexerCollector(), - builder.DB, + builder.ProtocolDB, builder.Storage.RegisterIndex, builder.Storage.Headers, - builder.Storage.Events, + builder.events, builder.Storage.Collections, builder.Storage.Transactions, - builder.Storage.LightTransactionResults, + builder.lightTransactionResults, builder.RootChainID.Chain(), indexerDerivedChainData, collectionExecutedMetric, @@ -1769,7 +1773,7 @@ func (builder *ObserverServiceBuilder) enqueueRPCServer() { return nil }) builder.Module("events storage", func(node *cmd.NodeConfig) error { - builder.Storage.Events = bstorage.NewEvents(node.Metrics.Cache, node.DB) + builder.events = store.NewEvents(node.Metrics.Cache, node.ProtocolDB) return nil }) builder.Module("reporter", func(node *cmd.NodeConfig) error { @@ -1777,11 +1781,11 @@ func (builder *ObserverServiceBuilder) enqueueRPCServer() { return nil }) builder.Module("events index", func(node *cmd.NodeConfig) error { - builder.EventsIndex = index.NewEventsIndex(builder.Reporter, builder.Storage.Events) + builder.EventsIndex = index.NewEventsIndex(builder.Reporter, builder.events) return nil }) builder.Module("transaction result index", func(node *cmd.NodeConfig) error { - builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.Storage.LightTransactionResults) + builder.TxResultsIndex = index.NewTransactionResultsIndex(builder.Reporter, builder.lightTransactionResults) return nil }) builder.Module("script executor", func(node *cmd.NodeConfig) error { diff --git a/cmd/scaffold.go b/cmd/scaffold.go index d0b6333b8a7..329126f6961 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -83,6 +83,7 @@ import ( "github.com/onflow/flow-go/storage" bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/storage/badger/operation" + "github.com/onflow/flow-go/storage/operation/badgerimpl" sutil "github.com/onflow/flow-go/storage/util" "github.com/onflow/flow-go/utils/logging" ) @@ -1108,9 +1109,12 @@ func (fnb *FlowNodeBuilder) initBadgerDB() error { return fmt.Errorf("could not open public db: %w", err) } fnb.DB = publicDB + // set badger db as protocol db + // TODO: making it dynamic to switch between badger and pebble + fnb.ProtocolDB = badgerimpl.ToDB(publicDB) fnb.ShutdownFunc(func() error { - if err := fnb.DB.Close(); err != nil { + if err := publicDB.Close(); err != nil { return fmt.Errorf("error closing protocol database: %w", err) } return nil @@ -1222,7 +1226,6 @@ func (fnb *FlowNodeBuilder) initStorage() error { collections := bstorage.NewCollections(fnb.DB, transactions) setups := bstorage.NewEpochSetups(fnb.Metrics.Cache, fnb.DB) epochCommits := bstorage.NewEpochCommits(fnb.Metrics.Cache, fnb.DB) - commits := bstorage.NewCommits(fnb.Metrics.Cache, fnb.DB) protocolState := bstorage.NewEpochProtocolStateEntries(fnb.Metrics.Cache, setups, epochCommits, fnb.DB, bstorage.DefaultEpochProtocolStateCacheSize, bstorage.DefaultProtocolStateIndexCacheSize) protocolKVStores := bstorage.NewProtocolKVStore(fnb.Metrics.Cache, fnb.DB, @@ -1232,8 +1235,6 @@ func (fnb *FlowNodeBuilder) initStorage() error { fnb.Storage = Storage{ Headers: headers, Guarantees: guarantees, - Receipts: receipts, - Results: results, Seals: seals, Index: index, Payloads: payloads, @@ -1246,7 +1247,9 @@ func (fnb *FlowNodeBuilder) initStorage() error { VersionBeacons: versionBeacons, EpochProtocolStateEntries: protocolState, ProtocolKVStore: protocolKVStores, - Commits: commits, + + Results: results, + Receipts: receipts, } return nil diff --git a/cmd/util/cmd/common/storage.go b/cmd/util/cmd/common/storage.go index 0fa58a9cf28..7a79156ca04 100644 --- a/cmd/util/cmd/common/storage.go +++ b/cmd/util/cmd/common/storage.go @@ -8,6 +8,8 @@ import ( "github.com/onflow/flow-go/storage" storagebadger "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/storage/badger/operation" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) func InitStorage(datadir string) *badger.DB { @@ -44,3 +46,22 @@ func InitStorages(db *badger.DB) *storage.All { return storagebadger.InitAll(metrics, db) } + +func InitExecutionStorages(bdb *badger.DB) *storage.Execution { + metrics := &metrics.NoopCollector{} + + db := badgerimpl.ToDB(bdb) + + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, storagebadger.DefaultCacheSize) + commits := store.NewCommits(metrics, db) + transactionResults := store.NewTransactionResults(metrics, db, storagebadger.DefaultCacheSize) + events := store.NewEvents(metrics, db) + return &storage.Execution{ + Results: results, + Receipts: receipts, + Commits: commits, + TransactionResults: transactionResults, + Events: events, + } +} diff --git a/cmd/util/cmd/exec-data-json-export/block_exporter.go b/cmd/util/cmd/exec-data-json-export/block_exporter.go index 2e178d08af6..27325b2d228 100644 --- a/cmd/util/cmd/exec-data-json-export/block_exporter.go +++ b/cmd/util/cmd/exec-data-json-export/block_exporter.go @@ -13,6 +13,8 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) type blockSummary struct { @@ -37,6 +39,8 @@ func ExportBlocks(blockID flow.Identifier, dbPath string, outputPath string) (fl db := common.InitStorage(dbPath) defer db.Close() + sdb := badgerimpl.ToDB(db) + cacheMetrics := &metrics.NoopCollector{} headers := badger.NewHeaders(cacheMetrics, db) index := badger.NewIndex(cacheMetrics, db) @@ -46,7 +50,7 @@ func ExportBlocks(blockID flow.Identifier, dbPath string, outputPath string) (fl receipts := badger.NewExecutionReceipts(cacheMetrics, db, results, badger.DefaultCacheSize) payloads := badger.NewPayloads(db, index, guarantees, seals, receipts, results) blocks := badger.NewBlocks(db, headers, payloads) - commits := badger.NewCommits(&metrics.NoopCollector{}, db) + commits := store.NewCommits(&metrics.NoopCollector{}, sdb) activeBlockID := blockID outputFile := filepath.Join(outputPath, "blocks.jsonl") diff --git a/cmd/util/cmd/exec-data-json-export/delta_snapshot_exporter.go b/cmd/util/cmd/exec-data-json-export/delta_snapshot_exporter.go index 68fbc9f4070..af8dc3ba898 100644 --- a/cmd/util/cmd/exec-data-json-export/delta_snapshot_exporter.go +++ b/cmd/util/cmd/exec-data-json-export/delta_snapshot_exporter.go @@ -12,7 +12,8 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage/badger" - "github.com/onflow/flow-go/storage/badger/operation" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/badgerimpl" ) type dSnapshot struct { @@ -26,6 +27,7 @@ func ExportDeltaSnapshots(blockID flow.Identifier, dbPath string, outputPath str // traverse backward from the given block (parent block) and fetch by blockHash db := common.InitStorage(dbPath) defer db.Close() + sdb := badgerimpl.ToDB(db) cacheMetrics := &metrics.NoopCollector{} headers := badger.NewHeaders(cacheMetrics, db) @@ -50,7 +52,7 @@ func ExportDeltaSnapshots(blockID flow.Identifier, dbPath string, outputPath str } var snap []*snapshot.ExecutionSnapshot - err = db.View(operation.RetrieveExecutionStateInteractions(activeBlockID, &snap)) + err = operation.RetrieveExecutionStateInteractions(sdb.Reader(), activeBlockID, &snap) if err != nil { return fmt.Errorf("could not load delta snapshot: %w", err) } diff --git a/cmd/util/cmd/exec-data-json-export/event_exporter.go b/cmd/util/cmd/exec-data-json-export/event_exporter.go index a47c8d4a01d..b516b4d4d3c 100644 --- a/cmd/util/cmd/exec-data-json-export/event_exporter.go +++ b/cmd/util/cmd/exec-data-json-export/event_exporter.go @@ -12,6 +12,8 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) type event struct { @@ -31,9 +33,11 @@ func ExportEvents(blockID flow.Identifier, dbPath string, outputPath string) err db := common.InitStorage(dbPath) defer db.Close() + sdb := badgerimpl.ToDB(db) + cacheMetrics := &metrics.NoopCollector{} headers := badger.NewHeaders(cacheMetrics, db) - events := badger.NewEvents(cacheMetrics, db) + events := store.NewEvents(cacheMetrics, sdb) activeBlockID := blockID outputFile := filepath.Join(outputPath, "events.jsonl") diff --git a/cmd/util/cmd/execution-state-extract/cmd.go b/cmd/util/cmd/execution-state-extract/cmd.go index ad9ddb50e30..420e34245f4 100644 --- a/cmd/util/cmd/execution-state-extract/cmd.go +++ b/cmd/util/cmd/execution-state-extract/cmd.go @@ -20,7 +20,8 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) var ( @@ -239,7 +240,7 @@ func run(*cobra.Command, []string) { defer db.Close() cache := &metrics.NoopCollector{} - commits := badger.NewCommits(cache, db) + commits := store.NewCommits(cache, badgerimpl.ToDB(db)) stateCommitment, err = commits.ByBlockID(blockID) if err != nil { diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index e630b925789..3650385920d 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -22,7 +22,8 @@ import ( "github.com/onflow/flow-go/ledger/complete/wal" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" ) @@ -38,7 +39,7 @@ func TestExtractExecutionState(t *testing.T) { withDirs(t, func(datadir, execdir, outdir string) { db := common.InitStorage(datadir) - commits := badger.NewCommits(metr, db) + commits := store.NewCommits(metr, badgerimpl.ToDB(db)) _, err := commits.ByBlockID(unittest.IdentifierFixture()) require.Error(t, err) @@ -49,7 +50,7 @@ func TestExtractExecutionState(t *testing.T) { withDirs(t, func(datadir, execdir, outdir string) { db := common.InitStorage(datadir) - commits := badger.NewCommits(metr, db) + commits := store.NewCommits(metr, badgerimpl.ToDB(db)) blockID := unittest.IdentifierFixture() stateCommitment := unittest.StateCommitmentFixture() @@ -84,7 +85,7 @@ func TestExtractExecutionState(t *testing.T) { ) db := common.InitStorage(datadir) - commits := badger.NewCommits(metr, db) + commits := store.NewCommits(metr, badgerimpl.ToDB(db)) // generate some oldLedger data size := 10 diff --git a/cmd/util/cmd/find-inconsistent-result/cmd.go b/cmd/util/cmd/find-inconsistent-result/cmd.go index 7a2ceea5036..24b6d352d91 100644 --- a/cmd/util/cmd/find-inconsistent-result/cmd.go +++ b/cmd/util/cmd/find-inconsistent-result/cmd.go @@ -11,7 +11,8 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/storage/badger/procedure" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/badgerimpl" ) var NoMissmatchFoundError = errors.New("No missmatch found") @@ -179,15 +180,20 @@ func findLastExecutedAndSealedHeight(state protocol.State, db *badger.DB) (uint6 } var blockID flow.Identifier - var lastExecuted uint64 - err = db.View(procedure.GetLastExecutedBlock(&lastExecuted, &blockID)) + reader := badgerimpl.ToDB(db).Reader() + err = operation.RetrieveExecutedBlock(reader, &blockID) if err != nil { - return 0, err + return 0, fmt.Errorf("could not lookup executed block %v: %w", blockID, err) + } + + lastExecuted, err := state.AtBlockID(blockID).Head() + if err != nil { + return 0, fmt.Errorf("could not retrieve executed header %v: %w", blockID, err) } // the last sealed executed is min(last_sealed, last_executed) - if lastExecuted < lastSealed.Height { - return lastExecuted, nil + if lastExecuted.Height < lastSealed.Height { + return lastExecuted.Height, nil } return lastSealed.Height, nil } diff --git a/cmd/util/cmd/read-badger/cmd/commits.go b/cmd/util/cmd/read-badger/cmd/commits.go index 873bf4746a5..7ad0d47d31d 100644 --- a/cmd/util/cmd/read-badger/cmd/commits.go +++ b/cmd/util/cmd/read-badger/cmd/commits.go @@ -4,6 +4,7 @@ import ( "github.com/rs/zerolog/log" "github.com/spf13/cobra" + "github.com/onflow/flow-go/cmd/util/cmd/common" findBlockByCommits "github.com/onflow/flow-go/cmd/util/cmd/read-badger/cmd/find-block-by-commits" "github.com/onflow/flow-go/model/flow" ) @@ -21,9 +22,11 @@ var commitsCmd = &cobra.Command{ Use: "commits", Short: "get commit by block ID", Run: func(cmd *cobra.Command, args []string) { - storages, db := InitStorages() + _, db := InitStorages() defer db.Close() + commits := common.InitExecutionStorages(db).Commits + log.Info().Msgf("got flag block id: %s", flagBlockID) blockID, err := flow.HexStringToIdentifier(flagBlockID) if err != nil { @@ -32,7 +35,7 @@ var commitsCmd = &cobra.Command{ } log.Info().Msgf("getting commit by block id: %v", blockID) - commit, err := storages.Commits.ByBlockID(blockID) + commit, err := commits.ByBlockID(blockID) if err != nil { log.Error().Err(err).Msgf("could not get commit for block id: %v", blockID) return diff --git a/cmd/util/cmd/read-badger/cmd/events.go b/cmd/util/cmd/read-badger/cmd/events.go index 981d0a65727..0a9ee47ca34 100644 --- a/cmd/util/cmd/read-badger/cmd/events.go +++ b/cmd/util/cmd/read-badger/cmd/events.go @@ -25,9 +25,11 @@ var eventsCmd = &cobra.Command{ Use: "events", Short: "Read events from badger", Run: func(cmd *cobra.Command, args []string) { - storages, db := InitStorages() + _, db := InitStorages() defer db.Close() + events := common.InitExecutionStorages(db).Events + if flagEventType != "" && flagTransactionID != "" { log.Error().Msg("provide only one of --transaction-id or --event-type") return @@ -49,7 +51,7 @@ var eventsCmd = &cobra.Command{ } log.Info().Msgf("getting events for block id: %v, transaction id: %v", blockID, transactionID) - events, err := storages.Events.ByBlockIDTransactionID(blockID, transactionID) + events, err := events.ByBlockIDTransactionID(blockID, transactionID) if err != nil { log.Error().Err(err).Msgf("could not get events for block id: %v, transaction id: %v", blockID, transactionID) return @@ -70,7 +72,7 @@ var eventsCmd = &cobra.Command{ } if _, ok := validEvents[flagEventType]; ok { log.Info().Msgf("getting events for block id: %v, event type: %s", blockID, flagEventType) - events, err := storages.Events.ByBlockIDEventType(blockID, flow.EventType(flagEventType)) + events, err := events.ByBlockIDEventType(blockID, flow.EventType(flagEventType)) if err != nil { log.Error().Err(err).Msgf("could not get events for block id: %v, event type: %s", blockID, flagEventType) return @@ -88,13 +90,13 @@ var eventsCmd = &cobra.Command{ // just fetch events for block log.Info().Msgf("getting events for block id: %v", blockID) - events, err := storages.Events.ByBlockID(blockID) + evts, err := events.ByBlockID(blockID) if err != nil { log.Error().Err(err).Msgf("could not get events for block id: %v", blockID) return } - for _, event := range events { + for _, event := range evts { common.PrettyPrint(event) } }, diff --git a/cmd/util/cmd/read-badger/cmd/find-block-by-commits/main.go b/cmd/util/cmd/read-badger/cmd/find-block-by-commits/main.go index b1fb1cf3106..93933cdf43b 100644 --- a/cmd/util/cmd/read-badger/cmd/find-block-by-commits/main.go +++ b/cmd/util/cmd/read-badger/cmd/find-block-by-commits/main.go @@ -10,6 +10,7 @@ import ( "github.com/rs/zerolog/log" "github.com/spf13/cobra" + "github.com/onflow/flow-go/cmd/util/cmd/common" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/storage" ) @@ -123,10 +124,12 @@ func run(*cobra.Command, []string) { } }() + en := common.InitExecutionStorages(db) + _, err = FindBlockIDByCommits( log.Logger, storage.Headers, - storage.Commits, + en.Commits, stateCommitments, flagStartHeight, flagEndHeight, diff --git a/cmd/util/cmd/read-badger/cmd/transaction_results.go b/cmd/util/cmd/read-badger/cmd/transaction_results.go index ebf604c687b..abbbe02b8a7 100644 --- a/cmd/util/cmd/read-badger/cmd/transaction_results.go +++ b/cmd/util/cmd/read-badger/cmd/transaction_results.go @@ -22,6 +22,8 @@ var transactionResultsCmd = &cobra.Command{ storages, db := InitStorages() defer db.Close() + transactionResults := common.InitExecutionStorages(db).TransactionResults + log.Info().Msgf("got flag block id: %s", flagBlockID) blockID, err := flow.HexStringToIdentifier(flagBlockID) if err != nil { @@ -50,7 +52,7 @@ var transactionResultsCmd = &cobra.Command{ } for _, txID := range txIDs { - transactionResult, err := storages.TransactionResults.ByBlockIDTransactionID(blockID, txID) + transactionResult, err := transactionResults.ByBlockIDTransactionID(blockID, txID) if err != nil { log.Error().Err(err).Msgf("could not get transaction result for block id and transaction id: %v", txID) return diff --git a/cmd/util/cmd/read-protocol-state/cmd/blocks.go b/cmd/util/cmd/read-protocol-state/cmd/blocks.go index 2db4172d536..aaa4cbe41cc 100644 --- a/cmd/util/cmd/read-protocol-state/cmd/blocks.go +++ b/cmd/util/cmd/read-protocol-state/cmd/blocks.go @@ -52,11 +52,11 @@ type Reader struct { commits storage.Commits } -func NewReader(state protocol.State, storages *storage.All) *Reader { +func NewReader(state protocol.State, blocks storage.Blocks, commits storage.Commits) *Reader { return &Reader{ state: state, - blocks: storages.Blocks, - commits: storages.Commits, + blocks: blocks, + commits: commits, } } @@ -153,11 +153,13 @@ func run(*cobra.Command, []string) { storages := common.InitStorages(db) state, err := common.InitProtocolState(db, storages) + en := common.InitExecutionStorages(db) + if err != nil { log.Fatal().Err(err).Msg("could not init protocol state") } - reader := NewReader(state, storages) + reader := NewReader(state, storages.Blocks, en.Commits) // making sure only one flag is being used err = checkOnlyOneFlagIsUsed(flagHeight, flagBlockID, flagFinal, flagSealed, flagExecuted) diff --git a/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height.go b/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height.go index c5a7e947c86..a4255ee6a89 100644 --- a/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height.go +++ b/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" "github.com/onflow/flow-go/storage/operation/pebbleimpl" storagepebble "github.com/onflow/flow-go/storage/pebble" "github.com/onflow/flow-go/storage/store" @@ -59,24 +60,26 @@ func run(*cobra.Command, []string) { log.Fatal().Msg("height must be above 0") } - db := common.InitStorage(flagDataDir) - storages := common.InitStorages(db) - state, err := common.InitProtocolState(db, storages) + bdb := common.InitStorage(flagDataDir) + storages := common.InitStorages(bdb) + state, err := common.InitProtocolState(bdb, storages) if err != nil { log.Fatal().Err(err).Msg("could not init protocol states") } + db := badgerimpl.ToDB(bdb) metrics := &metrics.NoopCollector{} - transactionResults := badger.NewTransactionResults(metrics, db, badger.DefaultCacheSize) - commits := badger.NewCommits(metrics, db) - collections := badger.NewCollections(db, badger.NewTransactions(metrics, db)) - results := badger.NewExecutionResults(metrics, db) - receipts := badger.NewExecutionReceipts(metrics, db, results, badger.DefaultCacheSize) - myReceipts := badger.NewMyExecutionReceipts(metrics, db, receipts) - headers := badger.NewHeaders(metrics, db) - events := badger.NewEvents(metrics, db) - serviceEvents := badger.NewServiceEvents(metrics, db) + transactionResults := store.NewTransactionResults(metrics, db, badger.DefaultCacheSize) + commits := store.NewCommits(metrics, db) + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, badger.DefaultCacheSize) + myReceipts := store.NewMyExecutionReceipts(metrics, db, receipts) + headers := badger.NewHeaders(metrics, bdb) + events := store.NewEvents(metrics, db) + serviceEvents := store.NewServiceEvents(metrics, db) + transactions := badger.NewTransactions(metrics, bdb) + collections := badger.NewCollections(bdb, transactions) // require the chunk data pack data must exist before returning the storage module chunkDataPacksPebbleDB, err := storagepebble.MustOpenDefaultPebbleDB(flagChunkDataPackDir) if err != nil { @@ -84,15 +87,14 @@ func run(*cobra.Command, []string) { } chunkDataPacksDB := pebbleimpl.ToDB(chunkDataPacksPebbleDB) chunkDataPacks := store.NewChunkDataPacks(metrics, chunkDataPacksDB, collections, 1000) - - writeBatch := badger.NewBatch(db) chunkBatch := chunkDataPacksDB.NewBatch() + writeBatch := db.NewBatch() + err = removeExecutionResultsFromHeight( writeBatch, chunkBatch, state, - headers, transactionResults, commits, chunkDataPacks, @@ -112,7 +114,7 @@ func run(*cobra.Command, []string) { log.Fatal().Err(err).Msgf("could not commit chunk batch at %v", flagHeight) } - err = writeBatch.Flush() + err = writeBatch.Commit() if err != nil { log.Fatal().Err(err).Msgf("could not flush write batch at %v", flagHeight) } @@ -134,17 +136,16 @@ func run(*cobra.Command, []string) { // use badger instances directly instead of stroage interfaces so that the interface don't // need to include the Remove methods func removeExecutionResultsFromHeight( - writeBatch *badger.Batch, + writeBatch storage.Batch, chunkBatch storage.Batch, protoState protocol.State, - headers *badger.Headers, - transactionResults *badger.TransactionResults, - commits *badger.Commits, + transactionResults storage.TransactionResults, + commits storage.Commits, chunkDataPacks storage.ChunkDataPacks, - results *badger.ExecutionResults, - myReceipts *badger.MyExecutionReceipts, - events *badger.Events, - serviceEvents *badger.ServiceEvents, + results storage.ExecutionResults, + myReceipts storage.MyExecutionReceipts, + events storage.Events, + serviceEvents storage.ServiceEvents, fromHeight uint64) error { log.Info().Msgf("removing results for blocks from height: %v", fromHeight) @@ -175,7 +176,7 @@ func removeExecutionResultsFromHeight( blockID := head.ID() - err = removeForBlockID(writeBatch, chunkBatch, headers, commits, transactionResults, results, chunkDataPacks, myReceipts, events, serviceEvents, blockID) + err = removeForBlockID(writeBatch, chunkBatch, commits, transactionResults, results, chunkDataPacks, myReceipts, events, serviceEvents, blockID) if err != nil { return fmt.Errorf("could not remove result for finalized block: %v, %w", blockID, err) } @@ -194,7 +195,7 @@ func removeExecutionResultsFromHeight( total = len(pendings) for _, pending := range pendings { - err = removeForBlockID(writeBatch, chunkBatch, headers, commits, transactionResults, results, chunkDataPacks, myReceipts, events, serviceEvents, pending) + err = removeForBlockID(writeBatch, chunkBatch, commits, transactionResults, results, chunkDataPacks, myReceipts, events, serviceEvents, pending) if err != nil { return fmt.Errorf("could not remove result for pending block %v: %w", pending, err) @@ -214,16 +215,15 @@ func removeExecutionResultsFromHeight( // All data to be removed will be removed in a batch write. // It bubbles up any error encountered func removeForBlockID( - writeBatch *badger.Batch, + writeBatch storage.Batch, chunkBatch storage.Batch, - headers *badger.Headers, - commits *badger.Commits, - transactionResults *badger.TransactionResults, - results *badger.ExecutionResults, + commits storage.Commits, + transactionResults storage.TransactionResults, + results storage.ExecutionResults, chunks storage.ChunkDataPacks, - myReceipts *badger.MyExecutionReceipts, - events *badger.Events, - serviceEvents *badger.ServiceEvents, + myReceipts storage.MyExecutionReceipts, + events storage.Events, + serviceEvents storage.ServiceEvents, blockID flow.Identifier, ) error { result, err := results.ByBlockID(blockID) diff --git a/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height_test.go b/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height_test.go index 8d168bf41cb..e108502542a 100644 --- a/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height_test.go +++ b/cmd/util/cmd/rollback-executed-height/cmd/rollback_executed_height_test.go @@ -14,6 +14,7 @@ import ( "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/trace" bstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" "github.com/onflow/flow-go/storage/operation/pebbleimpl" "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" @@ -22,34 +23,39 @@ import ( // Test save block execution related data, then remove it, and then // save again should still work func TestReExecuteBlock(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + unittest.RunWithBadgerDB(t, func(bdb *badger.DB) { unittest.RunWithPebbleDB(t, func(pdb *pebble.DB) { // bootstrap to init highest executed height bootstrapper := bootstrap.NewBootstrapper(unittest.Logger()) genesis := unittest.BlockHeaderFixture() rootSeal := unittest.Seal.Fixture(unittest.Seal.WithBlock(genesis)) + db := badgerimpl.ToDB(bdb) err := bootstrapper.BootstrapExecutionDatabase(db, rootSeal) require.NoError(t, err) // create all modules metrics := &metrics.NoopCollector{} - headers := bstorage.NewHeaders(metrics, db) - txResults := bstorage.NewTransactionResults(metrics, db, bstorage.DefaultCacheSize) - commits := bstorage.NewCommits(metrics, db) - chunkDataPacks := store.NewChunkDataPacks(metrics, pebbleimpl.ToDB(pdb), bstorage.NewCollections(db, bstorage.NewTransactions(metrics, db)), bstorage.DefaultCacheSize) - results := bstorage.NewExecutionResults(metrics, db) - receipts := bstorage.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) - myReceipts := bstorage.NewMyExecutionReceipts(metrics, db, receipts) - events := bstorage.NewEvents(metrics, db) - serviceEvents := bstorage.NewServiceEvents(metrics, db) - transactions := bstorage.NewTransactions(metrics, db) - collections := bstorage.NewCollections(db, transactions) + headers := bstorage.NewHeaders(metrics, bdb) + txResults := store.NewTransactionResults(metrics, db, bstorage.DefaultCacheSize) + commits := store.NewCommits(metrics, db) + chunkDataPacks := store.NewChunkDataPacks(metrics, pebbleimpl.ToDB(pdb), bstorage.NewCollections(bdb, bstorage.NewTransactions(metrics, bdb)), bstorage.DefaultCacheSize) + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) + myReceipts := store.NewMyExecutionReceipts(metrics, db, receipts) + events := store.NewEvents(metrics, db) + serviceEvents := store.NewServiceEvents(metrics, db) + transactions := bstorage.NewTransactions(metrics, bdb) + collections := bstorage.NewCollections(bdb, transactions) err = headers.Store(genesis) require.NoError(t, err) + getLatestFinalized := func() (uint64, error) { + return genesis.Height, nil + } + // create execution state module es := state.NewExecutionState( nil, @@ -64,6 +70,7 @@ func TestReExecuteBlock(t *testing.T) { serviceEvents, txResults, db, + getLatestFinalized, trace.NewNoopTracer(), nil, false, @@ -80,14 +87,13 @@ func TestReExecuteBlock(t *testing.T) { err = es.SaveExecutionResults(context.Background(), computationResult) require.NoError(t, err) - batch := bstorage.NewBatch(db) + batch := db.NewBatch() chunkBatch := pebbleimpl.ToDB(pdb).NewBatch() // remove execution results err = removeForBlockID( batch, chunkBatch, - headers, commits, txResults, results, @@ -104,7 +110,6 @@ func TestReExecuteBlock(t *testing.T) { err = removeForBlockID( batch, chunkBatch, - headers, commits, txResults, results, @@ -117,18 +122,17 @@ func TestReExecuteBlock(t *testing.T) { require.NoError(t, err) require.NoError(t, chunkBatch.Commit()) - err2 := batch.Flush() + err2 := batch.Commit() require.NoError(t, err2) - batch = bstorage.NewBatch(db) + batch = db.NewBatch() chunkBatch = pebbleimpl.ToDB(pdb).NewBatch() // remove again after flushing err = removeForBlockID( batch, chunkBatch, - headers, commits, txResults, results, @@ -141,7 +145,7 @@ func TestReExecuteBlock(t *testing.T) { require.NoError(t, err) require.NoError(t, chunkBatch.Commit()) - err2 = batch.Flush() + err2 = batch.Commit() require.NoError(t, err2) @@ -155,7 +159,7 @@ func TestReExecuteBlock(t *testing.T) { // Test save block execution related data, then remove it, and then // save again with different result should work func TestReExecuteBlockWithDifferentResult(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + unittest.RunWithBadgerDB(t, func(bdb *badger.DB) { unittest.RunWithPebbleDB(t, func(pdb *pebble.DB) { // bootstrap to init highest executed height @@ -163,27 +167,33 @@ func TestReExecuteBlockWithDifferentResult(t *testing.T) { genesis := unittest.BlockHeaderFixture() rootSeal := unittest.Seal.Fixture() unittest.Seal.WithBlock(genesis)(rootSeal) + + db := badgerimpl.ToDB(bdb) err := bootstrapper.BootstrapExecutionDatabase(db, rootSeal) require.NoError(t, err) // create all modules metrics := &metrics.NoopCollector{} - headers := bstorage.NewHeaders(metrics, db) - txResults := bstorage.NewTransactionResults(metrics, db, bstorage.DefaultCacheSize) - commits := bstorage.NewCommits(metrics, db) - chunkDataPacks := store.NewChunkDataPacks(metrics, pebbleimpl.ToDB(pdb), bstorage.NewCollections(db, bstorage.NewTransactions(metrics, db)), bstorage.DefaultCacheSize) - results := bstorage.NewExecutionResults(metrics, db) - receipts := bstorage.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) - myReceipts := bstorage.NewMyExecutionReceipts(metrics, db, receipts) - events := bstorage.NewEvents(metrics, db) - serviceEvents := bstorage.NewServiceEvents(metrics, db) - transactions := bstorage.NewTransactions(metrics, db) - collections := bstorage.NewCollections(db, transactions) + headers := bstorage.NewHeaders(metrics, bdb) + txResults := store.NewTransactionResults(metrics, db, bstorage.DefaultCacheSize) + commits := store.NewCommits(metrics, db) + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) + myReceipts := store.NewMyExecutionReceipts(metrics, db, receipts) + events := store.NewEvents(metrics, db) + serviceEvents := store.NewServiceEvents(metrics, db) + transactions := bstorage.NewTransactions(metrics, bdb) + collections := bstorage.NewCollections(bdb, transactions) + chunkDataPacks := store.NewChunkDataPacks(metrics, pebbleimpl.ToDB(pdb), collections, bstorage.DefaultCacheSize) err = headers.Store(genesis) require.NoError(t, err) + getLatestFinalized := func() (uint64, error) { + return genesis.Height, nil + } + // create execution state module es := state.NewExecutionState( nil, @@ -198,6 +208,7 @@ func TestReExecuteBlockWithDifferentResult(t *testing.T) { serviceEvents, txResults, db, + getLatestFinalized, trace.NewNoopTracer(), nil, false, @@ -221,14 +232,13 @@ func TestReExecuteBlockWithDifferentResult(t *testing.T) { err = es.SaveExecutionResults(context.Background(), computationResult) require.NoError(t, err) - batch := bstorage.NewBatch(db) + batch := db.NewBatch() chunkBatch := pebbleimpl.ToDB(pdb).NewBatch() // remove execution results err = removeForBlockID( batch, chunkBatch, - headers, commits, txResults, results, @@ -241,17 +251,16 @@ func TestReExecuteBlockWithDifferentResult(t *testing.T) { require.NoError(t, err) require.NoError(t, chunkBatch.Commit()) - err2 := batch.Flush() + err2 := batch.Commit() require.NoError(t, err2) - batch = bstorage.NewBatch(db) + batch = db.NewBatch() chunkBatch = pebbleimpl.ToDB(pdb).NewBatch() // remove again to test for duplicates handling err = removeForBlockID( batch, chunkBatch, - headers, commits, txResults, results, @@ -265,7 +274,7 @@ func TestReExecuteBlockWithDifferentResult(t *testing.T) { require.NoError(t, err) require.NoError(t, chunkBatch.Commit()) - err2 = batch.Flush() + err2 = batch.Commit() require.NoError(t, err2) computationResult2 := testutil.ComputationResultFixture(t) diff --git a/cmd/verification_builder.go b/cmd/verification_builder.go index f7c27b6455a..80ea76acd45 100644 --- a/cmd/verification_builder.go +++ b/cmd/verification_builder.go @@ -39,7 +39,6 @@ import ( "github.com/onflow/flow-go/state/protocol/blocktimer" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/badger" - "github.com/onflow/flow-go/storage/operation/badgerimpl" "github.com/onflow/flow-go/storage/store" ) @@ -158,11 +157,11 @@ func (v *VerificationNodeBuilder) LoadComponentsAndModules() { return nil }). Module("processed chunk index consumer progress", func(node *NodeConfig) error { - processedChunkIndex = store.NewConsumerProgress(badgerimpl.ToDB(node.DB), module.ConsumeProgressVerificationChunkIndex) + processedChunkIndex = store.NewConsumerProgress(node.ProtocolDB, module.ConsumeProgressVerificationChunkIndex) return nil }). Module("processed block height consumer progress", func(node *NodeConfig) error { - processedBlockHeight = store.NewConsumerProgress(badgerimpl.ToDB(node.DB), module.ConsumeProgressVerificationBlockHeight) + processedBlockHeight = store.NewConsumerProgress(node.ProtocolDB, module.ConsumeProgressVerificationBlockHeight) return nil }). Module("chunks queue", func(node *NodeConfig) error { diff --git a/engine/access/access_test.go b/engine/access/access_test.go index 032c3a0e1eb..4f6423f64f3 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -148,10 +148,11 @@ func (suite *Suite) SetupTest() { } func (suite *Suite) RunTest( - f func(handler *access.Handler, db *badger.DB, all *storage.All), + f func(handler *access.Handler, db *badger.DB, all *storage.All, en *storage.Execution), ) { unittest.RunWithBadgerDB(suite.T(), func(db *badger.DB) { all := util.StorageLayer(suite.T(), db) + en := util.ExecutionStorageLayer(suite.T(), db) var err error suite.backend, err = backend.New(backend.Params{ @@ -161,8 +162,8 @@ func (suite *Suite) RunTest( Headers: all.Headers, Collections: all.Collections, Transactions: all.Transactions, - ExecutionResults: all.Results, - ExecutionReceipts: all.Receipts, + ExecutionResults: en.Results, + ExecutionReceipts: en.Receipts, ChainID: suite.chainID, AccessMetrics: suite.metrics, MaxHeightRange: backend.DefaultMaxHeightRange, @@ -180,12 +181,12 @@ func (suite *Suite) RunTest( subscription.DefaultMaxGlobalStreams, access.WithBlockSignerDecoder(suite.signerIndicesDecoder), ) - f(handler, db, all) + f(handler, db, all, en) }) } func (suite *Suite) TestSendAndGetTransaction() { - suite.RunTest(func(handler *access.Handler, _ *badger.DB, _ *storage.All) { + suite.RunTest(func(handler *access.Handler, _ *badger.DB, _ *storage.All, _ *storage.Execution) { referenceBlock := unittest.BlockHeaderFixture() transaction := unittest.TransactionFixture() transaction.SetReferenceBlockID(referenceBlock.ID()) @@ -238,7 +239,7 @@ func (suite *Suite) TestSendAndGetTransaction() { } func (suite *Suite) TestSendExpiredTransaction() { - suite.RunTest(func(handler *access.Handler, _ *badger.DB, _ *storage.All) { + suite.RunTest(func(handler *access.Handler, _ *badger.DB, _ *storage.All, en *storage.Execution) { referenceBlock := suite.finalizedBlock transaction := unittest.TransactionFixture() @@ -377,7 +378,7 @@ func (suite *Suite) TestSendTransactionToRandomCollectionNode() { } func (suite *Suite) TestGetBlockByIDAndHeight() { - suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All) { + suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All, en *storage.Execution) { // test block1 get by ID block1 := unittest.BlockFixture() @@ -513,7 +514,7 @@ func (suite *Suite) TestGetBlockByIDAndHeight() { } func (suite *Suite) TestGetExecutionResultByBlockID() { - suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All) { + suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All, en *storage.Execution) { // test block1 get by ID nonexistingID := unittest.IdentifierFixture() @@ -523,8 +524,8 @@ func (suite *Suite) TestGetExecutionResultByBlockID() { unittest.WithExecutionResultBlockID(blockID), unittest.WithServiceEvents(3)) - require.NoError(suite.T(), all.Results.Store(er)) - require.NoError(suite.T(), all.Results.Index(blockID, er.ID())) + require.NoError(suite.T(), en.Results.Store(er)) + require.NoError(suite.T(), en.Results.Index(blockID, er.ID())) assertResp := func( resp *accessproto.ExecutionResultForBlockIDResponse, @@ -597,8 +598,7 @@ func (suite *Suite) TestGetExecutionResultByBlockID() { func (suite *Suite) TestGetSealedTransaction() { unittest.RunWithBadgerDB(suite.T(), func(db *badger.DB) { all := util.StorageLayer(suite.T(), db) - results := bstorage.NewExecutionResults(suite.metrics, db) - receipts := bstorage.NewExecutionReceipts(suite.metrics, db, results, bstorage.DefaultCacheSize) + en := util.ExecutionStorageLayer(suite.T(), db) enIdentities := unittest.IdentityListFixture(2, unittest.WithRole(flow.RoleExecution)) enNodeIDs := enIdentities.NodeIDs() @@ -646,7 +646,7 @@ func (suite *Suite) TestGetSealedTransaction() { execNodeIdentitiesProvider := commonrpc.NewExecutionNodeIdentitiesProvider( suite.log, suite.state, - receipts, + en.Receipts, enNodeIDs, nil, ) @@ -658,8 +658,8 @@ func (suite *Suite) TestGetSealedTransaction() { Headers: all.Headers, Collections: collections, Transactions: transactions, - ExecutionReceipts: receipts, - ExecutionResults: results, + ExecutionReceipts: en.Receipts, + ExecutionResults: en.Results, ChainID: suite.chainID, AccessMetrics: suite.metrics, ConnFactory: connFactory, @@ -704,8 +704,8 @@ func (suite *Suite) TestGetSealedTransaction() { all.Headers, collections, transactions, - results, - receipts, + en.Results, + en.Receipts, collectionExecutedMetric, processedHeight, lastFullBlockHeight, @@ -765,8 +765,7 @@ func (suite *Suite) TestGetSealedTransaction() { func (suite *Suite) TestGetTransactionResult() { unittest.RunWithBadgerDB(suite.T(), func(db *badger.DB) { all := util.StorageLayer(suite.T(), db) - results := bstorage.NewExecutionResults(suite.metrics, db) - receipts := bstorage.NewExecutionReceipts(suite.metrics, db, results, bstorage.DefaultCacheSize) + en := util.ExecutionStorageLayer(suite.T(), db) originID := unittest.IdentifierFixture() *suite.state = protocol.State{} @@ -836,7 +835,7 @@ func (suite *Suite) TestGetTransactionResult() { execNodeIdentitiesProvider := commonrpc.NewExecutionNodeIdentitiesProvider( suite.log, suite.state, - receipts, + en.Receipts, enNodeIDs, nil, ) @@ -847,8 +846,8 @@ func (suite *Suite) TestGetTransactionResult() { Headers: all.Headers, Collections: collections, Transactions: transactions, - ExecutionReceipts: receipts, - ExecutionResults: results, + ExecutionReceipts: en.Receipts, + ExecutionResults: en.Results, ChainID: suite.chainID, AccessMetrics: suite.metrics, ConnFactory: connFactory, @@ -895,8 +894,8 @@ func (suite *Suite) TestGetTransactionResult() { all.Headers, collections, transactions, - results, - receipts, + en.Results, + en.Receipts, collectionExecutedMetric, processedHeightInitializer, lastFullBlockHeight, @@ -1060,10 +1059,7 @@ func (suite *Suite) TestGetTransactionResult() { func (suite *Suite) TestExecuteScript() { unittest.RunWithBadgerDB(suite.T(), func(db *badger.DB) { all := util.StorageLayer(suite.T(), db) - transactions := bstorage.NewTransactions(suite.metrics, db) - collections := bstorage.NewCollections(db, transactions) - results := bstorage.NewExecutionResults(suite.metrics, db) - receipts := bstorage.NewExecutionReceipts(suite.metrics, db, results, bstorage.DefaultCacheSize) + en := util.ExecutionStorageLayer(suite.T(), db) 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) @@ -1075,7 +1071,7 @@ func (suite *Suite) TestExecuteScript() { execNodeIdentitiesProvider := commonrpc.NewExecutionNodeIdentitiesProvider( suite.log, suite.state, - receipts, + en.Receipts, nil, identities.NodeIDs(), ) @@ -1086,10 +1082,10 @@ func (suite *Suite) TestExecuteScript() { CollectionRPC: suite.collClient, Blocks: all.Blocks, Headers: all.Headers, - Collections: collections, - Transactions: transactions, - ExecutionReceipts: receipts, - ExecutionResults: results, + Collections: all.Collections, + Transactions: all.Transactions, + ExecutionReceipts: en.Receipts, + ExecutionResults: en.Results, ChainID: suite.chainID, AccessMetrics: suite.metrics, ConnFactory: connFactory, @@ -1122,7 +1118,7 @@ func (suite *Suite) TestExecuteScript() { collectionsToMarkFinalized, collectionsToMarkExecuted, blocksToMarkExecuted, - collections, + all.Collections, all.Blocks, blockTransactions, ) @@ -1150,10 +1146,10 @@ func (suite *Suite) TestExecuteScript() { suite.request, all.Blocks, all.Headers, - collections, - transactions, - results, - receipts, + all.Collections, + all.Transactions, + en.Results, + en.Receipts, collectionExecutedMetric, processedHeightInitializer, lastFullBlockHeight, @@ -1278,7 +1274,7 @@ func (suite *Suite) TestExecuteScript() { // TestAPICallNodeVersionInfo tests the GetNodeVersionInfo query and check response returns correct node version // information func (suite *Suite) TestAPICallNodeVersionInfo() { - suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All) { + suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All, en *storage.Execution) { req := &accessproto.GetNodeVersionInfoRequest{} resp, err := handler.GetNodeVersionInfo(context.Background(), req) require.NoError(suite.T(), err) @@ -1298,7 +1294,7 @@ func (suite *Suite) TestAPICallNodeVersionInfo() { // field in the response matches the finalized header from cache. It also tests that the LastFinalizedBlock field is // updated correctly when a block with a greater height is finalized. func (suite *Suite) TestLastFinalizedBlockHeightResult() { - suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All) { + suite.RunTest(func(handler *access.Handler, db *badger.DB, all *storage.All, en *storage.Execution) { block := unittest.BlockWithParentFixture(suite.finalizedBlock) newFinalizedBlock := unittest.BlockWithParentFixture(block.Header) diff --git a/engine/execution/state/bootstrap/bootstrap.go b/engine/execution/state/bootstrap/bootstrap.go index 97656092d09..59fce03aaf8 100644 --- a/engine/execution/state/bootstrap/bootstrap.go +++ b/engine/execution/state/bootstrap/bootstrap.go @@ -6,7 +6,6 @@ import ( "fmt" "github.com/cockroachdb/pebble" - "github.com/dgraph-io/badger/v2" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine/execution/state" @@ -16,7 +15,7 @@ import ( "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/storage/badger/operation" + "github.com/onflow/flow-go/storage/operation" pStorage "github.com/onflow/flow-go/storage/pebble" ) @@ -77,18 +76,10 @@ func (b *Bootstrapper) BootstrapLedger( // IsBootstrapped returns whether the execution database has been bootstrapped, if yes, returns the // root statecommitment -func (b *Bootstrapper) IsBootstrapped(db *badger.DB) (flow.StateCommitment, bool, error) { +func (b *Bootstrapper) IsBootstrapped(db storage.DB) (flow.StateCommitment, bool, error) { var commit flow.StateCommitment - err := db.View(func(txn *badger.Txn) error { - err := operation.LookupStateCommitment(flow.ZeroID, &commit)(txn) - if err != nil { - return fmt.Errorf("could not lookup state commitment: %w", err) - } - - return nil - }) - + err := operation.LookupStateCommitment(db.Reader(), flow.ZeroID, &commit) if errors.Is(err, storage.ErrNotFound) { return flow.DummyStateCommitment, false, nil } @@ -101,35 +92,35 @@ func (b *Bootstrapper) IsBootstrapped(db *badger.DB) (flow.StateCommitment, bool } func (b *Bootstrapper) BootstrapExecutionDatabase( - db *badger.DB, + db storage.DB, rootSeal *flow.Seal, ) error { commit := rootSeal.FinalState - err := operation.RetryOnConflict(db.Update, func(txn *badger.Txn) error { - - err := operation.InsertExecutedBlock(rootSeal.BlockID)(txn) + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + w := rw.Writer() + err := operation.UpdateExecutedBlock(w, rootSeal.BlockID) if err != nil { return fmt.Errorf("could not index initial genesis execution block: %w", err) } - err = operation.SkipDuplicates(operation.IndexExecutionResult(rootSeal.BlockID, rootSeal.ResultID))(txn) + err = operation.IndexExecutionResult(w, rootSeal.BlockID, rootSeal.ResultID) if err != nil { return fmt.Errorf("could not index result for root result: %w", err) } - err = operation.IndexStateCommitment(flow.ZeroID, commit)(txn) + err = operation.IndexStateCommitment(w, flow.ZeroID, commit) if err != nil { return fmt.Errorf("could not index void state commitment: %w", err) } - err = operation.IndexStateCommitment(rootSeal.BlockID, commit)(txn) + err = operation.IndexStateCommitment(w, rootSeal.BlockID, commit) if err != nil { return fmt.Errorf("could not index genesis state commitment: %w", err) } snapshots := make([]*snapshot.ExecutionSnapshot, 0) - err = operation.InsertExecutionStateInteractions(rootSeal.BlockID, snapshots)(txn) + err = operation.InsertExecutionStateInteractions(w, rootSeal.BlockID, snapshots) if err != nil { return fmt.Errorf("could not bootstrap execution state interactions: %w", err) } diff --git a/engine/execution/state/mock/execution_state.go b/engine/execution/state/mock/execution_state.go index 59ef8e56d5d..344cb0ad868 100644 --- a/engine/execution/state/mock/execution_state.go +++ b/engine/execution/state/mock/execution_state.go @@ -279,7 +279,7 @@ func (_m *ExecutionState) StateCommitmentByBlockID(_a0 flow.Identifier) (flow.St } // UpdateLastExecutedBlock provides a mock function with given fields: _a0, _a1 -func (_m *ExecutionState) UpdateLastExecutedBlock(_a0 context.Context, _a1 *flow.Header) error { +func (_m *ExecutionState) UpdateLastExecutedBlock(_a0 context.Context, _a1 flow.Identifier) error { ret := _m.Called(_a0, _a1) if len(ret) == 0 { @@ -287,7 +287,7 @@ func (_m *ExecutionState) UpdateLastExecutedBlock(_a0 context.Context, _a1 *flow } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *flow.Header) error); ok { + if rf, ok := ret.Get(0).(func(context.Context, flow.Identifier) error); ok { r0 = rf(_a0, _a1) } else { r0 = ret.Error(0) diff --git a/engine/execution/state/state.go b/engine/execution/state/state.go index f95f15a3efd..c0948ecf41e 100644 --- a/engine/execution/state/state.go +++ b/engine/execution/state/state.go @@ -7,8 +7,6 @@ import ( "math" "sync" - "github.com/dgraph-io/badger/v2" - "github.com/onflow/flow-go/engine/execution" "github.com/onflow/flow-go/engine/execution/storehouse" "github.com/onflow/flow-go/fvm/storage/snapshot" @@ -18,9 +16,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/storage" - badgerstorage "github.com/onflow/flow-go/storage/badger" - "github.com/onflow/flow-go/storage/badger/operation" - "github.com/onflow/flow-go/storage/badger/procedure" + "github.com/onflow/flow-go/storage/operation" ) var ErrExecutionStatePruned = fmt.Errorf("execution state is pruned") @@ -79,7 +75,7 @@ type FinalizedExecutionState interface { type ExecutionState interface { ReadOnlyExecutionState - UpdateLastExecutedBlock(context.Context, *flow.Header) error + UpdateLastExecutedBlock(context.Context, flow.Identifier) error SaveExecutionResults( ctx context.Context, @@ -104,7 +100,12 @@ type state struct { events storage.Events serviceEvents storage.ServiceEvents transactionResults storage.TransactionResults - db *badger.DB + db storage.DB + // make sure only one saving operation is running at a time, this is to prevent + // the reads within the saving operation to be dirty. such as ensuring no other own receipts + // is indexed for the same block. + savingResults sync.Mutex + getLatestFinalized func() (uint64, error) registerStore execution.RegisterStore // when it is true, registers are stored in both register store and ledger @@ -125,7 +126,8 @@ func NewExecutionState( events storage.Events, serviceEvents storage.ServiceEvents, transactionResults storage.TransactionResults, - db *badger.DB, + db storage.DB, + getLatestFinalized func() (uint64, error), tracer module.Tracer, registerStore execution.RegisterStore, enableRegisterStore bool, @@ -144,6 +146,8 @@ func NewExecutionState( serviceEvents: serviceEvents, transactionResults: transactionResults, db: db, + savingResults: sync.Mutex{}, + getLatestFinalized: getLatestFinalized, registerStore: registerStore, enableRegisterStore: enableRegisterStore, } @@ -385,7 +389,7 @@ func (s *state) SaveExecutionResults( } //outside batch because it requires read access - err = s.UpdateLastExecutedBlock(childCtx, result.ExecutableBlock.Block.Header) + err = s.UpdateLastExecutedBlock(childCtx, result.ExecutableBlock.ID()) if err != nil { return fmt.Errorf("cannot update highest executed block: %w", err) } @@ -396,8 +400,16 @@ func (s *state) saveExecutionResults( ctx context.Context, result *execution.ComputationResult, ) (err error) { - header := result.ExecutableBlock.Block.Header - blockID := header.ID() + // make sure only one saving operation is running at a time + // this is to prevent the reads within the saving operation to be dirty + // such as: + // - ensuring no other own receipts is indexed for the same block + // - ensuring no other receipts is indexed for the same block + // - ensuring no other results is indexed for the same block + s.savingResults.Lock() + defer s.savingResults.Unlock() + + blockID := result.ExecutableBlock.ID() err = s.chunkDataPacks.Store(result.AllChunkDataPacks()) if err != nil { @@ -409,72 +421,71 @@ func (s *state) saveExecutionResults( // as tightly as possible to let Badger manage it. // Note, that it does not guarantee atomicity as transactions has size limit, // but it's the closest thing to atomicity we could have - batch := badgerstorage.NewBatch(s.db) + return s.db.WithReaderBatchWriter(func(batch storage.ReaderBatchWriter) error { + + batch.AddCallback(func(err error) { + // Rollback if an error occurs during batch operations + if err != nil { + chunks := result.AllChunkDataPacks() + chunkIDs := make([]flow.Identifier, 0, len(chunks)) + for _, chunk := range chunks { + chunkIDs = append(chunkIDs, chunk.ID()) + } + _ = s.chunkDataPacks.Remove(chunkIDs) + } + }) - defer func() { - // Rollback if an error occurs during batch operations + err = s.events.BatchStore(blockID, []flow.EventsList{result.AllEvents()}, batch) if err != nil { - chunks := result.AllChunkDataPacks() - chunkIDs := make([]flow.Identifier, 0, len(chunks)) - for _, chunk := range chunks { - chunkIDs = append(chunkIDs, chunk.ID()) - } - _ = s.chunkDataPacks.Remove(chunkIDs) + return fmt.Errorf("cannot store events: %w", err) } - }() - - err = s.events.BatchStore(blockID, []flow.EventsList{result.AllEvents()}, batch) - if err != nil { - return fmt.Errorf("cannot store events: %w", err) - } - err = s.serviceEvents.BatchStore(blockID, result.AllServiceEvents(), batch) - if err != nil { - return fmt.Errorf("cannot store service events: %w", err) - } + err = s.serviceEvents.BatchStore(blockID, result.AllServiceEvents(), batch) + if err != nil { + return fmt.Errorf("cannot store service events: %w", err) + } - err = s.transactionResults.BatchStore( - blockID, - result.AllTransactionResults(), - batch) - if err != nil { - return fmt.Errorf("cannot store transaction result: %w", err) - } + err = s.transactionResults.BatchStore( + blockID, + result.AllTransactionResults(), + batch) + if err != nil { + return fmt.Errorf("cannot store transaction result: %w", err) + } - executionResult := &result.ExecutionReceipt.ExecutionResult - err = s.results.BatchStore(executionResult, batch) - if err != nil { - return fmt.Errorf("cannot store execution result: %w", err) - } + executionResult := &result.ExecutionReceipt.ExecutionResult + err = s.results.BatchStore(executionResult, batch) + if err != nil { + return fmt.Errorf("cannot store execution result: %w", err) + } - err = s.results.BatchIndex(blockID, executionResult.ID(), batch) - if err != nil { - return fmt.Errorf("cannot index execution result: %w", err) - } + err = s.results.BatchIndex(blockID, executionResult.ID(), batch) + if err != nil { + return fmt.Errorf("cannot index execution result: %w", err) + } - err = s.myReceipts.BatchStoreMyReceipt(result.ExecutionReceipt, batch) - if err != nil { - return fmt.Errorf("could not persist execution result: %w", err) - } + err = s.myReceipts.BatchStoreMyReceipt(result.ExecutionReceipt, batch) + if err != nil { + return fmt.Errorf("could not persist execution result: %w", err) + } - // the state commitment is the last data item to be stored, so that - // IsBlockExecuted can be implemented by checking whether state commitment exists - // in the database - err = s.commits.BatchStore(blockID, result.CurrentEndState(), batch) - if err != nil { - return fmt.Errorf("cannot store state commitment: %w", err) - } + // the state commitment is the last data item to be stored, so that + // IsBlockExecuted can be implemented by checking whether state commitment exists + // in the database + err = s.commits.BatchStore(blockID, result.CurrentEndState(), batch) + if err != nil { + return fmt.Errorf("cannot store state commitment: %w", err) + } - err = batch.Flush() - if err != nil { - return fmt.Errorf("batch flush error: %w", err) - } + return nil + }) - return nil } -func (s *state) UpdateLastExecutedBlock(ctx context.Context, header *flow.Header) error { - return operation.RetryOnConflict(s.db.Update, procedure.UpdateLastExecutedBlock(header)) +func (s *state) UpdateLastExecutedBlock(ctx context.Context, executedID flow.Identifier) error { + return s.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.UpdateExecutedBlock(rw.Writer(), executedID) + }) } // deprecated by storehouse's GetHighestFinalizedExecuted @@ -495,13 +506,17 @@ func (s *state) GetLastExecutedBlockID(ctx context.Context) (uint64, flow.Identi } var blockID flow.Identifier - var height uint64 - err := s.db.View(procedure.GetLastExecutedBlock(&height, &blockID)) + err := operation.RetrieveExecutedBlock(s.db.Reader(), &blockID) if err != nil { return 0, flow.ZeroID, err } - return height, blockID, nil + lastExecuted, err := s.headers.ByBlockID(blockID) + if err != nil { + return 0, flow.ZeroID, fmt.Errorf("could not retrieve executed header %v: %w", blockID, err) + } + + return lastExecuted.Height, blockID, nil } func (s *state) GetHighestFinalizedExecuted() (uint64, error) { @@ -510,10 +525,9 @@ func (s *state) GetHighestFinalizedExecuted() (uint64, error) { } // last finalized height - var finalizedHeight uint64 - err := s.db.View(operation.RetrieveFinalizedHeight(&finalizedHeight)) + finalizedHeight, err := s.getLatestFinalized() if err != nil { - return 0, fmt.Errorf("could not retrieve finalized height: %w", err) + return 0, fmt.Errorf("could not retrieve finalized: %w", err) } // last executed height diff --git a/engine/execution/state/state_storehouse_test.go b/engine/execution/state/state_storehouse_test.go index 70f30563525..b3e9aa75c2a 100644 --- a/engine/execution/state/state_storehouse_test.go +++ b/engine/execution/state/state_storehouse_test.go @@ -31,6 +31,7 @@ import ( badgerstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/storage/badger/operation" storage "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/storage/operation/badgerimpl" "github.com/onflow/flow-go/storage/pebble" "github.com/onflow/flow-go/utils/unittest" ) @@ -87,8 +88,14 @@ func prepareStorehouseTest(f func(t *testing.T, es state.ExecutionState, l *ledg headersDB := badgerstorage.NewHeaders(metrics, badgerDB) require.NoError(t, headersDB.Store(finalizedHeaders[10])) + getLatestFinalized := func() (uint64, error) { + return rootHeight, nil + } + es := state.NewExecutionState( - ls, stateCommitments, blocks, headers, collections, chunkDataPacks, results, myReceipts, events, serviceEvents, txResults, badgerDB, trace.NewNoopTracer(), + ls, stateCommitments, blocks, headers, collections, chunkDataPacks, results, myReceipts, events, serviceEvents, txResults, badgerimpl.ToDB(badgerDB), + getLatestFinalized, + trace.NewNoopTracer(), rs, true, ) diff --git a/engine/execution/state/state_test.go b/engine/execution/state/state_test.go index b9ba72b29f1..f545248a0ac 100644 --- a/engine/execution/state/state_test.go +++ b/engine/execution/state/state_test.go @@ -20,6 +20,7 @@ import ( "github.com/onflow/flow-go/module/trace" storageerr "github.com/onflow/flow-go/storage" storage "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/storage/operation/badgerimpl" "github.com/onflow/flow-go/utils/unittest" ) @@ -48,8 +49,13 @@ func prepareTest(f func(t *testing.T, es state.ExecutionState, l *ledger.Ledger, results := storage.NewExecutionResults(t) myReceipts := storage.NewMyExecutionReceipts(t) + getLatestFinalized := func() (uint64, error) { + return 0, nil + } + + db := badgerimpl.ToDB(badgerDB) es := state.NewExecutionState( - ls, stateCommitments, blocks, headers, collections, chunkDataPacks, results, myReceipts, events, serviceEvents, txResults, badgerDB, trace.NewNoopTracer(), + ls, stateCommitments, blocks, headers, collections, chunkDataPacks, results, myReceipts, events, serviceEvents, txResults, db, getLatestFinalized, trace.NewNoopTracer(), nil, false, ) diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index db1c7be4085..2235a36aa4a 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -429,8 +429,9 @@ func ConsensusNode(t *testing.T, hub *stub.Hub, identity bootstrap.NodeInfo, ide node := GenericNodeFromParticipants(t, hub, identity, identities, chainID) - resultsDB := storage.NewExecutionResults(node.Metrics, node.PublicDB) - receiptsDB := storage.NewExecutionReceipts(node.Metrics, node.PublicDB, resultsDB, storage.DefaultCacheSize) + db := badgerimpl.ToDB(node.PublicDB) + resultsDB := store.NewExecutionResults(node.Metrics, db) + receiptsDB := store.NewExecutionReceipts(node.Metrics, db, resultsDB, storage.DefaultCacheSize) guarantees, err := stdmap.NewGuarantees(1000) require.NoError(t, err) @@ -530,16 +531,17 @@ type CheckerMock struct { func ExecutionNode(t *testing.T, hub *stub.Hub, identity bootstrap.NodeInfo, identities []*flow.Identity, syncThreshold int, chainID flow.ChainID) testmock.ExecutionNode { node := GenericNodeFromParticipants(t, hub, identity, identities, chainID) + db := badgerimpl.ToDB(node.PublicDB) transactionsStorage := storage.NewTransactions(node.Metrics, node.PublicDB) collectionsStorage := storage.NewCollections(node.PublicDB, transactionsStorage) - eventsStorage := storage.NewEvents(node.Metrics, node.PublicDB) - serviceEventsStorage := storage.NewServiceEvents(node.Metrics, node.PublicDB) - txResultStorage := storage.NewTransactionResults(node.Metrics, node.PublicDB, storage.DefaultCacheSize) - commitsStorage := storage.NewCommits(node.Metrics, node.PublicDB) + eventsStorage := store.NewEvents(node.Metrics, db) + serviceEventsStorage := store.NewServiceEvents(node.Metrics, db) + txResultStorage := store.NewTransactionResults(node.Metrics, db, storage.DefaultCacheSize) + commitsStorage := store.NewCommits(node.Metrics, db) chunkDataPackStorage := store.NewChunkDataPacks(node.Metrics, badgerimpl.ToDB(node.PublicDB), collectionsStorage, 100) - results := storage.NewExecutionResults(node.Metrics, node.PublicDB) - receipts := storage.NewExecutionReceipts(node.Metrics, node.PublicDB, results, storage.DefaultCacheSize) - myReceipts := storage.NewMyExecutionReceipts(node.Metrics, node.PublicDB, receipts) + results := store.NewExecutionResults(node.Metrics, db) + receipts := store.NewExecutionReceipts(node.Metrics, db, results, storage.DefaultCacheSize) + myReceipts := store.NewMyExecutionReceipts(node.Metrics, db, receipts) versionBeacons := storage.NewVersionBeacons(node.PublicDB) headersStorage := storage.NewHeaders(node.Metrics, node.PublicDB) @@ -607,7 +609,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity bootstrap.NodeInfo, ide require.Equal(t, fmt.Sprintf("%x", rootSeal.FinalState), fmt.Sprintf("%x", commit)) require.Equal(t, rootSeal.ResultID, rootResult.ID()) - err = bootstrapper.BootstrapExecutionDatabase(node.PublicDB, rootSeal) + err = bootstrapper.BootstrapExecutionDatabase(db, rootSeal) require.NoError(t, err) registerDir := unittest.TempPebblePath(t) @@ -631,8 +633,15 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity bootstrap.NodeInfo, ide require.NoError(t, err) storehouseEnabled := true + getLatestFinalized := func() (uint64, error) { + final, err := protoState.Final().Head() + if err != nil { + return 0, err + } + return final.Height, nil + } execState := executionState.NewExecutionState( - ls, commitsStorage, node.Blocks, node.Headers, collectionsStorage, chunkDataPackStorage, results, myReceipts, eventsStorage, serviceEventsStorage, txResultStorage, node.PublicDB, node.Tracer, + ls, commitsStorage, node.Blocks, node.Headers, collectionsStorage, chunkDataPackStorage, results, myReceipts, eventsStorage, serviceEventsStorage, txResultStorage, db, getLatestFinalized, node.Tracer, // TODO: test with register store registerStore, storehouseEnabled, @@ -983,9 +992,10 @@ func VerificationNode(t testing.TB, } if node.Results == nil { - results := storage.NewExecutionResults(node.Metrics, node.PublicDB) + db := badgerimpl.ToDB(node.PublicDB) + results := store.NewExecutionResults(node.Metrics, db) node.Results = results - node.Receipts = storage.NewExecutionReceipts(node.Metrics, node.PublicDB, results, storage.DefaultCacheSize) + node.Receipts = store.NewExecutionReceipts(node.Metrics, db, results, storage.DefaultCacheSize) } if node.ProcessedChunkIndex == nil { diff --git a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go index 596a436684d..f0395a4584d 100644 --- a/integration/tests/access/cohort4/access_store_tx_error_messages_test.go +++ b/integration/tests/access/cohort4/access_store_tx_error_messages_test.go @@ -21,6 +21,8 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) const maxReceiptHeightMetric = "access_ingestion_max_receipt_height" @@ -218,7 +220,7 @@ func (s *AccessStoreTxErrorMessagesSuite) fetchTxErrorMessages(txResults []*sdk. require.NoError(s.T(), err, "could not open db") metrics := metrics.NewNoopCollector() - anTxErrorMessages := badger.NewTransactionResultErrorMessages(metrics, anDB, badger.DefaultCacheSize) + anTxErrorMessages := store.NewTransactionResultErrorMessages(metrics, badgerimpl.ToDB(anDB), badger.DefaultCacheSize) txResultErrorMessages := make([]*flow.TransactionResultErrorMessage, len(txResults)) for i, txResult := range txResults { diff --git a/module/state_synchronization/indexer/indexer_core.go b/module/state_synchronization/indexer/indexer_core.go index 22a6d16ea2a..9a709b840ba 100644 --- a/module/state_synchronization/indexer/indexer_core.go +++ b/module/state_synchronization/indexer/indexer_core.go @@ -15,7 +15,6 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/storage" - bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/utils/logging" ) @@ -30,7 +29,7 @@ type IndexerCore struct { collections storage.Collections transactions storage.Transactions results storage.LightTransactionResults - batcher bstorage.BatchBuilder + protocolDB storage.DB collectionExecutedMetric module.CollectionExecutedMetric @@ -44,7 +43,7 @@ type IndexerCore struct { func New( log zerolog.Logger, metrics module.ExecutionStateIndexerMetrics, - batcher bstorage.BatchBuilder, + protocolDB storage.DB, registers storage.RegisterIndex, headers storage.Headers, events storage.Events, @@ -66,7 +65,7 @@ func New( return &IndexerCore{ log: log, metrics: metrics, - batcher: batcher, + protocolDB: protocolDB, registers: registers, headers: headers, collections: collections, @@ -145,7 +144,7 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti results = append(results, chunk.TransactionResults...) } - batch := bstorage.NewBatch(c.batcher) + batch := c.protocolDB.NewBatch() err := c.events.BatchStore(data.BlockID, []flow.EventsList{events}, batch) if err != nil { @@ -157,7 +156,7 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti return fmt.Errorf("could not index transaction results at height %d: %w", header.Height, err) } - batch.Flush() + err = batch.Commit() if err != nil { return fmt.Errorf("batch flush error: %w", err) } diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index f446c0740a0..9bc9b8ae67d 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -26,6 +26,7 @@ import ( synctest "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/storage" storagemock "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/storage/operation/badgerimpl" pebbleStorage "github.com/onflow/flow-go/storage/pebble" "github.com/onflow/flow-go/utils/unittest" ) @@ -134,7 +135,7 @@ func (i *indexCoreTest) setStoreRegisters(f func(t *testing.T, entries flow.Regi func (i *indexCoreTest) setStoreEvents(f func(*testing.T, flow.Identifier, []flow.EventsList) error) *indexCoreTest { i.events. On("BatchStore", mock.AnythingOfType("flow.Identifier"), mock.AnythingOfType("[]flow.EventsList"), mock.Anything). - Return(func(blockID flow.Identifier, events []flow.EventsList, batch storage.BatchStorage) error { + Return(func(blockID flow.Identifier, events []flow.EventsList, batch storage.ReaderBatchWriter) error { require.NotNil(i.t, batch) return f(i.t, blockID, events) }) @@ -144,7 +145,7 @@ func (i *indexCoreTest) setStoreEvents(f func(*testing.T, flow.Identifier, []flo func (i *indexCoreTest) setStoreTransactionResults(f func(*testing.T, flow.Identifier, []flow.LightTransactionResult) error) *indexCoreTest { i.results. On("BatchStore", mock.AnythingOfType("flow.Identifier"), mock.AnythingOfType("[]flow.LightTransactionResult"), mock.Anything). - Return(func(blockID flow.Identifier, results []flow.LightTransactionResult, batch storage.BatchStorage) error { + Return(func(blockID flow.Identifier, results []flow.LightTransactionResult, batch storage.ReaderBatchWriter) error { require.NotNil(i.t, batch) return f(i.t, blockID, results) }) @@ -221,7 +222,7 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { indexer, err := New( log, metrics.NewNoopCollector(), - db, + badgerimpl.ToDB(db), i.registers, i.headers, i.events, @@ -699,7 +700,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { index, err := New( logger, metrics, - db, + badgerimpl.ToDB(db), registers, nil, nil, @@ -733,7 +734,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { index, err := New( logger, metrics, - db, + badgerimpl.ToDB(db), registers, nil, nil, @@ -760,7 +761,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { index, err := New( logger, metrics, - db, + badgerimpl.ToDB(db), registers, nil, nil, @@ -804,7 +805,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { index, err := New( logger, metrics, - db, + badgerimpl.ToDB(db), registers, nil, nil, diff --git a/storage/all.go b/storage/all.go index 26bb89bd454..d5474a94e08 100644 --- a/storage/all.go +++ b/storage/all.go @@ -2,27 +2,34 @@ 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 - TransactionResultErrorMessages TransactionResultErrorMessages - 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 + ChunkDataPacks ChunkDataPacks + Transactions Transactions + Collections Collections + EpochProtocolStateEntries EpochProtocolStateEntries + ProtocolKVStore ProtocolKVStore + VersionBeacons VersionBeacons + RegisterIndex RegisterIndex + + // These results are for reading and storing the result data from block payload + // EN uses a different results module to store their own results + // and receipts (see the Execution struct below) + Results ExecutionResults + Receipts ExecutionReceipts +} + +type Execution struct { + Results ExecutionResults + Receipts ExecutionReceipts + Commits Commits + TransactionResults TransactionResults + Events Events } diff --git a/storage/badger/all.go b/storage/badger/all.go index e0ea7483f73..458b8ca39fc 100644 --- a/storage/badger/all.go +++ b/storage/badger/all.go @@ -24,11 +24,8 @@ func InitAll(metrics module.CacheMetrics, db *badger.DB) *storage.All { protocolKVStore := NewProtocolKVStore(metrics, db, DefaultProtocolKVStoreCacheSize, DefaultProtocolKVStoreByBlockIDCacheSize) versionBeacons := NewVersionBeacons(db) - commits := NewCommits(metrics, db) transactions := NewTransactions(metrics, db) - transactionResults := NewTransactionResults(metrics, db, 10000) collections := NewCollections(db, transactions) - events := NewEvents(metrics, db) return &storage.All{ Headers: headers, @@ -45,10 +42,7 @@ func InitAll(metrics module.CacheMetrics, db *badger.DB) *storage.All { VersionBeacons: versionBeacons, Results: results, Receipts: receipts, - Commits: commits, Transactions: transactions, - TransactionResults: transactionResults, Collections: collections, - Events: events, } } diff --git a/storage/badger/commits.go b/storage/badger/commits.go deleted file mode 100644 index 11a4e4aa8e2..00000000000 --- a/storage/badger/commits.go +++ /dev/null @@ -1,89 +0,0 @@ -package badger - -import ( - "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" - "github.com/onflow/flow-go/storage/badger/transaction" -) - -type Commits struct { - db *badger.DB - cache *Cache[flow.Identifier, flow.StateCommitment] -} - -func NewCommits(collector module.CacheMetrics, db *badger.DB) *Commits { - - store := func(blockID flow.Identifier, commit flow.StateCommitment) func(*transaction.Tx) error { - return transaction.WithTx(operation.SkipDuplicates(operation.IndexStateCommitment(blockID, commit))) - } - - retrieve := func(blockID flow.Identifier) func(tx *badger.Txn) (flow.StateCommitment, error) { - return func(tx *badger.Txn) (flow.StateCommitment, error) { - var commit flow.StateCommitment - err := operation.LookupStateCommitment(blockID, &commit)(tx) - return commit, err - } - } - - c := &Commits{ - db: db, - cache: newCache[flow.Identifier, flow.StateCommitment](collector, metrics.ResourceCommit, - withLimit[flow.Identifier, flow.StateCommitment](1000), - withStore(store), - withRetrieve(retrieve), - ), - } - - return c -} - -func (c *Commits) storeTx(blockID flow.Identifier, commit flow.StateCommitment) func(*transaction.Tx) error { - return c.cache.PutTx(blockID, commit) -} - -func (c *Commits) retrieveTx(blockID flow.Identifier) func(tx *badger.Txn) (flow.StateCommitment, error) { - return func(tx *badger.Txn) (flow.StateCommitment, error) { - val, err := c.cache.Get(blockID)(tx) - if err != nil { - return flow.DummyStateCommitment, err - } - return val, nil - } -} - -func (c *Commits) Store(blockID flow.Identifier, commit flow.StateCommitment) error { - return operation.RetryOnConflictTx(c.db, transaction.Update, c.storeTx(blockID, commit)) -} - -// BatchStore stores Commit keyed by blockID in provided batch -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (c *Commits) BatchStore(blockID flow.Identifier, commit flow.StateCommitment, batch storage.BatchStorage) error { - // we can't cache while using batches, as it's unknown at this point when, and if - // the batch will be committed. Cache will be populated on read however. - writeBatch := batch.GetWriter() - return operation.BatchIndexStateCommitment(blockID, commit)(writeBatch) -} - -func (c *Commits) ByBlockID(blockID flow.Identifier) (flow.StateCommitment, error) { - tx := c.db.NewTransaction(false) - defer tx.Discard() - return c.retrieveTx(blockID)(tx) -} - -func (c *Commits) RemoveByBlockID(blockID flow.Identifier) error { - return c.db.Update(operation.SkipNonExist(operation.RemoveStateCommitment(blockID))) -} - -// BatchRemoveByBlockID removes Commit keyed by blockID in provided batch -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (c *Commits) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return operation.BatchRemoveStateCommitment(blockID)(writeBatch) -} diff --git a/storage/badger/common.go b/storage/badger/common.go deleted file mode 100644 index 77c6c5e7296..00000000000 --- a/storage/badger/common.go +++ /dev/null @@ -1,21 +0,0 @@ -package badger - -import ( - "errors" - "fmt" - - "github.com/dgraph-io/badger/v2" - - "github.com/onflow/flow-go/storage" -) - -func handleError(err error, t interface{}) error { - if err != nil { - if errors.Is(err, badger.ErrKeyNotFound) { - return storage.ErrNotFound - } - - return fmt.Errorf("could not retrieve %T: %w", t, err) - } - return nil -} diff --git a/storage/badger/my_receipts.go b/storage/badger/my_receipts.go deleted file mode 100644 index ff1584f44d6..00000000000 --- a/storage/badger/my_receipts.go +++ /dev/null @@ -1,159 +0,0 @@ -package badger - -import ( - "errors" - "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" - "github.com/onflow/flow-go/storage/badger/transaction" -) - -// MyExecutionReceipts holds and indexes Execution Receipts. -// MyExecutionReceipts is implemented as a wrapper around badger.ExecutionReceipts -// The wrapper adds the ability to "MY execution receipt", from the viewpoint -// of an individual Execution Node. -type MyExecutionReceipts struct { - genericReceipts *ExecutionReceipts - db *badger.DB - cache *Cache[flow.Identifier, *flow.ExecutionReceipt] -} - -// NewMyExecutionReceipts creates instance of MyExecutionReceipts which is a wrapper wrapper around badger.ExecutionReceipts -// It's useful for execution nodes to keep track of produced execution receipts. -func NewMyExecutionReceipts(collector module.CacheMetrics, db *badger.DB, receipts *ExecutionReceipts) *MyExecutionReceipts { - store := func(key flow.Identifier, receipt *flow.ExecutionReceipt) func(*transaction.Tx) error { - // assemble DB operations to store receipt (no execution) - storeReceiptOps := receipts.storeTx(receipt) - // assemble DB operations to index receipt as one of my own (no execution) - blockID := receipt.ExecutionResult.BlockID - receiptID := receipt.ID() - indexOwnReceiptOps := transaction.WithTx(func(tx *badger.Txn) error { - err := operation.IndexOwnExecutionReceipt(blockID, receiptID)(tx) - // check if we are storing same receipt - if errors.Is(err, storage.ErrAlreadyExists) { - var savedReceiptID flow.Identifier - err := operation.LookupOwnExecutionReceipt(blockID, &savedReceiptID)(tx) - if err != nil { - return err - } - - if savedReceiptID == receiptID { - // if we are storing same receipt we shouldn't error - return nil - } - - return fmt.Errorf("indexing my receipt %v failed: different receipt %v for the same block %v is already indexed", receiptID, - savedReceiptID, blockID) - } - return err - }) - - return func(tx *transaction.Tx) error { - err := storeReceiptOps(tx) // execute operations to store receipt - if err != nil { - return fmt.Errorf("could not store receipt: %w", err) - } - err = indexOwnReceiptOps(tx) // execute operations to index receipt as one of my own - if err != nil { - return fmt.Errorf("could not index receipt as one of my own: %w", err) - } - return nil - } - } - - retrieve := func(blockID flow.Identifier) func(tx *badger.Txn) (*flow.ExecutionReceipt, error) { - return func(tx *badger.Txn) (*flow.ExecutionReceipt, error) { - var receiptID flow.Identifier - err := operation.LookupOwnExecutionReceipt(blockID, &receiptID)(tx) - if err != nil { - return nil, fmt.Errorf("could not lookup receipt ID: %w", err) - } - receipt, err := receipts.byID(receiptID)(tx) - if err != nil { - return nil, err - } - return receipt, nil - } - } - - return &MyExecutionReceipts{ - genericReceipts: receipts, - db: db, - cache: newCache[flow.Identifier, *flow.ExecutionReceipt](collector, metrics.ResourceMyReceipt, - withLimit[flow.Identifier, *flow.ExecutionReceipt](flow.DefaultTransactionExpiry+100), - withStore(store), - withRetrieve(retrieve)), - } -} - -// storeMyReceipt assembles the operations to store the receipt and marks it as mine (trusted). -func (m *MyExecutionReceipts) storeMyReceipt(receipt *flow.ExecutionReceipt) func(*transaction.Tx) error { - return m.cache.PutTx(receipt.ExecutionResult.BlockID, receipt) -} - -// storeMyReceipt assembles the operations to retrieve my receipt for the given block ID. -func (m *MyExecutionReceipts) myReceipt(blockID flow.Identifier) func(*badger.Txn) (*flow.ExecutionReceipt, error) { - retrievalOps := m.cache.Get(blockID) // assemble DB operations to retrieve receipt (no execution) - return func(tx *badger.Txn) (*flow.ExecutionReceipt, error) { - val, err := retrievalOps(tx) // execute operations to retrieve receipt - if err != nil { - return nil, err - } - return val, nil - } -} - -// StoreMyReceipt stores the receipt and marks it as mine (trusted). My -// receipts are indexed by the block whose result they compute. Currently, -// we only support indexing a _single_ receipt per block. Attempting to -// store conflicting receipts for the same block will error. -func (m *MyExecutionReceipts) StoreMyReceipt(receipt *flow.ExecutionReceipt) error { - return operation.RetryOnConflictTx(m.db, transaction.Update, m.storeMyReceipt(receipt)) -} - -// BatchStoreMyReceipt stores blockID-to-my-receipt index entry keyed by blockID in a provided batch. -// No errors are expected during normal operation -// If entity fails marshalling, the error is wrapped in a generic error and returned. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (m *MyExecutionReceipts) BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, batch storage.BatchStorage) error { - - writeBatch := batch.GetWriter() - - err := m.genericReceipts.BatchStore(receipt, batch) - if err != nil { - return fmt.Errorf("cannot batch store generic execution receipt inside my execution receipt batch store: %w", err) - } - - err = operation.BatchIndexOwnExecutionReceipt(receipt.ExecutionResult.BlockID, receipt.ID())(writeBatch) - if err != nil { - return fmt.Errorf("cannot batch index own execution receipt inside my execution receipt batch store: %w", err) - } - - return nil -} - -// MyReceipt retrieves my receipt for the given block. -// Returns storage.ErrNotFound if no receipt was persisted for the block. -func (m *MyExecutionReceipts) MyReceipt(blockID flow.Identifier) (*flow.ExecutionReceipt, error) { - tx := m.db.NewTransaction(false) - defer tx.Discard() - return m.myReceipt(blockID)(tx) -} - -func (m *MyExecutionReceipts) RemoveIndexByBlockID(blockID flow.Identifier) error { - return m.db.Update(operation.SkipNonExist(operation.RemoveOwnExecutionReceipt(blockID))) -} - -// BatchRemoveIndexByBlockID removes blockID-to-my-execution-receipt index entry keyed by a blockID in a provided batch -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return operation.BatchRemoveOwnExecutionReceipt(blockID)(writeBatch) -} diff --git a/storage/badger/my_receipts_test.go b/storage/badger/my_receipts_test.go deleted file mode 100644 index 942c771f041..00000000000 --- a/storage/badger/my_receipts_test.go +++ /dev/null @@ -1,72 +0,0 @@ -package badger_test - -import ( - "testing" - - "github.com/dgraph-io/badger/v2" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/module/metrics" - bstorage "github.com/onflow/flow-go/storage/badger" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestMyExecutionReceiptsStorage(t *testing.T) { - withStore := func(t *testing.T, f func(store *bstorage.MyExecutionReceipts)) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { - metrics := metrics.NewNoopCollector() - results := bstorage.NewExecutionResults(metrics, db) - receipts := bstorage.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) - store := bstorage.NewMyExecutionReceipts(metrics, db, receipts) - - f(store) - }) - } - - t.Run("store one get one", func(t *testing.T) { - withStore(t, func(store *bstorage.MyExecutionReceipts) { - block := unittest.BlockFixture() - receipt1 := unittest.ReceiptForBlockFixture(&block) - - err := store.StoreMyReceipt(receipt1) - require.NoError(t, err) - - actual, err := store.MyReceipt(block.ID()) - require.NoError(t, err) - - require.Equal(t, receipt1, actual) - }) - }) - - t.Run("store same for the same block", func(t *testing.T) { - withStore(t, func(store *bstorage.MyExecutionReceipts) { - block := unittest.BlockFixture() - - receipt1 := unittest.ReceiptForBlockFixture(&block) - - err := store.StoreMyReceipt(receipt1) - require.NoError(t, err) - - err = store.StoreMyReceipt(receipt1) - require.NoError(t, err) - }) - }) - - t.Run("store different receipt for same block should fail", func(t *testing.T) { - withStore(t, func(store *bstorage.MyExecutionReceipts) { - block := unittest.BlockFixture() - - executor1 := unittest.IdentifierFixture() - executor2 := unittest.IdentifierFixture() - - receipt1 := unittest.ReceiptForBlockExecutorFixture(&block, executor1) - receipt2 := unittest.ReceiptForBlockExecutorFixture(&block, executor2) - - err := store.StoreMyReceipt(receipt1) - require.NoError(t, err) - - err = store.StoreMyReceipt(receipt2) - require.Error(t, err) - }) - }) -} diff --git a/storage/badger/operation/commits.go b/storage/badger/operation/commits.go deleted file mode 100644 index 260983da739..00000000000 --- a/storage/badger/operation/commits.go +++ /dev/null @@ -1,40 +0,0 @@ -package operation - -import ( - "github.com/dgraph-io/badger/v2" - - "github.com/onflow/flow-go/model/flow" -) - -// IndexStateCommitment indexes a state commitment. -// -// State commitments are keyed by the block whose execution results in the state with the given commit. -func IndexStateCommitment(blockID flow.Identifier, commit flow.StateCommitment) func(*badger.Txn) error { - return insert(makePrefix(codeCommit, blockID), commit) -} - -// BatchIndexStateCommitment indexes a state commitment into a batch -// -// State commitments are keyed by the block whose execution results in the state with the given commit. -func BatchIndexStateCommitment(blockID flow.Identifier, commit flow.StateCommitment) func(batch *badger.WriteBatch) error { - return batchWrite(makePrefix(codeCommit, blockID), commit) -} - -// LookupStateCommitment gets a state commitment keyed by block ID -// -// State commitments are keyed by the block whose execution results in the state with the given commit. -func LookupStateCommitment(blockID flow.Identifier, commit *flow.StateCommitment) func(*badger.Txn) error { - return retrieve(makePrefix(codeCommit, blockID), commit) -} - -// RemoveStateCommitment removes the state commitment by block ID -func RemoveStateCommitment(blockID flow.Identifier) func(*badger.Txn) error { - return remove(makePrefix(codeCommit, blockID)) -} - -// BatchRemoveStateCommitment batch removes the state commitment by block ID -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func BatchRemoveStateCommitment(blockID flow.Identifier) func(batch *badger.WriteBatch) error { - return batchRemove(makePrefix(codeCommit, blockID)) -} diff --git a/storage/badger/operation/commits_test.go b/storage/badger/operation/commits_test.go deleted file mode 100644 index 392331e935a..00000000000 --- a/storage/badger/operation/commits_test.go +++ /dev/null @@ -1,26 +0,0 @@ -package operation - -import ( - "testing" - - "github.com/dgraph-io/badger/v2" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestStateCommitments(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { - expected := unittest.StateCommitmentFixture() - id := unittest.IdentifierFixture() - err := db.Update(IndexStateCommitment(id, expected)) - require.Nil(t, err) - - var actual flow.StateCommitment - err = db.View(LookupStateCommitment(id, &actual)) - require.Nil(t, err) - assert.Equal(t, expected, actual) - }) -} diff --git a/storage/badger/operation/common.go b/storage/badger/operation/common.go index 1c293348231..2cb95faf14f 100644 --- a/storage/badger/operation/common.go +++ b/storage/badger/operation/common.go @@ -211,29 +211,6 @@ func removeByPrefix(prefix []byte) func(*badger.Txn) error { } } -// batchRemoveByPrefix removes all items under the keys match the given prefix in a batch write transaction. -// no error would be returned if no key was found with the given prefix. -// all error returned should be exception -func batchRemoveByPrefix(prefix []byte) func(tx *badger.Txn, writeBatch *badger.WriteBatch) error { - return func(tx *badger.Txn, writeBatch *badger.WriteBatch) error { - - opts := badger.DefaultIteratorOptions - opts.AllVersions = false - opts.PrefetchValues = false - it := tx.NewIterator(opts) - defer it.Close() - - for it.Seek(prefix); it.ValidForPrefix(prefix); it.Next() { - key := it.Item().KeyCopy(nil) - err := writeBatch.Delete(key) - if err != nil { - return irrecoverable.NewExceptionf("could not delete item in batch: %w", err) - } - } - return nil - } -} - // retrieve will retrieve the binary data under the given key from the badger DB // and decode it into the given entity. The provided entity needs to be a // pointer to an initialized entity of the correct type. diff --git a/storage/badger/operation/events.go b/storage/badger/operation/events.go deleted file mode 100644 index f7b0a1ee2d2..00000000000 --- a/storage/badger/operation/events.go +++ /dev/null @@ -1,113 +0,0 @@ -package operation - -import ( - "github.com/dgraph-io/badger/v2" - - "github.com/onflow/flow-go/model/flow" -) - -func eventPrefix(prefix byte, blockID flow.Identifier, event flow.Event) []byte { - return makePrefix(prefix, blockID, event.TransactionID, event.TransactionIndex, event.EventIndex) -} - -func InsertEvent(blockID flow.Identifier, event flow.Event) func(*badger.Txn) error { - return insert(eventPrefix(codeEvent, blockID, event), event) -} - -func BatchInsertEvent(blockID flow.Identifier, event flow.Event) func(batch *badger.WriteBatch) error { - return batchWrite(eventPrefix(codeEvent, blockID, event), event) -} - -func InsertServiceEvent(blockID flow.Identifier, event flow.Event) func(*badger.Txn) error { - return insert(eventPrefix(codeServiceEvent, blockID, event), event) -} - -func BatchInsertServiceEvent(blockID flow.Identifier, event flow.Event) func(batch *badger.WriteBatch) error { - return batchWrite(eventPrefix(codeServiceEvent, blockID, event), event) -} - -func RetrieveEvents(blockID flow.Identifier, transactionID flow.Identifier, events *[]flow.Event) func(*badger.Txn) error { - iterationFunc := eventIterationFunc(events) - return traverse(makePrefix(codeEvent, blockID, transactionID), iterationFunc) -} - -func LookupEventsByBlockID(blockID flow.Identifier, events *[]flow.Event) func(*badger.Txn) error { - iterationFunc := eventIterationFunc(events) - return traverse(makePrefix(codeEvent, blockID), iterationFunc) -} - -func LookupServiceEventsByBlockID(blockID flow.Identifier, events *[]flow.Event) func(*badger.Txn) error { - iterationFunc := eventIterationFunc(events) - return traverse(makePrefix(codeServiceEvent, blockID), iterationFunc) -} - -func LookupEventsByBlockIDEventType(blockID flow.Identifier, eventType flow.EventType, events *[]flow.Event) func(*badger.Txn) error { - iterationFunc := eventFilterIterationFunc(events, eventType) - return traverse(makePrefix(codeEvent, blockID), iterationFunc) -} - -func RemoveServiceEventsByBlockID(blockID flow.Identifier) func(*badger.Txn) error { - return removeByPrefix(makePrefix(codeServiceEvent, blockID)) -} - -// BatchRemoveServiceEventsByBlockID removes all service events for the given blockID. -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func BatchRemoveServiceEventsByBlockID(blockID flow.Identifier, batch *badger.WriteBatch) func(*badger.Txn) error { - return func(txn *badger.Txn) error { - return batchRemoveByPrefix(makePrefix(codeServiceEvent, blockID))(txn, batch) - } -} - -func RemoveEventsByBlockID(blockID flow.Identifier) func(*badger.Txn) error { - return removeByPrefix(makePrefix(codeEvent, blockID)) -} - -// BatchRemoveEventsByBlockID removes all events for the given blockID. -// No errors are expected during normal operation, even if no entries are matched. -// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func BatchRemoveEventsByBlockID(blockID flow.Identifier, batch *badger.WriteBatch) func(*badger.Txn) error { - return func(txn *badger.Txn) error { - return batchRemoveByPrefix(makePrefix(codeEvent, blockID))(txn, batch) - } - -} - -// eventIterationFunc returns an in iteration function which returns all events found during traversal or iteration -func eventIterationFunc(events *[]flow.Event) func() (checkFunc, createFunc, handleFunc) { - return func() (checkFunc, createFunc, handleFunc) { - check := func(key []byte) bool { - return true - } - var val flow.Event - create := func() interface{} { - return &val - } - handle := func() error { - *events = append(*events, val) - return nil - } - return check, create, handle - } -} - -// eventFilterIterationFunc returns an iteration function which filters the result by the given event type in the handleFunc -func eventFilterIterationFunc(events *[]flow.Event, eventType flow.EventType) func() (checkFunc, createFunc, handleFunc) { - return func() (checkFunc, createFunc, handleFunc) { - check := func(key []byte) bool { - return true - } - var val flow.Event - create := func() interface{} { - return &val - } - handle := func() error { - // filter out all events not of type eventType - if val.Type == eventType { - *events = append(*events, val) - } - return nil - } - return check, create, handle - } -} diff --git a/storage/badger/operation/transaction_results.go b/storage/badger/operation/transaction_results.go deleted file mode 100644 index c4264640364..00000000000 --- a/storage/badger/operation/transaction_results.go +++ /dev/null @@ -1,170 +0,0 @@ -package operation - -import ( - "fmt" - - "github.com/dgraph-io/badger/v2" - - "github.com/onflow/flow-go/model/flow" -) - -func InsertTransactionResult(blockID flow.Identifier, transactionResult *flow.TransactionResult) func(*badger.Txn) error { - return insert(makePrefix(codeTransactionResult, blockID, transactionResult.TransactionID), transactionResult) -} - -func BatchInsertTransactionResult(blockID flow.Identifier, transactionResult *flow.TransactionResult) func(batch *badger.WriteBatch) error { - return batchWrite(makePrefix(codeTransactionResult, blockID, transactionResult.TransactionID), transactionResult) -} - -func BatchIndexTransactionResult(blockID flow.Identifier, txIndex uint32, transactionResult *flow.TransactionResult) func(batch *badger.WriteBatch) error { - return batchWrite(makePrefix(codeTransactionResultIndex, blockID, txIndex), transactionResult) -} - -func RetrieveTransactionResult(blockID flow.Identifier, transactionID flow.Identifier, transactionResult *flow.TransactionResult) func(*badger.Txn) error { - return retrieve(makePrefix(codeTransactionResult, blockID, transactionID), transactionResult) -} -func RetrieveTransactionResultByIndex(blockID flow.Identifier, txIndex uint32, transactionResult *flow.TransactionResult) func(*badger.Txn) error { - return retrieve(makePrefix(codeTransactionResultIndex, blockID, txIndex), transactionResult) -} - -// LookupTransactionResultsByBlockIDUsingIndex retrieves all tx results for a block, by using -// tx_index index. This correctly handles cases of duplicate transactions within block. -func LookupTransactionResultsByBlockIDUsingIndex(blockID flow.Identifier, txResults *[]flow.TransactionResult) func(*badger.Txn) error { - - txErrIterFunc := func() (checkFunc, createFunc, handleFunc) { - check := func(_ []byte) bool { - return true - } - var val flow.TransactionResult - create := func() interface{} { - return &val - } - handle := func() error { - *txResults = append(*txResults, val) - return nil - } - return check, create, handle - } - - return traverse(makePrefix(codeTransactionResultIndex, blockID), txErrIterFunc) -} - -// RemoveTransactionResultsByBlockID removes the transaction results for the given blockID -func RemoveTransactionResultsByBlockID(blockID flow.Identifier) func(*badger.Txn) error { - return func(txn *badger.Txn) error { - - prefix := makePrefix(codeTransactionResult, blockID) - err := removeByPrefix(prefix)(txn) - if err != nil { - return fmt.Errorf("could not remove transaction results for block %v: %w", blockID, err) - } - - return nil - } -} - -// BatchRemoveTransactionResultsByBlockID removes transaction results for the given blockID in a provided batch. -// No errors are expected during normal operation, but it may return generic error -// if badger fails to process request -func BatchRemoveTransactionResultsByBlockID(blockID flow.Identifier, batch *badger.WriteBatch) func(*badger.Txn) error { - return func(txn *badger.Txn) error { - - prefix := makePrefix(codeTransactionResult, blockID) - err := batchRemoveByPrefix(prefix)(txn, batch) - if err != nil { - return fmt.Errorf("could not remove transaction results for block %v: %w", blockID, err) - } - - return nil - } -} - -func InsertLightTransactionResult(blockID flow.Identifier, transactionResult *flow.LightTransactionResult) func(*badger.Txn) error { - return insert(makePrefix(codeLightTransactionResult, blockID, transactionResult.TransactionID), transactionResult) -} - -func BatchInsertLightTransactionResult(blockID flow.Identifier, transactionResult *flow.LightTransactionResult) func(batch *badger.WriteBatch) error { - return batchWrite(makePrefix(codeLightTransactionResult, blockID, transactionResult.TransactionID), transactionResult) -} - -func BatchIndexLightTransactionResult(blockID flow.Identifier, txIndex uint32, transactionResult *flow.LightTransactionResult) func(batch *badger.WriteBatch) error { - return batchWrite(makePrefix(codeLightTransactionResultIndex, blockID, txIndex), transactionResult) -} - -func RetrieveLightTransactionResult(blockID flow.Identifier, transactionID flow.Identifier, transactionResult *flow.LightTransactionResult) func(*badger.Txn) error { - return retrieve(makePrefix(codeLightTransactionResult, blockID, transactionID), transactionResult) -} - -func RetrieveLightTransactionResultByIndex(blockID flow.Identifier, txIndex uint32, transactionResult *flow.LightTransactionResult) func(*badger.Txn) error { - return retrieve(makePrefix(codeLightTransactionResultIndex, blockID, txIndex), transactionResult) -} - -// LookupLightTransactionResultsByBlockIDUsingIndex retrieves all tx results for a block, but using -// tx_index index. This correctly handles cases of duplicate transactions within block. -func LookupLightTransactionResultsByBlockIDUsingIndex(blockID flow.Identifier, txResults *[]flow.LightTransactionResult) func(*badger.Txn) error { - - txErrIterFunc := func() (checkFunc, createFunc, handleFunc) { - check := func(_ []byte) bool { - return true - } - var val flow.LightTransactionResult - create := func() interface{} { - return &val - } - handle := func() error { - *txResults = append(*txResults, val) - return nil - } - return check, create, handle - } - - 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/procedure/executed.go b/storage/badger/procedure/executed.go deleted file mode 100644 index dfa1591c717..00000000000 --- a/storage/badger/procedure/executed.go +++ /dev/null @@ -1,45 +0,0 @@ -package procedure - -import ( - "errors" - "fmt" - - "github.com/dgraph-io/badger/v2" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/storage/badger/operation" -) - -// UpdateLastExecutedBlock updates the latest executed block to be the input block -func UpdateLastExecutedBlock(header *flow.Header) func(txn *badger.Txn) error { - return func(txn *badger.Txn) error { - err := operation.UpdateExecutedBlock(header.ID())(txn) - if err != nil { - return fmt.Errorf("cannot update highest executed block: %w", err) - } - - return nil - } -} - -// GetLastExecutedBlock retrieves the height and ID of the latest block executed by this node. -// Returns storage.ErrNotFound if no latest executed block has been stored. -func GetLastExecutedBlock(height *uint64, blockID *flow.Identifier) func(tx *badger.Txn) error { - return func(tx *badger.Txn) error { - var highest flow.Header - err := operation.RetrieveExecutedBlock(blockID)(tx) - if err != nil { - return fmt.Errorf("could not lookup executed block %v: %w", blockID, err) - } - err = operation.RetrieveHeader(*blockID, &highest)(tx) - if err != nil { - if errors.Is(err, storage.ErrNotFound) { - return fmt.Errorf("unexpected: latest executed block does not exist in storage: %s", err.Error()) - } - return fmt.Errorf("could not retrieve executed header %v: %w", blockID, err) - } - *height = highest.Height - return nil - } -} diff --git a/storage/badger/procedure/executed_test.go b/storage/badger/procedure/executed_test.go deleted file mode 100644 index 0f2f68b0012..00000000000 --- a/storage/badger/procedure/executed_test.go +++ /dev/null @@ -1,91 +0,0 @@ -package procedure - -import ( - "testing" - - "github.com/dgraph-io/badger/v2" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/storage/badger/operation" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestInsertExecuted(t *testing.T) { - chain, _, _ := unittest.ChainFixture(6) - unittest.RunWithBadgerDB(t, func(db *badger.DB) { - t.Run("setup and bootstrap", func(t *testing.T) { - for _, block := range chain { - require.NoError(t, db.Update(operation.InsertHeader(block.Header.ID(), block.Header))) - } - - root := chain[0].Header - require.NoError(t, - db.Update(operation.InsertExecutedBlock(root.ID())), - ) - - var height uint64 - var blockID flow.Identifier - require.NoError(t, - db.View(GetLastExecutedBlock(&height, &blockID)), - ) - - require.Equal(t, root.ID(), blockID) - require.Equal(t, root.Height, height) - }) - - t.Run("insert and get", func(t *testing.T) { - header1 := chain[1].Header - require.NoError(t, - db.Update(UpdateLastExecutedBlock(header1)), - ) - - var height uint64 - var blockID flow.Identifier - require.NoError(t, - db.View(GetLastExecutedBlock(&height, &blockID)), - ) - - require.Equal(t, header1.ID(), blockID) - require.Equal(t, header1.Height, height) - }) - - t.Run("insert more and get highest", func(t *testing.T) { - header2 := chain[2].Header - header3 := chain[3].Header - require.NoError(t, - db.Update(UpdateLastExecutedBlock(header2)), - ) - require.NoError(t, - db.Update(UpdateLastExecutedBlock(header3)), - ) - var height uint64 - var blockID flow.Identifier - require.NoError(t, - db.View(GetLastExecutedBlock(&height, &blockID)), - ) - - require.Equal(t, header3.ID(), blockID) - require.Equal(t, header3.Height, height) - }) - - t.Run("insert lower height later and get highest", func(t *testing.T) { - header5 := chain[5].Header - header4 := chain[4].Header - require.NoError(t, - db.Update(UpdateLastExecutedBlock(header5)), - ) - require.NoError(t, - db.Update(UpdateLastExecutedBlock(header4)), - ) - var height uint64 - var blockID flow.Identifier - require.NoError(t, - db.View(GetLastExecutedBlock(&height, &blockID)), - ) - - require.Equal(t, header4.ID(), blockID) - require.Equal(t, header4.Height, height) - }) - }) -} diff --git a/storage/badger/receipts.go b/storage/badger/receipts.go index b92c3961048..d566f46f261 100644 --- a/storage/badger/receipts.go +++ b/storage/badger/receipts.go @@ -21,6 +21,8 @@ type ExecutionReceipts struct { cache *Cache[flow.Identifier, *flow.ExecutionReceipt] } +var _ storage.ExecutionReceipts = (*ExecutionReceipts)(nil) + // NewExecutionReceipts Creates ExecutionReceipts instance which is a database of receipts which // supports storing and indexing receipts by receipt ID and block ID. func NewExecutionReceipts(collector module.CacheMetrics, db *badger.DB, results *ExecutionResults, cacheSize uint) *ExecutionReceipts { @@ -71,7 +73,7 @@ func NewExecutionReceipts(collector module.CacheMetrics, db *badger.DB, results return &ExecutionReceipts{ db: db, results: results, - cache: newCache[flow.Identifier, *flow.ExecutionReceipt](collector, metrics.ResourceReceipt, + cache: newCache(collector, metrics.ResourceReceipt, withLimit[flow.Identifier, *flow.ExecutionReceipt](cacheSize), withStore(store), withRetrieve(retrieve)), @@ -118,25 +120,8 @@ func (r *ExecutionReceipts) Store(receipt *flow.ExecutionReceipt) error { return operation.RetryOnConflictTx(r.db, transaction.Update, r.storeTx(receipt)) } -func (r *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - - err := r.results.BatchStore(&receipt.ExecutionResult, batch) - if err != nil { - return fmt.Errorf("cannot batch store execution result inside execution receipt batch store: %w", err) - } - - err = operation.BatchInsertExecutionReceiptMeta(receipt.ID(), receipt.Meta())(writeBatch) - if err != nil { - return fmt.Errorf("cannot batch store execution meta inside execution receipt batch store: %w", err) - } - - err = operation.BatchIndexExecutionReceipts(receipt.ExecutionResult.BlockID, receipt.ID())(writeBatch) - if err != nil { - return fmt.Errorf("cannot batch index execution receipt inside execution receipt batch store: %w", err) - } - - return nil +func (r *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, batch storage.ReaderBatchWriter) error { + return fmt.Errorf("not implemented") } func (r *ExecutionReceipts) ByID(receiptID flow.Identifier) (*flow.ExecutionReceipt, error) { diff --git a/storage/badger/results.go b/storage/badger/results.go index d4d1a4525b0..242bf0648a3 100644 --- a/storage/badger/results.go +++ b/storage/badger/results.go @@ -108,14 +108,12 @@ func (r *ExecutionResults) Store(result *flow.ExecutionResult) error { return operation.RetryOnConflictTx(r.db, transaction.Update, r.store(result)) } -func (r *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return operation.BatchInsertExecutionResult(result)(writeBatch) +func (r *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch storage.ReaderBatchWriter) error { + return fmt.Errorf("not implemented") } -func (r *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return operation.BatchIndexExecutionResult(blockID, resultID)(writeBatch) +func (r *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch storage.ReaderBatchWriter) error { + return fmt.Errorf("not implemented") } func (r *ExecutionResults) ByID(resultID flow.Identifier) (*flow.ExecutionResult, error) { @@ -160,7 +158,6 @@ func (r *ExecutionResults) RemoveIndexByBlockID(blockID flow.Identifier) error { // BatchRemoveIndexByBlockID removes blockID-to-executionResultID index entries keyed by blockID in a provided batch. // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (r *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return operation.BatchRemoveExecutionResultIndex(blockID)(writeBatch) +func (r *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { + return fmt.Errorf("not implemented") } diff --git a/storage/badger/results_test.go b/storage/badger/results_test.go index a23c8bf7232..e76ebdc4de9 100644 --- a/storage/badger/results_test.go +++ b/storage/badger/results_test.go @@ -54,34 +54,6 @@ func TestResultStoreTwice(t *testing.T) { }) } -func TestResultBatchStoreTwice(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { - metrics := metrics.NewNoopCollector() - store := bstorage.NewExecutionResults(metrics, db) - - result := unittest.ExecutionResultFixture() - blockID := unittest.IdentifierFixture() - - batch := bstorage.NewBatch(db) - err := store.BatchStore(result, batch) - require.NoError(t, err) - - err = store.BatchIndex(blockID, result.ID(), batch) - require.NoError(t, err) - - require.NoError(t, batch.Flush()) - - batch = bstorage.NewBatch(db) - err = store.BatchStore(result, batch) - require.NoError(t, err) - - err = store.BatchIndex(blockID, result.ID(), batch) - require.NoError(t, err) - - require.NoError(t, batch.Flush()) - }) -} - func TestResultStoreTwoDifferentResultsShouldFail(t *testing.T) { unittest.RunWithBadgerDB(t, func(db *badger.DB) { metrics := metrics.NewNoopCollector() diff --git a/storage/commits.go b/storage/commits.go index 1612c55cc9f..4fdcd5515a3 100644 --- a/storage/commits.go +++ b/storage/commits.go @@ -13,7 +13,7 @@ type Commits interface { // BatchStore stores Commit keyed by blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchStore(blockID flow.Identifier, commit flow.StateCommitment, batch BatchStorage) error + BatchStore(blockID flow.Identifier, commit flow.StateCommitment, batch ReaderBatchWriter) error // ByBlockID will retrieve a commit by its ID from persistent storage. ByBlockID(blockID flow.Identifier) (flow.StateCommitment, error) @@ -21,5 +21,5 @@ type Commits interface { // BatchRemoveByBlockID removes Commit keyed by blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchRemoveByBlockID(blockID flow.Identifier, batch BatchStorage) error + BatchRemoveByBlockID(blockID flow.Identifier, batch ReaderBatchWriter) error } diff --git a/storage/events.go b/storage/events.go index 6b9d9e6699c..86b72813904 100644 --- a/storage/events.go +++ b/storage/events.go @@ -10,7 +10,7 @@ type Events interface { Store(blockID flow.Identifier, blockEvents []flow.EventsList) error // BatchStore will store events for the given block ID in a given batch - BatchStore(blockID flow.Identifier, events []flow.EventsList, batch BatchStorage) error + BatchStore(blockID flow.Identifier, events []flow.EventsList, batch ReaderBatchWriter) error // ByBlockID returns the events for the given block ID ByBlockID(blockID flow.Identifier) ([]flow.Event, error) @@ -26,21 +26,21 @@ type Events interface { // BatchRemoveByBlockID removes events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. - // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchRemoveByBlockID(blockID flow.Identifier, batch BatchStorage) error + // If database unexpectedly fails to process the request, the error is wrapped in a generic error and returned. + BatchRemoveByBlockID(blockID flow.Identifier, batch ReaderBatchWriter) error } type ServiceEvents interface { // BatchStore stores service events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. - // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchStore(blockID flow.Identifier, events []flow.Event, batch BatchStorage) error + // If database unexpectedly fails to process the request, the error is wrapped in a generic error and returned. + BatchStore(blockID flow.Identifier, events []flow.Event, batch ReaderBatchWriter) error // ByBlockID returns the events for the given block ID ByBlockID(blockID flow.Identifier) ([]flow.Event, error) // BatchRemoveByBlockID removes service events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. - // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchRemoveByBlockID(blockID flow.Identifier, batch BatchStorage) error + // If database unexpectedly fails to process the request, the error is wrapped in a generic error and returned. + BatchRemoveByBlockID(blockID flow.Identifier, batch ReaderBatchWriter) error } diff --git a/storage/mock/commits.go b/storage/mock/commits.go index d8e1513dccb..1e1cd341813 100644 --- a/storage/mock/commits.go +++ b/storage/mock/commits.go @@ -15,7 +15,7 @@ type Commits struct { } // BatchRemoveByBlockID provides a mock function with given fields: blockID, batch -func (_m *Commits) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { +func (_m *Commits) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, batch) if len(ret) == 0 { @@ -23,7 +23,7 @@ func (_m *Commits) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.B } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, batch) } else { r0 = ret.Error(0) @@ -33,7 +33,7 @@ func (_m *Commits) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.B } // BatchStore provides a mock function with given fields: blockID, commit, batch -func (_m *Commits) BatchStore(blockID flow.Identifier, commit flow.StateCommitment, batch storage.BatchStorage) error { +func (_m *Commits) BatchStore(blockID flow.Identifier, commit flow.StateCommitment, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, commit, batch) if len(ret) == 0 { @@ -41,7 +41,7 @@ func (_m *Commits) BatchStore(blockID flow.Identifier, commit flow.StateCommitme } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, flow.StateCommitment, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, flow.StateCommitment, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, commit, batch) } else { r0 = ret.Error(0) diff --git a/storage/mock/events.go b/storage/mock/events.go index f23160b8c15..017bcf5fd1d 100644 --- a/storage/mock/events.go +++ b/storage/mock/events.go @@ -15,7 +15,7 @@ type Events struct { } // BatchRemoveByBlockID provides a mock function with given fields: blockID, batch -func (_m *Events) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { +func (_m *Events) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, batch) if len(ret) == 0 { @@ -23,7 +23,7 @@ func (_m *Events) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.Ba } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, batch) } else { r0 = ret.Error(0) @@ -33,7 +33,7 @@ func (_m *Events) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.Ba } // BatchStore provides a mock function with given fields: blockID, events, batch -func (_m *Events) BatchStore(blockID flow.Identifier, events []flow.EventsList, batch storage.BatchStorage) error { +func (_m *Events) BatchStore(blockID flow.Identifier, events []flow.EventsList, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, events, batch) if len(ret) == 0 { @@ -41,7 +41,7 @@ func (_m *Events) BatchStore(blockID flow.Identifier, events []flow.EventsList, } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.EventsList, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.EventsList, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, events, batch) } else { r0 = ret.Error(0) diff --git a/storage/mock/execution_receipts.go b/storage/mock/execution_receipts.go index da96d6dddb3..ee0b7e03135 100644 --- a/storage/mock/execution_receipts.go +++ b/storage/mock/execution_receipts.go @@ -15,7 +15,7 @@ type ExecutionReceipts struct { } // BatchStore provides a mock function with given fields: receipt, batch -func (_m *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, batch storage.BatchStorage) error { +func (_m *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, batch storage.ReaderBatchWriter) error { ret := _m.Called(receipt, batch) if len(ret) == 0 { @@ -23,7 +23,7 @@ func (_m *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, batch st } var r0 error - if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt, storage.ReaderBatchWriter) error); ok { r0 = rf(receipt, batch) } else { r0 = ret.Error(0) diff --git a/storage/mock/execution_results.go b/storage/mock/execution_results.go index f701356adae..bb13de105df 100644 --- a/storage/mock/execution_results.go +++ b/storage/mock/execution_results.go @@ -17,7 +17,7 @@ type ExecutionResults struct { } // BatchIndex provides a mock function with given fields: blockID, resultID, batch -func (_m *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch storage.BatchStorage) error { +func (_m *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, resultID, batch) if len(ret) == 0 { @@ -25,7 +25,7 @@ func (_m *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Id } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, resultID, batch) } else { r0 = ret.Error(0) @@ -35,7 +35,7 @@ func (_m *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Id } // BatchRemoveIndexByBlockID provides a mock function with given fields: blockID, batch -func (_m *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { +func (_m *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, batch) if len(ret) == 0 { @@ -43,7 +43,7 @@ func (_m *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, b } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, batch) } else { r0 = ret.Error(0) @@ -53,7 +53,7 @@ func (_m *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, b } // BatchStore provides a mock function with given fields: result, batch -func (_m *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch storage.BatchStorage) error { +func (_m *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch storage.ReaderBatchWriter) error { ret := _m.Called(result, batch) if len(ret) == 0 { @@ -61,7 +61,7 @@ func (_m *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch stora } var r0 error - if rf, ok := ret.Get(0).(func(*flow.ExecutionResult, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(*flow.ExecutionResult, storage.ReaderBatchWriter) error); ok { r0 = rf(result, batch) } else { r0 = ret.Error(0) diff --git a/storage/mock/light_transaction_results.go b/storage/mock/light_transaction_results.go index 5548377176b..46a8587fb15 100644 --- a/storage/mock/light_transaction_results.go +++ b/storage/mock/light_transaction_results.go @@ -14,14 +14,32 @@ type LightTransactionResults struct { mock.Mock } -// BatchStore provides a mock function with given fields: blockID, transactionResults, batch -func (_m *LightTransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch storage.BatchStorage) error { - ret := _m.Called(blockID, transactionResults, batch) +// BatchStore provides a mock function with given fields: blockID, transactionResults, rw +func (_m *LightTransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, rw storage.ReaderBatchWriter) error { + ret := _m.Called(blockID, transactionResults, rw) if len(ret) == 0 { panic("no return value specified for BatchStore") } + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.LightTransactionResult, storage.ReaderBatchWriter) error); ok { + r0 = rf(blockID, transactionResults, rw) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// BatchStoreBadger provides a mock function with given fields: blockID, transactionResults, batch +func (_m *LightTransactionResults) BatchStoreBadger(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch storage.BatchStorage) error { + ret := _m.Called(blockID, transactionResults, batch) + + if len(ret) == 0 { + panic("no return value specified for BatchStoreBadger") + } + var r0 error if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.LightTransactionResult, storage.BatchStorage) error); ok { r0 = rf(blockID, transactionResults, batch) diff --git a/storage/mock/my_execution_receipts.go b/storage/mock/my_execution_receipts.go index ea249927bb3..d694a8ca206 100644 --- a/storage/mock/my_execution_receipts.go +++ b/storage/mock/my_execution_receipts.go @@ -15,7 +15,7 @@ type MyExecutionReceipts struct { } // BatchRemoveIndexByBlockID provides a mock function with given fields: blockID, batch -func (_m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { +func (_m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, batch) if len(ret) == 0 { @@ -23,7 +23,7 @@ func (_m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, batch) } else { r0 = ret.Error(0) @@ -33,7 +33,7 @@ func (_m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier } // BatchStoreMyReceipt provides a mock function with given fields: receipt, batch -func (_m *MyExecutionReceipts) BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, batch storage.BatchStorage) error { +func (_m *MyExecutionReceipts) BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, batch storage.ReaderBatchWriter) error { ret := _m.Called(receipt, batch) if len(ret) == 0 { @@ -41,7 +41,7 @@ func (_m *MyExecutionReceipts) BatchStoreMyReceipt(receipt *flow.ExecutionReceip } var r0 error - if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt, storage.ReaderBatchWriter) error); ok { r0 = rf(receipt, batch) } else { r0 = ret.Error(0) @@ -80,24 +80,6 @@ func (_m *MyExecutionReceipts) MyReceipt(blockID flow.Identifier) (*flow.Executi return r0, r1 } -// StoreMyReceipt provides a mock function with given fields: receipt -func (_m *MyExecutionReceipts) StoreMyReceipt(receipt *flow.ExecutionReceipt) error { - ret := _m.Called(receipt) - - if len(ret) == 0 { - panic("no return value specified for StoreMyReceipt") - } - - var r0 error - if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt) error); ok { - r0 = rf(receipt) - } else { - r0 = ret.Error(0) - } - - return r0 -} - // NewMyExecutionReceipts creates a new instance of MyExecutionReceipts. 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 NewMyExecutionReceipts(t interface { diff --git a/storage/mock/service_events.go b/storage/mock/service_events.go index 923c8f700c5..fcb419e4fd1 100644 --- a/storage/mock/service_events.go +++ b/storage/mock/service_events.go @@ -15,7 +15,7 @@ type ServiceEvents struct { } // BatchRemoveByBlockID provides a mock function with given fields: blockID, batch -func (_m *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { +func (_m *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, batch) if len(ret) == 0 { @@ -23,7 +23,7 @@ func (_m *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, batch sto } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, batch) } else { r0 = ret.Error(0) @@ -33,7 +33,7 @@ func (_m *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, batch sto } // BatchStore provides a mock function with given fields: blockID, events, batch -func (_m *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event, batch storage.BatchStorage) error { +func (_m *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, events, batch) if len(ret) == 0 { @@ -41,7 +41,7 @@ func (_m *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.Event, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.Event, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, events, batch) } else { r0 = ret.Error(0) diff --git a/storage/mock/transaction_results.go b/storage/mock/transaction_results.go index bfe3e094fdf..7fcdae6f26d 100644 --- a/storage/mock/transaction_results.go +++ b/storage/mock/transaction_results.go @@ -14,8 +14,26 @@ type TransactionResults struct { mock.Mock } +// BatchRemoveByBlockID provides a mock function with given fields: id, batch +func (_m *TransactionResults) BatchRemoveByBlockID(id flow.Identifier, batch storage.ReaderBatchWriter) error { + ret := _m.Called(id, batch) + + if len(ret) == 0 { + panic("no return value specified for BatchRemoveByBlockID") + } + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier, storage.ReaderBatchWriter) error); ok { + r0 = rf(id, batch) + } else { + r0 = ret.Error(0) + } + + return r0 +} + // BatchStore provides a mock function with given fields: blockID, transactionResults, batch -func (_m *TransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch storage.BatchStorage) error { +func (_m *TransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch storage.ReaderBatchWriter) error { ret := _m.Called(blockID, transactionResults, batch) if len(ret) == 0 { @@ -23,7 +41,7 @@ func (_m *TransactionResults) BatchStore(blockID flow.Identifier, transactionRes } var r0 error - if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.TransactionResult, storage.BatchStorage) error); ok { + if rf, ok := ret.Get(0).(func(flow.Identifier, []flow.TransactionResult, storage.ReaderBatchWriter) error); ok { r0 = rf(blockID, transactionResults, batch) } else { r0 = ret.Error(0) diff --git a/storage/mocks/storage.go b/storage/mocks/storage.go index fddf323c5ad..448039c9346 100644 --- a/storage/mocks/storage.go +++ b/storage/mocks/storage.go @@ -435,7 +435,7 @@ func (m *MockCommits) EXPECT() *MockCommitsMockRecorder { } // BatchRemoveByBlockID mocks base method. -func (m *MockCommits) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.BatchStorage) error { +func (m *MockCommits) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchRemoveByBlockID", arg0, arg1) ret0, _ := ret[0].(error) @@ -449,7 +449,7 @@ func (mr *MockCommitsMockRecorder) BatchRemoveByBlockID(arg0, arg1 interface{}) } // BatchStore mocks base method. -func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.BatchStorage) error { +func (m *MockCommits) BatchStore(arg0 flow.Identifier, arg1 flow.StateCommitment, arg2 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) ret0, _ := ret[0].(error) @@ -515,7 +515,7 @@ func (m *MockEvents) EXPECT() *MockEventsMockRecorder { } // BatchRemoveByBlockID mocks base method. -func (m *MockEvents) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.BatchStorage) error { +func (m *MockEvents) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchRemoveByBlockID", arg0, arg1) ret0, _ := ret[0].(error) @@ -529,7 +529,7 @@ func (mr *MockEventsMockRecorder) BatchRemoveByBlockID(arg0, arg1 interface{}) * } // BatchStore mocks base method. -func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.BatchStorage) error { +func (m *MockEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.EventsList, arg2 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) ret0, _ := ret[0].(error) @@ -640,7 +640,7 @@ func (m *MockServiceEvents) EXPECT() *MockServiceEventsMockRecorder { } // BatchRemoveByBlockID mocks base method. -func (m *MockServiceEvents) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.BatchStorage) error { +func (m *MockServiceEvents) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchRemoveByBlockID", arg0, arg1) ret0, _ := ret[0].(error) @@ -654,7 +654,7 @@ func (mr *MockServiceEventsMockRecorder) BatchRemoveByBlockID(arg0, arg1 interfa } // BatchStore mocks base method. -func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.BatchStorage) error { +func (m *MockServiceEvents) BatchStore(arg0 flow.Identifier, arg1 []flow.Event, arg2 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) ret0, _ := ret[0].(error) @@ -705,8 +705,22 @@ func (m *MockTransactionResults) EXPECT() *MockTransactionResultsMockRecorder { return m.recorder } +// BatchRemoveByBlockID mocks base method. +func (m *MockTransactionResults) BatchRemoveByBlockID(arg0 flow.Identifier, arg1 storage.ReaderBatchWriter) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "BatchRemoveByBlockID", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// BatchRemoveByBlockID indicates an expected call of BatchRemoveByBlockID. +func (mr *MockTransactionResultsMockRecorder) BatchRemoveByBlockID(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BatchRemoveByBlockID", reflect.TypeOf((*MockTransactionResults)(nil).BatchRemoveByBlockID), arg0, arg1) +} + // BatchStore mocks base method. -func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.BatchStorage) error { +func (m *MockTransactionResults) BatchStore(arg0 flow.Identifier, arg1 []flow.TransactionResult, arg2 storage.ReaderBatchWriter) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BatchStore", arg0, arg1, arg2) ret0, _ := ret[0].(error) diff --git a/storage/operation/commits.go b/storage/operation/commits.go new file mode 100644 index 00000000000..1f80af9a768 --- /dev/null +++ b/storage/operation/commits.go @@ -0,0 +1,25 @@ +package operation + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +// IndexStateCommitment indexes a state commitment. +// +// State commitments are keyed by the block whose execution results in the state with the given commit. +func IndexStateCommitment(w storage.Writer, blockID flow.Identifier, commit flow.StateCommitment) error { + return UpsertByKey(w, MakePrefix(codeCommit, blockID), commit) +} + +// LookupStateCommitment gets a state commitment keyed by block ID +// +// State commitments are keyed by the block whose execution results in the state with the given commit. +func LookupStateCommitment(r storage.Reader, blockID flow.Identifier, commit *flow.StateCommitment) error { + return RetrieveByKey(r, MakePrefix(codeCommit, blockID), commit) +} + +// RemoveStateCommitment removes the state commitment by block ID +func RemoveStateCommitment(w storage.Writer, blockID flow.Identifier) error { + return RemoveByKey(w, MakePrefix(codeCommit, blockID)) +} diff --git a/storage/operation/commits_test.go b/storage/operation/commits_test.go new file mode 100644 index 00000000000..bee8dd2b21e --- /dev/null +++ b/storage/operation/commits_test.go @@ -0,0 +1,28 @@ +package operation_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestStateCommitments(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + expected := unittest.StateCommitmentFixture() + id := unittest.IdentifierFixture() + require.NoError(t, db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.IndexStateCommitment(rw.Writer(), id, expected) + })) + + var actual flow.StateCommitment + err := operation.LookupStateCommitment(db.Reader(), id, &actual) + require.Nil(t, err) + require.Equal(t, expected, actual) + }) +} diff --git a/storage/operation/events.go b/storage/operation/events.go new file mode 100644 index 00000000000..0a444408402 --- /dev/null +++ b/storage/operation/events.go @@ -0,0 +1,85 @@ +package operation + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +func eventPrefix(prefix byte, blockID flow.Identifier, event flow.Event) []byte { + return MakePrefix(prefix, blockID, event.TransactionID, event.TransactionIndex, event.EventIndex) +} + +func InsertEvent(w storage.Writer, blockID flow.Identifier, event flow.Event) error { + return UpsertByKey(w, eventPrefix(codeEvent, blockID, event), event) +} + +func InsertServiceEvent(w storage.Writer, blockID flow.Identifier, event flow.Event) error { + return UpsertByKey(w, eventPrefix(codeServiceEvent, blockID, event), event) +} + +func RetrieveEvents(r storage.Reader, blockID flow.Identifier, transactionID flow.Identifier, events *[]flow.Event) error { + iterationFunc := eventIterationFunc(events) + return TraverseByPrefix(r, MakePrefix(codeEvent, blockID, transactionID), iterationFunc, storage.DefaultIteratorOptions()) +} + +func LookupEventsByBlockID(r storage.Reader, blockID flow.Identifier, events *[]flow.Event) error { + iterationFunc := eventIterationFunc(events) + return TraverseByPrefix(r, MakePrefix(codeEvent, blockID), iterationFunc, storage.DefaultIteratorOptions()) +} + +func LookupServiceEventsByBlockID(r storage.Reader, blockID flow.Identifier, events *[]flow.Event) error { + iterationFunc := eventIterationFunc(events) + return TraverseByPrefix(r, MakePrefix(codeServiceEvent, blockID), iterationFunc, storage.DefaultIteratorOptions()) +} + +func LookupEventsByBlockIDEventType(r storage.Reader, blockID flow.Identifier, eventType flow.EventType, events *[]flow.Event) error { + iterationFunc := eventFilterIterationFunc(events, eventType) + return TraverseByPrefix(r, MakePrefix(codeEvent, blockID), iterationFunc, storage.DefaultIteratorOptions()) +} + +func RemoveServiceEventsByBlockID(r storage.Reader, w storage.Writer, blockID flow.Identifier) error { + return RemoveByKeyPrefix(r, w, MakePrefix(codeServiceEvent, blockID)) +} + +func RemoveEventsByBlockID(r storage.Reader, w storage.Writer, blockID flow.Identifier) error { + return RemoveByKeyPrefix(r, w, MakePrefix(codeEvent, blockID)) +} + +// eventIterationFunc returns an in iteration function which returns all events found during traversal or iteration +func eventIterationFunc(events *[]flow.Event) func() (CheckFunc, CreateFunc, HandleFunc) { + return func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(key []byte) (bool, error) { + return true, nil + } + var val flow.Event + create := func() interface{} { + return &val + } + handle := func() error { + *events = append(*events, val) + return nil + } + return check, create, handle + } +} + +// eventFilterIterationFunc returns an iteration function which filters the result by the given event type in the handleFunc +func eventFilterIterationFunc(events *[]flow.Event, eventType flow.EventType) func() (CheckFunc, CreateFunc, HandleFunc) { + return func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(key []byte) (bool, error) { + return true, nil + } + var val flow.Event + create := func() interface{} { + return &val + } + handle := func() error { + // filter out all events not of type eventType + if val.Type == eventType { + *events = append(*events, val) + } + return nil + } + return check, create, handle + } +} diff --git a/storage/badger/operation/events_test.go b/storage/operation/events_test.go similarity index 85% rename from storage/badger/operation/events_test.go rename to storage/operation/events_test.go index 9896c02fd69..7e8ce558241 100644 --- a/storage/badger/operation/events_test.go +++ b/storage/operation/events_test.go @@ -1,4 +1,4 @@ -package operation +package operation_test import ( "bytes" @@ -6,17 +6,19 @@ import ( "golang.org/x/exp/slices" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" "github.com/onflow/flow-go/utils/unittest" ) // TestRetrieveEventByBlockIDTxID tests event insertion, event retrieval by block id, block id and transaction id, // and block id and event type func TestRetrieveEventByBlockIDTxID(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { // create block ids, transaction ids and event types slices blockIDs := []flow.Identifier{flow.HashToID([]byte{0x01}), flow.HashToID([]byte{0x02})} @@ -46,7 +48,9 @@ func TestRetrieveEventByBlockIDTxID(t *testing.T) { event := unittest.EventFixture(etype, uint32(i), uint32(j), tx, 0) // insert event into the db - err := db.Update(InsertEvent(b, event)) + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.InsertEvent(rw.Writer(), b, event) + }) require.Nil(t, err) // update event arrays in the maps @@ -78,7 +82,7 @@ func TestRetrieveEventByBlockIDTxID(t *testing.T) { var actualEvents = make([]flow.Event, 0) // lookup events by block id - err := db.View(LookupEventsByBlockID(b, &actualEvents)) + err := operation.LookupEventsByBlockID(db.Reader(), b, &actualEvents) expectedEvents := blockMap[b.String()] assertFunc(err, expectedEvents, actualEvents) @@ -91,7 +95,7 @@ func TestRetrieveEventByBlockIDTxID(t *testing.T) { var actualEvents = make([]flow.Event, 0) //lookup events by block id and transaction id - err := db.View(RetrieveEvents(b, t, &actualEvents)) + err := operation.RetrieveEvents(db.Reader(), b, t, &actualEvents) expectedEvents := txMap[b.String()+"_"+t.String()] assertFunc(err, expectedEvents, actualEvents) @@ -105,7 +109,7 @@ func TestRetrieveEventByBlockIDTxID(t *testing.T) { var actualEvents = make([]flow.Event, 0) //lookup events by block id and transaction id - err := db.View(LookupEventsByBlockIDEventType(b, et, &actualEvents)) + err := operation.LookupEventsByBlockIDEventType(db.Reader(), b, et, &actualEvents) expectedEvents := typeMap[b.String()+"_"+string(et)] assertFunc(err, expectedEvents, actualEvents) diff --git a/storage/operation/executed.go b/storage/operation/executed.go new file mode 100644 index 00000000000..3c3e126586b --- /dev/null +++ b/storage/operation/executed.go @@ -0,0 +1,14 @@ +package operation + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +func UpdateExecutedBlock(w storage.Writer, blockID flow.Identifier) error { + return UpsertByKey(w, MakePrefix(codeExecutedBlock), blockID) +} + +func RetrieveExecutedBlock(r storage.Reader, blockID *flow.Identifier) error { + return RetrieveByKey(r, MakePrefix(codeExecutedBlock), blockID) +} diff --git a/storage/badger/operation/interactions.go b/storage/operation/interactions.go similarity index 60% rename from storage/badger/operation/interactions.go rename to storage/operation/interactions.go index 952b2f7a188..04c63635419 100644 --- a/storage/badger/operation/interactions.go +++ b/storage/operation/interactions.go @@ -3,23 +3,24 @@ package operation import ( "github.com/onflow/flow-go/fvm/storage/snapshot" "github.com/onflow/flow-go/model/flow" - - "github.com/dgraph-io/badger/v2" + "github.com/onflow/flow-go/storage" ) func InsertExecutionStateInteractions( + w storage.Writer, blockID flow.Identifier, executionSnapshots []*snapshot.ExecutionSnapshot, -) func(*badger.Txn) error { - return insert( - makePrefix(codeExecutionStateInteractions, blockID), +) error { + return UpsertByKey(w, + MakePrefix(codeExecutionStateInteractions, blockID), executionSnapshots) } func RetrieveExecutionStateInteractions( + r storage.Reader, blockID flow.Identifier, executionSnapshots *[]*snapshot.ExecutionSnapshot, -) func(*badger.Txn) error { - return retrieve( - makePrefix(codeExecutionStateInteractions, blockID), executionSnapshots) +) error { + return RetrieveByKey(r, + MakePrefix(codeExecutionStateInteractions, blockID), executionSnapshots) } diff --git a/storage/badger/operation/interactions_test.go b/storage/operation/interactions_test.go similarity index 72% rename from storage/badger/operation/interactions_test.go rename to storage/operation/interactions_test.go index b30d43d90bf..016fd2e5687 100644 --- a/storage/badger/operation/interactions_test.go +++ b/storage/operation/interactions_test.go @@ -1,19 +1,21 @@ -package operation +package operation_test import ( "testing" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/fvm/storage/snapshot" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" "github.com/onflow/flow-go/utils/unittest" ) func TestStateInteractionsInsertCheckRetrieve(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { id1 := flow.NewRegisterID( flow.BytesToAddress([]byte("\x89krg\u007fBN\x1d\xf5\xfb\xb8r\xbc4\xbd\x98Õ¼\xf1\xd0twU\xbf\x16N\xb4?,\xa0&;")), @@ -39,12 +41,14 @@ func TestStateInteractionsInsertCheckRetrieve(t *testing.T) { blockID := unittest.IdentifierFixture() - err := db.Update(InsertExecutionStateInteractions(blockID, interactions)) + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.InsertExecutionStateInteractions(rw.Writer(), blockID, interactions) + }) require.Nil(t, err) var readInteractions []*snapshot.ExecutionSnapshot - err = db.View(RetrieveExecutionStateInteractions(blockID, &readInteractions)) + err = operation.RetrieveExecutionStateInteractions(db.Reader(), blockID, &readInteractions) require.NoError(t, err) assert.Equal(t, interactions, readInteractions) diff --git a/storage/operation/receipts.go b/storage/operation/receipts.go new file mode 100644 index 00000000000..e6941eb5fe0 --- /dev/null +++ b/storage/operation/receipts.go @@ -0,0 +1,62 @@ +package operation + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +// InsertExecutionReceiptMeta inserts an execution receipt meta by ID. +func InsertExecutionReceiptMeta(w storage.Writer, receiptID flow.Identifier, meta *flow.ExecutionReceiptMeta) error { + return UpsertByKey(w, MakePrefix(codeExecutionReceiptMeta, receiptID), meta) +} + +// RetrieveExecutionReceiptMeta retrieves a execution receipt meta by ID. +func RetrieveExecutionReceiptMeta(r storage.Reader, receiptID flow.Identifier, meta *flow.ExecutionReceiptMeta) error { + return RetrieveByKey(r, MakePrefix(codeExecutionReceiptMeta, receiptID), meta) +} + +// IndexOwnExecutionReceipt inserts an execution receipt ID keyed by block ID +func IndexOwnExecutionReceipt(w storage.Writer, blockID flow.Identifier, receiptID flow.Identifier) error { + return UpsertByKey(w, MakePrefix(codeOwnBlockReceipt, blockID), receiptID) +} + +// LookupOwnExecutionReceipt finds execution receipt ID by block +func LookupOwnExecutionReceipt(r storage.Reader, blockID flow.Identifier, receiptID *flow.Identifier) error { + return RetrieveByKey(r, MakePrefix(codeOwnBlockReceipt, blockID), receiptID) +} + +// RemoveOwnExecutionReceipt removes own execution receipt index by blockID +func RemoveOwnExecutionReceipt(w storage.Writer, blockID flow.Identifier) error { + return RemoveByKey(w, MakePrefix(codeOwnBlockReceipt, blockID)) +} + +// IndexExecutionReceipts inserts an execution receipt ID keyed by block ID and receipt ID. +// one block could have multiple receipts, even if they are from the same executor +func IndexExecutionReceipts(w storage.Writer, blockID, receiptID flow.Identifier) error { + return UpsertByKey(w, MakePrefix(codeAllBlockReceipts, blockID, receiptID), receiptID) +} + +// LookupExecutionReceipts finds all execution receipts by block ID +func LookupExecutionReceipts(r storage.Reader, blockID flow.Identifier, receiptIDs *[]flow.Identifier) error { + iterationFunc := receiptIterationFunc(receiptIDs) + return TraverseByPrefix(r, MakePrefix(codeAllBlockReceipts, blockID), iterationFunc, storage.DefaultIteratorOptions()) +} + +// receiptIterationFunc returns an in iteration function which returns all receipt IDs found during traversal +func receiptIterationFunc(receiptIDs *[]flow.Identifier) func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(key []byte) (bool, error) { + return true, nil + } + + var receiptID flow.Identifier + create := func() interface{} { + return &receiptID + } + handle := func() error { + *receiptIDs = append(*receiptIDs, receiptID) + return nil + } + return func() (CheckFunc, CreateFunc, HandleFunc) { + return check, create, handle + } +} diff --git a/storage/operation/receipts_test.go b/storage/operation/receipts_test.go new file mode 100644 index 00000000000..5bef5ba3dd0 --- /dev/null +++ b/storage/operation/receipts_test.go @@ -0,0 +1,71 @@ +package operation_test + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestReceipts_InsertRetrieve(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + receipt := unittest.ExecutionReceiptFixture() + expected := receipt.Meta() + + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.InsertExecutionReceiptMeta(rw.Writer(), receipt.ID(), expected) + }) + require.Nil(t, err) + + var actual flow.ExecutionReceiptMeta + err = operation.RetrieveExecutionReceiptMeta(db.Reader(), receipt.ID(), &actual) + require.Nil(t, err) + + assert.Equal(t, expected, &actual) + }) +} + +func TestReceipts_Index(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + receipt := unittest.ExecutionReceiptFixture() + expected := receipt.ID() + blockID := receipt.ExecutionResult.BlockID + + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.IndexOwnExecutionReceipt(rw.Writer(), blockID, expected) + }) + require.Nil(t, err) + + var actual flow.Identifier + err = operation.LookupOwnExecutionReceipt(db.Reader(), blockID, &actual) + require.Nil(t, err) + + assert.Equal(t, expected, actual) + }) +} + +func TestReceipts_MultiIndex(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + expected := []flow.Identifier{unittest.IdentifierFixture(), unittest.IdentifierFixture()} + blockID := unittest.IdentifierFixture() + + require.NoError(t, db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + for _, id := range expected { + err := operation.IndexExecutionReceipts(rw.Writer(), blockID, id) + require.Nil(t, err) + } + return nil + })) + var actual []flow.Identifier + err := operation.LookupExecutionReceipts(db.Reader(), blockID, &actual) + require.Nil(t, err) + + assert.ElementsMatch(t, expected, actual) + }) +} diff --git a/storage/operation/results.go b/storage/operation/results.go new file mode 100644 index 00000000000..ec7f3c6f4c3 --- /dev/null +++ b/storage/operation/results.go @@ -0,0 +1,35 @@ +package operation + +import ( + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +// InsertExecutionResult inserts an execution result by ID. +func InsertExecutionResult(w storage.Writer, result *flow.ExecutionResult) error { + return UpsertByKey(w, MakePrefix(codeExecutionResult, result.ID()), result) +} + +// RetrieveExecutionResult retrieves a transaction by fingerprint. +func RetrieveExecutionResult(r storage.Reader, resultID flow.Identifier, result *flow.ExecutionResult) error { + return RetrieveByKey(r, MakePrefix(codeExecutionResult, resultID), result) +} + +// IndexExecutionResult inserts an execution result ID keyed by block ID +func IndexExecutionResult(w storage.Writer, blockID flow.Identifier, resultID flow.Identifier) error { + return UpsertByKey(w, MakePrefix(codeIndexExecutionResultByBlock, blockID), resultID) +} + +// LookupExecutionResult finds execution result ID by block +func LookupExecutionResult(r storage.Reader, blockID flow.Identifier, resultID *flow.Identifier) error { + return RetrieveByKey(r, MakePrefix(codeIndexExecutionResultByBlock, blockID), resultID) +} + +func ExistExecutionResult(r storage.Reader, blockID flow.Identifier) (bool, error) { + return KeyExists(r, MakePrefix(codeIndexExecutionResultByBlock, blockID)) +} + +// RemoveExecutionResultIndex removes execution result indexed by the given blockID +func RemoveExecutionResultIndex(w storage.Writer, blockID flow.Identifier) error { + return RemoveByKey(w, MakePrefix(codeIndexExecutionResultByBlock, blockID)) +} diff --git a/storage/operation/results_test.go b/storage/operation/results_test.go new file mode 100644 index 00000000000..a2f59425b1c --- /dev/null +++ b/storage/operation/results_test.go @@ -0,0 +1,31 @@ +package operation_test + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestResults_InsertRetrieve(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + expected := unittest.ExecutionResultFixture() + + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.InsertExecutionResult(rw.Writer(), expected) + }) + require.Nil(t, err) + + var actual flow.ExecutionResult + err = operation.RetrieveExecutionResult(db.Reader(), expected.ID(), &actual) + require.Nil(t, err) + + assert.Equal(t, expected, &actual) + }) +} diff --git a/storage/operation/transaction_results.go b/storage/operation/transaction_results.go new file mode 100644 index 00000000000..174a4f321a9 --- /dev/null +++ b/storage/operation/transaction_results.go @@ -0,0 +1,166 @@ +package operation + +import ( + "fmt" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" +) + +func InsertTransactionResult(w storage.Writer, blockID flow.Identifier, transactionResult *flow.TransactionResult) error { + return UpsertByKey(w, MakePrefix(codeTransactionResult, blockID, transactionResult.TransactionID), transactionResult) +} + +func IndexTransactionResult(w storage.Writer, blockID flow.Identifier, txIndex uint32, transactionResult *flow.TransactionResult) error { + return UpsertByKey(w, MakePrefix(codeTransactionResultIndex, blockID, txIndex), transactionResult) +} + +func RetrieveTransactionResult(r storage.Reader, blockID flow.Identifier, transactionID flow.Identifier, transactionResult *flow.TransactionResult) error { + return RetrieveByKey(r, MakePrefix(codeTransactionResult, blockID, transactionID), transactionResult) +} + +func RetrieveTransactionResultByIndex(r storage.Reader, blockID flow.Identifier, txIndex uint32, transactionResult *flow.TransactionResult) error { + return RetrieveByKey(r, MakePrefix(codeTransactionResultIndex, blockID, txIndex), transactionResult) +} + +// LookupTransactionResultsByBlockIDUsingIndex retrieves all tx results for a block, by using +// tx_index index. This correctly handles cases of duplicate transactions within block. +func LookupTransactionResultsByBlockIDUsingIndex(r storage.Reader, blockID flow.Identifier, txResults *[]flow.TransactionResult) error { + + txErrIterFunc := func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(_ []byte) (bool, error) { + return true, nil + } + var val flow.TransactionResult + create := func() interface{} { + return &val + } + handle := func() error { + *txResults = append(*txResults, val) + return nil + } + return check, create, handle + } + + return TraverseByPrefix(r, MakePrefix(codeTransactionResultIndex, blockID), txErrIterFunc, storage.DefaultIteratorOptions()) +} + +// RemoveTransactionResultsByBlockID removes the transaction results for the given blockID +func RemoveTransactionResultsByBlockID(r storage.Reader, w storage.Writer, blockID flow.Identifier) error { + prefix := MakePrefix(codeTransactionResult, blockID) + err := RemoveByKeyPrefix(r, w, prefix) + if err != nil { + return fmt.Errorf("could not remove transaction results for block %v: %w", blockID, err) + } + + return nil +} + +// BatchRemoveTransactionResultsByBlockID removes transaction results for the given blockID in a provided batch. +// No errors are expected during normal operation, but it may return generic error +// if badger fails to process request +func BatchRemoveTransactionResultsByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { + prefix := MakePrefix(codeTransactionResult, blockID) + err := RemoveByKeyPrefix(batch.GlobalReader(), batch.Writer(), prefix) + if err != nil { + return fmt.Errorf("could not remove transaction results for block %v: %w", blockID, err) + } + + return nil +} + +// deprecated +func InsertLightTransactionResult(w storage.Writer, blockID flow.Identifier, transactionResult *flow.LightTransactionResult) error { + return UpsertByKey(w, MakePrefix(codeLightTransactionResult, blockID, transactionResult.TransactionID), transactionResult) +} + +func BatchInsertLightTransactionResult(w storage.Writer, blockID flow.Identifier, transactionResult *flow.LightTransactionResult) error { + return UpsertByKey(w, MakePrefix(codeLightTransactionResult, blockID, transactionResult.TransactionID), transactionResult) +} + +func BatchIndexLightTransactionResult(w storage.Writer, blockID flow.Identifier, txIndex uint32, transactionResult *flow.LightTransactionResult) error { + return UpsertByKey(w, MakePrefix(codeLightTransactionResultIndex, blockID, txIndex), transactionResult) +} + +func RetrieveLightTransactionResult(r storage.Reader, blockID flow.Identifier, transactionID flow.Identifier, transactionResult *flow.LightTransactionResult) error { + return RetrieveByKey(r, MakePrefix(codeLightTransactionResult, blockID, transactionID), transactionResult) +} + +func RetrieveLightTransactionResultByIndex(r storage.Reader, blockID flow.Identifier, txIndex uint32, transactionResult *flow.LightTransactionResult) error { + return RetrieveByKey(r, MakePrefix(codeLightTransactionResultIndex, blockID, txIndex), transactionResult) +} + +// LookupLightTransactionResultsByBlockIDUsingIndex retrieves all tx results for a block, but using +// tx_index index. This correctly handles cases of duplicate transactions within block. +func LookupLightTransactionResultsByBlockIDUsingIndex(r storage.Reader, blockID flow.Identifier, txResults *[]flow.LightTransactionResult) error { + + txErrIterFunc := func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(_ []byte) (bool, error) { + return true, nil + } + var val flow.LightTransactionResult + create := func() interface{} { + return &val + } + handle := func() error { + *txResults = append(*txResults, val) + return nil + } + return check, create, handle + } + + return TraverseByPrefix(r, MakePrefix(codeLightTransactionResultIndex, blockID), txErrIterFunc, storage.DefaultIteratorOptions()) +} + +// BatchInsertTransactionResultErrorMessage inserts a transaction result error message by block ID and transaction ID +// into the database using a batch write. +func BatchInsertTransactionResultErrorMessage(w storage.Writer, blockID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) error { + return UpsertByKey(w, MakePrefix(codeTransactionResultErrorMessage, blockID, transactionResultErrorMessage.TransactionID), transactionResultErrorMessage) +} + +// BatchIndexTransactionResultErrorMessage indexes a transaction result error message by index within the block using a +// batch write. +func BatchIndexTransactionResultErrorMessage(w storage.Writer, blockID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) error { + return UpsertByKey(w, MakePrefix(codeTransactionResultErrorMessageIndex, blockID, transactionResultErrorMessage.Index), transactionResultErrorMessage) +} + +// RetrieveTransactionResultErrorMessage retrieves a transaction result error message by block ID and transaction ID. +func RetrieveTransactionResultErrorMessage(r storage.Reader, blockID flow.Identifier, transactionID flow.Identifier, transactionResultErrorMessage *flow.TransactionResultErrorMessage) error { + return RetrieveByKey(r, MakePrefix(codeTransactionResultErrorMessage, blockID, transactionID), transactionResultErrorMessage) +} + +// RetrieveTransactionResultErrorMessageByIndex retrieves a transaction result error message by block ID and index. +func RetrieveTransactionResultErrorMessageByIndex(r storage.Reader, blockID flow.Identifier, txIndex uint32, transactionResultErrorMessage *flow.TransactionResultErrorMessage) error { + return RetrieveByKey(r, MakePrefix(codeTransactionResultErrorMessageIndex, blockID, txIndex), transactionResultErrorMessage) +} + +// TransactionResultErrorMessagesExists checks whether tx result error messages exist in the database. +func TransactionResultErrorMessagesExists(r storage.Reader, blockID flow.Identifier, blockExists *bool) error { + exists, err := KeyExists(r, MakePrefix(codeTransactionResultErrorMessageIndex, blockID)) + if err != nil { + return err + } + *blockExists = exists + return nil +} + +// 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(r storage.Reader, blockID flow.Identifier, txResultErrorMessages *[]flow.TransactionResultErrorMessage) error { + txErrIterFunc := func() (CheckFunc, CreateFunc, HandleFunc) { + check := func(_ []byte) (bool, error) { + return true, nil + } + var val flow.TransactionResultErrorMessage + create := func() interface{} { + return &val + } + handle := func() error { + *txResultErrorMessages = append(*txResultErrorMessages, val) + return nil + } + return check, create, handle + } + + return TraverseByPrefix(r, MakePrefix(codeTransactionResultErrorMessageIndex, blockID), txErrIterFunc, storage.DefaultIteratorOptions()) +} diff --git a/storage/receipts.go b/storage/receipts.go index 0c1209e77cc..dd1ce3561eb 100644 --- a/storage/receipts.go +++ b/storage/receipts.go @@ -14,7 +14,7 @@ type ExecutionReceipts interface { Store(receipt *flow.ExecutionReceipt) error // BatchStore stores an execution receipt inside given batch - BatchStore(receipt *flow.ExecutionReceipt, batch BatchStorage) error + BatchStore(receipt *flow.ExecutionReceipt, batch ReaderBatchWriter) error // ByID retrieves an execution receipt by its ID. ByID(receiptID flow.Identifier) (*flow.ExecutionReceipt, error) @@ -28,17 +28,11 @@ type ExecutionReceipts interface { // them. Instead, it includes the "My" in the method name in order to highlight the notion // of "MY execution receipt", from the viewpoint of an individual Execution Node. type MyExecutionReceipts interface { - // StoreMyReceipt stores the receipt and marks it as mine (trusted). My - // receipts are indexed by the block whose result they compute. Currently, - // we only support indexing a _single_ receipt per block. Attempting to - // store conflicting receipts for the same block will error. - StoreMyReceipt(receipt *flow.ExecutionReceipt) error - // BatchStoreMyReceipt stores blockID-to-my-receipt index entry keyed by blockID in a provided batch. // No errors are expected during normal operation // If entity fails marshalling, the error is wrapped in a generic error and returned. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, batch BatchStorage) error + BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, batch ReaderBatchWriter) error // MyReceipt retrieves my receipt for the given block. MyReceipt(blockID flow.Identifier) (*flow.ExecutionReceipt, error) @@ -46,5 +40,5 @@ type MyExecutionReceipts interface { // BatchRemoveIndexByBlockID removes blockID-to-my-execution-receipt index entry keyed by a blockID in a provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchRemoveIndexByBlockID(blockID flow.Identifier, batch BatchStorage) error + BatchRemoveIndexByBlockID(blockID flow.Identifier, batch ReaderBatchWriter) error } diff --git a/storage/results.go b/storage/results.go index aa57c371b84..8e71fae60b1 100644 --- a/storage/results.go +++ b/storage/results.go @@ -11,13 +11,14 @@ type ExecutionResults interface { Store(result *flow.ExecutionResult) error // BatchStore stores an execution result in a given batch - BatchStore(result *flow.ExecutionResult, batch BatchStorage) error + BatchStore(result *flow.ExecutionResult, batch ReaderBatchWriter) error // ByID retrieves an execution result by its ID. Returns `ErrNotFound` if `resultID` is unknown. ByID(resultID flow.Identifier) (*flow.ExecutionResult, error) // ByIDTx returns a functor which retrieves the execution result by its ID, as part of a future database transaction. // When executing the functor, it returns `ErrNotFound` if no execution result with the respective ID is known. + // deprecated ByIDTx(resultID flow.Identifier) func(*transaction.Tx) (*flow.ExecutionResult, error) // Index indexes an execution result by block ID. @@ -27,7 +28,7 @@ type ExecutionResults interface { ForceIndex(blockID flow.Identifier, resultID flow.Identifier) error // BatchIndex indexes an execution result by block ID in a given batch - BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch BatchStorage) error + BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch ReaderBatchWriter) error // ByBlockID retrieves an execution result by block ID. ByBlockID(blockID flow.Identifier) (*flow.ExecutionResult, error) @@ -35,5 +36,5 @@ type ExecutionResults interface { // BatchRemoveIndexByBlockID removes blockID-to-executionResultID index entries keyed by blockID in a provided batch. // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. - BatchRemoveIndexByBlockID(blockID flow.Identifier, batch BatchStorage) error + BatchRemoveIndexByBlockID(blockID flow.Identifier, batch ReaderBatchWriter) error } diff --git a/storage/store/commits.go b/storage/store/commits.go new file mode 100644 index 00000000000..eb1599b72ba --- /dev/null +++ b/storage/store/commits.go @@ -0,0 +1,82 @@ +package store + +import ( + "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/operation" +) + +type Commits struct { + db storage.DB + cache *Cache[flow.Identifier, flow.StateCommitment] +} + +func NewCommits(collector module.CacheMetrics, db storage.DB) *Commits { + + store := func(rw storage.ReaderBatchWriter, blockID flow.Identifier, commit flow.StateCommitment) error { + return operation.IndexStateCommitment(rw.Writer(), blockID, commit) + } + + retrieve := func(r storage.Reader, blockID flow.Identifier) (flow.StateCommitment, error) { + var commit flow.StateCommitment + err := operation.LookupStateCommitment(r, blockID, &commit) + return commit, err + } + + c := &Commits{ + db: db, + cache: newCache(collector, metrics.ResourceCommit, + withLimit[flow.Identifier, flow.StateCommitment](1000), + withStore(store), + withRetrieve(retrieve), + ), + } + + return c +} + +func (c *Commits) storeTx(rw storage.ReaderBatchWriter, blockID flow.Identifier, commit flow.StateCommitment) error { + return c.cache.PutTx(rw, blockID, commit) +} + +func (c *Commits) retrieveTx(r storage.Reader, blockID flow.Identifier) (flow.StateCommitment, error) { + val, err := c.cache.Get(r, blockID) + if err != nil { + return flow.DummyStateCommitment, err + } + return val, nil +} + +func (c *Commits) Store(blockID flow.Identifier, commit flow.StateCommitment) error { + return c.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return c.storeTx(rw, blockID, commit) + }) +} + +// BatchStore stores Commit keyed by blockID in provided batch +// No errors are expected during normal operation, even if no entries are matched. +// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. +func (c *Commits) BatchStore(blockID flow.Identifier, commit flow.StateCommitment, rw storage.ReaderBatchWriter) error { + // we can't cache while using batches, as it's unknown at this point when, and if + // the batch will be committed. Cache will be populated on read however. + return operation.IndexStateCommitment(rw.Writer(), blockID, commit) +} + +func (c *Commits) ByBlockID(blockID flow.Identifier) (flow.StateCommitment, error) { + return c.retrieveTx(c.db.Reader(), blockID) +} + +func (c *Commits) RemoveByBlockID(blockID flow.Identifier) error { + return c.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveStateCommitment(rw.Writer(), blockID) + }) +} + +// BatchRemoveByBlockID removes Commit keyed by blockID in provided batch +// No errors are expected during normal operation, even if no entries are matched. +// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. +func (c *Commits) BatchRemoveByBlockID(blockID flow.Identifier, rw storage.ReaderBatchWriter) error { + return operation.RemoveStateCommitment(rw.Writer(), blockID) +} diff --git a/storage/badger/commit_test.go b/storage/store/commits_test.go similarity index 56% rename from storage/badger/commit_test.go rename to storage/store/commits_test.go index 25527c31c61..a1547e44d51 100644 --- a/storage/badger/commit_test.go +++ b/storage/store/commits_test.go @@ -1,10 +1,9 @@ -package badger_test +package store_test import ( "errors" "testing" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -12,32 +11,33 @@ import ( "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/unittest" - badgerstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" ) -// TestCommitsStoreAndRetrieve tests that a commit can be stored, retrieved and attempted to be stored again without an error +// TestCommitsStoreAndRetrieve tests that a commit can be store1d, retrieved and attempted to be stored again without an error func TestCommitsStoreAndRetrieve(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := badgerstorage.NewCommits(metrics, db) + store1 := store.NewCommits(metrics, db) // attempt to get a invalid commit - _, err := store.ByBlockID(unittest.IdentifierFixture()) + _, err := store1.ByBlockID(unittest.IdentifierFixture()) assert.True(t, errors.Is(err, storage.ErrNotFound)) - // store a commit in db + // store1 a commit in db blockID := unittest.IdentifierFixture() expected := unittest.StateCommitmentFixture() - err = store.Store(blockID, expected) + err = store1.Store(blockID, expected) require.NoError(t, err) // retrieve the commit by ID - actual, err := store.ByBlockID(blockID) + actual, err := store1.ByBlockID(blockID) require.NoError(t, err) assert.Equal(t, expected, actual) // re-insert the commit - should be idempotent - err = store.Store(blockID, expected) + err = store1.Store(blockID, expected) require.NoError(t, err) }) } diff --git a/storage/badger/events.go b/storage/store/events.go similarity index 67% rename from storage/badger/events.go rename to storage/store/events.go index ca7cb5105ec..929d666b0c0 100644 --- a/storage/badger/events.go +++ b/storage/store/events.go @@ -1,34 +1,32 @@ -package badger +package store 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" + "github.com/onflow/flow-go/storage/operation" ) type Events struct { - db *badger.DB + db storage.DB cache *Cache[flow.Identifier, []flow.Event] } -func NewEvents(collector module.CacheMetrics, db *badger.DB) *Events { - retrieve := func(blockID flow.Identifier) func(tx *badger.Txn) ([]flow.Event, error) { +var _ storage.Events = (*Events)(nil) + +func NewEvents(collector module.CacheMetrics, db storage.DB) *Events { + retrieve := func(r storage.Reader, blockID flow.Identifier) ([]flow.Event, error) { var events []flow.Event - return func(tx *badger.Txn) ([]flow.Event, error) { - err := operation.LookupEventsByBlockID(blockID, &events)(tx) - return events, handleError(err, flow.Event{}) - } + err := operation.LookupEventsByBlockID(r, blockID, &events) + return events, err } return &Events{ db: db, - cache: newCache[flow.Identifier, []flow.Event](collector, metrics.ResourceEvents, + cache: newCache(collector, metrics.ResourceEvents, withStore(noopStore[flow.Identifier, []flow.Event]), withRetrieve(retrieve)), } @@ -37,8 +35,8 @@ func NewEvents(collector module.CacheMetrics, db *badger.DB) *Events { // BatchStore stores events keyed by a blockID in provided batch // No errors are expected during normal operation, but it may return generic error // if badger fails to process request -func (e *Events) BatchStore(blockID flow.Identifier, blockEvents []flow.EventsList, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() +func (e *Events) BatchStore(blockID flow.Identifier, blockEvents []flow.EventsList, batch storage.ReaderBatchWriter) error { + writer := batch.Writer() // pre-allocating and indexing slice is faster than appending sliceSize := 0 @@ -52,7 +50,7 @@ func (e *Events) BatchStore(blockID flow.Identifier, blockEvents []flow.EventsLi for _, events := range blockEvents { for _, event := range events { - err := operation.BatchInsertEvent(blockID, event)(writeBatch) + err := operation.InsertEvent(writer, blockID, event) if err != nil { return fmt.Errorf("cannot batch insert event: %w", err) } @@ -64,33 +62,21 @@ func (e *Events) BatchStore(blockID flow.Identifier, blockEvents []flow.EventsLi callback := func() { e.cache.Insert(blockID, combinedEvents) } - batch.OnSucceed(callback) + storage.OnCommitSucceed(batch, callback) return nil } // Store will store events for the given block ID func (e *Events) Store(blockID flow.Identifier, blockEvents []flow.EventsList) error { - batch := NewBatch(e.db) - - err := e.BatchStore(blockID, blockEvents, batch) - if err != nil { - return err - } - - err = batch.Flush() - if err != nil { - return fmt.Errorf("cannot flush batch: %w", err) - } - - return nil + return e.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return e.BatchStore(blockID, blockEvents, rw) + }) } // ByBlockID returns the events for the given block ID // Note: This method will return an empty slice and no error if no entries for the blockID are found func (e *Events) ByBlockID(blockID flow.Identifier) ([]flow.Event, error) { - tx := e.db.NewTransaction(false) - defer tx.Discard() - val, err := e.cache.Get(blockID)(tx) + val, err := e.cache.Get(e.db.Reader(), blockID) if err != nil { return nil, err } @@ -102,7 +88,7 @@ func (e *Events) ByBlockID(blockID flow.Identifier) ([]flow.Event, error) { func (e *Events) ByBlockIDTransactionID(blockID flow.Identifier, txID flow.Identifier) ([]flow.Event, error) { events, err := e.ByBlockID(blockID) if err != nil { - return nil, handleError(err, flow.Event{}) + return nil, err } var matched []flow.Event @@ -119,7 +105,7 @@ func (e *Events) ByBlockIDTransactionID(blockID flow.Identifier, txID flow.Ident func (e *Events) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) ([]flow.Event, error) { events, err := e.ByBlockID(blockID) if err != nil { - return nil, handleError(err, flow.Event{}) + return nil, err } var matched []flow.Event @@ -136,7 +122,7 @@ func (e *Events) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint func (e *Events) ByBlockIDEventType(blockID flow.Identifier, eventType flow.EventType) ([]flow.Event, error) { events, err := e.ByBlockID(blockID) if err != nil { - return nil, handleError(err, flow.Event{}) + return nil, err } var matched []flow.Event @@ -150,34 +136,33 @@ func (e *Events) ByBlockIDEventType(blockID flow.Identifier, eventType flow.Even // RemoveByBlockID removes events by block ID func (e *Events) RemoveByBlockID(blockID flow.Identifier) error { - return e.db.Update(operation.RemoveEventsByBlockID(blockID)) + return e.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveEventsByBlockID(rw.GlobalReader(), rw.Writer(), blockID) + }) } // BatchRemoveByBlockID removes events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (e *Events) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return e.db.View(operation.BatchRemoveEventsByBlockID(blockID, writeBatch)) +func (e *Events) BatchRemoveByBlockID(blockID flow.Identifier, rw storage.ReaderBatchWriter) error { + return operation.RemoveEventsByBlockID(rw.GlobalReader(), rw.Writer(), blockID) } type ServiceEvents struct { - db *badger.DB + db storage.DB cache *Cache[flow.Identifier, []flow.Event] } -func NewServiceEvents(collector module.CacheMetrics, db *badger.DB) *ServiceEvents { - retrieve := func(blockID flow.Identifier) func(tx *badger.Txn) ([]flow.Event, error) { +func NewServiceEvents(collector module.CacheMetrics, db storage.DB) *ServiceEvents { + retrieve := func(r storage.Reader, blockID flow.Identifier) ([]flow.Event, error) { var events []flow.Event - return func(tx *badger.Txn) ([]flow.Event, error) { - err := operation.LookupServiceEventsByBlockID(blockID, &events)(tx) - return events, handleError(err, flow.Event{}) - } + err := operation.LookupServiceEventsByBlockID(r, blockID, &events) + return events, err } return &ServiceEvents{ db: db, - cache: newCache[flow.Identifier, []flow.Event](collector, metrics.ResourceEvents, + cache: newCache(collector, metrics.ResourceEvents, withStore(noopStore[flow.Identifier, []flow.Event]), withRetrieve(retrieve)), } @@ -186,10 +171,10 @@ func NewServiceEvents(collector module.CacheMetrics, db *badger.DB) *ServiceEven // BatchStore stores service events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (e *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() +func (e *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event, rw storage.ReaderBatchWriter) error { + writer := rw.Writer() for _, event := range events { - err := operation.BatchInsertServiceEvent(blockID, event)(writeBatch) + err := operation.InsertServiceEvent(writer, blockID, event) if err != nil { return fmt.Errorf("cannot batch insert service event: %w", err) } @@ -198,15 +183,13 @@ func (e *ServiceEvents) BatchStore(blockID flow.Identifier, events []flow.Event, callback := func() { e.cache.Insert(blockID, events) } - batch.OnSucceed(callback) + storage.OnCommitSucceed(rw, callback) return nil } // ByBlockID returns the events for the given block ID func (e *ServiceEvents) ByBlockID(blockID flow.Identifier) ([]flow.Event, error) { - tx := e.db.NewTransaction(false) - defer tx.Discard() - val, err := e.cache.Get(blockID)(tx) + val, err := e.cache.Get(e.db.Reader(), blockID) if err != nil { return nil, err } @@ -215,13 +198,16 @@ func (e *ServiceEvents) ByBlockID(blockID flow.Identifier) ([]flow.Event, error) // RemoveByBlockID removes service events by block ID func (e *ServiceEvents) RemoveByBlockID(blockID flow.Identifier) error { - return e.db.Update(operation.RemoveServiceEventsByBlockID(blockID)) + return e.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveServiceEventsByBlockID(rw.GlobalReader(), rw.Writer(), blockID) + }) } // BatchRemoveByBlockID removes service events keyed by a blockID in provided batch // No errors are expected during normal operation, even if no entries are matched. // If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. -func (e *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return e.db.View(operation.BatchRemoveServiceEventsByBlockID(blockID, writeBatch)) +func (e *ServiceEvents) BatchRemoveByBlockID(blockID flow.Identifier, rw storage.ReaderBatchWriter) error { + return e.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveServiceEventsByBlockID(rw.GlobalReader(), rw.Writer(), blockID) + }) } diff --git a/storage/badger/events_test.go b/storage/store/events_test.go similarity index 62% rename from storage/badger/events_test.go rename to storage/store/events_test.go index cb0e956395c..ae0e08846d6 100644 --- a/storage/badger/events_test.go +++ b/storage/store/events_test.go @@ -1,23 +1,24 @@ -package badger_test +package store_test import ( "math/rand" "testing" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" - badgerstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" ) func TestEventStoreRetrieve(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := badgerstorage.NewEvents(metrics, db) + events := store.NewEvents(metrics, db) blockID := unittest.IdentifierFixture() tx1ID := unittest.IdentifierFixture() @@ -32,16 +33,13 @@ func TestEventStoreRetrieve(t *testing.T) { {evt2_1}, } - batch := badgerstorage.NewBatch(db) - // store event - err := store.BatchStore(blockID, expected, batch) - require.NoError(t, err) - - err = batch.Flush() - require.NoError(t, err) + require.NoError(t, db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + // store event + return events.BatchStore(blockID, expected, rw) + })) // retrieve by blockID - actual, err := store.ByBlockID(blockID) + actual, err := events.ByBlockID(blockID) require.NoError(t, err) require.Len(t, actual, 3) require.Contains(t, actual, evt1_1) @@ -49,38 +47,38 @@ func TestEventStoreRetrieve(t *testing.T) { require.Contains(t, actual, evt2_1) // retrieve by blockID and event type - actual, err = store.ByBlockIDEventType(blockID, flow.EventAccountCreated) + actual, err = events.ByBlockIDEventType(blockID, flow.EventAccountCreated) require.NoError(t, err) require.Len(t, actual, 2) require.Contains(t, actual, evt1_1) require.Contains(t, actual, evt1_2) - actual, err = store.ByBlockIDEventType(blockID, flow.EventAccountUpdated) + actual, err = events.ByBlockIDEventType(blockID, flow.EventAccountUpdated) require.NoError(t, err) require.Len(t, actual, 1) require.Contains(t, actual, evt2_1) - events := systemcontracts.ServiceEventsForChain(flow.Emulator) + evts := systemcontracts.ServiceEventsForChain(flow.Emulator) - actual, err = store.ByBlockIDEventType(blockID, events.EpochSetup.EventType()) + actual, err = events.ByBlockIDEventType(blockID, evts.EpochSetup.EventType()) require.NoError(t, err) require.Len(t, actual, 0) // retrieve by blockID and transaction id - actual, err = store.ByBlockIDTransactionID(blockID, tx1ID) + actual, err = events.ByBlockIDTransactionID(blockID, tx1ID) require.NoError(t, err) require.Len(t, actual, 1) require.Contains(t, actual, evt1_1) // retrieve by blockID and transaction index - actual, err = store.ByBlockIDTransactionIndex(blockID, 1) + actual, err = events.ByBlockIDTransactionIndex(blockID, 1) require.NoError(t, err) require.Len(t, actual, 1) require.Contains(t, actual, evt1_2) // test loading from database - newStore := badgerstorage.NewEvents(metrics, db) + newStore := store.NewEvents(metrics, db) actual, err = newStore.ByBlockID(blockID) require.NoError(t, err) require.Len(t, actual, 3) @@ -91,33 +89,33 @@ func TestEventStoreRetrieve(t *testing.T) { } func TestEventRetrieveWithoutStore(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := badgerstorage.NewEvents(metrics, db) + events := store.NewEvents(metrics, db) blockID := unittest.IdentifierFixture() txID := unittest.IdentifierFixture() txIndex := rand.Uint32() // retrieve by blockID - events, err := store.ByBlockID(blockID) + evts, err := events.ByBlockID(blockID) require.NoError(t, err) - require.True(t, len(events) == 0) + require.True(t, len(evts) == 0) // retrieve by blockID and event type - events, err = store.ByBlockIDEventType(blockID, flow.EventAccountCreated) + evts, err = events.ByBlockIDEventType(blockID, flow.EventAccountCreated) require.NoError(t, err) - require.True(t, len(events) == 0) + require.True(t, len(evts) == 0) // retrieve by blockID and transaction id - events, err = store.ByBlockIDTransactionID(blockID, txID) + evts, err = events.ByBlockIDTransactionID(blockID, txID) require.NoError(t, err) - require.True(t, len(events) == 0) + require.True(t, len(evts) == 0) // retrieve by blockID and transaction id - events, err = store.ByBlockIDTransactionIndex(blockID, txIndex) + evts, err = events.ByBlockIDTransactionIndex(blockID, txIndex) require.NoError(t, err) - require.True(t, len(events) == 0) + require.True(t, len(evts) == 0) }) } diff --git a/storage/badger/light_transaction_results.go b/storage/store/light_transaction_results.go similarity index 52% rename from storage/badger/light_transaction_results.go rename to storage/store/light_transaction_results.go index 13e8863a276..44f3a52f973 100644 --- a/storage/badger/light_transaction_results.go +++ b/storage/store/light_transaction_results.go @@ -1,88 +1,78 @@ -package badger +package store 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" + "github.com/onflow/flow-go/storage/operation" ) var _ storage.LightTransactionResults = (*LightTransactionResults)(nil) type LightTransactionResults struct { - db *badger.DB + db storage.DB cache *Cache[string, flow.LightTransactionResult] indexCache *Cache[string, flow.LightTransactionResult] blockCache *Cache[string, []flow.LightTransactionResult] } -func NewLightTransactionResults(collector module.CacheMetrics, db *badger.DB, transactionResultsCacheSize uint) *LightTransactionResults { - retrieve := func(key string) func(tx *badger.Txn) (flow.LightTransactionResult, error) { +func NewLightTransactionResults(collector module.CacheMetrics, db storage.DB, transactionResultsCacheSize uint) *LightTransactionResults { + retrieve := func(r storage.Reader, key string) (flow.LightTransactionResult, error) { var txResult flow.LightTransactionResult - return func(tx *badger.Txn) (flow.LightTransactionResult, error) { - - blockID, txID, err := KeyToBlockIDTransactionID(key) - if err != nil { - return flow.LightTransactionResult{}, fmt.Errorf("could not convert key: %w", err) - } - - err = operation.RetrieveLightTransactionResult(blockID, txID, &txResult)(tx) - if err != nil { - return flow.LightTransactionResult{}, handleError(err, flow.LightTransactionResult{}) - } - return txResult, nil + blockID, txID, err := KeyToBlockIDTransactionID(key) + if err != nil { + return flow.LightTransactionResult{}, fmt.Errorf("could not convert key: %w", err) + } + + err = operation.RetrieveLightTransactionResult(r, blockID, txID, &txResult) + if err != nil { + return flow.LightTransactionResult{}, err } + return txResult, nil } - retrieveIndex := func(key string) func(tx *badger.Txn) (flow.LightTransactionResult, error) { + retrieveIndex := func(r storage.Reader, key string) (flow.LightTransactionResult, error) { var txResult flow.LightTransactionResult - return func(tx *badger.Txn) (flow.LightTransactionResult, error) { - - blockID, txIndex, err := KeyToBlockIDIndex(key) - if err != nil { - return flow.LightTransactionResult{}, fmt.Errorf("could not convert index key: %w", err) - } - - err = operation.RetrieveLightTransactionResultByIndex(blockID, txIndex, &txResult)(tx) - if err != nil { - return flow.LightTransactionResult{}, handleError(err, flow.LightTransactionResult{}) - } - return txResult, nil + blockID, txIndex, err := KeyToBlockIDIndex(key) + if err != nil { + return flow.LightTransactionResult{}, fmt.Errorf("could not convert index key: %w", err) + } + + err = operation.RetrieveLightTransactionResultByIndex(r, blockID, txIndex, &txResult) + if err != nil { + return flow.LightTransactionResult{}, err } + return txResult, nil } - retrieveForBlock := func(key string) func(tx *badger.Txn) ([]flow.LightTransactionResult, error) { + retrieveForBlock := func(r storage.Reader, key string) ([]flow.LightTransactionResult, error) { var txResults []flow.LightTransactionResult - return func(tx *badger.Txn) ([]flow.LightTransactionResult, error) { - - blockID, err := KeyToBlockID(key) - if err != nil { - return nil, fmt.Errorf("could not convert index key: %w", err) - } - - err = operation.LookupLightTransactionResultsByBlockIDUsingIndex(blockID, &txResults)(tx) - if err != nil { - return nil, handleError(err, flow.LightTransactionResult{}) - } - return txResults, nil + + blockID, err := KeyToBlockID(key) + if err != nil { + return nil, fmt.Errorf("could not convert index key: %w", err) + } + + err = operation.LookupLightTransactionResultsByBlockIDUsingIndex(r, blockID, &txResults) + if err != nil { + return nil, err } + return txResults, nil } return &LightTransactionResults{ db: db, - cache: newCache[string, flow.LightTransactionResult](collector, metrics.ResourceTransactionResults, + cache: newCache(collector, metrics.ResourceTransactionResults, withLimit[string, flow.LightTransactionResult](transactionResultsCacheSize), withStore(noopStore[string, flow.LightTransactionResult]), withRetrieve(retrieve), ), - indexCache: newCache[string, flow.LightTransactionResult](collector, metrics.ResourceTransactionResultIndices, + indexCache: newCache(collector, metrics.ResourceTransactionResultIndices, withLimit[string, flow.LightTransactionResult](transactionResultsCacheSize), withStore(noopStore[string, flow.LightTransactionResult]), withRetrieve(retrieveIndex), ), - blockCache: newCache[string, []flow.LightTransactionResult](collector, metrics.ResourceTransactionResultIndices, + blockCache: newCache(collector, metrics.ResourceTransactionResultIndices, withLimit[string, []flow.LightTransactionResult](transactionResultsCacheSize), withStore(noopStore[string, []flow.LightTransactionResult]), withRetrieve(retrieveForBlock), @@ -90,22 +80,22 @@ func NewLightTransactionResults(collector module.CacheMetrics, db *badger.DB, tr } } -func (tr *LightTransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() +func (tr *LightTransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, rw storage.ReaderBatchWriter) error { + w := rw.Writer() for i, result := range transactionResults { - err := operation.BatchInsertLightTransactionResult(blockID, &result)(writeBatch) + err := operation.BatchInsertLightTransactionResult(w, blockID, &result) if err != nil { return fmt.Errorf("cannot batch insert tx result: %w", err) } - err = operation.BatchIndexLightTransactionResult(blockID, uint32(i), &result)(writeBatch) + err = operation.BatchIndexLightTransactionResult(w, blockID, uint32(i), &result) if err != nil { return fmt.Errorf("cannot batch index tx result: %w", err) } } - batch.OnSucceed(func() { + storage.OnCommitSucceed(rw, func() { for i, result := range transactionResults { key := KeyFromBlockIDTransactionID(blockID, result.TransactionID) // cache for each transaction, so that it's faster to retrieve @@ -123,12 +113,14 @@ func (tr *LightTransactionResults) BatchStore(blockID flow.Identifier, transacti return nil } +func (tr *LightTransactionResults) BatchStoreBadger(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch storage.BatchStorage) error { + panic("LightTransactionResults BatchStoreBadger not implemented") +} + // ByBlockIDTransactionID returns the transaction result for the given block ID and transaction ID func (tr *LightTransactionResults) ByBlockIDTransactionID(blockID flow.Identifier, txID flow.Identifier) (*flow.LightTransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockIDTransactionID(blockID, txID) - transactionResult, err := tr.cache.Get(key)(tx) + transactionResult, err := tr.cache.Get(tr.db.Reader(), key) if err != nil { return nil, err } @@ -137,10 +129,8 @@ func (tr *LightTransactionResults) ByBlockIDTransactionID(blockID flow.Identifie // ByBlockIDTransactionIndex returns the transaction result for the given blockID and transaction index func (tr *LightTransactionResults) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) (*flow.LightTransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockIDIndex(blockID, txIndex) - transactionResult, err := tr.indexCache.Get(key)(tx) + transactionResult, err := tr.indexCache.Get(tr.db.Reader(), key) if err != nil { return nil, err } @@ -149,10 +139,8 @@ func (tr *LightTransactionResults) ByBlockIDTransactionIndex(blockID flow.Identi // ByBlockID gets all transaction results for a block, ordered by transaction index func (tr *LightTransactionResults) ByBlockID(blockID flow.Identifier) ([]flow.LightTransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockID(blockID) - transactionResults, err := tr.blockCache.Get(key)(tx) + transactionResults, err := tr.blockCache.Get(tr.db.Reader(), key) if err != nil { return nil, err } diff --git a/storage/badger/light_transaction_results_test.go b/storage/store/light_transaction_results_test.go similarity index 67% rename from storage/badger/light_transaction_results_test.go rename to storage/store/light_transaction_results_test.go index 61fc857e0bb..c3ea965ab72 100644 --- a/storage/badger/light_transaction_results_test.go +++ b/storage/store/light_transaction_results_test.go @@ -1,9 +1,8 @@ -package badger_test +package store_test import ( "testing" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" @@ -11,44 +10,40 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage" - bstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" "github.com/onflow/flow-go/utils/unittest" ) func TestBatchStoringLightTransactionResults(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewLightTransactionResults(metrics, db, 1000) + store1 := store.NewLightTransactionResults(metrics, db, 1000) blockID := unittest.IdentifierFixture() txResults := getLightTransactionResultsFixture(10) - t.Run("batch store results", func(t *testing.T) { - writeBatch := bstorage.NewBatch(db) - err := store.BatchStore(blockID, txResults, writeBatch) - require.NoError(t, err) - - err = writeBatch.Flush() - require.NoError(t, err) + t.Run("batch store1 results", func(t *testing.T) { + require.NoError(t, db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return store1.BatchStore(blockID, txResults, rw) + })) // add a results to a new block to validate they are not included in lookups - writeBatch = bstorage.NewBatch(db) - err = store.BatchStore(unittest.IdentifierFixture(), getLightTransactionResultsFixture(2), writeBatch) - require.NoError(t, err) + require.NoError(t, db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return store1.BatchStore(unittest.IdentifierFixture(), getLightTransactionResultsFixture(2), rw) + })) - err = writeBatch.Flush() - require.NoError(t, err) }) t.Run("read results with cache", func(t *testing.T) { for _, txResult := range txResults { - actual, err := store.ByBlockIDTransactionID(blockID, txResult.TransactionID) + actual, err := store1.ByBlockIDTransactionID(blockID, txResult.TransactionID) require.NoError(t, err) assert.Equal(t, txResult, *actual) } }) - newStore := bstorage.NewLightTransactionResults(metrics, db, 1000) + newStore := store.NewLightTransactionResults(metrics, db, 1000) t.Run("read results without cache", func(t *testing.T) { // test loading from database (without cache) // create a new instance using the same db so it has an empty cache @@ -62,7 +57,7 @@ func TestBatchStoringLightTransactionResults(t *testing.T) { t.Run("cached and non-cached results are equal", func(t *testing.T) { // check retrieving by index from both cache and db for i := len(txResults) - 1; i >= 0; i-- { - actual, err := store.ByBlockIDTransactionIndex(blockID, uint32(i)) + actual, err := store1.ByBlockIDTransactionIndex(blockID, uint32(i)) require.NoError(t, err) assert.Equal(t, txResults[i], *actual) @@ -73,7 +68,7 @@ func TestBatchStoringLightTransactionResults(t *testing.T) { }) t.Run("read all results for block", func(t *testing.T) { - actuals, err := store.ByBlockID(blockID) + actuals, err := store1.ByBlockID(blockID) require.NoError(t, err) assert.Equal(t, len(txResults), len(actuals)) @@ -85,18 +80,18 @@ func TestBatchStoringLightTransactionResults(t *testing.T) { } func TestReadingNotStoredLightTransactionResults(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewLightTransactionResults(metrics, db, 1000) + store1 := store.NewLightTransactionResults(metrics, db, 1000) blockID := unittest.IdentifierFixture() txID := unittest.IdentifierFixture() txIndex := rand.Uint32() - _, err := store.ByBlockIDTransactionID(blockID, txID) + _, err := store1.ByBlockIDTransactionID(blockID, txID) assert.ErrorIs(t, err, storage.ErrNotFound) - _, err = store.ByBlockIDTransactionIndex(blockID, txIndex) + _, err = store1.ByBlockIDTransactionIndex(blockID, txIndex) assert.ErrorIs(t, err, storage.ErrNotFound) }) } diff --git a/storage/store/my_receipts.go b/storage/store/my_receipts.go new file mode 100644 index 00000000000..9fbb8ef3011 --- /dev/null +++ b/storage/store/my_receipts.go @@ -0,0 +1,89 @@ +package store + +import ( + "fmt" + + "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/operation" +) + +// MyExecutionReceipts holds and indexes Execution Receipts. +// MyExecutionReceipts is implemented as a wrapper around badger.ExecutionReceipts +// The wrapper adds the ability to "MY execution receipt", from the viewpoint +// of an individual Execution Node. +type MyExecutionReceipts struct { + genericReceipts storage.ExecutionReceipts + db storage.DB + cache *Cache[flow.Identifier, *flow.ExecutionReceipt] +} + +// NewMyExecutionReceipts creates instance of MyExecutionReceipts which is a wrapper wrapper around badger.ExecutionReceipts +// It's useful for execution nodes to keep track of produced execution receipts. +func NewMyExecutionReceipts(collector module.CacheMetrics, db storage.DB, receipts storage.ExecutionReceipts) *MyExecutionReceipts { + retrieve := func(r storage.Reader, blockID flow.Identifier) (*flow.ExecutionReceipt, error) { + var receiptID flow.Identifier + err := operation.LookupOwnExecutionReceipt(r, blockID, &receiptID) + if err != nil { + return nil, fmt.Errorf("could not lookup receipt ID: %w", err) + } + receipt, err := receipts.ByID(receiptID) + if err != nil { + return nil, err + } + return receipt, nil + } + + return &MyExecutionReceipts{ + genericReceipts: receipts, + db: db, + cache: newCache(collector, metrics.ResourceMyReceipt, + withLimit[flow.Identifier, *flow.ExecutionReceipt](flow.DefaultTransactionExpiry+100), + withRetrieve(retrieve)), + } +} + +// storeMyReceipt assembles the operations to retrieve my receipt for the given block ID. +func (m *MyExecutionReceipts) myReceipt(blockID flow.Identifier) (*flow.ExecutionReceipt, error) { + return m.cache.Get(m.db.Reader(), blockID) // assemble DB operations to retrieve receipt (no execution) +} + +// BatchStoreMyReceipt stores blockID-to-my-receipt index entry keyed by blockID in a provided batch. +// No errors are expected during normal operation +// If entity fails marshalling, the error is wrapped in a generic error and returned. +// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. +func (m *MyExecutionReceipts) BatchStoreMyReceipt(receipt *flow.ExecutionReceipt, rw storage.ReaderBatchWriter) error { + + err := m.genericReceipts.BatchStore(receipt, rw) + if err != nil { + return fmt.Errorf("cannot batch store generic execution receipt inside my execution receipt batch store: %w", err) + } + + err = operation.IndexOwnExecutionReceipt(rw.Writer(), receipt.ExecutionResult.BlockID, receipt.ID()) + if err != nil { + return fmt.Errorf("cannot batch index own execution receipt inside my execution receipt batch store: %w", err) + } + + return nil +} + +// MyReceipt retrieves my receipt for the given block. +// Returns storage.ErrNotFound if no receipt was persisted for the block. +func (m *MyExecutionReceipts) MyReceipt(blockID flow.Identifier) (*flow.ExecutionReceipt, error) { + return m.myReceipt(blockID) +} + +func (m *MyExecutionReceipts) RemoveIndexByBlockID(blockID flow.Identifier) error { + return m.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveOwnExecutionReceipt(rw.Writer(), blockID) + }) +} + +// BatchRemoveIndexByBlockID removes blockID-to-my-execution-receipt index entry keyed by a blockID in a provided batch +// No errors are expected during normal operation, even if no entries are matched. +// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. +func (m *MyExecutionReceipts) BatchRemoveIndexByBlockID(blockID flow.Identifier, rw storage.ReaderBatchWriter) error { + return operation.RemoveOwnExecutionReceipt(rw.Writer(), blockID) +} diff --git a/storage/store/my_receipts_test.go b/storage/store/my_receipts_test.go new file mode 100644 index 00000000000..ce6e78bd702 --- /dev/null +++ b/storage/store/my_receipts_test.go @@ -0,0 +1,61 @@ +package store_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestMyExecutionReceiptsStorage(t *testing.T) { + withStore := func(t *testing.T, f func(store1 *store.MyExecutionReceipts, db storage.DB)) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, 100) + store1 := store.NewMyExecutionReceipts(metrics, db, receipts) + + f(store1, db) + }) + } + + t.Run("store1 one get one", func(t *testing.T) { + withStore(t, func(store1 *store.MyExecutionReceipts, db storage.DB) { + block := unittest.BlockFixture() + receipt1 := unittest.ReceiptForBlockFixture(&block) + + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return store1.BatchStoreMyReceipt(receipt1, rw) + }) + require.NoError(t, err) + + actual, err := store1.MyReceipt(block.ID()) + require.NoError(t, err) + + require.Equal(t, receipt1, actual) + }) + }) + + t.Run("store1 same for the same block", func(t *testing.T) { + withStore(t, func(store1 *store.MyExecutionReceipts, db storage.DB) { + block := unittest.BlockFixture() + + receipt1 := unittest.ReceiptForBlockFixture(&block) + + err := db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return store1.BatchStoreMyReceipt(receipt1, rw) + }) + require.NoError(t, err) + + err = db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return store1.BatchStoreMyReceipt(receipt1, rw) + }) + require.NoError(t, err) + }) + }) +} diff --git a/storage/store/receipts.go b/storage/store/receipts.go new file mode 100644 index 00000000000..679ab23a51b --- /dev/null +++ b/storage/store/receipts.go @@ -0,0 +1,112 @@ +package store + +import ( + "errors" + "fmt" + + "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/operation" +) + +// ExecutionReceipts implements storage for execution receipts. +type ExecutionReceipts struct { + db storage.DB + results storage.ExecutionResults + cache *Cache[flow.Identifier, *flow.ExecutionReceipt] +} + +// NewExecutionReceipts Creates ExecutionReceipts instance which is a database of receipts which +// supports storing and indexing receipts by receipt ID and block ID. +func NewExecutionReceipts(collector module.CacheMetrics, db storage.DB, results storage.ExecutionResults, cacheSize uint) *ExecutionReceipts { + store := func(rw storage.ReaderBatchWriter, receiptTD flow.Identifier, receipt *flow.ExecutionReceipt) error { + receiptID := receipt.ID() + + err := results.BatchStore(&receipt.ExecutionResult, rw) + if err != nil { + return fmt.Errorf("could not store result: %w", err) + } + err = operation.InsertExecutionReceiptMeta(rw.Writer(), receiptID, receipt.Meta()) + if err != nil { + return fmt.Errorf("could not store receipt metadata: %w", err) + } + err = operation.IndexExecutionReceipts(rw.Writer(), receipt.ExecutionResult.BlockID, receiptID) + if err != nil { + return fmt.Errorf("could not index receipt by the block it computes: %w", err) + } + return nil + } + + retrieve := func(r storage.Reader, receiptID flow.Identifier) (*flow.ExecutionReceipt, error) { + var meta flow.ExecutionReceiptMeta + err := operation.RetrieveExecutionReceiptMeta(r, receiptID, &meta) + if err != nil { + return nil, fmt.Errorf("could not retrieve receipt meta: %w", err) + } + result, err := results.ByID(meta.ResultID) + if err != nil { + return nil, fmt.Errorf("could not retrieve result: %w", err) + } + return flow.ExecutionReceiptFromMeta(meta, *result), nil + } + + return &ExecutionReceipts{ + db: db, + results: results, + cache: newCache(collector, metrics.ResourceReceipt, + withLimit[flow.Identifier, *flow.ExecutionReceipt](cacheSize), + withStore(store), + withRetrieve(retrieve)), + } +} + +// storeMyReceipt assembles the operations to store an arbitrary receipt. +func (r *ExecutionReceipts) storeTx(rw storage.ReaderBatchWriter, receipt *flow.ExecutionReceipt) error { + return r.cache.PutTx(rw, receipt.ID(), receipt) +} + +func (r *ExecutionReceipts) byID(receiptID flow.Identifier) (*flow.ExecutionReceipt, error) { + val, err := r.cache.Get(r.db.Reader(), receiptID) + if err != nil { + return nil, err + } + return val, nil +} + +func (r *ExecutionReceipts) byBlockID(blockID flow.Identifier) ([]*flow.ExecutionReceipt, error) { + var receiptIDs []flow.Identifier + err := operation.LookupExecutionReceipts(r.db.Reader(), blockID, &receiptIDs) + if err != nil && !errors.Is(err, storage.ErrNotFound) { + return nil, fmt.Errorf("could not find receipt index for block: %w", err) + } + + var receipts []*flow.ExecutionReceipt + for _, id := range receiptIDs { + receipt, err := r.byID(id) + if err != nil { + return nil, fmt.Errorf("could not find receipt with id %v: %w", id, err) + } + receipts = append(receipts, receipt) + } + return receipts, nil +} + +func (r *ExecutionReceipts) Store(receipt *flow.ExecutionReceipt) error { + return r.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return r.storeTx(rw, receipt) + }) +} + +func (r *ExecutionReceipts) BatchStore(receipt *flow.ExecutionReceipt, rw storage.ReaderBatchWriter) error { + return r.storeTx(rw, receipt) +} + +func (r *ExecutionReceipts) ByID(receiptID flow.Identifier) (*flow.ExecutionReceipt, error) { + return r.byID(receiptID) +} + +func (r *ExecutionReceipts) ByBlockID(blockID flow.Identifier) (flow.ExecutionReceiptList, error) { + return r.byBlockID(blockID) +} diff --git a/storage/store/receipts_test.go b/storage/store/receipts_test.go new file mode 100644 index 00000000000..75538dd84f4 --- /dev/null +++ b/storage/store/receipts_test.go @@ -0,0 +1,147 @@ +package store_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestExecutionReceiptsStorage(t *testing.T) { + withStore := func(t *testing.T, f func(store1 *store.ExecutionReceipts)) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + results := store.NewExecutionResults(metrics, db) + store1 := store.NewExecutionReceipts(metrics, db, results, 100) + f(store1) + }) + } + + t.Run("get empty", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block := unittest.BlockFixture() + receipts, err := store1.ByBlockID(block.ID()) + require.NoError(t, err) + require.Equal(t, 0, len(receipts)) + }) + }) + + t.Run("store1 one get one", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block := unittest.BlockFixture() + receipt1 := unittest.ReceiptForBlockFixture(&block) + + err := store1.Store(receipt1) + require.NoError(t, err) + + actual, err := store1.ByID(receipt1.ID()) + require.NoError(t, err) + + require.Equal(t, receipt1, actual) + + receipts, err := store1.ByBlockID(block.ID()) + require.NoError(t, err) + + require.Equal(t, flow.ExecutionReceiptList{receipt1}, receipts) + }) + }) + + t.Run("store1 two for the same block", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block := unittest.BlockFixture() + + executor1 := unittest.IdentifierFixture() + executor2 := unittest.IdentifierFixture() + + receipt1 := unittest.ReceiptForBlockExecutorFixture(&block, executor1) + receipt2 := unittest.ReceiptForBlockExecutorFixture(&block, executor2) + + err := store1.Store(receipt1) + require.NoError(t, err) + + err = store1.Store(receipt2) + require.NoError(t, err) + + receipts, err := store1.ByBlockID(block.ID()) + require.NoError(t, err) + + require.ElementsMatch(t, []*flow.ExecutionReceipt{receipt1, receipt2}, receipts) + }) + }) + + t.Run("store1 two for different blocks", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block1 := unittest.BlockFixture() + block2 := unittest.BlockFixture() + + executor1 := unittest.IdentifierFixture() + executor2 := unittest.IdentifierFixture() + + receipt1 := unittest.ReceiptForBlockExecutorFixture(&block1, executor1) + receipt2 := unittest.ReceiptForBlockExecutorFixture(&block2, executor2) + + err := store1.Store(receipt1) + require.NoError(t, err) + + err = store1.Store(receipt2) + require.NoError(t, err) + + receipts1, err := store1.ByBlockID(block1.ID()) + require.NoError(t, err) + + receipts2, err := store1.ByBlockID(block2.ID()) + require.NoError(t, err) + + require.ElementsMatch(t, []*flow.ExecutionReceipt{receipt1}, receipts1) + require.ElementsMatch(t, []*flow.ExecutionReceipt{receipt2}, receipts2) + }) + }) + + t.Run("indexing duplicated receipts should be ok", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block1 := unittest.BlockFixture() + + executor1 := unittest.IdentifierFixture() + receipt1 := unittest.ReceiptForBlockExecutorFixture(&block1, executor1) + + err := store1.Store(receipt1) + require.NoError(t, err) + + err = store1.Store(receipt1) + require.NoError(t, err) + + receipts, err := store1.ByBlockID(block1.ID()) + require.NoError(t, err) + + require.ElementsMatch(t, []*flow.ExecutionReceipt{receipt1}, receipts) + }) + }) + + t.Run("indexing receipt from the same executor for same block should succeed", func(t *testing.T) { + withStore(t, func(store1 *store.ExecutionReceipts) { + block1 := unittest.BlockFixture() + + executor1 := unittest.IdentifierFixture() + + receipt1 := unittest.ReceiptForBlockExecutorFixture(&block1, executor1) + receipt2 := unittest.ReceiptForBlockExecutorFixture(&block1, executor1) + + err := store1.Store(receipt1) + require.NoError(t, err) + + err = store1.Store(receipt2) + require.NoError(t, err) + + receipts, err := store1.ByBlockID(block1.ID()) + require.NoError(t, err) + + require.ElementsMatch(t, []*flow.ExecutionReceipt{receipt1, receipt2}, receipts) + }) + }) +} diff --git a/storage/store/results.go b/storage/store/results.go new file mode 100644 index 00000000000..696b6d114c8 --- /dev/null +++ b/storage/store/results.go @@ -0,0 +1,168 @@ +package store + +import ( + "fmt" + + "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/transaction" + "github.com/onflow/flow-go/storage/operation" +) + +// ExecutionResults implements persistent storage for execution results. +type ExecutionResults struct { + db storage.DB + cache *Cache[flow.Identifier, *flow.ExecutionResult] +} + +var _ storage.ExecutionResults = (*ExecutionResults)(nil) + +func NewExecutionResults(collector module.CacheMetrics, db storage.DB) *ExecutionResults { + + store := func(rw storage.ReaderBatchWriter, _ flow.Identifier, result *flow.ExecutionResult) error { + return operation.InsertExecutionResult(rw.Writer(), result) + } + + retrieve := func(r storage.Reader, resultID flow.Identifier) (*flow.ExecutionResult, error) { + var result flow.ExecutionResult + err := operation.RetrieveExecutionResult(r, resultID, &result) + return &result, err + } + + res := &ExecutionResults{ + db: db, + cache: newCache(collector, metrics.ResourceResult, + withLimit[flow.Identifier, *flow.ExecutionResult](flow.DefaultTransactionExpiry+100), + withStore(store), + withRetrieve(retrieve)), + } + + return res +} + +func (r *ExecutionResults) store(rw storage.ReaderBatchWriter, result *flow.ExecutionResult) error { + return r.cache.PutTx(rw, result.ID(), result) +} + +func (r *ExecutionResults) byID(resultID flow.Identifier) (*flow.ExecutionResult, error) { + val, err := r.cache.Get(r.db.Reader(), resultID) + if err != nil { + return nil, err + } + return val, nil +} + +func (r *ExecutionResults) byBlockID(blockID flow.Identifier) (*flow.ExecutionResult, error) { + var resultID flow.Identifier + err := operation.LookupExecutionResult(r.db.Reader(), blockID, &resultID) + if err != nil { + return nil, fmt.Errorf("could not lookup execution result ID: %w", err) + } + return r.byID(resultID) +} + +func (r *ExecutionResults) index(w storage.Writer, blockID, resultID flow.Identifier, force bool) error { + if !force { + // when not forcing the index, check if the result is already indexed + exist, err := operation.ExistExecutionResult(r.db.Reader(), blockID) + if err != nil { + return fmt.Errorf("could not check if execution result exists: %w", err) + } + + // if the result is already indexed, check if the stored result is the same + if exist { + var storedResultID flow.Identifier + err = operation.LookupExecutionResult(r.db.Reader(), blockID, &storedResultID) + if err != nil { + return fmt.Errorf("could not lookup execution result ID: %w", err) + } + + if storedResultID != resultID { + return fmt.Errorf("storing result that is different from the already stored one for block: %v, storing result: %v, stored result: %v. %w", + blockID, resultID, storedResultID, storage.ErrDataMismatch) + } + + // if the result is the same, we don't need to index it again + return nil + } + + // if the result is not indexed, we can index it + } + + err := operation.IndexExecutionResult(w, blockID, resultID) + if err == nil { + return nil + } + + return nil +} + +func (r *ExecutionResults) Store(result *flow.ExecutionResult) error { + return r.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return r.store(rw, result) + }) +} + +func (r *ExecutionResults) BatchStore(result *flow.ExecutionResult, batch storage.ReaderBatchWriter) error { + return r.store(batch, result) +} + +func (r *ExecutionResults) BatchIndex(blockID flow.Identifier, resultID flow.Identifier, batch storage.ReaderBatchWriter) error { + return operation.IndexExecutionResult(batch.Writer(), blockID, resultID) +} + +func (r *ExecutionResults) ByID(resultID flow.Identifier) (*flow.ExecutionResult, error) { + return r.byID(resultID) +} + +// TODO: deprecated, should be removed when protocol data is moved pebble +func (r *ExecutionResults) ByIDTx(resultID flow.Identifier) func(tx *transaction.Tx) (*flow.ExecutionResult, error) { + return func(tx *transaction.Tx) (*flow.ExecutionResult, error) { + return nil, fmt.Errorf("not implemented") + } +} + +// Index indexes an execution result by block ID. +// Note: this method call is not concurrent safe, because it checks if the different result is already indexed +// by the same blockID, and if it is, it returns an error. +// The caller needs to ensure that there is no concurrent call to this method with the same blockID. +func (r *ExecutionResults) Index(blockID flow.Identifier, resultID flow.Identifier) error { + err := r.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return r.index(rw.Writer(), blockID, resultID, false) + }) + + if err != nil { + return fmt.Errorf("could not index execution result: %w", err) + } + return nil +} + +func (r *ExecutionResults) ForceIndex(blockID flow.Identifier, resultID flow.Identifier) error { + err := r.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return r.index(rw.Writer(), blockID, resultID, true) + }) + + if err != nil { + return fmt.Errorf("could not index execution result: %w", err) + } + return nil +} + +func (r *ExecutionResults) ByBlockID(blockID flow.Identifier) (*flow.ExecutionResult, error) { + return r.byBlockID(blockID) +} + +func (r *ExecutionResults) RemoveIndexByBlockID(blockID flow.Identifier) error { + return r.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveExecutionResultIndex(rw.Writer(), blockID) + }) +} + +// BatchRemoveIndexByBlockID removes blockID-to-executionResultID index entries keyed by blockID in a provided batch. +// No errors are expected during normal operation, even if no entries are matched. +// If Badger unexpectedly fails to process the request, the error is wrapped in a generic error and returned. +func (r *ExecutionResults) BatchRemoveIndexByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { + return operation.RemoveExecutionResultIndex(batch.Writer(), blockID) +} diff --git a/storage/store/results_test.go b/storage/store/results_test.go new file mode 100644 index 00000000000..34f63e1f885 --- /dev/null +++ b/storage/store/results_test.go @@ -0,0 +1,138 @@ +package store_test + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestResultStoreAndRetrieve(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + store1 := store.NewExecutionResults(metrics, db) + + result := unittest.ExecutionResultFixture() + blockID := unittest.IdentifierFixture() + err := store1.Store(result) + require.NoError(t, err) + + err = store1.Index(blockID, result.ID()) + require.NoError(t, err) + + actual, err := store1.ByBlockID(blockID) + require.NoError(t, err) + + require.Equal(t, result, actual) + }) +} + +func TestResultStoreTwice(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + store1 := store.NewExecutionResults(metrics, db) + + result := unittest.ExecutionResultFixture() + blockID := unittest.IdentifierFixture() + err := store1.Store(result) + require.NoError(t, err) + + err = store1.Index(blockID, result.ID()) + require.NoError(t, err) + + err = store1.Store(result) + require.NoError(t, err) + + err = store1.Index(blockID, result.ID()) + require.NoError(t, err) + }) +} + +func TestResultBatchStoreTwice(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + store1 := store.NewExecutionResults(metrics, db) + + result := unittest.ExecutionResultFixture() + blockID := unittest.IdentifierFixture() + + require.NoError(t, db.WithReaderBatchWriter(func(batch storage.ReaderBatchWriter) error { + err := store1.BatchStore(result, batch) + require.NoError(t, err) + + err = store1.BatchIndex(blockID, result.ID(), batch) + require.NoError(t, err) + return nil + })) + + require.NoError(t, db.WithReaderBatchWriter(func(batch storage.ReaderBatchWriter) error { + err := store1.BatchStore(result, batch) + require.NoError(t, err) + + err = store1.BatchIndex(blockID, result.ID(), batch) + require.NoError(t, err) + + return nil + })) + }) +} + +func TestResultStoreTwoDifferentResultsShouldFail(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + store1 := store.NewExecutionResults(metrics, db) + + result1 := unittest.ExecutionResultFixture() + result2 := unittest.ExecutionResultFixture() + blockID := unittest.IdentifierFixture() + err := store1.Store(result1) + require.NoError(t, err) + + err = store1.Index(blockID, result1.ID()) + require.NoError(t, err) + + // we can store1 a different result, but we can't index + // a different result for that block, because it will mean + // one block has two different results. + err = store1.Store(result2) + require.NoError(t, err) + + err = store1.Index(blockID, result2.ID()) + require.Error(t, err) + require.True(t, errors.Is(err, storage.ErrDataMismatch)) + }) +} + +func TestResultStoreForceIndexOverridesMapping(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { + metrics := metrics.NewNoopCollector() + store1 := store.NewExecutionResults(metrics, db) + + result1 := unittest.ExecutionResultFixture() + result2 := unittest.ExecutionResultFixture() + blockID := unittest.IdentifierFixture() + err := store1.Store(result1) + require.NoError(t, err) + err = store1.Index(blockID, result1.ID()) + require.NoError(t, err) + + err = store1.Store(result2) + require.NoError(t, err) + + // force index + err = store1.ForceIndex(blockID, result2.ID()) + require.NoError(t, err) + + // retrieve index to make sure it points to second ER now + byBlockID, err := store1.ByBlockID(blockID) + + require.Equal(t, result2, byBlockID) + require.NoError(t, err) + }) +} diff --git a/storage/badger/transaction_result_error_messages.go b/storage/store/transaction_result_error_messages.go similarity index 64% rename from storage/badger/transaction_result_error_messages.go rename to storage/store/transaction_result_error_messages.go index e2abf659d5e..d9775643566 100644 --- a/storage/badger/transaction_result_error_messages.go +++ b/storage/store/transaction_result_error_messages.go @@ -1,74 +1,63 @@ -package badger +package store 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" + "github.com/onflow/flow-go/storage/operation" ) var _ storage.TransactionResultErrorMessages = (*TransactionResultErrorMessages)(nil) type TransactionResultErrorMessages struct { - db *badger.DB + db storage.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) { +func NewTransactionResultErrorMessages(collector module.CacheMetrics, db storage.DB, transactionResultsCacheSize uint) *TransactionResultErrorMessages { + retrieve := func(r storage.Reader, key string) (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 + blockID, txID, err := KeyToBlockIDTransactionID(key) + if err != nil { + return flow.TransactionResultErrorMessage{}, fmt.Errorf("could not convert key: %w", err) + } + + err = operation.RetrieveTransactionResultErrorMessage(r, blockID, txID, &txResultErrMsg) + if err != nil { + return flow.TransactionResultErrorMessage{}, err } + return txResultErrMsg, nil } - retrieveIndex := func(key string) func(tx *badger.Txn) (flow.TransactionResultErrorMessage, error) { + retrieveIndex := func(r storage.Reader, key string) (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 + blockID, txIndex, err := KeyToBlockIDIndex(key) + if err != nil { + return flow.TransactionResultErrorMessage{}, fmt.Errorf("could not convert index key: %w", err) } + + err = operation.RetrieveTransactionResultErrorMessageByIndex(r, blockID, txIndex, &txResultErrMsg) + if err != nil { + return flow.TransactionResultErrorMessage{}, err + } + return txResultErrMsg, nil } - retrieveForBlock := func(key string) func(tx *badger.Txn) ([]flow.TransactionResultErrorMessage, error) { + retrieveForBlock := func(r storage.Reader, key string) ([]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 + blockID, err := KeyToBlockID(key) + if err != nil { + return nil, fmt.Errorf("could not convert index key: %w", err) } + + err = operation.LookupTransactionResultErrorMessagesByBlockIDUsingIndex(r, blockID, &txResultErrMsg) + if err != nil { + return nil, err + } + return txResultErrMsg, nil } return &TransactionResultErrorMessages{ @@ -95,19 +84,9 @@ func NewTransactionResultErrorMessages(collector module.CacheMetrics, db *badger // // 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 + return t.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return t.batchStore(blockID, transactionResultErrorMessages, rw) + }) } // Exists returns true if transaction result error messages for the given ID have been stored. @@ -121,7 +100,7 @@ func (t *TransactionResultErrorMessages) Exists(blockID flow.Identifier) (bool, } // otherwise, check badger store var exists bool - err := t.db.View(operation.TransactionResultErrorMessagesExists(blockID, &exists)) + err := operation.TransactionResultErrorMessagesExists(t.db.Reader(), blockID, &exists) if err != nil { return false, fmt.Errorf("could not check existence: %w", err) } @@ -134,23 +113,22 @@ func (t *TransactionResultErrorMessages) Exists(blockID flow.Identifier) (bool, func (t *TransactionResultErrorMessages) batchStore( blockID flow.Identifier, transactionResultErrorMessages []flow.TransactionResultErrorMessage, - batch storage.BatchStorage, + batch storage.ReaderBatchWriter, ) error { - writeBatch := batch.GetWriter() - + writer := batch.Writer() for _, result := range transactionResultErrorMessages { - err := operation.BatchInsertTransactionResultErrorMessage(blockID, &result)(writeBatch) + err := operation.BatchInsertTransactionResultErrorMessage(writer, blockID, &result) if err != nil { return fmt.Errorf("cannot batch insert tx result error message: %w", err) } - err = operation.BatchIndexTransactionResultErrorMessage(blockID, &result)(writeBatch) + err = operation.BatchIndexTransactionResultErrorMessage(writer, blockID, &result) if err != nil { return fmt.Errorf("cannot batch index tx result error message: %w", err) } } - batch.OnSucceed(func() { + storage.OnCommitSucceed(batch, func() { for _, result := range transactionResultErrorMessages { key := KeyFromBlockIDTransactionID(blockID, result.TransactionID) // cache for each transaction, so that it's faster to retrieve @@ -171,10 +149,8 @@ func (t *TransactionResultErrorMessages) batchStore( // 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) + transactionResultErrorMessage, err := t.cache.Get(t.db.Reader(), key) if err != nil { return nil, err } @@ -186,10 +162,8 @@ func (t *TransactionResultErrorMessages) ByBlockIDTransactionID(blockID flow.Ide // 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) + transactionResultErrorMessage, err := t.indexCache.Get(t.db.Reader(), key) if err != nil { return nil, err } @@ -201,10 +175,8 @@ func (t *TransactionResultErrorMessages) ByBlockIDTransactionIndex(blockID flow. // // 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) + transactionResultErrorMessages, err := t.blockCache.Get(t.db.Reader(), key) if err != nil { return nil, err } diff --git a/storage/badger/transaction_result_error_messages_test.go b/storage/store/transaction_result_error_messages_test.go similarity index 71% rename from storage/badger/transaction_result_error_messages_test.go rename to storage/store/transaction_result_error_messages_test.go index e21e8aaf348..02238f0138c 100644 --- a/storage/badger/transaction_result_error_messages_test.go +++ b/storage/store/transaction_result_error_messages_test.go @@ -1,10 +1,9 @@ -package badger_test +package store_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" @@ -14,23 +13,24 @@ import ( "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/unittest" - bstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" ) func TestStoringTransactionResultErrorMessages(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + store1 := store.NewTransactionResultErrorMessages(metrics, db, 1000) blockID := unittest.IdentifierFixture() // test db Exists by block id - exists, err := store.Exists(blockID) + exists, err := store1.Exists(blockID) require.NoError(t, err) require.False(t, exists) // check retrieving by ByBlockID - messages, err := store.ByBlockID(blockID) + messages, err := store1.ByBlockID(blockID) require.NoError(t, err) require.Nil(t, messages) @@ -44,35 +44,35 @@ func TestStoringTransactionResultErrorMessages(t *testing.T) { } txErrorMessages = append(txErrorMessages, expected) } - err = store.Store(blockID, txErrorMessages) + err = store1.Store(blockID, txErrorMessages) require.NoError(t, err) // test db Exists by block id - exists, err = store.Exists(blockID) + exists, err = store1.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) + actual, err := store1.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) + actual, err := store1.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) require.NoError(t, err) assert.Equal(t, txErrorMessage, *actual) } // check retrieving by ByBlockID - actual, err := store.ByBlockID(blockID) + actual, err := store1.ByBlockID(blockID) require.NoError(t, err) assert.Equal(t, txErrorMessages, actual) // test loading from database - newStore := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + newStore := store.NewTransactionResultErrorMessages(metrics, db, 1000) for _, txErrorMessage := range txErrorMessages { actual, err := newStore.ByBlockIDTransactionID(blockID, txErrorMessage.TransactionID) require.NoError(t, err) @@ -81,7 +81,7 @@ func TestStoringTransactionResultErrorMessages(t *testing.T) { // check retrieving by index from both cache and db for i, txErrorMessage := range txErrorMessages { - actual, err := store.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) + actual, err := store1.ByBlockIDTransactionIndex(blockID, txErrorMessage.Index) require.NoError(t, err) assert.Equal(t, txErrorMessages[i], *actual) @@ -93,18 +93,18 @@ func TestStoringTransactionResultErrorMessages(t *testing.T) { } func TestReadingNotStoreTransactionResultErrorMessage(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewTransactionResultErrorMessages(metrics, db, 1000) + store1 := store.NewTransactionResultErrorMessages(metrics, db, 1000) blockID := unittest.IdentifierFixture() txID := unittest.IdentifierFixture() txIndex := rand.Uint32() - _, err := store.ByBlockIDTransactionID(blockID, txID) + _, err := store1.ByBlockIDTransactionID(blockID, txID) assert.ErrorIs(t, err, storage.ErrNotFound) - _, err = store.ByBlockIDTransactionIndex(blockID, txIndex) + _, err = store1.ByBlockIDTransactionIndex(blockID, txIndex) assert.ErrorIs(t, err, storage.ErrNotFound) }) } diff --git a/storage/badger/transaction_results.go b/storage/store/transaction_results.go similarity index 62% rename from storage/badger/transaction_results.go rename to storage/store/transaction_results.go index 1aca9e63b11..62e595579a3 100644 --- a/storage/badger/transaction_results.go +++ b/storage/store/transaction_results.go @@ -1,23 +1,21 @@ -package badger +package store import ( "encoding/binary" "encoding/hex" "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" + "github.com/onflow/flow-go/storage/operation" ) var _ storage.TransactionResults = (*TransactionResults)(nil) type TransactionResults struct { - db *badger.DB + db storage.DB cache *Cache[string, flow.TransactionResult] indexCache *Cache[string, flow.TransactionResult] blockCache *Cache[string, []flow.TransactionResult] @@ -84,68 +82,59 @@ func KeyToBlockID(key string) (flow.Identifier, error) { return blockID, err } -func NewTransactionResults(collector module.CacheMetrics, db *badger.DB, transactionResultsCacheSize uint) *TransactionResults { - retrieve := func(key string) func(tx *badger.Txn) (flow.TransactionResult, error) { +func NewTransactionResults(collector module.CacheMetrics, db storage.DB, transactionResultsCacheSize uint) *TransactionResults { + retrieve := func(r storage.Reader, key string) (flow.TransactionResult, error) { var txResult flow.TransactionResult - return func(tx *badger.Txn) (flow.TransactionResult, error) { - - blockID, txID, err := KeyToBlockIDTransactionID(key) - if err != nil { - return flow.TransactionResult{}, fmt.Errorf("could not convert key: %w", err) - } - - err = operation.RetrieveTransactionResult(blockID, txID, &txResult)(tx) - if err != nil { - return flow.TransactionResult{}, handleError(err, flow.TransactionResult{}) - } - return txResult, nil + blockID, txID, err := KeyToBlockIDTransactionID(key) + if err != nil { + return flow.TransactionResult{}, fmt.Errorf("could not convert key: %w", err) } + + err = operation.RetrieveTransactionResult(r, blockID, txID, &txResult) + if err != nil { + return flow.TransactionResult{}, err + } + return txResult, nil } - retrieveIndex := func(key string) func(tx *badger.Txn) (flow.TransactionResult, error) { + retrieveIndex := func(r storage.Reader, key string) (flow.TransactionResult, error) { var txResult flow.TransactionResult - return func(tx *badger.Txn) (flow.TransactionResult, error) { - - blockID, txIndex, err := KeyToBlockIDIndex(key) - if err != nil { - return flow.TransactionResult{}, fmt.Errorf("could not convert index key: %w", err) - } - - err = operation.RetrieveTransactionResultByIndex(blockID, txIndex, &txResult)(tx) - if err != nil { - return flow.TransactionResult{}, handleError(err, flow.TransactionResult{}) - } - return txResult, nil + blockID, txIndex, err := KeyToBlockIDIndex(key) + if err != nil { + return flow.TransactionResult{}, fmt.Errorf("could not convert index key: %w", err) + } + + err = operation.RetrieveTransactionResultByIndex(r, blockID, txIndex, &txResult) + if err != nil { + return flow.TransactionResult{}, err } + return txResult, nil } - retrieveForBlock := func(key string) func(tx *badger.Txn) ([]flow.TransactionResult, error) { + retrieveForBlock := func(r storage.Reader, key string) ([]flow.TransactionResult, error) { var txResults []flow.TransactionResult - return func(tx *badger.Txn) ([]flow.TransactionResult, error) { - - blockID, err := KeyToBlockID(key) - if err != nil { - return nil, fmt.Errorf("could not convert index key: %w", err) - } - - err = operation.LookupTransactionResultsByBlockIDUsingIndex(blockID, &txResults)(tx) - if err != nil { - return nil, handleError(err, flow.TransactionResult{}) - } - return txResults, nil + blockID, err := KeyToBlockID(key) + if err != nil { + return nil, fmt.Errorf("could not convert index key: %w", err) } + + err = operation.LookupTransactionResultsByBlockIDUsingIndex(r, blockID, &txResults) + if err != nil { + return nil, err + } + return txResults, nil } return &TransactionResults{ db: db, - cache: newCache[string, flow.TransactionResult](collector, metrics.ResourceTransactionResults, + cache: newCache(collector, metrics.ResourceTransactionResults, withLimit[string, flow.TransactionResult](transactionResultsCacheSize), withStore(noopStore[string, flow.TransactionResult]), withRetrieve(retrieve), ), - indexCache: newCache[string, flow.TransactionResult](collector, metrics.ResourceTransactionResultIndices, + indexCache: newCache(collector, metrics.ResourceTransactionResultIndices, withLimit[string, flow.TransactionResult](transactionResultsCacheSize), withStore(noopStore[string, flow.TransactionResult]), withRetrieve(retrieveIndex), ), - blockCache: newCache[string, []flow.TransactionResult](collector, metrics.ResourceTransactionResultIndices, + blockCache: newCache(collector, metrics.ResourceTransactionResultIndices, withLimit[string, []flow.TransactionResult](transactionResultsCacheSize), withStore(noopStore[string, []flow.TransactionResult]), withRetrieve(retrieveForBlock), @@ -154,22 +143,22 @@ func NewTransactionResults(collector module.CacheMetrics, db *badger.DB, transac } // BatchStore will store the transaction results for the given block ID in a batch -func (tr *TransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() +func (tr *TransactionResults) BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch storage.ReaderBatchWriter) error { + w := batch.Writer() for i, result := range transactionResults { - err := operation.BatchInsertTransactionResult(blockID, &result)(writeBatch) + err := operation.InsertTransactionResult(w, blockID, &result) if err != nil { return fmt.Errorf("cannot batch insert tx result: %w", err) } - err = operation.BatchIndexTransactionResult(blockID, uint32(i), &result)(writeBatch) + err = operation.IndexTransactionResult(w, blockID, uint32(i), &result) if err != nil { return fmt.Errorf("cannot batch index tx result: %w", err) } } - batch.OnSucceed(func() { + storage.OnCommitSucceed(batch, func() { for i, result := range transactionResults { key := KeyFromBlockIDTransactionID(blockID, result.TransactionID) // cache for each transaction, so that it's faster to retrieve @@ -189,10 +178,8 @@ func (tr *TransactionResults) BatchStore(blockID flow.Identifier, transactionRes // ByBlockIDTransactionID returns the runtime transaction result for the given block ID and transaction ID func (tr *TransactionResults) ByBlockIDTransactionID(blockID flow.Identifier, txID flow.Identifier) (*flow.TransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockIDTransactionID(blockID, txID) - transactionResult, err := tr.cache.Get(key)(tx) + transactionResult, err := tr.cache.Get(tr.db.Reader(), key) if err != nil { return nil, err } @@ -201,10 +188,8 @@ func (tr *TransactionResults) ByBlockIDTransactionID(blockID flow.Identifier, tx // ByBlockIDTransactionIndex returns the runtime transaction result for the given block ID and transaction index func (tr *TransactionResults) ByBlockIDTransactionIndex(blockID flow.Identifier, txIndex uint32) (*flow.TransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockIDIndex(blockID, txIndex) - transactionResult, err := tr.indexCache.Get(key)(tx) + transactionResult, err := tr.indexCache.Get(tr.db.Reader(), key) if err != nil { return nil, err } @@ -213,10 +198,8 @@ func (tr *TransactionResults) ByBlockIDTransactionIndex(blockID flow.Identifier, // ByBlockID gets all transaction results for a block, ordered by transaction index func (tr *TransactionResults) ByBlockID(blockID flow.Identifier) ([]flow.TransactionResult, error) { - tx := tr.db.NewTransaction(false) - defer tx.Discard() key := KeyFromBlockID(blockID) - transactionResults, err := tr.blockCache.Get(key)(tx) + transactionResults, err := tr.blockCache.Get(tr.db.Reader(), key) if err != nil { return nil, err } @@ -225,11 +208,12 @@ func (tr *TransactionResults) ByBlockID(blockID flow.Identifier) ([]flow.Transac // RemoveByBlockID removes transaction results by block ID func (tr *TransactionResults) RemoveByBlockID(blockID flow.Identifier) error { - return tr.db.Update(operation.RemoveTransactionResultsByBlockID(blockID)) + return tr.db.WithReaderBatchWriter(func(rw storage.ReaderBatchWriter) error { + return operation.RemoveTransactionResultsByBlockID(rw.GlobalReader(), rw.Writer(), blockID) + }) } // BatchRemoveByBlockID batch removes transaction results by block ID -func (tr *TransactionResults) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.BatchStorage) error { - writeBatch := batch.GetWriter() - return tr.db.View(operation.BatchRemoveTransactionResultsByBlockID(blockID, writeBatch)) +func (tr *TransactionResults) BatchRemoveByBlockID(blockID flow.Identifier, batch storage.ReaderBatchWriter) error { + return operation.BatchRemoveTransactionResultsByBlockID(blockID, batch) } diff --git a/storage/badger/transaction_results_test.go b/storage/store/transaction_results_test.go similarity index 62% rename from storage/badger/transaction_results_test.go rename to storage/store/transaction_results_test.go index 5ba30d74414..733280e8f05 100644 --- a/storage/badger/transaction_results_test.go +++ b/storage/store/transaction_results_test.go @@ -1,11 +1,10 @@ -package badger_test +package store_test import ( "fmt" mathRand "math/rand" "testing" - "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" @@ -15,13 +14,14 @@ import ( "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/unittest" - bstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/storage/store" ) func TestBatchStoringTransactionResults(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewTransactionResults(metrics, db, 1000) + st := store.NewTransactionResults(metrics, db, 1000) blockID := unittest.IdentifierFixture() txResults := make([]flow.TransactionResult, 0) @@ -33,53 +33,53 @@ func TestBatchStoringTransactionResults(t *testing.T) { } txResults = append(txResults, expected) } - writeBatch := bstorage.NewBatch(db) - err := store.BatchStore(blockID, txResults, writeBatch) + writeBatch := db.NewBatch() + err := st.BatchStore(blockID, txResults, writeBatch) require.NoError(t, err) - err = writeBatch.Flush() + err = writeBatch.Commit() require.NoError(t, err) for _, txResult := range txResults { - actual, err := store.ByBlockIDTransactionID(blockID, txResult.TransactionID) + actual, err := st.ByBlockIDTransactionID(blockID, txResult.TransactionID) require.NoError(t, err) assert.Equal(t, txResult, *actual) } // test loading from database - newStore := bstorage.NewTransactionResults(metrics, db, 1000) + newst := store.NewTransactionResults(metrics, db, 1000) for _, txResult := range txResults { - actual, err := newStore.ByBlockIDTransactionID(blockID, txResult.TransactionID) + actual, err := newst.ByBlockIDTransactionID(blockID, txResult.TransactionID) require.NoError(t, err) assert.Equal(t, txResult, *actual) } // check retrieving by index from both cache and db for i := len(txResults) - 1; i >= 0; i-- { - actual, err := store.ByBlockIDTransactionIndex(blockID, uint32(i)) + actual, err := st.ByBlockIDTransactionIndex(blockID, uint32(i)) require.NoError(t, err) assert.Equal(t, txResults[i], *actual) - actual, err = newStore.ByBlockIDTransactionIndex(blockID, uint32(i)) + actual, err = newst.ByBlockIDTransactionIndex(blockID, uint32(i)) require.NoError(t, err) assert.Equal(t, txResults[i], *actual) } }) } -func TestReadingNotStoreTransaction(t *testing.T) { - unittest.RunWithBadgerDB(t, func(db *badger.DB) { +func TestReadingNotstTransaction(t *testing.T) { + dbtest.RunWithDB(t, func(t *testing.T, db storage.DB) { metrics := metrics.NewNoopCollector() - store := bstorage.NewTransactionResults(metrics, db, 1000) + st := store.NewTransactionResults(metrics, db, 1000) blockID := unittest.IdentifierFixture() txID := unittest.IdentifierFixture() txIndex := rand.Uint32() - _, err := store.ByBlockIDTransactionID(blockID, txID) + _, err := st.ByBlockIDTransactionID(blockID, txID) assert.ErrorIs(t, err, storage.ErrNotFound) - _, err = store.ByBlockIDTransactionIndex(blockID, txIndex) + _, err = st.ByBlockIDTransactionIndex(blockID, txIndex) assert.ErrorIs(t, err, storage.ErrNotFound) }) } @@ -87,8 +87,8 @@ func TestReadingNotStoreTransaction(t *testing.T) { func TestKeyConversion(t *testing.T) { blockID := unittest.IdentifierFixture() txID := unittest.IdentifierFixture() - key := bstorage.KeyFromBlockIDTransactionID(blockID, txID) - bID, tID, err := bstorage.KeyToBlockIDTransactionID(key) + key := store.KeyFromBlockIDTransactionID(blockID, txID) + bID, tID, err := store.KeyToBlockIDTransactionID(key) require.NoError(t, err) require.Equal(t, blockID, bID) require.Equal(t, txID, tID) @@ -97,8 +97,8 @@ func TestKeyConversion(t *testing.T) { func TestIndexKeyConversion(t *testing.T) { blockID := unittest.IdentifierFixture() txIndex := mathRand.Uint32() - key := bstorage.KeyFromBlockIDIndex(blockID, txIndex) - bID, tID, err := bstorage.KeyToBlockIDIndex(key) + key := store.KeyFromBlockIDIndex(blockID, txIndex) + bID, tID, err := store.KeyToBlockIDIndex(key) require.NoError(t, err) require.Equal(t, blockID, bID) require.Equal(t, txIndex, tID) diff --git a/storage/transaction_results.go b/storage/transaction_results.go index 6aaacc3a880..affd4147b30 100644 --- a/storage/transaction_results.go +++ b/storage/transaction_results.go @@ -6,7 +6,7 @@ import "github.com/onflow/flow-go/model/flow" type TransactionResults interface { // BatchStore inserts a batch of transaction result into a batch - BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch BatchStorage) error + BatchStore(blockID flow.Identifier, transactionResults []flow.TransactionResult, batch ReaderBatchWriter) error // ByBlockIDTransactionID returns the transaction result for the given block ID and transaction ID ByBlockIDTransactionID(blockID flow.Identifier, transactionID flow.Identifier) (*flow.TransactionResult, error) @@ -16,13 +16,19 @@ type TransactionResults interface { // ByBlockID gets all transaction results for a block, ordered by transaction index ByBlockID(id flow.Identifier) ([]flow.TransactionResult, error) + + // RemoveByBlockID removes all transaction results for a block + BatchRemoveByBlockID(id flow.Identifier, batch ReaderBatchWriter) error } // LightTransactionResults represents persistent storage for light transaction result type LightTransactionResults interface { // BatchStore inserts a batch of transaction result into a batch - BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch BatchStorage) error + BatchStore(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, rw ReaderBatchWriter) error + + // deprecated + BatchStoreBadger(blockID flow.Identifier, transactionResults []flow.LightTransactionResult, batch BatchStorage) error // ByBlockIDTransactionID returns the transaction result for the given block ID and transaction ID ByBlockIDTransactionID(blockID flow.Identifier, transactionID flow.Identifier) (*flow.LightTransactionResult, error) diff --git a/storage/util/testing.go b/storage/util/testing.go index 89e7e523364..59e3c443090 100644 --- a/storage/util/testing.go +++ b/storage/util/testing.go @@ -11,6 +11,8 @@ import ( "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage" bstorage "github.com/onflow/flow-go/storage/badger" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/store" ) func StorageLayer(_ testing.TB, db *badger.DB) *storage.All { @@ -19,6 +21,25 @@ func StorageLayer(_ testing.TB, db *badger.DB) *storage.All { return all } +func ExecutionStorageLayer(_ testing.TB, bdb *badger.DB) *storage.Execution { + metrics := metrics.NewNoopCollector() + + db := badgerimpl.ToDB(bdb) + + results := store.NewExecutionResults(metrics, db) + receipts := store.NewExecutionReceipts(metrics, db, results, bstorage.DefaultCacheSize) + commits := store.NewCommits(metrics, db) + transactionResults := store.NewTransactionResults(metrics, db, bstorage.DefaultCacheSize) + events := store.NewEvents(metrics, db) + return &storage.Execution{ + Results: results, + Receipts: receipts, + Commits: commits, + TransactionResults: transactionResults, + Events: events, + } +} + func CreateFiles(t *testing.T, dir string, names ...string) { for _, name := range names { file, err := os.Create(filepath.Join(dir, name))