From 8d98f0fc8cb3339a3b63d4f200080060dd55c71b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 13 Sep 2024 16:53:53 -0700 Subject: [PATCH 01/25] add universal database operations --- cmd/bootstrap/utils/md5.go | 3 +- storage/batch.go | 8 +- storage/operation/badgerimpl/iterator.go | 65 ++++++ storage/operation/badgerimpl/reader.go | 54 +++++ storage/operation/badgerimpl/writer.go | 93 ++++++++ storage/operation/callbacks.go | 24 ++ storage/operation/codec.go | 34 +++ storage/operation/dbtest/helper.go | 60 +++++ storage/operation/pebbleimpl/iterator.go | 74 ++++++ storage/operation/pebbleimpl/reader.go | 47 ++++ storage/operation/pebbleimpl/writer.go | 83 +++++++ storage/operation/reads.go | 222 ++++++++++++++++++ storage/operation/reads_test.go | 189 +++++++++++++++ storage/operation/writes.go | 58 +++++ storage/operation/writes_test.go | 278 +++++++++++++++++++++++ storage/operations.go | 132 +++++++++++ utils/unittest/unittest.go | 70 ++++++ 17 files changed, 1492 insertions(+), 2 deletions(-) create mode 100644 storage/operation/badgerimpl/iterator.go create mode 100644 storage/operation/badgerimpl/reader.go create mode 100644 storage/operation/badgerimpl/writer.go create mode 100644 storage/operation/callbacks.go create mode 100644 storage/operation/codec.go create mode 100644 storage/operation/dbtest/helper.go create mode 100644 storage/operation/pebbleimpl/iterator.go create mode 100644 storage/operation/pebbleimpl/reader.go create mode 100644 storage/operation/pebbleimpl/writer.go create mode 100644 storage/operation/reads.go create mode 100644 storage/operation/reads_test.go create mode 100644 storage/operation/writes.go create mode 100644 storage/operation/writes_test.go create mode 100644 storage/operations.go diff --git a/cmd/bootstrap/utils/md5.go b/cmd/bootstrap/utils/md5.go index 65823fd6e96..4d4bbe21046 100644 --- a/cmd/bootstrap/utils/md5.go +++ b/cmd/bootstrap/utils/md5.go @@ -2,7 +2,8 @@ package utils // The google storage API only provides md5 and crc32 hence overriding the linter flag for md5 import ( - "crypto/md5" //nolint:gosec + // #nosec + "crypto/md5" "io" "os" ) diff --git a/storage/batch.go b/storage/batch.go index 3147fc5c0e7..23b9d39ac63 100644 --- a/storage/batch.go +++ b/storage/batch.go @@ -1,11 +1,17 @@ package storage -import "github.com/dgraph-io/badger/v2" +import ( + "github.com/dgraph-io/badger/v2" +) +// deprecated +// use Writer instead type Transaction interface { Set(key, val []byte) error } +// deprecated +// use ReaderBatchWriter instead // BatchStorage serves as an abstraction over batch storage, adding ability to add ability to add extra // callbacks which fire after the batch is successfully flushed. type BatchStorage interface { diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go new file mode 100644 index 00000000000..81ecda2d719 --- /dev/null +++ b/storage/operation/badgerimpl/iterator.go @@ -0,0 +1,65 @@ +package badgerimpl + +import ( + "bytes" + + "github.com/dgraph-io/badger/v2" + + "github.com/onflow/flow-go/storage" +) + +type badgerIterator struct { + iter *badger.Iterator + lowerBound []byte + upperBound []byte +} + +var _ storage.Iterator = (*badgerIterator)(nil) + +func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage.IteratorOption) *badgerIterator { + options := badger.DefaultIteratorOptions + if ops.IterateKeyOnly { + options.PrefetchValues = false + } + + tx := db.NewTransaction(false) + iter := tx.NewIterator(options) + + lowerBound, upperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + + return &badgerIterator{ + iter: iter, + lowerBound: lowerBound, + upperBound: upperBound, + } +} + +func (i *badgerIterator) SeekGE() { + i.iter.Seek(i.lowerBound) +} + +func (i *badgerIterator) Valid() bool { + // if it's beyond the upper bound, it's invalid + if !i.iter.Valid() { + return false + } + key := i.iter.Item().Key() + // "< 0" means the upperBound is exclusive + valid := bytes.Compare(key, i.upperBound) < 0 + return valid +} + +func (i *badgerIterator) Next() { + i.iter.Next() +} + +func (i *badgerIterator) IterItem() storage.IterItem { + return i.iter.Item() +} + +var _ storage.IterItem = (*badger.Item)(nil) + +func (i *badgerIterator) Close() error { + i.iter.Close() + return nil +} diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go new file mode 100644 index 00000000000..06158e634ff --- /dev/null +++ b/storage/operation/badgerimpl/reader.go @@ -0,0 +1,54 @@ +package badgerimpl + +import ( + "errors" + "io" + + "github.com/dgraph-io/badger/v2" + + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" +) + +type dbReader struct { + db *badger.DB +} + +type noopCloser struct{} + +var _ io.Closer = (*noopCloser)(nil) + +func (noopCloser) Close() error { return nil } + +func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { + tx := b.db.NewTransaction(false) + defer tx.Discard() + + item, err := tx.Get(key) + if err != nil { + if errors.Is(err, badger.ErrKeyNotFound) { + return nil, nil, storage.ErrNotFound + } + return nil, nil, irrecoverable.NewExceptionf("could not load data: %w", err) + } + + var value []byte + err = item.Value(func(val []byte) error { + value = append([]byte{}, val...) + return nil + }) + if err != nil { + return nil, nil, irrecoverable.NewExceptionf("could not load value: %w", err) + } + + return value, noopCloser{}, nil +} + +func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { + return newBadgerIterator(b.db, startPrefix, endPrefix, ops), nil +} + +// ToReader is a helper function to convert a *badger.DB to a Reader +func ToReader(db *badger.DB) storage.Reader { + return dbReader{db} +} diff --git a/storage/operation/badgerimpl/writer.go b/storage/operation/badgerimpl/writer.go new file mode 100644 index 00000000000..3837be3917f --- /dev/null +++ b/storage/operation/badgerimpl/writer.go @@ -0,0 +1,93 @@ +package badgerimpl + +import ( + "fmt" + + "github.com/dgraph-io/badger/v2" + + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + op "github.com/onflow/flow-go/storage/operation" +) + +type ReaderBatchWriter struct { + globalReader storage.Reader + batch *badger.WriteBatch + + callbacks op.Callbacks +} + +var _ storage.ReaderBatchWriter = (*ReaderBatchWriter)(nil) + +func (b *ReaderBatchWriter) GlobalReader() storage.Reader { + return b.globalReader +} + +func (b *ReaderBatchWriter) Writer() storage.Writer { + return b +} + +func (b *ReaderBatchWriter) BadgerWriteBatch() *badger.WriteBatch { + return b.batch +} + +func (b *ReaderBatchWriter) AddCallback(callback func(error)) { + b.callbacks.AddCallback(callback) +} + +func (b *ReaderBatchWriter) Commit() error { + err := b.batch.Flush() + + b.callbacks.NotifyCallbacks(err) + + return err +} + +func WithReaderBatchWriter(db *badger.DB, fn func(storage.ReaderBatchWriter) error) error { + batch := NewReaderBatchWriter(db) + + err := fn(batch) + if err != nil { + // fn might use lock to ensure concurrent safety while reading and writing data + // and the lock is usually released by a callback. + // in other words, fn might hold a lock to be released by a callback, + // we need to notify the callback for the locks to be released before + // returning the error. + batch.callbacks.NotifyCallbacks(err) + return err + } + + return batch.Commit() +} + +func NewReaderBatchWriter(db *badger.DB) *ReaderBatchWriter { + return &ReaderBatchWriter{ + globalReader: ToReader(db), + batch: db.NewWriteBatch(), + } +} + +var _ storage.Writer = (*ReaderBatchWriter)(nil) + +func (b *ReaderBatchWriter) Set(key, value []byte) error { + return b.batch.Set(key, value) +} + +func (b *ReaderBatchWriter) Delete(key []byte) error { + return b.batch.Delete(key) +} + +func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { + err := operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { + err := b.batch.Delete(key) + if err != nil { + return fmt.Errorf("could not add key to delete batch (%v): %w", key, err) + } + return nil + })(globalReader) + + if err != nil { + return fmt.Errorf("could not find keys by range to be deleted: %w", err) + } + return nil +} diff --git a/storage/operation/callbacks.go b/storage/operation/callbacks.go new file mode 100644 index 00000000000..40d414ded91 --- /dev/null +++ b/storage/operation/callbacks.go @@ -0,0 +1,24 @@ +package operation + +import "sync" + +type Callbacks struct { + sync.Mutex // protect callbacks + callbacks []func(error) +} + +func (b *Callbacks) AddCallback(callback func(error)) { + b.Lock() + defer b.Unlock() + + b.callbacks = append(b.callbacks, callback) +} + +func (b *Callbacks) NotifyCallbacks(err error) { + b.Lock() + defer b.Unlock() + + for _, callback := range b.callbacks { + callback(err) + } +} diff --git a/storage/operation/codec.go b/storage/operation/codec.go new file mode 100644 index 00000000000..43dc4c37f7a --- /dev/null +++ b/storage/operation/codec.go @@ -0,0 +1,34 @@ +package operation + +import ( + "encoding/binary" + "fmt" + + "github.com/onflow/flow-go/model/flow" +) + +// EncodeKeyPart encodes a value to be used as a part of a key to be stored in storage. +func EncodeKeyPart(v interface{}) []byte { + switch i := v.(type) { + case uint8: + return []byte{i} + case uint32: + b := make([]byte, 4) + binary.BigEndian.PutUint32(b, i) + return b + case uint64: + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, i) + return b + case string: + return []byte(i) + case flow.Role: + return []byte{byte(i)} + case flow.Identifier: + return i[:] + case flow.ChainID: + return []byte(i) + default: + panic(fmt.Sprintf("unsupported type to convert (%T)", v)) + } +} diff --git a/storage/operation/dbtest/helper.go b/storage/operation/dbtest/helper.go new file mode 100644 index 00000000000..64a166c2390 --- /dev/null +++ b/storage/operation/dbtest/helper.go @@ -0,0 +1,60 @@ +package dbtest + +import ( + "testing" + + "github.com/cockroachdb/pebble" + "github.com/dgraph-io/badger/v2" + + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation/badgerimpl" + "github.com/onflow/flow-go/storage/operation/pebbleimpl" + "github.com/onflow/flow-go/utils/unittest" +) + +// helper types and functions +type WithWriter func(func(storage.Writer) error) error + +func RunWithStorages(t *testing.T, fn func(*testing.T, storage.Reader, WithWriter)) { + t.Run("BadgerStorage", func(t *testing.T) { + unittest.RunWithBadgerDB(t, func(db *badger.DB) { + withWriter := func(writing func(storage.Writer) error) error { + writer := badgerimpl.NewReaderBatchWriter(db) + err := writing(writer) + if err != nil { + return err + } + + err = writer.Commit() + if err != nil { + return err + } + return nil + } + + reader := badgerimpl.ToReader(db) + fn(t, reader, withWriter) + }) + }) + + t.Run("PebbleStorage", func(t *testing.T) { + unittest.RunWithPebbleDB(t, func(db *pebble.DB) { + withWriter := func(writing func(storage.Writer) error) error { + writer := pebbleimpl.NewReaderBatchWriter(db) + err := writing(writer) + if err != nil { + return err + } + + err = writer.Commit() + if err != nil { + return err + } + return nil + } + + reader := pebbleimpl.ToReader(db) + fn(t, reader, withWriter) + }) + }) +} diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go new file mode 100644 index 00000000000..b6f3910cead --- /dev/null +++ b/storage/operation/pebbleimpl/iterator.go @@ -0,0 +1,74 @@ +package pebbleimpl + +import ( + "fmt" + + "github.com/cockroachdb/pebble" + + "github.com/onflow/flow-go/storage" +) + +type pebbleIterator struct { + iter *pebble.Iterator + lowerBound []byte +} + +var _ storage.Iterator = (*pebbleIterator)(nil) + +func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops storage.IteratorOption) (*pebbleIterator, error) { + lowerBound, upperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + + options := pebble.IterOptions{ + LowerBound: lowerBound, + UpperBound: upperBound, + } + + iter, err := reader.NewIter(&options) + if err != nil { + return nil, fmt.Errorf("can not create iterator: %w", err) + } + + return &pebbleIterator{ + iter: iter, + lowerBound: lowerBound, + }, nil +} + +func (i *pebbleIterator) SeekGE() { + i.iter.SeekGE(i.lowerBound) +} + +func (i *pebbleIterator) Valid() bool { + return i.iter.Valid() +} + +func (i *pebbleIterator) Next() { + i.iter.Next() +} + +func (i *pebbleIterator) IterItem() storage.IterItem { + return pebbleIterItem{iter: i.iter} +} + +type pebbleIterItem struct { + iter *pebble.Iterator +} + +var _ storage.IterItem = (*pebbleIterItem)(nil) + +func (i pebbleIterItem) Key() []byte { + return i.iter.Key() +} + +func (i pebbleIterItem) Value(fn func([]byte) error) error { + val, err := i.iter.ValueAndErr() + if err != nil { + return err + } + + return fn(val) +} + +func (i *pebbleIterator) Close() error { + return i.iter.Close() +} diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go new file mode 100644 index 00000000000..6cfdfd93da5 --- /dev/null +++ b/storage/operation/pebbleimpl/reader.go @@ -0,0 +1,47 @@ +package pebbleimpl + +import ( + "errors" + "io" + + "github.com/cockroachdb/pebble" + + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" +) + +type dbReader struct { + db *pebble.DB +} + +var _ storage.Reader = (*dbReader)(nil) + +type noopCloser struct{} + +var _ io.Closer = (*noopCloser)(nil) + +func (noopCloser) Close() error { return nil } + +func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { + value, closer, err := b.db.Get(key) + + if err != nil { + if errors.Is(err, pebble.ErrNotFound) { + return nil, nil, storage.ErrNotFound + } + + // exception while checking for the key + return nil, nil, irrecoverable.NewExceptionf("could not load data: %w", err) + } + + return value, closer, nil +} + +func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { + return newPebbleIterator(b.db, startPrefix, endPrefix, ops) +} + +// ToReader is a helper function to convert a *pebble.DB to a Reader +func ToReader(db *pebble.DB) storage.Reader { + return dbReader{db} +} diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go new file mode 100644 index 00000000000..ad639223209 --- /dev/null +++ b/storage/operation/pebbleimpl/writer.go @@ -0,0 +1,83 @@ +package pebbleimpl + +import ( + "github.com/cockroachdb/pebble" + + "github.com/onflow/flow-go/storage" + op "github.com/onflow/flow-go/storage/operation" +) + +type ReaderBatchWriter struct { + globalReader storage.Reader + batch *pebble.Batch + + callbacks op.Callbacks +} + +var _ storage.ReaderBatchWriter = (*ReaderBatchWriter)(nil) + +func (b *ReaderBatchWriter) GlobalReader() storage.Reader { + return b.globalReader +} + +func (b *ReaderBatchWriter) Writer() storage.Writer { + return b +} + +func (b *ReaderBatchWriter) PebbleWriterBatch() *pebble.Batch { + return b.batch +} + +func (b *ReaderBatchWriter) AddCallback(callback func(error)) { + b.callbacks.AddCallback(callback) +} + +func (b *ReaderBatchWriter) Commit() error { + err := b.batch.Commit(pebble.Sync) + + b.callbacks.NotifyCallbacks(err) + + return err +} + +func WithReaderBatchWriter(db *pebble.DB, fn func(storage.ReaderBatchWriter) error) error { + batch := NewReaderBatchWriter(db) + + err := fn(batch) + if err != nil { + // fn might use lock to ensure concurrent safety while reading and writing data + // and the lock is usually released by a callback. + // in other words, fn might hold a lock to be released by a callback, + // we need to notify the callback for the locks to be released before + // returning the error. + batch.callbacks.NotifyCallbacks(err) + return err + } + + return batch.Commit() +} + +func NewReaderBatchWriter(db *pebble.DB) *ReaderBatchWriter { + return &ReaderBatchWriter{ + globalReader: ToReader(db), + batch: db.NewBatch(), + } +} + +var _ storage.Writer = (*ReaderBatchWriter)(nil) + +func (b *ReaderBatchWriter) Set(key, value []byte) error { + return b.batch.Set(key, value, pebble.Sync) +} + +func (b *ReaderBatchWriter) Delete(key []byte) error { + return b.batch.Delete(key, pebble.Sync) +} + +// DeleteByRange deletes all keys with the given prefix defined by [startPrefix, endPrefix] (both inclusive). +func (b *ReaderBatchWriter) DeleteByRange(_ storage.Reader, startPrefix, endPrefix []byte) error { + // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). + // therefore, we need to increment the endPrefix to make it inclusive. + start, end := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + return b.batch.DeleteRange(start, end, pebble.Sync) +} diff --git a/storage/operation/reads.go b/storage/operation/reads.go new file mode 100644 index 00000000000..2e6be8dd3fe --- /dev/null +++ b/storage/operation/reads.go @@ -0,0 +1,222 @@ +package operation + +import ( + "bytes" + "errors" + "fmt" + + "github.com/vmihailenco/msgpack" + + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" +) + +// CheckFunc is a function that checks if the value should be read and decoded. +// return (true, nil) to read the value and pass it to the CreateFunc and HandleFunc for decoding +// return (false, nil) to skip reading the value +// return (false, err) if running into any exception, the iteration should be stopped. +type CheckFunc func(key []byte) (bool, error) + +// createFunc returns a pointer to an initialized entity that we can potentially +// decode the next value into during a badger DB iteration. +type CreateFunc func() interface{} + +// handleFunc is a function that starts the processing of the current key-value +// pair during a badger iteration. It should be called after the key was checked +// and the entity was decoded. +// No errors are expected during normal operation. Any errors will halt the iteration. +type HandleFunc func() error +type IterationFunc func() (CheckFunc, CreateFunc, HandleFunc) + +// IterateKeysInPrefixRange will iterate over all keys in the given range [startPrefix, endPrefix] (both inclusive) +func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(key []byte) error) func(storage.Reader) error { + return Iterate(startPrefix, endPrefix, func() (CheckFunc, CreateFunc, HandleFunc) { + return func(key []byte) (bool, error) { + err := check(key) + if err != nil { + return false, err + } + return false, nil + }, nil, nil + }, storage.IteratorOption{IterateKeyOnly: true}) +} + +// Iterate will iterate over all keys in the given range [startPrefix, endPrefix] (both inclusive) +func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { + return func(r storage.Reader) error { + + if len(startPrefix) == 0 { + return fmt.Errorf("startPrefix prefix is empty") + } + + if len(endPrefix) == 0 { + return fmt.Errorf("endPrefix prefix is empty") + } + + // Reverse iteration is not supported by pebble + if bytes.Compare(startPrefix, endPrefix) > 0 { + return fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } + + it, err := r.NewIter(startPrefix, endPrefix, opt) + if err != nil { + return fmt.Errorf("can not create iterator: %w", err) + } + defer it.Close() + + for it.SeekGE(); it.Valid(); it.Next() { + item := it.IterItem() + key := item.Key() + + // initialize processing functions for iteration + check, create, handle := iterFunc() + + keyCopy := make([]byte, len(key)) + copy(keyCopy, key) + + // check if we should process the item at all + shouldReadValue, err := check(keyCopy) + if err != nil { + return err + } + if !shouldReadValue { // skip reading value + continue + } + + err = item.Value(func(val []byte) error { + + // decode into the entity + entity := create() + err = msgpack.Unmarshal(val, entity) + if err != nil { + return irrecoverable.NewExceptionf("could not decode entity: %w", err) + } + + // process the entity + err = handle() + if err != nil { + return fmt.Errorf("could not handle entity: %w", err) + } + + return nil + }) + + if err != nil { + return fmt.Errorf("could not process value: %w", err) + } + } + + return nil + } +} + +// Traverse will iterate over all keys with the given prefix +func Traverse(prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { + return Iterate(prefix, PrefixUpperBound(prefix), iterFunc, opt) +} + +// PrefixUpperBound returns a key K such that all possible keys beginning with the input prefix +// sort lower than K according to the byte-wise lexicographic key ordering used by Pebble. +// This is used to define an upper bound for iteration, when we want to iterate over +// all keys beginning with a given prefix. +// referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration +func PrefixUpperBound(prefix []byte) []byte { + end := make([]byte, len(prefix)) + copy(end, prefix) + for i := len(end) - 1; i >= 0; i-- { + // increment the bytes by 1 + end[i] = end[i] + 1 + if end[i] != 0 { + return end[:i+1] + } + } + return nil // no upper-bound +} + +// Exists returns true if a key exists in the database. +// No errors are expected during normal operation. +func Exists(key []byte, keyExists *bool) func(storage.Reader) error { + return func(r storage.Reader) error { + _, closer, err := r.Get(key) + if err != nil { + // the key does not exist in the database + if errors.Is(err, storage.ErrNotFound) { + *keyExists = false + return nil + } + // exception while checking for the key + return irrecoverable.NewExceptionf("could not load data: %w", err) + } + defer closer.Close() + + // the key does exist in the database + *keyExists = true + return nil + } +} + +// retrieve will retrieve the binary data under the given key from the badger DB +// and decode it into the given entity. The provided entity needs to be a +// pointer to an initialized entity of the correct type. +// Error returns: +// - storage.ErrNotFound if the key does not exist in the database +// - generic error in case of unexpected failure from the database layer, or failure +// to decode an existing database value +func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { + return func(r storage.Reader) error { + val, closer, err := r.Get(key) + if err != nil { + return err + } + + defer closer.Close() + + err = msgpack.Unmarshal(val, entity) + if err != nil { + return irrecoverable.NewExceptionf("could not decode entity: %w", err) + } + return nil + } +} + +// FindHighestAtOrBelow finds the highest key with the given prefix and +// height equal to or below the given height. +func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func(storage.Reader) error { + return func(r storage.Reader) error { + if len(prefix) == 0 { + return fmt.Errorf("prefix must not be empty") + } + + key := append(prefix, EncodeKeyPart(height)...) + it, err := r.NewIter(prefix, key, storage.DefaultIteratorOptions()) + if err != nil { + return fmt.Errorf("can not create iterator: %w", err) + } + defer it.Close() + + var highestKey []byte + // find highest value below the given height + for it.SeekGE(); it.Valid(); it.Next() { + highestKey = it.IterItem().Key() + } + + if len(highestKey) == 0 { + return storage.ErrNotFound + } + + // read the value of the highest key + val, closer, err := r.Get(highestKey) + if err != nil { + return err + } + + defer closer.Close() + + err = msgpack.Unmarshal(val, entity) + if err != nil { + return irrecoverable.NewExceptionf("failed to decode value: %w", err) + } + + return nil + } +} diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go new file mode 100644 index 00000000000..e24bc15b5ae --- /dev/null +++ b/storage/operation/reads_test.go @@ -0,0 +1,189 @@ +package operation_test + +import ( + "bytes" + "fmt" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" +) + +func TestIterateKeysInPrefixRange(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + // Define the prefix range + prefixStart := []byte{0x10} + prefixEnd := []byte{0x20} + + // Create a range of keys around the prefix start/end values + keys := [][]byte{ + // before start -> not included in range + {0x09, 0xff}, + // within the start prefix -> included in range + {0x10, 0x00}, + {0x10, 0xff}, + // between start and end -> included in range + {0x15, 0x00}, + {0x1A, 0xff}, + // within the end prefix -> included in range + {0x20, 0x00}, + {0x20, 0xff}, + // after end -> not included in range + {0x21, 0x00}, + } + + // Keys expected to be in the prefix range + lastNToExclude := 1 + keysInRange := keys[1 : len(keys)-lastNToExclude] // these keys are between the start and end + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + // Forward iteration and check boundaries + var found [][]byte + require.NoError(t, operation.IterateKeysInPrefixRange(prefixStart, prefixEnd, func(key []byte) error { + found = append(found, key) + return nil + })(r), "should iterate forward without error") + require.ElementsMatch(t, keysInRange, found, "forward iteration should return the correct keys in range") + }) +} + +func TestTraverse(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + keys := [][]byte{ + {0x42, 0x00}, + {0xff}, + {0x42, 0x56}, + {0x00}, + {0x42, 0xff}, + } + vals := []uint64{11, 13, 17, 19, 23} + expected := []uint64{11, 23} + + // Insert the keys and values into storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for i, key := range keys { + err := operation.Upsert(key, vals[i])(writer) + if err != nil { + return err + } + } + return nil + })) + + actual := make([]uint64, 0, len(keys)) + + // Define the iteration logic + iterationFunc := func() (operation.CheckFunc, operation.CreateFunc, operation.HandleFunc) { + check := func(key []byte) (bool, error) { + // Skip the key {0x42, 0x56} + return !bytes.Equal(key, []byte{0x42, 0x56}), nil + } + var val uint64 + create := func() interface{} { + return &val + } + handle := func() error { + actual = append(actual, val) + return nil + } + return check, create, handle + } + + // Traverse the keys starting with prefix {0x42} + err := operation.Traverse([]byte{0x42}, iterationFunc, storage.DefaultIteratorOptions())(r) + require.NoError(t, err, "traverse should not return an error") + + // Assert that the actual values match the expected values + require.Equal(t, expected, actual, "traversed values should match expected values") + }) +} + +func TestFindHighestAtOrBelow(t *testing.T) { + // Helper function to insert an entity into the storage + insertEntity := func(writer storage.Writer, prefix []byte, height uint64, entity Entity) error { + key := append(prefix, operation.EncodeKeyPart(height)...) + return operation.Upsert(key, entity)(writer) + } + + // Entities to be inserted + entities := []struct { + height uint64 + entity Entity + }{ + {5, Entity{ID: 41}}, + {10, Entity{ID: 42}}, + {15, Entity{ID: 43}}, + } + + // Run test with multiple storage backends + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + prefix := []byte("test_prefix") + + // Insert entities into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, e := range entities { + if err := insertEntity(writer, prefix, e.height, e.entity); err != nil { + return err + } + } + return nil + })) + + // Declare entity to store the results of FindHighestAtOrBelow + var entity Entity + + // Test cases + tests := []struct { + name string + height uint64 + expectedValue uint64 + expectError bool + expectedErrMsg string + }{ + {"target first height exists", 5, 41, false, ""}, + {"target height exists", 10, 42, false, ""}, + {"target height above", 11, 42, false, ""}, + {"target height above highest", 20, 43, false, ""}, + {"target height below lowest", 4, 0, true, storage.ErrNotFound.Error()}, + {"empty prefix", 5, 0, true, "prefix must not be empty"}, + } + + // Execute test cases + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + prefixToUse := prefix + + if tt.name == "empty prefix" { + prefixToUse = []byte{} + } + + err := operation.FindHighestAtOrBelow( + prefixToUse, + tt.height, + &entity)(r) + + if tt.expectError { + require.Error(t, err, fmt.Sprintf("expected error but got nil, entity: %v", entity)) + require.Contains(t, err.Error(), tt.expectedErrMsg) + } else { + require.NoError(t, err) + require.Equal(t, tt.expectedValue, entity.ID) + } + }) + } + }) +} diff --git a/storage/operation/writes.go b/storage/operation/writes.go new file mode 100644 index 00000000000..3bbe08d12d2 --- /dev/null +++ b/storage/operation/writes.go @@ -0,0 +1,58 @@ +package operation + +import ( + "github.com/vmihailenco/msgpack" + + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/storage" +) + +// Upsert will encode the given entity using msgpack and will insert the resulting +// binary data under the provided key. +// If the key already exists, the value will be overwritten. +// Error returns: +// - generic error in case of unexpected failure from the database layer or +// encoding failure. +func Upsert(key []byte, val interface{}) func(storage.Writer) error { + return func(w storage.Writer) error { + value, err := msgpack.Marshal(val) + if err != nil { + return irrecoverable.NewExceptionf("failed to encode value: %w", err) + } + + err = w.Set(key, value) + if err != nil { + return irrecoverable.NewExceptionf("failed to store data: %w", err) + } + + 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: +// * generic error in case of unexpected database error +func Remove(key []byte) func(storage.Writer) error { + return func(w storage.Writer) error { + err := w.Delete(key) + if err != nil { + return irrecoverable.NewExceptionf("could not delete item: %w", err) + } + return nil + } +} + +// RemoveByPrefix removes all keys with the given prefix defined by [startPrefix, endPrefix] (both inclusive). +// If no keys exist with the given prefix, this is a no-op. +// Error returns: +// * generic error in case of unexpected database error +func RemoveByPrefix(reader storage.Reader, key []byte) func(storage.Writer) error { + return func(w storage.Writer) error { + err := w.DeleteByRange(reader, key, key) + if err != nil { + return irrecoverable.NewExceptionf("could not delete item: %w", err) + } + return nil + } +} diff --git a/storage/operation/writes_test.go b/storage/operation/writes_test.go new file mode 100644 index 00000000000..aa7b5020b1a --- /dev/null +++ b/storage/operation/writes_test.go @@ -0,0 +1,278 @@ +package operation_test + +import ( + "encoding/binary" + "errors" + "fmt" + "sync" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" + "github.com/onflow/flow-go/storage/operation/dbtest" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestReadWrite(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + e := Entity{ID: 1337} + + // Test read nothing should return not found + var item Entity + err := operation.Retrieve(e.Key(), &item)(r) + require.True(t, errors.Is(err, storage.ErrNotFound), "expected not found error") + + require.NoError(t, withWriter(operation.Upsert(e.Key(), e))) + + var readBack Entity + require.NoError(t, operation.Retrieve(e.Key(), &readBack)(r)) + require.Equal(t, e, readBack, "expected retrieved value to match written value") + + // Test write again should overwrite + newEntity := Entity{ID: 42} + require.NoError(t, withWriter(operation.Upsert(e.Key(), newEntity))) + + require.NoError(t, operation.Retrieve(e.Key(), &readBack)(r)) + require.Equal(t, newEntity, readBack, "expected overwritten value to be retrieved") + + // Test write should not overwrite a different key + anotherEntity := Entity{ID: 84} + require.NoError(t, withWriter(operation.Upsert(anotherEntity.Key(), anotherEntity))) + + var anotherReadBack Entity + require.NoError(t, operation.Retrieve(anotherEntity.Key(), &anotherReadBack)(r)) + require.Equal(t, anotherEntity, anotherReadBack, "expected different key to return different value") + }) +} + +func TestReadWriteMalformed(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + e := Entity{ID: 1337} + ue := UnencodeableEntity(e) + + // Test write should return encoding error + require.NoError(t, withWriter(func(writer storage.Writer) error { + err := operation.Upsert(e.Key(), ue)(writer) + require.Contains(t, err.Error(), errCantEncode.Error(), "expected encoding error") + return nil + })) + + // Test read should return decoding error + var exists bool + require.NoError(t, operation.Exists(e.Key(), &exists)(r)) + require.False(t, exists, "expected key to not exist") + }) +} + +// Verify multiple entities can be removed in one batch update +func TestBatchWrite(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + // Define multiple entities for batch insertion + entities := []Entity{ + {ID: 1337}, + {ID: 42}, + {ID: 84}, + } + + // Batch write: insert multiple entities in a single transaction + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, e := range entities { + if err := operation.Upsert(e.Key(), e)(writer); err != nil { + return err + } + } + return nil + })) + + // Verify that each entity can be read back + for _, e := range entities { + var readBack Entity + require.NoError(t, operation.Retrieve(e.Key(), &readBack)(r)) + require.Equal(t, e, readBack, "expected retrieved value to match written value for entity ID %d", e.ID) + } + + // Batch update: remove multiple entities in a single transaction + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, e := range entities { + if err := operation.Remove(e.Key())(writer); err != nil { + return err + } + } + return nil + })) + + // Verify that each entity has been removed + for _, e := range entities { + var readBack Entity + err := operation.Retrieve(e.Key(), &readBack)(r) + require.True(t, errors.Is(err, storage.ErrNotFound), "expected not found error for entity ID %d after removal", e.ID) + } + }) +} + +func TestRemove(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + e := Entity{ID: 1337} + + var exists bool + require.NoError(t, operation.Exists(e.Key(), &exists)(r)) + require.False(t, exists, "expected key to not exist") + + // Test delete nothing should return OK + require.NoError(t, withWriter(operation.Remove(e.Key()))) + + // Test write, delete, then read should return not found + require.NoError(t, withWriter(operation.Upsert(e.Key(), e))) + + require.NoError(t, operation.Exists(e.Key(), &exists)(r)) + require.True(t, exists, "expected key to exist") + + require.NoError(t, withWriter(operation.Remove(e.Key()))) + + var item Entity + err := operation.Retrieve(e.Key(), &item)(r) + require.True(t, errors.Is(err, storage.ErrNotFound), "expected not found error after delete") + }) +} + +func TestConcurrentWrite(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + var wg sync.WaitGroup + numWrites := 10 // number of concurrent writes + + for i := 0; i < numWrites; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + e := Entity{ID: uint64(i)} + + // Simulate a concurrent write to a different key + require.NoError(t, withWriter(operation.Upsert(e.Key(), e))) + + var readBack Entity + require.NoError(t, operation.Retrieve(e.Key(), &readBack)(r)) + require.Equal(t, e, readBack, "expected retrieved value to match written value for key %d", i) + }(i) + } + + wg.Wait() // Wait for all goroutines to finish + }) +} + +func TestConcurrentRemove(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + var wg sync.WaitGroup + numDeletes := 10 // number of concurrent deletions + + // First, insert entities to be deleted concurrently + for i := 0; i < numDeletes; i++ { + e := Entity{ID: uint64(i)} + require.NoError(t, withWriter(operation.Upsert(e.Key(), e))) + } + + // Now, perform concurrent deletes + for i := 0; i < numDeletes; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + e := Entity{ID: uint64(i)} + + // Simulate a concurrent delete + require.NoError(t, withWriter(operation.Remove(e.Key()))) + + // Check that the item is no longer retrievable + var item Entity + err := operation.Retrieve(e.Key(), &item)(r) + require.True(t, errors.Is(err, storage.ErrNotFound), "expected not found error after delete for key %d", i) + }(i) + } + + wg.Wait() // Wait for all goroutines to finish + }) +} + +func TestRemoveRange(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + + // Define the prefix + prefix := []byte{0x10} + + // Create a range of keys around the boundaries of the prefix + keys := [][]byte{ + // before prefix -> not included in range + {0x09, 0xff}, + // within the prefix -> included in range + {0x10, 0x00}, + {0x10, 0x50}, + {0x10, 0xff}, + // after end -> not included in range + {0x11, 0x00}, + {0x1A, 0xff}, + } + + // Keys expected to be in the prefix range + includeStart, includeEnd := 1, 3 + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + // Remove the keys in the prefix range + require.NoError(t, withWriter(operation.RemoveByPrefix(r, prefix))) + + lg := unittest.Logger().With().Logger() + // Verify that the keys in the prefix range have been removed + for i, key := range keys { + var exists bool + require.NoError(t, operation.Exists(key, &exists)(r)) + lg.Info().Msgf("key %x exists: %t", key, exists) + + deleted := includeStart <= i && i <= includeEnd + + // deleted item should not exist + require.Equal(t, !deleted, exists, + "expected key %x to be %s", key, map[bool]string{true: "deleted", false: "not deleted"}) + } + }) +} + +type Entity struct { + ID uint64 +} + +func (e Entity) Key() []byte { + byteSlice := make([]byte, 8) // uint64 is 8 bytes + binary.BigEndian.PutUint64(byteSlice, e.ID) + return byteSlice +} + +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 +} diff --git a/storage/operations.go b/storage/operations.go new file mode 100644 index 00000000000..c261d4ba28c --- /dev/null +++ b/storage/operations.go @@ -0,0 +1,132 @@ +package storage + +import ( + "io" +) + +// Iterator is an interface for iterating over key-value pairs in a storage backend. +type Iterator interface { + // SeekGE seeks to the smallest key greater than or equal to the given key. + SeekGE() + + // Valid returns whether the iterator is positioned at a valid key-value pair. + Valid() bool + + // Next advances the iterator to the next key-value pair. + Next() + + // Key returns the key of the current key-value pair, or nil if done. + IterItem() IterItem + + // Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. + Close() error +} + +// IterItem is an interface for iterating over key-value pairs in a storage backend. +type IterItem interface { + Key() []byte + + // Value returns the value of the current key-value pair + // The reason it takes a function is to follow badgerDB's API pattern + Value(func(val []byte) error) error +} + +type IteratorOption struct { + IterateKeyOnly bool // default false +} + +func DefaultIteratorOptions() IteratorOption { + return IteratorOption{ + IterateKeyOnly: false, // only needed for badger. ignored by pebble + } +} + +type Reader interface { + // Get gets the value for the given key. It returns ErrNotFound if the DB + // does not contain the key. + // + // The caller should not modify the contents of the returned slice, but it is + // safe to modify the contents of the argument after Get returns. The + // returned slice will remain valid until the returned Closer is closed. On + // success, the caller MUST call closer.Close() or a memory leak will occur. + Get(key []byte) (value []byte, closer io.Closer, err error) + + // NewIter returns a new Iterator for the given key range [startPrefix, endPrefix], both inclusive. + NewIter(startPrefix, endPrefix []byte, ops IteratorOption) (Iterator, error) +} + +// Writer is an interface for batch writing to a storage backend. +type Writer interface { + // Set sets the value for the given key. It overwrites any previous value + // for that key; a DB is not a multi-map. + // + // It is safe to modify the contents of the arguments after Set returns. + Set(k, v []byte) error + + // Delete deletes the value for the given key. Deletes are blind all will + // succeed even if the given key does not exist. + // + // It is safe to modify the contents of the arguments after Delete returns. + Delete(key []byte) error + + // DeleteByRange removes all keys with a prefix that falls within the + // range [start, end], both inclusive. + DeleteByRange(globalReader Reader, startPrefix, endPrefix []byte) error +} + +// ReaderBatchWriter is an interface for reading and writing to a storage backend. +type ReaderBatchWriter interface { + // GlobalReader returns a database-backed reader which reads the latest committed global database state ("read-committed isolation"). + // This reader will not read writes written to ReaderBatchWriter.Writer until the write batch is committed. + // This reader may observe different values for the same key on subsequent reads. + GlobalReader() Reader + + // Writer returns a writer associated with a batch of writes. The batch is pending until it is committed. + // When we `Write` into the batch, that write operation is added to the pending batch, but not committed. + // The commit operation is atomic w.r.t. the batch; either all writes are applied to the database, or no writes are. + // Note: + // - The writer cannot be used concurrently for writing. + Writer() Writer + + // AddCallback adds a callback to execute after the batch has been flush + // regardless the batch update is succeeded or failed. + // The error parameter is the error returned by the batch update. + AddCallback(func(error)) +} + +// OnCommitSucceed adds a callback to execute after the batch has been successfully committed. +func OnCommitSucceed(b ReaderBatchWriter, onSuccessFn func()) { + b.AddCallback(func(err error) { + if err == nil { + onSuccessFn() + } + }) +} + +func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte) { + // LowerBound specifies the smallest key to iterate and it's inclusive. + // UpperBound specifies the largest key to iterate and it's exclusive (not inclusive) + // in order to match all keys prefixed with the `end` bytes, we increment the bytes of end by 1, + // for instance, to iterate keys between "hello" and "world", + // we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" + // will all be included. + return startPrefix, prefixUpperBound(endPrefix) +} + +// prefixUpperBound returns a key K such that all possible keys beginning with the input prefix +// sort lower than K according to the byte-wise lexicographic key ordering used by Pebble. +// This is used to define an upper bound for iteration, when we want to iterate over +// all keys beginning with a given prefix. +// referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration +func prefixUpperBound(prefix []byte) []byte { + end := make([]byte, len(prefix)) + copy(end, prefix) + for i := len(end) - 1; i >= 0; i-- { + // increment the bytes by 1 + end[i] = end[i] + 1 + if end[i] != 0 { + return end[:i+1] + } + } + return nil // no upper-bound +} diff --git a/utils/unittest/unittest.go b/utils/unittest/unittest.go index 4d13b279087..d15f39cd27c 100644 --- a/utils/unittest/unittest.go +++ b/utils/unittest/unittest.go @@ -368,6 +368,11 @@ func TempBadgerDB(t testing.TB) (*badger.DB, string) { return db, dir } +func TempPebbleDB(t testing.TB) (*pebble.DB, string) { + dir := TempDir(t) + return PebbleDB(t, dir), dir +} + func TempPebblePath(t *testing.T) string { return path.Join(TempDir(t), "pebble"+strconv.Itoa(rand.Int())+".db") } @@ -380,6 +385,71 @@ 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 PebbleDB(t testing.TB, dir string) *pebble.DB { + db, err := pebble.Open(dir, &pebble.Options{}) + require.NoError(t, err) + return db +} + +func TypedPebbleDB(t testing.TB, dir string, create func(string, *pebble.Options) (*pebble.DB, error)) *pebble.DB { + db, err := create(dir, &pebble.Options{}) + require.NoError(t, err) + return db +} + +type PebbleWrapper struct { + db *pebble.DB +} + +func (p *PebbleWrapper) View(fn func(pebble.Reader) error) error { + return fn(p.db) +} + +func (p *PebbleWrapper) Update(fn func(pebble.Writer) error) error { + return fn(p.db) +} + +func (p *PebbleWrapper) DB() *pebble.DB { + return p.db +} + +func RunWithWrappedPebbleDB(t testing.TB, f func(p *PebbleWrapper)) { + RunWithTempDir(t, func(dir string) { + db, err := pebble.Open(dir, &pebble.Options{}) + require.NoError(t, err) + defer func() { + assert.NoError(t, db.Close()) + }() + f(&PebbleWrapper{db}) + }) + +} + +func RunWithTypedPebbleDB( + t testing.TB, + create func(string, *pebble.Options) (*pebble.DB, error), + f func(*pebble.DB)) { + RunWithTempDir(t, func(dir string) { + db, err := create(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++ { From 90a24264c6d0c652ddc02b5befe6bfe53d4d5a4d Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 1 Nov 2024 14:10:48 -0700 Subject: [PATCH 02/25] address review comments --- storage/operation/pebbleimpl/iterator.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index b6f3910cead..d8453b497f5 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -9,8 +9,7 @@ import ( ) type pebbleIterator struct { - iter *pebble.Iterator - lowerBound []byte + iter *pebble.Iterator } var _ storage.Iterator = (*pebbleIterator)(nil) @@ -29,13 +28,12 @@ func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops } return &pebbleIterator{ - iter: iter, - lowerBound: lowerBound, + iter: iter, }, nil } func (i *pebbleIterator) SeekGE() { - i.iter.SeekGE(i.lowerBound) + i.iter.First() } func (i *pebbleIterator) Valid() bool { From 326772e598b9d754b34d9e90670e7e71589eef7a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 1 Nov 2024 14:12:25 -0700 Subject: [PATCH 03/25] consolidate PrefixUpperbound --- storage/batch.go | 8 +++--- storage/operation/badgerimpl/iterator.go | 4 +-- storage/operation/badgerimpl/reader.go | 6 +---- storage/operation/pebbleimpl/iterator.go | 2 +- storage/operation/pebbleimpl/writer.go | 2 +- storage/operation/reads.go | 33 +++++++----------------- storage/operation/reads_test.go | 27 +++++++++---------- storage/operation/writes_test.go | 6 ++--- storage/operations.go | 33 +++++++++++++++++------- utils/unittest/unittest.go | 28 -------------------- 10 files changed, 57 insertions(+), 92 deletions(-) diff --git a/storage/batch.go b/storage/batch.go index 23b9d39ac63..bc9c4853294 100644 --- a/storage/batch.go +++ b/storage/batch.go @@ -4,16 +4,16 @@ import ( "github.com/dgraph-io/badger/v2" ) -// deprecated -// use Writer instead +// Deprecated: Transaction is being deprecated as part of the transition from Badger to Pebble. +// Use Writer instead of Transaction for all new code. type Transaction interface { Set(key, val []byte) error } -// deprecated -// use ReaderBatchWriter instead // BatchStorage serves as an abstraction over batch storage, adding ability to add ability to add extra // callbacks which fire after the batch is successfully flushed. +// Deprecated: BatchStorage is being deprecated as part of the transition from Badger to Pebble. +// Use ReaderBatchWriter instead of BatchStorage for all new code. type BatchStorage interface { GetWriter() *badger.WriteBatch diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index 81ecda2d719..e9f8b5dc6be 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -34,7 +34,7 @@ func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage } } -func (i *badgerIterator) SeekGE() { +func (i *badgerIterator) First() { i.iter.Seek(i.lowerBound) } @@ -44,7 +44,7 @@ func (i *badgerIterator) Valid() bool { return false } key := i.iter.Item().Key() - // "< 0" means the upperBound is exclusive + // "< 0" means "key < upperBound" valid := bytes.Compare(key, i.upperBound) < 0 return valid } diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index 06158e634ff..8d7d982d65e 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -32,11 +32,7 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { return nil, nil, irrecoverable.NewExceptionf("could not load data: %w", err) } - var value []byte - err = item.Value(func(val []byte) error { - value = append([]byte{}, val...) - return nil - }) + value, err := item.ValueCopy(nil) if err != nil { return nil, nil, irrecoverable.NewExceptionf("could not load value: %w", err) } diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index d8453b497f5..b1b6630cc51 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -32,7 +32,7 @@ func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops }, nil } -func (i *pebbleIterator) SeekGE() { +func (i *pebbleIterator) First() { i.iter.First() } diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index ad639223209..c6ccdff06b9 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -74,7 +74,7 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { return b.batch.Delete(key, pebble.Sync) } -// DeleteByRange deletes all keys with the given prefix defined by [startPrefix, endPrefix] (both inclusive). +// DeleteByRange deletes all keys with a prefix in the range [startPrefix, endPrefix] (both inclusive). func (b *ReaderBatchWriter) DeleteByRange(_ storage.Reader, startPrefix, endPrefix []byte) error { // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). // therefore, we need to increment the endPrefix to make it inclusive. diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 2e6be8dd3fe..1be299ab9d8 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -28,7 +28,7 @@ type CreateFunc func() interface{} type HandleFunc func() error type IterationFunc func() (CheckFunc, CreateFunc, HandleFunc) -// IterateKeysInPrefixRange will iterate over all keys in the given range [startPrefix, endPrefix] (both inclusive) +// IterateKeysInPrefixRange will iterate over all keys with prefixes in the range [startPrefix, endPrefix] (both inclusive) func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(key []byte) error) func(storage.Reader) error { return Iterate(startPrefix, endPrefix, func() (CheckFunc, CreateFunc, HandleFunc) { return func(key []byte) (bool, error) { @@ -41,7 +41,7 @@ func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(k }, storage.IteratorOption{IterateKeyOnly: true}) } -// Iterate will iterate over all keys in the given range [startPrefix, endPrefix] (both inclusive) +// Iterate will iterate over all keys with prefixes in the given range [startPrefix, endPrefix] (both inclusive) func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { return func(r storage.Reader) error { @@ -64,7 +64,7 @@ func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt s } defer it.Close() - for it.SeekGE(); it.Valid(); it.Next() { + for it.First(); it.Valid(); it.Next() { item := it.IterItem() key := item.Key() @@ -72,6 +72,9 @@ func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt s check, create, handle := iterFunc() keyCopy := make([]byte, len(key)) + + // The underlying database may re-use and modify the backing memory of the returned key. + // Tor safety we proactively make a copy before passing the key to the upper layer. copy(keyCopy, key) // check if we should process the item at all @@ -112,25 +115,7 @@ func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt s // Traverse will iterate over all keys with the given prefix func Traverse(prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { - return Iterate(prefix, PrefixUpperBound(prefix), iterFunc, opt) -} - -// PrefixUpperBound returns a key K such that all possible keys beginning with the input prefix -// sort lower than K according to the byte-wise lexicographic key ordering used by Pebble. -// This is used to define an upper bound for iteration, when we want to iterate over -// all keys beginning with a given prefix. -// referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration -func PrefixUpperBound(prefix []byte) []byte { - end := make([]byte, len(prefix)) - copy(end, prefix) - for i := len(end) - 1; i >= 0; i-- { - // increment the bytes by 1 - end[i] = end[i] + 1 - if end[i] != 0 { - return end[:i+1] - } - } - return nil // no upper-bound + return Iterate(prefix, prefix, iterFunc, opt) } // Exists returns true if a key exists in the database. @@ -155,7 +140,7 @@ func Exists(key []byte, keyExists *bool) func(storage.Reader) error { } } -// retrieve will retrieve the binary data under the given key from the badger DB +// Retrieve will retrieve the binary data under the given key from the database // and decode it into the given entity. The provided entity needs to be a // pointer to an initialized entity of the correct type. // Error returns: @@ -196,7 +181,7 @@ func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func var highestKey []byte // find highest value below the given height - for it.SeekGE(); it.Valid(); it.Next() { + for it.First(); it.Valid(); it.Next() { highestKey = it.IterItem().Key() } diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index e24bc15b5ae..b9addec418d 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -35,9 +35,9 @@ func TestIterateKeysInPrefixRange(t *testing.T) { {0x21, 0x00}, } - // Keys expected to be in the prefix range - lastNToExclude := 1 - keysInRange := keys[1 : len(keys)-lastNToExclude] // these keys are between the start and end + // The first and last keys are outside the prefix range, so we omit them + // from keysInRange, which is the set of keys we expect in the iteration + keysInRange := keys[1 : len(keys)-1] // Insert the keys into the storage require.NoError(t, withWriter(func(writer storage.Writer) error { @@ -63,20 +63,21 @@ func TestIterateKeysInPrefixRange(t *testing.T) { func TestTraverse(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { - keys := [][]byte{ - {0x42, 0x00}, - {0xff}, - {0x42, 0x56}, - {0x00}, - {0x42, 0xff}, + keyVals := map[[2]byte]uint64{ + {0x41, 0xff}: 3, + {0x42, 0x00}: 11, + {0xff}: 13, + {0x42, 0x56}: 17, + {0x00}: 19, + {0x42, 0xff}: 23, + {0x43, 0x00}: 33, } - vals := []uint64{11, 13, 17, 19, 23} expected := []uint64{11, 23} // Insert the keys and values into storage require.NoError(t, withWriter(func(writer storage.Writer) error { - for i, key := range keys { - err := operation.Upsert(key, vals[i])(writer) + for key, val := range keyVals { + err := operation.Upsert(key[:], val)(writer) if err != nil { return err } @@ -84,7 +85,7 @@ func TestTraverse(t *testing.T) { return nil })) - actual := make([]uint64, 0, len(keys)) + actual := make([]uint64, 0, len(keyVals)) // Define the iteration logic iterationFunc := func() (operation.CheckFunc, operation.CreateFunc, operation.HandleFunc) { diff --git a/storage/operation/writes_test.go b/storage/operation/writes_test.go index aa7b5020b1a..9355b5822db 100644 --- a/storage/operation/writes_test.go +++ b/storage/operation/writes_test.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/operation" "github.com/onflow/flow-go/storage/operation/dbtest" - "github.com/onflow/flow-go/utils/unittest" ) func TestReadWrite(t *testing.T) { @@ -230,16 +229,15 @@ func TestRemoveRange(t *testing.T) { // Remove the keys in the prefix range require.NoError(t, withWriter(operation.RemoveByPrefix(r, prefix))) - lg := unittest.Logger().With().Logger() // Verify that the keys in the prefix range have been removed for i, key := range keys { var exists bool require.NoError(t, operation.Exists(key, &exists)(r)) - lg.Info().Msgf("key %x exists: %t", key, exists) + t.Logf("key %x exists: %t", key, exists) deleted := includeStart <= i && i <= includeEnd - // deleted item should not exist + // An item that was not deleted must exist require.Equal(t, !deleted, exists, "expected key %x to be %s", key, map[bool]string{true: "deleted", false: "not deleted"}) } diff --git a/storage/operations.go b/storage/operations.go index c261d4ba28c..e0fe101f636 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -6,8 +6,8 @@ import ( // Iterator is an interface for iterating over key-value pairs in a storage backend. type Iterator interface { - // SeekGE seeks to the smallest key greater than or equal to the given key. - SeekGE() + // First seeks to the smallest key greater than or equal to the given key. + First() // Valid returns whether the iterator is positioned at a valid key-value pair. Valid() bool @@ -15,10 +15,11 @@ type Iterator interface { // Next advances the iterator to the next key-value pair. Next() - // Key returns the key of the current key-value pair, or nil if done. + // IterItem returns the current key-value pair, or nil if done. IterItem() IterItem // Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. + // No errors expected during normal operation Close() error } @@ -28,6 +29,7 @@ type IterItem interface { // Value returns the value of the current key-value pair // The reason it takes a function is to follow badgerDB's API pattern + // No errors expected during normal operation Value(func(val []byte) error) error } @@ -44,6 +46,7 @@ func DefaultIteratorOptions() IteratorOption { type Reader interface { // Get gets the value for the given key. It returns ErrNotFound if the DB // does not contain the key. + // other errors are exceptions // // The caller should not modify the contents of the returned slice, but it is // safe to modify the contents of the argument after Get returns. The @@ -51,7 +54,11 @@ type Reader interface { // success, the caller MUST call closer.Close() or a memory leak will occur. Get(key []byte) (value []byte, closer io.Closer, err error) - // NewIter returns a new Iterator for the given key range [startPrefix, endPrefix], both inclusive. + // NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. + // Specifically, all keys that meet ANY of the following conditions are included in the iteration: + // - have a prefix equal to startPrefix OR + // - have a prefix equal to the endPrefix OR + // - have a prefix that is lexicographically between startPrefix and endPrefix NewIter(startPrefix, endPrefix []byte, ops IteratorOption) (Iterator, error) } @@ -61,20 +68,26 @@ type Writer interface { // for that key; a DB is not a multi-map. // // It is safe to modify the contents of the arguments after Set returns. + // No errors expected during normal operation Set(k, v []byte) error // Delete deletes the value for the given key. Deletes are blind all will // succeed even if the given key does not exist. // // It is safe to modify the contents of the arguments after Delete returns. + // No errors expected during normal operation Delete(key []byte) error // DeleteByRange removes all keys with a prefix that falls within the // range [start, end], both inclusive. + // No errors expected during normal operation DeleteByRange(globalReader Reader, startPrefix, endPrefix []byte) error } // ReaderBatchWriter is an interface for reading and writing to a storage backend. +// It is useful for performing a related sequence of reads and writes, after which you would like +// to modify some non-database state if the sequence completed successfully (via AddCallback). +// If you are not using AddCallback, avoid using ReaderBatchWriter: use Reader and Writer directly. type ReaderBatchWriter interface { // GlobalReader returns a database-backed reader which reads the latest committed global database state ("read-committed isolation"). // This reader will not read writes written to ReaderBatchWriter.Writer until the write batch is committed. @@ -104,21 +117,21 @@ func OnCommitSucceed(b ReaderBatchWriter, onSuccessFn func()) { } func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte) { - // LowerBound specifies the smallest key to iterate and it's inclusive. - // UpperBound specifies the largest key to iterate and it's exclusive (not inclusive) - // in order to match all keys prefixed with the `end` bytes, we increment the bytes of end by 1, + // Return value lowerBound specifies the smallest key to iterate and it's inclusive. + // Return value upperBound specifies the largest key to iterate and it's exclusive (not inclusive) + // in order to match all keys prefixed with `endPrefix`, we increment the bytes of `endPrefix` by 1, // for instance, to iterate keys between "hello" and "world", // we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" // will all be included. - return startPrefix, prefixUpperBound(endPrefix) + return startPrefix, PrefixUpperBound(endPrefix) } -// prefixUpperBound returns a key K such that all possible keys beginning with the input prefix +// PrefixUpperBound returns a key K such that all possible keys beginning with the input prefix // sort lower than K according to the byte-wise lexicographic key ordering used by Pebble. // This is used to define an upper bound for iteration, when we want to iterate over // all keys beginning with a given prefix. // referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration -func prefixUpperBound(prefix []byte) []byte { +func PrefixUpperBound(prefix []byte) []byte { end := make([]byte, len(prefix)) copy(end, prefix) for i := len(end) - 1; i >= 0; i-- { diff --git a/utils/unittest/unittest.go b/utils/unittest/unittest.go index d15f39cd27c..5edcd3d477e 100644 --- a/utils/unittest/unittest.go +++ b/utils/unittest/unittest.go @@ -408,34 +408,6 @@ func TypedPebbleDB(t testing.TB, dir string, create func(string, *pebble.Options return db } -type PebbleWrapper struct { - db *pebble.DB -} - -func (p *PebbleWrapper) View(fn func(pebble.Reader) error) error { - return fn(p.db) -} - -func (p *PebbleWrapper) Update(fn func(pebble.Writer) error) error { - return fn(p.db) -} - -func (p *PebbleWrapper) DB() *pebble.DB { - return p.db -} - -func RunWithWrappedPebbleDB(t testing.TB, f func(p *PebbleWrapper)) { - RunWithTempDir(t, func(dir string) { - db, err := pebble.Open(dir, &pebble.Options{}) - require.NoError(t, err) - defer func() { - assert.NoError(t, db.Close()) - }() - f(&PebbleWrapper{db}) - }) - -} - func RunWithTypedPebbleDB( t testing.TB, create func(string, *pebble.Options) (*pebble.DB, error), From 17972e0e6a779473d71f9cc491b09b20ce0aff59 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 1 Nov 2024 17:05:58 -0700 Subject: [PATCH 04/25] update comments --- storage/operation/badgerimpl/iterator.go | 6 ++++++ storage/operation/badgerimpl/reader.go | 13 ++++++++++++ storage/operation/badgerimpl/writer.go | 27 ++++++++++++++++++++++++ storage/operation/pebbleimpl/iterator.go | 6 ++++++ storage/operation/pebbleimpl/reader.go | 13 ++++++++++++ storage/operation/pebbleimpl/writer.go | 27 +++++++++++++++++++++++- storage/operations.go | 1 + 7 files changed, 92 insertions(+), 1 deletion(-) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index e9f8b5dc6be..5cc5fc50340 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -34,10 +34,12 @@ func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage } } +// First seeks to the smallest key greater than or equal to the given key. func (i *badgerIterator) First() { i.iter.Seek(i.lowerBound) } +// Valid returns whether the iterator is positioned at a valid key-value pair. func (i *badgerIterator) Valid() bool { // if it's beyond the upper bound, it's invalid if !i.iter.Valid() { @@ -49,16 +51,20 @@ func (i *badgerIterator) Valid() bool { return valid } +// Next advances the iterator to the next key-value pair. func (i *badgerIterator) Next() { i.iter.Next() } +// IterItem returns the current key-value pair, or nil if done. func (i *badgerIterator) IterItem() storage.IterItem { return i.iter.Item() } var _ storage.IterItem = (*badger.Item)(nil) +// Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. +// No errors expected during normal operation func (i *badgerIterator) Close() error { i.iter.Close() return nil diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index 8d7d982d65e..a410067a6b7 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -20,6 +20,14 @@ var _ io.Closer = (*noopCloser)(nil) func (noopCloser) Close() error { return nil } +// Get gets the value for the given key. It returns ErrNotFound if the DB +// does not contain the key. +// other errors are exceptions +// +// The caller should not modify the contents of the returned slice, but it is +// safe to modify the contents of the argument after Get returns. The +// returned slice will remain valid until the returned Closer is closed. On +// success, the caller MUST call closer.Close() or a memory leak will occur. func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { tx := b.db.NewTransaction(false) defer tx.Discard() @@ -40,6 +48,11 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { return value, noopCloser{}, nil } +// NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. +// Specifically, all keys that meet ANY of the following conditions are included in the iteration: +// - have a prefix equal to startPrefix OR +// - have a prefix equal to the endPrefix OR +// - have a prefix that is lexicographically between startPrefix and endPrefix func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { return newBadgerIterator(b.db, startPrefix, endPrefix, ops), nil } diff --git a/storage/operation/badgerimpl/writer.go b/storage/operation/badgerimpl/writer.go index 3837be3917f..769187166ba 100644 --- a/storage/operation/badgerimpl/writer.go +++ b/storage/operation/badgerimpl/writer.go @@ -19,22 +19,36 @@ type ReaderBatchWriter struct { var _ storage.ReaderBatchWriter = (*ReaderBatchWriter)(nil) +// GlobalReader returns a database-backed reader which reads the latest committed global database state ("read-committed isolation"). +// This reader will not read writes written to ReaderBatchWriter.Writer until the write batch is committed. +// This reader may observe different values for the same key on subsequent reads. func (b *ReaderBatchWriter) GlobalReader() storage.Reader { return b.globalReader } +// Writer returns a writer associated with a batch of writes. The batch is pending until it is committed. +// When we `Write` into the batch, that write operation is added to the pending batch, but not committed. +// The commit operation is atomic w.r.t. the batch; either all writes are applied to the database, or no writes are. +// Note: +// - The writer cannot be used concurrently for writing. func (b *ReaderBatchWriter) Writer() storage.Writer { return b } +// BadgerWriteBatch returns the badger write batch func (b *ReaderBatchWriter) BadgerWriteBatch() *badger.WriteBatch { return b.batch } +// AddCallback adds a callback to execute after the batch has been flush +// regardless the batch update is succeeded or failed. +// The error parameter is the error returned by the batch update. func (b *ReaderBatchWriter) AddCallback(callback func(error)) { b.callbacks.AddCallback(callback) } +// Commit flushes the batch to the database. +// No errors expected during normal operation func (b *ReaderBatchWriter) Commit() error { err := b.batch.Flush() @@ -69,14 +83,27 @@ func NewReaderBatchWriter(db *badger.DB) *ReaderBatchWriter { var _ storage.Writer = (*ReaderBatchWriter)(nil) +// Set sets the value for the given key. It overwrites any previous value +// for that key; a DB is not a multi-map. +// +// It is safe to modify the contents of the arguments after Set returns. +// No errors expected during normal operation func (b *ReaderBatchWriter) Set(key, value []byte) error { return b.batch.Set(key, value) } +// Delete deletes the value for the given key. Deletes are blind all will +// succeed even if the given key does not exist. +// +// It is safe to modify the contents of the arguments after Delete returns. +// No errors expected during normal operation func (b *ReaderBatchWriter) Delete(key []byte) error { return b.batch.Delete(key) } +// DeleteByRange removes all keys with a prefix that falls within the +// range [start, end], both inclusive. +// No errors expected during normal operation func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { err := operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { err := b.batch.Delete(key) diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index b1b6630cc51..bc0cd2bae69 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -32,18 +32,22 @@ func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops }, nil } +// First seeks to the smallest key greater than or equal to the given key. func (i *pebbleIterator) First() { i.iter.First() } +// Valid returns whether the iterator is positioned at a valid key-value pair. func (i *pebbleIterator) Valid() bool { return i.iter.Valid() } +// Next advances the iterator to the next key-value pair. func (i *pebbleIterator) Next() { i.iter.Next() } +// IterItem returns the current key-value pair, or nil if done. func (i *pebbleIterator) IterItem() storage.IterItem { return pebbleIterItem{iter: i.iter} } @@ -67,6 +71,8 @@ func (i pebbleIterItem) Value(fn func([]byte) error) error { return fn(val) } +// Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. +// No errors expected during normal operation func (i *pebbleIterator) Close() error { return i.iter.Close() } diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go index 6cfdfd93da5..cff5a916048 100644 --- a/storage/operation/pebbleimpl/reader.go +++ b/storage/operation/pebbleimpl/reader.go @@ -22,6 +22,14 @@ var _ io.Closer = (*noopCloser)(nil) func (noopCloser) Close() error { return nil } +// Get gets the value for the given key. It returns ErrNotFound if the DB +// does not contain the key. +// other errors are exceptions +// +// The caller should not modify the contents of the returned slice, but it is +// safe to modify the contents of the argument after Get returns. The +// returned slice will remain valid until the returned Closer is closed. On +// success, the caller MUST call closer.Close() or a memory leak will occur. func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { value, closer, err := b.db.Get(key) @@ -37,6 +45,11 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { return value, closer, nil } +// NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. +// Specifically, all keys that meet ANY of the following conditions are included in the iteration: +// - have a prefix equal to startPrefix OR +// - have a prefix equal to the endPrefix OR +// - have a prefix that is lexicographically between startPrefix and endPrefix func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { return newPebbleIterator(b.db, startPrefix, endPrefix, ops) } diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index c6ccdff06b9..3525bb59f2c 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -16,10 +16,18 @@ type ReaderBatchWriter struct { var _ storage.ReaderBatchWriter = (*ReaderBatchWriter)(nil) +// GlobalReader returns a database-backed reader which reads the latest committed global database state ("read-committed isolation"). +// This reader will not read writes written to ReaderBatchWriter.Writer until the write batch is committed. +// This reader may observe different values for the same key on subsequent reads. func (b *ReaderBatchWriter) GlobalReader() storage.Reader { return b.globalReader } +// Writer returns a writer associated with a batch of writes. The batch is pending until it is committed. +// When we `Write` into the batch, that write operation is added to the pending batch, but not committed. +// The commit operation is atomic w.r.t. the batch; either all writes are applied to the database, or no writes are. +// Note: +// - The writer cannot be used concurrently for writing. func (b *ReaderBatchWriter) Writer() storage.Writer { return b } @@ -28,10 +36,15 @@ func (b *ReaderBatchWriter) PebbleWriterBatch() *pebble.Batch { return b.batch } +// AddCallback adds a callback to execute after the batch has been flush +// regardless the batch update is succeeded or failed. +// The error parameter is the error returned by the batch update. func (b *ReaderBatchWriter) AddCallback(callback func(error)) { b.callbacks.AddCallback(callback) } +// Commit flushes the batch to the database. +// No errors expected during normal operation func (b *ReaderBatchWriter) Commit() error { err := b.batch.Commit(pebble.Sync) @@ -66,15 +79,27 @@ func NewReaderBatchWriter(db *pebble.DB) *ReaderBatchWriter { var _ storage.Writer = (*ReaderBatchWriter)(nil) +// Set sets the value for the given key. It overwrites any previous value +// for that key; a DB is not a multi-map. +// +// It is safe to modify the contents of the arguments after Set returns. +// No errors expected during normal operation func (b *ReaderBatchWriter) Set(key, value []byte) error { return b.batch.Set(key, value, pebble.Sync) } +// Delete deletes the value for the given key. Deletes are blind all will +// succeed even if the given key does not exist. +// +// It is safe to modify the contents of the arguments after Delete returns. +// No errors expected during normal operation func (b *ReaderBatchWriter) Delete(key []byte) error { return b.batch.Delete(key, pebble.Sync) } -// DeleteByRange deletes all keys with a prefix in the range [startPrefix, endPrefix] (both inclusive). +// DeleteByRange removes all keys with a prefix that falls within the +// range [start, end], both inclusive. +// No errors expected during normal operation func (b *ReaderBatchWriter) DeleteByRange(_ storage.Reader, startPrefix, endPrefix []byte) error { // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). // therefore, we need to increment the endPrefix to make it inclusive. diff --git a/storage/operations.go b/storage/operations.go index e0fe101f636..d407da299e7 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -63,6 +63,7 @@ type Reader interface { } // Writer is an interface for batch writing to a storage backend. +// It cannot be used concurrently for writing. type Writer interface { // Set sets the value for the given key. It overwrites any previous value // for that key; a DB is not a multi-map. From eea7a2c71d9101a179f8cd375aef50c9f616aed9 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 12:21:20 -0800 Subject: [PATCH 05/25] add comments --- storage/operation/badgerimpl/iterator.go | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index 5cc5fc50340..a1cbb6dbe1f 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -41,12 +41,23 @@ func (i *badgerIterator) First() { // Valid returns whether the iterator is positioned at a valid key-value pair. func (i *badgerIterator) Valid() bool { - // if it's beyond the upper bound, it's invalid + // Note: we didn't specify the iteration range with the badger IteratorOptions, + // because the IterationOptions only allows us to specify a single prefix, whereas + // we need to specify a range of prefixes. So we have to manually check the bounds here. + // The First() method, which calls Seek(i.lowerBound), ensures the iteration starts from + // the lowerBound, and the upperbound is checked here by first checking if it's + // reaching the end of the iteration, then checking if the key is within the upperbound. + + // check if it's reaching the end of the iteration if !i.iter.Valid() { return false } + + // check if the key is within the upperbound (exclusive) key := i.iter.Item().Key() - // "< 0" means "key < upperBound" + // note: for the boundary case, + // upperBound is the exclusive upper bound, should not be included in the iteration, + // so if key == upperBound, it's invalid, should return false. valid := bytes.Compare(key, i.upperBound) < 0 return valid } From 2a36b5701533af61d267f40f3d4bce7c38d7a336 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 12:54:40 -0800 Subject: [PATCH 06/25] fix the boundary iteration case for badger --- storage/operation/badgerimpl/iterator.go | 5 +++ storage/operation/reads_test.go | 52 ++++++++++++++++++++++++ 2 files changed, 57 insertions(+) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index a1cbb6dbe1f..58c38d0a909 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -53,6 +53,11 @@ func (i *badgerIterator) Valid() bool { return false } + // if upper bound is nil, then there's no upper bound, so it's always valid + if i.upperBound == nil { + return true + } + // check if the key is within the upperbound (exclusive) key := i.iter.Item().Key() // note: for the boundary case, diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index b9addec418d..0857ce0740c 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -61,6 +61,58 @@ func TestIterateKeysInPrefixRange(t *testing.T) { }) } +func TestIterationBoundary(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + // Define the prefix range + prefixStart := []byte{0x01} + prefixEnd := []byte{0xff} + + // Create a range of keys around the prefix start/end values + keys := [][]byte{ + {0x00}, + {0x00, 0x00}, + {0x00, 0xff}, + {0x01}, + {0x01, 0x00}, + {0x01, 0xff}, + {0x02}, + {0xff}, + {0xff, 0x00}, + {0xff, 0xff}, + } + + expectedKeys := [][]byte{ + {0x01}, + {0x01, 0x00}, + {0x01, 0xff}, + {0x02}, + {0xff}, + {0xff, 0x00}, + {0xff, 0xff}, + } + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + // Forward iteration and check boundaries + var found [][]byte + require.NoError(t, operation.IterateKeysInPrefixRange(prefixStart, prefixEnd, func(key []byte) error { + found = append(found, key) + return nil + })(r), "should iterate forward without error") + require.ElementsMatch(t, expectedKeys, found, "forward iteration should return the correct keys in range") + }) +} + func TestTraverse(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { keyVals := map[[2]byte]uint64{ From 47664215decc4dd511f035a8eab602980f53adaf Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 13:21:22 -0800 Subject: [PATCH 07/25] add review comments --- storage/operation/badgerimpl/reader.go | 2 ++ storage/operation/badgerimpl/writer.go | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index a410067a6b7..1b2bd00d7c8 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -53,6 +53,8 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { // - have a prefix equal to startPrefix OR // - have a prefix equal to the endPrefix OR // - have a prefix that is lexicographically between startPrefix and endPrefix +// +// no errors are expected during normal operation func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { return newBadgerIterator(b.db, startPrefix, endPrefix, ops), nil } diff --git a/storage/operation/badgerimpl/writer.go b/storage/operation/badgerimpl/writer.go index 769187166ba..9957bc82445 100644 --- a/storage/operation/badgerimpl/writer.go +++ b/storage/operation/badgerimpl/writer.go @@ -20,7 +20,7 @@ type ReaderBatchWriter struct { var _ storage.ReaderBatchWriter = (*ReaderBatchWriter)(nil) // GlobalReader returns a database-backed reader which reads the latest committed global database state ("read-committed isolation"). -// This reader will not read writes written to ReaderBatchWriter.Writer until the write batch is committed. +// This reader will not read un-committed writes written to ReaderBatchWriter.Writer until the write batch is committed. // This reader may observe different values for the same key on subsequent reads. func (b *ReaderBatchWriter) GlobalReader() storage.Reader { return b.globalReader From a725247cc8783586916de9cf1f33fc9a43140591 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 14:57:33 -0800 Subject: [PATCH 08/25] fix DeleteRange --- storage/operation/pebbleimpl/writer.go | 17 ++++++++-- storage/operation/reads_test.go | 12 +++++++ storage/operation/writes_test.go | 47 ++++++++++++++++++++++++++ storage/operations.go | 34 ++++++++++++++----- 4 files changed, 99 insertions(+), 11 deletions(-) diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index 3525bb59f2c..af6da1b142b 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -1,9 +1,12 @@ package pebbleimpl import ( + "fmt" + "github.com/cockroachdb/pebble" "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/storage/operation" op "github.com/onflow/flow-go/storage/operation" ) @@ -100,9 +103,19 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { // DeleteByRange removes all keys with a prefix that falls within the // range [start, end], both inclusive. // No errors expected during normal operation -func (b *ReaderBatchWriter) DeleteByRange(_ storage.Reader, startPrefix, endPrefix []byte) error { +func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). // therefore, we need to increment the endPrefix to make it inclusive. start, end := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) - return b.batch.DeleteRange(start, end, pebble.Sync) + if len(end) > 0 { + return b.batch.DeleteRange(start, end, pebble.Sync) + } + + return operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { + err := b.batch.Delete(key, pebble.Sync) + if err != nil { + return fmt.Errorf("could not add key to delete batch (%v): %w", key, err) + } + return nil + })(globalReader) } diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index 0857ce0740c..a958a244d95 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -61,6 +61,18 @@ func TestIterateKeysInPrefixRange(t *testing.T) { }) } +// TestIterateInvalidRange tests that error should return if startPrefix > endPrefix +func TestIterateInvalidRange(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + + var found [][]byte + require.Error(t, operation.IterateKeysInPrefixRange([]byte{0x02}, []byte{0x01}, func(key []byte) error { + found = append(found, key) + return nil + })(r)) + }) +} + func TestIterationBoundary(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { // Define the prefix range diff --git a/storage/operation/writes_test.go b/storage/operation/writes_test.go index 9355b5822db..6f32480d77e 100644 --- a/storage/operation/writes_test.go +++ b/storage/operation/writes_test.go @@ -244,6 +244,53 @@ func TestRemoveRange(t *testing.T) { }) } +func TestRemoveFrom(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + + // Define the prefix + prefix := []byte{0xff} + + // Create a range of keys around the boundaries of the prefix + keys := [][]byte{ + {0x10, 0x00}, + {0xff}, + {0xff, 0x00}, + {0xff, 0xff}, + } + + // Keys expected to be in the prefix range + includeStart, includeEnd := 1, 3 + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + // Remove the keys in the prefix range + require.NoError(t, withWriter(operation.RemoveByPrefix(r, prefix))) + + // Verify that the keys in the prefix range have been removed + for i, key := range keys { + var exists bool + require.NoError(t, operation.Exists(key, &exists)(r)) + t.Logf("key %x exists: %t", key, exists) + + deleted := includeStart <= i && i <= includeEnd + + // An item that was not deleted must exist + require.Equal(t, !deleted, exists, + fmt.Errorf("a key %x should be deleted (%v), but actually exists (%v)", key, deleted, exists)) + } + }) +} + type Entity struct { ID uint64 } diff --git a/storage/operations.go b/storage/operations.go index d407da299e7..fc5984dba1c 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -5,17 +5,30 @@ import ( ) // Iterator is an interface for iterating over key-value pairs in a storage backend. +// A common usage is: +// +// defer it.Close() +// +// for it.First(); it.Valid(); it.Next() { +// item := it.IterItem() +// } type Iterator interface { // First seeks to the smallest key greater than or equal to the given key. + // This method must be called because it's necessary for the badger implementation + // to move the iteration cursor to the first key in the iteration range. + // This method must be called before calling Valid, Next, IterItem, or Close. First() // Valid returns whether the iterator is positioned at a valid key-value pair. + // If Valid returns false, the iterator is done and must be closed. Valid() bool // Next advances the iterator to the next key-value pair. + // The next key-value pair might be invalid, so you should call Valid() to check. Next() // IterItem returns the current key-value pair, or nil if done. + // A best practice is always to call Valid() before calling IterItem. IterItem() IterItem // Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. @@ -49,7 +62,7 @@ type Reader interface { // other errors are exceptions // // The caller should not modify the contents of the returned slice, but it is - // safe to modify the contents of the argument after Get returns. The + // safe to modify the contents of the `key` argument after Get returns. The // returned slice will remain valid until the returned Closer is closed. On // success, the caller MUST call closer.Close() or a memory leak will occur. Get(key []byte) (value []byte, closer io.Closer, err error) @@ -63,7 +76,7 @@ type Reader interface { } // Writer is an interface for batch writing to a storage backend. -// It cannot be used concurrently for writing. +// One Writer instance cannot be used concurrently by multiple goroutines. type Writer interface { // Set sets the value for the given key. It overwrites any previous value // for that key; a DB is not a multi-map. @@ -117,18 +130,21 @@ func OnCommitSucceed(b ReaderBatchWriter, onSuccessFn func()) { }) } +// StartEndPrefixToLowerUpperBound returns the lower and upper bounds for a range of keys +// specified by the start and end prefixes. +// the lower and upper bounds are used for the key iteration. +// The return value lowerBound specifies the smallest key to iterate and it's inclusive. +// The return value upperBound specifies the largest key to iterate and it's exclusive (not inclusive) +// in order to match all keys prefixed with `endPrefix`, we increment the bytes of `endPrefix` by 1, +// for instance, to iterate keys between "hello" and "world", +// we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" +// will all be included. func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte) { - // Return value lowerBound specifies the smallest key to iterate and it's inclusive. - // Return value upperBound specifies the largest key to iterate and it's exclusive (not inclusive) - // in order to match all keys prefixed with `endPrefix`, we increment the bytes of `endPrefix` by 1, - // for instance, to iterate keys between "hello" and "world", - // we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" - // will all be included. return startPrefix, PrefixUpperBound(endPrefix) } // PrefixUpperBound returns a key K such that all possible keys beginning with the input prefix -// sort lower than K according to the byte-wise lexicographic key ordering used by Pebble. +// sort lower than K according to the byte-wise lexicographic key ordering. // This is used to define an upper bound for iteration, when we want to iterate over // all keys beginning with a given prefix. // referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration From b5eaf02ea3e633c1486c63b0f5a7ac60639d7e84 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 15:12:05 -0800 Subject: [PATCH 09/25] refactor StartEndPrefixToLowerUpperBound --- storage/operation/badgerimpl/iterator.go | 18 ++++++++++-------- storage/operation/pebbleimpl/iterator.go | 7 ++++++- storage/operation/pebbleimpl/writer.go | 4 ++-- storage/operations.go | 14 ++++++++++++-- 4 files changed, 30 insertions(+), 13 deletions(-) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index 58c38d0a909..7d4eb8dea6c 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -9,9 +9,10 @@ import ( ) type badgerIterator struct { - iter *badger.Iterator - lowerBound []byte - upperBound []byte + iter *badger.Iterator + lowerBound []byte + upperBound []byte + hasUpperBound bool // whether there's an upper bound } var _ storage.Iterator = (*badgerIterator)(nil) @@ -25,12 +26,13 @@ func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage tx := db.NewTransaction(false) iter := tx.NewIterator(options) - lowerBound, upperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + lowerBound, upperBound, hasUpperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) return &badgerIterator{ - iter: iter, - lowerBound: lowerBound, - upperBound: upperBound, + iter: iter, + lowerBound: lowerBound, + upperBound: upperBound, + hasUpperBound: hasUpperBound, } } @@ -54,7 +56,7 @@ func (i *badgerIterator) Valid() bool { } // if upper bound is nil, then there's no upper bound, so it's always valid - if i.upperBound == nil { + if !i.hasUpperBound { return true } diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index bc0cd2bae69..6b72fed04bd 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -15,13 +15,18 @@ type pebbleIterator struct { var _ storage.Iterator = (*pebbleIterator)(nil) func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops storage.IteratorOption) (*pebbleIterator, error) { - lowerBound, upperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + lowerBound, upperBound, hasUpperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) options := pebble.IterOptions{ LowerBound: lowerBound, UpperBound: upperBound, } + // setting UpperBound to nil if there is no upper bound + if !hasUpperBound { + options.UpperBound = nil + } + iter, err := reader.NewIter(&options) if err != nil { return nil, fmt.Errorf("can not create iterator: %w", err) diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index af6da1b142b..00f7aec3add 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -106,8 +106,8 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). // therefore, we need to increment the endPrefix to make it inclusive. - start, end := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) - if len(end) > 0 { + start, end, hasUpperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) + if hasUpperBound { return b.batch.DeleteRange(start, end, pebble.Sync) } diff --git a/storage/operations.go b/storage/operations.go index fc5984dba1c..d2e0970e6d7 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -135,12 +135,20 @@ func OnCommitSucceed(b ReaderBatchWriter, onSuccessFn func()) { // the lower and upper bounds are used for the key iteration. // The return value lowerBound specifies the smallest key to iterate and it's inclusive. // The return value upperBound specifies the largest key to iterate and it's exclusive (not inclusive) +// The return value hasUpperBound specifies whether there is upperBound // in order to match all keys prefixed with `endPrefix`, we increment the bytes of `endPrefix` by 1, // for instance, to iterate keys between "hello" and "world", // we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" // will all be included. -func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte) { - return startPrefix, PrefixUpperBound(endPrefix) +func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte, hasUpperBound bool) { + // if the endPrefix is all 1s, such as []byte{0xff, 0xff, ...}, there is no upper-bound + // so we return the startPrefix as the lower-bound, and nil as the upper-bound, and false for hasUpperBound + upperBound = PrefixUpperBound(endPrefix) + if upperBound == nil { + return startPrefix, nil, false + } + + return startPrefix, upperBound, true } // PrefixUpperBound returns a key K such that all possible keys beginning with the input prefix @@ -148,6 +156,8 @@ func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, // This is used to define an upper bound for iteration, when we want to iterate over // all keys beginning with a given prefix. // referred to https://pkg.go.dev/github.com/cockroachdb/pebble#example-Iterator-PrefixIteration +// when the prefix is all 1s, such as []byte{0xff}, or []byte(0xff, 0xff} etc, there is no upper-bound +// It returns nil in this case. func PrefixUpperBound(prefix []byte) []byte { end := make([]byte, len(prefix)) copy(end, prefix) From 47ef8aa7a0344dbd0b5415bb814061404fc14b96 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 15:23:51 -0800 Subject: [PATCH 10/25] add prefix check in NewIter --- storage/operation/badgerimpl/reader.go | 7 +++++++ storage/operation/pebbleimpl/reader.go | 9 +++++++++ 2 files changed, 16 insertions(+) diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index 1b2bd00d7c8..29a11bfaeb9 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -1,7 +1,9 @@ package badgerimpl import ( + "bytes" "errors" + "fmt" "io" "github.com/dgraph-io/badger/v2" @@ -54,8 +56,13 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { // - have a prefix equal to the endPrefix OR // - have a prefix that is lexicographically between startPrefix and endPrefix // +// it returns error if the startPrefix key is greater than the endPrefix key // no errors are expected during normal operation func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { + if bytes.Compare(startPrefix, endPrefix) > 0 { + return nil, fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } + return newBadgerIterator(b.db, startPrefix, endPrefix, ops), nil } diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go index cff5a916048..648c4d78696 100644 --- a/storage/operation/pebbleimpl/reader.go +++ b/storage/operation/pebbleimpl/reader.go @@ -1,7 +1,9 @@ package pebbleimpl import ( + "bytes" "errors" + "fmt" "io" "github.com/cockroachdb/pebble" @@ -50,7 +52,14 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { // - have a prefix equal to startPrefix OR // - have a prefix equal to the endPrefix OR // - have a prefix that is lexicographically between startPrefix and endPrefix +// +// it returns error if the startPrefix key is greater than the endPrefix key +// no errors are expected during normal operation func (b dbReader) NewIter(startPrefix, endPrefix []byte, ops storage.IteratorOption) (storage.Iterator, error) { + if bytes.Compare(startPrefix, endPrefix) > 0 { + return nil, fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } + return newPebbleIterator(b.db, startPrefix, endPrefix, ops) } From 18c0a179ec03294e7351d5ca489271f19555e913 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 15:37:02 -0800 Subject: [PATCH 11/25] add RemoveByRange --- storage/operation/badgerimpl/writer.go | 3 +- storage/operation/pebbleimpl/writer.go | 8 +++- storage/operation/reads.go | 4 +- storage/operation/writes.go | 18 +++++++-- storage/operation/writes_test.go | 55 +++++++++++++++++++++++++- 5 files changed, 80 insertions(+), 8 deletions(-) diff --git a/storage/operation/badgerimpl/writer.go b/storage/operation/badgerimpl/writer.go index 9957bc82445..d0cc9ee389a 100644 --- a/storage/operation/badgerimpl/writer.go +++ b/storage/operation/badgerimpl/writer.go @@ -103,7 +103,8 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { // DeleteByRange removes all keys with a prefix that falls within the // range [start, end], both inclusive. -// No errors expected during normal operation +// It returns error if endPrefix < startPrefix +// no other errors are expected during normal operation func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { err := operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { err := b.batch.Delete(key) diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index 00f7aec3add..6746e5effea 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -1,6 +1,7 @@ package pebbleimpl import ( + "bytes" "fmt" "github.com/cockroachdb/pebble" @@ -102,8 +103,13 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { // DeleteByRange removes all keys with a prefix that falls within the // range [start, end], both inclusive. -// No errors expected during normal operation +// It returns error if endPrefix < startPrefix +// no other errors are expected during normal operation func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { + if bytes.Compare(startPrefix, endPrefix) > 0 { + return fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } + // DeleteRange takes the prefix range with start (inclusive) and end (exclusive, note: not inclusive). // therefore, we need to increment the endPrefix to make it inclusive. start, end, hasUpperBound := storage.StartEndPrefixToLowerUpperBound(startPrefix, endPrefix) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 1be299ab9d8..5fa73362fa0 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -17,11 +17,11 @@ import ( // return (false, err) if running into any exception, the iteration should be stopped. type CheckFunc func(key []byte) (bool, error) -// createFunc returns a pointer to an initialized entity that we can potentially +// CreateFunc returns a pointer to an initialized entity that we can potentially // decode the next value into during a badger DB iteration. type CreateFunc func() interface{} -// handleFunc is a function that starts the processing of the current key-value +// HandleFunc is a function that starts the processing of the current key-value // pair during a badger iteration. It should be called after the key was checked // and the entity was decoded. // No errors are expected during normal operation. Any errors will halt the iteration. diff --git a/storage/operation/writes.go b/storage/operation/writes.go index 3bbe08d12d2..590526b4686 100644 --- a/storage/operation/writes.go +++ b/storage/operation/writes.go @@ -1,6 +1,9 @@ package operation import ( + "bytes" + "fmt" + "github.com/vmihailenco/msgpack" "github.com/onflow/flow-go/module/irrecoverable" @@ -43,13 +46,22 @@ func Remove(key []byte) func(storage.Writer) error { } } -// RemoveByPrefix removes all keys with the given prefix defined by [startPrefix, endPrefix] (both inclusive). -// If no keys exist with the given prefix, this is a no-op. +// RemoveByPrefix removes all keys with the given prefix // Error returns: // * generic error in case of unexpected database error func RemoveByPrefix(reader storage.Reader, key []byte) func(storage.Writer) error { + return RemoveByRange(reader, key, key) +} + +// RemoveByRange removes all keys with a prefix that falls within the range [start, end], both inclusive. +// It returns error if endPrefix < startPrefix +// no other errors are expected during normal operation +func RemoveByRange(reader storage.Reader, startPrefix []byte, endPrefix []byte) func(storage.Writer) error { return func(w storage.Writer) error { - err := w.DeleteByRange(reader, key, key) + if bytes.Compare(startPrefix, endPrefix) > 0 { + return fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } + err := w.DeleteByRange(reader, startPrefix, endPrefix) if err != nil { return irrecoverable.NewExceptionf("could not delete item: %w", err) } diff --git a/storage/operation/writes_test.go b/storage/operation/writes_test.go index 6f32480d77e..4b549cc56c4 100644 --- a/storage/operation/writes_test.go +++ b/storage/operation/writes_test.go @@ -192,7 +192,7 @@ func TestConcurrentRemove(t *testing.T) { }) } -func TestRemoveRange(t *testing.T) { +func TestRemoveByPrefix(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { // Define the prefix @@ -244,6 +244,59 @@ func TestRemoveRange(t *testing.T) { }) } +func TestRemoveByRange(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + + startPrefix, endPrefix := []byte{0x10}, []byte{0x12} + // Create a range of keys around the boundaries of the prefix + keys := [][]byte{ + {0x09, 0xff}, + // within the range + {0x10, 0x00}, + {0x10, 0x50}, + {0x10, 0xff}, + {0x11}, + {0x12}, + {0x12, 0x00}, + {0x12, 0xff}, + // after end -> not included in range + {0x13}, + {0x1A, 0xff}, + } + + // Keys expected to be in the prefix range + includeStart, includeEnd := 1, 7 + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + // Remove the keys in the prefix range + require.NoError(t, withWriter(operation.RemoveByRange(r, startPrefix, endPrefix))) + + // Verify that the keys in the prefix range have been removed + for i, key := range keys { + var exists bool + require.NoError(t, operation.Exists(key, &exists)(r)) + t.Logf("key %x exists: %t", key, exists) + + deleted := includeStart <= i && i <= includeEnd + + // An item that was not deleted must exist + require.Equal(t, !deleted, exists, + "expected key %x to be %s", key, map[bool]string{true: "deleted", false: "not deleted"}) + } + }) +} + func TestRemoveFrom(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { From cd39e67620ae2c4e2e1b404d22e226f0db37e27b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 16:06:41 -0800 Subject: [PATCH 12/25] refactor to use KeyCopy --- storage/operation/pebbleimpl/iterator.go | 5 +++++ storage/operation/reads.go | 16 ++++++++++++---- storage/operations.go | 13 +++++++++++-- 3 files changed, 28 insertions(+), 6 deletions(-) diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index 6b72fed04bd..79d39068e21 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -67,6 +67,11 @@ func (i pebbleIterItem) Key() []byte { return i.iter.Key() } +// KeyCopy returns a copy of the key of the item, writing it to dst slice. +func (i pebbleIterItem) KeyCopy(dst []byte) []byte { + return append(dst[:0], i.iter.Key()...) +} + func (i pebbleIterItem) Value(fn func([]byte) error) error { val, err := i.iter.ValueAndErr() if err != nil { diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 5fa73362fa0..5690b9c6ead 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -28,7 +28,10 @@ type CreateFunc func() interface{} type HandleFunc func() error type IterationFunc func() (CheckFunc, CreateFunc, HandleFunc) -// IterateKeysInPrefixRange will iterate over all keys with prefixes in the range [startPrefix, endPrefix] (both inclusive) +// IterateKeysInPrefixRange will iterate over all entries in the database, where the key starts with a prefixes in +// the range [startPrefix, endPrefix] (both inclusive). We require that startPrefix <= endPrefix (otherwise this +// function errors). On every such key, the `check` function is called. If `check` errors, iteration is aborted. +// No errors expected during normal operations. func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(key []byte) error) func(storage.Reader) error { return Iterate(startPrefix, endPrefix, func() (CheckFunc, CreateFunc, HandleFunc) { return func(key []byte) (bool, error) { @@ -164,8 +167,10 @@ func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { } } -// FindHighestAtOrBelow finds the highest key with the given prefix and -// height equal to or below the given height. +// FindHighestAtOrBelow is for database entries that are indexed by block height. It is suitable to search +// keys with the format prefix` + `height` (where "+" denotes concatenation of binary strings). The height +// is encoded as Big-Endian (entries with numerically smaller height have lexicographically smaller key). +// The function finds the *highest* key with the given prefix and height equal to or below the given height. func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func(storage.Reader) error { return func(r storage.Reader) error { if len(prefix) == 0 { @@ -180,9 +185,12 @@ func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func defer it.Close() var highestKey []byte + // find highest value below the given height for it.First(); it.Valid(); it.Next() { - highestKey = it.IterItem().Key() + // copy the key to avoid the underlying slices of the key + // being modified by the Next() call + highestKey = it.IterItem().KeyCopy(highestKey) } if len(highestKey) == 0 { diff --git a/storage/operations.go b/storage/operations.go index d2e0970e6d7..b9724d779a7 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -27,8 +27,9 @@ type Iterator interface { // The next key-value pair might be invalid, so you should call Valid() to check. Next() - // IterItem returns the current key-value pair, or nil if done. - // A best practice is always to call Valid() before calling IterItem. + // IterItem returns the current key-value pair, or nil if Valid returns false. + // Always to call Valid() before calling IterItem. + // Note, the returned item is only valid until the Next() method is called. IterItem() IterItem // Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. @@ -38,8 +39,16 @@ type Iterator interface { // IterItem is an interface for iterating over key-value pairs in a storage backend. type IterItem interface { + // Key returns the key of the current key-value pair + // Key is only valid until the Iterator.Next() method is called + // If you need to use it outside its validity, please use KeyCopy Key() []byte + // KeyCopy returns a copy of the key of the item, writing it to dst slice. + // If nil is passed, or capacity of dst isn't sufficient, a new slice would be allocated and + // returned. + KeyCopy(dst []byte) []byte + // Value returns the value of the current key-value pair // The reason it takes a function is to follow badgerDB's API pattern // No errors expected during normal operation From c68ab0d35b13080fc63febe6130e1ce5e5ddc5ad Mon Sep 17 00:00:00 2001 From: Leo Zhang Date: Fri, 6 Dec 2024 09:45:01 -0800 Subject: [PATCH 13/25] Apply suggestions from code review Co-authored-by: Alexander Hentschel --- storage/operations.go | 1 + 1 file changed, 1 insertion(+) diff --git a/storage/operations.go b/storage/operations.go index b9724d779a7..594271a27e2 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -77,6 +77,7 @@ type Reader interface { Get(key []byte) (value []byte, closer io.Closer, err error) // NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. + // We require that startPrefix ≤ endPrefix (otherwise this function errors). // Specifically, all keys that meet ANY of the following conditions are included in the iteration: // - have a prefix equal to startPrefix OR // - have a prefix equal to the endPrefix OR From dac89f262dacf60b987fe101ff4c2030b3c43bd0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 6 Dec 2024 09:45:19 -0800 Subject: [PATCH 14/25] add comments --- storage/operation/reads_test.go | 1 + storage/operations.go | 2 ++ 2 files changed, 3 insertions(+) diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index a958a244d95..74c59b10a4a 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -73,6 +73,7 @@ func TestIterateInvalidRange(t *testing.T) { }) } +// TestIterationBoundary tests that when the prefixEnd is all 1s, the iteration should include the last key func TestIterationBoundary(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { // Define the prefix range diff --git a/storage/operations.go b/storage/operations.go index 594271a27e2..8c3392765e2 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -150,6 +150,8 @@ func OnCommitSucceed(b ReaderBatchWriter, onSuccessFn func()) { // for instance, to iterate keys between "hello" and "world", // we use "hello" as LowerBound, "worle" as UpperBound, so that "world", "world1", "worldffff...ffff" // will all be included. +// In the case that the endPrefix is all 1s, such as []byte{0xff, 0xff, ...}, there is no upper-bound, +// it returns (startPrefix, nil, false) func StartEndPrefixToLowerUpperBound(startPrefix, endPrefix []byte) (lowerBound, upperBound []byte, hasUpperBound bool) { // if the endPrefix is all 1s, such as []byte{0xff, 0xff, ...}, there is no upper-bound // so we return the startPrefix as the lower-bound, and nil as the upper-bound, and false for hasUpperBound From f69cbcc1ddae765638669d1823d951eebf187cfb Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 6 Dec 2024 10:30:12 -0800 Subject: [PATCH 15/25] update comments for Get method --- storage/operation/badgerimpl/reader.go | 5 ++-- storage/operation/pebbleimpl/reader.go | 5 ++-- storage/operation/reads.go | 37 +++++++++++++++++--------- storage/operations.go | 5 ++-- 4 files changed, 33 insertions(+), 19 deletions(-) diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index 29a11bfaeb9..aebe9b854d2 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -28,8 +28,9 @@ func (noopCloser) Close() error { return nil } // // The caller should not modify the contents of the returned slice, but it is // safe to modify the contents of the argument after Get returns. The -// returned slice will remain valid until the returned Closer is closed. On -// success, the caller MUST call closer.Close() or a memory leak will occur. +// returned slice will remain valid until the returned Closer is closed. +// when err == nil, the caller MUST call closer.Close() or a memory leak will occur. +// when err != nil, then the caller io.Closer is nil, and should not be called func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { tx := b.db.NewTransaction(false) defer tx.Discard() diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go index 648c4d78696..b19262fd2ac 100644 --- a/storage/operation/pebbleimpl/reader.go +++ b/storage/operation/pebbleimpl/reader.go @@ -30,8 +30,9 @@ func (noopCloser) Close() error { return nil } // // The caller should not modify the contents of the returned slice, but it is // safe to modify the contents of the argument after Get returns. The -// returned slice will remain valid until the returned Closer is closed. On -// success, the caller MUST call closer.Close() or a memory leak will occur. +// returned slice will remain valid until the returned Closer is closed. +// when err == nil, the caller MUST call closer.Close() or a memory leak will occur. +// when err != nil, then the caller io.Closer is nil, and should not be called func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { value, closer, err := b.db.Get(key) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 5690b9c6ead..7c00e4ec4c7 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -121,28 +121,39 @@ func Traverse(prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) return Iterate(prefix, prefix, iterFunc, opt) } -// Exists returns true if a key exists in the database. +// Exists takes a key and a pointer to an a boolean variable `keyExists` as inputs and returns an function. +// When this returned function is executed (and only then), it will write into the `keyExists` whether +// the key exists. // No errors are expected during normal operation. func Exists(key []byte, keyExists *bool) func(storage.Reader) error { return func(r storage.Reader) error { - _, closer, err := r.Get(key) + exists, err := KeyExists(r, key) if err != nil { - // the key does not exist in the database - if errors.Is(err, storage.ErrNotFound) { - *keyExists = false - return nil - } - // exception while checking for the key - return irrecoverable.NewExceptionf("could not load data: %w", err) + return err } - defer closer.Close() - - // the key does exist in the database - *keyExists = true + *keyExists = exists return nil } } +// KeyExists returns true if a key exists in the database. +// No errors are expected during normal operation. +func KeyExists(r storage.Reader, key []byte) (bool, error) { + _, closer, err := r.Get(key) + if err != nil { + // the key does not exist in the database + if errors.Is(err, storage.ErrNotFound) { + return false, nil + } + // exception while checking for the key + return false, irrecoverable.NewExceptionf("could not load data: %w", err) + } + defer closer.Close() + + // the key does exist in the database + return true, nil +} + // Retrieve will retrieve the binary data under the given key from the database // and decode it into the given entity. The provided entity needs to be a // pointer to an initialized entity of the correct type. diff --git a/storage/operations.go b/storage/operations.go index 8c3392765e2..dc6c25c3eb3 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -72,8 +72,9 @@ type Reader interface { // // The caller should not modify the contents of the returned slice, but it is // safe to modify the contents of the `key` argument after Get returns. The - // returned slice will remain valid until the returned Closer is closed. On - // success, the caller MUST call closer.Close() or a memory leak will occur. + // returned slice will remain valid until the returned Closer is closed. + // when err == nil, the caller MUST call closer.Close() or a memory leak will occur. + // when err != nil, then the caller io.Closer is nil, and should not be called Get(key []byte) (value []byte, closer io.Closer, err error) // NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. From 42f360eaf6c164de1fb97e4b61a5ee42918eb516 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 6 Dec 2024 10:37:20 -0800 Subject: [PATCH 16/25] making it optional to call closer when Get fails --- storage/operation/badgerimpl/reader.go | 7 +++---- storage/operation/pebbleimpl/reader.go | 5 ++--- storage/operations.go | 1 - 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index aebe9b854d2..ab1ca7d2c44 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -30,7 +30,6 @@ func (noopCloser) Close() error { return nil } // safe to modify the contents of the argument after Get returns. The // returned slice will remain valid until the returned Closer is closed. // when err == nil, the caller MUST call closer.Close() or a memory leak will occur. -// when err != nil, then the caller io.Closer is nil, and should not be called func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { tx := b.db.NewTransaction(false) defer tx.Discard() @@ -38,14 +37,14 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { item, err := tx.Get(key) if err != nil { if errors.Is(err, badger.ErrKeyNotFound) { - return nil, nil, storage.ErrNotFound + return nil, noopCloser{}, storage.ErrNotFound } - return nil, nil, irrecoverable.NewExceptionf("could not load data: %w", err) + return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load data: %w", err) } value, err := item.ValueCopy(nil) if err != nil { - return nil, nil, irrecoverable.NewExceptionf("could not load value: %w", err) + return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load value: %w", err) } return value, noopCloser{}, nil diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go index b19262fd2ac..56265427f1a 100644 --- a/storage/operation/pebbleimpl/reader.go +++ b/storage/operation/pebbleimpl/reader.go @@ -32,17 +32,16 @@ func (noopCloser) Close() error { return nil } // safe to modify the contents of the argument after Get returns. The // returned slice will remain valid until the returned Closer is closed. // when err == nil, the caller MUST call closer.Close() or a memory leak will occur. -// when err != nil, then the caller io.Closer is nil, and should not be called func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { value, closer, err := b.db.Get(key) if err != nil { if errors.Is(err, pebble.ErrNotFound) { - return nil, nil, storage.ErrNotFound + return nil, noopCloser{}, storage.ErrNotFound } // exception while checking for the key - return nil, nil, irrecoverable.NewExceptionf("could not load data: %w", err) + return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load data: %w", err) } return value, closer, nil diff --git a/storage/operations.go b/storage/operations.go index dc6c25c3eb3..e5e91caa583 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -74,7 +74,6 @@ type Reader interface { // safe to modify the contents of the `key` argument after Get returns. The // returned slice will remain valid until the returned Closer is closed. // when err == nil, the caller MUST call closer.Close() or a memory leak will occur. - // when err != nil, then the caller io.Closer is nil, and should not be called Get(key []byte) (value []byte, closer io.Closer, err error) // NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. From 05862f6d5c7e84510f956f5dc118f00776538efa Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 6 Dec 2024 10:51:35 -0800 Subject: [PATCH 17/25] refactor Retrieve with RetrieveByKey --- storage/operation/reads.go | 31 ++++++++++++++++++------------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 7c00e4ec4c7..10ef7106329 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -154,28 +154,33 @@ func KeyExists(r storage.Reader, key []byte) (bool, error) { return true, nil } -// Retrieve will retrieve the binary data under the given key from the database +// Retrieve returns a functor that retrieves the binary data under the given key from the database +func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { + return func(r storage.Reader) error { + return RetrieveByKey(r, key, entity) + } +} + +// RetrieveByKey will retrieve the binary data under the given key from the database // and decode it into the given entity. The provided entity needs to be a // pointer to an initialized entity of the correct type. // Error returns: // - storage.ErrNotFound if the key does not exist in the database // - generic error in case of unexpected failure from the database layer, or failure // to decode an existing database value -func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { - return func(r storage.Reader) error { - val, closer, err := r.Get(key) - if err != nil { - return err - } +func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) error { + val, closer, err := r.Get(key) + if err != nil { + return err + } - defer closer.Close() + defer closer.Close() - err = msgpack.Unmarshal(val, entity) - if err != nil { - return irrecoverable.NewExceptionf("could not decode entity: %w", err) - } - return nil + err = msgpack.Unmarshal(val, entity) + if err != nil { + return irrecoverable.NewExceptionf("could not decode entity: %w", err) } + return nil } // FindHighestAtOrBelow is for database entries that are indexed by block height. It is suitable to search From e5ae317bd47169e81d3ae0a1a33383e9b35fc56b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 6 Dec 2024 11:41:41 -0800 Subject: [PATCH 18/25] extract functions out of functors --- storage/operation/badgerimpl/writer.go | 2 +- storage/operation/pebbleimpl/writer.go | 4 +- storage/operation/reads.go | 213 ++++++++++++------------- storage/operation/reads_functors.go | 44 +++++ storage/operation/reads_test.go | 6 +- 5 files changed, 148 insertions(+), 121 deletions(-) create mode 100644 storage/operation/reads_functors.go diff --git a/storage/operation/badgerimpl/writer.go b/storage/operation/badgerimpl/writer.go index d0cc9ee389a..78a3ee386a6 100644 --- a/storage/operation/badgerimpl/writer.go +++ b/storage/operation/badgerimpl/writer.go @@ -106,7 +106,7 @@ func (b *ReaderBatchWriter) Delete(key []byte) error { // It returns error if endPrefix < startPrefix // no other errors are expected during normal operation func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPrefix, endPrefix []byte) error { - err := operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { + err := operation.Iterate(startPrefix, endPrefix, func(key []byte) error { err := b.batch.Delete(key) if err != nil { return fmt.Errorf("could not add key to delete batch (%v): %w", key, err) diff --git a/storage/operation/pebbleimpl/writer.go b/storage/operation/pebbleimpl/writer.go index 6746e5effea..c7caa4aed34 100644 --- a/storage/operation/pebbleimpl/writer.go +++ b/storage/operation/pebbleimpl/writer.go @@ -117,11 +117,11 @@ func (b *ReaderBatchWriter) DeleteByRange(globalReader storage.Reader, startPref return b.batch.DeleteRange(start, end, pebble.Sync) } - return operation.IterateKeysInPrefixRange(startPrefix, endPrefix, func(key []byte) error { + return operation.IterateKeysByPrefixRange(globalReader, startPrefix, endPrefix, func(key []byte) error { err := b.batch.Delete(key, pebble.Sync) if err != nil { return fmt.Errorf("could not add key to delete batch (%v): %w", key, err) } return nil - })(globalReader) + }) } diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 10ef7106329..8edf7fe2e0c 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -14,7 +14,10 @@ import ( // CheckFunc is a function that checks if the value should be read and decoded. // return (true, nil) to read the value and pass it to the CreateFunc and HandleFunc for decoding // return (false, nil) to skip reading the value -// return (false, err) if running into any exception, the iteration should be stopped. +// return (false, err) if running into any error, the iteration should be stopped. +// when making a CheckFunc to be used in the IterationFunc to iterate over the keys, a sentinel error +// can be defined and checked to stop the iteration early, such as finding the first key that match +// certain condition. type CheckFunc func(key []byte) (bool, error) // CreateFunc returns a pointer to an initialized entity that we can potentially @@ -28,12 +31,13 @@ type CreateFunc func() interface{} type HandleFunc func() error type IterationFunc func() (CheckFunc, CreateFunc, HandleFunc) -// IterateKeysInPrefixRange will iterate over all entries in the database, where the key starts with a prefixes in +// IterateKey will iterate over all entries in the database, where the key starts with a prefixes in // the range [startPrefix, endPrefix] (both inclusive). We require that startPrefix <= endPrefix (otherwise this // function errors). On every such key, the `check` function is called. If `check` errors, iteration is aborted. +// In other words, error returned by the iteration functions will be propagated to the caller. // No errors expected during normal operations. -func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(key []byte) error) func(storage.Reader) error { - return Iterate(startPrefix, endPrefix, func() (CheckFunc, CreateFunc, HandleFunc) { +func IterateKeysByPrefixRange(r storage.Reader, startPrefix []byte, endPrefix []byte, check func(key []byte) error) error { + return IterateKeys(r, startPrefix, endPrefix, func() (CheckFunc, CreateFunc, HandleFunc) { return func(key []byte) (bool, error) { err := check(key) if err != nil { @@ -44,100 +48,88 @@ func IterateKeysInPrefixRange(startPrefix []byte, endPrefix []byte, check func(k }, storage.IteratorOption{IterateKeyOnly: true}) } -// Iterate will iterate over all keys with prefixes in the given range [startPrefix, endPrefix] (both inclusive) -func Iterate(startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { - return func(r storage.Reader) error { +// IterateKey will iterate over all entries in the database, where the key starts with a prefixes in +// the range [startPrefix, endPrefix] (both inclusive). +// No errors expected during normal operations. +func IterateKeys(r storage.Reader, startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) error { + if len(startPrefix) == 0 { + return fmt.Errorf("startPrefix prefix is empty") + } - if len(startPrefix) == 0 { - return fmt.Errorf("startPrefix prefix is empty") - } + if len(endPrefix) == 0 { + return fmt.Errorf("endPrefix prefix is empty") + } - if len(endPrefix) == 0 { - return fmt.Errorf("endPrefix prefix is empty") - } + // Reverse iteration is not supported by pebble + if bytes.Compare(startPrefix, endPrefix) > 0 { + return fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") + } - // Reverse iteration is not supported by pebble - if bytes.Compare(startPrefix, endPrefix) > 0 { - return fmt.Errorf("startPrefix key must be less than or equal to endPrefix key") - } + it, err := r.NewIter(startPrefix, endPrefix, opt) + if err != nil { + return fmt.Errorf("can not create iterator: %w", err) + } + defer it.Close() - it, err := r.NewIter(startPrefix, endPrefix, opt) - if err != nil { - return fmt.Errorf("can not create iterator: %w", err) - } - defer it.Close() + for it.First(); it.Valid(); it.Next() { + item := it.IterItem() + key := item.Key() + + // initialize processing functions for iteration + check, create, handle := iterFunc() - for it.First(); it.Valid(); it.Next() { - item := it.IterItem() - key := item.Key() + keyCopy := make([]byte, len(key)) - // initialize processing functions for iteration - check, create, handle := iterFunc() + // The underlying database may re-use and modify the backing memory of the returned key. + // Tor safety we proactively make a copy before passing the key to the upper layer. + copy(keyCopy, key) - keyCopy := make([]byte, len(key)) + // check if we should process the item at all + shouldReadValue, err := check(keyCopy) + if err != nil { + return err + } + if !shouldReadValue { // skip reading value + continue + } - // The underlying database may re-use and modify the backing memory of the returned key. - // Tor safety we proactively make a copy before passing the key to the upper layer. - copy(keyCopy, key) + err = item.Value(func(val []byte) error { - // check if we should process the item at all - shouldReadValue, err := check(keyCopy) + // decode into the entity + entity := create() + err = msgpack.Unmarshal(val, entity) if err != nil { - return err + return irrecoverable.NewExceptionf("could not decode entity: %w", err) } - if !shouldReadValue { // skip reading value - continue - } - - err = item.Value(func(val []byte) error { - - // decode into the entity - entity := create() - err = msgpack.Unmarshal(val, entity) - if err != nil { - return irrecoverable.NewExceptionf("could not decode entity: %w", err) - } - - // process the entity - err = handle() - if err != nil { - return fmt.Errorf("could not handle entity: %w", err) - } - - return nil - }) + // process the entity + err = handle() if err != nil { - return fmt.Errorf("could not process value: %w", err) + return fmt.Errorf("could not handle entity: %w", err) } - } - return nil + return nil + }) + + if err != nil { + return fmt.Errorf("could not process value: %w", err) + } } + + return nil } // Traverse will iterate over all keys with the given prefix -func Traverse(prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { - return Iterate(prefix, prefix, iterFunc, opt) +// error returned by the iteration functions will be propagated to the caller. +// No other errors are expected during normal operation. +func TraverseByPrefix(r storage.Reader, prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) error { + return IterateKeys(r, prefix, prefix, iterFunc, opt) } -// Exists takes a key and a pointer to an a boolean variable `keyExists` as inputs and returns an function. +// KeyExists returns true if a key exists in the database. // When this returned function is executed (and only then), it will write into the `keyExists` whether // the key exists. // No errors are expected during normal operation. -func Exists(key []byte, keyExists *bool) func(storage.Reader) error { - return func(r storage.Reader) error { - exists, err := KeyExists(r, key) - if err != nil { - return err - } - *keyExists = exists - return nil - } -} - -// KeyExists returns true if a key exists in the database. -// No errors are expected during normal operation. func KeyExists(r storage.Reader, key []byte) (bool, error) { _, closer, err := r.Get(key) if err != nil { @@ -154,13 +146,6 @@ func KeyExists(r storage.Reader, key []byte) (bool, error) { return true, nil } -// Retrieve returns a functor that retrieves the binary data under the given key from the database -func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { - return func(r storage.Reader) error { - return RetrieveByKey(r, key, entity) - } -} - // RetrieveByKey will retrieve the binary data under the given key from the database // and decode it into the given entity. The provided entity needs to be a // pointer to an initialized entity of the correct type. @@ -183,49 +168,47 @@ func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) error { return nil } -// FindHighestAtOrBelow is for database entries that are indexed by block height. It is suitable to search +// FindHighestAtOrBelowByPrefix is for database entries that are indexed by block height. It is suitable to search // keys with the format prefix` + `height` (where "+" denotes concatenation of binary strings). The height // is encoded as Big-Endian (entries with numerically smaller height have lexicographically smaller key). // The function finds the *highest* key with the given prefix and height equal to or below the given height. -func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func(storage.Reader) error { - return func(r storage.Reader) error { - if len(prefix) == 0 { - return fmt.Errorf("prefix must not be empty") - } - - key := append(prefix, EncodeKeyPart(height)...) - it, err := r.NewIter(prefix, key, storage.DefaultIteratorOptions()) - if err != nil { - return fmt.Errorf("can not create iterator: %w", err) - } - defer it.Close() +func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64, entity interface{}) error { + if len(prefix) == 0 { + return fmt.Errorf("prefix must not be empty") + } - var highestKey []byte + key := append(prefix, EncodeKeyPart(height)...) + it, err := r.NewIter(prefix, key, storage.DefaultIteratorOptions()) + if err != nil { + return fmt.Errorf("can not create iterator: %w", err) + } + defer it.Close() - // find highest value below the given height - for it.First(); it.Valid(); it.Next() { - // copy the key to avoid the underlying slices of the key - // being modified by the Next() call - highestKey = it.IterItem().KeyCopy(highestKey) - } + var highestKey []byte - if len(highestKey) == 0 { - return storage.ErrNotFound - } + // find highest value below the given height + for it.First(); it.Valid(); it.Next() { + // copy the key to avoid the underlying slices of the key + // being modified by the Next() call + highestKey = it.IterItem().KeyCopy(highestKey) + } - // read the value of the highest key - val, closer, err := r.Get(highestKey) - if err != nil { - return err - } + if len(highestKey) == 0 { + return storage.ErrNotFound + } - defer closer.Close() + // read the value of the highest key + val, closer, err := r.Get(highestKey) + if err != nil { + return err + } - err = msgpack.Unmarshal(val, entity) - if err != nil { - return irrecoverable.NewExceptionf("failed to decode value: %w", err) - } + defer closer.Close() - return nil + err = msgpack.Unmarshal(val, entity) + if err != nil { + return irrecoverable.NewExceptionf("failed to decode value: %w", err) } + + return nil } diff --git a/storage/operation/reads_functors.go b/storage/operation/reads_functors.go new file mode 100644 index 00000000000..dace2e9ec02 --- /dev/null +++ b/storage/operation/reads_functors.go @@ -0,0 +1,44 @@ +package operation + +import "github.com/onflow/flow-go/storage" + +// Leo: This package includes deprecated functions that wraps the operation of reading from the database. +// They are needed because the original badger implementation is also implemented in the same wrapped function manner, +// since badger requires reads to be done in a transaction, which is stateful. +// Using these deprecated functions could minimize the changes during refactor and easier to review the changes. +// The simplified implementation of the functions are in the reads.go file, which are encouraged to be used instead. + +func Iterate(startPrefix []byte, endPrefix []byte, check func(key []byte) error) func(storage.Reader) error { + return func(r storage.Reader) error { + return IterateKeysByPrefixRange(r, startPrefix, endPrefix, check) + } +} + +func Traverse(prefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) func(storage.Reader) error { + return func(r storage.Reader) error { + return TraverseByPrefix(r, prefix, iterFunc, opt) + } +} + +func Retrieve(key []byte, entity interface{}) func(storage.Reader) error { + return func(r storage.Reader) error { + return RetrieveByKey(r, key, entity) + } +} + +func Exists(key []byte, keyExists *bool) func(storage.Reader) error { + return func(r storage.Reader) error { + exists, err := KeyExists(r, key) + if err != nil { + return err + } + *keyExists = exists + return nil + } +} + +func FindHighestAtOrBelow(prefix []byte, height uint64, entity interface{}) func(storage.Reader) error { + return func(r storage.Reader) error { + return FindHighestAtOrBelowByPrefix(r, prefix, height, entity) + } +} diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index 74c59b10a4a..efb5379dfc2 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -53,7 +53,7 @@ func TestIterateKeysInPrefixRange(t *testing.T) { // Forward iteration and check boundaries var found [][]byte - require.NoError(t, operation.IterateKeysInPrefixRange(prefixStart, prefixEnd, func(key []byte) error { + require.NoError(t, operation.Iterate(prefixStart, prefixEnd, func(key []byte) error { found = append(found, key) return nil })(r), "should iterate forward without error") @@ -66,7 +66,7 @@ func TestIterateInvalidRange(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { var found [][]byte - require.Error(t, operation.IterateKeysInPrefixRange([]byte{0x02}, []byte{0x01}, func(key []byte) error { + require.Error(t, operation.Iterate([]byte{0x02}, []byte{0x01}, func(key []byte) error { found = append(found, key) return nil })(r)) @@ -118,7 +118,7 @@ func TestIterationBoundary(t *testing.T) { // Forward iteration and check boundaries var found [][]byte - require.NoError(t, operation.IterateKeysInPrefixRange(prefixStart, prefixEnd, func(key []byte) error { + require.NoError(t, operation.Iterate(prefixStart, prefixEnd, func(key []byte) error { found = append(found, key) return nil })(r), "should iterate forward without error") From 88d86d291477de801b259fcd843a0878891eabf5 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:06:47 -0800 Subject: [PATCH 19/25] address review comments --- storage/operation/badgerimpl/iterator.go | 2 +- storage/operations.go | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index 7d4eb8dea6c..9acbc2054a9 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -19,7 +19,7 @@ var _ storage.Iterator = (*badgerIterator)(nil) func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage.IteratorOption) *badgerIterator { options := badger.DefaultIteratorOptions - if ops.IterateKeyOnly { + if ops.BadgerIterateKeyOnly { options.PrefetchValues = false } diff --git a/storage/operations.go b/storage/operations.go index e5e91caa583..76c61ffa267 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -56,12 +56,13 @@ type IterItem interface { } type IteratorOption struct { - IterateKeyOnly bool // default false + BadgerIterateKeyOnly bool // default false } func DefaultIteratorOptions() IteratorOption { return IteratorOption{ - IterateKeyOnly: false, // only needed for badger. ignored by pebble + // only needed for badger. ignored by pebble + BadgerIterateKeyOnly: false, } } From ef995d4c6cd46a5f6b46271b153a72b84c3da5a5 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:16:06 -0800 Subject: [PATCH 20/25] address review comments --- storage/operation/badgerimpl/iterator.go | 3 +- storage/operation/pebbleimpl/iterator.go | 40 ++++++------------------ storage/operation/reads.go | 2 +- storage/operation/reads_test.go | 37 ++++++++++++++++++++++ storage/operations.go | 4 ++- 5 files changed, 53 insertions(+), 33 deletions(-) diff --git a/storage/operation/badgerimpl/iterator.go b/storage/operation/badgerimpl/iterator.go index 9acbc2054a9..edd5dab2fd6 100644 --- a/storage/operation/badgerimpl/iterator.go +++ b/storage/operation/badgerimpl/iterator.go @@ -37,8 +37,9 @@ func newBadgerIterator(db *badger.DB, startPrefix, endPrefix []byte, ops storage } // First seeks to the smallest key greater than or equal to the given key. -func (i *badgerIterator) First() { +func (i *badgerIterator) First() bool { i.iter.Seek(i.lowerBound) + return i.Valid() } // Valid returns whether the iterator is positioned at a valid key-value pair. diff --git a/storage/operation/pebbleimpl/iterator.go b/storage/operation/pebbleimpl/iterator.go index 79d39068e21..49792d160c3 100644 --- a/storage/operation/pebbleimpl/iterator.go +++ b/storage/operation/pebbleimpl/iterator.go @@ -9,7 +9,7 @@ import ( ) type pebbleIterator struct { - iter *pebble.Iterator + *pebble.Iterator } var _ storage.Iterator = (*pebbleIterator)(nil) @@ -33,56 +33,36 @@ func newPebbleIterator(reader pebble.Reader, startPrefix, endPrefix []byte, ops } return &pebbleIterator{ - iter: iter, + iter, }, nil } -// First seeks to the smallest key greater than or equal to the given key. -func (i *pebbleIterator) First() { - i.iter.First() -} - -// Valid returns whether the iterator is positioned at a valid key-value pair. -func (i *pebbleIterator) Valid() bool { - return i.iter.Valid() +// IterItem returns the current key-value pair, or nil if done. +func (i *pebbleIterator) IterItem() storage.IterItem { + return pebbleIterItem{i.Iterator} } -// Next advances the iterator to the next key-value pair. +// Next seeks to the smallest key greater than or equal to the given key. func (i *pebbleIterator) Next() { - i.iter.Next() -} - -// IterItem returns the current key-value pair, or nil if done. -func (i *pebbleIterator) IterItem() storage.IterItem { - return pebbleIterItem{iter: i.iter} + i.Iterator.Next() } type pebbleIterItem struct { - iter *pebble.Iterator + *pebble.Iterator } var _ storage.IterItem = (*pebbleIterItem)(nil) -func (i pebbleIterItem) Key() []byte { - return i.iter.Key() -} - // KeyCopy returns a copy of the key of the item, writing it to dst slice. func (i pebbleIterItem) KeyCopy(dst []byte) []byte { - return append(dst[:0], i.iter.Key()...) + return append(dst[:0], i.Key()...) } func (i pebbleIterItem) Value(fn func([]byte) error) error { - val, err := i.iter.ValueAndErr() + val, err := i.ValueAndErr() if err != nil { return err } return fn(val) } - -// Close closes the iterator. Iterator must be closed, otherwise it causes memory leak. -// No errors expected during normal operation -func (i *pebbleIterator) Close() error { - return i.iter.Close() -} diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 8edf7fe2e0c..a6af1f2f11f 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -45,7 +45,7 @@ func IterateKeysByPrefixRange(r storage.Reader, startPrefix []byte, endPrefix [] } return false, nil }, nil, nil - }, storage.IteratorOption{IterateKeyOnly: true}) + }, storage.IteratorOption{BadgerIterateKeyOnly: true}) } // IterateKey will iterate over all entries in the database, where the key starts with a prefixes in diff --git a/storage/operation/reads_test.go b/storage/operation/reads_test.go index efb5379dfc2..0ba19e12023 100644 --- a/storage/operation/reads_test.go +++ b/storage/operation/reads_test.go @@ -12,6 +12,43 @@ import ( "github.com/onflow/flow-go/storage/operation/dbtest" ) +func TestFirst(t *testing.T) { + dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { + + // Create a range of keys around the prefix start/end values + keys := [][]byte{ + {0x10, 0x00}, + {0x10, 0xff}, + } + + // Insert the keys into the storage + require.NoError(t, withWriter(func(writer storage.Writer) error { + for _, key := range keys { + value := []byte{0x00} // value are skipped, doesn't matter + err := operation.Upsert(key, value)(writer) + if err != nil { + return err + } + } + return nil + })) + + iter, err := r.NewIter([]byte{0x20}, []byte{0x30}, storage.DefaultIteratorOptions()) + require.NoError(t, err) + + // Check that the iterator is at the first key and return false when matching no key + require.False(t, iter.First()) + require.NoError(t, iter.Close()) + + iter, err = r.NewIter([]byte{0x10}, []byte{0x30}, storage.DefaultIteratorOptions()) + require.NoError(t, err) + + // Check that the iterator is at the first key and return true when matching the first key + require.True(t, iter.First()) + require.NoError(t, iter.Close()) + }) +} + func TestIterateKeysInPrefixRange(t *testing.T) { dbtest.RunWithStorages(t, func(t *testing.T, r storage.Reader, withWriter dbtest.WithWriter) { // Define the prefix range diff --git a/storage/operations.go b/storage/operations.go index 76c61ffa267..1caea90b823 100644 --- a/storage/operations.go +++ b/storage/operations.go @@ -17,7 +17,9 @@ type Iterator interface { // This method must be called because it's necessary for the badger implementation // to move the iteration cursor to the first key in the iteration range. // This method must be called before calling Valid, Next, IterItem, or Close. - First() + // return true if the iterator is pointing to a valid key-value pair after calling First, + // return false otherwise. + First() bool // Valid returns whether the iterator is positioned at a valid key-value pair. // If Valid returns false, the iterator is done and must be closed. From ffe023c030c944ea080d185baa2f1a26d3179be8 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:36:29 -0800 Subject: [PATCH 21/25] use RWMutex --- storage/operation/callbacks.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storage/operation/callbacks.go b/storage/operation/callbacks.go index 40d414ded91..d27dd2f5337 100644 --- a/storage/operation/callbacks.go +++ b/storage/operation/callbacks.go @@ -3,8 +3,8 @@ package operation import "sync" type Callbacks struct { - sync.Mutex // protect callbacks - callbacks []func(error) + sync.RWMutex // protect callbacks + callbacks []func(error) } func (b *Callbacks) AddCallback(callback func(error)) { @@ -15,8 +15,8 @@ func (b *Callbacks) AddCallback(callback func(error)) { } func (b *Callbacks) NotifyCallbacks(err error) { - b.Lock() - defer b.Unlock() + b.RLock() + defer b.RUnlock() for _, callback := range b.callbacks { callback(err) From fdce57483b88420d127a2982b7e4ca8984058cf8 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:42:54 -0800 Subject: [PATCH 22/25] add noop closer --- storage/operation/badgerimpl/reader.go | 15 +++++---------- storage/operation/pebbleimpl/reader.go | 11 +++-------- utils/noop/closer.go | 9 +++++++++ 3 files changed, 17 insertions(+), 18 deletions(-) create mode 100644 utils/noop/closer.go diff --git a/storage/operation/badgerimpl/reader.go b/storage/operation/badgerimpl/reader.go index ab1ca7d2c44..f58c840cd7a 100644 --- a/storage/operation/badgerimpl/reader.go +++ b/storage/operation/badgerimpl/reader.go @@ -10,18 +10,13 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/noop" ) type dbReader struct { db *badger.DB } -type noopCloser struct{} - -var _ io.Closer = (*noopCloser)(nil) - -func (noopCloser) Close() error { return nil } - // Get gets the value for the given key. It returns ErrNotFound if the DB // does not contain the key. // other errors are exceptions @@ -37,17 +32,17 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { item, err := tx.Get(key) if err != nil { if errors.Is(err, badger.ErrKeyNotFound) { - return nil, noopCloser{}, storage.ErrNotFound + return nil, noop.Closer{}, storage.ErrNotFound } - return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load data: %w", err) + return nil, noop.Closer{}, irrecoverable.NewExceptionf("could not load data: %w", err) } value, err := item.ValueCopy(nil) if err != nil { - return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load value: %w", err) + return nil, noop.Closer{}, irrecoverable.NewExceptionf("could not load value: %w", err) } - return value, noopCloser{}, nil + return value, noop.Closer{}, nil } // NewIter returns a new Iterator for the given key prefix range [startPrefix, endPrefix], both inclusive. diff --git a/storage/operation/pebbleimpl/reader.go b/storage/operation/pebbleimpl/reader.go index 56265427f1a..4f0494fb681 100644 --- a/storage/operation/pebbleimpl/reader.go +++ b/storage/operation/pebbleimpl/reader.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/noop" ) type dbReader struct { @@ -18,12 +19,6 @@ type dbReader struct { var _ storage.Reader = (*dbReader)(nil) -type noopCloser struct{} - -var _ io.Closer = (*noopCloser)(nil) - -func (noopCloser) Close() error { return nil } - // Get gets the value for the given key. It returns ErrNotFound if the DB // does not contain the key. // other errors are exceptions @@ -37,11 +32,11 @@ func (b dbReader) Get(key []byte) ([]byte, io.Closer, error) { if err != nil { if errors.Is(err, pebble.ErrNotFound) { - return nil, noopCloser{}, storage.ErrNotFound + return nil, noop.Closer{}, storage.ErrNotFound } // exception while checking for the key - return nil, noopCloser{}, irrecoverable.NewExceptionf("could not load data: %w", err) + return nil, noop.Closer{}, irrecoverable.NewExceptionf("could not load data: %w", err) } return value, closer, nil diff --git a/utils/noop/closer.go b/utils/noop/closer.go new file mode 100644 index 00000000000..994038362de --- /dev/null +++ b/utils/noop/closer.go @@ -0,0 +1,9 @@ +package noop + +import "io" + +type Closer struct{} + +var _ io.Closer = (*Closer)(nil) + +func (Closer) Close() error { return nil } From eebde069ca8e989c11389666278c0601abf4e011 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:49:17 -0800 Subject: [PATCH 23/25] handle close error --- storage/operation/reads.go | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index a6af1f2f11f..3cedeba44cf 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -130,7 +130,7 @@ func TraverseByPrefix(r storage.Reader, prefix []byte, iterFunc IterationFunc, o // When this returned function is executed (and only then), it will write into the `keyExists` whether // the key exists. // No errors are expected during normal operation. -func KeyExists(r storage.Reader, key []byte) (bool, error) { +func KeyExists(r storage.Reader, key []byte) (exist bool, errExit error) { _, closer, err := r.Get(key) if err != nil { // the key does not exist in the database @@ -140,7 +140,9 @@ func KeyExists(r storage.Reader, key []byte) (bool, error) { // exception while checking for the key return false, irrecoverable.NewExceptionf("could not load data: %w", err) } - defer closer.Close() + defer func() { + errExit = closer.Close() + }() // the key does exist in the database return true, nil @@ -153,13 +155,15 @@ func KeyExists(r storage.Reader, key []byte) (bool, error) { // - storage.ErrNotFound if the key does not exist in the database // - generic error in case of unexpected failure from the database layer, or failure // to decode an existing database value -func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) error { +func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) (errExit error) { val, closer, err := r.Get(key) if err != nil { return err } - defer closer.Close() + defer func() { + errExit = closer.Close() + }() err = msgpack.Unmarshal(val, entity) if err != nil { @@ -172,7 +176,7 @@ func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) error { // keys with the format prefix` + `height` (where "+" denotes concatenation of binary strings). The height // is encoded as Big-Endian (entries with numerically smaller height have lexicographically smaller key). // The function finds the *highest* key with the given prefix and height equal to or below the given height. -func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64, entity interface{}) error { +func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64, entity interface{}) (errExit error) { if len(prefix) == 0 { return fmt.Errorf("prefix must not be empty") } @@ -182,7 +186,9 @@ func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64 if err != nil { return fmt.Errorf("can not create iterator: %w", err) } - defer it.Close() + defer func() { + errExit = it.Close() + }() var highestKey []byte @@ -203,7 +209,9 @@ func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64 return err } - defer closer.Close() + defer func() { + errExit = closer.Close() + }() err = msgpack.Unmarshal(val, entity) if err != nil { From 9f49267ec6ed4011eb7860df84eecec1291530f8 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:49:54 -0800 Subject: [PATCH 24/25] update comments --- storage/operation/reads.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 3cedeba44cf..1df0bdc85b4 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -31,7 +31,7 @@ type CreateFunc func() interface{} type HandleFunc func() error type IterationFunc func() (CheckFunc, CreateFunc, HandleFunc) -// IterateKey will iterate over all entries in the database, where the key starts with a prefixes in +// IterateKeysByPrefixRange will iterate over all entries in the database, where the key starts with a prefixes in // the range [startPrefix, endPrefix] (both inclusive). We require that startPrefix <= endPrefix (otherwise this // function errors). On every such key, the `check` function is called. If `check` errors, iteration is aborted. // In other words, error returned by the iteration functions will be propagated to the caller. @@ -48,7 +48,7 @@ func IterateKeysByPrefixRange(r storage.Reader, startPrefix []byte, endPrefix [] }, storage.IteratorOption{BadgerIterateKeyOnly: true}) } -// IterateKey will iterate over all entries in the database, where the key starts with a prefixes in +// IterateKeys will iterate over all entries in the database, where the key starts with a prefixes in // the range [startPrefix, endPrefix] (both inclusive). // No errors expected during normal operations. func IterateKeys(r storage.Reader, startPrefix []byte, endPrefix []byte, iterFunc IterationFunc, opt storage.IteratorOption) error { From 24d65bcc5dfe57007a5ed22abef181ba834652c4 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 12 Dec 2024 09:32:06 -0800 Subject: [PATCH 25/25] revert closer changes --- storage/operation/reads.go | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/storage/operation/reads.go b/storage/operation/reads.go index 1df0bdc85b4..6c14102bbc3 100644 --- a/storage/operation/reads.go +++ b/storage/operation/reads.go @@ -130,7 +130,7 @@ func TraverseByPrefix(r storage.Reader, prefix []byte, iterFunc IterationFunc, o // When this returned function is executed (and only then), it will write into the `keyExists` whether // the key exists. // No errors are expected during normal operation. -func KeyExists(r storage.Reader, key []byte) (exist bool, errExit error) { +func KeyExists(r storage.Reader, key []byte) (bool, error) { _, closer, err := r.Get(key) if err != nil { // the key does not exist in the database @@ -140,9 +140,7 @@ func KeyExists(r storage.Reader, key []byte) (exist bool, errExit error) { // exception while checking for the key return false, irrecoverable.NewExceptionf("could not load data: %w", err) } - defer func() { - errExit = closer.Close() - }() + defer closer.Close() // the key does exist in the database return true, nil @@ -155,15 +153,13 @@ func KeyExists(r storage.Reader, key []byte) (exist bool, errExit error) { // - storage.ErrNotFound if the key does not exist in the database // - generic error in case of unexpected failure from the database layer, or failure // to decode an existing database value -func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) (errExit error) { +func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) error { val, closer, err := r.Get(key) if err != nil { return err } - defer func() { - errExit = closer.Close() - }() + defer closer.Close() err = msgpack.Unmarshal(val, entity) if err != nil { @@ -176,7 +172,7 @@ func RetrieveByKey(r storage.Reader, key []byte, entity interface{}) (errExit er // keys with the format prefix` + `height` (where "+" denotes concatenation of binary strings). The height // is encoded as Big-Endian (entries with numerically smaller height have lexicographically smaller key). // The function finds the *highest* key with the given prefix and height equal to or below the given height. -func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64, entity interface{}) (errExit error) { +func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64, entity interface{}) error { if len(prefix) == 0 { return fmt.Errorf("prefix must not be empty") } @@ -186,9 +182,7 @@ func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64 if err != nil { return fmt.Errorf("can not create iterator: %w", err) } - defer func() { - errExit = it.Close() - }() + defer it.Close() var highestKey []byte @@ -209,9 +203,7 @@ func FindHighestAtOrBelowByPrefix(r storage.Reader, prefix []byte, height uint64 return err } - defer func() { - errExit = closer.Close() - }() + defer closer.Close() err = msgpack.Unmarshal(val, entity) if err != nil {