From 0b6499ad2ad1bcf216ec16483c3ecf09d82cef6b Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 25 Jul 2024 16:01:39 +0300 Subject: [PATCH 01/33] Refactored structure of execution data tracker --- .../node_builder/access_node_builder.go | 4 +- cmd/execution_builder.go | 4 +- cmd/observer/node_builder/observer_builder.go | 4 +- module/executiondatasync/tracker/storage.go | 463 +----------------- storage/badger/execution_data_tracker.go | 432 ++++++++++++++++ .../badger/execution_data_tracker_test.go | 51 +- storage/pebble/execution_data_tracker.go | 1 + 7 files changed, 486 insertions(+), 473 deletions(-) create mode 100644 storage/badger/execution_data_tracker.go rename module/executiondatasync/tracker/storage_test.go => storage/badger/execution_data_tracker_test.go (73%) create mode 100644 storage/pebble/execution_data_tracker.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 143110256be..edda383b210 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -666,11 +666,11 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = tracker.OpenStorage( + builder.ExecutionDataTracker, err = bstorage.NewStorageTracker( trackerDir, sealed.Height, node.Logger, - tracker.WithPruneCallback(func(c cid.Cid) error { + bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) }), diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index 7617f44b1af..28e4306407c 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -946,11 +946,11 @@ func (exeNode *ExecutionNode) LoadExecutionDataPruner( } trackerDir := filepath.Join(exeNode.exeConf.executionDataDir, "tracker") - exeNode.executionDataTracker, err = tracker.OpenStorage( + exeNode.executionDataTracker, err = storage.NewStorageTracker( trackerDir, sealed.Height, node.Logger, - tracker.WithPruneCallback(func(c cid.Cid) error { + storage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return exeNode.executionDataBlobstore.DeleteBlob(context.TODO(), c) }), diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 38538d1129c..d63a6770f83 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1231,11 +1231,11 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = tracker.OpenStorage( + builder.ExecutionDataTracker, err = bstorage.NewStorageTracker( trackerDir, sealed.Height, node.Logger, - tracker.WithPruneCallback(func(c cid.Cid) error { + bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) }), diff --git a/module/executiondatasync/tracker/storage.go b/module/executiondatasync/tracker/storage.go index c7677f79ca7..cf3a9441a81 100644 --- a/module/executiondatasync/tracker/storage.go +++ b/module/executiondatasync/tracker/storage.go @@ -2,106 +2,67 @@ package tracker import ( "encoding/binary" - "errors" - "fmt" - "sync" - "github.com/dgraph-io/badger/v2" - "github.com/hashicorp/go-multierror" "github.com/ipfs/go-cid" - "github.com/rs/zerolog" "github.com/onflow/flow-go/module/blobs" ) // badger key prefixes const ( - prefixGlobalState byte = iota + 1 // global state variables - prefixLatestHeight // tracks, for each blob, the latest height at which there exists a block whose execution data contains the blob - prefixBlobRecord // tracks the set of blobs at each height + PrefixGlobalState byte = iota + 1 // global state variables + PrefixLatestHeight // tracks, for each blob, the latest height at which there exists a block whose execution data contains the blob + PrefixBlobRecord // tracks the set of blobs at each height ) const ( - globalStateFulfilledHeight byte = iota + 1 // latest fulfilled block height - globalStatePrunedHeight // latest pruned block height + GlobalStateFulfilledHeight byte = iota + 1 // latest fulfilled block height + GlobalStatePrunedHeight // latest pruned block height ) -const cidsPerBatch = 16 // number of cids to track per batch +const CidsPerBatch = 16 // number of cids to track per batch -func retryOnConflict(db *badger.DB, fn func(txn *badger.Txn) error) error { - for { - err := db.Update(fn) - if errors.Is(err, badger.ErrConflict) { - continue - } - return err - } -} - -const globalStateKeyLength = 2 +const GlobalStateKeyLength = 2 -func makeGlobalStateKey(state byte) []byte { - globalStateKey := make([]byte, globalStateKeyLength) - globalStateKey[0] = prefixGlobalState +func MakeGlobalStateKey(state byte) []byte { + globalStateKey := make([]byte, GlobalStateKeyLength) + globalStateKey[0] = PrefixGlobalState globalStateKey[1] = state return globalStateKey } -const blobRecordKeyLength = 1 + 8 + blobs.CidLength +const BlobRecordKeyLength = 1 + 8 + blobs.CidLength -func makeBlobRecordKey(blockHeight uint64, c cid.Cid) []byte { - blobRecordKey := make([]byte, blobRecordKeyLength) - blobRecordKey[0] = prefixBlobRecord +func MakeBlobRecordKey(blockHeight uint64, c cid.Cid) []byte { + blobRecordKey := make([]byte, BlobRecordKeyLength) + blobRecordKey[0] = PrefixBlobRecord binary.BigEndian.PutUint64(blobRecordKey[1:], blockHeight) copy(blobRecordKey[1+8:], c.Bytes()) return blobRecordKey } -func parseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { +func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { blockHeight := binary.BigEndian.Uint64(key[1:]) c, err := cid.Cast(key[1+8:]) return blockHeight, c, err } -const latestHeightKeyLength = 1 + blobs.CidLength +const LatestHeightKeyLength = 1 + blobs.CidLength -func makeLatestHeightKey(c cid.Cid) []byte { - latestHeightKey := make([]byte, latestHeightKeyLength) - latestHeightKey[0] = prefixLatestHeight +func MakeLatestHeightKey(c cid.Cid) []byte { + latestHeightKey := make([]byte, LatestHeightKeyLength) + latestHeightKey[0] = PrefixLatestHeight copy(latestHeightKey[1:], c.Bytes()) return latestHeightKey } -func makeUint64Value(v uint64) []byte { +func MakeUint64Value(v uint64) []byte { value := make([]byte, 8) binary.BigEndian.PutUint64(value, v) return value } -func getUint64Value(item *badger.Item) (uint64, error) { - value, err := item.ValueCopy(nil) - if err != nil { - return 0, err - } - - return binary.BigEndian.Uint64(value), nil -} - -// getBatchItemCountLimit returns the maximum number of items that can be included in a single batch -// transaction based on the number / total size of updates per item. -func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePerItem int64) int { - totalSizePerItem := 2*writeCountPerItem + writeSizePerItem // 2 bytes per entry for user and internal meta - maxItemCountByWriteCount := db.MaxBatchCount() / writeCountPerItem - maxItemCountByWriteSize := db.MaxBatchSize() / totalSizePerItem - - if maxItemCountByWriteCount < maxItemCountByWriteSize { - return int(maxItemCountByWriteCount) - } else { - return int(maxItemCountByWriteSize) - } -} - -// TrackBlobsFun is passed to the UpdateFn provided to Storage.Update, +// TrackBlobsFn is passed to the UpdateFn provided to Storage.Update, // and can be called to track a list of cids at a given block height. // It returns an error if the update failed. type TrackBlobsFn func(blockHeight uint64, cids ...cid.Cid) error @@ -157,385 +118,3 @@ type Storage interface { // called with a value higher than the fulfilled height. PruneUpToHeight(height uint64) error } - -// The storage component tracks the following information: -// - the latest pruned height -// - the latest fulfilled height -// - the set of CIDs of the execution data blobs we know about at each height, so that -// once we prune a fulfilled height we can remove the blob data from local storage -// - for each CID, the most recent height that it was observed at, so that when pruning -// a fulfilled height we don't remove any blob data that is still needed at higher heights -// -// The storage component calls the given prune callback for a CID when the last height -// at which that CID appears is pruned. The prune callback can be used to delete the -// corresponding blob data from the blob store. -type storage struct { - // ensures that pruning operations are not run concurrently with any other db writes - // we acquire the read lock when we want to perform a non-prune WRITE - // we acquire the write lock when we want to perform a prune WRITE - mu sync.RWMutex - - db *badger.DB - pruneCallback PruneCallback - logger zerolog.Logger -} - -type StorageOption func(*storage) - -func WithPruneCallback(callback PruneCallback) StorageOption { - return func(s *storage) { - s.pruneCallback = callback - } -} - -func OpenStorage(dbPath string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*storage, error) { - lg := logger.With().Str("module", "tracker_storage").Logger() - db, err := badger.Open(badger.LSMOnlyOptions(dbPath)) - if err != nil { - return nil, fmt.Errorf("could not open tracker db: %w", err) - } - - storage := &storage{ - db: db, - pruneCallback: func(c cid.Cid) error { return nil }, - logger: lg, - } - - for _, opt := range opts { - opt(storage) - } - - lg.Info().Msgf("initialize storage with start height: %d", startHeight) - - if err := storage.init(startHeight); err != nil { - return nil, fmt.Errorf("failed to initialize storage: %w", err) - } - - lg.Info().Msgf("storage initialized") - - return storage, nil -} - -func (s *storage) init(startHeight uint64) error { - fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() - prunedHeight, prunedHeightErr := s.GetPrunedHeight() - - if fulfilledHeightErr == nil && prunedHeightErr == nil { - if prunedHeight > fulfilledHeight { - return fmt.Errorf( - "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", - prunedHeight, - fulfilledHeight, - ) - } - - s.logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) - // replay pruning in case it was interrupted during previous shutdown - if err := s.PruneUpToHeight(prunedHeight); err != nil { - return fmt.Errorf("failed to replay pruning: %w", err) - } - s.logger.Info().Msgf("finished pruning") - } else if errors.Is(fulfilledHeightErr, badger.ErrKeyNotFound) && errors.Is(prunedHeightErr, badger.ErrKeyNotFound) { - // db is empty, we need to bootstrap it - if err := s.bootstrap(startHeight); err != nil { - return fmt.Errorf("failed to bootstrap storage: %w", err) - } - } else { - return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() - } - - return nil -} - -func (s *storage) bootstrap(startHeight uint64) error { - fulfilledHeightKey := makeGlobalStateKey(globalStateFulfilledHeight) - fulfilledHeightValue := makeUint64Value(startHeight) - - prunedHeightKey := makeGlobalStateKey(globalStatePrunedHeight) - prunedHeightValue := makeUint64Value(startHeight) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } - - if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil - }) -} - -func (s *storage) Update(f UpdateFn) error { - s.mu.RLock() - defer s.mu.RUnlock() - return f(s.trackBlobs) -} - -func (s *storage) SetFulfilledHeight(height uint64) error { - fulfilledHeightKey := makeGlobalStateKey(globalStateFulfilledHeight) - fulfilledHeightValue := makeUint64Value(height) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } - - return nil - }) -} - -func (s *storage) GetFulfilledHeight() (uint64, error) { - fulfilledHeightKey := makeGlobalStateKey(globalStateFulfilledHeight) - var fulfilledHeight uint64 - - if err := s.db.View(func(txn *badger.Txn) error { - item, err := txn.Get(fulfilledHeightKey) - if err != nil { - return fmt.Errorf("failed to find fulfilled height entry: %w", err) - } - - fulfilledHeight, err = getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve fulfilled height value: %w", err) - } - - return nil - }); err != nil { - return 0, err - } - - return fulfilledHeight, nil -} - -func (s *storage) trackBlob(txn *badger.Txn, blockHeight uint64, c cid.Cid) error { - if err := txn.Set(makeBlobRecordKey(blockHeight, c), nil); err != nil { - return fmt.Errorf("failed to add blob record: %w", err) - } - - latestHeightKey := makeLatestHeightKey(c) - item, err := txn.Get(latestHeightKey) - if err != nil { - if !errors.Is(err, badger.ErrKeyNotFound) { - return fmt.Errorf("failed to get latest height: %w", err) - } - } else { - latestHeight, err := getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve latest height value: %w", err) - } - - // don't update the latest height if there is already a higher block height containing this blob - if latestHeight >= blockHeight { - return nil - } - } - - latestHeightValue := makeUint64Value(blockHeight) - - if err := txn.Set(latestHeightKey, latestHeightValue); err != nil { - return fmt.Errorf("failed to set latest height value: %w", err) - } - - return nil -} - -func (s *storage) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - cidsPerBatch := cidsPerBatch - maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, blobRecordKeyLength+latestHeightKeyLength+8) - if maxCidsPerBatch < cidsPerBatch { - cidsPerBatch = maxCidsPerBatch - } - - for len(cids) > 0 { - batchSize := cidsPerBatch - if len(cids) < batchSize { - batchSize = len(cids) - } - batch := cids[:batchSize] - - if err := retryOnConflict(s.db, func(txn *badger.Txn) error { - for _, c := range batch { - if err := s.trackBlob(txn, blockHeight, c); err != nil { - return fmt.Errorf("failed to track blob %s: %w", c.String(), err) - } - } - - return nil - }); err != nil { - return err - } - - cids = cids[batchSize:] - } - - return nil -} - -func (s *storage) batchDelete(deleteInfos []*deleteInfo) error { - return s.db.Update(func(txn *badger.Txn) error { - for _, dInfo := range deleteInfos { - if err := txn.Delete(makeBlobRecordKey(dInfo.height, dInfo.cid)); err != nil { - return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.cid.String(), err) - } - - if dInfo.deleteLatestHeightRecord { - if err := txn.Delete(makeLatestHeightKey(dInfo.cid)); err != nil { - return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.cid.String(), err) - } - } - } - - return nil - }) -} - -func (s *storage) batchDeleteItemLimit() int { - itemsPerBatch := 256 - maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, blobRecordKeyLength+latestHeightKeyLength) - if maxItemsPerBatch < itemsPerBatch { - itemsPerBatch = maxItemsPerBatch - } - return itemsPerBatch -} - -func (s *storage) PruneUpToHeight(height uint64) error { - blobRecordPrefix := []byte{prefixBlobRecord} - itemsPerBatch := s.batchDeleteItemLimit() - var batch []*deleteInfo - - s.mu.Lock() - defer s.mu.Unlock() - - if err := s.setPrunedHeight(height); err != nil { - return err - } - - if err := s.db.View(func(txn *badger.Txn) error { - it := txn.NewIterator(badger.IteratorOptions{ - PrefetchValues: false, - Prefix: blobRecordPrefix, - }) - defer it.Close() - - // iterate over blob records, calling pruneCallback for any CIDs that should be pruned - // and cleaning up the corresponding tracker records - for it.Seek(blobRecordPrefix); it.ValidForPrefix(blobRecordPrefix); it.Next() { - blobRecordItem := it.Item() - blobRecordKey := blobRecordItem.Key() - - blockHeight, blobCid, err := parseBlobRecordKey(blobRecordKey) - if err != nil { - return fmt.Errorf("malformed blob record key %v: %w", blobRecordKey, err) - } - - // iteration occurs in key order, so block heights are guaranteed to be ascending - if blockHeight > height { - break - } - - dInfo := &deleteInfo{ - cid: blobCid, - height: blockHeight, - } - - latestHeightKey := makeLatestHeightKey(blobCid) - latestHeightItem, err := txn.Get(latestHeightKey) - if err != nil { - return fmt.Errorf("failed to get latest height entry for Cid %s: %w", blobCid.String(), err) - } - - latestHeight, err := getUint64Value(latestHeightItem) - if err != nil { - return fmt.Errorf("failed to retrieve latest height value for Cid %s: %w", blobCid.String(), err) - } - - // a blob is only removable if it is not referenced by any blob tree at a higher height - if latestHeight < blockHeight { - // this should never happen - return fmt.Errorf( - "inconsistency detected: latest height recorded for Cid %s is %d, but blob record exists at height %d", - blobCid.String(), latestHeight, blockHeight, - ) - } - - // the current block height is the last to reference this CID, prune the CID and remove - // all tracker records - if latestHeight == blockHeight { - if err := s.pruneCallback(blobCid); err != nil { - return err - } - dInfo.deleteLatestHeightRecord = true - } - - // remove tracker records for pruned heights - batch = append(batch, dInfo) - if len(batch) == itemsPerBatch { - if err := s.batchDelete(batch); err != nil { - return err - } - batch = nil - } - } - - if len(batch) > 0 { - if err := s.batchDelete(batch); err != nil { - return err - } - } - - return nil - }); err != nil { - return err - } - - // this is a good time to do garbage collection - if err := s.db.RunValueLogGC(0.5); err != nil { - s.logger.Err(err).Msg("failed to run value log garbage collection") - } - - return nil -} - -func (s *storage) setPrunedHeight(height uint64) error { - prunedHeightKey := makeGlobalStateKey(globalStatePrunedHeight) - prunedHeightValue := makeUint64Value(height) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil - }) -} - -func (s *storage) GetPrunedHeight() (uint64, error) { - prunedHeightKey := makeGlobalStateKey(globalStatePrunedHeight) - var prunedHeight uint64 - - if err := s.db.View(func(txn *badger.Txn) error { - item, err := txn.Get(prunedHeightKey) - if err != nil { - return fmt.Errorf("failed to find pruned height entry: %w", err) - } - - prunedHeight, err = getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve pruned height value: %w", err) - } - - return nil - }); err != nil { - return 0, err - } - - return prunedHeight, nil -} - -type deleteInfo struct { - cid cid.Cid - height uint64 - deleteLatestHeightRecord bool -} diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go new file mode 100644 index 00000000000..08918e2cf02 --- /dev/null +++ b/storage/badger/execution_data_tracker.go @@ -0,0 +1,432 @@ +package badger + +import ( + "encoding/binary" + "errors" + "fmt" + "sync" + + "github.com/dgraph-io/badger/v2" + "github.com/hashicorp/go-multierror" + "github.com/ipfs/go-cid" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/module/executiondatasync/tracker" +) + +func getUint64Value(item *badger.Item) (uint64, error) { + value, err := item.ValueCopy(nil) + if err != nil { + return 0, err + } + + return binary.BigEndian.Uint64(value), nil +} + +// getBatchItemCountLimit returns the maximum number of items that can be included in a single batch +// transaction based on the number / total size of updates per item. +func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePerItem int64) int { + totalSizePerItem := 2*writeCountPerItem + writeSizePerItem // 2 bytes per entry for user and internal meta + maxItemCountByWriteCount := db.MaxBatchCount() / writeCountPerItem + maxItemCountByWriteSize := db.MaxBatchSize() / totalSizePerItem + + if maxItemCountByWriteCount < maxItemCountByWriteSize { + return int(maxItemCountByWriteCount) + } else { + return int(maxItemCountByWriteSize) + } +} + +func retryOnConflict(db *badger.DB, fn func(txn *badger.Txn) error) error { + for { + err := db.Update(fn) + if errors.Is(err, badger.ErrConflict) { + continue + } + return err + } +} + +type StorageOption func(*Storage) + +var _ tracker.Storage = (*Storage)(nil) + +// The Storage component tracks the following information: +// - the latest pruned height +// - the latest fulfilled height +// - the set of CIDs of the execution data blobs we know about at each height, so that +// once we prune a fulfilled height we can remove the blob data from local storage +// - for each CID, the most recent height that it was observed at, so that when pruning +// a fulfilled height we don't remove any blob data that is still needed at higher heights +// +// The storage component calls the given prune callback for a CID when the last height +// at which that CID appears is pruned. The prune callback can be used to delete the +// corresponding blob data from the blob store. +type Storage struct { + // ensures that pruning operations are not run concurrently with any other db writes + // we acquire the read lock when we want to perform a non-prune WRITE + // we acquire the write lock when we want to perform a prune WRITE + mu sync.RWMutex + + db *badger.DB + pruneCallback tracker.PruneCallback + logger zerolog.Logger +} + +func WithPruneCallback(callback tracker.PruneCallback) StorageOption { + return func(s *Storage) { + s.pruneCallback = callback + } +} + +func NewStorageTracker(dbPath string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*Storage, error) { + lg := logger.With().Str("module", "tracker_storage").Logger() + db, err := badger.Open(badger.LSMOnlyOptions(dbPath)) + if err != nil { + return nil, fmt.Errorf("could not open tracker db: %w", err) + } + + storage := &Storage{ + db: db, + pruneCallback: func(c cid.Cid) error { return nil }, + logger: lg, + } + + for _, opt := range opts { + opt(storage) + } + + lg.Info().Msgf("initialize storage with start height: %d", startHeight) + + if err := storage.init(startHeight); err != nil { + return nil, fmt.Errorf("failed to initialize storage: %w", err) + } + + lg.Info().Msgf("storage initialized") + + return storage, nil +} + +func (s *Storage) init(startHeight uint64) error { + fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() + prunedHeight, prunedHeightErr := s.GetPrunedHeight() + + if fulfilledHeightErr == nil && prunedHeightErr == nil { + if prunedHeight > fulfilledHeight { + return fmt.Errorf( + "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", + prunedHeight, + fulfilledHeight, + ) + } + + s.logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) + // replay pruning in case it was interrupted during previous shutdown + if err := s.PruneUpToHeight(prunedHeight); err != nil { + return fmt.Errorf("failed to replay pruning: %w", err) + } + s.logger.Info().Msgf("finished pruning") + } else if errors.Is(fulfilledHeightErr, badger.ErrKeyNotFound) && errors.Is(prunedHeightErr, badger.ErrKeyNotFound) { + // db is empty, we need to bootstrap it + if err := s.bootstrap(startHeight); err != nil { + return fmt.Errorf("failed to bootstrap storage: %w", err) + } + } else { + return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() + } + + return nil +} + +func (s *Storage) bootstrap(startHeight uint64) error { + fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) + fulfilledHeightValue := tracker.MakeUint64Value(startHeight) + + prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) + prunedHeightValue := tracker.MakeUint64Value(startHeight) + + return s.db.Update(func(txn *badger.Txn) error { + if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + + if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil + }) +} + +func (s *Storage) Update(f tracker.UpdateFn) error { + s.mu.RLock() + defer s.mu.RUnlock() + return f(s.trackBlobs) +} + +func (s *Storage) SetFulfilledHeight(height uint64) error { + fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) + fulfilledHeightValue := tracker.MakeUint64Value(height) + + return s.db.Update(func(txn *badger.Txn) error { + if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + + return nil + }) +} + +func (s *Storage) GetFulfilledHeight() (uint64, error) { + fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) + var fulfilledHeight uint64 + + if err := s.db.View(func(txn *badger.Txn) error { + item, err := txn.Get(fulfilledHeightKey) + if err != nil { + return fmt.Errorf("failed to find fulfilled height entry: %w", err) + } + + fulfilledHeight, err = getUint64Value(item) + if err != nil { + return fmt.Errorf("failed to retrieve fulfilled height value: %w", err) + } + + return nil + }); err != nil { + return 0, err + } + + return fulfilledHeight, nil +} + +func (s *Storage) trackBlob(txn *badger.Txn, blockHeight uint64, c cid.Cid) error { + if err := txn.Set(tracker.MakeBlobRecordKey(blockHeight, c), nil); err != nil { + return fmt.Errorf("failed to add blob record: %w", err) + } + + latestHeightKey := tracker.MakeLatestHeightKey(c) + item, err := txn.Get(latestHeightKey) + if err != nil { + if !errors.Is(err, badger.ErrKeyNotFound) { + return fmt.Errorf("failed to get latest height: %w", err) + } + } else { + latestHeight, err := getUint64Value(item) + if err != nil { + return fmt.Errorf("failed to retrieve latest height value: %w", err) + } + + // don't update the latest height if there is already a higher block height containing this blob + if latestHeight >= blockHeight { + return nil + } + } + + latestHeightValue := tracker.MakeUint64Value(blockHeight) + + if err := txn.Set(latestHeightKey, latestHeightValue); err != nil { + return fmt.Errorf("failed to set latest height value: %w", err) + } + + return nil +} + +func (s *Storage) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { + cidsPerBatch := tracker.CidsPerBatch + maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength+8) + if maxCidsPerBatch < cidsPerBatch { + cidsPerBatch = maxCidsPerBatch + } + + for len(cids) > 0 { + batchSize := cidsPerBatch + if len(cids) < batchSize { + batchSize = len(cids) + } + batch := cids[:batchSize] + + if err := retryOnConflict(s.db, func(txn *badger.Txn) error { + for _, c := range batch { + if err := s.trackBlob(txn, blockHeight, c); err != nil { + return fmt.Errorf("failed to track blob %s: %w", c.String(), err) + } + } + + return nil + }); err != nil { + return err + } + + cids = cids[batchSize:] + } + + return nil +} + +func (s *Storage) batchDelete(deleteInfos []*deleteInfo) error { + return s.db.Update(func(txn *badger.Txn) error { + for _, dInfo := range deleteInfos { + if err := txn.Delete(tracker.MakeBlobRecordKey(dInfo.height, dInfo.cid)); err != nil { + return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.cid.String(), err) + } + + if dInfo.deleteLatestHeightRecord { + if err := txn.Delete(tracker.MakeLatestHeightKey(dInfo.cid)); err != nil { + return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.cid.String(), err) + } + } + } + + return nil + }) +} + +func (s *Storage) batchDeleteItemLimit() int { + itemsPerBatch := 256 + maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength) + if maxItemsPerBatch < itemsPerBatch { + itemsPerBatch = maxItemsPerBatch + } + return itemsPerBatch +} + +func (s *Storage) PruneUpToHeight(height uint64) error { + blobRecordPrefix := []byte{tracker.PrefixBlobRecord} + itemsPerBatch := s.batchDeleteItemLimit() + var batch []*deleteInfo + + s.mu.Lock() + defer s.mu.Unlock() + + if err := s.setPrunedHeight(height); err != nil { + return err + } + + if err := s.db.View(func(txn *badger.Txn) error { + it := txn.NewIterator(badger.IteratorOptions{ + PrefetchValues: false, + Prefix: blobRecordPrefix, + }) + defer it.Close() + + // iterate over blob records, calling pruneCallback for any CIDs that should be pruned + // and cleaning up the corresponding tracker records + for it.Seek(blobRecordPrefix); it.ValidForPrefix(blobRecordPrefix); it.Next() { + blobRecordItem := it.Item() + blobRecordKey := blobRecordItem.Key() + + blockHeight, blobCid, err := tracker.ParseBlobRecordKey(blobRecordKey) + if err != nil { + return fmt.Errorf("malformed blob record key %v: %w", blobRecordKey, err) + } + + // iteration occurs in key order, so block heights are guaranteed to be ascending + if blockHeight > height { + break + } + + dInfo := &deleteInfo{ + cid: blobCid, + height: blockHeight, + } + + latestHeightKey := tracker.MakeLatestHeightKey(blobCid) + latestHeightItem, err := txn.Get(latestHeightKey) + if err != nil { + return fmt.Errorf("failed to get latest height entry for Cid %s: %w", blobCid.String(), err) + } + + latestHeight, err := getUint64Value(latestHeightItem) + if err != nil { + return fmt.Errorf("failed to retrieve latest height value for Cid %s: %w", blobCid.String(), err) + } + + // a blob is only removable if it is not referenced by any blob tree at a higher height + if latestHeight < blockHeight { + // this should never happen + return fmt.Errorf( + "inconsistency detected: latest height recorded for Cid %s is %d, but blob record exists at height %d", + blobCid.String(), latestHeight, blockHeight, + ) + } + + // the current block height is the last to reference this CID, prune the CID and remove + // all tracker records + if latestHeight == blockHeight { + if err := s.pruneCallback(blobCid); err != nil { + return err + } + dInfo.deleteLatestHeightRecord = true + } + + // remove tracker records for pruned heights + batch = append(batch, dInfo) + if len(batch) == itemsPerBatch { + if err := s.batchDelete(batch); err != nil { + return err + } + batch = nil + } + } + + if len(batch) > 0 { + if err := s.batchDelete(batch); err != nil { + return err + } + } + + return nil + }); err != nil { + return err + } + + // this is a good time to do garbage collection + if err := s.db.RunValueLogGC(0.5); err != nil { + s.logger.Err(err).Msg("failed to run value log garbage collection") + } + + return nil +} + +func (s *Storage) setPrunedHeight(height uint64) error { + prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) + prunedHeightValue := tracker.MakeUint64Value(height) + + return s.db.Update(func(txn *badger.Txn) error { + if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil + }) +} + +func (s *Storage) GetPrunedHeight() (uint64, error) { + prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) + var prunedHeight uint64 + + if err := s.db.View(func(txn *badger.Txn) error { + item, err := txn.Get(prunedHeightKey) + if err != nil { + return fmt.Errorf("failed to find pruned height entry: %w", err) + } + + prunedHeight, err = getUint64Value(item) + if err != nil { + return fmt.Errorf("failed to retrieve pruned height value: %w", err) + } + + return nil + }); err != nil { + return 0, err + } + + return prunedHeight, nil +} + +type deleteInfo struct { + cid cid.Cid + height uint64 + deleteLatestHeightRecord bool +} diff --git a/module/executiondatasync/tracker/storage_test.go b/storage/badger/execution_data_tracker_test.go similarity index 73% rename from module/executiondatasync/tracker/storage_test.go rename to storage/badger/execution_data_tracker_test.go index 76c7b613ab6..6080c60854b 100644 --- a/module/executiondatasync/tracker/storage_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -1,4 +1,4 @@ -package tracker +package badger import ( "crypto/rand" @@ -11,6 +11,7 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/module/blobs" + "github.com/onflow/flow-go/module/executiondatasync/tracker" ) func randomCid() cid.Cid { @@ -24,7 +25,7 @@ func randomCid() cid.Cid { func TestPrune(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - storage, err := OpenStorage(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] assert.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) @@ -41,7 +42,7 @@ func TestPrune(t *testing.T) { c3 := randomCid() c4 := randomCid() - require.NoError(t, storage.Update(func(tbf TrackBlobsFn) error { + require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c3, c4)) @@ -56,22 +57,22 @@ func TestPrune(t *testing.T) { assert.Len(t, expectedPrunedCIDs, 0) err = storage.db.View(func(txn *badger.Txn) error { - _, err := txn.Get(makeBlobRecordKey(1, c1)) + _, err := txn.Get(tracker.MakeBlobRecordKey(1, c1)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(makeLatestHeightKey(c1)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(makeBlobRecordKey(1, c2)) + _, err = txn.Get(tracker.MakeBlobRecordKey(1, c2)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(makeLatestHeightKey(c2)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(makeBlobRecordKey(2, c3)) + _, err = txn.Get(tracker.MakeBlobRecordKey(2, c3)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c3)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c3)) assert.NoError(t, err) - _, err = txn.Get(makeBlobRecordKey(2, c4)) + _, err = txn.Get(tracker.MakeBlobRecordKey(2, c4)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c4)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c4)) assert.NoError(t, err) return nil @@ -83,7 +84,7 @@ func TestPrune(t *testing.T) { // if that CID also exists at another height above the pruned height, the CID should not be pruned. func TestPruneNonLatestHeight(t *testing.T) { storageDir := t.TempDir() - storage, err := OpenStorage(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { assert.Fail(t, "unexpected CID pruned: %s", c.String()) return nil })) @@ -94,7 +95,7 @@ func TestPruneNonLatestHeight(t *testing.T) { c1 := randomCid() c2 := randomCid() - require.NoError(t, storage.Update(func(tbf TrackBlobsFn) error { + require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c1, c2)) @@ -107,13 +108,13 @@ func TestPruneNonLatestHeight(t *testing.T) { assert.Equal(t, uint64(1), prunedHeight) err = storage.db.View(func(txn *badger.Txn) error { - _, err = txn.Get(makeBlobRecordKey(2, c1)) + _, err = txn.Get(tracker.MakeBlobRecordKey(2, c1)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c1)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) assert.NoError(t, err) - _, err = txn.Get(makeBlobRecordKey(2, c2)) + _, err = txn.Get(tracker.MakeBlobRecordKey(2, c2)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c2)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) assert.NoError(t, err) return nil @@ -126,7 +127,7 @@ func TestPruneNonLatestHeight(t *testing.T) { func TestAscendingOrderOfRecords(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - storage, err := OpenStorage(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] assert.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) @@ -143,7 +144,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { c2 := randomCid() c3 := randomCid() - require.NoError(t, storage.Update(func(tbf TrackBlobsFn) error { + require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1)) require.NoError(t, tbf(2, c2)) // It is important to check if the record with height 256 does not precede @@ -162,21 +163,21 @@ func TestAscendingOrderOfRecords(t *testing.T) { err = storage.db.View(func(txn *badger.Txn) error { // expected that blob record with height 1 was removed - _, err := txn.Get(makeBlobRecordKey(1, c1)) + _, err := txn.Get(tracker.MakeBlobRecordKey(1, c1)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(makeLatestHeightKey(c1)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) assert.ErrorIs(t, err, badger.ErrKeyNotFound) // expected that blob record with height 2 exists - _, err = txn.Get(makeBlobRecordKey(2, c2)) + _, err = txn.Get(tracker.MakeBlobRecordKey(2, c2)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c2)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) assert.NoError(t, err) // expected that blob record with height 256 exists - _, err = txn.Get(makeBlobRecordKey(256, c3)) + _, err = txn.Get(tracker.MakeBlobRecordKey(256, c3)) assert.NoError(t, err) - _, err = txn.Get(makeLatestHeightKey(c3)) + _, err = txn.Get(tracker.MakeLatestHeightKey(c3)) assert.NoError(t, err) return nil diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go new file mode 100644 index 00000000000..c86d0bc9ec3 --- /dev/null +++ b/storage/pebble/execution_data_tracker.go @@ -0,0 +1 @@ +package pebble From 5893168d891cd40b0df51b590dbbe6913dd9fafd Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Fri, 26 Jul 2024 17:15:18 +0300 Subject: [PATCH 02/33] Refactored badger version of execution data tracker --- .../node_builder/access_node_builder.go | 2 +- cmd/execution_builder.go | 2 +- cmd/observer/node_builder/observer_builder.go | 2 +- module/executiondatasync/tracker/storage.go | 36 +--- storage/badger/execution_data_tracker.go | 198 ++++++------------ storage/badger/execution_data_tracker_test.go | 77 +++---- storage/badger/operation/common.go | 3 + .../operation/execution_data_tracker.go | 56 +++++ storage/badger/operation/prefix.go | 4 + .../operation/execution_data_tracker.go | 0 10 files changed, 175 insertions(+), 205 deletions(-) create mode 100644 storage/badger/operation/execution_data_tracker.go create mode 100644 storage/pebble/operation/execution_data_tracker.go diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index edda383b210..264657ce8d7 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -666,7 +666,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = bstorage.NewStorageTracker( + builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( trackerDir, sealed.Height, node.Logger, diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index 28e4306407c..4b02d05a28f 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -946,7 +946,7 @@ func (exeNode *ExecutionNode) LoadExecutionDataPruner( } trackerDir := filepath.Join(exeNode.exeConf.executionDataDir, "tracker") - exeNode.executionDataTracker, err = storage.NewStorageTracker( + exeNode.executionDataTracker, err = storage.NewExecutionDataTracker( trackerDir, sealed.Height, node.Logger, diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index d63a6770f83..c83c96c5989 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1231,7 +1231,7 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = bstorage.NewStorageTracker( + builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( trackerDir, sealed.Height, node.Logger, diff --git a/module/executiondatasync/tracker/storage.go b/module/executiondatasync/tracker/storage.go index cf3a9441a81..dfa19822082 100644 --- a/module/executiondatasync/tracker/storage.go +++ b/module/executiondatasync/tracker/storage.go @@ -8,7 +8,7 @@ import ( "github.com/onflow/flow-go/module/blobs" ) -// badger key prefixes +// key prefixes const ( PrefixGlobalState byte = iota + 1 // global state variables PrefixLatestHeight // tracks, for each blob, the latest height at which there exists a block whose execution data contains the blob @@ -21,25 +21,8 @@ const ( ) const CidsPerBatch = 16 // number of cids to track per batch - -const GlobalStateKeyLength = 2 - -func MakeGlobalStateKey(state byte) []byte { - globalStateKey := make([]byte, GlobalStateKeyLength) - globalStateKey[0] = PrefixGlobalState - globalStateKey[1] = state - return globalStateKey -} - const BlobRecordKeyLength = 1 + 8 + blobs.CidLength - -func MakeBlobRecordKey(blockHeight uint64, c cid.Cid) []byte { - blobRecordKey := make([]byte, BlobRecordKeyLength) - blobRecordKey[0] = PrefixBlobRecord - binary.BigEndian.PutUint64(blobRecordKey[1:], blockHeight) - copy(blobRecordKey[1+8:], c.Bytes()) - return blobRecordKey -} +const LatestHeightKeyLength = 1 + blobs.CidLength func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { blockHeight := binary.BigEndian.Uint64(key[1:]) @@ -47,21 +30,6 @@ func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { return blockHeight, c, err } -const LatestHeightKeyLength = 1 + blobs.CidLength - -func MakeLatestHeightKey(c cid.Cid) []byte { - latestHeightKey := make([]byte, LatestHeightKeyLength) - latestHeightKey[0] = PrefixLatestHeight - copy(latestHeightKey[1:], c.Bytes()) - return latestHeightKey -} - -func MakeUint64Value(v uint64) []byte { - value := make([]byte, 8) - binary.BigEndian.PutUint64(value, v) - return value -} - // TrackBlobsFn is passed to the UpdateFn provided to Storage.Update, // and can be called to track a list of cids at a given block height. // It returns an error if the update failed. diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 08918e2cf02..df8e617bc15 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -1,7 +1,6 @@ package badger import ( - "encoding/binary" "errors" "fmt" "sync" @@ -12,17 +11,10 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/badger/operation" ) -func getUint64Value(item *badger.Item) (uint64, error) { - value, err := item.ValueCopy(nil) - if err != nil { - return 0, err - } - - return binary.BigEndian.Uint64(value), nil -} - // getBatchItemCountLimit returns the maximum number of items that can be included in a single batch // transaction based on the number / total size of updates per item. func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePerItem int64) int { @@ -37,21 +29,11 @@ func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePer } } -func retryOnConflict(db *badger.DB, fn func(txn *badger.Txn) error) error { - for { - err := db.Update(fn) - if errors.Is(err, badger.ErrConflict) { - continue - } - return err - } -} - -type StorageOption func(*Storage) +type StorageOption func(*ExecutionDataTracker) -var _ tracker.Storage = (*Storage)(nil) +var _ tracker.Storage = (*ExecutionDataTracker)(nil) -// The Storage component tracks the following information: +// The ExecutionDataTracker component tracks the following information: // - the latest pruned height // - the latest fulfilled height // - the set of CIDs of the execution data blobs we know about at each height, so that @@ -62,7 +44,7 @@ var _ tracker.Storage = (*Storage)(nil) // The storage component calls the given prune callback for a CID when the last height // at which that CID appears is pruned. The prune callback can be used to delete the // corresponding blob data from the blob store. -type Storage struct { +type ExecutionDataTracker struct { // ensures that pruning operations are not run concurrently with any other db writes // we acquire the read lock when we want to perform a non-prune WRITE // we acquire the write lock when we want to perform a prune WRITE @@ -74,19 +56,19 @@ type Storage struct { } func WithPruneCallback(callback tracker.PruneCallback) StorageOption { - return func(s *Storage) { + return func(s *ExecutionDataTracker) { s.pruneCallback = callback } } -func NewStorageTracker(dbPath string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*Storage, error) { +func NewExecutionDataTracker(dbPath string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*ExecutionDataTracker, error) { lg := logger.With().Str("module", "tracker_storage").Logger() db, err := badger.Open(badger.LSMOnlyOptions(dbPath)) if err != nil { return nil, fmt.Errorf("could not open tracker db: %w", err) } - storage := &Storage{ + storage := &ExecutionDataTracker{ db: db, pruneCallback: func(c cid.Cid) error { return nil }, logger: lg, @@ -107,7 +89,7 @@ func NewStorageTracker(dbPath string, startHeight uint64, logger zerolog.Logger, return storage, nil } -func (s *Storage) init(startHeight uint64) error { +func (s *ExecutionDataTracker) init(startHeight uint64) error { fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() prunedHeight, prunedHeightErr := s.GetPrunedHeight() @@ -126,7 +108,7 @@ func (s *Storage) init(startHeight uint64) error { return fmt.Errorf("failed to replay pruning: %w", err) } s.logger.Info().Msgf("finished pruning") - } else if errors.Is(fulfilledHeightErr, badger.ErrKeyNotFound) && errors.Is(prunedHeightErr, badger.ErrKeyNotFound) { + } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { // db is empty, we need to bootstrap it if err := s.bootstrap(startHeight); err != nil { return fmt.Errorf("failed to bootstrap storage: %w", err) @@ -138,101 +120,74 @@ func (s *Storage) init(startHeight uint64) error { return nil } -func (s *Storage) bootstrap(startHeight uint64) error { - fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) - fulfilledHeightValue := tracker.MakeUint64Value(startHeight) - - prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) - prunedHeightValue := tracker.MakeUint64Value(startHeight) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } +func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { + err := s.db.Update(operation.InsertTrackerFulfilledHeight(startHeight)) + if err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } - if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } + err = s.db.Update(operation.InsertTrackerPrunedHeight(startHeight)) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } - return nil - }) + return nil } -func (s *Storage) Update(f tracker.UpdateFn) error { +func (s *ExecutionDataTracker) Update(f tracker.UpdateFn) error { s.mu.RLock() defer s.mu.RUnlock() return f(s.trackBlobs) } -func (s *Storage) SetFulfilledHeight(height uint64) error { - fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) - fulfilledHeightValue := tracker.MakeUint64Value(height) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(fulfilledHeightKey, fulfilledHeightValue); err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } +func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { + err := s.db.Update(operation.UpdateTrackerFulfilledHeight(height)) + if err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } - return nil - }) + return nil } -func (s *Storage) GetFulfilledHeight() (uint64, error) { - fulfilledHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStateFulfilledHeight) +func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { var fulfilledHeight uint64 - if err := s.db.View(func(txn *badger.Txn) error { - item, err := txn.Get(fulfilledHeightKey) - if err != nil { - return fmt.Errorf("failed to find fulfilled height entry: %w", err) - } - - fulfilledHeight, err = getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve fulfilled height value: %w", err) - } - - return nil - }); err != nil { + err := s.db.View(operation.RetrieveTrackerFulfilledHeight(&fulfilledHeight)) + if err != nil { return 0, err } return fulfilledHeight, nil } -func (s *Storage) trackBlob(txn *badger.Txn, blockHeight uint64, c cid.Cid) error { - if err := txn.Set(tracker.MakeBlobRecordKey(blockHeight, c), nil); err != nil { +func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c cid.Cid) error { + err := operation.InsertBlob(blockHeight, c)(tx) + if err != nil { return fmt.Errorf("failed to add blob record: %w", err) } - latestHeightKey := tracker.MakeLatestHeightKey(c) - item, err := txn.Get(latestHeightKey) + var latestHeight uint64 + err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(tx) if err != nil { - if !errors.Is(err, badger.ErrKeyNotFound) { + if !errors.Is(err, storage.ErrNotFound) { return fmt.Errorf("failed to get latest height: %w", err) } } else { - latestHeight, err := getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve latest height value: %w", err) - } - // don't update the latest height if there is already a higher block height containing this blob if latestHeight >= blockHeight { return nil } } - latestHeightValue := tracker.MakeUint64Value(blockHeight) - - if err := txn.Set(latestHeightKey, latestHeightValue); err != nil { + err = operation.UpsertTrackerLatestHeight(c, blockHeight)(tx) + if err != nil { return fmt.Errorf("failed to set latest height value: %w", err) } return nil } -func (s *Storage) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { +func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { cidsPerBatch := tracker.CidsPerBatch maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength+8) if maxCidsPerBatch < cidsPerBatch { @@ -246,7 +201,7 @@ func (s *Storage) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { } batch := cids[:batchSize] - if err := retryOnConflict(s.db, func(txn *badger.Txn) error { + if err := operation.RetryOnConflict(s.db.Update, func(txn *badger.Txn) error { for _, c := range batch { if err := s.trackBlob(txn, blockHeight, c); err != nil { return fmt.Errorf("failed to track blob %s: %w", c.String(), err) @@ -264,25 +219,25 @@ func (s *Storage) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { return nil } -func (s *Storage) batchDelete(deleteInfos []*deleteInfo) error { - return s.db.Update(func(txn *badger.Txn) error { - for _, dInfo := range deleteInfos { - if err := txn.Delete(tracker.MakeBlobRecordKey(dInfo.height, dInfo.cid)); err != nil { - return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.cid.String(), err) - } +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*deleteInfo) error { + for _, dInfo := range deleteInfos { + err := s.db.Update(operation.RemoveBlob(dInfo.height, dInfo.cid)) + if err != nil { + return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.cid.String(), err) + } - if dInfo.deleteLatestHeightRecord { - if err := txn.Delete(tracker.MakeLatestHeightKey(dInfo.cid)); err != nil { - return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.cid.String(), err) - } + if dInfo.deleteLatestHeightRecord { + err = s.db.Update(operation.RemoveTrackerLatestHeight(dInfo.cid)) + if err != nil { + return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.cid.String(), err) } } + } - return nil - }) + return nil } -func (s *Storage) batchDeleteItemLimit() int { +func (s *ExecutionDataTracker) batchDeleteItemLimit() int { itemsPerBatch := 256 maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength) if maxItemsPerBatch < itemsPerBatch { @@ -291,7 +246,7 @@ func (s *Storage) batchDeleteItemLimit() int { return itemsPerBatch } -func (s *Storage) PruneUpToHeight(height uint64) error { +func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{tracker.PrefixBlobRecord} itemsPerBatch := s.batchDeleteItemLimit() var batch []*deleteInfo @@ -331,17 +286,12 @@ func (s *Storage) PruneUpToHeight(height uint64) error { height: blockHeight, } - latestHeightKey := tracker.MakeLatestHeightKey(blobCid) - latestHeightItem, err := txn.Get(latestHeightKey) + var latestHeight uint64 + err = operation.RetrieveTrackerLatestHeight(blobCid, &latestHeight)(txn) if err != nil { return fmt.Errorf("failed to get latest height entry for Cid %s: %w", blobCid.String(), err) } - latestHeight, err := getUint64Value(latestHeightItem) - if err != nil { - return fmt.Errorf("failed to retrieve latest height value for Cid %s: %w", blobCid.String(), err) - } - // a blob is only removable if it is not referenced by any blob tree at a higher height if latestHeight < blockHeight { // this should never happen @@ -389,36 +339,20 @@ func (s *Storage) PruneUpToHeight(height uint64) error { return nil } -func (s *Storage) setPrunedHeight(height uint64) error { - prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) - prunedHeightValue := tracker.MakeUint64Value(height) - - return s.db.Update(func(txn *badger.Txn) error { - if err := txn.Set(prunedHeightKey, prunedHeightValue); err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } +func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { + err := s.db.Update(operation.UpdateTrackerPrunedHeight(height)) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } - return nil - }) + return nil } -func (s *Storage) GetPrunedHeight() (uint64, error) { - prunedHeightKey := tracker.MakeGlobalStateKey(tracker.GlobalStatePrunedHeight) +func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { var prunedHeight uint64 - if err := s.db.View(func(txn *badger.Txn) error { - item, err := txn.Get(prunedHeightKey) - if err != nil { - return fmt.Errorf("failed to find pruned height entry: %w", err) - } - - prunedHeight, err = getUint64Value(item) - if err != nil { - return fmt.Errorf("failed to retrieve pruned height value: %w", err) - } - - return nil - }); err != nil { + err := s.db.View(operation.RetrieveTrackerPrunedHeight(&prunedHeight)) + if err != nil { return 0, err } diff --git a/storage/badger/execution_data_tracker_test.go b/storage/badger/execution_data_tracker_test.go index 6080c60854b..88fbc75409d 100644 --- a/storage/badger/execution_data_tracker_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -12,6 +12,8 @@ import ( "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/badger/operation" ) func randomCid() cid.Cid { @@ -25,7 +27,7 @@ func randomCid() cid.Cid { func TestPrune(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] assert.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) @@ -42,37 +44,38 @@ func TestPrune(t *testing.T) { c3 := randomCid() c4 := randomCid() - require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c3, c4)) return nil })) - require.NoError(t, storage.PruneUpToHeight(1)) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) - prunedHeight, err := storage.GetPrunedHeight() + prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) assert.Equal(t, uint64(1), prunedHeight) assert.Len(t, expectedPrunedCIDs, 0) - err = storage.db.View(func(txn *badger.Txn) error { - _, err := txn.Get(tracker.MakeBlobRecordKey(1, c1)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(tracker.MakeBlobRecordKey(1, c2)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) - - _, err = txn.Get(tracker.MakeBlobRecordKey(2, c3)) + var latestHeight uint64 + err = executionDataTracker.db.View(func(txn *badger.Txn) error { + err = operation.RetrieveBlob(1, c1)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveBlob(1, c2)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) + + err = operation.RetrieveBlob(2, c3)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c3)) + err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeBlobRecordKey(2, c4)) + err = operation.RetrieveBlob(2, c4)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c4)) + err = operation.RetrieveTrackerLatestHeight(c4, &latestHeight)(txn) assert.NoError(t, err) return nil @@ -84,7 +87,7 @@ func TestPrune(t *testing.T) { // if that CID also exists at another height above the pruned height, the CID should not be pruned. func TestPruneNonLatestHeight(t *testing.T) { storageDir := t.TempDir() - storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + storage, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { assert.Fail(t, "unexpected CID pruned: %s", c.String()) return nil })) @@ -107,14 +110,15 @@ func TestPruneNonLatestHeight(t *testing.T) { require.NoError(t, err) assert.Equal(t, uint64(1), prunedHeight) + var latestHeight uint64 err = storage.db.View(func(txn *badger.Txn) error { - _, err = txn.Get(tracker.MakeBlobRecordKey(2, c1)) + err = operation.RetrieveBlob(2, c1)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeBlobRecordKey(2, c2)) + err = operation.RetrieveBlob(2, c2)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) assert.NoError(t, err) return nil @@ -127,7 +131,7 @@ func TestPruneNonLatestHeight(t *testing.T) { func TestAscendingOrderOfRecords(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - storage, err := NewStorageTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] assert.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) @@ -144,7 +148,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { c2 := randomCid() c3 := randomCid() - require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1)) require.NoError(t, tbf(2, c2)) // It is important to check if the record with height 256 does not precede @@ -153,31 +157,32 @@ func TestAscendingOrderOfRecords(t *testing.T) { return nil })) - require.NoError(t, storage.PruneUpToHeight(1)) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) - prunedHeight, err := storage.GetPrunedHeight() + prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) assert.Equal(t, uint64(1), prunedHeight) assert.Len(t, expectedPrunedCIDs, 0) - err = storage.db.View(func(txn *badger.Txn) error { + var latestHeight uint64 + err = executionDataTracker.db.View(func(txn *badger.Txn) error { // expected that blob record with height 1 was removed - _, err := txn.Get(tracker.MakeBlobRecordKey(1, c1)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) - _, err = txn.Get(tracker.MakeLatestHeightKey(c1)) - assert.ErrorIs(t, err, badger.ErrKeyNotFound) + err = operation.RetrieveBlob(1, c1)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) + assert.ErrorIs(t, err, storage.ErrNotFound) // expected that blob record with height 2 exists - _, err = txn.Get(tracker.MakeBlobRecordKey(2, c2)) + err = operation.RetrieveBlob(2, c2)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c2)) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) assert.NoError(t, err) // expected that blob record with height 256 exists - _, err = txn.Get(tracker.MakeBlobRecordKey(256, c3)) + err = operation.RetrieveBlob(256, c3)(txn) assert.NoError(t, err) - _, err = txn.Get(tracker.MakeLatestHeightKey(c3)) + err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(txn) assert.NoError(t, err) return nil diff --git a/storage/badger/operation/common.go b/storage/badger/operation/common.go index 1c293348231..4bd5a50b209 100644 --- a/storage/badger/operation/common.go +++ b/storage/badger/operation/common.go @@ -255,6 +255,9 @@ func retrieve(key []byte, entity interface{}) func(*badger.Txn) error { // get the value from the item err = item.Value(func(val []byte) error { + if entity == nil { + return nil + } err := msgpack.Unmarshal(val, entity) return err }) diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go new file mode 100644 index 00000000000..769f3e3ca31 --- /dev/null +++ b/storage/badger/operation/execution_data_tracker.go @@ -0,0 +1,56 @@ +package operation + +import ( + "github.com/dgraph-io/badger/v2" + "github.com/ipfs/go-cid" + + "github.com/onflow/flow-go/module/executiondatasync/tracker" +) + +func UpdateTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { + return update(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func InsertTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func RetrieveTrackerFulfilledHeight(height *uint64) func(*badger.Txn) error { + return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func UpdateTrackerPrunedHeight(height uint64) func(*badger.Txn) error { + return update(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func InsertTrackerPrunedHeight(height uint64) func(*badger.Txn) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func RetrieveTrackerPrunedHeight(height *uint64) func(*badger.Txn) error { + return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(*badger.Txn) error { + return upsert(makePrefix(tracker.PrefixLatestHeight, cid), height) +} + +func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(*badger.Txn) error { + return retrieve(makePrefix(tracker.PrefixLatestHeight, cid), height) +} + +func RemoveTrackerLatestHeight(cid cid.Cid) func(*badger.Txn) error { + return remove(makePrefix(tracker.PrefixLatestHeight, cid)) +} + +func InsertBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { + return insert(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) +} + +func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { + return retrieve(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) +} + +func RemoveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { + return remove(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid)) +} diff --git a/storage/badger/operation/prefix.go b/storage/badger/operation/prefix.go index ad909faf394..35af7b1f994 100644 --- a/storage/badger/operation/prefix.go +++ b/storage/badger/operation/prefix.go @@ -4,6 +4,8 @@ import ( "encoding/binary" "fmt" + "github.com/ipfs/go-cid" + "github.com/onflow/flow-go/model/flow" ) @@ -141,6 +143,8 @@ func b(v interface{}) []byte { return i[:] case flow.ChainID: return []byte(i) + case cid.Cid: + return i.Bytes() default: panic(fmt.Sprintf("unsupported type to convert (%T)", v)) } diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go new file mode 100644 index 00000000000..e69de29bb2d From 6f2460241656692a8946e3408aa603a83b8103e9 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Mon, 29 Jul 2024 22:58:52 +0300 Subject: [PATCH 03/33] Added pebble operations for execution data tracker, added basic pebble execution data tracker impl, refactored badger impl --- module/executiondatasync/tracker/storage.go | 6 + storage/badger/execution_data_tracker.go | 29 +- storage/pebble/execution_data_tracker.go | 329 ++++++++++++++++++ storage/pebble/operation/common.go | 21 +- .../operation/execution_data_tracker.go | 56 +++ storage/pebble/operation/prefix.go | 4 + 6 files changed, 426 insertions(+), 19 deletions(-) diff --git a/module/executiondatasync/tracker/storage.go b/module/executiondatasync/tracker/storage.go index dfa19822082..c1ee1d07764 100644 --- a/module/executiondatasync/tracker/storage.go +++ b/module/executiondatasync/tracker/storage.go @@ -86,3 +86,9 @@ type Storage interface { // called with a value higher than the fulfilled height. PruneUpToHeight(height uint64) error } + +type DeleteInfo struct { + Cid cid.Cid + Height uint64 + DeleteLatestHeightRecord bool +} diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index df8e617bc15..5a45c607e39 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -219,17 +219,17 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e return nil } -func (s *ExecutionDataTracker) batchDelete(deleteInfos []*deleteInfo) error { +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) error { for _, dInfo := range deleteInfos { - err := s.db.Update(operation.RemoveBlob(dInfo.height, dInfo.cid)) + err := s.db.Update(operation.RemoveBlob(dInfo.Height, dInfo.Cid)) if err != nil { - return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.cid.String(), err) + return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.Cid.String(), err) } - if dInfo.deleteLatestHeightRecord { - err = s.db.Update(operation.RemoveTrackerLatestHeight(dInfo.cid)) + if dInfo.DeleteLatestHeightRecord { + err = s.db.Update(operation.RemoveTrackerLatestHeight(dInfo.Cid)) if err != nil { - return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.cid.String(), err) + return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.Cid.String(), err) } } } @@ -249,7 +249,7 @@ func (s *ExecutionDataTracker) batchDeleteItemLimit() int { func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{tracker.PrefixBlobRecord} itemsPerBatch := s.batchDeleteItemLimit() - var batch []*deleteInfo + var batch []*tracker.DeleteInfo s.mu.Lock() defer s.mu.Unlock() @@ -258,6 +258,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { return err } + // TODO: move to separate 'iterate' function if err := s.db.View(func(txn *badger.Txn) error { it := txn.NewIterator(badger.IteratorOptions{ PrefetchValues: false, @@ -281,9 +282,9 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { break } - dInfo := &deleteInfo{ - cid: blobCid, - height: blockHeight, + dInfo := &tracker.DeleteInfo{ + Cid: blobCid, + Height: blockHeight, } var latestHeight uint64 @@ -307,7 +308,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { if err := s.pruneCallback(blobCid); err != nil { return err } - dInfo.deleteLatestHeightRecord = true + dInfo.DeleteLatestHeightRecord = true } // remove tracker records for pruned heights @@ -358,9 +359,3 @@ func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { return prunedHeight, nil } - -type deleteInfo struct { - cid cid.Cid - height uint64 - deleteLatestHeightRecord bool -} diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index c86d0bc9ec3..a2dfdda0cfd 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -1 +1,330 @@ package pebble + +import ( + "bytes" + "errors" + "fmt" + "sync" + + "github.com/cockroachdb/pebble" + "github.com/hashicorp/go-multierror" + "github.com/ipfs/go-cid" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/pebble/operation" +) + +type StorageOption func(*ExecutionDataTracker) + +var _ tracker.Storage = (*ExecutionDataTracker)(nil) + +// The ExecutionDataTracker component tracks the following information: +// - the latest pruned height +// - the latest fulfilled height +// - the set of CIDs of the execution data blobs we know about at each height, so that +// once we prune a fulfilled height we can remove the blob data from local storage +// - for each CID, the most recent height that it was observed at, so that when pruning +// a fulfilled height we don't remove any blob data that is still needed at higher heights +// +// The storage component calls the given prune callback for a CID when the last height +// at which that CID appears is pruned. The prune callback can be used to delete the +// corresponding blob data from the blob store. +type ExecutionDataTracker struct { + // ensures that pruning operations are not run concurrently with any other db writes + // we acquire the read lock when we want to perform a non-prune WRITE + // we acquire the write lock when we want to perform a prune WRITE + mu sync.RWMutex + + db *pebble.DB + pruneCallback tracker.PruneCallback + logger zerolog.Logger +} + +func WithPruneCallback(callback tracker.PruneCallback) StorageOption { + return func(s *ExecutionDataTracker) { + s.pruneCallback = callback + } +} + +func NewExecutionDataTracker(path string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*ExecutionDataTracker, error) { + lg := logger.With().Str("module", "tracker_storage").Logger() + db, err := pebble.Open(path, nil) + if err != nil { + return nil, fmt.Errorf("could not open db: %w", err) + } + + storage := &ExecutionDataTracker{ + db: db, + pruneCallback: func(c cid.Cid) error { return nil }, + logger: lg, + } + + for _, opt := range opts { + opt(storage) + } + + lg.Info().Msgf("initialize storage with start height: %d", startHeight) + + if err := storage.init(startHeight); err != nil { + return nil, fmt.Errorf("failed to initialize storage: %w", err) + } + + lg.Info().Msgf("storage initialized") + + return storage, nil +} + +func (s *ExecutionDataTracker) init(startHeight uint64) error { + fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() + prunedHeight, prunedHeightErr := s.GetPrunedHeight() + + if fulfilledHeightErr == nil && prunedHeightErr == nil { + if prunedHeight > fulfilledHeight { + return fmt.Errorf( + "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", + prunedHeight, + fulfilledHeight, + ) + } + + s.logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) + // replay pruning in case it was interrupted during previous shutdown + if err := s.PruneUpToHeight(prunedHeight); err != nil { + return fmt.Errorf("failed to replay pruning: %w", err) + } + s.logger.Info().Msgf("finished pruning") + } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { + // db is empty, we need to bootstrap it + if err := s.bootstrap(startHeight); err != nil { + return fmt.Errorf("failed to bootstrap storage: %w", err) + } + } else { + return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() + } + + return nil +} + +func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { + err := operation.InsertTrackerFulfilledHeight(startHeight)(s.db) + if err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + + err = operation.InsertTrackerPrunedHeight(startHeight)(s.db) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil +} + +func (s *ExecutionDataTracker) Update(f tracker.UpdateFn) error { + s.mu.RLock() + defer s.mu.RUnlock() + return f(s.trackBlobs) +} + +func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { + err := operation.UpdateTrackerFulfilledHeight(height)(s.db) + if err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + + return nil +} + +func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { + var fulfilledHeight uint64 + + err := operation.RetrieveTrackerFulfilledHeight(&fulfilledHeight)(s.db) + if err != nil { + return 0, err + } + + return fulfilledHeight, nil +} + +func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { + err := operation.InsertBlob(blockHeight, c)(s.db) + if err != nil { + return fmt.Errorf("failed to add blob record: %w", err) + } + + var latestHeight uint64 + err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(s.db) + if err != nil { + if !errors.Is(err, storage.ErrNotFound) { + return fmt.Errorf("failed to get latest height: %w", err) + } + } else { + // don't update the latest height if there is already a higher block height containing this blob + if latestHeight >= blockHeight { + return nil + } + } + + err = operation.UpsertTrackerLatestHeight(c, blockHeight)(s.db) + if err != nil { + return fmt.Errorf("failed to set latest height value: %w", err) + } + + return nil +} + +func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { + cidsPerBatch := tracker.CidsPerBatch + + for len(cids) > 0 { + batchSize := cidsPerBatch + if len(cids) < batchSize { + batchSize = len(cids) + } + batch := cids[:batchSize] + + for _, c := range batch { + if err := s.trackBlob(blockHeight, c); err != nil { + return fmt.Errorf("failed to track blob %s: %w", c.String(), err) + } + } + + cids = cids[batchSize:] + } + + return nil +} + +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) error { + for _, dInfo := range deleteInfos { + err := operation.RemoveBlob(dInfo.Height, dInfo.Cid)(s.db) + if err != nil { + return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.Cid.String(), err) + } + + if dInfo.DeleteLatestHeightRecord { + err = operation.RemoveTrackerLatestHeight(dInfo.Cid)(s.db) + if err != nil { + return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.Cid.String(), err) + } + } + } + + return nil +} + +var ffBytes = bytes.Repeat([]byte{0xFF}, 32) + +func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { + blobRecordPrefix := []byte{tracker.PrefixBlobRecord} + itemsPerBatch := 256 + var batch []*tracker.DeleteInfo + + s.mu.Lock() + defer s.mu.Unlock() + + if err := s.setPrunedHeight(height); err != nil { + return err + } + + // TODO: move to separate 'iterate' function and move common logic + err := func(tx pebble.Reader) error { + options := pebble.IterOptions{ + LowerBound: blobRecordPrefix, + UpperBound: append(blobRecordPrefix, ffBytes...), + } + + it, err := tx.NewIter(&options) + if err != nil { + return fmt.Errorf("can not create iterator: %w", err) + } + defer it.Close() + + // iterate over blob records, calling pruneCallback for any CIDs that should be pruned + // and cleaning up the corresponding tracker records + for it.SeekGE(blobRecordPrefix); it.Valid(); it.Next() { + blobRecordKey := it.Key() + + blockHeight, blobCid, err := tracker.ParseBlobRecordKey(blobRecordKey) + if err != nil { + return fmt.Errorf("malformed blob record key %v: %w", blobRecordKey, err) + } + + // iteration occurs in key order, so block heights are guaranteed to be ascending + if blockHeight > height { + break + } + + dInfo := &tracker.DeleteInfo{ + Cid: blobCid, + Height: blockHeight, + } + + var latestHeight uint64 + err = operation.RetrieveTrackerLatestHeight(blobCid, &latestHeight)(s.db) + if err != nil { + return fmt.Errorf("failed to get latest height entry for Cid %s: %w", blobCid.String(), err) + } + + // a blob is only removable if it is not referenced by any blob tree at a higher height + if latestHeight < blockHeight { + // this should never happen + return fmt.Errorf( + "inconsistency detected: latest height recorded for Cid %s is %d, but blob record exists at height %d", + blobCid.String(), latestHeight, blockHeight, + ) + } + + // the current block height is the last to reference this CID, prune the CID and remove + // all tracker records + if latestHeight == blockHeight { + if err := s.pruneCallback(blobCid); err != nil { + return err + } + dInfo.DeleteLatestHeightRecord = true + } + + // remove tracker records for pruned heights + batch = append(batch, dInfo) + if len(batch) == itemsPerBatch { + if err := s.batchDelete(batch); err != nil { + return err + } + batch = nil + } + } + + if len(batch) > 0 { + if err := s.batchDelete(batch); err != nil { + return err + } + } + + return nil + }(s.db) + if err != nil { + return err + } + return nil +} + +func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { + err := operation.UpdateTrackerPrunedHeight(height)(s.db) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil +} + +func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { + var prunedHeight uint64 + + err := operation.RetrieveTrackerPrunedHeight(&prunedHeight)(s.db) + if err != nil { + return 0, err + } + + return prunedHeight, nil +} diff --git a/storage/pebble/operation/common.go b/storage/pebble/operation/common.go index ad9e96c2c8b..7e5f0a847d6 100644 --- a/storage/pebble/operation/common.go +++ b/storage/pebble/operation/common.go @@ -17,7 +17,7 @@ func insert(key []byte, val interface{}) func(pebble.Writer) error { return irrecoverable.NewExceptionf("failed to encode value: %w", err) } - err = w.Set(key, value, nil) + err = w.Set(key, value, pebble.Sync) if err != nil { return irrecoverable.NewExceptionf("failed to store data: %w", err) } @@ -34,7 +34,10 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } defer closer.Close() - err = msgpack.Unmarshal(val, &sc) + if sc == nil { + return nil + } + err = msgpack.Unmarshal(val, sc) if err != nil { return irrecoverable.NewExceptionf("failed to decode value: %w", err) } @@ -42,6 +45,20 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } } +// remove removes the entity with the given key, if it exists. If it doesn't +// exist, this is a no-op. +// Error returns: +// * generic error in case of unexpected database error +func remove(key []byte) func(pebble.Writer) error { + return func(w pebble.Writer) error { + err := w.Delete(key, nil) + if err != nil { + return irrecoverable.NewExceptionf("could not delete item: %w", err) + } + return nil + } +} + func convertNotFoundError(err error) error { if errors.Is(err, pebble.ErrNotFound) { return storage.ErrNotFound diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index e69de29bb2d..36e591f5113 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -0,0 +1,56 @@ +package operation + +import ( + "github.com/cockroachdb/pebble" + "github.com/ipfs/go-cid" + + "github.com/onflow/flow-go/module/executiondatasync/tracker" +) + +func UpdateTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func InsertTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func RetrieveTrackerFulfilledHeight(height *uint64) func(r pebble.Reader) error { + return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) +} + +func UpdateTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func InsertTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func RetrieveTrackerPrunedHeight(height *uint64) func(r pebble.Reader) error { + return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) +} + +func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixLatestHeight, cid), height) +} + +func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(r pebble.Reader) error { + return retrieve(makePrefix(tracker.PrefixLatestHeight, cid), height) +} + +func RemoveTrackerLatestHeight(cid cid.Cid) func(w pebble.Writer) error { + return remove(makePrefix(tracker.PrefixLatestHeight, cid)) +} + +func InsertBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { + return insert(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) +} + +func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(r pebble.Reader) error { + return retrieve(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) +} + +func RemoveBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { + return remove(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid)) +} diff --git a/storage/pebble/operation/prefix.go b/storage/pebble/operation/prefix.go index ef7cdf3c08d..112a980122f 100644 --- a/storage/pebble/operation/prefix.go +++ b/storage/pebble/operation/prefix.go @@ -4,6 +4,8 @@ import ( "encoding/binary" "fmt" + "github.com/ipfs/go-cid" + "github.com/onflow/flow-go/model/flow" ) @@ -45,6 +47,8 @@ func b(v interface{}) []byte { return i[:] case flow.ChainID: return []byte(i) + case cid.Cid: + return i.Bytes() default: panic(fmt.Sprintf("unsupported type to convert (%T)", v)) } From cb61531833428320e11037f17e10db50bd3d3ae4 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Mon, 29 Jul 2024 23:00:59 +0300 Subject: [PATCH 04/33] Added functional test for pebble execution data tracker impl, updated test for badger impl --- storage/badger/execution_data_tracker_test.go | 10 +- storage/pebble/execution_data_tracker_test.go | 177 ++++++++++++++++++ 2 files changed, 182 insertions(+), 5 deletions(-) create mode 100644 storage/pebble/execution_data_tracker_test.go diff --git a/storage/badger/execution_data_tracker_test.go b/storage/badger/execution_data_tracker_test.go index 88fbc75409d..0579fb46cc0 100644 --- a/storage/badger/execution_data_tracker_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -87,7 +87,7 @@ func TestPrune(t *testing.T) { // if that CID also exists at another height above the pruned height, the CID should not be pruned. func TestPruneNonLatestHeight(t *testing.T) { storageDir := t.TempDir() - storage, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { assert.Fail(t, "unexpected CID pruned: %s", c.String()) return nil })) @@ -98,20 +98,20 @@ func TestPruneNonLatestHeight(t *testing.T) { c1 := randomCid() c2 := randomCid() - require.NoError(t, storage.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c1, c2)) return nil })) - require.NoError(t, storage.PruneUpToHeight(1)) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) - prunedHeight, err := storage.GetPrunedHeight() + prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) assert.Equal(t, uint64(1), prunedHeight) var latestHeight uint64 - err = storage.db.View(func(txn *badger.Txn) error { + err = executionDataTracker.db.View(func(txn *badger.Txn) error { err = operation.RetrieveBlob(2, c1)(txn) assert.NoError(t, err) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) diff --git a/storage/pebble/execution_data_tracker_test.go b/storage/pebble/execution_data_tracker_test.go new file mode 100644 index 00000000000..26baf6d98e4 --- /dev/null +++ b/storage/pebble/execution_data_tracker_test.go @@ -0,0 +1,177 @@ +package pebble + +import ( + "crypto/rand" + "testing" + + "github.com/ipfs/go-cid" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/module/blobs" + "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/pebble/operation" +) + +func randomCid() cid.Cid { + data := make([]byte, 1024) + _, _ = rand.Read(data) + return blobs.NewBlob(data).Cid() +} + +// TestPrune tests that when a height is pruned, all CIDs appearing at or below the pruned +// height, and their associated tracking data, should be removed from the database. +func TestPrune(t *testing.T) { + expectedPrunedCIDs := make(map[cid.Cid]struct{}) + storageDir := t.TempDir() + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) + require.NoError(t, err) + + // c1 and c2 are for height 1, and c3 and c4 are for height 2 + // after pruning up to height 1, only c1 and c2 should be pruned + c1 := randomCid() + expectedPrunedCIDs[c1] = struct{}{} + c2 := randomCid() + expectedPrunedCIDs[c2] = struct{}{} + c3 := randomCid() + c4 := randomCid() + + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, tbf(1, c1, c2)) + require.NoError(t, tbf(2, c3, c4)) + + return nil + })) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) + + prunedHeight, err := executionDataTracker.GetPrunedHeight() + require.NoError(t, err) + assert.Equal(t, uint64(1), prunedHeight) + + assert.Len(t, expectedPrunedCIDs, 0) + + var latestHeight uint64 + + err = operation.RetrieveBlob(1, c1)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveBlob(1, c2)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + + err = operation.RetrieveBlob(2, c3)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveBlob(2, c4)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c4, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) +} + +// TestPruneNonLatestHeight test that when pruning a height at which a CID exists, +// if that CID also exists at another height above the pruned height, the CID should not be pruned. +func TestPruneNonLatestHeight(t *testing.T) { + storageDir := t.TempDir() + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + assert.Fail(t, "unexpected CID pruned: %s", c.String()) + return nil + })) + require.NoError(t, err) + + // c1 and c2 appear both at height 1 and 2 + // therefore, when pruning up to height 1, both c1 and c2 should be retained + c1 := randomCid() + c2 := randomCid() + + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, tbf(1, c1, c2)) + require.NoError(t, tbf(2, c1, c2)) + + return nil + })) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) + + prunedHeight, err := executionDataTracker.GetPrunedHeight() + require.NoError(t, err) + assert.Equal(t, uint64(1), prunedHeight) + + var latestHeight uint64 + + err = operation.RetrieveBlob(2, c1)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveBlob(2, c2)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) +} + +// TestAscendingOrderOfRecords tests that order of data is ascending and all CIDs appearing at or below the pruned +// height, and their associated tracking data, should be removed from the database. +func TestAscendingOrderOfRecords(t *testing.T) { + expectedPrunedCIDs := make(map[cid.Cid]struct{}) + storageDir := t.TempDir() + executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) + require.NoError(t, err) + + // c1 is for height 1, + // c2 is for height 2, + // c3 is for height 256 + // pruning up to height 1 will check if order of the records is ascending, c1 should be pruned + c1 := randomCid() + expectedPrunedCIDs[c1] = struct{}{} + c2 := randomCid() + c3 := randomCid() + + require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, tbf(1, c1)) + require.NoError(t, tbf(2, c2)) + // It is important to check if the record with height 256 does not precede + // the record with height 1 during pruning. + require.NoError(t, tbf(256, c3)) + + return nil + })) + require.NoError(t, executionDataTracker.PruneUpToHeight(1)) + + prunedHeight, err := executionDataTracker.GetPrunedHeight() + require.NoError(t, err) + assert.Equal(t, uint64(1), prunedHeight) + + assert.Len(t, expectedPrunedCIDs, 0) + + var latestHeight uint64 + // expected that blob record with height 1 was removed + err = operation.RetrieveBlob(1, c1)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) + assert.ErrorIs(t, err, storage.ErrNotFound) + + // expected that blob record with height 2 exists + err = operation.RetrieveBlob(2, c2)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) + + // expected that blob record with height 256 exists + err = operation.RetrieveBlob(256, c3)(executionDataTracker.db) + assert.NoError(t, err) + err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(executionDataTracker.db) + assert.NoError(t, err) +} From e2630cd3ea647663c3b8438a4c8b14eaa45f643f Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Mon, 29 Jul 2024 23:01:28 +0300 Subject: [PATCH 05/33] Updated AN and ON builders --- .../node_builder/access_node_builder.go | 30 +++++++++++++------ cmd/observer/node_builder/observer_builder.go | 30 +++++++++++++------ 2 files changed, 42 insertions(+), 18 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 264657ce8d7..db5bbb60b01 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -666,15 +666,27 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( - trackerDir, - sealed.Height, - node.Logger, - bstorage.WithPruneCallback(func(c cid.Cid) error { - // TODO: use a proper context here - return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) - }), - ) + if executionDataDBMode == execution_data.ExecutionDataDBModeBadger { + builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( + trackerDir, + sealed.Height, + node.Logger, + bstorage.WithPruneCallback(func(c cid.Cid) error { + // TODO: use a proper context here + return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) + }), + ) + } else { + builder.ExecutionDataTracker, err = pstorage.NewExecutionDataTracker( + trackerDir, + sealed.Height, + node.Logger, + pstorage.WithPruneCallback(func(c cid.Cid) error { + // TODO: use a proper context here + return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) + }), + ) + } if err != nil { return nil, fmt.Errorf("failed to create execution data tracker: %w", err) } diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index c83c96c5989..a1c8c275245 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1231,15 +1231,27 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS } trackerDir := filepath.Join(builder.executionDataDir, "tracker") - builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( - trackerDir, - sealed.Height, - node.Logger, - bstorage.WithPruneCallback(func(c cid.Cid) error { - // TODO: use a proper context here - return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) - }), - ) + if executionDataDBMode == execution_data.ExecutionDataDBModeBadger { + builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( + trackerDir, + sealed.Height, + node.Logger, + bstorage.WithPruneCallback(func(c cid.Cid) error { + // TODO: use a proper context here + return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) + }), + ) + } else { + builder.ExecutionDataTracker, err = pstorage.NewExecutionDataTracker( + trackerDir, + sealed.Height, + node.Logger, + pstorage.WithPruneCallback(func(c cid.Cid) error { + // TODO: use a proper context here + return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) + }), + ) + } if err != nil { return nil, fmt.Errorf("failed to create execution data tracker: %w", err) } From fd141605c42332fa049dee8e2b7339e08af89164 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Mon, 29 Jul 2024 23:20:44 +0300 Subject: [PATCH 06/33] Added comments --- storage/pebble/execution_data_tracker.go | 1 + 1 file changed, 1 insertion(+) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index a2dfdda0cfd..0bcf871bc68 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -76,6 +76,7 @@ func NewExecutionDataTracker(path string, startHeight uint64, logger zerolog.Log return storage, nil } +// TODO: move common logic into separate function to avoid duplication of code func (s *ExecutionDataTracker) init(startHeight uint64) error { fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() prunedHeight, prunedHeightErr := s.GetPrunedHeight() From 4d675d7a5a90e2a3308040ecf4a61c0b3f07cd42 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 30 Jul 2024 17:52:59 +0300 Subject: [PATCH 07/33] Moved interface from tracker to storage --- Makefile | 1 - .../node_builder/access_node_builder.go | 3 +- cmd/execution_builder.go | 35 +++++++++---------- cmd/observer/node_builder/observer_builder.go | 3 +- .../computation/computer/computer_test.go | 2 +- .../execution_verification_test.go | 2 +- .../computation/manager_benchmark_test.go | 4 +-- engine/execution/computation/manager_test.go | 2 +- engine/execution/computation/programs_test.go | 2 +- engine/testutil/nodes.go | 2 +- engine/verification/utils/unittest/fixture.go | 2 +- fvm/fvm_bench_test.go | 2 +- .../execution_data/downloader.go | 7 ++-- module/executiondatasync/provider/provider.go | 11 +++--- .../provider/provider_test.go | 2 +- module/executiondatasync/pruner/pruner.go | 6 ++-- .../executiondatasync/pruner/pruner_test.go | 8 ++--- module/executiondatasync/tracker/mock/util.go | 19 ---------- storage/badger/execution_data_tracker.go | 25 +++++++------ storage/badger/execution_data_tracker_test.go | 7 ++-- .../operation/execution_data_tracker.go | 26 +++++++------- .../execution_data_tracker.go | 18 +++++----- .../mock/execution_data_tracker.go | 28 +++++++-------- storage/mocks/util.go | 20 +++++++++++ storage/pebble/execution_data_tracker.go | 21 ++++++----- storage/pebble/execution_data_tracker_test.go | 7 ++-- .../operation/execution_data_tracker.go | 26 +++++++------- .../tracker => storage/util}/noop.go | 12 ++++--- 28 files changed, 150 insertions(+), 153 deletions(-) delete mode 100644 module/executiondatasync/tracker/mock/util.go rename module/executiondatasync/tracker/storage.go => storage/execution_data_tracker.go (87%) rename module/executiondatasync/tracker/mock/storage.go => storage/mock/execution_data_tracker.go (70%) create mode 100644 storage/mocks/util.go rename {module/executiondatasync/tracker => storage/util}/noop.go (66%) diff --git a/Makefile b/Makefile index 19bd4da9990..f57d893e82f 100644 --- a/Makefile +++ b/Makefile @@ -211,7 +211,6 @@ generate-mocks: install-mock-generators mockery --name 'ExecForkActor' --structname 'ExecForkActorMock' --dir=module/mempool/consensus/mock/ --case=underscore --output="./module/mempool/consensus/mock/" --outpkg="mock" mockery --name '.*' --dir=engine/verification/fetcher/ --case=underscore --output="./engine/verification/fetcher/mock" --outpkg="mockfetcher" mockery --name '.*' --dir=./cmd/util/ledger/reporters --case=underscore --output="./cmd/util/ledger/reporters/mock" --outpkg="mock" - mockery --name 'Storage' --dir=module/executiondatasync/tracker --case=underscore --output="module/executiondatasync/tracker/mock" --outpkg="mocktracker" mockery --name 'ScriptExecutor' --dir=module/execution --case=underscore --output="module/execution/mock" --outpkg="mock" mockery --name 'StorageSnapshot' --dir=fvm/storage/snapshot --case=underscore --output="fvm/storage/snapshot/mock" --outpkg="mock" diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5de4141a2f5..a6c4b945f71 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -73,7 +73,6 @@ import ( execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" "github.com/onflow/flow-go/module/executiondatasync/pruner" edstorage "github.com/onflow/flow-go/module/executiondatasync/storage" - "github.com/onflow/flow-go/module/executiondatasync/tracker" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" "github.com/onflow/flow-go/module/grpcserver" "github.com/onflow/flow-go/module/id" @@ -322,7 +321,7 @@ type FlowAccessNodeBuilder struct { collectionExecutedMetric module.CollectionExecutedMetric ExecutionDataPruner *pruner.Pruner ExecutionDatastoreManager edstorage.DatastoreManager - ExecutionDataTracker tracker.Storage + ExecutionDataTracker storage.ExecutionDataTracker versionControl *version.VersionControl // The sync engine participants provider is the libp2p peer store for the access node diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index 4b02d05a28f..efcb8fbe13d 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -78,7 +78,6 @@ import ( "github.com/onflow/flow-go/module/executiondatasync/execution_data" exedataprovider "github.com/onflow/flow-go/module/executiondatasync/provider" "github.com/onflow/flow-go/module/executiondatasync/pruner" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/module/finalizedreader" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" "github.com/onflow/flow-go/module/mempool/queue" @@ -90,8 +89,8 @@ import ( "github.com/onflow/flow-go/state/protocol" badgerState "github.com/onflow/flow-go/state/protocol/badger" "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" + bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/storage/badger/procedure" storagepebble "github.com/onflow/flow-go/storage/pebble" sutil "github.com/onflow/flow-go/storage/util" @@ -133,11 +132,11 @@ type ExecutionNode struct { committee hotstuff.DynamicCommittee ledgerStorage *ledger.Ledger registerStore *storehouse.RegisterStore - events *storage.Events - serviceEvents *storage.ServiceEvents - txResults *storage.TransactionResults - results *storage.ExecutionResults - myReceipts *storage.MyExecutionReceipts + events *bstorage.Events + serviceEvents *bstorage.ServiceEvents + txResults *bstorage.TransactionResults + results *bstorage.ExecutionResults + myReceipts *bstorage.MyExecutionReceipts providerEngine exeprovider.ProviderEngine checkerEng *checker.Engine syncCore *chainsync.Core @@ -157,7 +156,7 @@ type ExecutionNode struct { executionDataDatastore *badgerds.Datastore executionDataPruner *pruner.Pruner executionDataBlobstore blobs.Blobstore - executionDataTracker tracker.Storage + executionDataTracker storage.ExecutionDataTracker blobService network.BlobService blobserviceDependable *module.ProxiedReadyDoneAware } @@ -281,7 +280,7 @@ func (exeNode *ExecutionNode) LoadExecutionMetrics(node *NodeConfig) error { err := node.DB.View(procedure.GetHighestExecutedBlock(&height, &blockID)) if err != nil { // database has not been bootstrapped yet - if errors.Is(err, storageerr.ErrNotFound) { + if errors.Is(err, storage.ErrNotFound) { return nil } return fmt.Errorf("could not get highest executed block: %w", err) @@ -300,8 +299,8 @@ func (exeNode *ExecutionNode) LoadSyncCore(node *NodeConfig) error { func (exeNode *ExecutionNode) LoadExecutionReceiptsStorage( 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)) + exeNode.results = bstorage.NewExecutionResults(node.Metrics.Cache, node.DB) + exeNode.myReceipts = bstorage.NewMyExecutionReceipts(node.Metrics.Cache, node.DB, node.Storage.Receipts.(*bstorage.ExecutionReceipts)) return nil } @@ -440,7 +439,7 @@ func (exeNode *ExecutionNode) LoadGCPBlockDataUploader( exeNode.events, exeNode.results, exeNode.txResults, - storage.NewComputationResultUploadStatus(node.DB), + bstorage.NewComputationResultUploadStatus(node.DB), execution_data.NewDownloader(exeNode.blobService), exeNode.collector) if retryableUploader == nil { @@ -739,9 +738,9 @@ func (exeNode *ExecutionNode) LoadExecutionState( 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) + exeNode.events = bstorage.NewEvents(node.Metrics.Cache, node.DB) + exeNode.serviceEvents = bstorage.NewServiceEvents(node.Metrics.Cache, node.DB) + exeNode.txResults = bstorage.NewTransactionResults(node.Metrics.Cache, node.DB, exeNode.exeConf.transactionResultsCacheSize) exeNode.executionState = state.NewExecutionState( exeNode.ledgerStorage, @@ -946,11 +945,11 @@ func (exeNode *ExecutionNode) LoadExecutionDataPruner( } trackerDir := filepath.Join(exeNode.exeConf.executionDataDir, "tracker") - exeNode.executionDataTracker, err = storage.NewExecutionDataTracker( + exeNode.executionDataTracker, err = bstorage.NewExecutionDataTracker( trackerDir, sealed.Height, node.Logger, - storage.WithPruneCallback(func(c cid.Cid) error { + bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return exeNode.executionDataBlobstore.DeleteBlob(context.TODO(), c) }), diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 2c1b1afa330..6cb4c775d35 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -72,7 +72,6 @@ import ( execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" "github.com/onflow/flow-go/module/executiondatasync/pruner" edstorage "github.com/onflow/flow-go/module/executiondatasync/storage" - "github.com/onflow/flow-go/module/executiondatasync/tracker" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" "github.com/onflow/flow-go/module/grpcserver" "github.com/onflow/flow-go/module/id" @@ -287,7 +286,7 @@ type ObserverServiceBuilder struct { ExecutionDataBlobstore blobs.Blobstore ExecutionDataPruner *pruner.Pruner ExecutionDatastoreManager edstorage.DatastoreManager - ExecutionDataTracker tracker.Storage + ExecutionDataTracker storage.ExecutionDataTracker RegistersAsyncStore *execution.RegistersAsyncStore Reporter *index.Reporter diff --git a/engine/execution/computation/computer/computer_test.go b/engine/execution/computation/computer/computer_test.go index 0bb95f11cf2..1322023dda6 100644 --- a/engine/execution/computation/computer/computer_test.go +++ b/engine/execution/computation/computer/computer_test.go @@ -49,12 +49,12 @@ import ( "github.com/onflow/flow-go/module/epochs" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" modulemock "github.com/onflow/flow-go/module/mock" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/execution/computation/execution_verification_test.go b/engine/execution/computation/execution_verification_test.go index c949b378df4..170e05e2780 100644 --- a/engine/execution/computation/execution_verification_test.go +++ b/engine/execution/computation/execution_verification_test.go @@ -39,10 +39,10 @@ import ( "github.com/onflow/flow-go/module/chunks" "github.com/onflow/flow-go/module/executiondatasync/execution_data" exedataprovider "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/metrics" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/execution/computation/manager_benchmark_test.go b/engine/execution/computation/manager_benchmark_test.go index d22b1ec9a8b..dfdc942ce4c 100644 --- a/engine/execution/computation/manager_benchmark_test.go +++ b/engine/execution/computation/manager_benchmark_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - blockstore "github.com/ipfs/boxo/blockstore" + "github.com/ipfs/boxo/blockstore" "github.com/ipfs/go-datastore" dssync "github.com/ipfs/go-datastore/sync" "github.com/onflow/cadence/runtime" @@ -25,12 +25,12 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" exedataprovider "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/execution/computation/manager_test.go b/engine/execution/computation/manager_test.go index 7aee62d7b40..e3d728ab7de 100644 --- a/engine/execution/computation/manager_test.go +++ b/engine/execution/computation/manager_test.go @@ -39,12 +39,12 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/execution/computation/programs_test.go b/engine/execution/computation/programs_test.go index 75467f4054d..59446a6403e 100644 --- a/engine/execution/computation/programs_test.go +++ b/engine/execution/computation/programs_test.go @@ -26,12 +26,12 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" module "github.com/onflow/flow-go/module/mock" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index b6d1037b500..969027f4f36 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -77,7 +77,6 @@ import ( "github.com/onflow/flow-go/module/compliance" "github.com/onflow/flow-go/module/executiondatasync/execution_data" exedataprovider "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/finalizedreader" confinalizer "github.com/onflow/flow-go/module/finalizer/consensus" "github.com/onflow/flow-go/module/id" @@ -105,6 +104,7 @@ import ( "github.com/onflow/flow-go/state/protocol/events/gadgets" "github.com/onflow/flow-go/state/protocol/util" storage "github.com/onflow/flow-go/storage/badger" + mocktracker "github.com/onflow/flow-go/storage/mocks" storagepebble "github.com/onflow/flow-go/storage/pebble" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/verification/utils/unittest/fixture.go b/engine/verification/utils/unittest/fixture.go index 25cea9d934c..a9ca72a77d6 100644 --- a/engine/verification/utils/unittest/fixture.go +++ b/engine/verification/utils/unittest/fixture.go @@ -25,12 +25,12 @@ import ( "github.com/onflow/flow-go/module/epochs" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/state/cluster" + mocktracker "github.com/onflow/flow-go/storage/mocks" envMock "github.com/onflow/flow-go/fvm/environment/mock" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" moduleMock "github.com/onflow/flow-go/module/mock" diff --git a/fvm/fvm_bench_test.go b/fvm/fvm_bench_test.go index aa1a06b6a6d..389fb2d7fd4 100644 --- a/fvm/fvm_bench_test.go +++ b/fvm/fvm_bench_test.go @@ -48,11 +48,11 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/metrics" moduleMock "github.com/onflow/flow-go/module/mock" requesterunit "github.com/onflow/flow-go/module/state_synchronization/requester/unittest" "github.com/onflow/flow-go/module/trace" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/module/executiondatasync/execution_data/downloader.go b/module/executiondatasync/execution_data/downloader.go index 3a8fa4ece67..45f81154b02 100644 --- a/module/executiondatasync/execution_data/downloader.go +++ b/module/executiondatasync/execution_data/downloader.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/storage" ) @@ -32,7 +31,7 @@ type downloader struct { blobService network.BlobService maxBlobSize int serializer Serializer - storage tracker.Storage + storage storage.ExecutionDataTracker headers storage.Headers } @@ -46,7 +45,7 @@ func WithSerializer(serializer Serializer) DownloaderOption { } // WithExecutionDataTracker configures the execution data tracker and the storage headers for the downloader -func WithExecutionDataTracker(storage tracker.Storage, headers storage.Headers) DownloaderOption { +func WithExecutionDataTracker(storage storage.ExecutionDataTracker, headers storage.Headers) DownloaderOption { return func(d *downloader) { d.storage = storage d.headers = headers @@ -233,7 +232,7 @@ func (d *downloader) trackBlobs(blockID flow.Identifier, cids []cid.Cid) error { return nil } - return d.storage.Update(func(trackBlobs tracker.TrackBlobsFn) error { + return d.storage.Update(func(trackBlobs storage.TrackBlobsFn) error { header, err := d.headers.ByBlockID(blockID) if err != nil { return err diff --git a/module/executiondatasync/provider/provider.go b/module/executiondatasync/provider/provider.go index c67ddb81a76..1b944e4ad1f 100644 --- a/module/executiondatasync/provider/provider.go +++ b/module/executiondatasync/provider/provider.go @@ -15,8 +15,9 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/executiondatasync/execution_data" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/util" ) type ProviderOption func(*ExecutionDataProvider) @@ -37,7 +38,7 @@ type ExecutionDataProvider struct { metrics module.ExecutionDataProviderMetrics maxBlobSize int blobService network.BlobService - storage tracker.Storage + storage storage.ExecutionDataTracker cidsProvider *ExecutionDataCIDProvider } @@ -48,11 +49,11 @@ func NewProvider( metrics module.ExecutionDataProviderMetrics, serializer execution_data.Serializer, blobService network.BlobService, - storage tracker.Storage, + storage storage.ExecutionDataTracker, opts ...ProviderOption, ) *ExecutionDataProvider { if storage == nil { - storage = &tracker.NoopStorage{} + storage = &util.NoopStorage{} } p := &ExecutionDataProvider{ @@ -95,7 +96,7 @@ func (p *ExecutionDataProvider) storeBlobs(parent context.Context, blockHeight u p.logger.Debug().Array("cids", cidArr).Uint64("height", blockHeight).Msg("storing blobs") } - err := p.storage.Update(func(trackBlobs tracker.TrackBlobsFn) error { + err := p.storage.Update(func(trackBlobs storage.TrackBlobsFn) error { ctx, cancel := context.WithCancel(parent) defer cancel() diff --git a/module/executiondatasync/provider/provider_test.go b/module/executiondatasync/provider/provider_test.go index a101ea7e05c..22e1b526477 100644 --- a/module/executiondatasync/provider/provider_test.go +++ b/module/executiondatasync/provider/provider_test.go @@ -18,10 +18,10 @@ import ( "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/executiondatasync/provider" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/mocknetwork" + mocktracker "github.com/onflow/flow-go/storage/mocks" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/module/executiondatasync/pruner/pruner.go b/module/executiondatasync/pruner/pruner.go index 99cbe565d83..f59ebf1e362 100644 --- a/module/executiondatasync/pruner/pruner.go +++ b/module/executiondatasync/pruner/pruner.go @@ -13,8 +13,8 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/executiondatasync/execution_data" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" ) // ErrNoRegisteredHeightRecorders represents an error indicating that pruner did not register any execution data height recorders. @@ -45,7 +45,7 @@ const ( // A height is considered fulfilled once it has both been executed, // tracked, and sealed. type Pruner struct { - storage tracker.Storage + storage storage.ExecutionDataTracker pruneCallback func(ctx context.Context) error lastFulfilledHeight uint64 @@ -105,7 +105,7 @@ func WithPruningInterval(interval time.Duration) PrunerOption { } // NewPruner creates a new Pruner. -func NewPruner(logger zerolog.Logger, metrics module.ExecutionDataPrunerMetrics, storage tracker.Storage, opts ...PrunerOption) (*Pruner, error) { +func NewPruner(logger zerolog.Logger, metrics module.ExecutionDataPrunerMetrics, storage storage.ExecutionDataTracker, opts ...PrunerOption) (*Pruner, error) { lastPrunedHeight, err := storage.GetPrunedHeight() if err != nil { return nil, fmt.Errorf("failed to get pruned height: %w", err) diff --git a/module/executiondatasync/pruner/pruner_test.go b/module/executiondatasync/pruner/pruner_test.go index 42158b3b7fb..ddaedf1e3d8 100644 --- a/module/executiondatasync/pruner/pruner_test.go +++ b/module/executiondatasync/pruner/pruner_test.go @@ -10,14 +10,14 @@ import ( exedatamock "github.com/onflow/flow-go/module/executiondatasync/execution_data/mock" "github.com/onflow/flow-go/module/executiondatasync/pruner" - mocktracker "github.com/onflow/flow-go/module/executiondatasync/tracker/mock" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/metrics" + mockstorage "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" ) func TestBasicPrune(t *testing.T) { - trackerStorage := new(mocktracker.Storage) + trackerStorage := new(mockstorage.ExecutionDataTracker) trackerStorage.On("GetFulfilledHeight").Return(uint64(0), nil).Once() trackerStorage.On("GetPrunedHeight").Return(uint64(0), nil).Once() @@ -65,7 +65,7 @@ func TestBasicPrune(t *testing.T) { } func TestUpdateThreshold(t *testing.T) { - trackerStorage := new(mocktracker.Storage) + trackerStorage := new(mockstorage.ExecutionDataTracker) trackerStorage.On("GetFulfilledHeight").Return(uint64(15), nil).Once() trackerStorage.On("GetPrunedHeight").Return(uint64(0), nil).Once() @@ -106,7 +106,7 @@ func TestUpdateThreshold(t *testing.T) { } func TestUpdateHeightRangeTarget(t *testing.T) { - trackerStorage := new(mocktracker.Storage) + trackerStorage := new(mockstorage.ExecutionDataTracker) trackerStorage.On("GetFulfilledHeight").Return(uint64(10), nil).Once() trackerStorage.On("GetPrunedHeight").Return(uint64(0), nil).Once() diff --git a/module/executiondatasync/tracker/mock/util.go b/module/executiondatasync/tracker/mock/util.go deleted file mode 100644 index 485ed1f13ee..00000000000 --- a/module/executiondatasync/tracker/mock/util.go +++ /dev/null @@ -1,19 +0,0 @@ -package mocktracker - -import ( - "github.com/ipfs/go-cid" - mock "github.com/stretchr/testify/mock" - - tracker "github.com/onflow/flow-go/module/executiondatasync/tracker" -) - -func NewMockStorage() *Storage { - trackerStorage := new(Storage) - trackerStorage.On("Update", mock.Anything).Return(func(fn tracker.UpdateFn) error { - return fn(func(uint64, ...cid.Cid) error { return nil }) - }) - - trackerStorage.On("SetFulfilledHeight", mock.Anything).Return(nil) - - return trackerStorage -} diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 5a45c607e39..4f19192c4bc 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -10,7 +10,6 @@ import ( "github.com/ipfs/go-cid" "github.com/rs/zerolog" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/badger/operation" ) @@ -31,7 +30,7 @@ func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePer type StorageOption func(*ExecutionDataTracker) -var _ tracker.Storage = (*ExecutionDataTracker)(nil) +var _ storage.ExecutionDataTracker = (*ExecutionDataTracker)(nil) // The ExecutionDataTracker component tracks the following information: // - the latest pruned height @@ -51,11 +50,11 @@ type ExecutionDataTracker struct { mu sync.RWMutex db *badger.DB - pruneCallback tracker.PruneCallback + pruneCallback storage.PruneCallback logger zerolog.Logger } -func WithPruneCallback(callback tracker.PruneCallback) StorageOption { +func WithPruneCallback(callback storage.PruneCallback) StorageOption { return func(s *ExecutionDataTracker) { s.pruneCallback = callback } @@ -134,7 +133,7 @@ func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { return nil } -func (s *ExecutionDataTracker) Update(f tracker.UpdateFn) error { +func (s *ExecutionDataTracker) Update(f storage.UpdateFn) error { s.mu.RLock() defer s.mu.RUnlock() return f(s.trackBlobs) @@ -188,8 +187,8 @@ func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c c } func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - cidsPerBatch := tracker.CidsPerBatch - maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength+8) + cidsPerBatch := storage.CidsPerBatch + maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength+8) if maxCidsPerBatch < cidsPerBatch { cidsPerBatch = maxCidsPerBatch } @@ -219,7 +218,7 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e return nil } -func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) error { +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { for _, dInfo := range deleteInfos { err := s.db.Update(operation.RemoveBlob(dInfo.Height, dInfo.Cid)) if err != nil { @@ -239,7 +238,7 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) er func (s *ExecutionDataTracker) batchDeleteItemLimit() int { itemsPerBatch := 256 - maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, tracker.BlobRecordKeyLength+tracker.LatestHeightKeyLength) + maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength) if maxItemsPerBatch < itemsPerBatch { itemsPerBatch = maxItemsPerBatch } @@ -247,9 +246,9 @@ func (s *ExecutionDataTracker) batchDeleteItemLimit() int { } func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { - blobRecordPrefix := []byte{tracker.PrefixBlobRecord} + blobRecordPrefix := []byte{storage.PrefixBlobRecord} itemsPerBatch := s.batchDeleteItemLimit() - var batch []*tracker.DeleteInfo + var batch []*storage.DeleteInfo s.mu.Lock() defer s.mu.Unlock() @@ -272,7 +271,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordItem := it.Item() blobRecordKey := blobRecordItem.Key() - blockHeight, blobCid, err := tracker.ParseBlobRecordKey(blobRecordKey) + blockHeight, blobCid, err := storage.ParseBlobRecordKey(blobRecordKey) if err != nil { return fmt.Errorf("malformed blob record key %v: %w", blobRecordKey, err) } @@ -282,7 +281,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { break } - dInfo := &tracker.DeleteInfo{ + dInfo := &storage.DeleteInfo{ Cid: blobCid, Height: blockHeight, } diff --git a/storage/badger/execution_data_tracker_test.go b/storage/badger/execution_data_tracker_test.go index 0579fb46cc0..6aba559a184 100644 --- a/storage/badger/execution_data_tracker_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/module/blobs" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/badger/operation" ) @@ -44,7 +43,7 @@ func TestPrune(t *testing.T) { c3 := randomCid() c4 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c3, c4)) @@ -98,7 +97,7 @@ func TestPruneNonLatestHeight(t *testing.T) { c1 := randomCid() c2 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c1, c2)) @@ -148,7 +147,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { c2 := randomCid() c3 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1)) require.NoError(t, tbf(2, c2)) // It is important to check if the record with height 256 does not precede diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index 769f3e3ca31..768975f0379 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -4,53 +4,53 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/ipfs/go-cid" - "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" ) func UpdateTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { - return update(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func InsertTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func RetrieveTrackerFulfilledHeight(height *uint64) func(*badger.Txn) error { - return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func UpdateTrackerPrunedHeight(height uint64) func(*badger.Txn) error { - return update(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func InsertTrackerPrunedHeight(height uint64) func(*badger.Txn) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func RetrieveTrackerPrunedHeight(height *uint64) func(*badger.Txn) error { - return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(*badger.Txn) error { - return upsert(makePrefix(tracker.PrefixLatestHeight, cid), height) + return upsert(makePrefix(storage.PrefixLatestHeight, cid), height) } func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(*badger.Txn) error { - return retrieve(makePrefix(tracker.PrefixLatestHeight, cid), height) + return retrieve(makePrefix(storage.PrefixLatestHeight, cid), height) } func RemoveTrackerLatestHeight(cid cid.Cid) func(*badger.Txn) error { - return remove(makePrefix(tracker.PrefixLatestHeight, cid)) + return remove(makePrefix(storage.PrefixLatestHeight, cid)) } func InsertBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { - return insert(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) + return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { - return retrieve(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) + return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } func RemoveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { - return remove(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid)) + return remove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) } diff --git a/module/executiondatasync/tracker/storage.go b/storage/execution_data_tracker.go similarity index 87% rename from module/executiondatasync/tracker/storage.go rename to storage/execution_data_tracker.go index c1ee1d07764..e56c934819a 100644 --- a/module/executiondatasync/tracker/storage.go +++ b/storage/execution_data_tracker.go @@ -1,4 +1,4 @@ -package tracker +package storage import ( "encoding/binary" @@ -30,28 +30,28 @@ func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { return blockHeight, c, err } -// TrackBlobsFn is passed to the UpdateFn provided to Storage.Update, +// TrackBlobsFn is passed to the UpdateFn provided to ExecutionDataTracker.Update, // and can be called to track a list of cids at a given block height. // It returns an error if the update failed. type TrackBlobsFn func(blockHeight uint64, cids ...cid.Cid) error -// UpdateFn is implemented by the user and passed to Storage.Update, +// UpdateFn is implemented by the user and passed to ExecutionDataTracker.Update, // which ensures that it will never be run concurrently with any call -// to Storage.Prune. -// Any returned error will be returned from the surrounding call to Storage.Update. -// The function must never make any calls to the Storage interface itself, +// to ExecutionDataTracker.Prune. +// Any returned error will be returned from the surrounding call to ExecutionDataTracker.Update. +// The function must never make any calls to the ExecutionDataTracker interface itself, // and should instead only modify the storage via the provided TrackBlobsFn. type UpdateFn func(TrackBlobsFn) error // PruneCallback is a function which can be provided by the user which // is called for each CID when the last height at which that CID appears // is pruned. -// Any returned error will be returned from the surrounding call to Storage.Prune. +// Any returned error will be returned from the surrounding call to ExecutionDataTracker.Prune. // The prune callback can be used to delete the corresponding // blob data from the blob store. type PruneCallback func(cid.Cid) error -type Storage interface { +type ExecutionDataTracker interface { // Update is used to track new blob CIDs. // It can be used to track blobs for both sealed and unsealed // heights, and the same blob may be added multiple times for @@ -80,7 +80,7 @@ type Storage interface { // PruneUpToHeight removes all data from storage corresponding // to block heights up to and including the given height, // and updates the latest pruned height value. - // It locks the Storage and ensures that no other writes + // It locks the ExecutionDataTracker and ensures that no other writes // can occur during the pruning. // It is up to the caller to ensure that this is never // called with a value higher than the fulfilled height. diff --git a/module/executiondatasync/tracker/mock/storage.go b/storage/mock/execution_data_tracker.go similarity index 70% rename from module/executiondatasync/tracker/mock/storage.go rename to storage/mock/execution_data_tracker.go index c9fd144785c..cd0e4f7dc7a 100644 --- a/module/executiondatasync/tracker/mock/storage.go +++ b/storage/mock/execution_data_tracker.go @@ -1,19 +1,19 @@ // Code generated by mockery v2.43.2. DO NOT EDIT. -package mocktracker +package mock import ( - tracker "github.com/onflow/flow-go/module/executiondatasync/tracker" + storage "github.com/onflow/flow-go/storage" mock "github.com/stretchr/testify/mock" ) -// Storage is an autogenerated mock type for the Storage type -type Storage struct { +// ExecutionDataTracker is an autogenerated mock type for the ExecutionDataTracker type +type ExecutionDataTracker struct { mock.Mock } // GetFulfilledHeight provides a mock function with given fields: -func (_m *Storage) GetFulfilledHeight() (uint64, error) { +func (_m *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { ret := _m.Called() if len(ret) == 0 { @@ -41,7 +41,7 @@ func (_m *Storage) GetFulfilledHeight() (uint64, error) { } // GetPrunedHeight provides a mock function with given fields: -func (_m *Storage) GetPrunedHeight() (uint64, error) { +func (_m *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { ret := _m.Called() if len(ret) == 0 { @@ -69,7 +69,7 @@ func (_m *Storage) GetPrunedHeight() (uint64, error) { } // PruneUpToHeight provides a mock function with given fields: height -func (_m *Storage) PruneUpToHeight(height uint64) error { +func (_m *ExecutionDataTracker) PruneUpToHeight(height uint64) error { ret := _m.Called(height) if len(ret) == 0 { @@ -87,7 +87,7 @@ func (_m *Storage) PruneUpToHeight(height uint64) error { } // SetFulfilledHeight provides a mock function with given fields: height -func (_m *Storage) SetFulfilledHeight(height uint64) error { +func (_m *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { ret := _m.Called(height) if len(ret) == 0 { @@ -105,7 +105,7 @@ func (_m *Storage) SetFulfilledHeight(height uint64) error { } // Update provides a mock function with given fields: _a0 -func (_m *Storage) Update(_a0 tracker.UpdateFn) error { +func (_m *ExecutionDataTracker) Update(_a0 storage.UpdateFn) error { ret := _m.Called(_a0) if len(ret) == 0 { @@ -113,7 +113,7 @@ func (_m *Storage) Update(_a0 tracker.UpdateFn) error { } var r0 error - if rf, ok := ret.Get(0).(func(tracker.UpdateFn) error); ok { + if rf, ok := ret.Get(0).(func(storage.UpdateFn) error); ok { r0 = rf(_a0) } else { r0 = ret.Error(0) @@ -122,13 +122,13 @@ func (_m *Storage) Update(_a0 tracker.UpdateFn) error { return r0 } -// NewStorage creates a new instance of Storage. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// NewExecutionDataTracker creates a new instance of ExecutionDataTracker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. -func NewStorage(t interface { +func NewExecutionDataTracker(t interface { mock.TestingT Cleanup(func()) -}) *Storage { - mock := &Storage{} +}) *ExecutionDataTracker { + mock := &ExecutionDataTracker{} mock.Mock.Test(t) t.Cleanup(func() { mock.AssertExpectations(t) }) diff --git a/storage/mocks/util.go b/storage/mocks/util.go new file mode 100644 index 00000000000..ff34e7da2d6 --- /dev/null +++ b/storage/mocks/util.go @@ -0,0 +1,20 @@ +package mocks + +import ( + "github.com/ipfs/go-cid" + "github.com/stretchr/testify/mock" + + "github.com/onflow/flow-go/storage" + storagemock "github.com/onflow/flow-go/storage/mock" +) + +func NewMockStorage() *storagemock.ExecutionDataTracker { + trackerStorage := new(storagemock.ExecutionDataTracker) + trackerStorage.On("Update", mock.Anything).Return(func(fn storage.UpdateFn) error { + return fn(func(uint64, ...cid.Cid) error { return nil }) + }) + + trackerStorage.On("SetFulfilledHeight", mock.Anything).Return(nil) + + return trackerStorage +} diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 0bcf871bc68..8307f435bca 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -11,14 +11,13 @@ import ( "github.com/ipfs/go-cid" "github.com/rs/zerolog" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/pebble/operation" ) type StorageOption func(*ExecutionDataTracker) -var _ tracker.Storage = (*ExecutionDataTracker)(nil) +var _ storage.ExecutionDataTracker = (*ExecutionDataTracker)(nil) // The ExecutionDataTracker component tracks the following information: // - the latest pruned height @@ -38,11 +37,11 @@ type ExecutionDataTracker struct { mu sync.RWMutex db *pebble.DB - pruneCallback tracker.PruneCallback + pruneCallback storage.PruneCallback logger zerolog.Logger } -func WithPruneCallback(callback tracker.PruneCallback) StorageOption { +func WithPruneCallback(callback storage.PruneCallback) StorageOption { return func(s *ExecutionDataTracker) { s.pruneCallback = callback } @@ -122,7 +121,7 @@ func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { return nil } -func (s *ExecutionDataTracker) Update(f tracker.UpdateFn) error { +func (s *ExecutionDataTracker) Update(f storage.UpdateFn) error { s.mu.RLock() defer s.mu.RUnlock() return f(s.trackBlobs) @@ -176,7 +175,7 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { } func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - cidsPerBatch := tracker.CidsPerBatch + cidsPerBatch := storage.CidsPerBatch for len(cids) > 0 { batchSize := cidsPerBatch @@ -197,7 +196,7 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e return nil } -func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) error { +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { for _, dInfo := range deleteInfos { err := operation.RemoveBlob(dInfo.Height, dInfo.Cid)(s.db) if err != nil { @@ -218,9 +217,9 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*tracker.DeleteInfo) er var ffBytes = bytes.Repeat([]byte{0xFF}, 32) func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { - blobRecordPrefix := []byte{tracker.PrefixBlobRecord} + blobRecordPrefix := []byte{storage.PrefixBlobRecord} itemsPerBatch := 256 - var batch []*tracker.DeleteInfo + var batch []*storage.DeleteInfo s.mu.Lock() defer s.mu.Unlock() @@ -247,7 +246,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { for it.SeekGE(blobRecordPrefix); it.Valid(); it.Next() { blobRecordKey := it.Key() - blockHeight, blobCid, err := tracker.ParseBlobRecordKey(blobRecordKey) + blockHeight, blobCid, err := storage.ParseBlobRecordKey(blobRecordKey) if err != nil { return fmt.Errorf("malformed blob record key %v: %w", blobRecordKey, err) } @@ -257,7 +256,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { break } - dInfo := &tracker.DeleteInfo{ + dInfo := &storage.DeleteInfo{ Cid: blobCid, Height: blockHeight, } diff --git a/storage/pebble/execution_data_tracker_test.go b/storage/pebble/execution_data_tracker_test.go index 26baf6d98e4..fc6f8fb3a88 100644 --- a/storage/pebble/execution_data_tracker_test.go +++ b/storage/pebble/execution_data_tracker_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/module/blobs" - "github.com/onflow/flow-go/module/executiondatasync/tracker" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/pebble/operation" ) @@ -43,7 +42,7 @@ func TestPrune(t *testing.T) { c3 := randomCid() c4 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c3, c4)) @@ -93,7 +92,7 @@ func TestPruneNonLatestHeight(t *testing.T) { c1 := randomCid() c2 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1, c2)) require.NoError(t, tbf(2, c1, c2)) @@ -139,7 +138,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { c2 := randomCid() c3 := randomCid() - require.NoError(t, executionDataTracker.Update(func(tbf tracker.TrackBlobsFn) error { + require.NoError(t, executionDataTracker.Update(func(tbf storage.TrackBlobsFn) error { require.NoError(t, tbf(1, c1)) require.NoError(t, tbf(2, c2)) // It is important to check if the record with height 256 does not precede diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index 36e591f5113..b9027ea3362 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -4,53 +4,53 @@ import ( "github.com/cockroachdb/pebble" "github.com/ipfs/go-cid" - "github.com/onflow/flow-go/module/executiondatasync/tracker" + "github.com/onflow/flow-go/storage" ) func UpdateTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func InsertTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func RetrieveTrackerFulfilledHeight(height *uint64) func(r pebble.Reader) error { - return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStateFulfilledHeight), height) + return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } func UpdateTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func InsertTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func RetrieveTrackerPrunedHeight(height *uint64) func(r pebble.Reader) error { - return retrieve(makePrefix(tracker.PrefixGlobalState, tracker.GlobalStatePrunedHeight), height) + return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixLatestHeight, cid), height) + return insert(makePrefix(storage.PrefixLatestHeight, cid), height) } func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(r pebble.Reader) error { - return retrieve(makePrefix(tracker.PrefixLatestHeight, cid), height) + return retrieve(makePrefix(storage.PrefixLatestHeight, cid), height) } func RemoveTrackerLatestHeight(cid cid.Cid) func(w pebble.Writer) error { - return remove(makePrefix(tracker.PrefixLatestHeight, cid)) + return remove(makePrefix(storage.PrefixLatestHeight, cid)) } func InsertBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { - return insert(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) + return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(r pebble.Reader) error { - return retrieve(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid), nil) + return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } func RemoveBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { - return remove(makePrefix(tracker.PrefixBlobRecord, blockHeight, cid)) + return remove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) } diff --git a/module/executiondatasync/tracker/noop.go b/storage/util/noop.go similarity index 66% rename from module/executiondatasync/tracker/noop.go rename to storage/util/noop.go index 552c1cbf2ca..d48b7a866d6 100644 --- a/module/executiondatasync/tracker/noop.go +++ b/storage/util/noop.go @@ -1,12 +1,16 @@ -package tracker +package util -import "github.com/ipfs/go-cid" +import ( + "github.com/ipfs/go-cid" + + "github.com/onflow/flow-go/storage" +) type NoopStorage struct{} -var _ Storage = (*NoopStorage)(nil) +var _ storage.ExecutionDataTracker = (*NoopStorage)(nil) -func (s *NoopStorage) Update(update UpdateFn) error { +func (s *NoopStorage) Update(update storage.UpdateFn) error { return update(func(blockHeight uint64, cids ...cid.Cid) error { return nil }) From 0f3dd84891bf6b008cb0879f0700011259c51401 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Wed, 31 Jul 2024 16:40:45 +0300 Subject: [PATCH 08/33] Updated pebble execution data tracker --- storage/badger/execution_data_tracker.go | 2 +- storage/execution_data_tracker.go | 1 + storage/pebble/execution_data_tracker.go | 16 +++------------- 3 files changed, 5 insertions(+), 14 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 4f19192c4bc..de18c4534fd 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -237,7 +237,7 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er } func (s *ExecutionDataTracker) batchDeleteItemLimit() int { - itemsPerBatch := 256 + itemsPerBatch := storage.DeleteItemsPerBatch maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength) if maxItemsPerBatch < itemsPerBatch { itemsPerBatch = maxItemsPerBatch diff --git a/storage/execution_data_tracker.go b/storage/execution_data_tracker.go index e56c934819a..3b752884276 100644 --- a/storage/execution_data_tracker.go +++ b/storage/execution_data_tracker.go @@ -23,6 +23,7 @@ const ( const CidsPerBatch = 16 // number of cids to track per batch const BlobRecordKeyLength = 1 + 8 + blobs.CidLength const LatestHeightKeyLength = 1 + blobs.CidLength +const DeleteItemsPerBatch = 256 func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { blockHeight := binary.BigEndian.Uint64(key[1:]) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 8307f435bca..4d39e3be2aa 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -175,22 +175,12 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { } func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - cidsPerBatch := storage.CidsPerBatch - for len(cids) > 0 { - batchSize := cidsPerBatch - if len(cids) < batchSize { - batchSize = len(cids) - } - batch := cids[:batchSize] - - for _, c := range batch { + for _, c := range cids { if err := s.trackBlob(blockHeight, c); err != nil { return fmt.Errorf("failed to track blob %s: %w", c.String(), err) } } - - cids = cids[batchSize:] } return nil @@ -214,11 +204,11 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er return nil } -var ffBytes = bytes.Repeat([]byte{0xFF}, 32) +var ffBytes = bytes.Repeat([]byte{0xFF}, storage.BlobRecordKeyLength) func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{storage.PrefixBlobRecord} - itemsPerBatch := 256 + itemsPerBatch := storage.DeleteItemsPerBatch var batch []*storage.DeleteInfo s.mu.Lock() From 437d0a733b6f4382733816240b5dc96d43b1852d Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Wed, 31 Jul 2024 18:18:27 +0300 Subject: [PATCH 09/33] Added comment for ffBytes, fixed check --- storage/pebble/execution_data_tracker.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 4d39e3be2aa..fceb1c9e52b 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -175,7 +175,7 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { } func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - for len(cids) > 0 { + if len(cids) > 0 { for _, c := range cids { if err := s.trackBlob(blockHeight, c); err != nil { return fmt.Errorf("failed to track blob %s: %w", c.String(), err) @@ -204,6 +204,9 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er return nil } +// for forward iteration, add the 0xff-bytes suffix to the end +// prefix, to ensure we include all keys with that prefix before +// finishing. var ffBytes = bytes.Repeat([]byte{0xFF}, storage.BlobRecordKeyLength) func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { From d9218c94ba39a7806e15bf0b427b11498ca524dc Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Wed, 31 Jul 2024 19:48:42 +0300 Subject: [PATCH 10/33] Added integration test for pebble version of execution data pruning, updated integration tests to avoid code dublication --- .../cohort3/execution_data_pruning_test.go | 48 ++++++++----------- .../cohort3/execution_state_sync_test.go | 14 ++++-- .../pebble_execution_data_pruning_test.go | 26 ++++++++++ 3 files changed, 54 insertions(+), 34 deletions(-) create mode 100644 integration/tests/access/cohort3/pebble_execution_data_pruning_test.go diff --git a/integration/tests/access/cohort3/execution_data_pruning_test.go b/integration/tests/access/cohort3/execution_data_pruning_test.go index 312ee60347c..7ce3c4fd22d 100644 --- a/integration/tests/access/cohort3/execution_data_pruning_test.go +++ b/integration/tests/access/cohort3/execution_data_pruning_test.go @@ -3,22 +3,17 @@ package cohort3 import ( "context" "fmt" - "path/filepath" "testing" "time" - badgerds "github.com/ipfs/go-ds-badger2" "github.com/rs/zerolog" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" sdk "github.com/onflow/flow-go-sdk" "github.com/onflow/flow-go/integration/testnet" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/executiondatasync/execution_data" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage/badger" @@ -41,9 +36,10 @@ type ExecutionDataPruningSuite struct { accessNodeName string observerNodeName string // threshold defines the maximum height range and how frequently pruning is performed. - threshold uint64 - heightRangeTarget uint64 - pruningInterval string + threshold uint64 + heightRangeTarget uint64 + pruningInterval string + executionDataDBMode execution_data.ExecutionDataDBMode // root context for the current test ctx context.Context @@ -60,6 +56,10 @@ func (s *ExecutionDataPruningSuite) TearDownTest() { } func (s *ExecutionDataPruningSuite) SetupTest() { + s.setup(execution_data.ExecutionDataDBModeBadger) +} + +func (s *ExecutionDataPruningSuite) setup(executionDataDBMode execution_data.ExecutionDataDBMode) { s.log = unittest.LoggerForTest(s.Suite.T(), zerolog.InfoLevel) s.log.Info().Msg("================> SetupTest") defer func() { @@ -69,6 +69,7 @@ func (s *ExecutionDataPruningSuite) SetupTest() { s.threshold = 50 s.heightRangeTarget = 100 s.pruningInterval = "10s" + s.executionDataDBMode = executionDataDBMode // access node s.accessNodeName = testnet.PrimaryAN @@ -83,6 +84,7 @@ func (s *ExecutionDataPruningSuite) SetupTest() { testnet.WithAdditionalFlagf("--execution-state-dir=%s", testnet.DefaultExecutionStateDir), testnet.WithAdditionalFlagf("--public-network-execution-data-sync-enabled=true"), testnet.WithAdditionalFlagf("--event-query-mode=local-only"), + testnet.WithAdditionalFlagf("--execution-data-db=%s", s.executionDataDBMode.String()), testnet.WithAdditionalFlagf("--execution-data-height-range-target=%d", s.heightRangeTarget), testnet.WithAdditionalFlagf("--execution-data-height-range-threshold=%d", s.threshold), testnet.WithAdditionalFlagf(fmt.Sprintf("--execution-data-pruning-interval=%s", s.pruningInterval)), @@ -137,18 +139,13 @@ func (s *ExecutionDataPruningSuite) SetupTest() { s.net.Start(s.ctx) } -// getGRPCClient is the helper func to create an access api client -func (s *ExecutionDataPruningSuite) getGRPCClient(address string) (accessproto.AccessAPIClient, error) { - conn, err := grpc.Dial(address, grpc.WithTransportCredentials(insecure.NewCredentials())) - if err != nil { - return nil, err - } - - client := accessproto.NewAccessAPIClient(conn) - return client, nil +// TestHappyPath tests the execution data pruning process using badger DB in a happy path scenario. +func (s *ExecutionDataPruningSuite) TestHappyPath() { + s.executionDataPruningTest() } -// TestHappyPath tests the execution data pruning process in a happy path scenario. +// executionDataPruningTest tests the execution data pruning process in a happy path scenario. + // The test follows these steps: // // 1. Define a target block height (waitingBlockHeight) for which execution data will be indexed. @@ -160,7 +157,7 @@ func (s *ExecutionDataPruningSuite) getGRPCClient(address string) (accessproto.A // 3. Stop all Flow network containers to simulate a network shutdown and ensure the indexing process is complete. // 4. Verify the results of execution data pruning: // - Check that the Access and Observer Nodes execution data up to the pruning threshold height has been correctly pruned. -func (s *ExecutionDataPruningSuite) TestHappyPath() { +func (s *ExecutionDataPruningSuite) executionDataPruningTest() { accessNode := s.net.ContainerByName(s.accessNodeName) observerNode := s.net.ContainerByName(s.observerNodeName) @@ -171,7 +168,7 @@ func (s *ExecutionDataPruningSuite) TestHappyPath() { metrics := metrics.NewNoopCollector() // start an execution data service using the Access Node's execution data db - anEds := s.nodeExecutionDataStore(accessNode) + anEds := nodeExecutionDataStore(s.T(), accessNode, s.executionDataDBMode) // setup storage objects needed to get the execution data id anDB, err := accessNode.DB() @@ -182,7 +179,7 @@ func (s *ExecutionDataPruningSuite) TestHappyPath() { // start an execution data service using the Observer Node's execution data db - onEds := s.nodeExecutionDataStore(observerNode) + onEds := nodeExecutionDataStore(s.T(), observerNode, s.executionDataDBMode) // setup storage objects needed to get the execution data id onDB, err := observerNode.DB() require.NoError(s.T(), err, "could not open db") @@ -197,7 +194,7 @@ func (s *ExecutionDataPruningSuite) TestHappyPath() { func (s *ExecutionDataPruningSuite) waitUntilExecutionDataForBlockIndexed(waitingBlockHeight uint64) { observerNode := s.net.ContainerByName(s.observerNodeName) - grpcClient, err := s.getGRPCClient(observerNode.Addr(testnet.GRPCPort)) + grpcClient, err := getAccessAPIClient(observerNode.Addr(testnet.GRPCPort)) s.Require().NoError(err) // creating execution data api client @@ -281,10 +278,3 @@ func (s *ExecutionDataPruningSuite) checkResults( require.ErrorAs(s.T(), err, &blobNotFoundError) } } - -func (s *ExecutionDataPruningSuite) nodeExecutionDataStore(node *testnet.Container) execution_data.ExecutionDataStore { - ds, err := badgerds.NewDatastore(filepath.Join(node.ExecutionDataDBPath(), "blobstore"), &badgerds.DefaultOptions) - require.NoError(s.T(), err, "could not get execution datastore") - - return execution_data.NewExecutionDataStore(blobs.NewBlobstore(ds), execution_data.DefaultSerializer) -} diff --git a/integration/tests/access/cohort3/execution_state_sync_test.go b/integration/tests/access/cohort3/execution_state_sync_test.go index 4ff4e77383e..a1ab09ccc32 100644 --- a/integration/tests/access/cohort3/execution_state_sync_test.go +++ b/integration/tests/access/cohort3/execution_state_sync_test.go @@ -166,7 +166,7 @@ func (s *ExecutionStateSyncSuite) executionStateSyncTest() { // start an execution data service using the Access Node's execution data db an := s.net.ContainerByID(s.bridgeID) - anEds := s.nodeExecutionDataStore(an) + anEds := nodeExecutionDataStore(s.T(), an, s.executionDataDBMode) // setup storage objects needed to get the execution data id anDB, err := an.DB() @@ -177,7 +177,7 @@ func (s *ExecutionStateSyncSuite) executionStateSyncTest() { // start an execution data service using the Observer Node's execution data db on := s.net.ContainerByName(s.observerName) - onEds := s.nodeExecutionDataStore(on) + onEds := nodeExecutionDataStore(s.T(), on, s.executionDataDBMode) // setup storage objects needed to get the execution data id onDB, err := on.DB() @@ -216,17 +216,21 @@ func (s *ExecutionStateSyncSuite) executionStateSyncTest() { } } -func (s *ExecutionStateSyncSuite) nodeExecutionDataStore(node *testnet.Container) execution_data.ExecutionDataStore { +func nodeExecutionDataStore( + t *testing.T, + node *testnet.Container, + executionDataDBMode execution_data.ExecutionDataDBMode, +) execution_data.ExecutionDataStore { var ds datastore.Batching var err error dsPath := filepath.Join(node.ExecutionDataDBPath(), "blobstore") - if s.executionDataDBMode == execution_data.ExecutionDataDBModePebble { + if executionDataDBMode == execution_data.ExecutionDataDBModePebble { ds, err = pebbleds.NewDatastore(dsPath, nil) } else { ds, err = badgerds.NewDatastore(dsPath, &badgerds.DefaultOptions) } - require.NoError(s.T(), err, "could not get execution datastore") + require.NoError(t, err, "could not get execution datastore") return execution_data.NewExecutionDataStore(blobs.NewBlobstore(ds), execution_data.DefaultSerializer) } diff --git a/integration/tests/access/cohort3/pebble_execution_data_pruning_test.go b/integration/tests/access/cohort3/pebble_execution_data_pruning_test.go new file mode 100644 index 00000000000..3bef48dd7e8 --- /dev/null +++ b/integration/tests/access/cohort3/pebble_execution_data_pruning_test.go @@ -0,0 +1,26 @@ +package cohort3 + +import ( + "testing" + + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/module/executiondatasync/execution_data" +) + +func TestPebbleExecutionDataPruning(t *testing.T) { + suite.Run(t, new(PebbleExecutionDataPruningSuite)) +} + +type PebbleExecutionDataPruningSuite struct { + ExecutionDataPruningSuite +} + +func (s *PebbleExecutionDataPruningSuite) SetupTest() { + s.setup(execution_data.ExecutionDataDBModePebble) +} + +// TestHappyPath tests the execution data pruning process using pebble DB in a happy path scenario. +func (s *PebbleExecutionDataPruningSuite) TestHappyPath() { + s.executionDataPruningTest() +} From 63573d60029d97291c843f055e5d04be7c35910d Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Wed, 31 Jul 2024 22:11:40 +0300 Subject: [PATCH 11/33] Updated integration test --- integration/tests/access/cohort3/execution_data_pruning_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/integration/tests/access/cohort3/execution_data_pruning_test.go b/integration/tests/access/cohort3/execution_data_pruning_test.go index 7ce3c4fd22d..ebcc7e1d823 100644 --- a/integration/tests/access/cohort3/execution_data_pruning_test.go +++ b/integration/tests/access/cohort3/execution_data_pruning_test.go @@ -123,6 +123,7 @@ func (s *ExecutionDataPruningSuite) setup(executionDataDBMode execution_data.Exe "--execution-data-retry-delay=1s", "--event-query-mode=local-only", "--local-service-api-enabled=true", + fmt.Sprintf("--execution-data-db=%s", s.executionDataDBMode.String()), fmt.Sprintf("--execution-data-height-range-target=%d", s.heightRangeTarget), fmt.Sprintf("--execution-data-height-range-threshold=%d", s.threshold), fmt.Sprintf("--execution-data-pruning-interval=%s", s.pruningInterval), From edadc783ae4631f5441aefcc3ca07e6456d705df Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 1 Aug 2024 15:44:49 +0300 Subject: [PATCH 12/33] Updated naming --- storage/badger/execution_data_tracker.go | 15 +++++++-------- storage/pebble/execution_data_tracker.go | 15 +++++++-------- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index de18c4534fd..ce3c2a1350e 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -67,25 +67,25 @@ func NewExecutionDataTracker(dbPath string, startHeight uint64, logger zerolog.L return nil, fmt.Errorf("could not open tracker db: %w", err) } - storage := &ExecutionDataTracker{ + tracker := &ExecutionDataTracker{ db: db, pruneCallback: func(c cid.Cid) error { return nil }, logger: lg, } for _, opt := range opts { - opt(storage) + opt(tracker) } - lg.Info().Msgf("initialize storage with start height: %d", startHeight) + lg.Info().Msgf("initialize tracker with start height: %d", startHeight) - if err := storage.init(startHeight); err != nil { - return nil, fmt.Errorf("failed to initialize storage: %w", err) + if err := tracker.init(startHeight); err != nil { + return nil, fmt.Errorf("failed to initialize tracker: %w", err) } - lg.Info().Msgf("storage initialized") + lg.Info().Msgf("tracker initialized") - return storage, nil + return tracker, nil } func (s *ExecutionDataTracker) init(startHeight uint64) error { @@ -257,7 +257,6 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { return err } - // TODO: move to separate 'iterate' function if err := s.db.View(func(txn *badger.Txn) error { it := txn.NewIterator(badger.IteratorOptions{ PrefetchValues: false, diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index fceb1c9e52b..815cc5ded9a 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -54,25 +54,25 @@ func NewExecutionDataTracker(path string, startHeight uint64, logger zerolog.Log return nil, fmt.Errorf("could not open db: %w", err) } - storage := &ExecutionDataTracker{ + tracker := &ExecutionDataTracker{ db: db, pruneCallback: func(c cid.Cid) error { return nil }, logger: lg, } for _, opt := range opts { - opt(storage) + opt(tracker) } - lg.Info().Msgf("initialize storage with start height: %d", startHeight) + lg.Info().Msgf("initialize tracker with start height: %d", startHeight) - if err := storage.init(startHeight); err != nil { - return nil, fmt.Errorf("failed to initialize storage: %w", err) + if err := tracker.init(startHeight); err != nil { + return nil, fmt.Errorf("failed to initialize tracker: %w", err) } - lg.Info().Msgf("storage initialized") + lg.Info().Msgf("tracker initialized") - return storage, nil + return tracker, nil } // TODO: move common logic into separate function to avoid duplication of code @@ -221,7 +221,6 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { return err } - // TODO: move to separate 'iterate' function and move common logic err := func(tx pebble.Reader) error { options := pebble.IterOptions{ LowerBound: blobRecordPrefix, From 68a02238dd49c90759cad27da75a69d28afa5315 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 1 Aug 2024 16:07:08 +0300 Subject: [PATCH 13/33] Generated mocks --- storage/mock/prune_callback.go | 45 +++++++++++++++++++++++++++++ storage/mock/track_blobs_fn.go | 52 ++++++++++++++++++++++++++++++++++ storage/mock/update_fn.go | 45 +++++++++++++++++++++++++++++ 3 files changed, 142 insertions(+) create mode 100644 storage/mock/prune_callback.go create mode 100644 storage/mock/track_blobs_fn.go create mode 100644 storage/mock/update_fn.go diff --git a/storage/mock/prune_callback.go b/storage/mock/prune_callback.go new file mode 100644 index 00000000000..3ef0a3f2c6a --- /dev/null +++ b/storage/mock/prune_callback.go @@ -0,0 +1,45 @@ +// Code generated by mockery v2.43.2. DO NOT EDIT. + +package mock + +import ( + cid "github.com/ipfs/go-cid" + mock "github.com/stretchr/testify/mock" +) + +// PruneCallback is an autogenerated mock type for the PruneCallback type +type PruneCallback struct { + mock.Mock +} + +// Execute provides a mock function with given fields: _a0 +func (_m *PruneCallback) Execute(_a0 cid.Cid) error { + ret := _m.Called(_a0) + + if len(ret) == 0 { + panic("no return value specified for Execute") + } + + var r0 error + if rf, ok := ret.Get(0).(func(cid.Cid) error); ok { + r0 = rf(_a0) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// NewPruneCallback creates a new instance of PruneCallback. 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 NewPruneCallback(t interface { + mock.TestingT + Cleanup(func()) +}) *PruneCallback { + mock := &PruneCallback{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/storage/mock/track_blobs_fn.go b/storage/mock/track_blobs_fn.go new file mode 100644 index 00000000000..327d2d506a8 --- /dev/null +++ b/storage/mock/track_blobs_fn.go @@ -0,0 +1,52 @@ +// Code generated by mockery v2.43.2. DO NOT EDIT. + +package mock + +import ( + cid "github.com/ipfs/go-cid" + mock "github.com/stretchr/testify/mock" +) + +// TrackBlobsFn is an autogenerated mock type for the TrackBlobsFn type +type TrackBlobsFn struct { + mock.Mock +} + +// Execute provides a mock function with given fields: blockHeight, cids +func (_m *TrackBlobsFn) Execute(blockHeight uint64, cids ...cid.Cid) error { + _va := make([]interface{}, len(cids)) + for _i := range cids { + _va[_i] = cids[_i] + } + var _ca []interface{} + _ca = append(_ca, blockHeight) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for Execute") + } + + var r0 error + if rf, ok := ret.Get(0).(func(uint64, ...cid.Cid) error); ok { + r0 = rf(blockHeight, cids...) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// NewTrackBlobsFn creates a new instance of TrackBlobsFn. 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 NewTrackBlobsFn(t interface { + mock.TestingT + Cleanup(func()) +}) *TrackBlobsFn { + mock := &TrackBlobsFn{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/storage/mock/update_fn.go b/storage/mock/update_fn.go new file mode 100644 index 00000000000..eed01a28b8b --- /dev/null +++ b/storage/mock/update_fn.go @@ -0,0 +1,45 @@ +// Code generated by mockery v2.43.2. DO NOT EDIT. + +package mock + +import ( + storage "github.com/onflow/flow-go/storage" + mock "github.com/stretchr/testify/mock" +) + +// UpdateFn is an autogenerated mock type for the UpdateFn type +type UpdateFn struct { + mock.Mock +} + +// Execute provides a mock function with given fields: _a0 +func (_m *UpdateFn) Execute(_a0 storage.TrackBlobsFn) error { + ret := _m.Called(_a0) + + if len(ret) == 0 { + panic("no return value specified for Execute") + } + + var r0 error + if rf, ok := ret.Get(0).(func(storage.TrackBlobsFn) error); ok { + r0 = rf(_a0) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// NewUpdateFn creates a new instance of UpdateFn. 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 NewUpdateFn(t interface { + mock.TestingT + Cleanup(func()) +}) *UpdateFn { + mock := &UpdateFn{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} From a77db84f0f968f28a587e6b8de0645306da8f030 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 1 Aug 2024 17:21:41 +0300 Subject: [PATCH 14/33] Updated naming for operations on storages --- storage/badger/execution_data_tracker.go | 4 ++-- storage/badger/operation/common.go | 8 +++++--- storage/badger/operation/execution_data_tracker.go | 4 ++-- storage/pebble/execution_data_tracker.go | 4 ++-- storage/pebble/operation/common.go | 2 ++ storage/pebble/operation/execution_data_tracker.go | 4 ++-- 6 files changed, 15 insertions(+), 11 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index ce3c2a1350e..8a670fbc399 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -120,12 +120,12 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { } func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { - err := s.db.Update(operation.InsertTrackerFulfilledHeight(startHeight)) + err := s.db.Update(operation.InitTrackerFulfilledHeight(startHeight)) if err != nil { return fmt.Errorf("failed to set fulfilled height value: %w", err) } - err = s.db.Update(operation.InsertTrackerPrunedHeight(startHeight)) + err = s.db.Update(operation.InitTrackerPrunedHeight(startHeight)) if err != nil { return fmt.Errorf("failed to set pruned height value: %w", err) } diff --git a/storage/badger/operation/common.go b/storage/badger/operation/common.go index 4bd5a50b209..624fe28174a 100644 --- a/storage/badger/operation/common.go +++ b/storage/badger/operation/common.go @@ -253,11 +253,13 @@ func retrieve(key []byte, entity interface{}) func(*badger.Txn) error { return irrecoverable.NewExceptionf("could not load data: %w", err) } + // in case the value is not needed + if entity == nil { + return nil + } + // get the value from the item err = item.Value(func(val []byte) error { - if entity == nil { - return nil - } err := msgpack.Unmarshal(val, entity) return err }) diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index 768975f0379..d8a659b183d 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -11,7 +11,7 @@ func UpdateTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } -func InsertTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { +func InitTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } @@ -23,7 +23,7 @@ func UpdateTrackerPrunedHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } -func InsertTrackerPrunedHeight(height uint64) func(*badger.Txn) error { +func InitTrackerPrunedHeight(height uint64) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 815cc5ded9a..90a0be3f814 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -108,12 +108,12 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { } func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { - err := operation.InsertTrackerFulfilledHeight(startHeight)(s.db) + err := operation.InitTrackerFulfilledHeight(startHeight)(s.db) if err != nil { return fmt.Errorf("failed to set fulfilled height value: %w", err) } - err = operation.InsertTrackerPrunedHeight(startHeight)(s.db) + err = operation.InitTrackerPrunedHeight(startHeight)(s.db) if err != nil { return fmt.Errorf("failed to set pruned height value: %w", err) } diff --git a/storage/pebble/operation/common.go b/storage/pebble/operation/common.go index 7e5f0a847d6..48953137eaf 100644 --- a/storage/pebble/operation/common.go +++ b/storage/pebble/operation/common.go @@ -34,9 +34,11 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } defer closer.Close() + // in case the value is not needed if sc == nil { return nil } + err = msgpack.Unmarshal(val, sc) if err != nil { return irrecoverable.NewExceptionf("failed to decode value: %w", err) diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index b9027ea3362..d3e364812be 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -11,7 +11,7 @@ func UpdateTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } -func InsertTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { +func InitTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } @@ -23,7 +23,7 @@ func UpdateTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } -func InsertTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { +func InitTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } From 9a2e3bee038562b4337c26947bfd437933a98602 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 1 Aug 2024 20:19:05 +0300 Subject: [PATCH 15/33] Added documentation --- storage/badger/execution_data_tracker.go | 157 +++++++++++++---- .../operation/execution_data_tracker.go | 12 ++ storage/execution_data_tracker.go | 14 +- storage/pebble/execution_data_tracker.go | 164 +++++++++++++----- .../operation/execution_data_tracker.go | 12 ++ 5 files changed, 280 insertions(+), 79 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 8a670fbc399..abcfcdfcdc0 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -32,21 +32,20 @@ type StorageOption func(*ExecutionDataTracker) var _ storage.ExecutionDataTracker = (*ExecutionDataTracker)(nil) -// The ExecutionDataTracker component tracks the following information: -// - the latest pruned height -// - the latest fulfilled height -// - the set of CIDs of the execution data blobs we know about at each height, so that -// once we prune a fulfilled height we can remove the blob data from local storage -// - for each CID, the most recent height that it was observed at, so that when pruning -// a fulfilled height we don't remove any blob data that is still needed at higher heights +// The ExecutionDataTracker component manages the following information: +// - The latest pruned height +// - The latest fulfilled height +// - The set of CIDs of execution data blobs known at each height, allowing removal of blob data from local storage +// once a fulfilled height is pruned +// - For each CID, the most recent height at which it was observed, ensuring that blob data needed at higher heights +// is not removed during pruning // -// The storage component calls the given prune callback for a CID when the last height -// at which that CID appears is pruned. The prune callback can be used to delete the -// corresponding blob data from the blob store. +// The component invokes the provided prune callback for a CID when the last height at which that CID appears is pruned. +// This callback can be used to delete the corresponding blob data from the blob store. type ExecutionDataTracker struct { - // ensures that pruning operations are not run concurrently with any other db writes - // we acquire the read lock when we want to perform a non-prune WRITE - // we acquire the write lock when we want to perform a prune WRITE + // Ensures that pruning operations are not run concurrently with other database writes. + // Acquires the read lock for non-prune WRITE operations. + // Acquires the write lock for prune WRITE operations. mu sync.RWMutex db *badger.DB @@ -54,15 +53,30 @@ type ExecutionDataTracker struct { logger zerolog.Logger } +// WithPruneCallback is used to configure the ExecutionDataTracker with a custom prune callback. func WithPruneCallback(callback storage.PruneCallback) StorageOption { return func(s *ExecutionDataTracker) { s.pruneCallback = callback } } -func NewExecutionDataTracker(dbPath string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*ExecutionDataTracker, error) { +// NewExecutionDataTracker initializes a new ExecutionDataTracker. +// +// Parameters: +// - path: The file path for the underlying Pebble database. +// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. +// - logger: The logger for logging tracker operations. +// - opts: Additional configuration options such as custom prune callbacks. +// +// No errors are expected during normal operation. +func NewExecutionDataTracker( + path string, + startHeight uint64, + logger zerolog.Logger, + opts ...StorageOption, +) (*ExecutionDataTracker, error) { lg := logger.With().Str("module", "tracker_storage").Logger() - db, err := badger.Open(badger.LSMOnlyOptions(dbPath)) + db, err := badger.Open(badger.LSMOnlyOptions(path)) if err != nil { return nil, fmt.Errorf("could not open tracker db: %w", err) } @@ -88,6 +102,12 @@ func NewExecutionDataTracker(dbPath string, startHeight uint64, logger zerolog.L return tracker, nil } +// init initializes the ExecutionDataTracker by setting the fulfilled and pruned heights. +// +// Parameters: +// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) init(startHeight uint64) error { fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() prunedHeight, prunedHeightErr := s.GetPrunedHeight() @@ -119,6 +139,12 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { return nil } +// bootstrap sets the initial fulfilled and pruned heights to startHeight in an empty database. +// +// Parameters: +// - startHeight: The initial height to set for both fulfilled and pruned heights. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { err := s.db.Update(operation.InitTrackerFulfilledHeight(startHeight)) if err != nil { @@ -133,12 +159,28 @@ func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { return nil } +// Update is used to track new blob CIDs. +// It can be used to track blobs for both sealed and unsealed +// heights, and the same blob may be added multiple times for +// different heights. +// The same blob may also be added multiple times for the same +// height, but it will only be tracked once per height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) Update(f storage.UpdateFn) error { s.mu.RLock() defer s.mu.RUnlock() return f(s.trackBlobs) } +// SetFulfilledHeight updates the fulfilled height value, +// which is the lowest from the highest block heights `h` such that all +// heights <= `h` are sealed and the sealed execution data +// has been downloaded or indexed. +// It is up to the caller to ensure that this is never +// called with a value lower than the pruned height. +// +// No errors are expected during normal operation func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { err := s.db.Update(operation.UpdateTrackerFulfilledHeight(height)) if err != nil { @@ -148,6 +190,9 @@ func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { return nil } +// GetFulfilledHeight returns the current fulfilled height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { var fulfilledHeight uint64 @@ -159,6 +204,43 @@ func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { return fulfilledHeight, nil } +// setPrunedHeight updates the current pruned height. +// +// No errors are expected during normal operation. +func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { + err := s.db.Update(operation.UpdateTrackerPrunedHeight(height)) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil +} + +// GetPrunedHeight returns the current pruned height. +// +// No errors are expected during normal operation. +func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { + var prunedHeight uint64 + + err := s.db.View(operation.RetrieveTrackerPrunedHeight(&prunedHeight)) + if err != nil { + return 0, err + } + + return prunedHeight, nil +} + +// trackBlob tracks a single blob CID at the specified block height. +// This method first inserts a record of the blob at the specified block height. +// It then checks if the current block height is greater than the previously recorded +// latest height for this CID. If so, it updates the latest height to the current block height. +// +// Parameters: +// - tx: The BadgerDB transaction in which the operation is performed. +// - blockHeight: The height at which the blob was observed. +// - c: The CID of the blob to be tracked. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c cid.Cid) error { err := operation.InsertBlob(blockHeight, c)(tx) if err != nil { @@ -186,6 +268,13 @@ func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c c return nil } +// trackBlobs tracks multiple blob CIDs at the specified block height. +// +// Parameters: +// - blockHeight: The height at which the blobs were observed. +// - cids: The CIDs of the blobs to be tracked. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { cidsPerBatch := storage.CidsPerBatch maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength+8) @@ -218,6 +307,14 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e return nil } +// batchDelete deletes multiple blobs from the storage in a batch operation. +// +// Parameters: +// - deleteInfos: Information about the blobs to be deleted, including their heights and whether to delete the latest height record. +// +// Ensures that all specified blobs are deleted from the storage. +// +// Returns an error if the batch deletion fails. func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { for _, dInfo := range deleteInfos { err := s.db.Update(operation.RemoveBlob(dInfo.Height, dInfo.Cid)) @@ -236,6 +333,7 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er return nil } +// batchDeleteItemLimit determines the maximum number of items that can be deleted in a batch operation. func (s *ExecutionDataTracker) batchDeleteItemLimit() int { itemsPerBatch := storage.DeleteItemsPerBatch maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength) @@ -245,6 +343,15 @@ func (s *ExecutionDataTracker) batchDeleteItemLimit() int { return itemsPerBatch } +// PruneUpToHeight removes all data from storage corresponding +// to block heights up to and including the given height, +// and updates the latest pruned height value. +// It locks the ExecutionDataTracker and ensures that no other writes +// can occur during the pruning. +// It is up to the caller to ensure that this is never +// called with a value higher than the fulfilled height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{storage.PrefixBlobRecord} itemsPerBatch := s.batchDeleteItemLimit() @@ -337,23 +444,3 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { return nil } - -func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { - err := s.db.Update(operation.UpdateTrackerPrunedHeight(height)) - if err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil -} - -func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { - var prunedHeight uint64 - - err := s.db.View(operation.RetrieveTrackerPrunedHeight(&prunedHeight)) - if err != nil { - return 0, err - } - - return prunedHeight, nil -} diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index d8a659b183d..5ee5bec50f3 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -7,50 +7,62 @@ import ( "github.com/onflow/flow-go/storage" ) +// UpdateTrackerFulfilledHeight updates the fulfilled height in the execution data tracker storage. func UpdateTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// InitTrackerFulfilledHeight initializes the fulfilled height for the execution data tracker storage. func InitTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// RetrieveTrackerFulfilledHeight retrieves the fulfilled height from the execution data tracker storage. func RetrieveTrackerFulfilledHeight(height *uint64) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// UpdateTrackerPrunedHeight updates the pruned height in the execution data tracker storage. func UpdateTrackerPrunedHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// InitTrackerPrunedHeight initializes the pruned height for the execution data tracker storage. func InitTrackerPrunedHeight(height uint64) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// RetrieveTrackerPrunedHeight retrieves the pruned height from the execution data tracker storage. func RetrieveTrackerPrunedHeight(height *uint64) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// UpsertTrackerLatestHeight set the latest height for the given CID in the execution data tracker storage. func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(*badger.Txn) error { return upsert(makePrefix(storage.PrefixLatestHeight, cid), height) } +// RetrieveTrackerLatestHeight retrieves the latest height for the given CID from the execution data tracker storage. func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixLatestHeight, cid), height) } +// RemoveTrackerLatestHeight removes the latest height for the given CID from the execution data tracker storage. func RemoveTrackerLatestHeight(cid cid.Cid) func(*badger.Txn) error { return remove(makePrefix(storage.PrefixLatestHeight, cid)) } +// InsertBlob inserts a blob record for the given block height and CID into the execution data tracker storage. func InsertBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } +// RetrieveBlob retrieves a blob record for the given block height and CID from the execution data tracker storage. func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } +// RemoveBlob removes a blob record for the given block height and CID from the execution data tracker storage. func RemoveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { return remove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) } diff --git a/storage/execution_data_tracker.go b/storage/execution_data_tracker.go index 3b752884276..1d56e801227 100644 --- a/storage/execution_data_tracker.go +++ b/storage/execution_data_tracker.go @@ -20,11 +20,13 @@ const ( GlobalStatePrunedHeight // latest pruned block height ) -const CidsPerBatch = 16 // number of cids to track per batch const BlobRecordKeyLength = 1 + 8 + blobs.CidLength const LatestHeightKeyLength = 1 + blobs.CidLength -const DeleteItemsPerBatch = 256 +const CidsPerBatch = 16 // number of cids to track per batch +const DeleteItemsPerBatch = 256 // number of items to delete per batch + +// ParseBlobRecordKey parses a blob record key and returns the block height and CID. func ParseBlobRecordKey(key []byte) (uint64, cid.Cid, error) { blockHeight := binary.BigEndian.Uint64(key[1:]) c, err := cid.Cast(key[1+8:]) @@ -62,19 +64,22 @@ type ExecutionDataTracker interface { Update(UpdateFn) error // GetFulfilledHeight returns the current fulfilled height. + // // No errors are expected during normal operation. GetFulfilledHeight() (uint64, error) // SetFulfilledHeight updates the fulfilled height value, - // which is the highest block height `h` such that all + // which is the lowest from the highest block heights `h` such that all // heights <= `h` are sealed and the sealed execution data - // has been downloaded. + // has been downloaded or indexed. // It is up to the caller to ensure that this is never // called with a value lower than the pruned height. + // // No errors are expected during normal operation SetFulfilledHeight(height uint64) error // GetPrunedHeight returns the current pruned height. + // // No errors are expected during normal operation. GetPrunedHeight() (uint64, error) @@ -88,6 +93,7 @@ type ExecutionDataTracker interface { PruneUpToHeight(height uint64) error } +// DeleteInfo contains information for a deletion operation. type DeleteInfo struct { Cid cid.Cid Height uint64 diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 90a0be3f814..a98f5d34dc0 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -15,25 +15,29 @@ import ( "github.com/onflow/flow-go/storage/pebble/operation" ) +// The `ffBytes` constant is used to ensure that all keys with a specific prefix are included during forward iteration. +// By appending a suffix of 0xff bytes to the prefix, all keys that start with the given prefix are captured before +// completing the iteration. +var ffBytes = bytes.Repeat([]byte{0xFF}, storage.BlobRecordKeyLength) + type StorageOption func(*ExecutionDataTracker) var _ storage.ExecutionDataTracker = (*ExecutionDataTracker)(nil) -// The ExecutionDataTracker component tracks the following information: -// - the latest pruned height -// - the latest fulfilled height -// - the set of CIDs of the execution data blobs we know about at each height, so that -// once we prune a fulfilled height we can remove the blob data from local storage -// - for each CID, the most recent height that it was observed at, so that when pruning -// a fulfilled height we don't remove any blob data that is still needed at higher heights +// The ExecutionDataTracker component manages the following information: +// - The latest pruned height +// - The latest fulfilled height +// - The set of CIDs of execution data blobs known at each height, allowing removal of blob data from local storage +// once a fulfilled height is pruned +// - For each CID, the most recent height at which it was observed, ensuring that blob data needed at higher heights +// is not removed during pruning // -// The storage component calls the given prune callback for a CID when the last height -// at which that CID appears is pruned. The prune callback can be used to delete the -// corresponding blob data from the blob store. +// The component invokes the provided prune callback for a CID when the last height at which that CID appears is pruned. +// This callback can be used to delete the corresponding blob data from the blob store. type ExecutionDataTracker struct { - // ensures that pruning operations are not run concurrently with any other db writes - // we acquire the read lock when we want to perform a non-prune WRITE - // we acquire the write lock when we want to perform a prune WRITE + // Ensures that pruning operations are not run concurrently with other database writes. + // Acquires the read lock for non-prune WRITE operations. + // Acquires the write lock for prune WRITE operations. mu sync.RWMutex db *pebble.DB @@ -41,13 +45,28 @@ type ExecutionDataTracker struct { logger zerolog.Logger } +// WithPruneCallback is used to configure the ExecutionDataTracker with a custom prune callback. func WithPruneCallback(callback storage.PruneCallback) StorageOption { return func(s *ExecutionDataTracker) { s.pruneCallback = callback } } -func NewExecutionDataTracker(path string, startHeight uint64, logger zerolog.Logger, opts ...StorageOption) (*ExecutionDataTracker, error) { +// NewExecutionDataTracker initializes a new ExecutionDataTracker. +// +// Parameters: +// - path: The file path for the underlying Pebble database. +// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. +// - logger: The logger for logging tracker operations. +// - opts: Additional configuration options such as custom prune callbacks. +// +// No errors are expected during normal operation. +func NewExecutionDataTracker( + path string, + startHeight uint64, + logger zerolog.Logger, + opts ...StorageOption, +) (*ExecutionDataTracker, error) { lg := logger.With().Str("module", "tracker_storage").Logger() db, err := pebble.Open(path, nil) if err != nil { @@ -76,6 +95,12 @@ func NewExecutionDataTracker(path string, startHeight uint64, logger zerolog.Log } // TODO: move common logic into separate function to avoid duplication of code +// init initializes the ExecutionDataTracker by setting the fulfilled and pruned heights. +// +// Parameters: +// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) init(startHeight uint64) error { fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() prunedHeight, prunedHeightErr := s.GetPrunedHeight() @@ -96,7 +121,7 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { } s.logger.Info().Msgf("finished pruning") } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { - // db is empty, we need to bootstrap it + // db is empty, need to bootstrap it if err := s.bootstrap(startHeight); err != nil { return fmt.Errorf("failed to bootstrap storage: %w", err) } @@ -107,6 +132,12 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { return nil } +// bootstrap sets the initial fulfilled and pruned heights to startHeight in an empty database. +// +// Parameters: +// - startHeight: The initial height to set for both fulfilled and pruned heights. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { err := operation.InitTrackerFulfilledHeight(startHeight)(s.db) if err != nil { @@ -121,12 +152,28 @@ func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { return nil } +// Update is used to track new blob CIDs. +// It can be used to track blobs for both sealed and unsealed +// heights, and the same blob may be added multiple times for +// different heights. +// The same blob may also be added multiple times for the same +// height, but it will only be tracked once per height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) Update(f storage.UpdateFn) error { s.mu.RLock() defer s.mu.RUnlock() return f(s.trackBlobs) } +// SetFulfilledHeight updates the fulfilled height value, +// which is the lowest from the highest block heights `h` such that all +// heights <= `h` are sealed and the sealed execution data +// has been downloaded or indexed. +// It is up to the caller to ensure that this is never +// called with a value lower than the pruned height. +// +// No errors are expected during normal operation func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { err := operation.UpdateTrackerFulfilledHeight(height)(s.db) if err != nil { @@ -136,6 +183,9 @@ func (s *ExecutionDataTracker) SetFulfilledHeight(height uint64) error { return nil } +// GetFulfilledHeight returns the current fulfilled height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { var fulfilledHeight uint64 @@ -147,6 +197,42 @@ func (s *ExecutionDataTracker) GetFulfilledHeight() (uint64, error) { return fulfilledHeight, nil } +// setPrunedHeight updates the current pruned height. +// +// No errors are expected during normal operation. +func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { + err := operation.UpdateTrackerPrunedHeight(height)(s.db) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil +} + +// GetPrunedHeight returns the current pruned height. +// +// No errors are expected during normal operation. +func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { + var prunedHeight uint64 + + err := operation.RetrieveTrackerPrunedHeight(&prunedHeight)(s.db) + if err != nil { + return 0, err + } + + return prunedHeight, nil +} + +// trackBlob tracks a single blob CID at the specified block height. +// This method first inserts a record of the blob at the specified block height. +// It then checks if the current block height is greater than the previously recorded +// latest height for this CID. If so, it updates the latest height to the current block height. +// +// Parameters: +// - blockHeight: The height at which the blob was observed. +// - c: The CID of the blob to be tracked. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { err := operation.InsertBlob(blockHeight, c)(s.db) if err != nil { @@ -174,6 +260,13 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { return nil } +// trackBlobs tracks multiple blob CIDs at the specified block height. +// +// Parameters: +// - blockHeight: The height at which the blobs were observed. +// - cids: The CIDs of the blobs to be tracked. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { if len(cids) > 0 { for _, c := range cids { @@ -186,6 +279,13 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e return nil } +// batchDelete deletes multiple blobs from the storage in a batch operation. +// +// Parameters: +// - deleteInfos: Information about the blobs to be deleted, including their heights and +// whether to delete the latest height record. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { for _, dInfo := range deleteInfos { err := operation.RemoveBlob(dInfo.Height, dInfo.Cid)(s.db) @@ -204,11 +304,15 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er return nil } -// for forward iteration, add the 0xff-bytes suffix to the end -// prefix, to ensure we include all keys with that prefix before -// finishing. -var ffBytes = bytes.Repeat([]byte{0xFF}, storage.BlobRecordKeyLength) - +// PruneUpToHeight removes all data from storage corresponding +// to block heights up to and including the given height, +// and updates the latest pruned height value. +// It locks the ExecutionDataTracker and ensures that no other writes +// can occur during the pruning. +// It is up to the caller to ensure that this is never +// called with a value higher than the fulfilled height. +// +// No errors are expected during normal operation. func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{storage.PrefixBlobRecord} itemsPerBatch := storage.DeleteItemsPerBatch @@ -300,23 +404,3 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { } return nil } - -func (s *ExecutionDataTracker) setPrunedHeight(height uint64) error { - err := operation.UpdateTrackerPrunedHeight(height)(s.db) - if err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil -} - -func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { - var prunedHeight uint64 - - err := operation.RetrieveTrackerPrunedHeight(&prunedHeight)(s.db) - if err != nil { - return 0, err - } - - return prunedHeight, nil -} diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index d3e364812be..3e99ee2354c 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -7,50 +7,62 @@ import ( "github.com/onflow/flow-go/storage" ) +// UpdateTrackerFulfilledHeight updates the fulfilled height in the execution data tracker storage. func UpdateTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// InitTrackerFulfilledHeight initializes the fulfilled height for the execution data tracker storage. func InitTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// RetrieveTrackerFulfilledHeight retrieves the fulfilled height from the execution data tracker storage. func RetrieveTrackerFulfilledHeight(height *uint64) func(r pebble.Reader) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } +// UpdateTrackerPrunedHeight updates the pruned height in the execution data tracker storage. func UpdateTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// InitTrackerPrunedHeight initializes the pruned height for the execution data tracker storage. func InitTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// RetrieveTrackerPrunedHeight retrieves the pruned height from the execution data tracker storage. func RetrieveTrackerPrunedHeight(height *uint64) func(r pebble.Reader) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } +// UpsertTrackerLatestHeight set the latest height for the given CID in the execution data tracker storage. func UpsertTrackerLatestHeight(cid cid.Cid, height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixLatestHeight, cid), height) } +// RetrieveTrackerLatestHeight retrieves the latest height for the given CID from the execution data tracker storage. func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(r pebble.Reader) error { return retrieve(makePrefix(storage.PrefixLatestHeight, cid), height) } +// RemoveTrackerLatestHeight removes the latest height for the given CID from the execution data tracker storage. func RemoveTrackerLatestHeight(cid cid.Cid) func(w pebble.Writer) error { return remove(makePrefix(storage.PrefixLatestHeight, cid)) } +// InsertBlob inserts a blob record for the given block height and CID into the execution data tracker storage. func InsertBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } +// RetrieveBlob retrieves a blob record for the given block height and CID from the execution data tracker storage. func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(r pebble.Reader) error { return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } +// RemoveBlob removes a blob record for the given block height and CID from the execution data tracker storage. func RemoveBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { return remove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) } From 4132ead40e22744515fd14585d975395df8a2fa7 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 1 Aug 2024 20:24:14 +0300 Subject: [PATCH 16/33] Updated order of arguments in NewExecutionDataTracker --- .../node_builder/access_node_builder.go | 4 +- cmd/execution_builder.go | 2 +- cmd/observer/node_builder/observer_builder.go | 4 +- storage/badger/execution_data_tracker.go | 4 +- storage/badger/execution_data_tracker_test.go | 44 ++++++++++++------- storage/pebble/execution_data_tracker.go | 4 +- storage/pebble/execution_data_tracker_test.go | 44 ++++++++++++------- 7 files changed, 65 insertions(+), 41 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index c530d4362a9..9be53e6c051 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -671,9 +671,9 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess trackerDir := filepath.Join(builder.executionDataDir, "tracker") if executionDataDBMode == execution_data.ExecutionDataDBModeBadger { builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( + node.Logger, trackerDir, sealed.Height, - node.Logger, bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) @@ -681,9 +681,9 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess ) } else { builder.ExecutionDataTracker, err = pstorage.NewExecutionDataTracker( + node.Logger, trackerDir, sealed.Height, - node.Logger, pstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index efcb8fbe13d..d30cc671bbd 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -946,9 +946,9 @@ func (exeNode *ExecutionNode) LoadExecutionDataPruner( trackerDir := filepath.Join(exeNode.exeConf.executionDataDir, "tracker") exeNode.executionDataTracker, err = bstorage.NewExecutionDataTracker( + node.Logger, trackerDir, sealed.Height, - node.Logger, bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return exeNode.executionDataBlobstore.DeleteBlob(context.TODO(), c) diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index f75258e84af..3e4a5e2c122 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1240,9 +1240,9 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS trackerDir := filepath.Join(builder.executionDataDir, "tracker") if executionDataDBMode == execution_data.ExecutionDataDBModeBadger { builder.ExecutionDataTracker, err = bstorage.NewExecutionDataTracker( + node.Logger, trackerDir, sealed.Height, - node.Logger, bstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) @@ -1250,9 +1250,9 @@ func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverS ) } else { builder.ExecutionDataTracker, err = pstorage.NewExecutionDataTracker( + node.Logger, trackerDir, sealed.Height, - node.Logger, pstorage.WithPruneCallback(func(c cid.Cid) error { // TODO: use a proper context here return builder.ExecutionDataBlobstore.DeleteBlob(context.TODO(), c) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index abcfcdfcdc0..d4bd3bfd1f2 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -63,16 +63,16 @@ func WithPruneCallback(callback storage.PruneCallback) StorageOption { // NewExecutionDataTracker initializes a new ExecutionDataTracker. // // Parameters: +// - logger: The logger for logging tracker operations. // - path: The file path for the underlying Pebble database. // - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. -// - logger: The logger for logging tracker operations. // - opts: Additional configuration options such as custom prune callbacks. // // No errors are expected during normal operation. func NewExecutionDataTracker( + logger zerolog.Logger, path string, startHeight uint64, - logger zerolog.Logger, opts ...StorageOption, ) (*ExecutionDataTracker, error) { lg := logger.With().Str("module", "tracker_storage").Logger() diff --git a/storage/badger/execution_data_tracker_test.go b/storage/badger/execution_data_tracker_test.go index 6aba559a184..2275d0ad9b1 100644 --- a/storage/badger/execution_data_tracker_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -26,12 +26,16 @@ func randomCid() cid.Cid { func TestPrune(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) - delete(expectedPrunedCIDs, c) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) require.NoError(t, err) // c1 and c2 are for height 1, and c3 and c4 are for height 2 @@ -86,10 +90,14 @@ func TestPrune(t *testing.T) { // if that CID also exists at another height above the pruned height, the CID should not be pruned. func TestPruneNonLatestHeight(t *testing.T) { storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - assert.Fail(t, "unexpected CID pruned: %s", c.String()) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + assert.Fail(t, "unexpected CID pruned: %s", c.String()) + return nil + })) require.NoError(t, err) // c1 and c2 appear both at height 1 and 2 @@ -130,12 +138,16 @@ func TestPruneNonLatestHeight(t *testing.T) { func TestAscendingOrderOfRecords(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) - delete(expectedPrunedCIDs, c) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) require.NoError(t, err) // c1 is for height 1, diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index a98f5d34dc0..2b3979d306a 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -55,16 +55,16 @@ func WithPruneCallback(callback storage.PruneCallback) StorageOption { // NewExecutionDataTracker initializes a new ExecutionDataTracker. // // Parameters: +// - logger: The logger for logging tracker operations. // - path: The file path for the underlying Pebble database. // - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. -// - logger: The logger for logging tracker operations. // - opts: Additional configuration options such as custom prune callbacks. // // No errors are expected during normal operation. func NewExecutionDataTracker( + logger zerolog.Logger, path string, startHeight uint64, - logger zerolog.Logger, opts ...StorageOption, ) (*ExecutionDataTracker, error) { lg := logger.With().Str("module", "tracker_storage").Logger() diff --git a/storage/pebble/execution_data_tracker_test.go b/storage/pebble/execution_data_tracker_test.go index fc6f8fb3a88..4febfcf8cc9 100644 --- a/storage/pebble/execution_data_tracker_test.go +++ b/storage/pebble/execution_data_tracker_test.go @@ -25,12 +25,16 @@ func randomCid() cid.Cid { func TestPrune(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) - delete(expectedPrunedCIDs, c) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) require.NoError(t, err) // c1 and c2 are for height 1, and c3 and c4 are for height 2 @@ -81,10 +85,14 @@ func TestPrune(t *testing.T) { // if that CID also exists at another height above the pruned height, the CID should not be pruned. func TestPruneNonLatestHeight(t *testing.T) { storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - assert.Fail(t, "unexpected CID pruned: %s", c.String()) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + assert.Fail(t, "unexpected CID pruned: %s", c.String()) + return nil + })) require.NoError(t, err) // c1 and c2 appear both at height 1 and 2 @@ -121,12 +129,16 @@ func TestPruneNonLatestHeight(t *testing.T) { func TestAscendingOrderOfRecords(t *testing.T) { expectedPrunedCIDs := make(map[cid.Cid]struct{}) storageDir := t.TempDir() - executionDataTracker, err := NewExecutionDataTracker(storageDir, 0, zerolog.Nop(), WithPruneCallback(func(c cid.Cid) error { - _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) - delete(expectedPrunedCIDs, c) - return nil - })) + executionDataTracker, err := NewExecutionDataTracker( + zerolog.Nop(), + storageDir, + 0, + WithPruneCallback(func(c cid.Cid) error { + _, ok := expectedPrunedCIDs[c] + assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + delete(expectedPrunedCIDs, c) + return nil + })) require.NoError(t, err) // c1 is for height 1, From 10bcd53eb84c2485e6ba703cac4ea84aad6949d2 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 11:28:16 +0300 Subject: [PATCH 17/33] Renamed to NoopExecutionDataTracker --- module/executiondatasync/provider/provider.go | 2 +- storage/util/noop.go | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/module/executiondatasync/provider/provider.go b/module/executiondatasync/provider/provider.go index 1b944e4ad1f..420d39acce1 100644 --- a/module/executiondatasync/provider/provider.go +++ b/module/executiondatasync/provider/provider.go @@ -53,7 +53,7 @@ func NewProvider( opts ...ProviderOption, ) *ExecutionDataProvider { if storage == nil { - storage = &util.NoopStorage{} + storage = &util.NoopExecutionDataTracker{} } p := &ExecutionDataProvider{ diff --git a/storage/util/noop.go b/storage/util/noop.go index d48b7a866d6..ea4dfd0d8c2 100644 --- a/storage/util/noop.go +++ b/storage/util/noop.go @@ -6,28 +6,28 @@ import ( "github.com/onflow/flow-go/storage" ) -type NoopStorage struct{} +type NoopExecutionDataTracker struct{} -var _ storage.ExecutionDataTracker = (*NoopStorage)(nil) +var _ storage.ExecutionDataTracker = (*NoopExecutionDataTracker)(nil) -func (s *NoopStorage) Update(update storage.UpdateFn) error { +func (s *NoopExecutionDataTracker) Update(update storage.UpdateFn) error { return update(func(blockHeight uint64, cids ...cid.Cid) error { return nil }) } -func (s *NoopStorage) GetFulfilledHeight() (uint64, error) { +func (s *NoopExecutionDataTracker) GetFulfilledHeight() (uint64, error) { return 0, nil } -func (s *NoopStorage) SetFulfilledHeight(uint64) error { +func (s *NoopExecutionDataTracker) SetFulfilledHeight(uint64) error { return nil } -func (s *NoopStorage) GetPrunedHeight() (uint64, error) { +func (s *NoopExecutionDataTracker) GetPrunedHeight() (uint64, error) { return 0, nil } -func (s *NoopStorage) PruneUpToHeight(height uint64) error { +func (s *NoopExecutionDataTracker) PruneUpToHeight(height uint64) error { return nil } From 72fcbb5cd2060c76079dcd7aaeeeff1f9d03a43f Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 11:38:30 +0300 Subject: [PATCH 18/33] Renamed to NewMockExecutionDataTracker --- .../computation/computer/computer_test.go | 18 +++++++++--------- .../computation/execution_verification_test.go | 2 +- .../computation/manager_benchmark_test.go | 2 +- engine/execution/computation/manager_test.go | 14 +++++++------- engine/execution/computation/programs_test.go | 4 ++-- engine/testutil/nodes.go | 2 +- engine/verification/utils/unittest/fixture.go | 2 +- fvm/fvm_bench_test.go | 2 +- .../provider/provider_test.go | 2 +- storage/mocks/util.go | 2 +- 10 files changed, 25 insertions(+), 25 deletions(-) diff --git a/engine/execution/computation/computer/computer_test.go b/engine/execution/computation/computer/computer_test.go index 1322023dda6..2ab3f0276dd 100644 --- a/engine/execution/computation/computer/computer_test.go +++ b/engine/execution/computation/computer/computer_test.go @@ -179,7 +179,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { Times(1) // 1 block bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -317,7 +317,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { committer := new(computermock.ViewCommitter) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -421,7 +421,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { comm := new(computermock.ViewCommitter) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -477,7 +477,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { committer := new(computermock.ViewCommitter) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -710,7 +710,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { vm := fvm.NewVirtualMachine() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -821,7 +821,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { vm := fvm.NewVirtualMachine() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -934,7 +934,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { vm := fvm.NewVirtualMachine() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -980,7 +980,7 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { bservice := requesterunit.MockBlobService( blockstore.NewBlockstore( dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -1297,7 +1297,7 @@ func Test_ExecutingSystemCollection(t *testing.T) { Return(nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), diff --git a/engine/execution/computation/execution_verification_test.go b/engine/execution/computation/execution_verification_test.go index 170e05e2780..950100980b6 100644 --- a/engine/execution/computation/execution_verification_test.go +++ b/engine/execution/computation/execution_verification_test.go @@ -753,7 +753,7 @@ func executeBlockAndVerifyWithParameters(t *testing.T, ledgerCommiter := committer.NewLedgerViewCommitter(ledger, tracer) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := exedataprovider.NewProvider( zerolog.Nop(), diff --git a/engine/execution/computation/manager_benchmark_test.go b/engine/execution/computation/manager_benchmark_test.go index dfdc942ce4c..de0424bd16d 100644 --- a/engine/execution/computation/manager_benchmark_test.go +++ b/engine/execution/computation/manager_benchmark_test.go @@ -180,7 +180,7 @@ func benchmarkComputeBlock( Return(nil, nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := exedataprovider.NewProvider( zerolog.Nop(), diff --git a/engine/execution/computation/manager_test.go b/engine/execution/computation/manager_test.go index e3d728ab7de..a8d282c1c3b 100644 --- a/engine/execution/computation/manager_test.go +++ b/engine/execution/computation/manager_test.go @@ -124,7 +124,7 @@ func TestComputeBlockWithStorage(t *testing.T) { Return(nil, nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -253,7 +253,7 @@ func TestExecuteScript(t *testing.T) { )) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -320,7 +320,7 @@ func TestExecuteScript_BalanceScriptFailsIfViewIsEmpty(t *testing.T) { )) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -366,7 +366,7 @@ func TestExecuteScripPanicsAreHandled(t *testing.T) { header := unittest.BlockHeaderFixture() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -416,7 +416,7 @@ func TestExecuteScript_LongScriptsAreLogged(t *testing.T) { header := unittest.BlockHeaderFixture() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -469,7 +469,7 @@ func TestExecuteScript_ShortScriptsAreNotLogged(t *testing.T) { header := unittest.BlockHeaderFixture() bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -819,7 +819,7 @@ func Test_EventEncodingFailsOnlyTxAndCarriesOn(t *testing.T) { Return(nil, nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), diff --git a/engine/execution/computation/programs_test.go b/engine/execution/computation/programs_test.go index 59446a6403e..efdb6702c00 100644 --- a/engine/execution/computation/programs_test.go +++ b/engine/execution/computation/programs_test.go @@ -118,7 +118,7 @@ func TestPrograms_TestContractUpdates(t *testing.T) { Return(nil, nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), @@ -231,7 +231,7 @@ func TestPrograms_TestBlockForks(t *testing.T) { Return(nil, nil) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 969027f4f36..8c11ab8b29b 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -670,7 +670,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity bootstrap.NodeInfo, ide committer := committer.NewLedgerViewCommitter(ls, node.Tracer) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := exedataprovider.NewProvider( zerolog.Nop(), diff --git a/engine/verification/utils/unittest/fixture.go b/engine/verification/utils/unittest/fixture.go index a9ca72a77d6..26546bfaaeb 100644 --- a/engine/verification/utils/unittest/fixture.go +++ b/engine/verification/utils/unittest/fixture.go @@ -274,7 +274,7 @@ func ExecutionResultFixture(t *testing.T, committer := committer.NewLedgerViewCommitter(led, trace.NewNoopTracer()) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), diff --git a/fvm/fvm_bench_test.go b/fvm/fvm_bench_test.go index 389fb2d7fd4..780b977f74a 100644 --- a/fvm/fvm_bench_test.go +++ b/fvm/fvm_bench_test.go @@ -208,7 +208,7 @@ func NewBasicBlockExecutor(tb testing.TB, chain flow.Chain, logger zerolog.Logge require.NoError(tb, err) bservice := requesterunit.MockBlobService(blockstore.NewBlockstore(dssync.MutexWrap(datastore.NewMapDatastore()))) - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() prov := provider.NewProvider( zerolog.Nop(), diff --git a/module/executiondatasync/provider/provider_test.go b/module/executiondatasync/provider/provider_test.go index 22e1b526477..a4f8978ab11 100644 --- a/module/executiondatasync/provider/provider_test.go +++ b/module/executiondatasync/provider/provider_test.go @@ -41,7 +41,7 @@ func getBlobservice(t *testing.T, ds datastore.Batching) network.BlobService { } func getProvider(blobService network.BlobService) provider.Provider { - trackerStorage := mocktracker.NewMockStorage() + trackerStorage := mocktracker.NewMockExecutionDataTracker() return provider.NewProvider( unittest.Logger(), diff --git a/storage/mocks/util.go b/storage/mocks/util.go index ff34e7da2d6..b0bc8ee47e5 100644 --- a/storage/mocks/util.go +++ b/storage/mocks/util.go @@ -8,7 +8,7 @@ import ( storagemock "github.com/onflow/flow-go/storage/mock" ) -func NewMockStorage() *storagemock.ExecutionDataTracker { +func NewMockExecutionDataTracker() *storagemock.ExecutionDataTracker { trackerStorage := new(storagemock.ExecutionDataTracker) trackerStorage.On("Update", mock.Anything).Return(func(fn storage.UpdateFn) error { return fn(func(uint64, ...cid.Cid) error { return nil }) From 2b609d4dabf9025204834b95321ba15fe8859f86 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 11:47:27 +0300 Subject: [PATCH 19/33] Added constant value for discardRatio, changed it according to comment --- storage/badger/execution_data_tracker.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index d4bd3bfd1f2..2d65cc2babd 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -14,6 +14,8 @@ import ( "github.com/onflow/flow-go/storage/badger/operation" ) +const defaultDiscardRatio = 0.25 + // getBatchItemCountLimit returns the maximum number of items that can be included in a single batch // transaction based on the number / total size of updates per item. func getBatchItemCountLimit(db *badger.DB, writeCountPerItem int64, writeSizePerItem int64) int { @@ -438,7 +440,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { } // this is a good time to do garbage collection - if err := s.db.RunValueLogGC(0.5); err != nil { + if err := s.db.RunValueLogGC(defaultDiscardRatio); err != nil { s.logger.Err(err).Msg("failed to run value log garbage collection") } From 9788c5847326b0f02b2b6f7a5342d786da67d10a Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 12:12:57 +0300 Subject: [PATCH 20/33] Updated execution data db prefixes to use fixed values, updated comment --- storage/execution_data_tracker.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/storage/execution_data_tracker.go b/storage/execution_data_tracker.go index 1d56e801227..5b442ee53e7 100644 --- a/storage/execution_data_tracker.go +++ b/storage/execution_data_tracker.go @@ -8,16 +8,17 @@ import ( "github.com/onflow/flow-go/module/blobs" ) -// key prefixes +// These key prefixes used exclusively in the execution data tracker database. +// They help categorize and manage different types of execution data. const ( - PrefixGlobalState byte = iota + 1 // global state variables - PrefixLatestHeight // tracks, for each blob, the latest height at which there exists a block whose execution data contains the blob - PrefixBlobRecord // tracks the set of blobs at each height + PrefixGlobalState byte = 1 // global state variables + PrefixLatestHeight byte = 2 // tracks, for each blob, the latest height at which there exists a block whose execution data contains the blob + PrefixBlobRecord byte = 3 // tracks the set of blobs at each height ) const ( - GlobalStateFulfilledHeight byte = iota + 1 // latest fulfilled block height - GlobalStatePrunedHeight // latest pruned block height + GlobalStateFulfilledHeight byte = 1 // latest fulfilled block height + GlobalStatePrunedHeight byte = 2 // latest pruned block height ) const BlobRecordKeyLength = 1 + 8 + blobs.CidLength From 481242bb0735d588b38184b6e85b8f09b0b8f9e0 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 13:43:45 +0300 Subject: [PATCH 21/33] Updated heights initialization for badger db --- storage/badger/execution_data_tracker.go | 22 +-------------- .../operation/execution_data_tracker.go | 27 ++++++++++++------- 2 files changed, 18 insertions(+), 31 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 2d65cc2babd..f3b8d43bcf2 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -131,7 +131,7 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { s.logger.Info().Msgf("finished pruning") } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { // db is empty, we need to bootstrap it - if err := s.bootstrap(startHeight); err != nil { + if err := s.db.Update(operation.InitTrackerHeights(startHeight)); err != nil { return fmt.Errorf("failed to bootstrap storage: %w", err) } } else { @@ -141,26 +141,6 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { return nil } -// bootstrap sets the initial fulfilled and pruned heights to startHeight in an empty database. -// -// Parameters: -// - startHeight: The initial height to set for both fulfilled and pruned heights. -// -// No errors are expected during normal operation. -func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { - err := s.db.Update(operation.InitTrackerFulfilledHeight(startHeight)) - if err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } - - err = s.db.Update(operation.InitTrackerPrunedHeight(startHeight)) - if err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil -} - // Update is used to track new blob CIDs. // It can be used to track blobs for both sealed and unsealed // heights, and the same blob may be added multiple times for diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index 5ee5bec50f3..a314090b249 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -1,22 +1,34 @@ package operation import ( + "fmt" + "github.com/dgraph-io/badger/v2" "github.com/ipfs/go-cid" "github.com/onflow/flow-go/storage" ) +// InitTrackerHeights initializes the fulfilled and the pruned heights for the execution data tracker storage. +// +// No errors are expected during normal operations. +func InitTrackerHeights(height uint64) func(*badger.Txn) error { + return func(tx *badger.Txn) error { + if err := insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height)(tx); err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + if err := insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height)(tx); err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + return nil + } +} + // UpdateTrackerFulfilledHeight updates the fulfilled height in the execution data tracker storage. func UpdateTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } -// InitTrackerFulfilledHeight initializes the fulfilled height for the execution data tracker storage. -func InitTrackerFulfilledHeight(height uint64) func(*badger.Txn) error { - return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) -} - // RetrieveTrackerFulfilledHeight retrieves the fulfilled height from the execution data tracker storage. func RetrieveTrackerFulfilledHeight(height *uint64) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) @@ -27,11 +39,6 @@ func UpdateTrackerPrunedHeight(height uint64) func(*badger.Txn) error { return update(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } -// InitTrackerPrunedHeight initializes the pruned height for the execution data tracker storage. -func InitTrackerPrunedHeight(height uint64) func(*badger.Txn) error { - return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) -} - // RetrieveTrackerPrunedHeight retrieves the pruned height from the execution data tracker storage. func RetrieveTrackerPrunedHeight(height *uint64) func(*badger.Txn) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) From ca68741ad7592337183c5dbc99f5bb3bcf159b44 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 15:54:05 +0300 Subject: [PATCH 22/33] Removed unnecessary option for pebble Set operation --- storage/pebble/operation/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/pebble/operation/common.go b/storage/pebble/operation/common.go index 48953137eaf..81d467c70b6 100644 --- a/storage/pebble/operation/common.go +++ b/storage/pebble/operation/common.go @@ -17,7 +17,7 @@ func insert(key []byte, val interface{}) func(pebble.Writer) error { return irrecoverable.NewExceptionf("failed to encode value: %w", err) } - err = w.Set(key, value, pebble.Sync) + err = w.Set(key, value, nil) if err != nil { return irrecoverable.NewExceptionf("failed to store data: %w", err) } From 302298017ca9263948185555b4125dcf7cc09ce7 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 17:29:30 +0300 Subject: [PATCH 23/33] Updated badger operations, removed no needed check in common retrieve for badger version, updated tests --- storage/badger/execution_data_tracker_test.go | 83 +++++++++++-------- storage/badger/operation/common.go | 5 -- .../operation/execution_data_tracker.go | 6 +- 3 files changed, 50 insertions(+), 44 deletions(-) diff --git a/storage/badger/execution_data_tracker_test.go b/storage/badger/execution_data_tracker_test.go index 2275d0ad9b1..7fe912673f0 100644 --- a/storage/badger/execution_data_tracker_test.go +++ b/storage/badger/execution_data_tracker_test.go @@ -7,7 +7,6 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/ipfs/go-cid" "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/module/blobs" @@ -32,7 +31,7 @@ func TestPrune(t *testing.T) { 0, WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + require.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) return nil })) @@ -57,29 +56,34 @@ func TestPrune(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) - assert.Len(t, expectedPrunedCIDs, 0) + require.Len(t, expectedPrunedCIDs, 0) var latestHeight uint64 + var exists bool err = executionDataTracker.db.View(func(txn *badger.Txn) error { - err = operation.RetrieveBlob(1, c1)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c1, &exists)(txn) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) - err = operation.RetrieveBlob(1, c2)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c2, &exists)(txn) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) - err = operation.RetrieveBlob(2, c3)(txn) - assert.NoError(t, err) + err = operation.BlobExist(2, c3, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(txn) - assert.NoError(t, err) - err = operation.RetrieveBlob(2, c4)(txn) - assert.NoError(t, err) + require.NoError(t, err) + err = operation.BlobExist(2, c4, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c4, &latestHeight)(txn) - assert.NoError(t, err) + require.NoError(t, err) return nil }) @@ -95,7 +99,7 @@ func TestPruneNonLatestHeight(t *testing.T) { storageDir, 0, WithPruneCallback(func(c cid.Cid) error { - assert.Fail(t, "unexpected CID pruned: %s", c.String()) + require.Fail(t, "unexpected CID pruned: %s", c.String()) return nil })) require.NoError(t, err) @@ -115,18 +119,21 @@ func TestPruneNonLatestHeight(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) var latestHeight uint64 + var exists bool err = executionDataTracker.db.View(func(txn *badger.Txn) error { - err = operation.RetrieveBlob(2, c1)(txn) - assert.NoError(t, err) + err = operation.BlobExist(2, c1, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) - assert.NoError(t, err) - err = operation.RetrieveBlob(2, c2)(txn) - assert.NoError(t, err) + require.NoError(t, err) + err = operation.BlobExist(2, c2, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) - assert.NoError(t, err) + require.NoError(t, err) return nil }) @@ -144,7 +151,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { 0, WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + require.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) return nil })) @@ -172,29 +179,33 @@ func TestAscendingOrderOfRecords(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) - assert.Len(t, expectedPrunedCIDs, 0) + require.Len(t, expectedPrunedCIDs, 0) var latestHeight uint64 + var exists bool err = executionDataTracker.db.View(func(txn *badger.Txn) error { // expected that blob record with height 1 was removed - err = operation.RetrieveBlob(1, c1)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c1, &exists)(txn) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(txn) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) // expected that blob record with height 2 exists - err = operation.RetrieveBlob(2, c2)(txn) - assert.NoError(t, err) + err = operation.BlobExist(2, c2, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(txn) - assert.NoError(t, err) + require.NoError(t, err) // expected that blob record with height 256 exists - err = operation.RetrieveBlob(256, c3)(txn) - assert.NoError(t, err) + err = operation.BlobExist(256, c3, &exists)(txn) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(txn) - assert.NoError(t, err) + require.NoError(t, err) return nil }) diff --git a/storage/badger/operation/common.go b/storage/badger/operation/common.go index 624fe28174a..1c293348231 100644 --- a/storage/badger/operation/common.go +++ b/storage/badger/operation/common.go @@ -253,11 +253,6 @@ func retrieve(key []byte, entity interface{}) func(*badger.Txn) error { return irrecoverable.NewExceptionf("could not load data: %w", err) } - // in case the value is not needed - if entity == nil { - return nil - } - // get the value from the item err = item.Value(func(val []byte) error { err := msgpack.Unmarshal(val, entity) diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index a314090b249..20b5b900d7a 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -64,9 +64,9 @@ func InsertBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } -// RetrieveBlob retrieves a blob record for the given block height and CID from the execution data tracker storage. -func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { - return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) +// BlobExist checks whether a blob record exists in the execution data tracker storage. +func BlobExist(blockHeight uint64, cid cid.Cid, blobExists *bool) func(*badger.Txn) error { + return exists(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), blobExists) } // RemoveBlob removes a blob record for the given block height and CID from the execution data tracker storage. From abd546b4d0c0402dc13f3cc6d2c4103b39171f98 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Thu, 8 Aug 2024 17:55:59 +0300 Subject: [PATCH 24/33] Updated pebble operations, removed no needed check in common retrieve for pebble version, updated tests --- storage/pebble/execution_data_tracker_test.go | 84 +++++++++++-------- storage/pebble/operation/common.go | 23 +++-- .../operation/execution_data_tracker.go | 6 +- 3 files changed, 69 insertions(+), 44 deletions(-) diff --git a/storage/pebble/execution_data_tracker_test.go b/storage/pebble/execution_data_tracker_test.go index 4febfcf8cc9..8290069e234 100644 --- a/storage/pebble/execution_data_tracker_test.go +++ b/storage/pebble/execution_data_tracker_test.go @@ -6,7 +6,6 @@ import ( "github.com/ipfs/go-cid" "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/module/blobs" @@ -31,7 +30,7 @@ func TestPrune(t *testing.T) { 0, WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + require.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) return nil })) @@ -56,29 +55,34 @@ func TestPrune(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) - assert.Len(t, expectedPrunedCIDs, 0) + require.Len(t, expectedPrunedCIDs, 0) var latestHeight uint64 + var exists bool - err = operation.RetrieveBlob(1, c1)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c1, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) - err = operation.RetrieveBlob(1, c2)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c2, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) - err = operation.RetrieveBlob(2, c3)(executionDataTracker.db) - assert.NoError(t, err) + err = operation.BlobExist(2, c3, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) - err = operation.RetrieveBlob(2, c4)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) + err = operation.BlobExist(2, c4, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c4, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) } // TestPruneNonLatestHeight test that when pruning a height at which a CID exists, @@ -90,7 +94,7 @@ func TestPruneNonLatestHeight(t *testing.T) { storageDir, 0, WithPruneCallback(func(c cid.Cid) error { - assert.Fail(t, "unexpected CID pruned: %s", c.String()) + require.Fail(t, "unexpected CID pruned: %s", c.String()) return nil })) require.NoError(t, err) @@ -110,18 +114,21 @@ func TestPruneNonLatestHeight(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) var latestHeight uint64 + var exists bool - err = operation.RetrieveBlob(2, c1)(executionDataTracker.db) - assert.NoError(t, err) + err = operation.BlobExist(2, c1, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) - err = operation.RetrieveBlob(2, c2)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) + err = operation.BlobExist(2, c2, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) } // TestAscendingOrderOfRecords tests that order of data is ascending and all CIDs appearing at or below the pruned @@ -135,7 +142,7 @@ func TestAscendingOrderOfRecords(t *testing.T) { 0, WithPruneCallback(func(c cid.Cid) error { _, ok := expectedPrunedCIDs[c] - assert.True(t, ok, "unexpected CID pruned: %s", c.String()) + require.True(t, ok, "unexpected CID pruned: %s", c.String()) delete(expectedPrunedCIDs, c) return nil })) @@ -163,26 +170,31 @@ func TestAscendingOrderOfRecords(t *testing.T) { prunedHeight, err := executionDataTracker.GetPrunedHeight() require.NoError(t, err) - assert.Equal(t, uint64(1), prunedHeight) + require.Equal(t, uint64(1), prunedHeight) - assert.Len(t, expectedPrunedCIDs, 0) + require.Len(t, expectedPrunedCIDs, 0) var latestHeight uint64 + var exists bool + // expected that blob record with height 1 was removed - err = operation.RetrieveBlob(1, c1)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) + err = operation.BlobExist(1, c1, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.False(t, exists) err = operation.RetrieveTrackerLatestHeight(c1, &latestHeight)(executionDataTracker.db) - assert.ErrorIs(t, err, storage.ErrNotFound) + require.ErrorIs(t, err, storage.ErrNotFound) // expected that blob record with height 2 exists - err = operation.RetrieveBlob(2, c2)(executionDataTracker.db) - assert.NoError(t, err) + err = operation.BlobExist(2, c2, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c2, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) // expected that blob record with height 256 exists - err = operation.RetrieveBlob(256, c3)(executionDataTracker.db) - assert.NoError(t, err) + err = operation.BlobExist(256, c3, &exists)(executionDataTracker.db) + require.NoError(t, err) + require.True(t, exists) err = operation.RetrieveTrackerLatestHeight(c3, &latestHeight)(executionDataTracker.db) - assert.NoError(t, err) + require.NoError(t, err) } diff --git a/storage/pebble/operation/common.go b/storage/pebble/operation/common.go index 81d467c70b6..ffd2ccb5861 100644 --- a/storage/pebble/operation/common.go +++ b/storage/pebble/operation/common.go @@ -34,11 +34,6 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } defer closer.Close() - // in case the value is not needed - if sc == nil { - return nil - } - err = msgpack.Unmarshal(val, sc) if err != nil { return irrecoverable.NewExceptionf("failed to decode value: %w", err) @@ -47,6 +42,24 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } } +func exists(key []byte, keyExists *bool) func(r pebble.Reader) error { + return func(r pebble.Reader) error { + _, closer, err := r.Get(key) + if err != nil { + if errors.Is(err, pebble.ErrNotFound) { + *keyExists = false + return nil + } + + // exception while checking for the key + return irrecoverable.NewExceptionf("could not load data: %w", err) + } + *keyExists = true + defer closer.Close() + return nil + } +} + // remove removes the entity with the given key, if it exists. If it doesn't // exist, this is a no-op. // Error returns: diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index 3e99ee2354c..fc47f32e3d7 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -57,9 +57,9 @@ func InsertBlob(blockHeight uint64, cid cid.Cid) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) } -// RetrieveBlob retrieves a blob record for the given block height and CID from the execution data tracker storage. -func RetrieveBlob(blockHeight uint64, cid cid.Cid) func(r pebble.Reader) error { - return retrieve(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), nil) +// BlobExist checks whether a blob record exists in the execution data tracker storage. +func BlobExist(blockHeight uint64, cid cid.Cid, blobExists *bool) func(r pebble.Reader) error { + return exists(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), blobExists) } // RemoveBlob removes a blob record for the given block height and CID from the execution data tracker storage. From d6f8b9ab7d6f2d10a57be71b4383812b8cb99d79 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Fri, 9 Aug 2024 17:04:52 +0300 Subject: [PATCH 25/33] Updated badger version of batchDelete --- storage/badger/execution_data_tracker.go | 18 +++++++++++++----- .../badger/operation/execution_data_tracker.go | 14 ++++++++------ 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index f3b8d43bcf2..cd74fc5f1be 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -296,22 +296,30 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e // // Ensures that all specified blobs are deleted from the storage. // -// Returns an error if the batch deletion fails. +// No errors are expected during normal operation. func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { + batch := NewBatch(s.db) + for _, dInfo := range deleteInfos { - err := s.db.Update(operation.RemoveBlob(dInfo.Height, dInfo.Cid)) + writeBatch := batch.GetWriter() + err := operation.BatchRemoveBlob(dInfo.Height, dInfo.Cid)(writeBatch) if err != nil { - return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.Cid.String(), err) + return fmt.Errorf("cannot batch remove blob: %w", err) } if dInfo.DeleteLatestHeightRecord { - err = s.db.Update(operation.RemoveTrackerLatestHeight(dInfo.Cid)) + err = operation.BatchRemoveTrackerLatestHeight(dInfo.Cid)(writeBatch) if err != nil { - return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.Cid.String(), err) + return fmt.Errorf("cannot batch remove latest height record: %w", err) } } } + err := batch.Flush() + if err != nil { + return fmt.Errorf("cannot flush batch to remove execution data: %w", err) + } + return nil } diff --git a/storage/badger/operation/execution_data_tracker.go b/storage/badger/operation/execution_data_tracker.go index 20b5b900d7a..752b6573328 100644 --- a/storage/badger/operation/execution_data_tracker.go +++ b/storage/badger/operation/execution_data_tracker.go @@ -54,9 +54,10 @@ func RetrieveTrackerLatestHeight(cid cid.Cid, height *uint64) func(*badger.Txn) return retrieve(makePrefix(storage.PrefixLatestHeight, cid), height) } -// RemoveTrackerLatestHeight removes the latest height for the given CID from the execution data tracker storage. -func RemoveTrackerLatestHeight(cid cid.Cid) func(*badger.Txn) error { - return remove(makePrefix(storage.PrefixLatestHeight, cid)) +// BatchRemoveTrackerLatestHeight removes the latest height for the given CID in provided batch from the execution data +// tracker storage. +func BatchRemoveTrackerLatestHeight(cid cid.Cid) func(batch *badger.WriteBatch) error { + return batchRemove(makePrefix(storage.PrefixLatestHeight, cid)) } // InsertBlob inserts a blob record for the given block height and CID into the execution data tracker storage. @@ -69,7 +70,8 @@ func BlobExist(blockHeight uint64, cid cid.Cid, blobExists *bool) func(*badger.T return exists(makePrefix(storage.PrefixBlobRecord, blockHeight, cid), blobExists) } -// RemoveBlob removes a blob record for the given block height and CID from the execution data tracker storage. -func RemoveBlob(blockHeight uint64, cid cid.Cid) func(*badger.Txn) error { - return remove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) +// BatchRemoveBlob removes a blob record for the given block height and CID in provided batch from the execution data +// tracker storage. +func BatchRemoveBlob(blockHeight uint64, cid cid.Cid) func(batch *badger.WriteBatch) error { + return batchRemove(makePrefix(storage.PrefixBlobRecord, blockHeight, cid)) } From 34e04eec57c1bfa33cf840f8b107cc507bcf6c24 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 13:13:24 +0300 Subject: [PATCH 26/33] Updated pebble version of execution data tracker by using batch according to comments --- storage/batch.go | 11 +- storage/pebble/execution_data_tracker.go | 109 ++++++++---------- storage/pebble/operation/common.go | 67 +++++++++++ .../operation/execution_data_tracker.go | 27 +++-- 4 files changed, 146 insertions(+), 68 deletions(-) diff --git a/storage/batch.go b/storage/batch.go index 3147fc5c0e7..f1dc4f37af4 100644 --- a/storage/batch.go +++ b/storage/batch.go @@ -1,6 +1,9 @@ package storage -import "github.com/dgraph-io/badger/v2" +import ( + "github.com/cockroachdb/pebble" + "github.com/dgraph-io/badger/v2" +) type Transaction interface { Set(key, val []byte) error @@ -20,3 +23,9 @@ type BatchStorage interface { // Flush will flush the write batch and update the cache. Flush() error } + +type PebbleReaderBatchWriter interface { + ReaderWriter() (pebble.Reader, pebble.Writer) + IndexedBatch() *pebble.Batch + AddCallback(func()) +} diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 2b3979d306a..379766b9937 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -121,8 +121,8 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { } s.logger.Info().Msgf("finished pruning") } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { - // db is empty, need to bootstrap it - if err := s.bootstrap(startHeight); err != nil { + // db is empty, need to initialize it + if err := operation.WithReaderBatchWriter(s.db, operation.InitTrackerHeights(startHeight)); err != nil { return fmt.Errorf("failed to bootstrap storage: %w", err) } } else { @@ -132,26 +132,6 @@ func (s *ExecutionDataTracker) init(startHeight uint64) error { return nil } -// bootstrap sets the initial fulfilled and pruned heights to startHeight in an empty database. -// -// Parameters: -// - startHeight: The initial height to set for both fulfilled and pruned heights. -// -// No errors are expected during normal operation. -func (s *ExecutionDataTracker) bootstrap(startHeight uint64) error { - err := operation.InitTrackerFulfilledHeight(startHeight)(s.db) - if err != nil { - return fmt.Errorf("failed to set fulfilled height value: %w", err) - } - - err = operation.InitTrackerPrunedHeight(startHeight)(s.db) - if err != nil { - return fmt.Errorf("failed to set pruned height value: %w", err) - } - - return nil -} - // Update is used to track new blob CIDs. // It can be used to track blobs for both sealed and unsealed // heights, and the same blob may be added multiple times for @@ -233,31 +213,34 @@ func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { // - c: The CID of the blob to be tracked. // // No errors are expected during normal operation. -func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { - err := operation.InsertBlob(blockHeight, c)(s.db) - if err != nil { - return fmt.Errorf("failed to add blob record: %w", err) - } +func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) func(tx storage.PebbleReaderBatchWriter) error { + return func(tx storage.PebbleReaderBatchWriter) error { + r, w := tx.ReaderWriter() + err := operation.InsertBlob(blockHeight, c)(w) + if err != nil { + return fmt.Errorf("failed to add blob record: %w", err) + } - var latestHeight uint64 - err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(s.db) - if err != nil { - if !errors.Is(err, storage.ErrNotFound) { - return fmt.Errorf("failed to get latest height: %w", err) + var latestHeight uint64 + err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(r) + if err != nil { + if !errors.Is(err, storage.ErrNotFound) { + return fmt.Errorf("failed to get latest height: %w", err) + } + } else { + // don't update the latest height if there is already a higher block height containing this blob + if latestHeight >= blockHeight { + return nil + } } - } else { - // don't update the latest height if there is already a higher block height containing this blob - if latestHeight >= blockHeight { - return nil + + err = operation.UpsertTrackerLatestHeight(c, blockHeight)(w) + if err != nil { + return fmt.Errorf("failed to set latest height value: %w", err) } - } - err = operation.UpsertTrackerLatestHeight(c, blockHeight)(s.db) - if err != nil { - return fmt.Errorf("failed to set latest height value: %w", err) + return nil } - - return nil } // trackBlobs tracks multiple blob CIDs at the specified block height. @@ -269,11 +252,15 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) error { // No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { if len(cids) > 0 { - for _, c := range cids { - if err := s.trackBlob(blockHeight, c); err != nil { - return fmt.Errorf("failed to track blob %s: %w", c.String(), err) + return operation.WithReaderBatchWriter(s.db, func(tx storage.PebbleReaderBatchWriter) error { + for _, c := range cids { + if err := s.trackBlob(blockHeight, c)(tx); err != nil { + return fmt.Errorf("failed to track blob %s: %w", c.String(), err) + } } - } + + return nil + }) } return nil @@ -286,22 +273,26 @@ func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) e // whether to delete the latest height record. // // No errors are expected during normal operation. -func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) error { - for _, dInfo := range deleteInfos { - err := operation.RemoveBlob(dInfo.Height, dInfo.Cid)(s.db) - if err != nil { - return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.Cid.String(), err) - } +func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) func(storage.PebbleReaderBatchWriter) error { + return func(tx storage.PebbleReaderBatchWriter) error { + _, w := tx.ReaderWriter() - if dInfo.DeleteLatestHeightRecord { - err = operation.RemoveTrackerLatestHeight(dInfo.Cid)(s.db) + for _, dInfo := range deleteInfos { + err := operation.RemoveBlob(dInfo.Height, dInfo.Cid)(w) if err != nil { - return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.Cid.String(), err) + return fmt.Errorf("failed to delete blob record for Cid %s: %w", dInfo.Cid.String(), err) + } + + if dInfo.DeleteLatestHeightRecord { + err = operation.RemoveTrackerLatestHeight(dInfo.Cid)(w) + if err != nil { + return fmt.Errorf("failed to delete latest height record for Cid %s: %w", dInfo.Cid.String(), err) + } } } - } - return nil + return nil + } } // PruneUpToHeight removes all data from storage corresponding @@ -384,7 +375,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { // remove tracker records for pruned heights batch = append(batch, dInfo) if len(batch) == itemsPerBatch { - if err := s.batchDelete(batch); err != nil { + if err := operation.WithReaderBatchWriter(s.db, s.batchDelete(batch)); err != nil { return err } batch = nil @@ -392,7 +383,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { } if len(batch) > 0 { - if err := s.batchDelete(batch); err != nil { + if err := operation.WithReaderBatchWriter(s.db, s.batchDelete(batch)); err != nil { return err } } diff --git a/storage/pebble/operation/common.go b/storage/pebble/operation/common.go index ffd2ccb5861..9cda1f77254 100644 --- a/storage/pebble/operation/common.go +++ b/storage/pebble/operation/common.go @@ -10,6 +10,70 @@ import ( "github.com/onflow/flow-go/storage" ) +// ReaderBatchWriter is a struct that manages batch operations in a Pebble database. +// It allows for combining multiple database writes into a single batch and provides +// a way to execute callbacks upon batch commit. +type ReaderBatchWriter struct { + db *pebble.DB + batch *pebble.Batch + callbacks []func() +} + +var _ storage.PebbleReaderBatchWriter = (*ReaderBatchWriter)(nil) + +// ReaderWriter returns the Pebble reader and writer. +// This method provides access to the database and the batch for read-write operations. +func (b *ReaderBatchWriter) ReaderWriter() (pebble.Reader, pebble.Writer) { + return b.db, b.batch +} + +// IndexedBatch returns the current batch for indexed write operations. +func (b *ReaderBatchWriter) IndexedBatch() *pebble.Batch { + return b.batch +} + +// Commit commits the current batch to the database. +// This method finalizes the batch operations and persists them to the database. +func (b *ReaderBatchWriter) Commit() error { + return b.batch.Commit(nil) +} + +// AddCallback adds a callback function to be executed after the batch is committed. +// The callbacks are executed in the order they are added. +func (b *ReaderBatchWriter) AddCallback(callback func()) { + b.callbacks = append(b.callbacks, callback) +} + +// NewPebbleReaderBatchWriter creates a new ReaderBatchWriter for the given Pebble database. +// This function initializes the ReaderBatchWriter with a new indexed batch. +func NewPebbleReaderBatchWriter(db *pebble.DB) *ReaderBatchWriter { + return &ReaderBatchWriter{ + db: db, + batch: db.NewIndexedBatch(), + } +} + +// WithReaderBatchWriter executes a function with a ReaderBatchWriter and commits the batch. +// This function ensures that the batch is committed and callbacks are executed. +func WithReaderBatchWriter(db *pebble.DB, fn func(storage.PebbleReaderBatchWriter) error) error { + batch := NewPebbleReaderBatchWriter(db) + err := fn(batch) + if err != nil { + return err + } + err = batch.Commit() + if err != nil { + return err + } + + for _, callback := range batch.callbacks { + callback() + } + + return nil +} + +// insert returns a function that inserts a key-value pair into the Pebble database. func insert(key []byte, val interface{}) func(pebble.Writer) error { return func(w pebble.Writer) error { value, err := msgpack.Marshal(val) @@ -26,6 +90,7 @@ func insert(key []byte, val interface{}) func(pebble.Writer) error { } } +// retrieve returns a function that retrieves a value from the Pebble database by key. func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { return func(r pebble.Reader) error { val, closer, err := r.Get(key) @@ -42,6 +107,8 @@ func retrieve(key []byte, sc interface{}) func(r pebble.Reader) error { } } +// exists returns a function that checks whether a key exists in the Pebble database. +// It sets the provided boolean pointer to true if the key exists, or false otherwise. func exists(key []byte, keyExists *bool) func(r pebble.Reader) error { return func(r pebble.Reader) error { _, closer, err := r.Get(key) diff --git a/storage/pebble/operation/execution_data_tracker.go b/storage/pebble/operation/execution_data_tracker.go index fc47f32e3d7..4aea35025a0 100644 --- a/storage/pebble/operation/execution_data_tracker.go +++ b/storage/pebble/operation/execution_data_tracker.go @@ -1,6 +1,8 @@ package operation import ( + "fmt" + "github.com/cockroachdb/pebble" "github.com/ipfs/go-cid" @@ -12,9 +14,23 @@ func UpdateTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) } -// InitTrackerFulfilledHeight initializes the fulfilled height for the execution data tracker storage. -func InitTrackerFulfilledHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height) +// InitTrackerHeights initializes the fulfilled and pruned heights for the execution data tracker storage. +func InitTrackerHeights(height uint64) func(storage.PebbleReaderBatchWriter) error { + return func(tx storage.PebbleReaderBatchWriter) error { + _, w := tx.ReaderWriter() + + err := insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStateFulfilledHeight), height)(w) + if err != nil { + return fmt.Errorf("failed to set fulfilled height value: %w", err) + } + + err = insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height)(w) + if err != nil { + return fmt.Errorf("failed to set pruned height value: %w", err) + } + + return nil + } } // RetrieveTrackerFulfilledHeight retrieves the fulfilled height from the execution data tracker storage. @@ -27,11 +43,6 @@ func UpdateTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) } -// InitTrackerPrunedHeight initializes the pruned height for the execution data tracker storage. -func InitTrackerPrunedHeight(height uint64) func(w pebble.Writer) error { - return insert(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) -} - // RetrieveTrackerPrunedHeight retrieves the pruned height from the execution data tracker storage. func RetrieveTrackerPrunedHeight(height *uint64) func(r pebble.Reader) error { return retrieve(makePrefix(storage.PrefixGlobalState, storage.GlobalStatePrunedHeight), height) From 3b8509df656d296c94cca156256440ae70a767af Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 13:23:46 +0300 Subject: [PATCH 27/33] Fixed godoc, updated logger values --- storage/badger/execution_data_tracker.go | 4 ++-- storage/pebble/execution_data_tracker.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index cd74fc5f1be..1ab55bef9f0 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -66,7 +66,7 @@ func WithPruneCallback(callback storage.PruneCallback) StorageOption { // // Parameters: // - logger: The logger for logging tracker operations. -// - path: The file path for the underlying Pebble database. +// - path: The file path for the underlying Badger database. // - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. // - opts: Additional configuration options such as custom prune callbacks. // @@ -77,7 +77,7 @@ func NewExecutionDataTracker( startHeight uint64, opts ...StorageOption, ) (*ExecutionDataTracker, error) { - lg := logger.With().Str("module", "tracker_storage").Logger() + lg := logger.With().Str("module", "badger_storage_tracker").Logger() db, err := badger.Open(badger.LSMOnlyOptions(path)) if err != nil { return nil, fmt.Errorf("could not open tracker db: %w", err) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 379766b9937..718a11e3bf2 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -67,7 +67,7 @@ func NewExecutionDataTracker( startHeight uint64, opts ...StorageOption, ) (*ExecutionDataTracker, error) { - lg := logger.With().Str("module", "tracker_storage").Logger() + lg := logger.With().Str("module", "pebble_storage_tracker").Logger() db, err := pebble.Open(path, nil) if err != nil { return nil, fmt.Errorf("could not open db: %w", err) From b8181abe772761bdfb12443a555eec1659792ff6 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 13:37:59 +0300 Subject: [PATCH 28/33] Updated trackBlob according to comment --- storage/badger/execution_data_tracker.go | 6 ++---- storage/pebble/execution_data_tracker.go | 6 ++---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 1ab55bef9f0..17cf678b1c6 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -235,11 +235,9 @@ func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c c if !errors.Is(err, storage.ErrNotFound) { return fmt.Errorf("failed to get latest height: %w", err) } - } else { + } else if latestHeight >= blockHeight { // don't update the latest height if there is already a higher block height containing this blob - if latestHeight >= blockHeight { - return nil - } + return nil } err = operation.UpsertTrackerLatestHeight(c, blockHeight)(tx) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 718a11e3bf2..78adff60c93 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -227,11 +227,9 @@ func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) func(tx if !errors.Is(err, storage.ErrNotFound) { return fmt.Errorf("failed to get latest height: %w", err) } - } else { + } else if latestHeight >= blockHeight { // don't update the latest height if there is already a higher block height containing this blob - if latestHeight >= blockHeight { - return nil - } + return nil } err = operation.UpsertTrackerLatestHeight(c, blockHeight)(w) From 640de213ccf7eb4d33e550b83375f506edf2fc3f Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 14:44:06 +0300 Subject: [PATCH 29/33] Separated read operation from write batch operations --- storage/pebble/execution_data_tracker.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 78adff60c93..8ae2b815637 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -215,14 +215,14 @@ func (s *ExecutionDataTracker) GetPrunedHeight() (uint64, error) { // No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlob(blockHeight uint64, c cid.Cid) func(tx storage.PebbleReaderBatchWriter) error { return func(tx storage.PebbleReaderBatchWriter) error { - r, w := tx.ReaderWriter() + _, w := tx.ReaderWriter() err := operation.InsertBlob(blockHeight, c)(w) if err != nil { return fmt.Errorf("failed to add blob record: %w", err) } var latestHeight uint64 - err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(r) + err = operation.RetrieveTrackerLatestHeight(c, &latestHeight)(s.db) if err != nil { if !errors.Is(err, storage.ErrNotFound) { return fmt.Errorf("failed to get latest height: %w", err) From 65e16146a00937f0fe775787fd852bee6ffd9310 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 14:58:36 +0300 Subject: [PATCH 30/33] Moved common logic for maximum number of items in a single batch for badger db --- storage/badger/execution_data_tracker.go | 25 ++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index 17cf678b1c6..d5315460157 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -256,11 +256,7 @@ func (s *ExecutionDataTracker) trackBlob(tx *badger.Txn, blockHeight uint64, c c // // No errors are expected during normal operation. func (s *ExecutionDataTracker) trackBlobs(blockHeight uint64, cids ...cid.Cid) error { - cidsPerBatch := storage.CidsPerBatch - maxCidsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength+8) - if maxCidsPerBatch < cidsPerBatch { - cidsPerBatch = maxCidsPerBatch - } + cidsPerBatch := s.batchItemLimit(storage.CidsPerBatch, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength+8) for len(cids) > 0 { batchSize := cidsPerBatch @@ -321,10 +317,19 @@ func (s *ExecutionDataTracker) batchDelete(deleteInfos []*storage.DeleteInfo) er return nil } -// batchDeleteItemLimit determines the maximum number of items that can be deleted in a batch operation. -func (s *ExecutionDataTracker) batchDeleteItemLimit() int { - itemsPerBatch := storage.DeleteItemsPerBatch - maxItemsPerBatch := getBatchItemCountLimit(s.db, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength) +// batchItemLimit returns the maximum number of items that can +// be processed in a single batch operation based on the limits. +// +// Parameters: +// - itemsPerBatch: The initial number of items to process in a single batch. +// - writeCountPerItem: The number of write operations required per item. +// - writeSizePerItem: The size in bytes of each item to be written. +func (s *ExecutionDataTracker) batchItemLimit( + itemsPerBatch int, + writeCountPerItem int64, + writeSizePerItem int64, +) int { + maxItemsPerBatch := getBatchItemCountLimit(s.db, writeCountPerItem, writeSizePerItem) if maxItemsPerBatch < itemsPerBatch { itemsPerBatch = maxItemsPerBatch } @@ -342,7 +347,7 @@ func (s *ExecutionDataTracker) batchDeleteItemLimit() int { // No errors are expected during normal operation. func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { blobRecordPrefix := []byte{storage.PrefixBlobRecord} - itemsPerBatch := s.batchDeleteItemLimit() + itemsPerBatch := s.batchItemLimit(storage.DeleteItemsPerBatch, 2, storage.BlobRecordKeyLength+storage.LatestHeightKeyLength) var batch []*storage.DeleteInfo s.mu.Lock() From feb984d7c0b17e2f3d0f9d7e9a521c77d6f4a75d Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 16:12:27 +0300 Subject: [PATCH 31/33] Updated naming --- storage/pebble/execution_data_tracker.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index 8ae2b815637..b36d6e1a9a0 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -314,13 +314,13 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { return err } - err := func(tx pebble.Reader) error { + err := func(r pebble.Reader) error { options := pebble.IterOptions{ LowerBound: blobRecordPrefix, UpperBound: append(blobRecordPrefix, ffBytes...), } - it, err := tx.NewIter(&options) + it, err := r.NewIter(&options) if err != nil { return fmt.Errorf("can not create iterator: %w", err) } @@ -347,7 +347,7 @@ func (s *ExecutionDataTracker) PruneUpToHeight(height uint64) error { } var latestHeight uint64 - err = operation.RetrieveTrackerLatestHeight(blobCid, &latestHeight)(s.db) + err = operation.RetrieveTrackerLatestHeight(blobCid, &latestHeight)(r) if err != nil { return fmt.Errorf("failed to get latest height entry for Cid %s: %w", blobCid.String(), err) } From b12dc9b62b39aba0a9f7a3f94ad029a5a08ca100 Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Tue, 13 Aug 2024 17:55:16 +0300 Subject: [PATCH 32/33] Moved common logic for tracker initialization --- storage/badger/execution_data_tracker.go | 48 +++++----------------- storage/execution_data_tracker.go | 51 ++++++++++++++++++++++++ storage/pebble/execution_data_tracker.go | 49 +++++------------------ 3 files changed, 69 insertions(+), 79 deletions(-) diff --git a/storage/badger/execution_data_tracker.go b/storage/badger/execution_data_tracker.go index d5315460157..d705f595981 100644 --- a/storage/badger/execution_data_tracker.go +++ b/storage/badger/execution_data_tracker.go @@ -6,7 +6,6 @@ import ( "sync" "github.com/dgraph-io/badger/v2" - "github.com/hashicorp/go-multierror" "github.com/ipfs/go-cid" "github.com/rs/zerolog" @@ -95,7 +94,15 @@ func NewExecutionDataTracker( lg.Info().Msgf("initialize tracker with start height: %d", startHeight) - if err := tracker.init(startHeight); err != nil { + err = storage.InitTracker( + tracker, + lg, + startHeight, + func(startHeight uint64) error { + return db.Update(operation.InitTrackerHeights(startHeight)) + }, + ) + if err != nil { return nil, fmt.Errorf("failed to initialize tracker: %w", err) } @@ -104,43 +111,6 @@ func NewExecutionDataTracker( return tracker, nil } -// init initializes the ExecutionDataTracker by setting the fulfilled and pruned heights. -// -// Parameters: -// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. -// -// No errors are expected during normal operation. -func (s *ExecutionDataTracker) init(startHeight uint64) error { - fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() - prunedHeight, prunedHeightErr := s.GetPrunedHeight() - - if fulfilledHeightErr == nil && prunedHeightErr == nil { - if prunedHeight > fulfilledHeight { - return fmt.Errorf( - "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", - prunedHeight, - fulfilledHeight, - ) - } - - s.logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) - // replay pruning in case it was interrupted during previous shutdown - if err := s.PruneUpToHeight(prunedHeight); err != nil { - return fmt.Errorf("failed to replay pruning: %w", err) - } - s.logger.Info().Msgf("finished pruning") - } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { - // db is empty, we need to bootstrap it - if err := s.db.Update(operation.InitTrackerHeights(startHeight)); err != nil { - return fmt.Errorf("failed to bootstrap storage: %w", err) - } - } else { - return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() - } - - return nil -} - // Update is used to track new blob CIDs. // It can be used to track blobs for both sealed and unsealed // heights, and the same blob may be added multiple times for diff --git a/storage/execution_data_tracker.go b/storage/execution_data_tracker.go index 5b442ee53e7..651cc5252cb 100644 --- a/storage/execution_data_tracker.go +++ b/storage/execution_data_tracker.go @@ -2,8 +2,12 @@ package storage import ( "encoding/binary" + "errors" + "fmt" + "github.com/hashicorp/go-multierror" "github.com/ipfs/go-cid" + "github.com/rs/zerolog" "github.com/onflow/flow-go/module/blobs" ) @@ -100,3 +104,50 @@ type DeleteInfo struct { Height uint64 DeleteLatestHeightRecord bool } + +// InitTracker initializes an ExecutionDataTracker. +// +// This function is shared by multiple tracker implementations (e.g., Badger-based and Pebble-based) to avoid code duplication. +// +// Parameters: +// - tracker: The ExecutionDataTracker implementation being initialized. +// - logger: The logger for logging tracker operations. +// - startHeight: The initial height to be set if no previous tracker heights are found. +// - updateHeightsFn: A function that initializes the tracker heights in the database if they are not found. +// +// Returns an error if any inconsistency is detected or if there is a failure in the initialization process. +func InitTracker( + tracker ExecutionDataTracker, + logger zerolog.Logger, + startHeight uint64, + initHeightsFn func(startHeight uint64) error, +) error { + fulfilledHeight, fulfilledHeightErr := tracker.GetFulfilledHeight() + prunedHeight, prunedHeightErr := tracker.GetPrunedHeight() + + if fulfilledHeightErr == nil && prunedHeightErr == nil { + if prunedHeight > fulfilledHeight { + return fmt.Errorf( + "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", + prunedHeight, + fulfilledHeight, + ) + } + + logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) + // replay pruning in case it was interrupted during previous shutdown + if err := tracker.PruneUpToHeight(prunedHeight); err != nil { + return fmt.Errorf("failed to replay pruning: %w", err) + } + logger.Info().Msgf("finished pruning") + } else if errors.Is(fulfilledHeightErr, ErrNotFound) && errors.Is(prunedHeightErr, ErrNotFound) { + // db is empty, need to initialize it + if err := initHeightsFn(startHeight); err != nil { + return fmt.Errorf("failed to bootstrap storage: %w", err) + } + } else { + return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() + } + + return nil +} diff --git a/storage/pebble/execution_data_tracker.go b/storage/pebble/execution_data_tracker.go index b36d6e1a9a0..b754286797f 100644 --- a/storage/pebble/execution_data_tracker.go +++ b/storage/pebble/execution_data_tracker.go @@ -7,7 +7,6 @@ import ( "sync" "github.com/cockroachdb/pebble" - "github.com/hashicorp/go-multierror" "github.com/ipfs/go-cid" "github.com/rs/zerolog" @@ -85,7 +84,15 @@ func NewExecutionDataTracker( lg.Info().Msgf("initialize tracker with start height: %d", startHeight) - if err := tracker.init(startHeight); err != nil { + err = storage.InitTracker( + tracker, + lg, + startHeight, + func(startHeight uint64) error { + return operation.WithReaderBatchWriter(db, operation.InitTrackerHeights(startHeight)) + }, + ) + if err != nil { return nil, fmt.Errorf("failed to initialize tracker: %w", err) } @@ -94,44 +101,6 @@ func NewExecutionDataTracker( return tracker, nil } -// TODO: move common logic into separate function to avoid duplication of code -// init initializes the ExecutionDataTracker by setting the fulfilled and pruned heights. -// -// Parameters: -// - startHeight: The initial fulfilled height to be set if no previous fulfilled height is found. -// -// No errors are expected during normal operation. -func (s *ExecutionDataTracker) init(startHeight uint64) error { - fulfilledHeight, fulfilledHeightErr := s.GetFulfilledHeight() - prunedHeight, prunedHeightErr := s.GetPrunedHeight() - - if fulfilledHeightErr == nil && prunedHeightErr == nil { - if prunedHeight > fulfilledHeight { - return fmt.Errorf( - "inconsistency detected: pruned height (%d) is greater than fulfilled height (%d)", - prunedHeight, - fulfilledHeight, - ) - } - - s.logger.Info().Msgf("prune from height %v up to height %d", fulfilledHeight, prunedHeight) - // replay pruning in case it was interrupted during previous shutdown - if err := s.PruneUpToHeight(prunedHeight); err != nil { - return fmt.Errorf("failed to replay pruning: %w", err) - } - s.logger.Info().Msgf("finished pruning") - } else if errors.Is(fulfilledHeightErr, storage.ErrNotFound) && errors.Is(prunedHeightErr, storage.ErrNotFound) { - // db is empty, need to initialize it - if err := operation.WithReaderBatchWriter(s.db, operation.InitTrackerHeights(startHeight)); err != nil { - return fmt.Errorf("failed to bootstrap storage: %w", err) - } - } else { - return multierror.Append(fulfilledHeightErr, prunedHeightErr).ErrorOrNil() - } - - return nil -} - // Update is used to track new blob CIDs. // It can be used to track blobs for both sealed and unsealed // heights, and the same blob may be added multiple times for From 95ac8ae24a6897afc9f5ca45fecf4839ddf9d31f Mon Sep 17 00:00:00 2001 From: UlyanaAndrukhiv Date: Wed, 14 Aug 2024 18:09:45 +0300 Subject: [PATCH 33/33] Added functional tests for common pebble operations --- storage/pebble/operation/common_test.go | 275 ++++++++++++++++++++++++ utils/unittest/unittest.go | 11 + 2 files changed, 286 insertions(+) create mode 100644 storage/pebble/operation/common_test.go diff --git a/storage/pebble/operation/common_test.go b/storage/pebble/operation/common_test.go new file mode 100644 index 00000000000..9db08a0d3c3 --- /dev/null +++ b/storage/pebble/operation/common_test.go @@ -0,0 +1,275 @@ +package operation + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/pebble" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/vmihailenco/msgpack" + + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/unittest" +) + +var upsert = insert +var update = insert + +type Entity struct { + ID uint64 +} + +type UnencodeableEntity Entity + +var errCantEncode = fmt.Errorf("encoding not supported") +var errCantDecode = fmt.Errorf("decoding not supported") + +func (a UnencodeableEntity) MarshalJSON() ([]byte, error) { + return nil, errCantEncode +} + +func (a *UnencodeableEntity) UnmarshalJSON(b []byte) error { + return errCantDecode +} + +func (a UnencodeableEntity) MarshalMsgpack() ([]byte, error) { + return nil, errCantEncode +} + +func (a UnencodeableEntity) UnmarshalMsgpack(b []byte) error { + return errCantDecode +} + +func TestInsertValid(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := insert(key, e)(db) + require.NoError(t, err) + + var act []byte + act, closer, err := db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + + assert.Equal(t, val, act) + }) +} + +func TestInsertDuplicate(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + + // persist first time + err := insert(key, e)(db) + require.NoError(t, err) + + e2 := Entity{ID: 1338} + val, _ := msgpack.Marshal(e2) + + // persist again will override + err = insert(key, e2)(db) + require.NoError(t, err) + + // ensure old value did not insert + act, closer, err := db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + + assert.Equal(t, val, act) + }) +} + +func TestInsertEncodingError(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + + err := insert(key, UnencodeableEntity(e))(db) + require.Error(t, err, errCantEncode) + require.NotErrorIs(t, err, storage.ErrNotFound) + }) +} + +func TestUpdateValid(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := db.Set(key, []byte{}, nil) + require.NoError(t, err) + + err = insert(key, e)(db) + require.NoError(t, err) + + act, closer, err := db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + + assert.Equal(t, val, act) + }) +} + +func TestUpdateEncodingError(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := db.Set(key, val, nil) + require.NoError(t, err) + + err = insert(key, UnencodeableEntity(e))(db) + require.Error(t, err) + require.NotErrorIs(t, err, storage.ErrNotFound) + + // ensure value did not change + act, closer, err := db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + + assert.Equal(t, val, act) + }) +} + +func TestUpsertEntry(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + // first upsert an non-existed entry + err := insert(key, e)(db) + require.NoError(t, err) + + act, closer, err := db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + require.NoError(t, err) + + assert.Equal(t, val, act) + + // next upsert the value with the same key + newEntity := Entity{ID: 1338} + newVal, _ := msgpack.Marshal(newEntity) + err = upsert(key, newEntity)(db) + require.NoError(t, err) + + act, closer, err = db.Get(key) + require.NoError(t, err) + defer require.NoError(t, closer.Close()) + + assert.Equal(t, newVal, act) + }) +} + +func TestRetrieveValid(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := db.Set(key, val, nil) + require.NoError(t, err) + + var act Entity + err = retrieve(key, &act)(db) + require.NoError(t, err) + + assert.Equal(t, e, act) + }) +} + +func TestRetrieveMissing(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + key := []byte{0x01, 0x02, 0x03} + + var act Entity + err := retrieve(key, &act)(db) + require.ErrorIs(t, err, storage.ErrNotFound) + }) +} + +func TestRetrieveUnencodeable(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := db.Set(key, val, nil) + require.NoError(t, err) + + var act *UnencodeableEntity + err = retrieve(key, &act)(db) + require.Error(t, err) + require.NotErrorIs(t, err, storage.ErrNotFound) + }) +} + +// TestExists verifies that `exists` returns correct results in different scenarios. +func TestExists(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + t.Run("non-existent key", func(t *testing.T) { + key := unittest.RandomBytes(32) + var _exists bool + err := exists(key, &_exists)(db) + require.NoError(t, err) + assert.False(t, _exists) + }) + + t.Run("existent key", func(t *testing.T) { + key := unittest.RandomBytes(32) + err := insert(key, unittest.RandomBytes(256))(db) + require.NoError(t, err) + + var _exists bool + err = exists(key, &_exists)(db) + require.NoError(t, err) + assert.True(t, _exists) + }) + + t.Run("removed key", func(t *testing.T) { + key := unittest.RandomBytes(32) + // insert, then remove the key + err := insert(key, unittest.RandomBytes(256))(db) + require.NoError(t, err) + err = remove(key)(db) + require.NoError(t, err) + + var _exists bool + err = exists(key, &_exists)(db) + require.NoError(t, err) + assert.False(t, _exists) + }) + }) +} + +func TestRemove(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + e := Entity{ID: 1337} + key := []byte{0x01, 0x02, 0x03} + val, _ := msgpack.Marshal(e) + + err := db.Set(key, val, nil) + require.NoError(t, err) + + t.Run("should be able to remove", func(t *testing.T) { + err := remove(key)(db) + assert.NoError(t, err) + + _, _, err = db.Get(key) + assert.ErrorIs(t, convertNotFoundError(err), storage.ErrNotFound) + }) + + t.Run("should ok when removing non-existing value", func(t *testing.T) { + nonexistantKey := append(key, 0x01) + err := remove(nonexistantKey)(db) + assert.NoError(t, err) + }) + }) +} diff --git a/utils/unittest/unittest.go b/utils/unittest/unittest.go index 4d13b279087..d977be6695f 100644 --- a/utils/unittest/unittest.go +++ b/utils/unittest/unittest.go @@ -380,6 +380,17 @@ func TempPebbleDBWithOpts(t testing.TB, opts *pebble.Options) (*pebble.DB, strin return db, dbpath } +func RunWithPebbleDB(t testing.TB, f func(*pebble.DB)) { + RunWithTempDir(t, func(dir string) { + db, err := pebble.Open(dir, &pebble.Options{}) + require.NoError(t, err) + defer func() { + assert.NoError(t, db.Close()) + }() + f(db) + }) +} + func Concurrently(n int, f func(int)) { var wg sync.WaitGroup for i := 0; i < n; i++ {