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/76] 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/76] 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/76] 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/76] 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 2ba50ede0983bbae10b0913a7f9883bd279892d7 Mon Sep 17 00:00:00 2001 From: sjonpaulbrown Date: Wed, 20 Nov 2024 17:04:13 -0700 Subject: [PATCH 05/76] Update workflow to support on-demand private builds --- .github/workflows/builds.yml | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/.github/workflows/builds.yml b/.github/workflows/builds.yml index cad6b7ad3c6..a9818c975e8 100644 --- a/.github/workflows/builds.yml +++ b/.github/workflows/builds.yml @@ -43,6 +43,10 @@ on: type: boolean description: 'Build amd64 `without_adx` and `without_netgo_without_adx` images, and arm64 images' required: false + private_build: + type: boolean + description: 'Build private images' + required: false jobs: # matrix_builder generates a matrix that includes the roles selected in the input @@ -109,16 +113,21 @@ jobs: credentials_json: ${{ secrets.GCR_SERVICE_KEY_SECRET }} - name: Set up Google Cloud SDK uses: google-github-actions/setup-gcloud@v1 - - name: Authenticate docker with gcloud + + - name: Authenticate Docker with gcloud run: | - gcloud auth configure-docker + if [[ "${{ github.event.inputs.private_build }}" == "true" ]]; then + gcloud auth configure-docker us-central1-docker.pkg.dev + else + gcloud auth configure-docker + fi - name: Build/Push ${{ matrix.role }} amd64 images with adx (default) env: IMAGE_TAG: ${{ inputs.docker_tag }} CADENCE_DEPLOY_KEY: ${{ secrets.CADENCE_DEPLOY_KEY }} run: | - make docker-build-${{ matrix.role }}-with-adx docker-push-${{ matrix.role }}-with-adx + make docker-build-${{ matrix.role }}-with-adx docker-push-${{ matrix.role }}-with-adx CONTAINER_REGISTRY=${{ github.event.inputs.private_build == "true" && vars.PRIVATE_REGISTRY || vars.PUBLIC_REGISTRY }} - name: Build/Push ${{ matrix.role }} amd64 images without netgo and without adx, arm64 images if: ${{ inputs.include_alternative_builds }} @@ -128,7 +137,7 @@ jobs: run: | make docker-build-${{ matrix.role }}-without-adx docker-push-${{ matrix.role }}-without-adx \ docker-build-${{ matrix.role }}-without-netgo-without-adx docker-push-${{ matrix.role }}-without-netgo-without-adx \ - docker-cross-build-${{ matrix.role }}-arm docker-push-${{ matrix.role }}-arm + docker-cross-build-${{ matrix.role }}-arm docker-push-${{ matrix.role }}-arm CONTAINER_REGISTRY=${{ github.event.inputs.private_build == "true" && vars.PRIVATE_REGISTRY || vars.PUBLIC_REGISTRY }} From 9825c34d4467801ae3567807af9ae96ce1671971 Mon Sep 17 00:00:00 2001 From: sjonpaulbrown Date: Wed, 20 Nov 2024 17:15:21 -0700 Subject: [PATCH 06/76] Set registry to be used --- .github/workflows/builds.yml | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/.github/workflows/builds.yml b/.github/workflows/builds.yml index a9818c975e8..b3e1670af33 100644 --- a/.github/workflows/builds.yml +++ b/.github/workflows/builds.yml @@ -122,12 +122,21 @@ jobs: gcloud auth configure-docker fi + - name: Set CONTAINER_REGISTRY + id: set-registry + run: | + if [[ "${{ github.event.inputs.private_build }}" == "true" ]]; then + echo "CONTAINER_REGISTRY=${{ vars.PRIVATE_REGISTRY }}" >> $GITHUB_ENV + else + echo "CONTAINER_REGISTRY=${{ vars.PUBLIC_REGISTRY }}" >> $GITHUB_ENV + fi + - name: Build/Push ${{ matrix.role }} amd64 images with adx (default) env: IMAGE_TAG: ${{ inputs.docker_tag }} CADENCE_DEPLOY_KEY: ${{ secrets.CADENCE_DEPLOY_KEY }} run: | - make docker-build-${{ matrix.role }}-with-adx docker-push-${{ matrix.role }}-with-adx CONTAINER_REGISTRY=${{ github.event.inputs.private_build == "true" && vars.PRIVATE_REGISTRY || vars.PUBLIC_REGISTRY }} + make docker-build-${{ matrix.role }}-with-adx docker-push-${{ matrix.role }}-with-adx CONTAINER_REGISTRY=$CONTAINER_REGISTRY - name: Build/Push ${{ matrix.role }} amd64 images without netgo and without adx, arm64 images if: ${{ inputs.include_alternative_builds }} @@ -137,7 +146,7 @@ jobs: run: | make docker-build-${{ matrix.role }}-without-adx docker-push-${{ matrix.role }}-without-adx \ docker-build-${{ matrix.role }}-without-netgo-without-adx docker-push-${{ matrix.role }}-without-netgo-without-adx \ - docker-cross-build-${{ matrix.role }}-arm docker-push-${{ matrix.role }}-arm CONTAINER_REGISTRY=${{ github.event.inputs.private_build == "true" && vars.PRIVATE_REGISTRY || vars.PUBLIC_REGISTRY }} + docker-cross-build-${{ matrix.role }}-arm docker-push-${{ matrix.role }}-arm CONTAINER_REGISTRY=$CONTAINER_REGISTRY From 49906474c1040ff84b630ce6cfcc7a4632219453 Mon Sep 17 00:00:00 2001 From: Jordan Schalm Date: Fri, 22 Nov 2024 14:19:05 -0800 Subject: [PATCH 07/76] improve dkg engine package docs - put the godoc directly above package declaration so it appears as godoc - remove tabs to correctly align diagram --- engine/consensus/dkg/doc.go | 101 +++++++++++++++++------------------- 1 file changed, 48 insertions(+), 53 deletions(-) diff --git a/engine/consensus/dkg/doc.go b/engine/consensus/dkg/doc.go index 2c72beabc5a..af9393534cf 100644 --- a/engine/consensus/dkg/doc.go +++ b/engine/consensus/dkg/doc.go @@ -1,54 +1,49 @@ -/* - -Package dkg implements engines for the DKG protocol. - -ReactorEngine - -ReactorEngine implements triggers to control the lifecycle of DKG runs. A new -DKG protocol is started when an EpochSetup event is sealed and finalized. The -subsequent phase transitions are triggered when specified views are encountered -(specifically when the first block of a given view is finalized). In between -phase transitions the engine regularly queries the DKG smart-contract to read -broadcast messages. - -MessagingEngine - -MessagingEngine is a network engine that enables consensus nodes to securely -exchange private DKG messages. Note that broadcast messages are not exchanged -through this engine, but rather via the DKG smart-contract. - -Architecture - -For every new epoch, the ReactorEngine instantiates a new DKGController with a -new Broker using the provided ControllerFactory. The ControllerFactory ties new -DKGControllers to the MessagingEngine via a BrokerTunnel which exposes channels -to relay incoming and outgoing messages (cf. module/dkg). - - EpochSetup/OnView - | - v - +---------------+ - | ReactorEngine | - +---------------+ - | - v -*~~~~~~~~~~~~~~~~~~~~~* (one/epoch) -| +---------------+ | -| | Controller | | -| +---------------+ | -| | | -| v | -| +---------------+ | -| | Broker | | -| +---------------+ | -*~~~~~~~~|~~~~~~~~~\~~* - tunnel smart-contract client - | \ - +--------------+ +------------------+ - | Messaging | | DKGSmartContract | - | Engine | | | - +--------------+ +------------------+ - -*/ - +// Package dkg implements engines for the DKG protocol. +// +// # Reactor Engine +// +// The [ReactorEngine] implements triggers to control the lifecycle of DKG runs. A new +// DKG protocol is started when an EpochSetup event is sealed and finalized. The +// subsequent phase transitions are triggered when specified views are encountered +// (specifically when the first block of a given view is finalized). In between +// phase transitions the engine regularly queries the DKG smart-contract to read +// broadcast messages. +// +// # Messaging Engine +// +// The [MessagingEngine] is a network engine that enables consensus nodes to securely exchange +// private (not broadcast) DKG messages. Broadcast messages are sent via the DKG smart contract. +// +// # Architecture +// +// For every new epoch, the [ReactorEngine] instantiates a new [module.DKGController] with a +// new [module.DKGBroker] using the provided ControllerFactory. The ControllerFactory ties new +// DKGControllers to the [MessagingEngine] via a BrokerTunnel which exposes channels +// to relay incoming and outgoing messages (see package module/dkg). +// +// EpochSetup/EpochCommit/OnView +// | +// v +// +---------------+ +// | ReactorEngine | +// +---------------+ +// | +// v +// *~~~~~~~~~~~~~~~~~~~~~* <- Epoch-scoped +// | +---------------+ | +// | | Controller | | +// | +---------------+ | +// | | | +// | v | +// | +---------------+ | +// | | Broker | | +// | +---------------+ | +// *~~~~~~~~|~~~~~~~~~\~~* +// | \ +// BrokerTunnel DKGContractClient +// | \ +// +--------------+ +------------------+ +// | Messaging | | FlowDKG smart | +// | Engine | | contract | +// +--------------+ +------------------+ package dkg From 12d132010bb82d92cbfe3d9be3cbadf42317eb90 Mon Sep 17 00:00:00 2001 From: Jordan Schalm Date: Mon, 25 Nov 2024 09:05:28 -0800 Subject: [PATCH 08/76] dkg package documentation tweaks --- engine/consensus/dkg/doc.go | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/engine/consensus/dkg/doc.go b/engine/consensus/dkg/doc.go index af9393534cf..2feff12aa42 100644 --- a/engine/consensus/dkg/doc.go +++ b/engine/consensus/dkg/doc.go @@ -2,12 +2,13 @@ // // # Reactor Engine // -// The [ReactorEngine] implements triggers to control the lifecycle of DKG runs. A new -// DKG protocol is started when an EpochSetup event is sealed and finalized. The -// subsequent phase transitions are triggered when specified views are encountered -// (specifically when the first block of a given view is finalized). In between -// phase transitions the engine regularly queries the DKG smart-contract to read -// broadcast messages. +// The [ReactorEngine] implements triggers to control the lifecycle of DKG instances. +// A new DKG instance is started when an EpochSetup service event is sealed. +// The subsequent phase transitions are triggered when specified views are encountered. +// Specifically, phase transitions for a view V are triggered when the first block with view >=V is finalized. +// Between phase transitions, we periodically query the DKG smart-contract ("whiteboard") to read broadcast messages. +// Before transitioning the state machine to the next phase, we query the whiteboard w.r.t. the final view +// of the phase - this ensures all participants eventually observe the same set of messages for each phase. // // # Messaging Engine // @@ -16,12 +17,12 @@ // // # Architecture // -// For every new epoch, the [ReactorEngine] instantiates a new [module.DKGController] with a -// new [module.DKGBroker] using the provided ControllerFactory. The ControllerFactory ties new -// DKGControllers to the [MessagingEngine] via a BrokerTunnel which exposes channels -// to relay incoming and outgoing messages (see package module/dkg). +// In the happy path, one DKG instance runs every epoch. For each DKG instance, the [ReactorEngine] +// instantiates a new, epoch-scoped [module.DKGController] and [module.DKGBroker] using the provided ControllerFactory. +// The ControllerFactory ties new DKGControllers to the [MessagingEngine] via a BrokerTunnel, +// which exposes channels to relay incoming and outgoing messages (see package module/dkg for details). // -// EpochSetup/EpochCommit/OnView +// EpochSetup/EpochCommit/OnView events // | // v // +---------------+ @@ -29,7 +30,7 @@ // +---------------+ // | // v -// *~~~~~~~~~~~~~~~~~~~~~* <- Epoch-scoped +// *~~~~~~~~~~~~~~~~~~~~~* <- Epoch-scoped components // | +---------------+ | // | | Controller | | // | +---------------+ | From e7ca1181503ffed22f62a8589d3992e70e28e144 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 21 Nov 2024 11:42:55 -0800 Subject: [PATCH 09/76] add verify evm offchain replay util cmd --- cmd/util/cmd/root.go | 2 + .../cmd/verify-evm-offchain-replay/main.go | 87 +++++++++ .../cmd/verify-evm-offchain-replay/verify.go | 93 ++++++++++ fvm/evm/offchain/utils/collection_test.go | 48 ++--- fvm/evm/offchain/utils/verify.go | 168 ++++++++++++++++++ 5 files changed, 374 insertions(+), 24 deletions(-) create mode 100644 cmd/util/cmd/verify-evm-offchain-replay/main.go create mode 100644 cmd/util/cmd/verify-evm-offchain-replay/verify.go create mode 100644 fvm/evm/offchain/utils/verify.go diff --git a/cmd/util/cmd/root.go b/cmd/util/cmd/root.go index cefd8db691d..146fb2b5af8 100644 --- a/cmd/util/cmd/root.go +++ b/cmd/util/cmd/root.go @@ -41,6 +41,7 @@ import ( "github.com/onflow/flow-go/cmd/util/cmd/snapshot" system_addresses "github.com/onflow/flow-go/cmd/util/cmd/system-addresses" truncate_database "github.com/onflow/flow-go/cmd/util/cmd/truncate-database" + verify_evm_offchain_replay "github.com/onflow/flow-go/cmd/util/cmd/verify-evm-offchain-replay" "github.com/onflow/flow-go/cmd/util/cmd/version" "github.com/onflow/flow-go/module/profiler" ) @@ -126,6 +127,7 @@ func addCommands() { rootCmd.AddCommand(debug_script.Cmd) rootCmd.AddCommand(generate_authorization_fixes.Cmd) rootCmd.AddCommand(evm_state_exporter.Cmd) + rootCmd.AddCommand(verify_evm_offchain_replay.Cmd) } func initConfig() { diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go new file mode 100644 index 00000000000..9f56587306e --- /dev/null +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -0,0 +1,87 @@ +package verify + +import ( + "fmt" + "strconv" + "strings" + + "github.com/rs/zerolog/log" + "github.com/spf13/cobra" + + "github.com/onflow/flow-go/model/flow" +) + +var ( + flagDatadir string + flagExecutionDataDir string + flagEVMStateGobDir string + flagChain string + flagFromTo string +) + +// usage example +// +// ./util verify-evm-offchain-replay --chain flow-testnet --from-to 211176671-211177000 +// --datadir /var/flow/data/protocol --execution_data_dir /var/flow/data/execution_data +var Cmd = &cobra.Command{ + Use: "verify-evm-offchain-replay", + Short: "verify evm offchain replay with execution data", + Run: run, +} + +func init() { + Cmd.Flags().StringVar(&flagChain, "chain", "", "Chain name") + _ = Cmd.MarkFlagRequired("chain") + + Cmd.Flags().StringVar(&flagDatadir, "datadir", "/var/flow/data/protocol", + "directory that stores the protocol state") + + Cmd.Flags().StringVar(&flagExecutionDataDir, "execution_data_dir", "/var/flow/data/execution_data", + "directory that stores the execution state") + + Cmd.Flags().StringVar(&flagFromTo, "from_to", "", + "the flow height range to verify blocks, i.e, 1-1000, 1000-2000, 2000-3000, etc.") + + Cmd.Flags().StringVar(&flagEVMStateGobDir, "evm_state_gob_dir", "/var/flow/data/evm_state_gob", + "directory that stores the evm state gob files as checkpoint") +} + +func run(*cobra.Command, []string) { + _ = flow.ChainID(flagChain).Chain() + + from, to, err := parseFromTo(flagFromTo) + if err != nil { + log.Fatal().Err(err).Msg("could not parse from_to") + } + + log.Info().Msgf("verifying range from %d to %d", from, to) + err = Verify(from, to, flow.Testnet, flagDatadir, flagExecutionDataDir) + if err != nil { + log.Fatal().Err(err).Msg("could not verify last k height") + } + log.Info().Msgf("successfully verified range from %d to %d", from, to) + +} + +func parseFromTo(fromTo string) (from, to uint64, err error) { + parts := strings.Split(fromTo, "-") + if len(parts) != 2 { + return 0, 0, fmt.Errorf("invalid format: expected 'from-to', got '%s'", fromTo) + } + + from, err = strconv.ParseUint(strings.TrimSpace(parts[0]), 10, 64) + if err != nil { + return 0, 0, fmt.Errorf("invalid 'from' value: %w", err) + } + + to, err = strconv.ParseUint(strings.TrimSpace(parts[1]), 10, 64) + if err != nil { + return 0, 0, fmt.Errorf("invalid 'to' value: %w", err) + } + + if from > to { + return 0, 0, fmt.Errorf("'from' value (%d) must be less than or equal to 'to' value (%d)", from, to) + } + + return from, to, nil +} diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go new file mode 100644 index 00000000000..1a907be669a --- /dev/null +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -0,0 +1,93 @@ +package verify + +import ( + "fmt" + "io" + "os" + "path/filepath" + + "github.com/dgraph-io/badger/v2" + badgerds "github.com/ipfs/go-ds-badger2" + + "github.com/onflow/flow-go/cmd/util/cmd/common" + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm" + "github.com/onflow/flow-go/fvm/evm/offchain/utils" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/blobs" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage" +) + +func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, executionDataDir string, evmStateGobDir string) error { + db, storages, executionDataStore, dsStore, err := initStorages(chainID, dataDir, executionDataDir) + if err != nil { + return fmt.Errorf("could not initialize storages: %w", err) + } + + defer db.Close() + defer dsStore.Close() + + var store *testutils.TestValueStore + isRoot := isEVMRootHeight(chainID, from) + if isRoot { + store = testutils.GetSimpleValueStore() + as := environment.NewAccountStatus() + rootAddr := evm.StorageAccountAddress(chainID) + err = store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes()) + if err != nil { + return err + } + } else { + // TODO: recover from gob + } + + return utils.OffchainReplayBackwardCompatibilityTest( + chainID, + from, + to, + storages.Headers, + storages.Results, + executionDataStore, + store, + ) +} + +func initStorages(chainID flow.ChainID, dataDir string, executionDataDir string) ( + *badger.DB, + *storage.All, + execution_data.ExecutionDataGetter, + io.Closer, + error, +) { + db := common.InitStorage(dataDir) + + storages := common.InitStorages(db) + + datastoreDir := filepath.Join(executionDataDir, "blobstore") + err := os.MkdirAll(datastoreDir, 0700) + if err != nil { + return nil, nil, nil, nil, err + } + dsOpts := &badgerds.DefaultOptions + ds, err := badgerds.NewDatastore(datastoreDir, dsOpts) + if err != nil { + return nil, nil, nil, nil, err + } + + executionDataBlobstore := blobs.NewBlobstore(ds) + executionDataStore := execution_data.NewExecutionDataStore(executionDataBlobstore, execution_data.DefaultSerializer) + + return db, storages, executionDataStore, ds, nil +} + +// EVM Root Height is the first block that has EVM Block Event where the EVM block height is 1 +func isEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { + if chainID == flow.Testnet { + return flowHeight == 211176671 + } else if chainID == flow.Mainnet { + return flowHeight == 85981136 + } + return flowHeight == 1 +} diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index a18ce4a81ac..827bb918601 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -41,7 +41,7 @@ func TestTestnetBackwardCompatibility(t *testing.T) { // > ~/Downloads/events_devnet51_1.jsonl // ... // - // 2) comment the above t.Skip, and update the events file paths and checkpoint dir + // 2) comment the above t.Skip, and update the events file paths and evmStateGob dir // to run the tests BackwardCompatibleSinceEVMGenesisBlock( t, flow.Testnet, []string{ @@ -65,47 +65,47 @@ func TestTestnetBackwardCompatibility(t *testing.T) { // --start 211176670 --end 211176770 --network testnet --host access-001.devnet51.nodes.onflow.org:9000 // // During the replay process, it will generate `values_.gob` and -// `allocators_.gob` checkpoint files for each height. If these checkpoint files exist, +// `allocators_.gob` checkpoint files for each height. If these checkpoint gob files exist, // the corresponding event JSON files will be skipped to optimize replay. func BackwardCompatibleSinceEVMGenesisBlock( t *testing.T, chainID flow.ChainID, eventsFilePaths []string, // ordered EVM events in JSONL format - checkpointDir string, - checkpointEndHeight uint64, // EVM height of an EVM state that a checkpoint was created for + evmStateGob string, + evmStateEndHeight uint64, // EVM height of an EVM state that a evmStateGob file was created for ) { // ensure that event files is not an empty array require.True(t, len(eventsFilePaths) > 0) - log.Info().Msgf("replaying EVM events from %v to %v, with checkpoints in %s, and checkpointEndHeight: %v", + log.Info().Msgf("replaying EVM events from %v to %v, with evmStateGob file in %s, and evmStateEndHeight: %v", eventsFilePaths[0], eventsFilePaths[len(eventsFilePaths)-1], - checkpointDir, checkpointEndHeight) + evmStateGob, evmStateEndHeight) - store, checkpointEndHeightOrZero := initStorageWithCheckpoints(t, chainID, checkpointDir, checkpointEndHeight) + store, evmStateEndHeightOrZero := initStorageWithEVMStateGob(t, chainID, evmStateGob, evmStateEndHeight) // the events to replay - nextHeight := checkpointEndHeightOrZero + 1 + nextHeight := evmStateEndHeightOrZero + 1 // replay each event files for _, eventsFilePath := range eventsFilePaths { log.Info().Msgf("replaying events from %v, nextHeight: %v", eventsFilePath, nextHeight) - checkpointEndHeight := replayEvents(t, chainID, store, eventsFilePath, checkpointDir, nextHeight) - nextHeight = checkpointEndHeight + 1 + evmStateEndHeight := replayEvents(t, chainID, store, eventsFilePath, evmStateGob, nextHeight) + nextHeight = evmStateEndHeight + 1 } log.Info(). Msgf("succhessfully replayed all events and state changes are consistent with onchain state change. nextHeight: %v", nextHeight) } -func initStorageWithCheckpoints(t *testing.T, chainID flow.ChainID, checkpointDir string, checkpointEndHeight uint64) ( +func initStorageWithEVMStateGob(t *testing.T, chainID flow.ChainID, evmStateGob string, evmStateEndHeight uint64) ( *TestValueStore, uint64, ) { rootAddr := evm.StorageAccountAddress(chainID) - // if there is no checkpoint, create a empty store and initialize the account status, + // if there is no evmStateGob file, create a empty store and initialize the account status, // return 0 as the genesis height - if checkpointEndHeight == 0 { + if evmStateEndHeight == 0 { store := GetSimpleValueStore() as := environment.NewAccountStatus() require.NoError(t, store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes())) @@ -113,19 +113,19 @@ func initStorageWithCheckpoints(t *testing.T, chainID flow.ChainID, checkpointDi return store, 0 } - valueFileName, allocatorFileName := checkpointFileNamesByEndHeight(checkpointDir, checkpointEndHeight) + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGob, evmStateEndHeight) values, err := deserialize(valueFileName) require.NoError(t, err) allocators, err := deserializeAllocator(allocatorFileName) require.NoError(t, err) store := GetSimpleValueStorePopulated(values, allocators) - return store, checkpointEndHeight + return store, evmStateEndHeight } func replayEvents( t *testing.T, chainID flow.ChainID, - store *TestValueStore, eventsFilePath string, checkpointDir string, initialNextHeight uint64) uint64 { + store *TestValueStore, eventsFilePath string, evmStateGob string, initialNextHeight uint64) uint64 { rootAddr := evm.StorageAccountAddress(chainID) @@ -172,22 +172,22 @@ func replayEvents( return nil }) - checkpointEndHeight := nextHeight - 1 + evmStateEndHeight := nextHeight - 1 - log.Info().Msgf("finished replaying events from %v to %v, creating checkpoint", initialNextHeight, checkpointEndHeight) - valuesFile, allocatorsFile := dumpCheckpoint(t, store, checkpointDir, checkpointEndHeight) - log.Info().Msgf("checkpoint created: %v, %v", valuesFile, allocatorsFile) + log.Info().Msgf("finished replaying events from %v to %v, creating evm state gobs", initialNextHeight, evmStateEndHeight) + valuesFile, allocatorsFile := dumpEVMStateToGobFiles(t, store, evmStateGob, evmStateEndHeight) + log.Info().Msgf("evm state gobs created: %v, %v", valuesFile, allocatorsFile) - return checkpointEndHeight + return evmStateEndHeight } -func checkpointFileNamesByEndHeight(dir string, endHeight uint64) (string, string) { +func evmStateGobFileNamesByEndHeight(dir string, endHeight uint64) (string, string) { return filepath.Join(dir, fmt.Sprintf("values_%d.gob", endHeight)), filepath.Join(dir, fmt.Sprintf("allocators_%d.gob", endHeight)) } -func dumpCheckpoint(t *testing.T, store *TestValueStore, dir string, checkpointEndHeight uint64) (string, string) { - valuesFileName, allocatorsFileName := checkpointFileNamesByEndHeight(dir, checkpointEndHeight) +func dumpEVMStateToGobFiles(t *testing.T, store *TestValueStore, dir string, evmStateEndHeight uint64) (string, string) { + valuesFileName, allocatorsFileName := evmStateGobFileNamesByEndHeight(dir, evmStateEndHeight) values, allocators := store.Dump() require.NoError(t, serialize(valuesFileName, values)) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go new file mode 100644 index 00000000000..bf3ed506adf --- /dev/null +++ b/fvm/evm/offchain/utils/verify.go @@ -0,0 +1,168 @@ +package utils + +import ( + "context" + "errors" + "strings" + + "github.com/rs/zerolog/log" + + "github.com/onflow/cadence" + "github.com/onflow/cadence/encoding/ccf" + + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm" + "github.com/onflow/flow-go/fvm/evm/events" + "github.com/onflow/flow-go/fvm/evm/offchain/blocks" + evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" + "github.com/onflow/flow-go/fvm/evm/offchain/sync" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + "github.com/onflow/flow-go/storage" +) + +func OffchainReplayBackwardCompatibilityTest( + chainID flow.ChainID, + flowStartHeight uint64, + flowEndHeight uint64, + headers storage.Headers, + results storage.ExecutionResults, + executionDataStore execution_data.ExecutionDataGetter, + store environment.ValueStore, +) error { + rootAddr := evm.StorageAccountAddress(chainID) + rootAddrStr := string(rootAddr.Bytes()) + + bpStorage := evmStorage.NewEphemeralStorage(store) + bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) + if err != nil { + return err + } + + for height := flowStartHeight; height <= flowEndHeight; height++ { + blockID, err := headers.BlockIDByHeight(height) + if err != nil { + return err + } + + result, err := results.ByBlockID(blockID) + if err != nil { + return err + } + + executionData, err := executionDataStore.Get(context.Background(), result.ExecutionDataID) + if err != nil { + return err + } + + events := flow.EventsList{} + payloads := []*ledger.Payload{} + + for _, chunkData := range executionData.ChunkExecutionDatas { + events = append(events, chunkData.Events...) + payloads = append(payloads, chunkData.TrieUpdate.Payloads...) + } + + updates := make(map[flow.RegisterID]flow.RegisterValue, len(payloads)) + for i := len(payloads) - 1; i >= 0; i-- { + regID, regVal, err := convert.PayloadToRegister(payloads[i]) + if err != nil { + return err + } + + // skip non-evm-account registers + if regID.Owner != rootAddrStr { + continue + } + + // when iterating backwards, duplicated register updates are stale updates, + // so skipping them + if _, ok := updates[regID]; !ok { + updates[regID] = regVal + } + } + + // parse events + evmBlockEvent, evmTxEvents, err := parseEVMEvents(events) + if err != nil { + return err + } + + err = bp.OnBlockReceived(evmBlockEvent) + if err != nil { + return err + } + + sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) + cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log.Logger, nil, true) + res, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) + if err != nil { + return err + } + + // commit all changes + for k, v := range res.StorageRegisterUpdates() { + err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) + if err != nil { + return err + } + } + + err = bp.OnBlockExecuted(evmBlockEvent.Height, res) + if err != nil { + return err + } + + // verify and commit all block hash list changes + for k, v := range bpStorage.StorageRegisterUpdates() { + // verify the block hash list changes are included in the trie update + + err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) + if err != nil { + return err + } + } + } + + return nil +} + +func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.TransactionEventPayload, error) { + var blockEvent *events.BlockEventPayload + txEvents := make([]events.TransactionEventPayload, 0) + + for _, e := range evts { + evtType := string(e.Type) + if strings.Contains(evtType, "BlockExecuted") { + if blockEvent != nil { + return nil, nil, errors.New("multiple block events in a single block") + } + + ev, err := ccf.Decode(nil, e.Payload) + if err != nil { + return nil, nil, err + } + + blockEventPayload, err := events.DecodeBlockEventPayload(ev.(cadence.Event)) + if err != nil { + return nil, nil, err + } + blockEvent = blockEventPayload + } else if strings.Contains(evtType, "TransactionExecuted") { + ev, err := ccf.Decode(nil, e.Payload) + if err != nil { + return nil, nil, err + } + txEv, err := events.DecodeTransactionEventPayload(ev.(cadence.Event)) + if err != nil { + return nil, nil, err + } + txEvents = append(txEvents, *txEv) + } + } + + return blockEvent, txEvents, nil +} From b560792abe1239e44d4354ead3367436b909f707 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 21 Nov 2024 14:49:32 -0800 Subject: [PATCH 10/76] refactor serailization with gob --- .../cmd/verify-evm-offchain-replay/main.go | 2 +- .../cmd/verify-evm-offchain-replay/verify.go | 38 +++++++- fvm/evm/offchain/utils/collection_test.go | 91 +------------------ fvm/evm/testutils/gob.go | 88 ++++++++++++++++++ 4 files changed, 129 insertions(+), 90 deletions(-) create mode 100644 fvm/evm/testutils/gob.go diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go index 9f56587306e..76581e8a471 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/main.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -55,7 +55,7 @@ func run(*cobra.Command, []string) { } log.Info().Msgf("verifying range from %d to %d", from, to) - err = Verify(from, to, flow.Testnet, flagDatadir, flagExecutionDataDir) + err = Verify(from, to, flow.Testnet, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir) if err != nil { log.Fatal().Err(err).Msg("could not verify last k height") } diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go index 1a907be669a..bbdd9911c21 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/verify.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -40,10 +40,21 @@ func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, execut return err } } else { - // TODO: recover from gob + prev := from - 1 + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, prev) + values, err := testutils.DeserializeState(valueFileName) + if err != nil { + return err + } + + allocators, err := testutils.DeserializeAllocator(allocatorFileName) + if err != nil { + return err + } + store = testutils.GetSimpleValueStorePopulated(values, allocators) } - return utils.OffchainReplayBackwardCompatibilityTest( + err = utils.OffchainReplayBackwardCompatibilityTest( chainID, from, to, @@ -52,6 +63,23 @@ func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, execut executionDataStore, store, ) + + if err != nil { + return err + } + + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, to) + values, allocators := store.Dump() + err = testutils.SerializeState(valueFileName, values) + if err != nil { + return err + } + err = testutils.SerializeAllocator(allocatorFileName, allocators) + if err != nil { + return err + } + + return nil } func initStorages(chainID flow.ChainID, dataDir string, executionDataDir string) ( @@ -91,3 +119,9 @@ func isEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { } return flowHeight == 1 } + +func evmStateGobFileNamesByEndHeight(evmStateGobDir string, endHeight uint64) (string, string) { + valueFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("values-%d.gob", endHeight)) + allocatorFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("allocators-%d.gob", endHeight)) + return valueFileName, allocatorFileName +} diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index 827bb918601..e5b3059661b 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -2,7 +2,6 @@ package utils_test import ( "bufio" - "encoding/gob" "encoding/hex" "encoding/json" "fmt" @@ -114,9 +113,9 @@ func initStorageWithEVMStateGob(t *testing.T, chainID flow.ChainID, evmStateGob } valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGob, evmStateEndHeight) - values, err := deserialize(valueFileName) + values, err := DeserializeState(valueFileName) require.NoError(t, err) - allocators, err := deserializeAllocator(allocatorFileName) + allocators, err := DeserializeAllocator(allocatorFileName) require.NoError(t, err) store := GetSimpleValueStorePopulated(values, allocators) return store, evmStateEndHeight @@ -190,8 +189,8 @@ func dumpEVMStateToGobFiles(t *testing.T, store *TestValueStore, dir string, evm valuesFileName, allocatorsFileName := evmStateGobFileNamesByEndHeight(dir, evmStateEndHeight) values, allocators := store.Dump() - require.NoError(t, serialize(valuesFileName, values)) - require.NoError(t, serializeAllocator(allocatorsFileName, allocators)) + require.NoError(t, SerializeState(valuesFileName, values)) + require.NoError(t, SerializeAllocator(allocatorsFileName, allocators)) return valuesFileName, allocatorsFileName } @@ -244,85 +243,3 @@ func scanEventFilesAndRun( t.Fatal(err) } } - -// Serialize function: saves map data to a file -func serialize(filename string, data map[string][]byte) error { - // Create a file to save data - file, err := os.Create(filename) - if err != nil { - return err - } - defer file.Close() - - // Use gob to encode data - encoder := gob.NewEncoder(file) - err = encoder.Encode(data) - if err != nil { - return err - } - - return nil -} - -// Deserialize function: reads map data from a file -func deserialize(filename string) (map[string][]byte, error) { - // Open the file for reading - file, err := os.Open(filename) - if err != nil { - return nil, err - } - defer file.Close() - - // Prepare the map to store decoded data - var data map[string][]byte - - // Use gob to decode data - decoder := gob.NewDecoder(file) - err = decoder.Decode(&data) - if err != nil { - return nil, err - } - - return data, nil -} - -// Serialize function: saves map data to a file -func serializeAllocator(filename string, data map[string]uint64) error { - // Create a file to save data - file, err := os.Create(filename) - if err != nil { - return err - } - defer file.Close() - - // Use gob to encode data - encoder := gob.NewEncoder(file) - err = encoder.Encode(data) - if err != nil { - return err - } - - return nil -} - -// Deserialize function: reads map data from a file -func deserializeAllocator(filename string) (map[string]uint64, error) { - // Open the file for reading - file, err := os.Open(filename) - if err != nil { - return nil, err - } - defer file.Close() - - // Prepare the map to store decoded data - var data map[string]uint64 - - // Use gob to decode data - decoder := gob.NewDecoder(file) - err = decoder.Decode(&data) - if err != nil { - return nil, err - } - - return data, nil -} diff --git a/fvm/evm/testutils/gob.go b/fvm/evm/testutils/gob.go new file mode 100644 index 00000000000..1c944a1e9e3 --- /dev/null +++ b/fvm/evm/testutils/gob.go @@ -0,0 +1,88 @@ +package testutils + +import ( + "encoding/gob" + "os" +) + +// Serialize function: saves map data to a file +func SerializeState(filename string, data map[string][]byte) error { + // Create a file to save data + file, err := os.Create(filename) + if err != nil { + return err + } + defer file.Close() + + // Use gob to encode data + encoder := gob.NewEncoder(file) + err = encoder.Encode(data) + if err != nil { + return err + } + + return nil +} + +// Deserialize function: reads map data from a file +func DeserializeState(filename string) (map[string][]byte, error) { + // Open the file for reading + file, err := os.Open(filename) + if err != nil { + return nil, err + } + defer file.Close() + + // Prepare the map to store decoded data + var data map[string][]byte + + // Use gob to decode data + decoder := gob.NewDecoder(file) + err = decoder.Decode(&data) + if err != nil { + return nil, err + } + + return data, nil +} + +// Serialize function: saves map data to a file +func SerializeAllocator(filename string, data map[string]uint64) error { + // Create a file to save data + file, err := os.Create(filename) + if err != nil { + return err + } + defer file.Close() + + // Use gob to encode data + encoder := gob.NewEncoder(file) + err = encoder.Encode(data) + if err != nil { + return err + } + + return nil +} + +// Deserialize function: reads map data from a file +func DeserializeAllocator(filename string) (map[string]uint64, error) { + // Open the file for reading + file, err := os.Open(filename) + if err != nil { + return nil, err + } + defer file.Close() + + // Prepare the map to store decoded data + var data map[string]uint64 + + // Use gob to decode data + decoder := gob.NewDecoder(file) + err = decoder.Decode(&data) + if err != nil { + return nil, err + } + + return data, nil +} From 626747690801913f433774995bcc24f78f2a11fd Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 21 Nov 2024 15:20:56 -0800 Subject: [PATCH 11/76] add logging --- .../cmd/verify-evm-offchain-replay/main.go | 2 +- .../cmd/verify-evm-offchain-replay/verify.go | 23 ++++++++++++++++--- fvm/evm/offchain/utils/verify.go | 8 ++++--- 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go index 76581e8a471..2459a35cd59 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/main.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -55,7 +55,7 @@ func run(*cobra.Command, []string) { } log.Info().Msgf("verifying range from %d to %d", from, to) - err = Verify(from, to, flow.Testnet, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir) + err = Verify(log.Logger, from, to, flow.Testnet, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir) if err != nil { log.Fatal().Err(err).Msg("could not verify last k height") } diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go index bbdd9911c21..f75cd8278b6 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/verify.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -8,6 +8,7 @@ import ( "github.com/dgraph-io/badger/v2" badgerds "github.com/ipfs/go-ds-badger2" + "github.com/rs/zerolog" "github.com/onflow/flow-go/cmd/util/cmd/common" "github.com/onflow/flow-go/fvm/environment" @@ -20,7 +21,16 @@ import ( "github.com/onflow/flow-go/storage" ) -func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, executionDataDir string, evmStateGobDir string) error { +// Verify verifies the offchain replay of EVM blocks from the given height range +// and updates the EVM state gob files with the latest state +func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, dataDir string, executionDataDir string, evmStateGobDir string) error { + log.Info(). + Str("chain", chainID.String()). + Str("dataDir", dataDir). + Str("executionDataDir", executionDataDir). + Str("evmStateGobDir", evmStateGobDir). + Msgf("verifying range from %d to %d", from, to) + db, storages, executionDataStore, dsStore, err := initStorages(chainID, dataDir, executionDataDir) if err != nil { return fmt.Errorf("could not initialize storages: %w", err) @@ -32,6 +42,8 @@ func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, execut var store *testutils.TestValueStore isRoot := isEVMRootHeight(chainID, from) if isRoot { + log.Info().Msgf("initializing EVM state for root height %d", from) + store = testutils.GetSimpleValueStore() as := environment.NewAccountStatus() rootAddr := evm.StorageAccountAddress(chainID) @@ -41,20 +53,23 @@ func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, execut } } else { prev := from - 1 + log.Info().Msgf("loading EVM state from previous height %d", prev) + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, prev) values, err := testutils.DeserializeState(valueFileName) if err != nil { - return err + return fmt.Errorf("could not deserialize state %v: %w", valueFileName, err) } allocators, err := testutils.DeserializeAllocator(allocatorFileName) if err != nil { - return err + return fmt.Errorf("could not deserialize allocator %v: %w", allocatorFileName, err) } store = testutils.GetSimpleValueStorePopulated(values, allocators) } err = utils.OffchainReplayBackwardCompatibilityTest( + log, chainID, from, to, @@ -79,6 +94,8 @@ func Verify(from uint64, to uint64, chainID flow.ChainID, dataDir string, execut return err } + log.Info().Msgf("saved EVM state to %s and %s", valueFileName, allocatorFileName) + return nil } diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index bf3ed506adf..ae99e827acb 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -5,10 +5,9 @@ import ( "errors" "strings" - "github.com/rs/zerolog/log" - "github.com/onflow/cadence" "github.com/onflow/cadence/encoding/ccf" + "github.com/rs/zerolog" "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" @@ -25,6 +24,7 @@ import ( ) func OffchainReplayBackwardCompatibilityTest( + log zerolog.Logger, chainID flow.ChainID, flowStartHeight uint64, flowEndHeight uint64, @@ -97,7 +97,7 @@ func OffchainReplayBackwardCompatibilityTest( } sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) - cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log.Logger, nil, true) + cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) res, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) if err != nil { return err @@ -125,6 +125,8 @@ func OffchainReplayBackwardCompatibilityTest( return err } } + + log.Info().Msgf("verified block %d", height) } return nil From fd24bb89877939fc90ade8bcec2881c4a5705f28 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 06:28:14 -0800 Subject: [PATCH 12/76] update error message --- cmd/util/cmd/verify-evm-offchain-replay/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go index 2459a35cd59..0bc6eef8187 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/main.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -57,7 +57,7 @@ func run(*cobra.Command, []string) { log.Info().Msgf("verifying range from %d to %d", from, to) err = Verify(log.Logger, from, to, flow.Testnet, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir) if err != nil { - log.Fatal().Err(err).Msg("could not verify last k height") + log.Fatal().Err(err).Msg("could not verify height") } log.Info().Msgf("successfully verified range from %d to %d", from, to) From c2560bc05c0f03395607dee11576d6da9850f5b6 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 08:44:36 -0800 Subject: [PATCH 13/76] add register checks --- fvm/evm/offchain/utils/verify.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index ae99e827acb..2045de36f22 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -1,8 +1,10 @@ package utils import ( + "bytes" "context" "errors" + "fmt" "strings" "github.com/onflow/cadence" @@ -124,6 +126,21 @@ func OffchainReplayBackwardCompatibilityTest( if err != nil { return err } + + expectedUpdate, ok := updates[k] + if !ok { + return fmt.Errorf("missing update for register %v, %v", k, expectedUpdate) + } + + if !bytes.Equal(expectedUpdate, v) { + return fmt.Errorf("unexpected update for register %v, expected %v, got %v", k, expectedUpdate, v) + } + + delete(updates, k) + } + + if len(updates) > 0 { + return fmt.Errorf("missing updates for registers %v", updates) } log.Info().Msgf("verified block %d", height) From 52f7f6c02b043c4f1dbe3f40cc1ec4bef45d7332 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 10:30:11 -0800 Subject: [PATCH 14/76] store block proposal in replay --- fvm/evm/offchain/blocks/provider.go | 50 +++++++++++++++++++++-- fvm/evm/offchain/sync/replay.go | 37 +++++++++-------- fvm/evm/offchain/sync/replayer.go | 18 ++++---- fvm/evm/offchain/sync/replayer_test.go | 4 +- fvm/evm/offchain/utils/collection_test.go | 2 +- fvm/evm/offchain/utils/verify.go | 39 +++++++++++++++++- 6 files changed, 119 insertions(+), 31 deletions(-) diff --git a/fvm/evm/offchain/blocks/provider.go b/fvm/evm/offchain/blocks/provider.go index 9111be4ac64..b9da39bd468 100644 --- a/fvm/evm/offchain/blocks/provider.go +++ b/fvm/evm/offchain/blocks/provider.go @@ -4,6 +4,7 @@ import ( "fmt" "github.com/onflow/flow-go/fvm/evm/events" + "github.com/onflow/flow-go/fvm/evm/handler" "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" ) @@ -13,7 +14,10 @@ import ( // a OnBlockReceived call before block execution and // a follow up OnBlockExecuted call after block execution. type BasicProvider struct { + chainID flow.ChainID blks *Blocks + rootAddr flow.Address + storage types.BackendStorage latestBlockPayload *events.BlockEventPayload } @@ -28,7 +32,12 @@ func NewBasicProvider( if err != nil { return nil, err } - return &BasicProvider{blks: blks}, nil + return &BasicProvider{ + chainID: chainID, + blks: blks, + rootAddr: rootAddr, + storage: storage, + }, nil } // GetSnapshotAt returns a block snapshot at the given height @@ -61,14 +70,49 @@ func (p *BasicProvider) OnBlockReceived(blockEvent *events.BlockEventPayload) er // OnBlockExecuted should be called after executing blocks. func (p *BasicProvider) OnBlockExecuted( height uint64, - resCol types.ReplayResultCollector) error { + resCol types.ReplayResultCollector, + blockProposal *types.BlockProposal, +) error { // we push the block hash after execution, so the behaviour of the blockhash is // identical to the evm.handler. if p.latestBlockPayload.Height != height { return fmt.Errorf("active block height doesn't match expected: %d, got: %d", p.latestBlockPayload.Height, height) } + + blockBytes, err := blockProposal.Block.ToBytes() + if err != nil { + return types.NewFatalError(err) + } + + // do the same as handler.CommitBlockProposal + err = p.storage.SetValue( + p.rootAddr[:], + []byte(handler.BlockStoreLatestBlockKey), + blockBytes, + ) + if err != nil { + return err + } + + blockProposalBytes, err := blockProposal.ToBytes() + if err != nil { + return types.NewFatalError(err) + } + + hash := p.latestBlockPayload.Hash + // update block proposal + err = p.storage.SetValue( + p.rootAddr[:], + []byte(handler.BlockStoreLatestBlockProposalKey), + blockProposalBytes, + ) + if err != nil { + return err + } + + // update block hash list return p.blks.PushBlockHash( p.latestBlockPayload.Height, - p.latestBlockPayload.Hash, + hash, ) } diff --git a/fvm/evm/offchain/sync/replay.go b/fvm/evm/offchain/sync/replay.go index 4516f37007d..e85fc21658c 100644 --- a/fvm/evm/offchain/sync/replay.go +++ b/fvm/evm/offchain/sync/replay.go @@ -30,25 +30,26 @@ func ReplayBlockExecution( transactionEvents []events.TransactionEventPayload, blockEvent *events.BlockEventPayload, validateResults bool, -) error { +) ([]*types.Result, error) { // check the passed block event if blockEvent == nil { - return fmt.Errorf("nil block event has been passed") + return nil, fmt.Errorf("nil block event has been passed") } // create a base block context for all transactions // tx related context values will be replaced during execution ctx, err := blockSnapshot.BlockContext() if err != nil { - return err + return nil, err } // update the tracer ctx.Tracer = tracer gasConsumedSoFar := uint64(0) txHashes := make(types.TransactionHashes, len(transactionEvents)) + results := make([]*types.Result, 0, len(transactionEvents)) for idx, tx := range transactionEvents { - err = replayTransactionExecution( + result, err := replayTransactionExecution( rootAddr, ctx, uint(idx), @@ -58,28 +59,30 @@ func ReplayBlockExecution( validateResults, ) if err != nil { - return fmt.Errorf("transaction execution failed, txIndex: %d, err: %w", idx, err) + return nil, fmt.Errorf("transaction execution failed, txIndex: %d, err: %w", idx, err) } gasConsumedSoFar += tx.GasConsumed txHashes[idx] = tx.Hash + + results = append(results, result) } if validateResults { // check transaction inclusion txHashRoot := gethTypes.DeriveSha(txHashes, gethTrie.NewStackTrie(nil)) if txHashRoot != blockEvent.TransactionHashRoot { - return fmt.Errorf("transaction root hash doesn't match [%x] != [%x]", txHashRoot, blockEvent.TransactionHashRoot) + return nil, fmt.Errorf("transaction root hash doesn't match [%x] != [%x]", txHashRoot, blockEvent.TransactionHashRoot) } // check total gas used if blockEvent.TotalGasUsed != gasConsumedSoFar { - return fmt.Errorf("total gas used doesn't match [%d] != [%d]", gasConsumedSoFar, blockEvent.TotalGasUsed) + return nil, fmt.Errorf("total gas used doesn't match [%d] != [%d]", gasConsumedSoFar, blockEvent.TotalGasUsed) } // no need to check the receipt root hash given we have checked the logs and other // values during tx execution. } - return nil + return results, nil } func replayTransactionExecution( @@ -90,7 +93,7 @@ func replayTransactionExecution( ledger atree.Ledger, txEvent *events.TransactionEventPayload, validate bool, -) error { +) (*types.Result, error) { // create emulator em := emulator.NewEmulator(ledger, rootAddr) @@ -102,7 +105,7 @@ func replayTransactionExecution( if len(txEvent.PrecompiledCalls) > 0 { pcs, err := types.AggregatedPrecompileCallsFromEncoded(txEvent.PrecompiledCalls) if err != nil { - return fmt.Errorf("error decoding precompiled calls [%x]: %w", txEvent.Payload, err) + return nil, fmt.Errorf("error decoding precompiled calls [%x]: %w", txEvent.Payload, err) } ctx.ExtraPrecompiledContracts = precompiles.AggregatedPrecompiledCallsToPrecompiledContracts(pcs) } @@ -110,7 +113,7 @@ func replayTransactionExecution( // create a new block view bv, err := em.NewBlockView(ctx) if err != nil { - return err + return nil, err } var res *types.Result @@ -119,31 +122,31 @@ func replayTransactionExecution( if txEvent.TransactionType == types.DirectCallTxType { call, err := types.DirectCallFromEncoded(txEvent.Payload) if err != nil { - return fmt.Errorf("failed to RLP-decode direct call [%x]: %w", txEvent.Payload, err) + return nil, fmt.Errorf("failed to RLP-decode direct call [%x]: %w", txEvent.Payload, err) } res, err = bv.DirectCall(call) if err != nil { - return fmt.Errorf("failed to execute direct call [%x]: %w", txEvent.Hash, err) + return nil, fmt.Errorf("failed to execute direct call [%x]: %w", txEvent.Hash, err) } } else { gethTx := &gethTypes.Transaction{} if err := gethTx.UnmarshalBinary(txEvent.Payload); err != nil { - return fmt.Errorf("failed to RLP-decode transaction [%x]: %w", txEvent.Payload, err) + return nil, fmt.Errorf("failed to RLP-decode transaction [%x]: %w", txEvent.Payload, err) } res, err = bv.RunTransaction(gethTx) if err != nil { - return fmt.Errorf("failed to run transaction [%x]: %w", txEvent.Hash, err) + return nil, fmt.Errorf("failed to run transaction [%x]: %w", txEvent.Hash, err) } } // validate results if validate { if err := ValidateResult(res, txEvent); err != nil { - return fmt.Errorf("transaction replay failed (txHash %x): %w", txEvent.Hash, err) + return nil, fmt.Errorf("transaction replay failed (txHash %x): %w", txEvent.Hash, err) } } - return nil + return res, nil } func ValidateResult( diff --git a/fvm/evm/offchain/sync/replayer.go b/fvm/evm/offchain/sync/replayer.go index 25ccdc10cbf..33411b7c133 100644 --- a/fvm/evm/offchain/sync/replayer.go +++ b/fvm/evm/offchain/sync/replayer.go @@ -46,7 +46,11 @@ func NewReplayer( // ReplayBlock replays the execution of the transactions of an EVM block // using the provided transactionEvents and blockEvents, -// which include all the context data for re-executing the transactions, and returns the replay result. +// which include all the context data for re-executing the transactions, and returns +// the replay result and the result of each transaction. +// the replay result contains the register updates, and the result of each transaction +// contains the execution result of each transaction, which is useful for recontstructing +// the EVM block proposal. // this method can be called concurrently if underlying storage // tracer and block snapshot provider support concurrency. // @@ -56,11 +60,11 @@ func NewReplayer( func (cr *Replayer) ReplayBlock( transactionEvents []events.TransactionEventPayload, blockEvent *events.BlockEventPayload, -) (types.ReplayResultCollector, error) { +) (types.ReplayResultCollector, []*types.Result, error) { // prepare storage st, err := cr.storageProvider.GetSnapshotAt(blockEvent.Height) if err != nil { - return nil, err + return nil, nil, err } // create storage @@ -69,11 +73,11 @@ func (cr *Replayer) ReplayBlock( // get block snapshot bs, err := cr.blockProvider.GetSnapshotAt(blockEvent.Height) if err != nil { - return nil, err + return nil, nil, err } // replay transactions - err = ReplayBlockExecution( + results, err := ReplayBlockExecution( cr.chainID, cr.rootAddr, state, @@ -84,8 +88,8 @@ func (cr *Replayer) ReplayBlock( cr.validateResults, ) if err != nil { - return nil, err + return nil, nil, err } - return state, nil + return state, results, nil } diff --git a/fvm/evm/offchain/sync/replayer_test.go b/fvm/evm/offchain/sync/replayer_test.go index f7c05ab63b5..d193163283b 100644 --- a/fvm/evm/offchain/sync/replayer_test.go +++ b/fvm/evm/offchain/sync/replayer_test.go @@ -162,9 +162,11 @@ func TestChainReplay(t *testing.T) { sp := NewTestStorageProvider(snapshot, 1) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, zerolog.Logger{}, nil, true) - res, err := cr.ReplayBlock(txEventPayloads, blockEventPayload) + res, results, err := cr.ReplayBlock(txEventPayloads, blockEventPayload) require.NoError(t, err) + require.Len(t, results, totalTxCount) + err = bp.OnBlockExecuted(blockEventPayload.Height, res) require.NoError(t, err) diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index e5b3059661b..ae8b10a0e59 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -147,7 +147,7 @@ func replayEvents( sp := NewTestStorageProvider(store, blockEventPayload.Height) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, zerolog.Logger{}, nil, true) - res, err := cr.ReplayBlock(txEvents, blockEventPayload) + res, _, err := cr.ReplayBlock(txEvents, blockEventPayload) require.NoError(t, err) // commit all changes diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 2045de36f22..a3f3e871f13 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -18,6 +18,7 @@ import ( evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" "github.com/onflow/flow-go/fvm/evm/offchain/sync" "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/model/flow" @@ -100,7 +101,7 @@ func OffchainReplayBackwardCompatibilityTest( sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) - res, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) + res, results, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) if err != nil { return err } @@ -113,7 +114,9 @@ func OffchainReplayBackwardCompatibilityTest( } } - err = bp.OnBlockExecuted(evmBlockEvent.Height, res) + blockProposal := reconstructProposal(evmBlockEvent, evmTxEvents, results) + + err = bp.OnBlockExecuted(evmBlockEvent.Height, res, blockProposal) if err != nil { return err } @@ -185,3 +188,35 @@ func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.T return blockEvent, txEvents, nil } + +func reconstructProposal( + blockEvent *events.BlockEventPayload, + txEvents []events.TransactionEventPayload, + results []*types.Result, +) *types.BlockProposal { + receipts := make([]types.LightReceipt, 0, len(results)) + + for _, result := range results { + receipts = append(receipts, *result.LightReceipt()) + } + + txHashes := make(types.TransactionHashes, 0, len(txEvents)) + for _, tx := range txEvents { + txHashes = append(txHashes, tx.Hash) + } + + return &types.BlockProposal{ + Block: types.Block{ + ParentBlockHash: blockEvent.ParentBlockHash, + Height: blockEvent.Height, + Timestamp: blockEvent.Timestamp, + TotalSupply: blockEvent.TotalSupply.Big(), + ReceiptRoot: blockEvent.ReceiptRoot, + TransactionHashRoot: blockEvent.TransactionHashRoot, + TotalGasUsed: blockEvent.TotalGasUsed, + PrevRandao: blockEvent.PrevRandao, + }, + Receipts: receipts, + TxHashes: txHashes, + } +} From b96010e15f1e7d7d0664861214611ed516c03600 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 11:18:53 -0800 Subject: [PATCH 15/76] fix tests --- fvm/evm/offchain/blocks/block_proposal.go | 38 +++++++++++++++++++++++ fvm/evm/offchain/sync/replayer_test.go | 14 ++++----- fvm/evm/offchain/utils/collection_test.go | 6 ++-- fvm/evm/offchain/utils/verify.go | 35 +-------------------- 4 files changed, 50 insertions(+), 43 deletions(-) create mode 100644 fvm/evm/offchain/blocks/block_proposal.go diff --git a/fvm/evm/offchain/blocks/block_proposal.go b/fvm/evm/offchain/blocks/block_proposal.go new file mode 100644 index 00000000000..877ba3303fe --- /dev/null +++ b/fvm/evm/offchain/blocks/block_proposal.go @@ -0,0 +1,38 @@ +package blocks + +import ( + "github.com/onflow/flow-go/fvm/evm/events" + "github.com/onflow/flow-go/fvm/evm/types" +) + +func ReconstructProposal( + blockEvent *events.BlockEventPayload, + txEvents []events.TransactionEventPayload, + results []*types.Result, +) *types.BlockProposal { + receipts := make([]types.LightReceipt, 0, len(results)) + + for _, result := range results { + receipts = append(receipts, *result.LightReceipt()) + } + + txHashes := make(types.TransactionHashes, 0, len(txEvents)) + for _, tx := range txEvents { + txHashes = append(txHashes, tx.Hash) + } + + return &types.BlockProposal{ + Block: types.Block{ + ParentBlockHash: blockEvent.ParentBlockHash, + Height: blockEvent.Height, + Timestamp: blockEvent.Timestamp, + TotalSupply: blockEvent.TotalSupply.Big(), + ReceiptRoot: blockEvent.ReceiptRoot, + TransactionHashRoot: blockEvent.TransactionHashRoot, + TotalGasUsed: blockEvent.TotalGasUsed, + PrevRandao: blockEvent.PrevRandao, + }, + Receipts: receipts, + TxHashes: txHashes, + } +} diff --git a/fvm/evm/offchain/sync/replayer_test.go b/fvm/evm/offchain/sync/replayer_test.go index d193163283b..2da1a5ba76b 100644 --- a/fvm/evm/offchain/sync/replayer_test.go +++ b/fvm/evm/offchain/sync/replayer_test.go @@ -12,6 +12,7 @@ import ( "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/events" "github.com/onflow/flow-go/fvm/evm/offchain/blocks" + "github.com/onflow/flow-go/fvm/evm/offchain/storage" "github.com/onflow/flow-go/fvm/evm/offchain/sync" . "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/fvm/evm/types" @@ -154,7 +155,8 @@ func TestChainReplay(t *testing.T) { // check replay - bp, err := blocks.NewBasicProvider(chainID, snapshot, rootAddr) + bpStorage := storage.NewEphemeralStorage(snapshot) + bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) require.NoError(t, err) err = bp.OnBlockReceived(blockEventPayload) @@ -167,14 +169,12 @@ func TestChainReplay(t *testing.T) { require.Len(t, results, totalTxCount) - err = bp.OnBlockExecuted(blockEventPayload.Height, res) + proposal := blocks.ReconstructProposal(blockEventPayload, txEventPayloads, results) + + err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) require.NoError(t, err) - // TODO: verify the state delta - // currently the backend storage doesn't work well with this - // changes needed to make this work, which is left for future PRs - // - // for k, v := range result.StorageRegisterUpdates() { + // for k, v := range bpStorage.StorageRegisterUpdates() { // ret, err := backend.GetValue([]byte(k.Owner), []byte(k.Key)) // require.NoError(t, err) // require.Equal(t, ret[:], v[:]) diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index ae8b10a0e59..a4385c7f664 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -147,7 +147,7 @@ func replayEvents( sp := NewTestStorageProvider(store, blockEventPayload.Height) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, zerolog.Logger{}, nil, true) - res, _, err := cr.ReplayBlock(txEvents, blockEventPayload) + res, results, err := cr.ReplayBlock(txEvents, blockEventPayload) require.NoError(t, err) // commit all changes @@ -156,7 +156,9 @@ func replayEvents( require.NoError(t, err) } - err = bp.OnBlockExecuted(blockEventPayload.Height, res) + proposal := blocks.ReconstructProposal(blockEventPayload, txEvents, results) + + err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) require.NoError(t, err) // commit all block hash list changes diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index a3f3e871f13..3a3d9d9b9ce 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -18,7 +18,6 @@ import ( evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" "github.com/onflow/flow-go/fvm/evm/offchain/sync" "github.com/onflow/flow-go/fvm/evm/testutils" - "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/model/flow" @@ -114,7 +113,7 @@ func OffchainReplayBackwardCompatibilityTest( } } - blockProposal := reconstructProposal(evmBlockEvent, evmTxEvents, results) + blockProposal := blocks.ReconstructProposal(evmBlockEvent, evmTxEvents, results) err = bp.OnBlockExecuted(evmBlockEvent.Height, res, blockProposal) if err != nil { @@ -188,35 +187,3 @@ func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.T return blockEvent, txEvents, nil } - -func reconstructProposal( - blockEvent *events.BlockEventPayload, - txEvents []events.TransactionEventPayload, - results []*types.Result, -) *types.BlockProposal { - receipts := make([]types.LightReceipt, 0, len(results)) - - for _, result := range results { - receipts = append(receipts, *result.LightReceipt()) - } - - txHashes := make(types.TransactionHashes, 0, len(txEvents)) - for _, tx := range txEvents { - txHashes = append(txHashes, tx.Hash) - } - - return &types.BlockProposal{ - Block: types.Block{ - ParentBlockHash: blockEvent.ParentBlockHash, - Height: blockEvent.Height, - Timestamp: blockEvent.Timestamp, - TotalSupply: blockEvent.TotalSupply.Big(), - ReceiptRoot: blockEvent.ReceiptRoot, - TransactionHashRoot: blockEvent.TransactionHashRoot, - TotalGasUsed: blockEvent.TotalGasUsed, - PrevRandao: blockEvent.PrevRandao, - }, - Receipts: receipts, - TxHashes: txHashes, - } -} From faccfb684a370575f3d18694d6ff93b1d75ffd57 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 11:43:45 -0800 Subject: [PATCH 16/76] update error message --- fvm/evm/offchain/utils/verify.go | 76 +++++++++++++++++++++++++------- 1 file changed, 61 insertions(+), 15 deletions(-) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 3a3d9d9b9ce..64c50fce3b7 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -68,7 +68,7 @@ func OffchainReplayBackwardCompatibilityTest( payloads = append(payloads, chunkData.TrieUpdate.Payloads...) } - updates := make(map[flow.RegisterID]flow.RegisterValue, len(payloads)) + expectedUpdates := make(map[flow.RegisterID]flow.RegisterValue, len(payloads)) for i := len(payloads) - 1; i >= 0; i-- { regID, regVal, err := convert.PayloadToRegister(payloads[i]) if err != nil { @@ -82,8 +82,8 @@ func OffchainReplayBackwardCompatibilityTest( // when iterating backwards, duplicated register updates are stale updates, // so skipping them - if _, ok := updates[regID]; !ok { - updates[regID] = regVal + if _, ok := expectedUpdates[regID]; !ok { + expectedUpdates[regID] = regVal } } @@ -105,12 +105,16 @@ func OffchainReplayBackwardCompatibilityTest( return err } + actualUpdates := make(map[flow.RegisterID]flow.RegisterValue, len(expectedUpdates)) + // commit all changes for k, v := range res.StorageRegisterUpdates() { err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) if err != nil { return err } + + actualUpdates[k] = v } blockProposal := blocks.ReconstructProposal(evmBlockEvent, evmTxEvents, results) @@ -129,20 +133,12 @@ func OffchainReplayBackwardCompatibilityTest( return err } - expectedUpdate, ok := updates[k] - if !ok { - return fmt.Errorf("missing update for register %v, %v", k, expectedUpdate) - } - - if !bytes.Equal(expectedUpdate, v) { - return fmt.Errorf("unexpected update for register %v, expected %v, got %v", k, expectedUpdate, v) - } - - delete(updates, k) + actualUpdates[k] = v } - if len(updates) > 0 { - return fmt.Errorf("missing updates for registers %v", updates) + err = verifyRegisterUpdates(expectedUpdates, actualUpdates) + if err != nil { + return err } log.Info().Msgf("verified block %d", height) @@ -187,3 +183,53 @@ func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.T return blockEvent, txEvents, nil } + +func verifyRegisterUpdates(expectedUpdates map[flow.RegisterID]flow.RegisterValue, actualUpdates map[flow.RegisterID]flow.RegisterValue) error { + missingUpdates := make(map[flow.RegisterID]flow.RegisterValue) + additionalUpdates := make(map[flow.RegisterID]flow.RegisterValue) + mismatchingUpdates := make(map[flow.RegisterID][2]flow.RegisterValue) + + for k, v := range expectedUpdates { + if actualVal, ok := actualUpdates[k]; !ok { + missingUpdates[k] = v + } else if !bytes.Equal(v, actualVal) { + mismatchingUpdates[k] = [2]flow.RegisterValue{v, actualVal} + } + + delete(actualUpdates, k) + } + + for k, v := range actualUpdates { + additionalUpdates[k] = v + } + + // Build a combined error message + var errorMessage strings.Builder + + if len(missingUpdates) > 0 { + errorMessage.WriteString("Missing register updates:\n") + for id, value := range missingUpdates { + errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ExpectedValue: %v\n", id, value)) + } + } + + if len(additionalUpdates) > 0 { + errorMessage.WriteString("Additional register updates:\n") + for id, value := range additionalUpdates { + errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ActualValue: %v\n", id, value)) + } + } + + if len(mismatchingUpdates) > 0 { + errorMessage.WriteString("Mismatching register updates:\n") + for id, values := range mismatchingUpdates { + errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ExpectedValue: %v, ActualValue: %v\n", id, values[0], values[1])) + } + } + + if errorMessage.Len() > 0 { + return errors.New(errorMessage.String()) + } + + return nil +} From 6d3de03b03413d3a72bb2f7fa81ae2eff58ba535 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 11:45:59 -0800 Subject: [PATCH 17/76] update error message --- fvm/evm/offchain/utils/verify.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 64c50fce3b7..9afb272acec 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -209,21 +209,21 @@ func verifyRegisterUpdates(expectedUpdates map[flow.RegisterID]flow.RegisterValu if len(missingUpdates) > 0 { errorMessage.WriteString("Missing register updates:\n") for id, value := range missingUpdates { - errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ExpectedValue: %v\n", id, value)) + errorMessage.WriteString(fmt.Sprintf(" RegisterKey: %v, ExpectedValue: %x\n", id.Key, value)) } } if len(additionalUpdates) > 0 { errorMessage.WriteString("Additional register updates:\n") for id, value := range additionalUpdates { - errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ActualValue: %v\n", id, value)) + errorMessage.WriteString(fmt.Sprintf(" RegisterKey: %v, ActualValue: %x\n", id.Key, value)) } } if len(mismatchingUpdates) > 0 { errorMessage.WriteString("Mismatching register updates:\n") for id, values := range mismatchingUpdates { - errorMessage.WriteString(fmt.Sprintf(" RegisterID: %v, ExpectedValue: %v, ActualValue: %v\n", id, values[0], values[1])) + errorMessage.WriteString(fmt.Sprintf(" RegisterKey: %v, ExpectedValue: %x, ActualValue: %x\n", id.Key, values[0], values[1])) } } From 26b4481bc246cfb00348fc196d3344f8fdc2f80a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 12:06:47 -0800 Subject: [PATCH 18/76] add account status updates --- fvm/evm/offchain/blocks/block_proposal.go | 8 ++----- fvm/evm/offchain/sync/replayer_test.go | 2 +- fvm/evm/offchain/utils/collection_test.go | 2 +- fvm/evm/offchain/utils/verify.go | 27 +++++++++++++++++++---- 4 files changed, 27 insertions(+), 12 deletions(-) diff --git a/fvm/evm/offchain/blocks/block_proposal.go b/fvm/evm/offchain/blocks/block_proposal.go index 877ba3303fe..cd1d68ed517 100644 --- a/fvm/evm/offchain/blocks/block_proposal.go +++ b/fvm/evm/offchain/blocks/block_proposal.go @@ -7,18 +7,14 @@ import ( func ReconstructProposal( blockEvent *events.BlockEventPayload, - txEvents []events.TransactionEventPayload, results []*types.Result, ) *types.BlockProposal { receipts := make([]types.LightReceipt, 0, len(results)) + txHashes := make(types.TransactionHashes, 0, len(results)) for _, result := range results { receipts = append(receipts, *result.LightReceipt()) - } - - txHashes := make(types.TransactionHashes, 0, len(txEvents)) - for _, tx := range txEvents { - txHashes = append(txHashes, tx.Hash) + txHashes = append(txHashes, result.TxHash) } return &types.BlockProposal{ diff --git a/fvm/evm/offchain/sync/replayer_test.go b/fvm/evm/offchain/sync/replayer_test.go index 2da1a5ba76b..3668e445c84 100644 --- a/fvm/evm/offchain/sync/replayer_test.go +++ b/fvm/evm/offchain/sync/replayer_test.go @@ -169,7 +169,7 @@ func TestChainReplay(t *testing.T) { require.Len(t, results, totalTxCount) - proposal := blocks.ReconstructProposal(blockEventPayload, txEventPayloads, results) + proposal := blocks.ReconstructProposal(blockEventPayload, results) err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) require.NoError(t, err) diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index a4385c7f664..8e292530534 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -156,7 +156,7 @@ func replayEvents( require.NoError(t, err) } - proposal := blocks.ReconstructProposal(blockEventPayload, txEvents, results) + proposal := blocks.ReconstructProposal(blockEventPayload, results) err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) require.NoError(t, err) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 9afb272acec..740989eac23 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -25,6 +25,16 @@ import ( "github.com/onflow/flow-go/storage" ) +// EVM Root Height is the first block that has EVM Block Event where the EVM block height is 1 +func isEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { + if chainID == flow.Testnet { + return flowHeight == 211176671 + } else if chainID == flow.Mainnet { + return flowHeight == 85981136 + } + return flowHeight == 1 +} + func OffchainReplayBackwardCompatibilityTest( log zerolog.Logger, chainID flow.ChainID, @@ -44,6 +54,14 @@ func OffchainReplayBackwardCompatibilityTest( return err } + // setup account status at EVM root block + if isEVMRootHeight(chainID, flowStartHeight) { + err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), environment.NewAccountStatus().ToBytes()) + if err != nil { + return err + } + } + for height := flowStartHeight; height <= flowEndHeight; height++ { blockID, err := headers.BlockIDByHeight(height) if err != nil { @@ -87,7 +105,7 @@ func OffchainReplayBackwardCompatibilityTest( } } - // parse events + // parse EVM events evmBlockEvent, evmTxEvents, err := parseEVMEvents(events) if err != nil { return err @@ -107,7 +125,7 @@ func OffchainReplayBackwardCompatibilityTest( actualUpdates := make(map[flow.RegisterID]flow.RegisterValue, len(expectedUpdates)) - // commit all changes + // commit all register changes from the EVM state transition for k, v := range res.StorageRegisterUpdates() { err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) if err != nil { @@ -117,14 +135,15 @@ func OffchainReplayBackwardCompatibilityTest( actualUpdates[k] = v } - blockProposal := blocks.ReconstructProposal(evmBlockEvent, evmTxEvents, results) + blockProposal := blocks.ReconstructProposal(evmBlockEvent, results) err = bp.OnBlockExecuted(evmBlockEvent.Height, res, blockProposal) if err != nil { return err } - // verify and commit all block hash list changes + // commit all register changes from non-EVM state transition, such + // as block hash list changes for k, v := range bpStorage.StorageRegisterUpdates() { // verify the block hash list changes are included in the trie update From 7fc20ce7306984382282d0fcdda39eabc6dfbf9c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 16:32:42 -0800 Subject: [PATCH 19/76] update provider --- fvm/evm/offchain/utils/verify.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 740989eac23..f059874bbb9 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -48,21 +48,21 @@ func OffchainReplayBackwardCompatibilityTest( rootAddr := evm.StorageAccountAddress(chainID) rootAddrStr := string(rootAddr.Bytes()) - bpStorage := evmStorage.NewEphemeralStorage(store) - bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) - if err != nil { - return err - } - - // setup account status at EVM root block - if isEVMRootHeight(chainID, flowStartHeight) { - err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), environment.NewAccountStatus().ToBytes()) + for height := flowStartHeight; height <= flowEndHeight; height++ { + bpStorage := evmStorage.NewEphemeralStorage(store) + bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) if err != nil { return err } - } - for height := flowStartHeight; height <= flowEndHeight; height++ { + // setup account status at EVM root block + if isEVMRootHeight(chainID, flowStartHeight) { + err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), environment.NewAccountStatus().ToBytes()) + if err != nil { + return err + } + } + blockID, err := headers.BlockIDByHeight(height) if err != nil { return err From 2774fc38b65afd0c8fb918133575905f2279683e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 16:39:28 -0800 Subject: [PATCH 20/76] update verifable keys --- fvm/evm/handler/blockHashList.go | 9 +++++++++ fvm/evm/offchain/utils/verify.go | 14 ++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/fvm/evm/handler/blockHashList.go b/fvm/evm/handler/blockHashList.go index 91eefded24e..0db2aff73f9 100644 --- a/fvm/evm/handler/blockHashList.go +++ b/fvm/evm/handler/blockHashList.go @@ -3,6 +3,7 @@ package handler import ( "encoding/binary" "fmt" + "strings" gethCommon "github.com/onflow/go-ethereum/common" @@ -26,6 +27,14 @@ const ( heightEncodingSize ) +func IsBlockHashListBucketKeyFormat(id flow.RegisterID) bool { + return strings.HasPrefix(id.Key, "BlockHashListBucket") +} + +func IsBlockHashListMetaKey(id flow.RegisterID) bool { + return id.Key == blockHashListMetaKey +} + // BlockHashList stores the last `capacity` number of block hashes // // Under the hood it breaks the list of hashes into diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index f059874bbb9..cfa48d39f9d 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -14,6 +14,7 @@ import ( "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/events" + "github.com/onflow/flow-go/fvm/evm/handler" "github.com/onflow/flow-go/fvm/evm/offchain/blocks" evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" "github.com/onflow/flow-go/fvm/evm/offchain/sync" @@ -98,6 +99,10 @@ func OffchainReplayBackwardCompatibilityTest( continue } + if !verifiableKeys(regID) { + continue + } + // when iterating backwards, duplicated register updates are stale updates, // so skipping them if _, ok := expectedUpdates[regID]; !ok { @@ -152,7 +157,12 @@ func OffchainReplayBackwardCompatibilityTest( return err } + if !verifiableKeys(k) { + continue + } + actualUpdates[k] = v + } err = verifyRegisterUpdates(expectedUpdates, actualUpdates) @@ -166,6 +176,10 @@ func OffchainReplayBackwardCompatibilityTest( return nil } +func verifiableKeys(key flow.RegisterID) bool { + return handler.IsBlockHashListBucketKeyFormat(key) || handler.IsBlockHashListMetaKey(key) +} + func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.TransactionEventPayload, error) { var blockEvent *events.BlockEventPayload txEvents := make([]events.TransactionEventPayload, 0) From 3086e38449a9bd74b676e06ed64cbef993c6cb88 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 16:42:00 -0800 Subject: [PATCH 21/76] update verifable keys --- fvm/evm/offchain/utils/verify.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index cfa48d39f9d..a269e81ec1b 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -137,6 +137,10 @@ func OffchainReplayBackwardCompatibilityTest( return err } + if !verifiableKeys(k) { + continue + } + actualUpdates[k] = v } From be8c93ce1bf563f95db4f5d42a6dd58d5c156b10 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 20:37:34 -0800 Subject: [PATCH 22/76] skip register verification --- fvm/evm/offchain/utils/verify.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index a269e81ec1b..9a6f6a45d87 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -14,7 +14,6 @@ import ( "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/events" - "github.com/onflow/flow-go/fvm/evm/handler" "github.com/onflow/flow-go/fvm/evm/offchain/blocks" evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" "github.com/onflow/flow-go/fvm/evm/offchain/sync" @@ -58,7 +57,8 @@ func OffchainReplayBackwardCompatibilityTest( // setup account status at EVM root block if isEVMRootHeight(chainID, flowStartHeight) { - err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), environment.NewAccountStatus().ToBytes()) + err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), + environment.NewAccountStatus().ToBytes()) if err != nil { return err } @@ -166,7 +166,6 @@ func OffchainReplayBackwardCompatibilityTest( } actualUpdates[k] = v - } err = verifyRegisterUpdates(expectedUpdates, actualUpdates) @@ -181,7 +180,8 @@ func OffchainReplayBackwardCompatibilityTest( } func verifiableKeys(key flow.RegisterID) bool { - return handler.IsBlockHashListBucketKeyFormat(key) || handler.IsBlockHashListMetaKey(key) + return false + // return handler.IsBlockHashListBucketKeyFormat(key) || handler.IsBlockHashListMetaKey(key) } func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.TransactionEventPayload, error) { From 5f3fd1fbbb920507a4f012c1ea42a824b7cf3858 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 09:42:21 -0800 Subject: [PATCH 23/76] refactor verify.go --- fvm/evm/offchain/utils/verify.go | 61 +++++++++++++++++++++++--------- 1 file changed, 45 insertions(+), 16 deletions(-) diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 9a6f6a45d87..f09f392b6dd 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -26,7 +26,7 @@ import ( ) // EVM Root Height is the first block that has EVM Block Event where the EVM block height is 1 -func isEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { +func IsEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { if chainID == flow.Testnet { return flowHeight == 211176671 } else if chainID == flow.Mainnet { @@ -44,10 +44,27 @@ func OffchainReplayBackwardCompatibilityTest( results storage.ExecutionResults, executionDataStore execution_data.ExecutionDataGetter, store environment.ValueStore, + onHeightReplayed func(uint64) error, ) error { rootAddr := evm.StorageAccountAddress(chainID) rootAddrStr := string(rootAddr.Bytes()) + if IsEVMRootHeight(chainID, flowStartHeight) { + log.Info().Msgf("initializing EVM state for root height %d", flowStartHeight) + + as := environment.NewAccountStatus() + rootAddr := evm.StorageAccountAddress(chainID) + err := store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes()) + if err != nil { + return err + } + } + + // pendingEVMTxEvents are tx events that are executed block included in a flow block that + // didn't emit EVM block event, which is caused when the system tx to emit EVM block fails. + // we accumulate these pending txs, and replay them when we encounter a block with EVM block event. + pendingEVMTxEvents := make([]events.TransactionEventPayload, 0) + for height := flowStartHeight; height <= flowEndHeight; height++ { bpStorage := evmStorage.NewEphemeralStorage(store) bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) @@ -55,15 +72,6 @@ func OffchainReplayBackwardCompatibilityTest( return err } - // setup account status at EVM root block - if isEVMRootHeight(chainID, flowStartHeight) { - err = bpStorage.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), - environment.NewAccountStatus().ToBytes()) - if err != nil { - return err - } - } - blockID, err := headers.BlockIDByHeight(height) if err != nil { return err @@ -76,14 +84,14 @@ func OffchainReplayBackwardCompatibilityTest( executionData, err := executionDataStore.Get(context.Background(), result.ExecutionDataID) if err != nil { - return err + return fmt.Errorf("could not get execution data %v for block %d: %w", result.ExecutionDataID, height, err) } - events := flow.EventsList{} + evts := flow.EventsList{} payloads := []*ledger.Payload{} for _, chunkData := range executionData.ChunkExecutionDatas { - events = append(events, chunkData.Events...) + evts = append(evts, chunkData.Events...) payloads = append(payloads, chunkData.TrieUpdate.Payloads...) } @@ -111,11 +119,29 @@ func OffchainReplayBackwardCompatibilityTest( } // parse EVM events - evmBlockEvent, evmTxEvents, err := parseEVMEvents(events) + evmBlockEvent, evmTxEvents, err := parseEVMEvents(evts) if err != nil { return err } + pendingEVMTxEvents = append(pendingEVMTxEvents, evmTxEvents...) + + if evmBlockEvent == nil { + log.Info().Msgf("block has no EVM block, height :%v, txEvents: %v", height, len(evmTxEvents)) + + err = onHeightReplayed(height) + if err != nil { + return err + } + continue + } + + // when we encounter a block with EVM block event, we replay the pending txs accumulated + // from previous blocks that had no EVM block event. + evmTxEventsIncludedInBlock := pendingEVMTxEvents + // reset pendingEVMTxEvents + pendingEVMTxEvents = make([]events.TransactionEventPayload, 0) + err = bp.OnBlockReceived(evmBlockEvent) if err != nil { return err @@ -123,7 +149,7 @@ func OffchainReplayBackwardCompatibilityTest( sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) - res, results, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) + res, results, err := cr.ReplayBlock(evmTxEventsIncludedInBlock, evmBlockEvent) if err != nil { return err } @@ -173,7 +199,10 @@ func OffchainReplayBackwardCompatibilityTest( return err } - log.Info().Msgf("verified block %d", height) + err = onHeightReplayed(height) + if err != nil { + return err + } } return nil From 4fb8f6e62fc1c4140d4e33009f2344dcab8c6504 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 11:24:31 -0800 Subject: [PATCH 24/76] refactor block replay verification --- .../cmd/verify-evm-offchain-replay/verify.go | 1 + fvm/evm/offchain/utils/replay.go | 101 +++++++ fvm/evm/offchain/utils/verify.go | 246 +++++++++--------- 3 files changed, 220 insertions(+), 128 deletions(-) create mode 100644 fvm/evm/offchain/utils/replay.go diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go index f75cd8278b6..b7bb0ab0e87 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/verify.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -77,6 +77,7 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da storages.Results, executionDataStore, store, + func(uint64) error { return nil }, ) if err != nil { diff --git a/fvm/evm/offchain/utils/replay.go b/fvm/evm/offchain/utils/replay.go new file mode 100644 index 00000000000..1c556f82d19 --- /dev/null +++ b/fvm/evm/offchain/utils/replay.go @@ -0,0 +1,101 @@ +package utils + +import ( + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm/events" + "github.com/onflow/flow-go/fvm/evm/offchain/blocks" + evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" + "github.com/onflow/flow-go/fvm/evm/offchain/sync" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/model/flow" +) + +func ReplayEVMEventsToStore( + log zerolog.Logger, + store environment.ValueStore, + chainID flow.ChainID, + rootAddr flow.Address, + evmBlockEvent *events.BlockEventPayload, // EVM block event + evmTxEvents []events.TransactionEventPayload, // EVM transaction event +) ( + map[flow.RegisterID]flow.RegisterValue, // EVM state transition updates + map[flow.RegisterID]flow.RegisterValue, // block provider updates + error, +) { + + bpStorage := evmStorage.NewEphemeralStorage(store) + bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) + if err != nil { + return nil, nil, err + } + + err = bp.OnBlockReceived(evmBlockEvent) + if err != nil { + return nil, nil, err + } + + sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) + cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) + res, results, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) + if err != nil { + return nil, nil, err + } + + // commit all register changes from the EVM state transition + for k, v := range res.StorageRegisterUpdates() { + err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) + if err != nil { + return nil, nil, err + } + } + + blockProposal := blocks.ReconstructProposal(evmBlockEvent, results) + + err = bp.OnBlockExecuted(evmBlockEvent.Height, res, blockProposal) + if err != nil { + return nil, nil, err + } + + // commit all register changes from non-EVM state transition, such + // as block hash list changes + for k, v := range bpStorage.StorageRegisterUpdates() { + // verify the block hash list changes are included in the trie update + + err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) + if err != nil { + return nil, nil, err + } + } + + return res.StorageRegisterUpdates(), bpStorage.StorageRegisterUpdates(), nil +} + +type EVMEventsAccumulator struct { + pendingEVMTxEvents []events.TransactionEventPayload +} + +func NewEVMEventsAccumulator() *EVMEventsAccumulator { + return &EVMEventsAccumulator{ + pendingEVMTxEvents: make([]events.TransactionEventPayload, 0), + } +} + +func (a *EVMEventsAccumulator) HasBlockEvent( + evmBlockEvent *events.BlockEventPayload, + evmTxEvents []events.TransactionEventPayload) ( + *events.BlockEventPayload, + []events.TransactionEventPayload, + bool, // true if there is an EVM block event +) { + a.pendingEVMTxEvents = append(a.pendingEVMTxEvents, evmTxEvents...) + + // if there is no EVM block event, we will accumulate the pending txs + if evmBlockEvent == nil { + return evmBlockEvent, a.pendingEVMTxEvents, false + } + + // if there is an EVM block event, we return the EVM block and the accumulated tx events + return evmBlockEvent, a.pendingEVMTxEvents, true +} diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index f09f392b6dd..c007e4976bc 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -14,10 +14,6 @@ import ( "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/events" - "github.com/onflow/flow-go/fvm/evm/offchain/blocks" - evmStorage "github.com/onflow/flow-go/fvm/evm/offchain/storage" - "github.com/onflow/flow-go/fvm/evm/offchain/sync" - "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/model/flow" @@ -35,6 +31,24 @@ func IsEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { return flowHeight == 1 } +// IsSporkHeight returns true if the given flow height is a spork height for the given chainID +// At spork height, there is no EVM events +func IsSporkHeight(chainID flow.ChainID, flowHeight uint64) bool { + if IsEVMRootHeight(chainID, flowHeight) { + return true + } + + if chainID == flow.Testnet { + return flowHeight == 218215349 // Testnet 52 + } else if chainID == flow.Mainnet { + return flowHeight == 88226267 // Mainnet 26 + } + return false +} + +// OffchainReplayBackwardCompatibilityTest replays the offchain EVM state transition for a given range of flow blocks, +// the replay will also verify the StateUpdateChecksum of the EVM state transition from each transaction execution. +// the updated register values will be saved to the given value store. func OffchainReplayBackwardCompatibilityTest( log zerolog.Logger, chainID flow.ChainID, @@ -49,84 +63,40 @@ func OffchainReplayBackwardCompatibilityTest( rootAddr := evm.StorageAccountAddress(chainID) rootAddrStr := string(rootAddr.Bytes()) - if IsEVMRootHeight(chainID, flowStartHeight) { - log.Info().Msgf("initializing EVM state for root height %d", flowStartHeight) - - as := environment.NewAccountStatus() - rootAddr := evm.StorageAccountAddress(chainID) - err := store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes()) - if err != nil { - return err - } - } - // pendingEVMTxEvents are tx events that are executed block included in a flow block that // didn't emit EVM block event, which is caused when the system tx to emit EVM block fails. // we accumulate these pending txs, and replay them when we encounter a block with EVM block event. - pendingEVMTxEvents := make([]events.TransactionEventPayload, 0) + pendingEVMEvents := NewEVMEventsAccumulator() for height := flowStartHeight; height <= flowEndHeight; height++ { - bpStorage := evmStorage.NewEphemeralStorage(store) - bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) - if err != nil { - return err - } - - blockID, err := headers.BlockIDByHeight(height) - if err != nil { - return err - } - - result, err := results.ByBlockID(blockID) - if err != nil { - return err - } - - executionData, err := executionDataStore.Get(context.Background(), result.ExecutionDataID) - if err != nil { - return fmt.Errorf("could not get execution data %v for block %d: %w", result.ExecutionDataID, height, err) - } - - evts := flow.EventsList{} - payloads := []*ledger.Payload{} - - for _, chunkData := range executionData.ChunkExecutionDatas { - evts = append(evts, chunkData.Events...) - payloads = append(payloads, chunkData.TrieUpdate.Payloads...) - } + // account status initialization for the root account at the EVM root height + if IsEVMRootHeight(chainID, height) { + log.Info().Msgf("initializing EVM state for root height %d", flowStartHeight) - expectedUpdates := make(map[flow.RegisterID]flow.RegisterValue, len(payloads)) - for i := len(payloads) - 1; i >= 0; i-- { - regID, regVal, err := convert.PayloadToRegister(payloads[i]) + as := environment.NewAccountStatus() + rootAddr := evm.StorageAccountAddress(chainID) + err := store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes()) if err != nil { return err } + } - // skip non-evm-account registers - if regID.Owner != rootAddrStr { - continue - } - - if !verifiableKeys(regID) { - continue - } - - // when iterating backwards, duplicated register updates are stale updates, - // so skipping them - if _, ok := expectedUpdates[regID]; !ok { - expectedUpdates[regID] = regVal - } + if IsSporkHeight(chainID, height) { + // spork root block has no EVM events + continue } - // parse EVM events - evmBlockEvent, evmTxEvents, err := parseEVMEvents(evts) + // get EVM events and register updates at the flow height + evmBlockEvent, evmTxEvents, registerUpdates, err := evmEventsAndRegisterUpdatesAtFlowHeight( + height, + headers, results, executionDataStore, rootAddrStr) if err != nil { - return err + return fmt.Errorf("failed to get EVM events and register updates at height %d: %w", height, err) } - pendingEVMTxEvents = append(pendingEVMTxEvents, evmTxEvents...) + blockEvent, txEvents, hasBlockEvent := pendingEVMEvents.HasBlockEvent(evmBlockEvent, evmTxEvents) - if evmBlockEvent == nil { + if !hasBlockEvent { log.Info().Msgf("block has no EVM block, height :%v, txEvents: %v", height, len(evmTxEvents)) err = onHeightReplayed(height) @@ -136,65 +106,19 @@ func OffchainReplayBackwardCompatibilityTest( continue } - // when we encounter a block with EVM block event, we replay the pending txs accumulated - // from previous blocks that had no EVM block event. - evmTxEventsIncludedInBlock := pendingEVMTxEvents - // reset pendingEVMTxEvents - pendingEVMTxEvents = make([]events.TransactionEventPayload, 0) - - err = bp.OnBlockReceived(evmBlockEvent) - if err != nil { - return err - } - - sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) - cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) - res, results, err := cr.ReplayBlock(evmTxEventsIncludedInBlock, evmBlockEvent) - if err != nil { - return err - } - - actualUpdates := make(map[flow.RegisterID]flow.RegisterValue, len(expectedUpdates)) - - // commit all register changes from the EVM state transition - for k, v := range res.StorageRegisterUpdates() { - err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) - if err != nil { - return err - } - - if !verifiableKeys(k) { - continue - } - - actualUpdates[k] = v - } - - blockProposal := blocks.ReconstructProposal(evmBlockEvent, results) - - err = bp.OnBlockExecuted(evmBlockEvent.Height, res, blockProposal) + evmUpdates, blockProviderUpdates, err := ReplayEVMEventsToStore( + log, + store, + chainID, + rootAddr, + blockEvent, + txEvents, + ) if err != nil { - return err - } - - // commit all register changes from non-EVM state transition, such - // as block hash list changes - for k, v := range bpStorage.StorageRegisterUpdates() { - // verify the block hash list changes are included in the trie update - - err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) - if err != nil { - return err - } - - if !verifiableKeys(k) { - continue - } - - actualUpdates[k] = v + return fmt.Errorf("fail to replay events: %w", err) } - err = verifyRegisterUpdates(expectedUpdates, actualUpdates) + err = verifyEVMRegisterUpdates(registerUpdates, evmUpdates, blockProviderUpdates) if err != nil { return err } @@ -208,11 +132,6 @@ func OffchainReplayBackwardCompatibilityTest( return nil } -func verifiableKeys(key flow.RegisterID) bool { - return false - // return handler.IsBlockHashListBucketKeyFormat(key) || handler.IsBlockHashListMetaKey(key) -} - func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.TransactionEventPayload, error) { var blockEvent *events.BlockEventPayload txEvents := make([]events.TransactionEventPayload, 0) @@ -250,7 +169,78 @@ func parseEVMEvents(evts flow.EventsList) (*events.BlockEventPayload, []events.T return blockEvent, txEvents, nil } -func verifyRegisterUpdates(expectedUpdates map[flow.RegisterID]flow.RegisterValue, actualUpdates map[flow.RegisterID]flow.RegisterValue) error { +func evmEventsAndRegisterUpdatesAtFlowHeight( + flowHeight uint64, + headers storage.Headers, + results storage.ExecutionResults, + executionDataStore execution_data.ExecutionDataGetter, + rootAddr string, +) ( + *events.BlockEventPayload, // EVM block event, might be nil if there is no block Event at this height + []events.TransactionEventPayload, // EVM transaction event + map[flow.RegisterID]flow.RegisterValue, // update registers + error, +) { + + blockID, err := headers.BlockIDByHeight(flowHeight) + if err != nil { + return nil, nil, nil, err + } + + result, err := results.ByBlockID(blockID) + if err != nil { + return nil, nil, nil, err + } + + executionData, err := executionDataStore.Get(context.Background(), result.ExecutionDataID) + if err != nil { + return nil, nil, nil, + fmt.Errorf("could not get execution data %v for block %d: %w", + result.ExecutionDataID, flowHeight, err) + } + + evts := flow.EventsList{} + payloads := []*ledger.Payload{} + + for _, chunkData := range executionData.ChunkExecutionDatas { + evts = append(evts, chunkData.Events...) + payloads = append(payloads, chunkData.TrieUpdate.Payloads...) + } + + updates := make(map[flow.RegisterID]flow.RegisterValue, len(payloads)) + for i := len(payloads) - 1; i >= 0; i-- { + regID, regVal, err := convert.PayloadToRegister(payloads[i]) + if err != nil { + return nil, nil, nil, err + } + + // find the register updates for the root account + if regID.Owner == rootAddr { + updates[regID] = regVal + } + } + + // parse EVM events + evmBlockEvent, evmTxEvents, err := parseEVMEvents(evts) + if err != nil { + return nil, nil, nil, err + } + return evmBlockEvent, evmTxEvents, updates, nil +} + +func verifyEVMRegisterUpdates( + registerUpdates map[flow.RegisterID]flow.RegisterValue, + evmUpdates map[flow.RegisterID]flow.RegisterValue, + blockProviderUpdates map[flow.RegisterID]flow.RegisterValue, +) error { + // skip the register level validation + // since the register is not stored at the same slab id as the on-chain EVM + // instead, we will compare by exporting the logic EVM state, which contains + // accounts, codes and slots. + return nil +} + +func VerifyRegisterUpdates(expectedUpdates map[flow.RegisterID]flow.RegisterValue, actualUpdates map[flow.RegisterID]flow.RegisterValue) error { missingUpdates := make(map[flow.RegisterID]flow.RegisterValue) additionalUpdates := make(map[flow.RegisterID]flow.RegisterValue) mismatchingUpdates := make(map[flow.RegisterID][2]flow.RegisterValue) From a69b60134bfe92a59b4e05951f74f51c20021ee9 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 11:34:55 -0800 Subject: [PATCH 25/76] refactor util tests --- fvm/evm/offchain/utils/collection_test.go | 42 ++++++----------------- 1 file changed, 10 insertions(+), 32 deletions(-) diff --git a/fvm/evm/offchain/utils/collection_test.go b/fvm/evm/offchain/utils/collection_test.go index 8e292530534..5dad9b86658 100644 --- a/fvm/evm/offchain/utils/collection_test.go +++ b/fvm/evm/offchain/utils/collection_test.go @@ -10,7 +10,6 @@ import ( "strings" "testing" - "github.com/rs/zerolog" "github.com/rs/zerolog/log" "github.com/stretchr/testify/require" @@ -20,9 +19,6 @@ import ( "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/events" - "github.com/onflow/flow-go/fvm/evm/offchain/blocks" - "github.com/onflow/flow-go/fvm/evm/offchain/storage" - "github.com/onflow/flow-go/fvm/evm/offchain/sync" "github.com/onflow/flow-go/fvm/evm/offchain/utils" . "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/model/flow" @@ -128,10 +124,6 @@ func replayEvents( rootAddr := evm.StorageAccountAddress(chainID) - bpStorage := storage.NewEphemeralStorage(store) - bp, err := blocks.NewBasicProvider(chainID, bpStorage, rootAddr) - require.NoError(t, err) - nextHeight := initialNextHeight scanEventFilesAndRun(t, eventsFilePath, @@ -142,31 +134,17 @@ func replayEvents( nextHeight, blockEventPayload.Height) } - err = bp.OnBlockReceived(blockEventPayload) - require.NoError(t, err) - - sp := NewTestStorageProvider(store, blockEventPayload.Height) - cr := sync.NewReplayer(chainID, rootAddr, sp, bp, zerolog.Logger{}, nil, true) - res, results, err := cr.ReplayBlock(txEvents, blockEventPayload) - require.NoError(t, err) - - // commit all changes - for k, v := range res.StorageRegisterUpdates() { - err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) - require.NoError(t, err) - } - - proposal := blocks.ReconstructProposal(blockEventPayload, results) - - err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) - require.NoError(t, err) - - // commit all block hash list changes - for k, v := range bpStorage.StorageRegisterUpdates() { - err = store.SetValue([]byte(k.Owner), []byte(k.Key), v) - require.NoError(t, err) + _, _, err := utils.ReplayEVMEventsToStore( + log.Logger, + store, + chainID, + rootAddr, + blockEventPayload, + txEvents, + ) + if err != nil { + return fmt.Errorf("fail to replay events: %w", err) } - // verify the block height is sequential without gap nextHeight++ From a41e107138ff30e013895d097493192ba04ee642 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 15:30:43 -0800 Subject: [PATCH 26/76] remove unused chainID arg --- cmd/util/cmd/verify-evm-offchain-replay/verify.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go index b7bb0ab0e87..d3e0b5c6d1d 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/verify.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -31,7 +31,7 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da Str("evmStateGobDir", evmStateGobDir). Msgf("verifying range from %d to %d", from, to) - db, storages, executionDataStore, dsStore, err := initStorages(chainID, dataDir, executionDataDir) + db, storages, executionDataStore, dsStore, err := initStorages(dataDir, executionDataDir) if err != nil { return fmt.Errorf("could not initialize storages: %w", err) } @@ -100,7 +100,7 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da return nil } -func initStorages(chainID flow.ChainID, dataDir string, executionDataDir string) ( +func initStorages(dataDir string, executionDataDir string) ( *badger.DB, *storage.All, execution_data.ExecutionDataGetter, From 4677d9702c0ebe7963ff356ef9f320d4161b705b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 16:53:13 -0800 Subject: [PATCH 27/76] refactor verify --- .../cmd/verify-evm-offchain-replay/main.go | 13 +-- .../cmd/verify-evm-offchain-replay/verify.go | 100 +++++++++++------- 2 files changed, 71 insertions(+), 42 deletions(-) diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go index 0bc6eef8187..d1027fb8a74 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/main.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -17,11 +17,12 @@ var ( flagEVMStateGobDir string flagChain string flagFromTo string + flagSaveEveryNBlocks uint64 ) // usage example // -// ./util verify-evm-offchain-replay --chain flow-testnet --from-to 211176671-211177000 +// ./util verify-evm-offchain-replay --chain flow-testnet --from_to 211176671-211177000 // --datadir /var/flow/data/protocol --execution_data_dir /var/flow/data/execution_data var Cmd = &cobra.Command{ Use: "verify-evm-offchain-replay", @@ -44,23 +45,23 @@ func init() { Cmd.Flags().StringVar(&flagEVMStateGobDir, "evm_state_gob_dir", "/var/flow/data/evm_state_gob", "directory that stores the evm state gob files as checkpoint") + + Cmd.Flags().Uint64Var(&flagSaveEveryNBlocks, "save_every", uint64(1_000_000), + "save the evm state gob files every N blocks") } func run(*cobra.Command, []string) { - _ = flow.ChainID(flagChain).Chain() + chainID := flow.ChainID(flagChain) from, to, err := parseFromTo(flagFromTo) if err != nil { log.Fatal().Err(err).Msg("could not parse from_to") } - log.Info().Msgf("verifying range from %d to %d", from, to) - err = Verify(log.Logger, from, to, flow.Testnet, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir) + err = Verify(log.Logger, from, to, chainID, flagDatadir, flagExecutionDataDir, flagEVMStateGobDir, flagSaveEveryNBlocks) if err != nil { log.Fatal().Err(err).Msg("could not verify height") } - log.Info().Msgf("successfully verified range from %d to %d", from, to) - } func parseFromTo(fromTo string) (from, to uint64, err error) { diff --git a/cmd/util/cmd/verify-evm-offchain-replay/verify.go b/cmd/util/cmd/verify-evm-offchain-replay/verify.go index d3e0b5c6d1d..47b34c72afa 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/verify.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/verify.go @@ -9,10 +9,9 @@ import ( "github.com/dgraph-io/badger/v2" badgerds "github.com/ipfs/go-ds-badger2" "github.com/rs/zerolog" + "github.com/rs/zerolog/log" "github.com/onflow/flow-go/cmd/util/cmd/common" - "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/offchain/utils" "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/model/flow" @@ -23,13 +22,26 @@ import ( // Verify verifies the offchain replay of EVM blocks from the given height range // and updates the EVM state gob files with the latest state -func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, dataDir string, executionDataDir string, evmStateGobDir string) error { - log.Info(). +func Verify( + log zerolog.Logger, + from uint64, + to uint64, + chainID flow.ChainID, + dataDir string, + executionDataDir string, + evmStateGobDir string, + saveEveryNBlocks uint64, +) error { + lg := log.With(). + Uint64("from", from).Uint64("to", to). Str("chain", chainID.String()). Str("dataDir", dataDir). Str("executionDataDir", executionDataDir). Str("evmStateGobDir", evmStateGobDir). - Msgf("verifying range from %d to %d", from, to) + Uint64("saveEveryNBlocks", saveEveryNBlocks). + Logger() + + lg.Info().Msgf("verifying range from %d to %d", from, to) db, storages, executionDataStore, dsStore, err := initStorages(dataDir, executionDataDir) if err != nil { @@ -40,34 +52,32 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da defer dsStore.Close() var store *testutils.TestValueStore - isRoot := isEVMRootHeight(chainID, from) - if isRoot { - log.Info().Msgf("initializing EVM state for root height %d", from) + // root block require the account status registers to be saved + isRoot := utils.IsEVMRootHeight(chainID, from) + if isRoot { store = testutils.GetSimpleValueStore() - as := environment.NewAccountStatus() - rootAddr := evm.StorageAccountAddress(chainID) - err = store.SetValue(rootAddr[:], []byte(flow.AccountStatusKey), as.ToBytes()) - if err != nil { - return err - } } else { prev := from - 1 - log.Info().Msgf("loading EVM state from previous height %d", prev) - - valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, prev) - values, err := testutils.DeserializeState(valueFileName) + store, err = loadState(prev, evmStateGobDir) if err != nil { - return fmt.Errorf("could not deserialize state %v: %w", valueFileName, err) + return fmt.Errorf("could not load EVM state from previous height %d: %w", prev, err) } + } - allocators, err := testutils.DeserializeAllocator(allocatorFileName) - if err != nil { - return fmt.Errorf("could not deserialize allocator %v: %w", allocatorFileName, err) + // save state every N blocks + onHeightReplayed := func(height uint64) error { + log.Info().Msgf("replayed height %d", height) + if height%saveEveryNBlocks == 0 { + err := saveState(store, height, evmStateGobDir) + if err != nil { + return err + } } - store = testutils.GetSimpleValueStorePopulated(values, allocators) + return nil } + // replay blocks err = utils.OffchainReplayBackwardCompatibilityTest( log, chainID, @@ -77,16 +87,27 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da storages.Results, executionDataStore, store, - func(uint64) error { return nil }, + onHeightReplayed, ) if err != nil { return err } - valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, to) + err = saveState(store, to, evmStateGobDir) + if err != nil { + return err + } + + lg.Info().Msgf("successfully verified range from %d to %d", from, to) + + return nil +} + +func saveState(store *testutils.TestValueStore, height uint64, gobDir string) error { + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(gobDir, height) values, allocators := store.Dump() - err = testutils.SerializeState(valueFileName, values) + err := testutils.SerializeState(valueFileName, values) if err != nil { return err } @@ -100,6 +121,23 @@ func Verify(log zerolog.Logger, from uint64, to uint64, chainID flow.ChainID, da return nil } +func loadState(height uint64, gobDir string) (*testutils.TestValueStore, error) { + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(gobDir, height) + values, err := testutils.DeserializeState(valueFileName) + if err != nil { + return nil, fmt.Errorf("could not deserialize state %v: %w", valueFileName, err) + } + + allocators, err := testutils.DeserializeAllocator(allocatorFileName) + if err != nil { + return nil, fmt.Errorf("could not deserialize allocator %v: %w", allocatorFileName, err) + } + store := testutils.GetSimpleValueStorePopulated(values, allocators) + + log.Info().Msgf("loaded EVM state for height %d from gob file %v", height, valueFileName) + return store, nil +} + func initStorages(dataDir string, executionDataDir string) ( *badger.DB, *storage.All, @@ -128,16 +166,6 @@ func initStorages(dataDir string, executionDataDir string) ( return db, storages, executionDataStore, ds, nil } -// EVM Root Height is the first block that has EVM Block Event where the EVM block height is 1 -func isEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { - if chainID == flow.Testnet { - return flowHeight == 211176671 - } else if chainID == flow.Mainnet { - return flowHeight == 85981136 - } - return flowHeight == 1 -} - func evmStateGobFileNamesByEndHeight(evmStateGobDir string, endHeight uint64) (string, string) { valueFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("values-%d.gob", endHeight)) allocatorFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("allocators-%d.gob", endHeight)) From bf4d4294d8b484a7d8124295f65c20c47dc798e9 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 22:50:17 -0800 Subject: [PATCH 28/76] fix spork heights --- cmd/util/cmd/verify-evm-offchain-replay/main.go | 2 +- fvm/evm/offchain/utils/verify.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/util/cmd/verify-evm-offchain-replay/main.go b/cmd/util/cmd/verify-evm-offchain-replay/main.go index d1027fb8a74..d42c9841435 100644 --- a/cmd/util/cmd/verify-evm-offchain-replay/main.go +++ b/cmd/util/cmd/verify-evm-offchain-replay/main.go @@ -22,7 +22,7 @@ var ( // usage example // -// ./util verify-evm-offchain-replay --chain flow-testnet --from_to 211176671-211177000 +// ./util verify-evm-offchain-replay --chain flow-testnet --from_to 211176670-211177000 // --datadir /var/flow/data/protocol --execution_data_dir /var/flow/data/execution_data var Cmd = &cobra.Command{ Use: "verify-evm-offchain-replay", diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index c007e4976bc..3cfe410f315 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -24,9 +24,9 @@ import ( // EVM Root Height is the first block that has EVM Block Event where the EVM block height is 1 func IsEVMRootHeight(chainID flow.ChainID, flowHeight uint64) bool { if chainID == flow.Testnet { - return flowHeight == 211176671 + return flowHeight == 211176670 } else if chainID == flow.Mainnet { - return flowHeight == 85981136 + return flowHeight == 85981135 } return flowHeight == 1 } From b82fc5e0d4b0e513b00e46352f2695b9adf7d019 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 23:27:33 -0800 Subject: [PATCH 29/76] fix pendingEVMTxEvents --- fvm/evm/offchain/utils/replay.go | 5 ++++- fvm/evm/offchain/utils/verify.go | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/fvm/evm/offchain/utils/replay.go b/fvm/evm/offchain/utils/replay.go index 1c556f82d19..d6cb222fa73 100644 --- a/fvm/evm/offchain/utils/replay.go +++ b/fvm/evm/offchain/utils/replay.go @@ -96,6 +96,9 @@ func (a *EVMEventsAccumulator) HasBlockEvent( return evmBlockEvent, a.pendingEVMTxEvents, false } + pendingEVMTxEvents := a.pendingEVMTxEvents + // reset pending events + a.pendingEVMTxEvents = make([]events.TransactionEventPayload, 0) // if there is an EVM block event, we return the EVM block and the accumulated tx events - return evmBlockEvent, a.pendingEVMTxEvents, true + return evmBlockEvent, pendingEVMTxEvents, true } diff --git a/fvm/evm/offchain/utils/verify.go b/fvm/evm/offchain/utils/verify.go index 3cfe410f315..9335beb6230 100644 --- a/fvm/evm/offchain/utils/verify.go +++ b/fvm/evm/offchain/utils/verify.go @@ -79,6 +79,8 @@ func OffchainReplayBackwardCompatibilityTest( if err != nil { return err } + + continue } if IsSporkHeight(chainID, height) { From 556d9036b2540911d1f5578f461329a82e3f8f2b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 06:43:35 -0800 Subject: [PATCH 30/76] keep ReplayBlock unchanged --- fvm/evm/offchain/sync/replayer.go | 11 ++++++++++- fvm/evm/offchain/sync/replayer_test.go | 8 +------- fvm/evm/offchain/utils/replay.go | 2 +- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/fvm/evm/offchain/sync/replayer.go b/fvm/evm/offchain/sync/replayer.go index 33411b7c133..96df01d58a0 100644 --- a/fvm/evm/offchain/sync/replayer.go +++ b/fvm/evm/offchain/sync/replayer.go @@ -45,6 +45,15 @@ func NewReplayer( } // ReplayBlock replays the execution of the transactions of an EVM block +func (cr *Replayer) ReplayBlock( + transactionEvents []events.TransactionEventPayload, + blockEvent *events.BlockEventPayload, +) (types.ReplayResultCollector, error) { + res, _, err := cr.ReplayBlockEvents(transactionEvents, blockEvent) + return res, err +} + +// ReplayBlockEvents replays the execution of the transactions of an EVM block // using the provided transactionEvents and blockEvents, // which include all the context data for re-executing the transactions, and returns // the replay result and the result of each transaction. @@ -57,7 +66,7 @@ func NewReplayer( // Warning! the list of transaction events has to be sorted based on their // execution, sometimes the access node might return events out of order // it needs to be sorted by txIndex and eventIndex respectively. -func (cr *Replayer) ReplayBlock( +func (cr *Replayer) ReplayBlockEvents( transactionEvents []events.TransactionEventPayload, blockEvent *events.BlockEventPayload, ) (types.ReplayResultCollector, []*types.Result, error) { diff --git a/fvm/evm/offchain/sync/replayer_test.go b/fvm/evm/offchain/sync/replayer_test.go index 3668e445c84..06262b5811e 100644 --- a/fvm/evm/offchain/sync/replayer_test.go +++ b/fvm/evm/offchain/sync/replayer_test.go @@ -164,7 +164,7 @@ func TestChainReplay(t *testing.T) { sp := NewTestStorageProvider(snapshot, 1) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, zerolog.Logger{}, nil, true) - res, results, err := cr.ReplayBlock(txEventPayloads, blockEventPayload) + res, results, err := cr.ReplayBlockEvents(txEventPayloads, blockEventPayload) require.NoError(t, err) require.Len(t, results, totalTxCount) @@ -173,12 +173,6 @@ func TestChainReplay(t *testing.T) { err = bp.OnBlockExecuted(blockEventPayload.Height, res, proposal) require.NoError(t, err) - - // for k, v := range bpStorage.StorageRegisterUpdates() { - // ret, err := backend.GetValue([]byte(k.Owner), []byte(k.Key)) - // require.NoError(t, err) - // require.Equal(t, ret[:], v[:]) - // } }) }) }) diff --git a/fvm/evm/offchain/utils/replay.go b/fvm/evm/offchain/utils/replay.go index d6cb222fa73..5aba8affcd1 100644 --- a/fvm/evm/offchain/utils/replay.go +++ b/fvm/evm/offchain/utils/replay.go @@ -38,7 +38,7 @@ func ReplayEVMEventsToStore( sp := testutils.NewTestStorageProvider(store, evmBlockEvent.Height) cr := sync.NewReplayer(chainID, rootAddr, sp, bp, log, nil, true) - res, results, err := cr.ReplayBlock(evmTxEvents, evmBlockEvent) + res, results, err := cr.ReplayBlockEvents(evmTxEvents, evmBlockEvent) if err != nil { return nil, nil, err } From 2af3c9eb8760d7ace3f9378d860c0ddf9ca56c4a Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 23:12:55 -0800 Subject: [PATCH 31/76] from gobs --- cmd/util/cmd/export-evm-state/cmd.go | 67 +++++++++++++++++++++++++--- 1 file changed, 62 insertions(+), 5 deletions(-) diff --git a/cmd/util/cmd/export-evm-state/cmd.go b/cmd/util/cmd/export-evm-state/cmd.go index 2927b9a313a..985c7d1b644 100644 --- a/cmd/util/cmd/export-evm-state/cmd.go +++ b/cmd/util/cmd/export-evm-state/cmd.go @@ -3,13 +3,17 @@ package evm_exporter import ( "fmt" "os" + "path/filepath" "github.com/rs/zerolog/log" "github.com/spf13/cobra" + "github.com/onflow/atree" + "github.com/onflow/flow-go/cmd/util/ledger/util" "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/model/flow" @@ -20,6 +24,8 @@ var ( flagExecutionStateDir string flagOutputDir string flagStateCommitment string + flagEVMStateGobDir string + flagEVMStateGobHeight uint64 ) var Cmd = &cobra.Command{ @@ -34,7 +40,6 @@ func init() { Cmd.Flags().StringVar(&flagExecutionStateDir, "execution-state-dir", "", "Execution Node state dir (where WAL logs are written") - _ = Cmd.MarkFlagRequired("execution-state-dir") Cmd.Flags().StringVar(&flagOutputDir, "output-dir", "", "Directory to write new Execution State to") @@ -42,13 +47,26 @@ func init() { Cmd.Flags().StringVar(&flagStateCommitment, "state-commitment", "", "State commitment (hex-encoded, 64 characters)") + + Cmd.Flags().StringVar(&flagEVMStateGobDir, "evm_state_gob_dir", "/var/flow/data/evm_state_gob", + "directory that stores the evm state gob files as checkpoint") + + Cmd.Flags().Uint64Var(&flagEVMStateGobHeight, "evm_state_gob_height", 0, + "the flow height of the evm state gob files") } func run(*cobra.Command, []string) { log.Info().Msg("start exporting evm state") - err := ExportEVMState(flagChain, flagExecutionStateDir, flagStateCommitment, flagOutputDir) - if err != nil { - log.Fatal().Err(err).Msg("cannot get export evm state") + if flagExecutionStateDir != "" { + err := ExportEVMState(flagChain, flagExecutionStateDir, flagStateCommitment, flagOutputDir) + if err != nil { + log.Fatal().Err(err).Msg("cannot get export evm state") + } + } else if flagEVMStateGobDir != "" { + err := ExportEVMStateFromGob(flagChain, flagEVMStateGobDir, flagEVMStateGobHeight, flagOutputDir) + if err != nil { + log.Fatal().Err(err).Msg("cannot get export evm state from gob files") + } } } @@ -83,7 +101,40 @@ func ExportEVMState( payloadsLedger := util.NewPayloadsLedger(filteredPayloads) - exporter, err := state.NewExporter(payloadsLedger, storageRoot) + return ExportEVMStateFromPayloads(payloadsLedger, storageRoot, outputPath) +} + +func ExportEVMStateFromGob( + chainName string, + evmStateGobDir string, + flowHeight uint64, + outputPath string) error { + + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(evmStateGobDir, flowHeight) + chainID := flow.ChainID(chainName) + + storageRoot := evm.StorageAccountAddress(chainID) + valuesGob, err := testutils.DeserializeState(valueFileName) + if err != nil { + return err + } + + allocatorGobs, err := testutils.DeserializeAllocator(allocatorFileName) + if err != nil { + return err + } + + store := testutils.GetSimpleValueStorePopulated(valuesGob, allocatorGobs) + + return ExportEVMStateFromPayloads(store, storageRoot, outputPath) +} + +func ExportEVMStateFromPayloads( + ledger atree.Ledger, + storageRoot flow.Address, + outputPath string, +) error { + exporter, err := state.NewExporter(ledger, storageRoot) if err != nil { return fmt.Errorf("failed to create exporter: %w", err) } @@ -107,3 +158,9 @@ func ExportEVMState( } return nil } + +func evmStateGobFileNamesByEndHeight(evmStateGobDir string, endHeight uint64) (string, string) { + valueFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("values-%d.gob", endHeight)) + allocatorFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("allocators-%d.gob", endHeight)) + return valueFileName, allocatorFileName +} From d8751bd7d5a84c703a423d1190e31a081c398f49 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 23:16:34 -0800 Subject: [PATCH 32/76] fix folder exists issue --- cmd/util/cmd/export-evm-state/cmd.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/cmd/util/cmd/export-evm-state/cmd.go b/cmd/util/cmd/export-evm-state/cmd.go index 985c7d1b644..6cb05da37ba 100644 --- a/cmd/util/cmd/export-evm-state/cmd.go +++ b/cmd/util/cmd/export-evm-state/cmd.go @@ -146,12 +146,6 @@ func ExportEVMStateFromPayloads( } } - fi, err := os.Create(outputPath) - if err != nil { - return err - } - defer fi.Close() - err = exporter.Export(outputPath) if err != nil { return fmt.Errorf("failed to export: %w", err) From 066c4e2c1bdb40afda5fc35cb0156e466c4db01f Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Fri, 22 Nov 2024 23:17:54 -0800 Subject: [PATCH 33/76] fix creating files --- fvm/evm/emulator/state/exporter.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fvm/evm/emulator/state/exporter.go b/fvm/evm/emulator/state/exporter.go index 49f3a0fdbd8..595c451b672 100644 --- a/fvm/evm/emulator/state/exporter.go +++ b/fvm/evm/emulator/state/exporter.go @@ -37,7 +37,7 @@ func NewExporter(ledger atree.Ledger, root flow.Address) (*Exporter, error) { } func (e *Exporter) Export(path string) error { - af, err := os.OpenFile(filepath.Join(path, ExportedAccountsFileName), os.O_RDWR, 0644) + af, err := os.Create(filepath.Join(path, ExportedAccountsFileName)) if err != nil { return err } @@ -48,7 +48,7 @@ func (e *Exporter) Export(path string) error { return err } - cf, err := os.OpenFile(filepath.Join(path, ExportedCodesFileName), os.O_RDWR, 0644) + cf, err := os.Create(filepath.Join(path, ExportedCodesFileName)) if err != nil { return err } @@ -59,7 +59,7 @@ func (e *Exporter) Export(path string) error { return err } - sf, err := os.OpenFile(filepath.Join(path, ExportedSlotsFileName), os.O_RDWR, 0644) + sf, err := os.Create(filepath.Join(path, ExportedSlotsFileName)) if err != nil { return err } From 6dbedf15c6125da2a970e433067f5293913a5a0e Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 16:42:14 -0800 Subject: [PATCH 34/76] add state diff --- fvm/evm/emulator/state/diff.go | 91 +++++++++++++++++++++ fvm/evm/emulator/state/diff_test.go | 19 +++++ fvm/evm/emulator/state/importer.go | 121 ++++++++++++++++++++++++++++ 3 files changed, 231 insertions(+) create mode 100644 fvm/evm/emulator/state/diff.go create mode 100644 fvm/evm/emulator/state/diff_test.go create mode 100644 fvm/evm/emulator/state/importer.go diff --git a/fvm/evm/emulator/state/diff.go b/fvm/evm/emulator/state/diff.go new file mode 100644 index 00000000000..5f79d4102b3 --- /dev/null +++ b/fvm/evm/emulator/state/diff.go @@ -0,0 +1,91 @@ +package state + +import ( + "bytes" + "fmt" +) + +func AccountEqual(a, b *Account) bool { + if a.Address != b.Address { + return false + } + if !bytes.Equal(a.Balance.Bytes(), b.Balance.Bytes()) { + return false + } + if a.Nonce != b.Nonce { + return false + } + if a.CodeHash != b.CodeHash { + return false + } + + // CollectionID could be different + return true +} + +// find the difference and return as error +func Diff(a *EVMState, b *EVMState) []error { + var differences []error + + // Compare Accounts + for addr, accA := range a.Accounts { + if accB, exists := b.Accounts[addr]; exists { + if !AccountEqual(accA, accB) { + differences = append(differences, fmt.Errorf("account %s differs", addr.Hex())) + } + } else { + differences = append(differences, fmt.Errorf("account %s exists in a but not in b", addr.Hex())) + } + } + for addr := range b.Accounts { + if _, exists := a.Accounts[addr]; !exists { + differences = append(differences, fmt.Errorf("account %s exists in b but not in a", addr.Hex())) + } + } + + // Compare Slots + for addr, slotsA := range a.Slots { + slotsB, exists := b.Slots[addr] + if !exists { + differences = append(differences, fmt.Errorf("slots for address %s exist in a but not in b", addr.Hex())) + continue + } + for key, valueA := range slotsA { + if valueB, exists := slotsB[key]; exists { + if valueA.Value != valueB.Value { + differences = append(differences, fmt.Errorf("slot value for address %s and key %s differs", addr.Hex(), key.Hex())) + } + } else { + differences = append(differences, fmt.Errorf("slot with key %s for address %s exists in a but not in b", key.Hex(), addr.Hex())) + } + } + for key := range slotsB { + if _, exists := slotsA[key]; !exists { + differences = append(differences, fmt.Errorf("slot with key %s for address %s exists in b but not in a", key.Hex(), addr.Hex())) + } + } + } + for addr := range b.Slots { + if _, exists := a.Slots[addr]; !exists { + differences = append(differences, fmt.Errorf("slots for address %s exist in b but not in a", addr.Hex())) + } + } + + // Compare Codes + for hash, codeA := range a.Codes { + if codeB, exists := b.Codes[hash]; exists { + if !bytes.Equal(codeA.Code, codeB.Code) { + differences = append(differences, fmt.Errorf("code for hash %s differs", hash.Hex())) + } + } else { + differences = append(differences, fmt.Errorf("code with hash %s exists in a but not in b", hash.Hex())) + } + } + for hash := range b.Codes { + if _, exists := a.Codes[hash]; !exists { + differences = append(differences, fmt.Errorf("code with hash %s exists in b but not in a", hash.Hex())) + } + } + + return differences +} diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go new file mode 100644 index 00000000000..598a35ca7f5 --- /dev/null +++ b/fvm/evm/emulator/state/diff_test.go @@ -0,0 +1,19 @@ +package state + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestStateDiff(t *testing.T) { + enState, err := ImportEVMState("~/Downloads/compare-state/evm-state-from-checkpoint/") + require.NoError(t, err) + + offchainState, err := ImportEVMState("~/Downloads/compare-state/evm-state-from-offchain/") + require.NoError(t, err) + + differences := Diff(enState, offchainState) + + require.Len(t, differences, 0) +} diff --git a/fvm/evm/emulator/state/importer.go b/fvm/evm/emulator/state/importer.go new file mode 100644 index 00000000000..52a81f1d481 --- /dev/null +++ b/fvm/evm/emulator/state/importer.go @@ -0,0 +1,121 @@ +package state + +import ( + "bufio" + "fmt" + "os" + "path/filepath" + + gethCommon "github.com/onflow/go-ethereum/common" + + "github.com/onflow/flow-go/fvm/evm/types" +) + +type EVMState struct { + Accounts map[gethCommon.Address]*Account + Codes map[gethCommon.Hash]*CodeInContext + // account address -> key -> value + Slots map[gethCommon.Address]map[gethCommon.Hash]*types.SlotEntry +} + +func ToEVMState( + accounts map[gethCommon.Address]*Account, + codes []*CodeInContext, + slots []*types.SlotEntry, +) (*EVMState, error) { + state := &EVMState{ + Accounts: accounts, + Codes: make(map[gethCommon.Hash]*CodeInContext), + Slots: make(map[gethCommon.Address]map[gethCommon.Hash]*types.SlotEntry), + } + + // Process codes + for _, code := range codes { + if _, ok := state.Codes[code.Hash]; ok { + return nil, fmt.Errorf("duplicate code hash: %s", code.Hash) + } + state.Codes[code.Hash] = code + } + + // Process slots + for _, slot := range slots { + if _, ok := state.Slots[slot.Address]; !ok { + state.Slots[slot.Address] = make(map[gethCommon.Hash]*types.SlotEntry) + } + + if _, ok := state.Slots[slot.Address][slot.Key]; ok { + return nil, fmt.Errorf("duplicate slot key: %s", slot.Key) + } + + state.Slots[slot.Address][slot.Key] = slot + } + + return state, nil +} + +func ImportEVMState(path string) (*EVMState, error) { + accounts := make(map[gethCommon.Address]*Account) + var codes []*CodeInContext + var slots []*types.SlotEntry + + // Import accounts + accountsFile, err := os.Open(filepath.Join(path, ExportedAccountsFileName)) + if err != nil { + return nil, fmt.Errorf("error opening accounts file: %w", err) + } + defer accountsFile.Close() + + scanner := bufio.NewScanner(accountsFile) + for scanner.Scan() { + acc, err := DecodeAccount(scanner.Bytes()) + if err != nil { + return nil, fmt.Errorf("error decoding account: %w", err) + } + accounts[acc.Address] = acc + } + + if err := scanner.Err(); err != nil { + return nil, fmt.Errorf("error reading accounts file: %w", err) + } + + // Import codes + codesFile, err := os.Open(filepath.Join(path, ExportedCodesFileName)) + if err != nil { + return nil, fmt.Errorf("error opening codes file: %w", err) + } + defer codesFile.Close() + + scanner = bufio.NewScanner(codesFile) + for scanner.Scan() { + code, err := CodeInContextFromEncoded(scanner.Bytes()) + if err != nil { + return nil, fmt.Errorf("error decoding code in context: %w", err) + } + codes = append(codes, code) + } + if err := scanner.Err(); err != nil { + return nil, fmt.Errorf("error reading codes file: %w", err) + } + + // Import slots + slotsFile, err := os.Open(filepath.Join(path, ExportedSlotsFileName)) + if err != nil { + return nil, fmt.Errorf("error opening slots file: %w", err) + } + defer slotsFile.Close() + + scanner = bufio.NewScanner(slotsFile) + for scanner.Scan() { + slot, err := types.SlotEntryFromEncoded(scanner.Bytes()) + if err != nil { + return nil, fmt.Errorf("error decoding slot entry: %w", err) + } + slots = append(slots, slot) + } + + if err := scanner.Err(); err != nil { + return nil, fmt.Errorf("error reading slots file: %w", err) + } + + return ToEVMState(accounts, codes, slots) +} From d1ea26159d7d66b7e22559b1652cd3035eecb13d Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 19:20:31 -0800 Subject: [PATCH 35/76] debug --- fvm/evm/emulator/state/diff_test.go | 4 +-- fvm/evm/emulator/state/exporter.go | 20 +++++++++++++ fvm/evm/emulator/state/importer.go | 44 +++++++++++++++-------------- 3 files changed, 45 insertions(+), 23 deletions(-) diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 598a35ca7f5..1a1a3161af4 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -7,10 +7,10 @@ import ( ) func TestStateDiff(t *testing.T) { - enState, err := ImportEVMState("~/Downloads/compare-state/evm-state-from-checkpoint/") + offchainState, err := ImportEVMState("/Users/leozhang/Downloads/compare-state/evm-state-from-gobs/") require.NoError(t, err) - offchainState, err := ImportEVMState("~/Downloads/compare-state/evm-state-from-offchain/") + enState, err := ImportEVMState("/Users/leozhang/Downloads/compare-state/evm-state-from-checkpoint/") require.NoError(t, err) differences := Diff(enState, offchainState) diff --git a/fvm/evm/emulator/state/exporter.go b/fvm/evm/emulator/state/exporter.go index 595c451b672..09c71af277b 100644 --- a/fvm/evm/emulator/state/exporter.go +++ b/fvm/evm/emulator/state/exporter.go @@ -1,6 +1,7 @@ package state import ( + "fmt" "io" "os" "path/filepath" @@ -8,6 +9,7 @@ import ( "github.com/onflow/atree" gethCommon "github.com/onflow/go-ethereum/common" + "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" ) @@ -96,6 +98,12 @@ func (e *Exporter) exportAccounts(writer io.Writer) ([]gethCommon.Address, error if err != nil { return nil, err } + + _, err = DecodeAccount(encoded) + if err != nil { + return nil, fmt.Errorf("account can not be decoded: %w", err) + } + // write every account on a new line _, err = writer.Write(append(encoded, byte('\n'))) if err != nil { @@ -123,6 +131,12 @@ func (e *Exporter) exportCodes(writer io.Writer) error { if err != nil { return err } + + _, err = CodeInContextFromEncoded(encoded) + if err != nil { + return fmt.Errorf("error decoding code in context: %w", err) + } + // write every codes on a new line _, err = writer.Write(append(encoded, byte('\n'))) if err != nil { @@ -151,6 +165,12 @@ func (e *Exporter) exportSlots(addresses []gethCommon.Address, writer io.Writer) if err != nil { return err } + + _, err = types.SlotEntryFromEncoded(encoded) + if err != nil { + return fmt.Errorf("error decoding slot entry: %w", err) + } + // write every codes on a new line _, err = writer.Write(append(encoded, byte('\n'))) if err != nil { diff --git a/fvm/evm/emulator/state/importer.go b/fvm/evm/emulator/state/importer.go index 52a81f1d481..2149b716616 100644 --- a/fvm/evm/emulator/state/importer.go +++ b/fvm/evm/emulator/state/importer.go @@ -58,26 +58,6 @@ func ImportEVMState(path string) (*EVMState, error) { var codes []*CodeInContext var slots []*types.SlotEntry - // Import accounts - accountsFile, err := os.Open(filepath.Join(path, ExportedAccountsFileName)) - if err != nil { - return nil, fmt.Errorf("error opening accounts file: %w", err) - } - defer accountsFile.Close() - - scanner := bufio.NewScanner(accountsFile) - for scanner.Scan() { - acc, err := DecodeAccount(scanner.Bytes()) - if err != nil { - return nil, fmt.Errorf("error decoding account: %w", err) - } - accounts[acc.Address] = acc - } - - if err := scanner.Err(); err != nil { - return nil, fmt.Errorf("error reading accounts file: %w", err) - } - // Import codes codesFile, err := os.Open(filepath.Join(path, ExportedCodesFileName)) if err != nil { @@ -85,7 +65,7 @@ func ImportEVMState(path string) (*EVMState, error) { } defer codesFile.Close() - scanner = bufio.NewScanner(codesFile) + scanner := bufio.NewScanner(codesFile) for scanner.Scan() { code, err := CodeInContextFromEncoded(scanner.Bytes()) if err != nil { @@ -113,6 +93,28 @@ func ImportEVMState(path string) (*EVMState, error) { slots = append(slots, slot) } + // Import accounts + accountsFile, err := os.Open(filepath.Join(path, ExportedAccountsFileName)) + if err != nil { + return nil, fmt.Errorf("error opening accounts file: %w", err) + } + defer accountsFile.Close() + + scanner = bufio.NewScanner(accountsFile) + for scanner.Scan() { + acc, err := DecodeAccount(scanner.Bytes()) + if err != nil { + fmt.Println("error decoding account: ", err, scanner.Bytes()) + } else { + fmt.Println("decoded account", acc.Address) + } + accounts[acc.Address] = acc + } + + if err := scanner.Err(); err != nil { + return nil, fmt.Errorf("error reading accounts file: %w", err) + } + if err := scanner.Err(); err != nil { return nil, fmt.Errorf("error reading slots file: %w", err) } From b202ebc85945230c85536c6937ec8b9b5bce05cb Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 20:12:41 -0800 Subject: [PATCH 36/76] refactor importer --- fvm/evm/emulator/state/diff_test.go | 4 +- fvm/evm/emulator/state/importer.go | 60 ++++++++++++----------------- 2 files changed, 27 insertions(+), 37 deletions(-) diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 1a1a3161af4..60d5a699a88 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -7,10 +7,10 @@ import ( ) func TestStateDiff(t *testing.T) { - offchainState, err := ImportEVMState("/Users/leozhang/Downloads/compare-state/evm-state-from-gobs/") + offchainState, err := ImportEVMState("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) - enState, err := ImportEVMState("/Users/leozhang/Downloads/compare-state/evm-state-from-checkpoint/") + enState, err := ImportEVMState("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) differences := Diff(enState, offchainState) diff --git a/fvm/evm/emulator/state/importer.go b/fvm/evm/emulator/state/importer.go index 2149b716616..c210787a7fe 100644 --- a/fvm/evm/emulator/state/importer.go +++ b/fvm/evm/emulator/state/importer.go @@ -1,10 +1,10 @@ package state import ( - "bufio" "fmt" - "os" + "io/ioutil" "path/filepath" + "strings" gethCommon "github.com/onflow/go-ethereum/common" @@ -57,36 +57,34 @@ func ImportEVMState(path string) (*EVMState, error) { accounts := make(map[gethCommon.Address]*Account) var codes []*CodeInContext var slots []*types.SlotEntry - // Import codes - codesFile, err := os.Open(filepath.Join(path, ExportedCodesFileName)) + codesData, err := ioutil.ReadFile(filepath.Join(path, ExportedCodesFileName)) if err != nil { return nil, fmt.Errorf("error opening codes file: %w", err) } - defer codesFile.Close() - - scanner := bufio.NewScanner(codesFile) - for scanner.Scan() { - code, err := CodeInContextFromEncoded(scanner.Bytes()) + codesLines := strings.Split(string(codesData), "\n") + for _, line := range codesLines { + if line == "" { + continue + } + code, err := CodeInContextFromEncoded([]byte(line)) if err != nil { return nil, fmt.Errorf("error decoding code in context: %w", err) } codes = append(codes, code) } - if err := scanner.Err(); err != nil { - return nil, fmt.Errorf("error reading codes file: %w", err) - } // Import slots - slotsFile, err := os.Open(filepath.Join(path, ExportedSlotsFileName)) + slotsData, err := ioutil.ReadFile(filepath.Join(path, ExportedSlotsFileName)) if err != nil { return nil, fmt.Errorf("error opening slots file: %w", err) } - defer slotsFile.Close() - - scanner = bufio.NewScanner(slotsFile) - for scanner.Scan() { - slot, err := types.SlotEntryFromEncoded(scanner.Bytes()) + slotsLines := strings.Split(string(slotsData), "\n") + for _, line := range slotsLines { + if line == "" { + continue + } + slot, err := types.SlotEntryFromEncoded([]byte(line)) if err != nil { return nil, fmt.Errorf("error decoding slot entry: %w", err) } @@ -94,30 +92,22 @@ func ImportEVMState(path string) (*EVMState, error) { } // Import accounts - accountsFile, err := os.Open(filepath.Join(path, ExportedAccountsFileName)) + accountsData, err := ioutil.ReadFile(filepath.Join(path, ExportedAccountsFileName)) if err != nil { return nil, fmt.Errorf("error opening accounts file: %w", err) } - defer accountsFile.Close() - - scanner = bufio.NewScanner(accountsFile) - for scanner.Scan() { - acc, err := DecodeAccount(scanner.Bytes()) + accountsLines := strings.Split(string(accountsData), "\n") + for _, line := range accountsLines { + if line == "" { + continue + } + acc, err := DecodeAccount([]byte(line)) if err != nil { - fmt.Println("error decoding account: ", err, scanner.Bytes()) + fmt.Println("error decoding account: ", err, line) } else { fmt.Println("decoded account", acc.Address) + accounts[acc.Address] = acc } - accounts[acc.Address] = acc - } - - if err := scanner.Err(); err != nil { - return nil, fmt.Errorf("error reading accounts file: %w", err) } - - if err := scanner.Err(); err != nil { - return nil, fmt.Errorf("error reading slots file: %w", err) - } - return ToEVMState(accounts, codes, slots) } From 2564f0352a839f3945b786e0e42fda72233ffe51 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 20:53:49 -0800 Subject: [PATCH 37/76] export evm state as gobs --- fvm/evm/emulator/state/diff_test.go | 4 +- fvm/evm/emulator/state/exporter.go | 26 +++++++++ fvm/evm/emulator/state/extract.go | 84 +++++++++++++++++++++++++++++ fvm/evm/emulator/state/importer.go | 24 +++++++++ 4 files changed, 136 insertions(+), 2 deletions(-) create mode 100644 fvm/evm/emulator/state/extract.go diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 60d5a699a88..913b6e3bd7f 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -7,10 +7,10 @@ import ( ) func TestStateDiff(t *testing.T) { - offchainState, err := ImportEVMState("/var/flow2/evm-state-from-gobs-218215348/") + offchainState, err := ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) - enState, err := ImportEVMState("/var/flow2/evm-state-from-gobs-218215348/") + enState, err := ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) differences := Diff(enState, offchainState) diff --git a/fvm/evm/emulator/state/exporter.go b/fvm/evm/emulator/state/exporter.go index 09c71af277b..2dd3028b287 100644 --- a/fvm/evm/emulator/state/exporter.go +++ b/fvm/evm/emulator/state/exporter.go @@ -1,6 +1,7 @@ package state import ( + "encoding/gob" "fmt" "io" "os" @@ -17,6 +18,7 @@ const ( ExportedAccountsFileName = "accounts.bin" ExportedCodesFileName = "codes.bin" ExportedSlotsFileName = "slots.bin" + ExportedStateGobFileName = "state.gob" ) type Exporter struct { @@ -38,6 +40,30 @@ func NewExporter(ledger atree.Ledger, root flow.Address) (*Exporter, error) { }, nil } +func (e *Exporter) ExportGob(path string) error { + fileName := filepath.Join(path, ExportedStateGobFileName) + // Open the file for reading + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + state, err := Extract(e.ledger, e.root, e.baseView) + if err != nil { + return err + } + + // Use gob to encode data + encoder := gob.NewEncoder(file) + err = encoder.Encode(state) + if err != nil { + return err + } + + return nil +} + func (e *Exporter) Export(path string) error { af, err := os.Create(filepath.Join(path, ExportedAccountsFileName)) if err != nil { diff --git a/fvm/evm/emulator/state/extract.go b/fvm/evm/emulator/state/extract.go new file mode 100644 index 00000000000..47d78d41d99 --- /dev/null +++ b/fvm/evm/emulator/state/extract.go @@ -0,0 +1,84 @@ +package state + +import ( + "github.com/onflow/atree" + gethCommon "github.com/onflow/go-ethereum/common" + + "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" +) + +func Extract( + ledger atree.Ledger, + root flow.Address, + baseView *BaseView, +) (*EVMState, error) { + + accounts := make(map[gethCommon.Address]*Account, 0) + + itr, err := baseView.AccountIterator() + + if err != nil { + return nil, err + } + // make a list of accounts with storage + addrWithSlots := make([]gethCommon.Address, 0) + for { + // TODO: we can optimize by returning the encoded value + acc, err := itr.Next() + if err != nil { + return nil, err + } + if acc == nil { + break + } + if acc.HasStoredValues() { + addrWithSlots = append(addrWithSlots, acc.Address) + } + accounts[acc.Address] = acc + } + + codes := make(map[gethCommon.Hash]*CodeInContext, 0) + codeItr, err := baseView.CodeIterator() + if err != nil { + return nil, err + } + for { + cic, err := codeItr.Next() + if err != nil { + return nil, err + } + if cic == nil { + break + } + codes[cic.Hash] = cic + } + + // account address -> key -> value + slots := make(map[gethCommon.Address]map[gethCommon.Hash]*types.SlotEntry) + + for _, addr := range addrWithSlots { + slots[addr] = make(map[gethCommon.Hash]*types.SlotEntry) + slotItr, err := baseView.AccountStorageIterator(addr) + if err != nil { + return nil, err + } + for { + slot, err := slotItr.Next() + if err != nil { + return nil, err + } + if slot == nil { + break + } + + slots[addr][slot.Key] = slot + } + } + + return &EVMState{ + Accounts: accounts, + Codes: codes, + Slots: slots, + }, nil +} diff --git a/fvm/evm/emulator/state/importer.go b/fvm/evm/emulator/state/importer.go index c210787a7fe..132846512f4 100644 --- a/fvm/evm/emulator/state/importer.go +++ b/fvm/evm/emulator/state/importer.go @@ -1,8 +1,10 @@ package state import ( + "encoding/gob" "fmt" "io/ioutil" + "os" "path/filepath" "strings" @@ -53,6 +55,28 @@ func ToEVMState( return state, nil } +func ImportEVMStateFromGob(path string) (*EVMState, error) { + fileName := filepath.Join(path, ExportedStateGobFileName) + // Open the file for reading + file, err := os.Open(fileName) + if err != nil { + return nil, err + } + defer file.Close() + + // Prepare the map to store decoded data + var data EVMState + + // Use gob to decode data + decoder := gob.NewDecoder(file) + err = decoder.Decode(&data) + if err != nil { + return nil, err + } + + return &data, nil +} + func ImportEVMState(path string) (*EVMState, error) { accounts := make(map[gethCommon.Address]*Account) var codes []*CodeInContext From 86fc3be20b150c7006ec84258b28c7e4b2d35613 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 20:55:34 -0800 Subject: [PATCH 38/76] as gob --- cmd/util/cmd/export-evm-state/cmd.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/util/cmd/export-evm-state/cmd.go b/cmd/util/cmd/export-evm-state/cmd.go index 6cb05da37ba..c29ac6c1436 100644 --- a/cmd/util/cmd/export-evm-state/cmd.go +++ b/cmd/util/cmd/export-evm-state/cmd.go @@ -146,7 +146,7 @@ func ExportEVMStateFromPayloads( } } - err = exporter.Export(outputPath) + err = exporter.ExportGob(outputPath) if err != nil { return fmt.Errorf("failed to export: %w", err) } From eb8d42804b629b5ac44df7ecc0e2d556bdab3be0 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 20:57:19 -0800 Subject: [PATCH 39/76] open file --- fvm/evm/emulator/state/exporter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fvm/evm/emulator/state/exporter.go b/fvm/evm/emulator/state/exporter.go index 2dd3028b287..f3c8072e070 100644 --- a/fvm/evm/emulator/state/exporter.go +++ b/fvm/evm/emulator/state/exporter.go @@ -43,7 +43,7 @@ func NewExporter(ledger atree.Ledger, root flow.Address) (*Exporter, error) { func (e *Exporter) ExportGob(path string) error { fileName := filepath.Join(path, ExportedStateGobFileName) // Open the file for reading - file, err := os.Open(fileName) + file, err := os.Create(fileName) if err != nil { return err } From 24e36fd47a92c62f4e425375bf9d2ac5986e8d8b Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Sat, 23 Nov 2024 21:32:19 -0800 Subject: [PATCH 40/76] show account diff --- fvm/evm/emulator/state/diff.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fvm/evm/emulator/state/diff.go b/fvm/evm/emulator/state/diff.go index 5f79d4102b3..bae539bd5db 100644 --- a/fvm/evm/emulator/state/diff.go +++ b/fvm/evm/emulator/state/diff.go @@ -31,7 +31,7 @@ func Diff(a *EVMState, b *EVMState) []error { for addr, accA := range a.Accounts { if accB, exists := b.Accounts[addr]; exists { if !AccountEqual(accA, accB) { - differences = append(differences, fmt.Errorf("account %s differs", addr.Hex())) + differences = append(differences, fmt.Errorf("account %s differs, accA %v, accB %v", addr.Hex(), accA, accB)) } } else { differences = append(differences, fmt.Errorf("account %s exists in a but not in b", addr.Hex())) From 5fe1eea336ad0bf0db5b1ad59a2ddfbcac0acef2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 25 Nov 2024 10:24:38 -0800 Subject: [PATCH 41/76] updat export gob method --- fvm/evm/emulator/state/exporter.go | 2 +- fvm/evm/emulator/state/extract.go | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/fvm/evm/emulator/state/exporter.go b/fvm/evm/emulator/state/exporter.go index f3c8072e070..f1cb9bcfa10 100644 --- a/fvm/evm/emulator/state/exporter.go +++ b/fvm/evm/emulator/state/exporter.go @@ -49,7 +49,7 @@ func (e *Exporter) ExportGob(path string) error { } defer file.Close() - state, err := Extract(e.ledger, e.root, e.baseView) + state, err := Extract(e.root, e.baseView) if err != nil { return err } diff --git a/fvm/evm/emulator/state/extract.go b/fvm/evm/emulator/state/extract.go index 47d78d41d99..e0bb30d82aa 100644 --- a/fvm/evm/emulator/state/extract.go +++ b/fvm/evm/emulator/state/extract.go @@ -1,7 +1,6 @@ package state import ( - "github.com/onflow/atree" gethCommon "github.com/onflow/go-ethereum/common" "github.com/onflow/flow-go/fvm/evm/types" @@ -9,7 +8,6 @@ import ( ) func Extract( - ledger atree.Ledger, root flow.Address, baseView *BaseView, ) (*EVMState, error) { From fb7b4bc84d6fffde3a594bb478e0a9b4a059139c Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 25 Nov 2024 10:33:59 -0800 Subject: [PATCH 42/76] fix lint --- fvm/evm/offchain/blocks/block_context.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fvm/evm/offchain/blocks/block_context.go b/fvm/evm/offchain/blocks/block_context.go index 2da73f9cb32..5b5095a9d2f 100644 --- a/fvm/evm/offchain/blocks/block_context.go +++ b/fvm/evm/offchain/blocks/block_context.go @@ -83,6 +83,11 @@ const blockHashListFixHCUEVMHeightMainnet = 8357079 // PR: https://github.com/onflow/flow-go/pull/6734 const blockHashListFixHCUEVMHeightTestnet = 16848829 +// Testnet52 - Spork +// Flow Block: 218215350 cc7188f0bdac4c442cc3ee072557d7f7c8ca4462537da945b148d5d0efa7a1ff +// PR: https://github.com/onflow/flow-go/pull/6377 +// const blockHashListBugIntroducedHCUEVMHeightTestnet = 7038679 + // Testnet51 - Height Coordinated Upgrade 1 // Flow Block: 212562161 1a520608c5457f228405c4c30fc39c8a0af7cf915fb2ede7ec5ccffc2a000f57 // PR: https://github.com/onflow/flow-go/pull/6380 From 888bddbac9dc6cda48640d78b243878f15067997 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 07:05:08 -0800 Subject: [PATCH 43/76] fix coinbaseAddressChangeEVMHeightTestnet --- fvm/evm/offchain/blocks/block_context.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fvm/evm/offchain/blocks/block_context.go b/fvm/evm/offchain/blocks/block_context.go index 5b5095a9d2f..669f2dda998 100644 --- a/fvm/evm/offchain/blocks/block_context.go +++ b/fvm/evm/offchain/blocks/block_context.go @@ -91,7 +91,7 @@ const blockHashListFixHCUEVMHeightTestnet = 16848829 // Testnet51 - Height Coordinated Upgrade 1 // Flow Block: 212562161 1a520608c5457f228405c4c30fc39c8a0af7cf915fb2ede7ec5ccffc2a000f57 // PR: https://github.com/onflow/flow-go/pull/6380 -const coinbaseAddressChangeEVMHeightTestnet = 1385491 +const coinbaseAddressChangeEVMHeightTestnet = 1385490 var genesisCoinbaseAddressTestnet = types.Address(gethCommon.HexToAddress("0000000000000000000000021169100eecb7c1a6")) From cf326885405e6e9d9426659924697bfc5f0826f2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 26 Nov 2024 19:31:19 -0800 Subject: [PATCH 44/76] add diff test cases --- fvm/evm/emulator/state/diff_test.go | 62 +++++++++++++++++++++++++++-- 1 file changed, 58 insertions(+), 4 deletions(-) diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 913b6e3bd7f..68f798e05c6 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -1,19 +1,73 @@ -package state +package state_test import ( + "fmt" + "path/filepath" "testing" + "github.com/onflow/flow-go/fvm/evm" + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/model/flow" "github.com/stretchr/testify/require" ) func TestStateDiff(t *testing.T) { - offchainState, err := ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") + offchainState, err := state.ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) - enState, err := ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") + enState, err := state.ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) - differences := Diff(enState, offchainState) + differences := state.Diff(enState, offchainState) require.Len(t, differences, 0) } + +func TestEVMStateDiff(t *testing.T) { + + state1 := EVMStateFromReplayGobDir(t, "/var/flow2/evm-state-from-gobs-218215348/", uint64(218215348)) + // state2 := EVMStateFromReplayGobDir(t, "/var/flow2/evm-state-from-gobs-218215348/", uint64(218215348)) + state2 := EVMStateFromCheckpointExtract(t, "/var/flow2/evm-state-from-checkpoint-218215348/") + + differences := state.Diff(state1, state2) + + for i, diff := range differences { + fmt.Printf("Difference %d: %v\n", i, diff) + } + + require.Len(t, differences, 0) +} + +func EVMStateFromCheckpointExtract(t *testing.T, dir string) *state.EVMState { + enState, err := state.ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") + require.NoError(t, err) + return enState +} + +func EVMStateFromReplayGobDir(t *testing.T, gobDir string, flowHeight uint64) *state.EVMState { + valueFileName, allocatorFileName := evmStateGobFileNamesByEndHeight(gobDir, flowHeight) + chainID := flow.Testnet + + allocatorGobs, err := testutils.DeserializeAllocator(allocatorFileName) + require.NoError(t, err) + + storageRoot := evm.StorageAccountAddress(chainID) + valuesGob, err := testutils.DeserializeState(valueFileName) + require.NoError(t, err) + + store := testutils.GetSimpleValueStorePopulated(valuesGob, allocatorGobs) + + bv, err := state.NewBaseView(store, storageRoot) + require.NoError(t, err) + + evmState, err := state.Extract(storageRoot, bv) + require.NoError(t, err) + return evmState +} + +func evmStateGobFileNamesByEndHeight(evmStateGobDir string, endHeight uint64) (string, string) { + valueFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("values-%d.gob", endHeight)) + allocatorFileName := filepath.Join(evmStateGobDir, fmt.Sprintf("allocators-%d.gob", endHeight)) + return valueFileName, allocatorFileName +} From d50ddb78c18718d7d7ef0f0b55af7f1eea42ca01 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 06:46:15 -0800 Subject: [PATCH 45/76] fix lint --- fvm/evm/emulator/state/diff_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 68f798e05c6..057c4314fa4 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -5,11 +5,12 @@ import ( "path/filepath" "testing" + "github.com/stretchr/testify/require" + "github.com/onflow/flow-go/fvm/evm" "github.com/onflow/flow-go/fvm/evm/emulator/state" "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/model/flow" - "github.com/stretchr/testify/require" ) func TestStateDiff(t *testing.T) { From 10d4b9db7f3275280d04f65055f3b0aa1c0919fd Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 09:38:11 -0800 Subject: [PATCH 46/76] update error message --- fvm/evm/offchain/storage/readonly.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fvm/evm/offchain/storage/readonly.go b/fvm/evm/offchain/storage/readonly.go index 4ed33a6fe44..6c66e7c1e43 100644 --- a/fvm/evm/offchain/storage/readonly.go +++ b/fvm/evm/offchain/storage/readonly.go @@ -1,7 +1,7 @@ package storage import ( - "errors" + "fmt" "github.com/onflow/atree" @@ -29,7 +29,7 @@ func (s *ReadOnlyStorage) GetValue(owner []byte, key []byte) ([]byte, error) { // SetValue returns an error if called func (s *ReadOnlyStorage) SetValue(owner, key, value []byte) error { - return errors.New("unexpected call received") + return fmt.Errorf("unexpected call received for SetValue with owner: %x, key: %v, value: %x", owner, key, value) } // ValueExists checks if a register exists @@ -40,5 +40,5 @@ func (s *ReadOnlyStorage) ValueExists(owner []byte, key []byte) (bool, error) { // AllocateSlabIndex returns an error if called func (s *ReadOnlyStorage) AllocateSlabIndex(owner []byte) (atree.SlabIndex, error) { - return atree.SlabIndex{}, errors.New("unexpected call received") + return atree.SlabIndex{}, fmt.Errorf("unexpected call received for AllocateSlabIndex with owner: %x", owner) } From 59a1d05f56115848b71d882637fa033bd1276db2 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 09:41:43 -0800 Subject: [PATCH 47/76] use panic --- fvm/evm/offchain/storage/readonly.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fvm/evm/offchain/storage/readonly.go b/fvm/evm/offchain/storage/readonly.go index 6c66e7c1e43..730c8670dcb 100644 --- a/fvm/evm/offchain/storage/readonly.go +++ b/fvm/evm/offchain/storage/readonly.go @@ -29,7 +29,7 @@ func (s *ReadOnlyStorage) GetValue(owner []byte, key []byte) ([]byte, error) { // SetValue returns an error if called func (s *ReadOnlyStorage) SetValue(owner, key, value []byte) error { - return fmt.Errorf("unexpected call received for SetValue with owner: %x, key: %v, value: %x", owner, key, value) + panic(fmt.Sprintf("unexpected call received for SetValue with owner: %x, key: %v, value: %x", owner, key, value)) } // ValueExists checks if a register exists @@ -40,5 +40,5 @@ func (s *ReadOnlyStorage) ValueExists(owner []byte, key []byte) (bool, error) { // AllocateSlabIndex returns an error if called func (s *ReadOnlyStorage) AllocateSlabIndex(owner []byte) (atree.SlabIndex, error) { - return atree.SlabIndex{}, fmt.Errorf("unexpected call received for AllocateSlabIndex with owner: %x", owner) + panic(fmt.Errorf("unexpected call received for AllocateSlabIndex with owner: %x", owner)) } From 2ff3bee18d812931bab6044de1f3d9e051b72793 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 09:55:17 -0800 Subject: [PATCH 48/76] include missing key in error message --- fvm/evm/emulator/state/base.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fvm/evm/emulator/state/base.go b/fvm/evm/emulator/state/base.go index 9f11ce6e3f0..9295bf6ac9f 100644 --- a/fvm/evm/emulator/state/base.go +++ b/fvm/evm/emulator/state/base.go @@ -74,13 +74,13 @@ func NewBaseView(ledger atree.Ledger, rootAddress flow.Address) (*BaseView, erro // fetch the account collection, if not exist, create one view.accounts, view.accountSetupOnCommit, err = view.fetchOrCreateCollection(AccountsStorageIDKey) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to fetch or create account collection with key %v: %w", AccountsStorageIDKey, err) } // fetch the code collection, if not exist, create one view.codes, view.codeSetupOnCommit, err = view.fetchOrCreateCollection(CodesStorageIDKey) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to fetch or create code collection with key %v: %w", CodesStorageIDKey, err) } return view, nil From b73baac082e79e38449d487329f65265240d3e09 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Wed, 27 Nov 2024 10:00:13 -0800 Subject: [PATCH 49/76] update error message --- fvm/evm/emulator/state/base.go | 2 +- fvm/evm/offchain/storage/readonly.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/fvm/evm/emulator/state/base.go b/fvm/evm/emulator/state/base.go index 9295bf6ac9f..97df2076957 100644 --- a/fvm/evm/emulator/state/base.go +++ b/fvm/evm/emulator/state/base.go @@ -485,7 +485,7 @@ func (v *BaseView) fetchOrCreateCollection(path string) (collection *Collection, } if len(collectionID) == 0 { collection, err = v.collectionProvider.NewCollection() - return collection, true, err + return collection, true, fmt.Errorf("fail to create collection with key %v: %w", path, err) } collection, err = v.collectionProvider.CollectionByID(collectionID) return collection, false, err diff --git a/fvm/evm/offchain/storage/readonly.go b/fvm/evm/offchain/storage/readonly.go index 730c8670dcb..6c66e7c1e43 100644 --- a/fvm/evm/offchain/storage/readonly.go +++ b/fvm/evm/offchain/storage/readonly.go @@ -29,7 +29,7 @@ func (s *ReadOnlyStorage) GetValue(owner []byte, key []byte) ([]byte, error) { // SetValue returns an error if called func (s *ReadOnlyStorage) SetValue(owner, key, value []byte) error { - panic(fmt.Sprintf("unexpected call received for SetValue with owner: %x, key: %v, value: %x", owner, key, value)) + return fmt.Errorf("unexpected call received for SetValue with owner: %x, key: %v, value: %x", owner, key, value) } // ValueExists checks if a register exists @@ -40,5 +40,5 @@ func (s *ReadOnlyStorage) ValueExists(owner []byte, key []byte) (bool, error) { // AllocateSlabIndex returns an error if called func (s *ReadOnlyStorage) AllocateSlabIndex(owner []byte) (atree.SlabIndex, error) { - panic(fmt.Errorf("unexpected call received for AllocateSlabIndex with owner: %x", owner)) + return atree.SlabIndex{}, fmt.Errorf("unexpected call received for AllocateSlabIndex with owner: %x", owner) } From e7f94c538f0a9cf2ebdc6d4b473a973ae49fd6ed Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Mon, 2 Dec 2024 14:44:06 -0800 Subject: [PATCH 50/76] apply review comments --- fvm/evm/offchain/blocks/block_context.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/fvm/evm/offchain/blocks/block_context.go b/fvm/evm/offchain/blocks/block_context.go index 669f2dda998..ecbc8813c76 100644 --- a/fvm/evm/offchain/blocks/block_context.go +++ b/fvm/evm/offchain/blocks/block_context.go @@ -83,11 +83,6 @@ const blockHashListFixHCUEVMHeightMainnet = 8357079 // PR: https://github.com/onflow/flow-go/pull/6734 const blockHashListFixHCUEVMHeightTestnet = 16848829 -// Testnet52 - Spork -// Flow Block: 218215350 cc7188f0bdac4c442cc3ee072557d7f7c8ca4462537da945b148d5d0efa7a1ff -// PR: https://github.com/onflow/flow-go/pull/6377 -// const blockHashListBugIntroducedHCUEVMHeightTestnet = 7038679 - // Testnet51 - Height Coordinated Upgrade 1 // Flow Block: 212562161 1a520608c5457f228405c4c30fc39c8a0af7cf915fb2ede7ec5ccffc2a000f57 // PR: https://github.com/onflow/flow-go/pull/6380 From fa089b0a59a70bf08933f183024c9cd217e223d3 Mon Sep 17 00:00:00 2001 From: Jordan Schalm Date: Tue, 3 Dec 2024 15:22:25 -0800 Subject: [PATCH 51/76] remove ex-package links these don't resolve correctly when testing locally --- engine/consensus/dkg/doc.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/consensus/dkg/doc.go b/engine/consensus/dkg/doc.go index 2feff12aa42..5bf29b43eaa 100644 --- a/engine/consensus/dkg/doc.go +++ b/engine/consensus/dkg/doc.go @@ -18,8 +18,8 @@ // # Architecture // // In the happy path, one DKG instance runs every epoch. For each DKG instance, the [ReactorEngine] -// instantiates a new, epoch-scoped [module.DKGController] and [module.DKGBroker] using the provided ControllerFactory. -// The ControllerFactory ties new DKGControllers to the [MessagingEngine] via a BrokerTunnel, +// instantiates a new, epoch-scoped module.DKGController and module.DKGBroker using the provided dkg.ControllerFactory. +// The dkg.ControllerFactory ties new module.DKGController's to the [MessagingEngine] via a dkg.BrokerTunnel, // which exposes channels to relay incoming and outgoing messages (see package module/dkg for details). // // EpochSetup/EpochCommit/OnView events From 3f1de8f49f95c4bf909ceb192610dca53a3fe4eb Mon Sep 17 00:00:00 2001 From: Jordan Schalm Date: Tue, 3 Dec 2024 15:23:20 -0800 Subject: [PATCH 52/76] Apply suggestions from code review Co-authored-by: Alexander Hentschel --- engine/consensus/dkg/doc.go | 47 +++++++++++++++++-------------------- 1 file changed, 21 insertions(+), 26 deletions(-) diff --git a/engine/consensus/dkg/doc.go b/engine/consensus/dkg/doc.go index 5bf29b43eaa..15ee9e1e06a 100644 --- a/engine/consensus/dkg/doc.go +++ b/engine/consensus/dkg/doc.go @@ -5,7 +5,7 @@ // The [ReactorEngine] implements triggers to control the lifecycle of DKG instances. // A new DKG instance is started when an EpochSetup service event is sealed. // The subsequent phase transitions are triggered when specified views are encountered. -// Specifically, phase transitions for a view V are triggered when the first block with view >=V is finalized. +// Specifically, phase transitions for a view V are triggered when the first block with view ≥V is finalized. // Between phase transitions, we periodically query the DKG smart-contract ("whiteboard") to read broadcast messages. // Before transitioning the state machine to the next phase, we query the whiteboard w.r.t. the final view // of the phase - this ensures all participants eventually observe the same set of messages for each phase. @@ -22,29 +22,24 @@ // The dkg.ControllerFactory ties new module.DKGController's to the [MessagingEngine] via a dkg.BrokerTunnel, // which exposes channels to relay incoming and outgoing messages (see package module/dkg for details). // -// EpochSetup/EpochCommit/OnView events -// | -// v -// +---------------+ -// | ReactorEngine | -// +---------------+ -// | -// v -// *~~~~~~~~~~~~~~~~~~~~~* <- Epoch-scoped components -// | +---------------+ | -// | | Controller | | -// | +---------------+ | -// | | | -// | v | -// | +---------------+ | -// | | Broker | | -// | +---------------+ | -// *~~~~~~~~|~~~~~~~~~\~~* -// | \ -// BrokerTunnel DKGContractClient -// | \ -// +--------------+ +------------------+ -// | Messaging | | FlowDKG smart | -// | Engine | | contract | -// +--------------+ +------------------+ +// EpochSetup/EpochCommit/OnView events +// ↓ +// ┏━━━━━━━━━━━━━━━━━┓ +// ┃ ReactorEngine ┃ +// ┗━━━━━━━━━━━━━━━━━┛ +// ↓ +// ┏━━━━━━━━━━━━━━━━━┓ ╮ +// ┃ Controller ┃ │ +// ┗━━━━━━━━━━━━━━━━━┛ │ +// ↓ ┝ Epoch-scoped components +// ┏━━━━━━━━━━━━━━━━━┓ │ +// ┃ Broker ┃ │ +// ┗━━━━━━━━━━━━━━━━━┛ ╯ +// │ │ +// BrokerTunnel DKGContractClient +// ↓ ↓ +// ┏━━━━━━━━━━━━━━┓ ┏━━━━━━━━━━━━━━━━━━┓ +// ┃ Messaging ┃ ┃ FlowDKG smart ┃ +// ┃ Engine ┃ ┃ contract ┃ +// ┗━━━━━━━━━━━━━━┛ ┗━━━━━━━━━━━━━━━━━━┛ package dkg From 4d419c4ab515af55e4f0c33899dca934a62afc38 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 08:01:00 -0800 Subject: [PATCH 53/76] disable tests --- fvm/evm/emulator/state/diff_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fvm/evm/emulator/state/diff_test.go b/fvm/evm/emulator/state/diff_test.go index 057c4314fa4..4abb6868795 100644 --- a/fvm/evm/emulator/state/diff_test.go +++ b/fvm/evm/emulator/state/diff_test.go @@ -13,7 +13,7 @@ import ( "github.com/onflow/flow-go/model/flow" ) -func TestStateDiff(t *testing.T) { +func StateDiff(t *testing.T) { offchainState, err := state.ImportEVMStateFromGob("/var/flow2/evm-state-from-gobs-218215348/") require.NoError(t, err) @@ -25,7 +25,7 @@ func TestStateDiff(t *testing.T) { require.Len(t, differences, 0) } -func TestEVMStateDiff(t *testing.T) { +func EVMStateDiff(t *testing.T) { state1 := EVMStateFromReplayGobDir(t, "/var/flow2/evm-state-from-gobs-218215348/", uint64(218215348)) // state2 := EVMStateFromReplayGobDir(t, "/var/flow2/evm-state-from-gobs-218215348/", uint64(218215348)) From 2d4bb1065751a7629e037d389a4d988368eec1bb Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 08:11:14 -0800 Subject: [PATCH 54/76] fix error handling --- fvm/evm/emulator/state/base.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fvm/evm/emulator/state/base.go b/fvm/evm/emulator/state/base.go index 97df2076957..0f690b7367a 100644 --- a/fvm/evm/emulator/state/base.go +++ b/fvm/evm/emulator/state/base.go @@ -485,7 +485,10 @@ func (v *BaseView) fetchOrCreateCollection(path string) (collection *Collection, } if len(collectionID) == 0 { collection, err = v.collectionProvider.NewCollection() - return collection, true, fmt.Errorf("fail to create collection with key %v: %w", path, err) + if err != nil { + return collection, true, fmt.Errorf("fail to create collection with key %v: %w", path, err) + } + return collection, true, nil } collection, err = v.collectionProvider.CollectionByID(collectionID) return collection, false, err From eea7a2c71d9101a179f8cd375aef50c9f616aed9 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Thu, 5 Dec 2024 12:21:20 -0800 Subject: [PATCH 55/76] 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 56/76] 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 57/76] 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 58/76] 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 59/76] 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 60/76] 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 61/76] 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 62/76] 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 63/76] 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 64/76] 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 65/76] 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 66/76] 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 67/76] 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 68/76] 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 0c778a7c1d69b441d4a711b7ba14a6f61e978212 Mon Sep 17 00:00:00 2001 From: sjonpaulbrown Date: Fri, 6 Dec 2024 14:39:31 -0700 Subject: [PATCH 69/76] Update Makefile to support private util builds --- Makefile | 1 + 1 file changed, 1 insertion(+) diff --git a/Makefile b/Makefile index 36075edd094..32feb2a5184 100644 --- a/Makefile +++ b/Makefile @@ -859,6 +859,7 @@ docker-all-tools: tool-util tool-remove-execution-fork PHONY: docker-build-util docker-build-util: docker build -f cmd/Dockerfile --build-arg TARGET=./cmd/util --build-arg GOARCH=$(GOARCH) --build-arg VERSION=$(IMAGE_TAG) --build-arg CGO_FLAG=$(DISABLE_ADX) --target production \ + --secret id=cadence_deploy_key,env=CADENCE_DEPLOY_KEY --build-arg GOPRIVATE=$(GOPRIVATE) \ -t "$(CONTAINER_REGISTRY)/util:latest" \ -t "$(CONTAINER_REGISTRY)/util:$(IMAGE_TAG)" . From 88d86d291477de801b259fcd843a0878891eabf5 Mon Sep 17 00:00:00 2001 From: "Leo Zhang (zhangchiqing)" Date: Tue, 10 Dec 2024 14:06:47 -0800 Subject: [PATCH 70/76] 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 71/76] 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 72/76] 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 73/76] 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 74/76] 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 75/76] 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 76/76] 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 {