core/state/snapshot: implement storage iterator (#20971)
* core/state/snapshot: implement storage iterator * core/state/snapshot, tests: implement helper function * core/state/snapshot: fix storage issue If an account is deleted in the tx_1 but recreated in the tx_2, the it can happen that in this diff layer, both destructedSet and storageData records this account. In this case, the storage iterator should be able to iterate the slots belong to new account but disable further iteration in deeper layers(belong to old account) * core/state/snapshot: address peter and martin's comment * core/state: address comments * core/state/snapshot: fix test
This commit is contained in:
parent
1264c19f11
commit
26d271dfbb
@ -24,8 +24,10 @@ import (
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
|
||||
// Account is a slim version of a state.Account, where the root and code hash
|
||||
// are replaced with a nil byte slice for empty accounts.
|
||||
// Account is a modified version of a state.Account, where the root is replaced
|
||||
// with a byte slice. This format can be used to represent full-consensus format
|
||||
// or slim-snapshot format which replaces the empty root and code hash as nil
|
||||
// byte slice.
|
||||
type Account struct {
|
||||
Nonce uint64
|
||||
Balance *big.Int
|
||||
@ -33,9 +35,8 @@ type Account struct {
|
||||
CodeHash []byte
|
||||
}
|
||||
|
||||
// AccountRLP converts a state.Account content into a slim snapshot version RLP
|
||||
// encoded.
|
||||
func AccountRLP(nonce uint64, balance *big.Int, root common.Hash, codehash []byte) []byte {
|
||||
// SlimAccount converts a state.Account content into a slim snapshot account
|
||||
func SlimAccount(nonce uint64, balance *big.Int, root common.Hash, codehash []byte) Account {
|
||||
slim := Account{
|
||||
Nonce: nonce,
|
||||
Balance: balance,
|
||||
@ -46,9 +47,40 @@ func AccountRLP(nonce uint64, balance *big.Int, root common.Hash, codehash []byt
|
||||
if !bytes.Equal(codehash, emptyCode[:]) {
|
||||
slim.CodeHash = codehash
|
||||
}
|
||||
data, err := rlp.EncodeToBytes(slim)
|
||||
return slim
|
||||
}
|
||||
|
||||
// SlimAccountRLP converts a state.Account content into a slim snapshot
|
||||
// version RLP encoded.
|
||||
func SlimAccountRLP(nonce uint64, balance *big.Int, root common.Hash, codehash []byte) []byte {
|
||||
data, err := rlp.EncodeToBytes(SlimAccount(nonce, balance, root, codehash))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return data
|
||||
}
|
||||
|
||||
// FullAccount decodes the data on the 'slim RLP' format and return
|
||||
// the consensus format account.
|
||||
func FullAccount(data []byte) (Account, error) {
|
||||
var account Account
|
||||
if err := rlp.DecodeBytes(data, &account); err != nil {
|
||||
return Account{}, err
|
||||
}
|
||||
if len(account.Root) == 0 {
|
||||
account.Root = emptyRoot[:]
|
||||
}
|
||||
if len(account.CodeHash) == 0 {
|
||||
account.CodeHash = emptyCode[:]
|
||||
}
|
||||
return account, nil
|
||||
}
|
||||
|
||||
// FullAccountRLP converts data on the 'slim RLP' format into the full RLP-format.
|
||||
func FullAccountRLP(data []byte) ([]byte, error) {
|
||||
account, err := FullAccount(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return rlp.EncodeToBytes(account)
|
||||
}
|
||||
|
@ -17,6 +17,8 @@
|
||||
package snapshot
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -27,80 +29,239 @@ import (
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
// conversionAccount is used for converting between full and slim format. When
|
||||
// doing this, we can consider 'balance' as a byte array, as it has already
|
||||
// been converted from big.Int into an rlp-byteslice.
|
||||
type conversionAccount struct {
|
||||
Nonce uint64
|
||||
Balance []byte
|
||||
Root []byte
|
||||
CodeHash []byte
|
||||
}
|
||||
|
||||
// SlimToFull converts data on the 'slim RLP' format into the full RLP-format
|
||||
func SlimToFull(data []byte) ([]byte, error) {
|
||||
acc := &conversionAccount{}
|
||||
if err := rlp.DecodeBytes(data, acc); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(acc.Root) == 0 {
|
||||
acc.Root = emptyRoot[:]
|
||||
}
|
||||
if len(acc.CodeHash) == 0 {
|
||||
acc.CodeHash = emptyCode[:]
|
||||
}
|
||||
fullData, err := rlp.EncodeToBytes(acc)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return fullData, nil
|
||||
}
|
||||
|
||||
// trieKV represents a trie key-value pair
|
||||
type trieKV struct {
|
||||
key common.Hash
|
||||
value []byte
|
||||
}
|
||||
|
||||
type trieGeneratorFn func(in chan (trieKV), out chan (common.Hash))
|
||||
type (
|
||||
// trieGeneratorFn is the interface of trie generation which can
|
||||
// be implemented by different trie algorithm.
|
||||
trieGeneratorFn func(in chan (trieKV), out chan (common.Hash))
|
||||
|
||||
// GenerateTrieRoot takes an account iterator and reproduces the root hash.
|
||||
func GenerateTrieRoot(it AccountIterator) common.Hash {
|
||||
return generateTrieRoot(it, stdGenerate)
|
||||
// leafCallbackFn is the callback invoked at the leaves of the trie,
|
||||
// returns the subtrie root with the specified subtrie identifier.
|
||||
leafCallbackFn func(hash common.Hash, stat *generateStats) common.Hash
|
||||
)
|
||||
|
||||
// GenerateAccountTrieRoot takes an account iterator and reproduces the root hash.
|
||||
func GenerateAccountTrieRoot(it AccountIterator) (common.Hash, error) {
|
||||
return generateTrieRoot(it, common.Hash{}, stdGenerate, nil, &generateStats{start: time.Now()}, true)
|
||||
}
|
||||
|
||||
func generateTrieRoot(it AccountIterator, generatorFn trieGeneratorFn) common.Hash {
|
||||
// GenerateStorageTrieRoot takes a storage iterator and reproduces the root hash.
|
||||
func GenerateStorageTrieRoot(account common.Hash, it StorageIterator) (common.Hash, error) {
|
||||
return generateTrieRoot(it, account, stdGenerate, nil, &generateStats{start: time.Now()}, true)
|
||||
}
|
||||
|
||||
// VerifyState takes the whole snapshot tree as the input, traverses all the accounts
|
||||
// as well as the corresponding storages and compares the re-computed hash with the
|
||||
// original one(state root and the storage root).
|
||||
func VerifyState(snaptree *Tree, root common.Hash) error {
|
||||
acctIt, err := snaptree.AccountIterator(root, common.Hash{})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
got, err := generateTrieRoot(acctIt, common.Hash{}, stdGenerate, func(account common.Hash, stat *generateStats) common.Hash {
|
||||
storageIt, err := snaptree.StorageIterator(root, account, common.Hash{})
|
||||
if err != nil {
|
||||
return common.Hash{}
|
||||
}
|
||||
hash, err := generateTrieRoot(storageIt, account, stdGenerate, nil, stat, false)
|
||||
if err != nil {
|
||||
return common.Hash{}
|
||||
}
|
||||
return hash
|
||||
}, &generateStats{start: time.Now()}, true)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if got != root {
|
||||
return fmt.Errorf("State root hash mismatch, got %x, want %x", got, root)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// generateStats is a collection of statistics gathered by the trie generator
|
||||
// for logging purposes.
|
||||
type generateStats struct {
|
||||
accounts uint64
|
||||
slots uint64
|
||||
curAccount common.Hash
|
||||
curSlot common.Hash
|
||||
start time.Time
|
||||
lock sync.RWMutex
|
||||
}
|
||||
|
||||
// progress records the progress trie generator made recently.
|
||||
func (stat *generateStats) progress(accounts, slots uint64, curAccount common.Hash, curSlot common.Hash) {
|
||||
stat.lock.Lock()
|
||||
defer stat.lock.Unlock()
|
||||
|
||||
stat.accounts += accounts
|
||||
stat.slots += slots
|
||||
if curAccount != (common.Hash{}) {
|
||||
stat.curAccount = curAccount
|
||||
}
|
||||
if curSlot != (common.Hash{}) {
|
||||
stat.curSlot = curSlot
|
||||
}
|
||||
}
|
||||
|
||||
// report prints the cumulative progress statistic smartly.
|
||||
func (stat *generateStats) report() {
|
||||
stat.lock.RLock()
|
||||
defer stat.lock.RUnlock()
|
||||
|
||||
var ctx []interface{}
|
||||
if stat.curSlot != (common.Hash{}) {
|
||||
ctx = append(ctx, []interface{}{
|
||||
"in", stat.curAccount,
|
||||
"at", stat.curSlot,
|
||||
}...)
|
||||
} else {
|
||||
ctx = append(ctx, []interface{}{"at", stat.curAccount}...)
|
||||
}
|
||||
// Add the usual measurements
|
||||
ctx = append(ctx, []interface{}{"accounts", stat.accounts}...)
|
||||
if stat.slots != 0 {
|
||||
ctx = append(ctx, []interface{}{"slots", stat.slots}...)
|
||||
}
|
||||
ctx = append(ctx, []interface{}{"elapsed", common.PrettyDuration(time.Since(stat.start))}...)
|
||||
log.Info("Generating trie hash from snapshot", ctx...)
|
||||
}
|
||||
|
||||
// reportDone prints the last log when the whole generation is finished.
|
||||
func (stat *generateStats) reportDone() {
|
||||
stat.lock.RLock()
|
||||
defer stat.lock.RUnlock()
|
||||
|
||||
var ctx []interface{}
|
||||
ctx = append(ctx, []interface{}{"accounts", stat.accounts}...)
|
||||
if stat.slots != 0 {
|
||||
ctx = append(ctx, []interface{}{"slots", stat.slots}...)
|
||||
}
|
||||
ctx = append(ctx, []interface{}{"elapsed", common.PrettyDuration(time.Since(stat.start))}...)
|
||||
log.Info("Generated trie hash from snapshot", ctx...)
|
||||
}
|
||||
|
||||
// generateTrieRoot generates the trie hash based on the snapshot iterator.
|
||||
// It can be used for generating account trie, storage trie or even the
|
||||
// whole state which connects the accounts and the corresponding storages.
|
||||
func generateTrieRoot(it Iterator, account common.Hash, generatorFn trieGeneratorFn, leafCallback leafCallbackFn, stats *generateStats, report bool) (common.Hash, error) {
|
||||
var (
|
||||
in = make(chan trieKV) // chan to pass leaves
|
||||
out = make(chan common.Hash) // chan to collect result
|
||||
wg sync.WaitGroup
|
||||
in = make(chan trieKV) // chan to pass leaves
|
||||
out = make(chan common.Hash, 1) // chan to collect result
|
||||
stoplog = make(chan bool, 1) // 1-size buffer, works when logging is not enabled
|
||||
wg sync.WaitGroup
|
||||
)
|
||||
// Spin up a go-routine for trie hash re-generation
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
generatorFn(in, out)
|
||||
wg.Done()
|
||||
}()
|
||||
// Feed leaves
|
||||
start := time.Now()
|
||||
logged := time.Now()
|
||||
accounts := 0
|
||||
for it.Next() {
|
||||
slimData := it.Account()
|
||||
fullData, _ := SlimToFull(slimData)
|
||||
l := trieKV{it.Hash(), fullData}
|
||||
in <- l
|
||||
if time.Since(logged) > 8*time.Second {
|
||||
log.Info("Generating trie hash from snapshot",
|
||||
"at", l.key, "accounts", accounts, "elapsed", time.Since(start))
|
||||
logged = time.Now()
|
||||
}
|
||||
accounts++
|
||||
|
||||
// Spin up a go-routine for progress logging
|
||||
if report && stats != nil {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
timer := time.NewTimer(0)
|
||||
defer timer.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-timer.C:
|
||||
stats.report()
|
||||
timer.Reset(time.Second * 8)
|
||||
case success := <-stoplog:
|
||||
if success {
|
||||
stats.reportDone()
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
close(in)
|
||||
result := <-out
|
||||
log.Info("Generated trie hash from snapshot", "accounts", accounts, "elapsed", time.Since(start))
|
||||
wg.Wait()
|
||||
return result
|
||||
// stop is a helper function to shutdown the background threads
|
||||
// and return the re-generated trie hash.
|
||||
stop := func(success bool) common.Hash {
|
||||
close(in)
|
||||
result := <-out
|
||||
stoplog <- success
|
||||
wg.Wait()
|
||||
return result
|
||||
}
|
||||
var (
|
||||
logged = time.Now()
|
||||
processed = uint64(0)
|
||||
leaf trieKV
|
||||
last common.Hash
|
||||
)
|
||||
// Start to feed leaves
|
||||
for it.Next() {
|
||||
if account == (common.Hash{}) {
|
||||
var (
|
||||
err error
|
||||
fullData []byte
|
||||
)
|
||||
if leafCallback == nil {
|
||||
fullData, err = FullAccountRLP(it.(AccountIterator).Account())
|
||||
if err != nil {
|
||||
stop(false)
|
||||
return common.Hash{}, err
|
||||
}
|
||||
} else {
|
||||
account, err := FullAccount(it.(AccountIterator).Account())
|
||||
if err != nil {
|
||||
stop(false)
|
||||
return common.Hash{}, err
|
||||
}
|
||||
// Apply the leaf callback. Normally the callback is used to traverse
|
||||
// the storage trie and re-generate the subtrie root.
|
||||
subroot := leafCallback(it.Hash(), stats)
|
||||
if !bytes.Equal(account.Root, subroot.Bytes()) {
|
||||
stop(false)
|
||||
return common.Hash{}, fmt.Errorf("invalid subroot(%x), want %x, got %x", it.Hash(), account.Root, subroot)
|
||||
}
|
||||
fullData, err = rlp.EncodeToBytes(account)
|
||||
if err != nil {
|
||||
stop(false)
|
||||
return common.Hash{}, err
|
||||
}
|
||||
}
|
||||
leaf = trieKV{it.Hash(), fullData}
|
||||
} else {
|
||||
leaf = trieKV{it.Hash(), common.CopyBytes(it.(StorageIterator).Slot())}
|
||||
}
|
||||
in <- leaf
|
||||
|
||||
// Accumulate the generaation statistic if it's required.
|
||||
processed++
|
||||
if time.Since(logged) > 3*time.Second && stats != nil {
|
||||
if account == (common.Hash{}) {
|
||||
stats.progress(processed, 0, it.Hash(), common.Hash{})
|
||||
} else {
|
||||
stats.progress(0, processed, account, it.Hash())
|
||||
}
|
||||
logged, processed = time.Now(), 0
|
||||
}
|
||||
last = it.Hash()
|
||||
}
|
||||
// Commit the last part statistic.
|
||||
if processed > 0 && stats != nil {
|
||||
if account == (common.Hash{}) {
|
||||
stats.progress(processed, 0, last, common.Hash{})
|
||||
} else {
|
||||
stats.progress(0, processed, account, last)
|
||||
}
|
||||
}
|
||||
result := stop(true)
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// stdGenerate is a very basic hexary trie builder which uses the same Trie
|
||||
|
@ -105,6 +105,13 @@ type diffLayer struct {
|
||||
root common.Hash // Root hash to which this snapshot diff belongs to
|
||||
stale uint32 // Signals that the layer became stale (state progressed)
|
||||
|
||||
// destructSet is a very special helper marker. If an account is marked as
|
||||
// deleted, then it's recorded in this set. However it's allowed that an account
|
||||
// is included here but still available in other sets(e.g. storageData). The
|
||||
// reason is the diff layer includes all the changes in a *block*. It can
|
||||
// happen that in the tx_1, account A is self-destructed while in the tx_2
|
||||
// it's recreated. But we still need this marker to indicate the "old" A is
|
||||
// deleted, all data in other set belongs to the "new" A.
|
||||
destructSet map[common.Hash]struct{} // Keyed markers for deleted (and potentially) recreated accounts
|
||||
accountList []common.Hash // List of account for iteration. If it exists, it's sorted, otherwise it's nil
|
||||
accountData map[common.Hash][]byte // Keyed accounts for direct retrival (nil means deleted)
|
||||
@ -169,6 +176,7 @@ func newDiffLayer(parent snapshot, root common.Hash, destructs map[common.Hash]s
|
||||
destructSet: destructs,
|
||||
accountData: accounts,
|
||||
storageData: storage,
|
||||
storageList: make(map[common.Hash][]common.Hash),
|
||||
}
|
||||
switch parent := parent.(type) {
|
||||
case *diskLayer:
|
||||
@ -194,11 +202,6 @@ func newDiffLayer(parent snapshot, root common.Hash, destructs map[common.Hash]s
|
||||
dl.memory += uint64(common.HashLength + len(data))
|
||||
snapshotDirtyAccountWriteMeter.Mark(int64(len(data)))
|
||||
}
|
||||
// Fill the storage hashes and sort them for the iterator
|
||||
dl.storageList = make(map[common.Hash][]common.Hash)
|
||||
for accountHash := range destructs {
|
||||
dl.storageList[accountHash] = nil
|
||||
}
|
||||
// Determine memory size and track the dirty writes
|
||||
for _, slots := range storage {
|
||||
for _, data := range slots {
|
||||
@ -206,7 +209,7 @@ func newDiffLayer(parent snapshot, root common.Hash, destructs map[common.Hash]s
|
||||
snapshotDirtyStorageWriteMeter.Mark(int64(len(data)))
|
||||
}
|
||||
}
|
||||
dl.memory += uint64(len(dl.storageList) * common.HashLength)
|
||||
dl.memory += uint64(len(destructs) * common.HashLength)
|
||||
return dl
|
||||
}
|
||||
|
||||
@ -287,6 +290,8 @@ func (dl *diffLayer) Account(hash common.Hash) (*Account, error) {
|
||||
|
||||
// AccountRLP directly retrieves the account RLP associated with a particular
|
||||
// hash in the snapshot slim data format.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (dl *diffLayer) AccountRLP(hash common.Hash) ([]byte, error) {
|
||||
// Check the bloom filter first whether there's even a point in reaching into
|
||||
// all the maps in all the layers below
|
||||
@ -347,6 +352,8 @@ func (dl *diffLayer) accountRLP(hash common.Hash, depth int) ([]byte, error) {
|
||||
// Storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account. If the slot is unknown to this diff, it's parent
|
||||
// is consulted.
|
||||
//
|
||||
// Note the returned slot is not a copy, please don't modify it.
|
||||
func (dl *diffLayer) Storage(accountHash, storageHash common.Hash) ([]byte, error) {
|
||||
// Check the bloom filter first whether there's even a point in reaching into
|
||||
// all the maps in all the layers below
|
||||
@ -502,22 +509,29 @@ func (dl *diffLayer) AccountList() []common.Hash {
|
||||
}
|
||||
}
|
||||
sort.Sort(hashes(dl.accountList))
|
||||
dl.memory += uint64(len(dl.accountList) * common.HashLength)
|
||||
return dl.accountList
|
||||
}
|
||||
|
||||
// StorageList returns a sorted list of all storage slot hashes in this difflayer
|
||||
// for the given account.
|
||||
// for the given account. If the whole storage is destructed in this layer, then
|
||||
// an additional flag *destructed = true* will be returned, otherwise the flag is
|
||||
// false. Besides, the returned list will include the hash of deleted storage slot.
|
||||
// Note a special case is an account is deleted in a prior tx but is recreated in
|
||||
// the following tx with some storage slots set. In this case the returned list is
|
||||
// not empty but the flag is true.
|
||||
//
|
||||
// Note, the returned slice is not a copy, so do not modify it.
|
||||
func (dl *diffLayer) StorageList(accountHash common.Hash) []common.Hash {
|
||||
func (dl *diffLayer) StorageList(accountHash common.Hash) ([]common.Hash, bool) {
|
||||
// If an old list already exists, return it
|
||||
dl.lock.RLock()
|
||||
list := dl.storageList[accountHash]
|
||||
_, destructed := dl.destructSet[accountHash]
|
||||
if list, exist := dl.storageList[accountHash]; exist {
|
||||
dl.lock.RUnlock()
|
||||
return list, destructed // The list might be nil
|
||||
}
|
||||
dl.lock.RUnlock()
|
||||
|
||||
if list != nil {
|
||||
return list
|
||||
}
|
||||
// No old sorted account list exists, generate a new one
|
||||
dl.lock.Lock()
|
||||
defer dl.lock.Unlock()
|
||||
@ -529,5 +543,6 @@ func (dl *diffLayer) StorageList(accountHash common.Hash) []common.Hash {
|
||||
}
|
||||
sort.Sort(hashes(storageList))
|
||||
dl.storageList[accountHash] = storageList
|
||||
return storageList
|
||||
dl.memory += uint64(len(dl.storageList)*common.HashLength + common.HashLength)
|
||||
return storageList, destructed
|
||||
}
|
||||
|
@ -109,7 +109,8 @@ func TestMergeBasics(t *testing.T) {
|
||||
if have, want := len(merged.storageList), i; have != want {
|
||||
t.Errorf("[1] storageList wrong: have %v, want %v", have, want)
|
||||
}
|
||||
if have, want := len(merged.StorageList(aHash)), len(sMap); have != want {
|
||||
list, _ := merged.StorageList(aHash)
|
||||
if have, want := len(list), len(sMap); have != want {
|
||||
t.Errorf("[2] StorageList() wrong: have %v, want %v", have, want)
|
||||
}
|
||||
if have, want := len(merged.storageList[aHash]), len(sMap); have != want {
|
||||
|
@ -42,7 +42,7 @@ var (
|
||||
)
|
||||
|
||||
// generatorStats is a collection of statistics gathered by the snapshot generator
|
||||
// for logging purposes.
|
||||
// for logging purposes.
|
||||
type generatorStats struct {
|
||||
wiping chan struct{} // Notification channel if wiping is in progress
|
||||
origin uint64 // Origin prefix where generation started
|
||||
@ -167,7 +167,7 @@ func (dl *diskLayer) generate(stats *generatorStats) {
|
||||
if err := rlp.DecodeBytes(accIt.Value, &acc); err != nil {
|
||||
log.Crit("Invalid account encountered during snapshot creation", "err", err)
|
||||
}
|
||||
data := AccountRLP(acc.Nonce, acc.Balance, acc.Root, acc.CodeHash)
|
||||
data := SlimAccountRLP(acc.Nonce, acc.Balance, acc.Root, acc.CodeHash)
|
||||
|
||||
// If the account is not yet in-progress, write it out
|
||||
if accMarker == nil || !bytes.Equal(accountHash[:], accMarker) {
|
||||
|
@ -26,9 +26,9 @@ import (
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
)
|
||||
|
||||
// AccountIterator is an iterator to step over all the accounts in a snapshot,
|
||||
// which may or may npt be composed of multiple layers.
|
||||
type AccountIterator interface {
|
||||
// Iterator is a iterator to step over all the accounts or the specific
|
||||
// storage in a snapshot which may or may not be composed of multiple layers.
|
||||
type Iterator interface {
|
||||
// Next steps the iterator forward one element, returning false if exhausted,
|
||||
// or an error if iteration failed for some reason (e.g. root being iterated
|
||||
// becomes stale and garbage collected).
|
||||
@ -38,18 +38,35 @@ type AccountIterator interface {
|
||||
// caused a premature iteration exit (e.g. snapshot stack becoming stale).
|
||||
Error() error
|
||||
|
||||
// Hash returns the hash of the account the iterator is currently at.
|
||||
// Hash returns the hash of the account or storage slot the iterator is
|
||||
// currently at.
|
||||
Hash() common.Hash
|
||||
|
||||
// Account returns the RLP encoded slim account the iterator is currently at.
|
||||
// An error will be returned if the iterator becomes invalid (e.g. snaph
|
||||
Account() []byte
|
||||
|
||||
// Release releases associated resources. Release should always succeed and
|
||||
// can be called multiple times without causing error.
|
||||
Release()
|
||||
}
|
||||
|
||||
// AccountIterator is a iterator to step over all the accounts in a snapshot,
|
||||
// which may or may not be composed of multiple layers.
|
||||
type AccountIterator interface {
|
||||
Iterator
|
||||
|
||||
// Account returns the RLP encoded slim account the iterator is currently at.
|
||||
// An error will be returned if the iterator becomes invalid
|
||||
Account() []byte
|
||||
}
|
||||
|
||||
// StorageIterator is a iterator to step over the specific storage in a snapshot,
|
||||
// which may or may not be composed of multiple layers.
|
||||
type StorageIterator interface {
|
||||
Iterator
|
||||
|
||||
// Slot returns the storage slot the iterator is currently at. An error will
|
||||
// be returned if the iterator becomes invalid
|
||||
Slot() []byte
|
||||
}
|
||||
|
||||
// diffAccountIterator is an account iterator that steps over the accounts (both
|
||||
// live and deleted) contained within a single diff layer. Higher order iterators
|
||||
// will use the deleted accounts to skip deeper iterators.
|
||||
@ -120,6 +137,8 @@ func (it *diffAccountIterator) Hash() common.Hash {
|
||||
// This method assumes that flattening does not delete elements from
|
||||
// the accountdata mapping (writing nil into it is fine though), and will panic
|
||||
// if elements have been deleted.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (it *diffAccountIterator) Account() []byte {
|
||||
it.layer.lock.RLock()
|
||||
blob, ok := it.layer.accountData[it.curHash]
|
||||
@ -164,7 +183,7 @@ func (it *diskAccountIterator) Next() bool {
|
||||
}
|
||||
// Try to advance the iterator and release it if we reached the end
|
||||
for {
|
||||
if !it.it.Next() || !bytes.HasPrefix(it.it.Key(), rawdb.SnapshotAccountPrefix) {
|
||||
if !it.it.Next() {
|
||||
it.it.Release()
|
||||
it.it = nil
|
||||
return false
|
||||
@ -182,12 +201,15 @@ func (it *diskAccountIterator) Next() bool {
|
||||
// A diff layer is immutable after creation content wise and can always be fully
|
||||
// iterated without error, so this method always returns nil.
|
||||
func (it *diskAccountIterator) Error() error {
|
||||
if it.it == nil {
|
||||
return nil // Iterator is exhausted and released
|
||||
}
|
||||
return it.it.Error()
|
||||
}
|
||||
|
||||
// Hash returns the hash of the account the iterator is currently at.
|
||||
func (it *diskAccountIterator) Hash() common.Hash {
|
||||
return common.BytesToHash(it.it.Key())
|
||||
return common.BytesToHash(it.it.Key()) // The prefix will be truncated
|
||||
}
|
||||
|
||||
// Account returns the RLP encoded slim account the iterator is currently at.
|
||||
@ -203,3 +225,176 @@ func (it *diskAccountIterator) Release() {
|
||||
it.it = nil
|
||||
}
|
||||
}
|
||||
|
||||
// diffStorageIterator is a storage iterator that steps over the specific storage
|
||||
// (both live and deleted) contained within a single diff layer. Higher order
|
||||
// iterators will use the deleted slot to skip deeper iterators.
|
||||
type diffStorageIterator struct {
|
||||
// curHash is the current hash the iterator is positioned on. The field is
|
||||
// explicitly tracked since the referenced diff layer might go stale after
|
||||
// the iterator was positioned and we don't want to fail accessing the old
|
||||
// hash as long as the iterator is not touched any more.
|
||||
curHash common.Hash
|
||||
account common.Hash
|
||||
|
||||
layer *diffLayer // Live layer to retrieve values from
|
||||
keys []common.Hash // Keys left in the layer to iterate
|
||||
fail error // Any failures encountered (stale)
|
||||
}
|
||||
|
||||
// StorageIterator creates a storage iterator over a single diff layer.
|
||||
// Execept the storage iterator is returned, there is an additional flag
|
||||
// "destructed" returned. If it's true then it means the whole storage is
|
||||
// destructed in this layer(maybe recreated too), don't bother deeper layer
|
||||
// for storage retrieval.
|
||||
func (dl *diffLayer) StorageIterator(account common.Hash, seek common.Hash) (StorageIterator, bool) {
|
||||
// Create the storage for this account even it's marked
|
||||
// as destructed. The iterator is for the new one which
|
||||
// just has the same adddress as the deleted one.
|
||||
hashes, destructed := dl.StorageList(account)
|
||||
index := sort.Search(len(hashes), func(i int) bool {
|
||||
return bytes.Compare(seek[:], hashes[i][:]) <= 0
|
||||
})
|
||||
// Assemble and returned the already seeked iterator
|
||||
return &diffStorageIterator{
|
||||
layer: dl,
|
||||
account: account,
|
||||
keys: hashes[index:],
|
||||
}, destructed
|
||||
}
|
||||
|
||||
// Next steps the iterator forward one element, returning false if exhausted.
|
||||
func (it *diffStorageIterator) Next() bool {
|
||||
// If the iterator was already stale, consider it a programmer error. Although
|
||||
// we could just return false here, triggering this path would probably mean
|
||||
// somebody forgot to check for Error, so lets blow up instead of undefined
|
||||
// behavior that's hard to debug.
|
||||
if it.fail != nil {
|
||||
panic(fmt.Sprintf("called Next of failed iterator: %v", it.fail))
|
||||
}
|
||||
// Stop iterating if all keys were exhausted
|
||||
if len(it.keys) == 0 {
|
||||
return false
|
||||
}
|
||||
if it.layer.Stale() {
|
||||
it.fail, it.keys = ErrSnapshotStale, nil
|
||||
return false
|
||||
}
|
||||
// Iterator seems to be still alive, retrieve and cache the live hash
|
||||
it.curHash = it.keys[0]
|
||||
// key cached, shift the iterator and notify the user of success
|
||||
it.keys = it.keys[1:]
|
||||
return true
|
||||
}
|
||||
|
||||
// Error returns any failure that occurred during iteration, which might have
|
||||
// caused a premature iteration exit (e.g. snapshot stack becoming stale).
|
||||
func (it *diffStorageIterator) Error() error {
|
||||
return it.fail
|
||||
}
|
||||
|
||||
// Hash returns the hash of the storage slot the iterator is currently at.
|
||||
func (it *diffStorageIterator) Hash() common.Hash {
|
||||
return it.curHash
|
||||
}
|
||||
|
||||
// Slot returns the raw storage slot value the iterator is currently at.
|
||||
// This method may _fail_, if the underlying layer has been flattened between
|
||||
// the call to Next and Value. That type of error will set it.Err.
|
||||
// This method assumes that flattening does not delete elements from
|
||||
// the storage mapping (writing nil into it is fine though), and will panic
|
||||
// if elements have been deleted.
|
||||
//
|
||||
// Note the returned slot is not a copy, please don't modify it.
|
||||
func (it *diffStorageIterator) Slot() []byte {
|
||||
it.layer.lock.RLock()
|
||||
storage, ok := it.layer.storageData[it.account]
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("iterator referenced non-existent account storage: %x", it.account))
|
||||
}
|
||||
// Storage slot might be nil(deleted), but it must exist
|
||||
blob, ok := storage[it.curHash]
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("iterator referenced non-existent storage slot: %x", it.curHash))
|
||||
}
|
||||
it.layer.lock.RUnlock()
|
||||
if it.layer.Stale() {
|
||||
it.fail, it.keys = ErrSnapshotStale, nil
|
||||
}
|
||||
return blob
|
||||
}
|
||||
|
||||
// Release is a noop for diff account iterators as there are no held resources.
|
||||
func (it *diffStorageIterator) Release() {}
|
||||
|
||||
// diskStorageIterator is a storage iterator that steps over the live storage
|
||||
// contained within a disk layer.
|
||||
type diskStorageIterator struct {
|
||||
layer *diskLayer
|
||||
account common.Hash
|
||||
it ethdb.Iterator
|
||||
}
|
||||
|
||||
// StorageIterator creates a storage iterator over a disk layer.
|
||||
// If the whole storage is destructed, then all entries in the disk
|
||||
// layer are deleted already. So the "destructed" flag returned here
|
||||
// is always false.
|
||||
func (dl *diskLayer) StorageIterator(account common.Hash, seek common.Hash) (StorageIterator, bool) {
|
||||
pos := common.TrimRightZeroes(seek[:])
|
||||
return &diskStorageIterator{
|
||||
layer: dl,
|
||||
account: account,
|
||||
it: dl.diskdb.NewIterator(append(rawdb.SnapshotStoragePrefix, account.Bytes()...), pos),
|
||||
}, false
|
||||
}
|
||||
|
||||
// Next steps the iterator forward one element, returning false if exhausted.
|
||||
func (it *diskStorageIterator) Next() bool {
|
||||
// If the iterator was already exhausted, don't bother
|
||||
if it.it == nil {
|
||||
return false
|
||||
}
|
||||
// Try to advance the iterator and release it if we reached the end
|
||||
for {
|
||||
if !it.it.Next() {
|
||||
it.it.Release()
|
||||
it.it = nil
|
||||
return false
|
||||
}
|
||||
if len(it.it.Key()) == len(rawdb.SnapshotStoragePrefix)+common.HashLength+common.HashLength {
|
||||
break
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// Error returns any failure that occurred during iteration, which might have
|
||||
// caused a premature iteration exit (e.g. snapshot stack becoming stale).
|
||||
//
|
||||
// A diff layer is immutable after creation content wise and can always be fully
|
||||
// iterated without error, so this method always returns nil.
|
||||
func (it *diskStorageIterator) Error() error {
|
||||
if it.it == nil {
|
||||
return nil // Iterator is exhausted and released
|
||||
}
|
||||
return it.it.Error()
|
||||
}
|
||||
|
||||
// Hash returns the hash of the storage slot the iterator is currently at.
|
||||
func (it *diskStorageIterator) Hash() common.Hash {
|
||||
return common.BytesToHash(it.it.Key()) // The prefix will be truncated
|
||||
}
|
||||
|
||||
// Slot returns the raw strorage slot content the iterator is currently at.
|
||||
func (it *diskStorageIterator) Slot() []byte {
|
||||
return it.it.Value()
|
||||
}
|
||||
|
||||
// Release releases the database snapshot held during iteration.
|
||||
func (it *diskStorageIterator) Release() {
|
||||
// The iterator is auto-released on exhaustion, so make sure it's still alive
|
||||
if it.it != nil {
|
||||
it.it.Release()
|
||||
it.it = nil
|
||||
}
|
||||
}
|
||||
|
@ -22,35 +22,91 @@ import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
// binaryAccountIterator is a simplistic iterator to step over the accounts in
|
||||
// a snapshot, which may or may npt be composed of multiple layers. Performance
|
||||
// binaryIterator is a simplistic iterator to step over the accounts or storage
|
||||
// in a snapshot, which may or may not be composed of multiple layers. Performance
|
||||
// wise this iterator is slow, it's meant for cross validating the fast one,
|
||||
type binaryAccountIterator struct {
|
||||
a AccountIterator
|
||||
b AccountIterator
|
||||
aDone bool
|
||||
bDone bool
|
||||
k common.Hash
|
||||
fail error
|
||||
type binaryIterator struct {
|
||||
a Iterator
|
||||
b Iterator
|
||||
aDone bool
|
||||
bDone bool
|
||||
accountIterator bool
|
||||
k common.Hash
|
||||
account common.Hash
|
||||
fail error
|
||||
}
|
||||
|
||||
// newBinaryAccountIterator creates a simplistic account iterator to step over
|
||||
// all the accounts in a slow, but eaily verifiable way.
|
||||
func (dl *diffLayer) newBinaryAccountIterator() AccountIterator {
|
||||
// initBinaryAccountIterator creates a simplistic iterator to step over all the
|
||||
// accounts in a slow, but eaily verifiable way. Note this function is used for
|
||||
// initialization, use `newBinaryAccountIterator` as the API.
|
||||
func (dl *diffLayer) initBinaryAccountIterator() Iterator {
|
||||
parent, ok := dl.parent.(*diffLayer)
|
||||
if !ok {
|
||||
// parent is the disk layer
|
||||
l := &binaryAccountIterator{
|
||||
a: dl.AccountIterator(common.Hash{}),
|
||||
b: dl.Parent().AccountIterator(common.Hash{}),
|
||||
l := &binaryIterator{
|
||||
a: dl.AccountIterator(common.Hash{}),
|
||||
b: dl.Parent().AccountIterator(common.Hash{}),
|
||||
accountIterator: true,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = !l.b.Next()
|
||||
return l
|
||||
}
|
||||
l := &binaryAccountIterator{
|
||||
a: dl.AccountIterator(common.Hash{}),
|
||||
b: parent.newBinaryAccountIterator(),
|
||||
l := &binaryIterator{
|
||||
a: dl.AccountIterator(common.Hash{}),
|
||||
b: parent.initBinaryAccountIterator(),
|
||||
accountIterator: true,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = !l.b.Next()
|
||||
return l
|
||||
}
|
||||
|
||||
// initBinaryStorageIterator creates a simplistic iterator to step over all the
|
||||
// storage slots in a slow, but eaily verifiable way. Note this function is used
|
||||
// for initialization, use `newBinaryStorageIterator` as the API.
|
||||
func (dl *diffLayer) initBinaryStorageIterator(account common.Hash) Iterator {
|
||||
parent, ok := dl.parent.(*diffLayer)
|
||||
if !ok {
|
||||
// If the storage in this layer is already destructed, discard all
|
||||
// deeper layers but still return an valid single-branch iterator.
|
||||
a, destructed := dl.StorageIterator(account, common.Hash{})
|
||||
if destructed {
|
||||
l := &binaryIterator{
|
||||
a: a,
|
||||
account: account,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = true
|
||||
return l
|
||||
}
|
||||
// The parent is disk layer, don't need to take care "destructed"
|
||||
// anymore.
|
||||
b, _ := dl.Parent().StorageIterator(account, common.Hash{})
|
||||
l := &binaryIterator{
|
||||
a: a,
|
||||
b: b,
|
||||
account: account,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = !l.b.Next()
|
||||
return l
|
||||
}
|
||||
// If the storage in this layer is already destructed, discard all
|
||||
// deeper layers but still return an valid single-branch iterator.
|
||||
a, destructed := dl.StorageIterator(account, common.Hash{})
|
||||
if destructed {
|
||||
l := &binaryIterator{
|
||||
a: a,
|
||||
account: account,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = true
|
||||
return l
|
||||
}
|
||||
l := &binaryIterator{
|
||||
a: a,
|
||||
b: parent.initBinaryStorageIterator(account),
|
||||
account: account,
|
||||
}
|
||||
l.aDone = !l.a.Next()
|
||||
l.bDone = !l.b.Next()
|
||||
@ -60,7 +116,7 @@ func (dl *diffLayer) newBinaryAccountIterator() AccountIterator {
|
||||
// Next steps the iterator forward one element, returning false if exhausted,
|
||||
// or an error if iteration failed for some reason (e.g. root being iterated
|
||||
// becomes stale and garbage collected).
|
||||
func (it *binaryAccountIterator) Next() bool {
|
||||
func (it *binaryIterator) Next() bool {
|
||||
if it.aDone && it.bDone {
|
||||
return false
|
||||
}
|
||||
@ -92,19 +148,24 @@ first:
|
||||
|
||||
// Error returns any failure that occurred during iteration, which might have
|
||||
// caused a premature iteration exit (e.g. snapshot stack becoming stale).
|
||||
func (it *binaryAccountIterator) Error() error {
|
||||
func (it *binaryIterator) Error() error {
|
||||
return it.fail
|
||||
}
|
||||
|
||||
// Hash returns the hash of the account the iterator is currently at.
|
||||
func (it *binaryAccountIterator) Hash() common.Hash {
|
||||
func (it *binaryIterator) Hash() common.Hash {
|
||||
return it.k
|
||||
}
|
||||
|
||||
// Account returns the RLP encoded slim account the iterator is currently at, or
|
||||
// nil if the iterated snapshot stack became stale (you can check Error after
|
||||
// to see if it failed or not).
|
||||
func (it *binaryAccountIterator) Account() []byte {
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (it *binaryIterator) Account() []byte {
|
||||
if !it.accountIterator {
|
||||
return nil
|
||||
}
|
||||
// The topmost iterator must be `diffAccountIterator`
|
||||
blob, err := it.a.(*diffAccountIterator).layer.AccountRLP(it.k)
|
||||
if err != nil {
|
||||
@ -114,8 +175,39 @@ func (it *binaryAccountIterator) Account() []byte {
|
||||
return blob
|
||||
}
|
||||
|
||||
// Slot returns the raw storage slot data the iterator is currently at, or
|
||||
// nil if the iterated snapshot stack became stale (you can check Error after
|
||||
// to see if it failed or not).
|
||||
//
|
||||
// Note the returned slot is not a copy, please don't modify it.
|
||||
func (it *binaryIterator) Slot() []byte {
|
||||
if it.accountIterator {
|
||||
return nil
|
||||
}
|
||||
blob, err := it.a.(*diffStorageIterator).layer.Storage(it.account, it.k)
|
||||
if err != nil {
|
||||
it.fail = err
|
||||
return nil
|
||||
}
|
||||
return blob
|
||||
}
|
||||
|
||||
// Release recursively releases all the iterators in the stack.
|
||||
func (it *binaryAccountIterator) Release() {
|
||||
func (it *binaryIterator) Release() {
|
||||
it.a.Release()
|
||||
it.b.Release()
|
||||
}
|
||||
|
||||
// newBinaryAccountIterator creates a simplistic account iterator to step over
|
||||
// all the accounts in a slow, but eaily verifiable way.
|
||||
func (dl *diffLayer) newBinaryAccountIterator() AccountIterator {
|
||||
iter := dl.initBinaryAccountIterator()
|
||||
return iter.(AccountIterator)
|
||||
}
|
||||
|
||||
// newBinaryStorageIterator creates a simplistic account iterator to step over
|
||||
// all the storage slots in a slow, but eaily verifiable way.
|
||||
func (dl *diffLayer) newBinaryStorageIterator(account common.Hash) StorageIterator {
|
||||
iter := dl.initBinaryStorageIterator(account)
|
||||
return iter.(StorageIterator)
|
||||
}
|
||||
|
@ -24,23 +24,23 @@ import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
// weightedAccountIterator is an account iterator with an assigned weight. It is
|
||||
// used to prioritise which account is the correct one if multiple iterators find
|
||||
// the same one (modified in multiple consecutive blocks).
|
||||
type weightedAccountIterator struct {
|
||||
it AccountIterator
|
||||
// weightedIterator is a iterator with an assigned weight. It is used to prioritise
|
||||
// which account or storage slot is the correct one if multiple iterators find the
|
||||
// same one (modified in multiple consecutive blocks).
|
||||
type weightedIterator struct {
|
||||
it Iterator
|
||||
priority int
|
||||
}
|
||||
|
||||
// weightedAccountIterators is a set of iterators implementing the sort.Interface.
|
||||
type weightedAccountIterators []*weightedAccountIterator
|
||||
// weightedIterators is a set of iterators implementing the sort.Interface.
|
||||
type weightedIterators []*weightedIterator
|
||||
|
||||
// Len implements sort.Interface, returning the number of active iterators.
|
||||
func (its weightedAccountIterators) Len() int { return len(its) }
|
||||
func (its weightedIterators) Len() int { return len(its) }
|
||||
|
||||
// Less implements sort.Interface, returning which of two iterators in the stack
|
||||
// is before the other.
|
||||
func (its weightedAccountIterators) Less(i, j int) bool {
|
||||
func (its weightedIterators) Less(i, j int) bool {
|
||||
// Order the iterators primarily by the account hashes
|
||||
hashI := its[i].it.Hash()
|
||||
hashJ := its[j].it.Hash()
|
||||
@ -51,45 +51,64 @@ func (its weightedAccountIterators) Less(i, j int) bool {
|
||||
case 1:
|
||||
return false
|
||||
}
|
||||
// Same account in multiple layers, split by priority
|
||||
// Same account/storage-slot in multiple layers, split by priority
|
||||
return its[i].priority < its[j].priority
|
||||
}
|
||||
|
||||
// Swap implements sort.Interface, swapping two entries in the iterator stack.
|
||||
func (its weightedAccountIterators) Swap(i, j int) {
|
||||
func (its weightedIterators) Swap(i, j int) {
|
||||
its[i], its[j] = its[j], its[i]
|
||||
}
|
||||
|
||||
// fastAccountIterator is a more optimized multi-layer iterator which maintains a
|
||||
// fastIterator is a more optimized multi-layer iterator which maintains a
|
||||
// direct mapping of all iterators leading down to the bottom layer.
|
||||
type fastAccountIterator struct {
|
||||
tree *Tree // Snapshot tree to reinitialize stale sub-iterators with
|
||||
root common.Hash // Root hash to reinitialize stale sub-iterators through
|
||||
curAccount []byte
|
||||
type fastIterator struct {
|
||||
tree *Tree // Snapshot tree to reinitialize stale sub-iterators with
|
||||
root common.Hash // Root hash to reinitialize stale sub-iterators through
|
||||
|
||||
iterators weightedAccountIterators
|
||||
curAccount []byte
|
||||
curSlot []byte
|
||||
|
||||
iterators weightedIterators
|
||||
initiated bool
|
||||
account bool
|
||||
fail error
|
||||
}
|
||||
|
||||
// newFastAccountIterator creates a new hierarhical account iterator with one
|
||||
// newFastIterator creates a new hierarhical account or storage iterator with one
|
||||
// element per diff layer. The returned combo iterator can be used to walk over
|
||||
// the entire snapshot diff stack simultaneously.
|
||||
func newFastAccountIterator(tree *Tree, root common.Hash, seek common.Hash) (AccountIterator, error) {
|
||||
func newFastIterator(tree *Tree, root common.Hash, account common.Hash, seek common.Hash, accountIterator bool) (*fastIterator, error) {
|
||||
snap := tree.Snapshot(root)
|
||||
if snap == nil {
|
||||
return nil, fmt.Errorf("unknown snapshot: %x", root)
|
||||
}
|
||||
fi := &fastAccountIterator{
|
||||
tree: tree,
|
||||
root: root,
|
||||
fi := &fastIterator{
|
||||
tree: tree,
|
||||
root: root,
|
||||
account: accountIterator,
|
||||
}
|
||||
current := snap.(snapshot)
|
||||
for depth := 0; current != nil; depth++ {
|
||||
fi.iterators = append(fi.iterators, &weightedAccountIterator{
|
||||
it: current.AccountIterator(seek),
|
||||
priority: depth,
|
||||
})
|
||||
if accountIterator {
|
||||
fi.iterators = append(fi.iterators, &weightedIterator{
|
||||
it: current.AccountIterator(seek),
|
||||
priority: depth,
|
||||
})
|
||||
} else {
|
||||
// If the whole storage is destructed in this layer, don't
|
||||
// bother deeper layer anymore. But we should still keep
|
||||
// the iterator for this layer, since the iterator can contain
|
||||
// some valid slots which belongs to the re-created account.
|
||||
it, destructed := current.StorageIterator(account, seek)
|
||||
fi.iterators = append(fi.iterators, &weightedIterator{
|
||||
it: it,
|
||||
priority: depth,
|
||||
})
|
||||
if destructed {
|
||||
break
|
||||
}
|
||||
}
|
||||
current = current.Parent()
|
||||
}
|
||||
fi.init()
|
||||
@ -98,7 +117,7 @@ func newFastAccountIterator(tree *Tree, root common.Hash, seek common.Hash) (Acc
|
||||
|
||||
// init walks over all the iterators and resolves any clashes between them, after
|
||||
// which it prepares the stack for step-by-step iteration.
|
||||
func (fi *fastAccountIterator) init() {
|
||||
func (fi *fastIterator) init() {
|
||||
// Track which account hashes are iterators positioned on
|
||||
var positioned = make(map[common.Hash]int)
|
||||
|
||||
@ -153,7 +172,7 @@ func (fi *fastAccountIterator) init() {
|
||||
}
|
||||
|
||||
// Next steps the iterator forward one element, returning false if exhausted.
|
||||
func (fi *fastAccountIterator) Next() bool {
|
||||
func (fi *fastIterator) Next() bool {
|
||||
if len(fi.iterators) == 0 {
|
||||
return false
|
||||
}
|
||||
@ -161,21 +180,25 @@ func (fi *fastAccountIterator) Next() bool {
|
||||
// Don't forward first time -- we had to 'Next' once in order to
|
||||
// do the sorting already
|
||||
fi.initiated = true
|
||||
fi.curAccount = fi.iterators[0].it.Account()
|
||||
if fi.account {
|
||||
fi.curAccount = fi.iterators[0].it.(AccountIterator).Account()
|
||||
} else {
|
||||
fi.curSlot = fi.iterators[0].it.(StorageIterator).Slot()
|
||||
}
|
||||
if innerErr := fi.iterators[0].it.Error(); innerErr != nil {
|
||||
fi.fail = innerErr
|
||||
return false
|
||||
}
|
||||
if fi.curAccount != nil {
|
||||
if fi.curAccount != nil || fi.curSlot != nil {
|
||||
return true
|
||||
}
|
||||
// Implicit else: we've hit a nil-account, and need to fall through to the
|
||||
// loop below to land on something non-nil
|
||||
// Implicit else: we've hit a nil-account or nil-slot, and need to
|
||||
// fall through to the loop below to land on something non-nil
|
||||
}
|
||||
// If an account is deleted in one of the layers, the key will still be there,
|
||||
// but the actual value will be nil. However, the iterator should not
|
||||
// export nil-values (but instead simply omit the key), so we need to loop
|
||||
// here until we either
|
||||
// If an account or a slot is deleted in one of the layers, the key will
|
||||
// still be there, but the actual value will be nil. However, the iterator
|
||||
// should not export nil-values (but instead simply omit the key), so we
|
||||
// need to loop here until we either
|
||||
// - get a non-nil value,
|
||||
// - hit an error,
|
||||
// - or exhaust the iterator
|
||||
@ -183,12 +206,16 @@ func (fi *fastAccountIterator) Next() bool {
|
||||
if !fi.next(0) {
|
||||
return false // exhausted
|
||||
}
|
||||
fi.curAccount = fi.iterators[0].it.Account()
|
||||
if fi.account {
|
||||
fi.curAccount = fi.iterators[0].it.(AccountIterator).Account()
|
||||
} else {
|
||||
fi.curSlot = fi.iterators[0].it.(StorageIterator).Slot()
|
||||
}
|
||||
if innerErr := fi.iterators[0].it.Error(); innerErr != nil {
|
||||
fi.fail = innerErr
|
||||
return false // error
|
||||
}
|
||||
if fi.curAccount != nil {
|
||||
if fi.curAccount != nil || fi.curSlot != nil {
|
||||
break // non-nil value found
|
||||
}
|
||||
}
|
||||
@ -201,7 +228,7 @@ func (fi *fastAccountIterator) Next() bool {
|
||||
// For example, if the iterated hashes become [2,3,5,5,8,9,10], then we should
|
||||
// invoke next(3), which will call Next on elem 3 (the second '5') and will
|
||||
// cascade along the list, applying the same operation if needed.
|
||||
func (fi *fastAccountIterator) next(idx int) bool {
|
||||
func (fi *fastIterator) next(idx int) bool {
|
||||
// If this particular iterator got exhausted, remove it and return true (the
|
||||
// next one is surely not exhausted yet, otherwise it would have been removed
|
||||
// already).
|
||||
@ -262,7 +289,7 @@ func (fi *fastAccountIterator) next(idx int) bool {
|
||||
}
|
||||
|
||||
// move advances an iterator to another position in the list.
|
||||
func (fi *fastAccountIterator) move(index, newpos int) {
|
||||
func (fi *fastIterator) move(index, newpos int) {
|
||||
elem := fi.iterators[index]
|
||||
copy(fi.iterators[index:], fi.iterators[index+1:newpos+1])
|
||||
fi.iterators[newpos] = elem
|
||||
@ -270,23 +297,30 @@ func (fi *fastAccountIterator) move(index, newpos int) {
|
||||
|
||||
// Error returns any failure that occurred during iteration, which might have
|
||||
// caused a premature iteration exit (e.g. snapshot stack becoming stale).
|
||||
func (fi *fastAccountIterator) Error() error {
|
||||
func (fi *fastIterator) Error() error {
|
||||
return fi.fail
|
||||
}
|
||||
|
||||
// Hash returns the current key
|
||||
func (fi *fastAccountIterator) Hash() common.Hash {
|
||||
func (fi *fastIterator) Hash() common.Hash {
|
||||
return fi.iterators[0].it.Hash()
|
||||
}
|
||||
|
||||
// Account returns the current key
|
||||
func (fi *fastAccountIterator) Account() []byte {
|
||||
// Account returns the current account blob.
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (fi *fastIterator) Account() []byte {
|
||||
return fi.curAccount
|
||||
}
|
||||
|
||||
// Slot returns the current storage slot.
|
||||
// Note the returned slot is not a copy, please don't modify it.
|
||||
func (fi *fastIterator) Slot() []byte {
|
||||
return fi.curSlot
|
||||
}
|
||||
|
||||
// Release iterates over all the remaining live layer iterators and releases each
|
||||
// of thme individually.
|
||||
func (fi *fastAccountIterator) Release() {
|
||||
func (fi *fastIterator) Release() {
|
||||
for _, it := range fi.iterators {
|
||||
it.it.Release()
|
||||
}
|
||||
@ -294,9 +328,23 @@ func (fi *fastAccountIterator) Release() {
|
||||
}
|
||||
|
||||
// Debug is a convencience helper during testing
|
||||
func (fi *fastAccountIterator) Debug() {
|
||||
func (fi *fastIterator) Debug() {
|
||||
for _, it := range fi.iterators {
|
||||
fmt.Printf("[p=%v v=%v] ", it.priority, it.it.Hash()[0])
|
||||
}
|
||||
fmt.Println()
|
||||
}
|
||||
|
||||
// newFastAccountIterator creates a new hierarhical account iterator with one
|
||||
// element per diff layer. The returned combo iterator can be used to walk over
|
||||
// the entire snapshot diff stack simultaneously.
|
||||
func newFastAccountIterator(tree *Tree, root common.Hash, seek common.Hash) (AccountIterator, error) {
|
||||
return newFastIterator(tree, root, common.Hash{}, seek, true)
|
||||
}
|
||||
|
||||
// newFastStorageIterator creates a new hierarhical storage iterator with one
|
||||
// element per diff layer. The returned combo iterator can be used to walk over
|
||||
// the entire snapshot diff stack simultaneously.
|
||||
func newFastStorageIterator(tree *Tree, root common.Hash, account common.Hash, seek common.Hash) (StorageIterator, error) {
|
||||
return newFastIterator(tree, root, account, seek, false)
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
)
|
||||
|
||||
// TestAccountIteratorBasics tests some simple single-layer iteration
|
||||
// TestAccountIteratorBasics tests some simple single-layer(diff and disk) iteration
|
||||
func TestAccountIteratorBasics(t *testing.T) {
|
||||
var (
|
||||
destructs = make(map[common.Hash]struct{})
|
||||
@ -53,9 +53,55 @@ func TestAccountIteratorBasics(t *testing.T) {
|
||||
}
|
||||
}
|
||||
// Add some (identical) layers on top
|
||||
parent := newDiffLayer(emptyLayer(), common.Hash{}, copyDestructs(destructs), copyAccounts(accounts), copyStorage(storage))
|
||||
it := parent.AccountIterator(common.Hash{})
|
||||
verifyIterator(t, 100, it)
|
||||
diffLayer := newDiffLayer(emptyLayer(), common.Hash{}, copyDestructs(destructs), copyAccounts(accounts), copyStorage(storage))
|
||||
it := diffLayer.AccountIterator(common.Hash{})
|
||||
verifyIterator(t, 100, it, verifyNothing) // Nil is allowed for single layer iterator
|
||||
|
||||
diskLayer := diffToDisk(diffLayer)
|
||||
it = diskLayer.AccountIterator(common.Hash{})
|
||||
verifyIterator(t, 100, it, verifyNothing) // Nil is allowed for single layer iterator
|
||||
}
|
||||
|
||||
// TestStorageIteratorBasics tests some simple single-layer(diff and disk) iteration for storage
|
||||
func TestStorageIteratorBasics(t *testing.T) {
|
||||
var (
|
||||
nilStorage = make(map[common.Hash]int)
|
||||
accounts = make(map[common.Hash][]byte)
|
||||
storage = make(map[common.Hash]map[common.Hash][]byte)
|
||||
)
|
||||
// Fill some random data
|
||||
for i := 0; i < 10; i++ {
|
||||
h := randomHash()
|
||||
accounts[h] = randomAccount()
|
||||
|
||||
accStorage := make(map[common.Hash][]byte)
|
||||
value := make([]byte, 32)
|
||||
|
||||
var nilstorage int
|
||||
for i := 0; i < 100; i++ {
|
||||
rand.Read(value)
|
||||
if rand.Intn(2) == 0 {
|
||||
accStorage[randomHash()] = common.CopyBytes(value)
|
||||
} else {
|
||||
accStorage[randomHash()] = nil // delete slot
|
||||
nilstorage += 1
|
||||
}
|
||||
}
|
||||
storage[h] = accStorage
|
||||
nilStorage[h] = nilstorage
|
||||
}
|
||||
// Add some (identical) layers on top
|
||||
diffLayer := newDiffLayer(emptyLayer(), common.Hash{}, nil, copyAccounts(accounts), copyStorage(storage))
|
||||
for account := range accounts {
|
||||
it, _ := diffLayer.StorageIterator(account, common.Hash{})
|
||||
verifyIterator(t, 100, it, verifyNothing) // Nil is allowed for single layer iterator
|
||||
}
|
||||
|
||||
diskLayer := diffToDisk(diffLayer)
|
||||
for account := range accounts {
|
||||
it, _ := diskLayer.StorageIterator(account, common.Hash{})
|
||||
verifyIterator(t, 100-nilStorage[account], it, verifyNothing) // Nil is allowed for single layer iterator
|
||||
}
|
||||
}
|
||||
|
||||
type testIterator struct {
|
||||
@ -87,6 +133,10 @@ func (ti *testIterator) Account() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ti *testIterator) Slot() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ti *testIterator) Release() {}
|
||||
|
||||
func TestFastIteratorBasics(t *testing.T) {
|
||||
@ -102,13 +152,12 @@ func TestFastIteratorBasics(t *testing.T) {
|
||||
{9, 10}, {10, 13, 15, 16}},
|
||||
expKeys: []byte{0, 1, 2, 7, 8, 9, 10, 13, 14, 15, 16}},
|
||||
} {
|
||||
var iterators []*weightedAccountIterator
|
||||
var iterators []*weightedIterator
|
||||
for i, data := range tc.lists {
|
||||
it := newTestIterator(data...)
|
||||
iterators = append(iterators, &weightedAccountIterator{it, i})
|
||||
|
||||
iterators = append(iterators, &weightedIterator{it, i})
|
||||
}
|
||||
fi := &fastAccountIterator{
|
||||
fi := &fastIterator{
|
||||
iterators: iterators,
|
||||
initiated: false,
|
||||
}
|
||||
@ -122,7 +171,15 @@ func TestFastIteratorBasics(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func verifyIterator(t *testing.T, expCount int, it AccountIterator) {
|
||||
type verifyContent int
|
||||
|
||||
const (
|
||||
verifyNothing verifyContent = iota
|
||||
verifyAccount
|
||||
verifyStorage
|
||||
)
|
||||
|
||||
func verifyIterator(t *testing.T, expCount int, it Iterator, verify verifyContent) {
|
||||
t.Helper()
|
||||
|
||||
var (
|
||||
@ -134,10 +191,13 @@ func verifyIterator(t *testing.T, expCount int, it AccountIterator) {
|
||||
if bytes.Compare(last[:], hash[:]) >= 0 {
|
||||
t.Errorf("wrong order: %x >= %x", last, hash)
|
||||
}
|
||||
if it.Account() == nil {
|
||||
count++
|
||||
if verify == verifyAccount && len(it.(AccountIterator).Account()) == 0 {
|
||||
t.Errorf("iterator returned nil-value for hash %x", hash)
|
||||
} else if verify == verifyStorage && len(it.(StorageIterator).Slot()) == 0 {
|
||||
t.Errorf("iterator returned nil-value for hash %x", hash)
|
||||
}
|
||||
count++
|
||||
last = hash
|
||||
}
|
||||
if count != expCount {
|
||||
t.Errorf("iterator count mismatch: have %d, want %d", count, expCount)
|
||||
@ -173,11 +233,11 @@ func TestAccountIteratorTraversal(t *testing.T) {
|
||||
// Verify the single and multi-layer iterators
|
||||
head := snaps.Snapshot(common.HexToHash("0x04"))
|
||||
|
||||
verifyIterator(t, 3, head.(snapshot).AccountIterator(common.Hash{}))
|
||||
verifyIterator(t, 7, head.(*diffLayer).newBinaryAccountIterator())
|
||||
verifyIterator(t, 3, head.(snapshot).AccountIterator(common.Hash{}), verifyNothing)
|
||||
verifyIterator(t, 7, head.(*diffLayer).newBinaryAccountIterator(), verifyAccount)
|
||||
|
||||
it, _ := snaps.AccountIterator(common.HexToHash("0x04"), common.Hash{})
|
||||
verifyIterator(t, 7, it)
|
||||
verifyIterator(t, 7, it, verifyAccount)
|
||||
it.Release()
|
||||
|
||||
// Test after persist some bottom-most layers into the disk,
|
||||
@ -188,10 +248,58 @@ func TestAccountIteratorTraversal(t *testing.T) {
|
||||
}()
|
||||
aggregatorMemoryLimit = 0 // Force pushing the bottom-most layer into disk
|
||||
snaps.Cap(common.HexToHash("0x04"), 2)
|
||||
verifyIterator(t, 7, head.(*diffLayer).newBinaryAccountIterator())
|
||||
verifyIterator(t, 7, head.(*diffLayer).newBinaryAccountIterator(), verifyAccount)
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.Hash{})
|
||||
verifyIterator(t, 7, it)
|
||||
verifyIterator(t, 7, it, verifyAccount)
|
||||
it.Release()
|
||||
}
|
||||
|
||||
func TestStorageIteratorTraversal(t *testing.T) {
|
||||
// Create an empty base layer and a snapshot tree out of it
|
||||
base := &diskLayer{
|
||||
diskdb: rawdb.NewMemoryDatabase(),
|
||||
root: common.HexToHash("0x01"),
|
||||
cache: fastcache.New(1024 * 500),
|
||||
}
|
||||
snaps := &Tree{
|
||||
layers: map[common.Hash]snapshot{
|
||||
base.root: base,
|
||||
},
|
||||
}
|
||||
// Stack three diff layers on top with various overlaps
|
||||
snaps.Update(common.HexToHash("0x02"), common.HexToHash("0x01"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x01", "0x02", "0x03"}}, nil))
|
||||
|
||||
snaps.Update(common.HexToHash("0x03"), common.HexToHash("0x02"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x04", "0x05", "0x06"}}, nil))
|
||||
|
||||
snaps.Update(common.HexToHash("0x04"), common.HexToHash("0x03"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x01", "0x02", "0x03"}}, nil))
|
||||
|
||||
// Verify the single and multi-layer iterators
|
||||
head := snaps.Snapshot(common.HexToHash("0x04"))
|
||||
|
||||
diffIter, _ := head.(snapshot).StorageIterator(common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 3, diffIter, verifyNothing)
|
||||
verifyIterator(t, 6, head.(*diffLayer).newBinaryStorageIterator(common.HexToHash("0xaa")), verifyStorage)
|
||||
|
||||
it, _ := snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 6, it, verifyStorage)
|
||||
it.Release()
|
||||
|
||||
// Test after persist some bottom-most layers into the disk,
|
||||
// the functionalities still work.
|
||||
limit := aggregatorMemoryLimit
|
||||
defer func() {
|
||||
aggregatorMemoryLimit = limit
|
||||
}()
|
||||
aggregatorMemoryLimit = 0 // Force pushing the bottom-most layer into disk
|
||||
snaps.Cap(common.HexToHash("0x04"), 2)
|
||||
verifyIterator(t, 6, head.(*diffLayer).newBinaryStorageIterator(common.HexToHash("0xaa")), verifyStorage)
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 6, it, verifyStorage)
|
||||
it.Release()
|
||||
}
|
||||
|
||||
@ -291,6 +399,105 @@ func TestAccountIteratorTraversalValues(t *testing.T) {
|
||||
it.Release()
|
||||
}
|
||||
|
||||
func TestStorageIteratorTraversalValues(t *testing.T) {
|
||||
// Create an empty base layer and a snapshot tree out of it
|
||||
base := &diskLayer{
|
||||
diskdb: rawdb.NewMemoryDatabase(),
|
||||
root: common.HexToHash("0x01"),
|
||||
cache: fastcache.New(1024 * 500),
|
||||
}
|
||||
snaps := &Tree{
|
||||
layers: map[common.Hash]snapshot{
|
||||
base.root: base,
|
||||
},
|
||||
}
|
||||
wrapStorage := func(storage map[common.Hash][]byte) map[common.Hash]map[common.Hash][]byte {
|
||||
return map[common.Hash]map[common.Hash][]byte{
|
||||
common.HexToHash("0xaa"): storage,
|
||||
}
|
||||
}
|
||||
// Create a batch of storage sets to seed subsequent layers with
|
||||
var (
|
||||
a = make(map[common.Hash][]byte)
|
||||
b = make(map[common.Hash][]byte)
|
||||
c = make(map[common.Hash][]byte)
|
||||
d = make(map[common.Hash][]byte)
|
||||
e = make(map[common.Hash][]byte)
|
||||
f = make(map[common.Hash][]byte)
|
||||
g = make(map[common.Hash][]byte)
|
||||
h = make(map[common.Hash][]byte)
|
||||
)
|
||||
for i := byte(2); i < 0xff; i++ {
|
||||
a[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 0, i))
|
||||
if i > 20 && i%2 == 0 {
|
||||
b[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 1, i))
|
||||
}
|
||||
if i%4 == 0 {
|
||||
c[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 2, i))
|
||||
}
|
||||
if i%7 == 0 {
|
||||
d[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 3, i))
|
||||
}
|
||||
if i%8 == 0 {
|
||||
e[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 4, i))
|
||||
}
|
||||
if i > 50 || i < 85 {
|
||||
f[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 5, i))
|
||||
}
|
||||
if i%64 == 0 {
|
||||
g[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 6, i))
|
||||
}
|
||||
if i%128 == 0 {
|
||||
h[common.Hash{i}] = []byte(fmt.Sprintf("layer-%d, key %d", 7, i))
|
||||
}
|
||||
}
|
||||
// Assemble a stack of snapshots from the account layers
|
||||
snaps.Update(common.HexToHash("0x02"), common.HexToHash("0x01"), nil, randomAccountSet("0xaa"), wrapStorage(a))
|
||||
snaps.Update(common.HexToHash("0x03"), common.HexToHash("0x02"), nil, randomAccountSet("0xaa"), wrapStorage(b))
|
||||
snaps.Update(common.HexToHash("0x04"), common.HexToHash("0x03"), nil, randomAccountSet("0xaa"), wrapStorage(c))
|
||||
snaps.Update(common.HexToHash("0x05"), common.HexToHash("0x04"), nil, randomAccountSet("0xaa"), wrapStorage(d))
|
||||
snaps.Update(common.HexToHash("0x06"), common.HexToHash("0x05"), nil, randomAccountSet("0xaa"), wrapStorage(e))
|
||||
snaps.Update(common.HexToHash("0x07"), common.HexToHash("0x06"), nil, randomAccountSet("0xaa"), wrapStorage(e))
|
||||
snaps.Update(common.HexToHash("0x08"), common.HexToHash("0x07"), nil, randomAccountSet("0xaa"), wrapStorage(g))
|
||||
snaps.Update(common.HexToHash("0x09"), common.HexToHash("0x08"), nil, randomAccountSet("0xaa"), wrapStorage(h))
|
||||
|
||||
it, _ := snaps.StorageIterator(common.HexToHash("0x09"), common.HexToHash("0xaa"), common.Hash{})
|
||||
head := snaps.Snapshot(common.HexToHash("0x09"))
|
||||
for it.Next() {
|
||||
hash := it.Hash()
|
||||
want, err := head.Storage(common.HexToHash("0xaa"), hash)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to retrieve expected storage slot: %v", err)
|
||||
}
|
||||
if have := it.Slot(); !bytes.Equal(want, have) {
|
||||
t.Fatalf("hash %x: slot mismatch: have %x, want %x", hash, have, want)
|
||||
}
|
||||
}
|
||||
it.Release()
|
||||
|
||||
// Test after persist some bottom-most layers into the disk,
|
||||
// the functionalities still work.
|
||||
limit := aggregatorMemoryLimit
|
||||
defer func() {
|
||||
aggregatorMemoryLimit = limit
|
||||
}()
|
||||
aggregatorMemoryLimit = 0 // Force pushing the bottom-most layer into disk
|
||||
snaps.Cap(common.HexToHash("0x09"), 2)
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x09"), common.HexToHash("0xaa"), common.Hash{})
|
||||
for it.Next() {
|
||||
hash := it.Hash()
|
||||
want, err := head.Storage(common.HexToHash("0xaa"), hash)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to retrieve expected slot: %v", err)
|
||||
}
|
||||
if have := it.Slot(); !bytes.Equal(want, have) {
|
||||
t.Fatalf("hash %x: slot mismatch: have %x, want %x", hash, have, want)
|
||||
}
|
||||
}
|
||||
it.Release()
|
||||
}
|
||||
|
||||
// This testcase is notorious, all layers contain the exact same 200 accounts.
|
||||
func TestAccountIteratorLargeTraversal(t *testing.T) {
|
||||
// Create a custom account factory to recreate the same addresses
|
||||
@ -319,11 +526,11 @@ func TestAccountIteratorLargeTraversal(t *testing.T) {
|
||||
}
|
||||
// Iterate the entire stack and ensure everything is hit only once
|
||||
head := snaps.Snapshot(common.HexToHash("0x80"))
|
||||
verifyIterator(t, 200, head.(snapshot).AccountIterator(common.Hash{}))
|
||||
verifyIterator(t, 200, head.(*diffLayer).newBinaryAccountIterator())
|
||||
verifyIterator(t, 200, head.(snapshot).AccountIterator(common.Hash{}), verifyNothing)
|
||||
verifyIterator(t, 200, head.(*diffLayer).newBinaryAccountIterator(), verifyAccount)
|
||||
|
||||
it, _ := snaps.AccountIterator(common.HexToHash("0x80"), common.Hash{})
|
||||
verifyIterator(t, 200, it)
|
||||
verifyIterator(t, 200, it, verifyAccount)
|
||||
it.Release()
|
||||
|
||||
// Test after persist some bottom-most layers into the disk,
|
||||
@ -335,10 +542,10 @@ func TestAccountIteratorLargeTraversal(t *testing.T) {
|
||||
aggregatorMemoryLimit = 0 // Force pushing the bottom-most layer into disk
|
||||
snaps.Cap(common.HexToHash("0x80"), 2)
|
||||
|
||||
verifyIterator(t, 200, head.(*diffLayer).newBinaryAccountIterator())
|
||||
verifyIterator(t, 200, head.(*diffLayer).newBinaryAccountIterator(), verifyAccount)
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x80"), common.Hash{})
|
||||
verifyIterator(t, 200, it)
|
||||
verifyIterator(t, 200, it, verifyAccount)
|
||||
it.Release()
|
||||
}
|
||||
|
||||
@ -406,46 +613,105 @@ func TestAccountIteratorSeek(t *testing.T) {
|
||||
// Construct various iterators and ensure their traversal is correct
|
||||
it, _ := snaps.AccountIterator(common.HexToHash("0x02"), common.HexToHash("0xdd"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 3, it) // expected: ee, f0, ff
|
||||
verifyIterator(t, 3, it, verifyAccount) // expected: ee, f0, ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x02"), common.HexToHash("0xaa"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 4, it) // expected: aa, ee, f0, ff
|
||||
verifyIterator(t, 4, it, verifyAccount) // expected: aa, ee, f0, ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x02"), common.HexToHash("0xff"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 1, it) // expected: ff
|
||||
verifyIterator(t, 1, it, verifyAccount) // expected: ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x02"), common.HexToHash("0xff1"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 0, it) // expected: nothing
|
||||
verifyIterator(t, 0, it, verifyAccount) // expected: nothing
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.HexToHash("0xbb"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 6, it) // expected: bb, cc, dd, ee, f0, ff
|
||||
verifyIterator(t, 6, it, verifyAccount) // expected: bb, cc, dd, ee, f0, ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.HexToHash("0xef"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 2, it) // expected: f0, ff
|
||||
verifyIterator(t, 2, it, verifyAccount) // expected: f0, ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.HexToHash("0xf0"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 2, it) // expected: f0, ff
|
||||
verifyIterator(t, 2, it, verifyAccount) // expected: f0, ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.HexToHash("0xff"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 1, it) // expected: ff
|
||||
verifyIterator(t, 1, it, verifyAccount) // expected: ff
|
||||
|
||||
it, _ = snaps.AccountIterator(common.HexToHash("0x04"), common.HexToHash("0xff1"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 0, it) // expected: nothing
|
||||
|
||||
verifyIterator(t, 0, it, verifyAccount) // expected: nothing
|
||||
}
|
||||
|
||||
// TestIteratorDeletions tests that the iterator behaves correct when there are
|
||||
func TestStorageIteratorSeek(t *testing.T) {
|
||||
// Create a snapshot stack with some initial data
|
||||
base := &diskLayer{
|
||||
diskdb: rawdb.NewMemoryDatabase(),
|
||||
root: common.HexToHash("0x01"),
|
||||
cache: fastcache.New(1024 * 500),
|
||||
}
|
||||
snaps := &Tree{
|
||||
layers: map[common.Hash]snapshot{
|
||||
base.root: base,
|
||||
},
|
||||
}
|
||||
// Stack three diff layers on top with various overlaps
|
||||
snaps.Update(common.HexToHash("0x02"), common.HexToHash("0x01"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x01", "0x03", "0x05"}}, nil))
|
||||
|
||||
snaps.Update(common.HexToHash("0x03"), common.HexToHash("0x02"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x02", "0x05", "0x06"}}, nil))
|
||||
|
||||
snaps.Update(common.HexToHash("0x04"), common.HexToHash("0x03"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x01", "0x05", "0x08"}}, nil))
|
||||
|
||||
// Account set is now
|
||||
// 02: 01, 03, 05
|
||||
// 03: 01, 02, 03, 05 (, 05), 06
|
||||
// 04: 01(, 01), 02, 03, 05(, 05, 05), 06, 08
|
||||
// Construct various iterators and ensure their traversal is correct
|
||||
it, _ := snaps.StorageIterator(common.HexToHash("0x02"), common.HexToHash("0xaa"), common.HexToHash("0x01"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 3, it, verifyStorage) // expected: 01, 03, 05
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x02"), common.HexToHash("0xaa"), common.HexToHash("0x02"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 2, it, verifyStorage) // expected: 03, 05
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x02"), common.HexToHash("0xaa"), common.HexToHash("0x5"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 1, it, verifyStorage) // expected: 05
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x02"), common.HexToHash("0xaa"), common.HexToHash("0x6"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 0, it, verifyStorage) // expected: nothing
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.HexToHash("0x01"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 6, it, verifyStorage) // expected: 01, 02, 03, 05, 06, 08
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.HexToHash("0x05"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 3, it, verifyStorage) // expected: 05, 06, 08
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.HexToHash("0x08"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 1, it, verifyStorage) // expected: 08
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.HexToHash("0x09"))
|
||||
defer it.Release()
|
||||
verifyIterator(t, 0, it, verifyStorage) // expected: nothing
|
||||
}
|
||||
|
||||
// TestAccountIteratorDeletions tests that the iterator behaves correct when there are
|
||||
// deleted accounts (where the Account() value is nil). The iterator
|
||||
// should not output any accounts or nil-values for those cases.
|
||||
func TestIteratorDeletions(t *testing.T) {
|
||||
func TestAccountIteratorDeletions(t *testing.T) {
|
||||
// Create an empty base layer and a snapshot tree out of it
|
||||
base := &diskLayer{
|
||||
diskdb: rawdb.NewMemoryDatabase(),
|
||||
@ -474,7 +740,7 @@ func TestIteratorDeletions(t *testing.T) {
|
||||
// The output should be 11,33,44,55
|
||||
it, _ := snaps.AccountIterator(common.HexToHash("0x04"), common.Hash{})
|
||||
// Do a quick check
|
||||
verifyIterator(t, 4, it)
|
||||
verifyIterator(t, 4, it, verifyAccount)
|
||||
it.Release()
|
||||
|
||||
// And a more detailed verification that we indeed do not see '0x22'
|
||||
@ -491,6 +757,63 @@ func TestIteratorDeletions(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestStorageIteratorDeletions(t *testing.T) {
|
||||
// Create an empty base layer and a snapshot tree out of it
|
||||
base := &diskLayer{
|
||||
diskdb: rawdb.NewMemoryDatabase(),
|
||||
root: common.HexToHash("0x01"),
|
||||
cache: fastcache.New(1024 * 500),
|
||||
}
|
||||
snaps := &Tree{
|
||||
layers: map[common.Hash]snapshot{
|
||||
base.root: base,
|
||||
},
|
||||
}
|
||||
// Stack three diff layers on top with various overlaps
|
||||
snaps.Update(common.HexToHash("0x02"), common.HexToHash("0x01"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x01", "0x03", "0x05"}}, nil))
|
||||
|
||||
snaps.Update(common.HexToHash("0x03"), common.HexToHash("0x02"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x02", "0x04", "0x06"}}, [][]string{{"0x01", "0x03"}}))
|
||||
|
||||
// The output should be 02,04,05,06
|
||||
it, _ := snaps.StorageIterator(common.HexToHash("0x03"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 4, it, verifyStorage)
|
||||
it.Release()
|
||||
|
||||
// The output should be 04,05,06
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x03"), common.HexToHash("0xaa"), common.HexToHash("0x03"))
|
||||
verifyIterator(t, 3, it, verifyStorage)
|
||||
it.Release()
|
||||
|
||||
// Destruct the whole storage
|
||||
destructed := map[common.Hash]struct{}{
|
||||
common.HexToHash("0xaa"): {},
|
||||
}
|
||||
snaps.Update(common.HexToHash("0x04"), common.HexToHash("0x03"), destructed, nil, nil)
|
||||
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x04"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 0, it, verifyStorage)
|
||||
it.Release()
|
||||
|
||||
// Re-insert the slots of the same account
|
||||
snaps.Update(common.HexToHash("0x05"), common.HexToHash("0x04"), nil,
|
||||
randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x07", "0x08", "0x09"}}, nil))
|
||||
|
||||
// The output should be 07,08,09
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x05"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 3, it, verifyStorage)
|
||||
it.Release()
|
||||
|
||||
// Destruct the whole storage but re-create the account in the same layer
|
||||
snaps.Update(common.HexToHash("0x06"), common.HexToHash("0x05"), destructed, randomAccountSet("0xaa"), randomStorageSet([]string{"0xaa"}, [][]string{{"0x11", "0x12"}}, nil))
|
||||
it, _ = snaps.StorageIterator(common.HexToHash("0x06"), common.HexToHash("0xaa"), common.Hash{})
|
||||
verifyIterator(t, 2, it, verifyStorage) // The output should be 11,12
|
||||
it.Release()
|
||||
|
||||
verifyIterator(t, 2, snaps.Snapshot(common.HexToHash("0x06")).(*diffLayer).newBinaryStorageIterator(common.HexToHash("0xaa")), verifyStorage)
|
||||
}
|
||||
|
||||
// BenchmarkAccountIteratorTraversal is a bit a bit notorious -- all layers contain the
|
||||
// exact same 200 accounts. That means that we need to process 2000 items, but
|
||||
// only spit out 200 values eventually.
|
||||
|
@ -138,6 +138,9 @@ type snapshot interface {
|
||||
|
||||
// AccountIterator creates an account iterator over an arbitrary layer.
|
||||
AccountIterator(seek common.Hash) AccountIterator
|
||||
|
||||
// StorageIterator creates a storage iterator over an arbitrary layer.
|
||||
StorageIterator(account common.Hash, seek common.Hash) (StorageIterator, bool)
|
||||
}
|
||||
|
||||
// SnapshotTree is an Ethereum state snapshot tree. It consists of one persistent
|
||||
@ -601,3 +604,9 @@ func (t *Tree) Rebuild(root common.Hash) {
|
||||
func (t *Tree) AccountIterator(root common.Hash, seek common.Hash) (AccountIterator, error) {
|
||||
return newFastAccountIterator(t, root, seek)
|
||||
}
|
||||
|
||||
// StorageIterator creates a new storage iterator for the specified root hash and
|
||||
// account. The iterator will be move to the specific start position.
|
||||
func (t *Tree) StorageIterator(root common.Hash, account common.Hash, seek common.Hash) (StorageIterator, error) {
|
||||
return newFastStorageIterator(t, root, account, seek)
|
||||
}
|
||||
|
@ -60,6 +60,29 @@ func randomAccountSet(hashes ...string) map[common.Hash][]byte {
|
||||
return accounts
|
||||
}
|
||||
|
||||
// randomStorageSet generates a set of random slots with the given strings as
|
||||
// the slot addresses.
|
||||
func randomStorageSet(accounts []string, hashes [][]string, nilStorage [][]string) map[common.Hash]map[common.Hash][]byte {
|
||||
storages := make(map[common.Hash]map[common.Hash][]byte)
|
||||
for index, account := range accounts {
|
||||
storages[common.HexToHash(account)] = make(map[common.Hash][]byte)
|
||||
|
||||
if index < len(hashes) {
|
||||
hashes := hashes[index]
|
||||
for _, hash := range hashes {
|
||||
storages[common.HexToHash(account)][common.HexToHash(hash)] = randomHash().Bytes()
|
||||
}
|
||||
}
|
||||
if index < len(nilStorage) {
|
||||
nils := nilStorage[index]
|
||||
for _, hash := range nils {
|
||||
storages[common.HexToHash(account)][common.HexToHash(hash)] = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
return storages
|
||||
}
|
||||
|
||||
// Tests that if a disk layer becomes stale, no active external references will
|
||||
// be returned with junk data. This version of the test flattens every diff layer
|
||||
// to check internal corner case around the bottom-most memory accumulator.
|
||||
|
@ -472,7 +472,7 @@ func (s *StateDB) updateStateObject(obj *stateObject) {
|
||||
// enough to track account updates at commit time, deletions need tracking
|
||||
// at transaction boundary level to ensure we capture state clearing.
|
||||
if s.snap != nil {
|
||||
s.snapAccounts[obj.addrHash] = snapshot.AccountRLP(obj.data.Nonce, obj.data.Balance, obj.data.Root, obj.data.CodeHash)
|
||||
s.snapAccounts[obj.addrHash] = snapshot.SlimAccountRLP(obj.data.Nonce, obj.data.Balance, obj.data.Root, obj.data.CodeHash)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -147,15 +147,8 @@ func (t *BlockTest) Run(snapshotter bool) error {
|
||||
}
|
||||
// Cross-check the snapshot-to-hash against the trie hash
|
||||
if snapshotter {
|
||||
snapTree := chain.Snapshot()
|
||||
root := chain.CurrentBlock().Root()
|
||||
it, err := snapTree.AccountIterator(root, common.Hash{})
|
||||
if err != nil {
|
||||
return fmt.Errorf("Could not create iterator for root %x: %v", root, err)
|
||||
}
|
||||
generatedRoot := snapshot.GenerateTrieRoot(it)
|
||||
if generatedRoot != root {
|
||||
return fmt.Errorf("Snapshot corruption, got %d exp %d", generatedRoot, root)
|
||||
if err := snapshot.VerifyState(chain.Snapshot(), chain.CurrentBlock().Root()); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return t.validateImportedHeaders(chain, validBlocks)
|
||||
|
Loading…
Reference in New Issue
Block a user