triedb/pathdb: track flat state changes in pathdb
This pull request ports some changes from the main state snapshot integration one, specifically introducing the flat state tracking in pathdb. Note, the tracked flat state changes are only held in memory and won't be persisted in the disk. Meanwhile, the correspoding state retrieval in persistent state is also not supported yet. The states management in disk is more complicated and will be implemented in a separate pull request.
This commit is contained in:
parent
459bb4a647
commit
2074ea80a9
|
@ -60,6 +60,10 @@ type backend interface {
|
|||
// An error will be returned if the specified state is not available.
|
||||
NodeReader(root common.Hash) (database.NodeReader, error)
|
||||
|
||||
// StateReader returns a reader for accessing flat states within the specified
|
||||
// state. An error will be returned if the specified state is not available.
|
||||
StateReader(root common.Hash) (database.StateReader, error)
|
||||
|
||||
// Initialized returns an indicator if the state data is already initialized
|
||||
// according to the state scheme.
|
||||
Initialized(genesisRoot common.Hash) bool
|
||||
|
@ -122,6 +126,13 @@ func (db *Database) NodeReader(blockRoot common.Hash) (database.NodeReader, erro
|
|||
return db.backend.NodeReader(blockRoot)
|
||||
}
|
||||
|
||||
// StateReader returns a reader that allows access to the state data associated
|
||||
// with the specified state. An error will be returned if the specified state is
|
||||
// not available.
|
||||
func (db *Database) StateReader(blockRoot common.Hash) (database.StateReader, error) {
|
||||
return db.backend.StateReader(blockRoot)
|
||||
}
|
||||
|
||||
// Update performs a state transition by committing dirty nodes contained in the
|
||||
// given set in order to update state from the specified parent to the specified
|
||||
// root. The held pre-images accumulated up to this point will be flushed in case
|
||||
|
|
|
@ -18,6 +18,7 @@ package database
|
|||
|
||||
import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
)
|
||||
|
||||
// NodeReader wraps the Node method of a backing trie reader.
|
||||
|
@ -37,3 +38,31 @@ type NodeDatabase interface {
|
|||
// An error will be returned if the specified state is not available.
|
||||
NodeReader(stateRoot common.Hash) (NodeReader, error)
|
||||
}
|
||||
|
||||
// StateReader wraps the Account and Storage method of a backing state reader.
|
||||
type StateReader interface {
|
||||
// Account directly retrieves the account associated with a particular hash in
|
||||
// the slim data format. An error will be returned if the read operation exits
|
||||
// abnormally. Specifically, if the layer is already stale.
|
||||
//
|
||||
// Note:
|
||||
// - the returned account object is safe to modify
|
||||
// - no error will be returned if the requested account is not found in database
|
||||
Account(hash common.Hash) (*types.SlimAccount, error)
|
||||
|
||||
// Storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account. An error will be returned if the read operation
|
||||
// exits abnormally.
|
||||
//
|
||||
// Note:
|
||||
// - the returned storage data is not a copy, please don't modify it
|
||||
// - no error will be returned if the requested slot is not found in database
|
||||
Storage(accountHash, storageHash common.Hash) ([]byte, error)
|
||||
}
|
||||
|
||||
// StateDatabase warps the methods of a backing state store.
|
||||
type StateDatabase interface {
|
||||
// StateReader returns a state reader associated with the specific state.
|
||||
// An error will be returned if the specified state is not available.
|
||||
StateReader(stateRoot common.Hash) (StateReader, error)
|
||||
}
|
||||
|
|
|
@ -635,3 +635,9 @@ func (reader *reader) Node(owner common.Hash, path []byte, hash common.Hash) ([]
|
|||
blob, _ := reader.db.node(hash)
|
||||
return blob, nil
|
||||
}
|
||||
|
||||
// StateReader returns a reader that allows access to the state data associated
|
||||
// with the specified state.
|
||||
func (db *Database) StateReader(root common.Hash) (database.StateReader, error) {
|
||||
return nil, errors.New("not implemented")
|
||||
}
|
||||
|
|
|
@ -36,37 +36,53 @@ type buffer struct {
|
|||
layers uint64 // The number of diff layers aggregated inside
|
||||
limit uint64 // The maximum memory allowance in bytes
|
||||
nodes *nodeSet // Aggregated trie node set
|
||||
states *stateSet // Aggregated state set
|
||||
}
|
||||
|
||||
// newBuffer initializes the buffer with the provided states and trie nodes.
|
||||
func newBuffer(limit int, nodes *nodeSet, layers uint64) *buffer {
|
||||
func newBuffer(limit int, nodes *nodeSet, states *stateSet, layers uint64) *buffer {
|
||||
// Don't panic for lazy users if any provided set is nil
|
||||
if nodes == nil {
|
||||
nodes = newNodeSet(nil)
|
||||
}
|
||||
if states == nil {
|
||||
states = newStates(nil, nil, nil)
|
||||
}
|
||||
return &buffer{
|
||||
layers: layers,
|
||||
limit: uint64(limit),
|
||||
nodes: nodes,
|
||||
states: states,
|
||||
}
|
||||
}
|
||||
|
||||
// account retrieves the account blob with account address hash.
|
||||
func (b *buffer) account(hash common.Hash) ([]byte, bool) {
|
||||
return b.states.account(hash)
|
||||
}
|
||||
|
||||
// storage retrieves the storage slot with account address hash and slot key.
|
||||
func (b *buffer) storage(addrHash common.Hash, storageHash common.Hash) ([]byte, bool) {
|
||||
return b.states.storage(addrHash, storageHash)
|
||||
}
|
||||
|
||||
// node retrieves the trie node with node path and its trie identifier.
|
||||
func (b *buffer) node(owner common.Hash, path []byte) (*trienode.Node, bool) {
|
||||
return b.nodes.node(owner, path)
|
||||
}
|
||||
|
||||
// commit merges the provided states and trie nodes into the buffer.
|
||||
func (b *buffer) commit(nodes *nodeSet) *buffer {
|
||||
func (b *buffer) commit(nodes *nodeSet, states *stateSet) *buffer {
|
||||
b.layers++
|
||||
b.nodes.merge(nodes)
|
||||
b.states.merge(states)
|
||||
return b
|
||||
}
|
||||
|
||||
// revert is the reverse operation of commit. It also merges the provided states
|
||||
// and trie nodes into the buffer. The key difference is that the provided state
|
||||
// set should reverse the changes made by the most recent state transition.
|
||||
func (b *buffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[string]*trienode.Node) error {
|
||||
func (b *buffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[string]*trienode.Node, accounts map[common.Hash][]byte, storages map[common.Hash]map[common.Hash][]byte) error {
|
||||
// Short circuit if no embedded state transition to revert
|
||||
if b.layers == 0 {
|
||||
return errStateUnrecoverable
|
||||
|
@ -79,6 +95,7 @@ func (b *buffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[strin
|
|||
return nil
|
||||
}
|
||||
b.nodes.revert(db, nodes)
|
||||
b.states.revert(accounts, storages)
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -86,6 +103,7 @@ func (b *buffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[strin
|
|||
func (b *buffer) reset() {
|
||||
b.layers = 0
|
||||
b.nodes.reset()
|
||||
b.states.reset()
|
||||
}
|
||||
|
||||
// empty returns an indicator if buffer is empty.
|
||||
|
@ -101,7 +119,7 @@ func (b *buffer) full() bool {
|
|||
|
||||
// size returns the approximate memory size of the held content.
|
||||
func (b *buffer) size() uint64 {
|
||||
return b.nodes.size
|
||||
return b.states.size + b.nodes.size
|
||||
}
|
||||
|
||||
// flush persists the in-memory dirty trie node into the disk if the configured
|
||||
|
|
|
@ -68,6 +68,24 @@ type layer interface {
|
|||
// - no error will be returned if the requested node is not found in database.
|
||||
node(owner common.Hash, path []byte, depth int) ([]byte, common.Hash, *nodeLoc, error)
|
||||
|
||||
// account directly retrieves the account RLP associated with a particular
|
||||
// hash in the slim data format. An error will be returned if the read
|
||||
// operation exits abnormally. Specifically, if the layer is already stale.
|
||||
//
|
||||
// Note:
|
||||
// - the returned account is not a copy, please don't modify it.
|
||||
// - no error will be returned if the requested account is not found in database.
|
||||
account(hash common.Hash, depth int) ([]byte, error)
|
||||
|
||||
// storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account. An error will be returned if the read operation
|
||||
// exits abnormally. Specifically, if the layer is already stale.
|
||||
//
|
||||
// Note:
|
||||
// - the returned storage data is not a copy, please don't modify it.
|
||||
// - no error will be returned if the requested slot is not found in database.
|
||||
storage(accountHash, storageHash common.Hash, depth int) ([]byte, error)
|
||||
|
||||
// rootHash returns the root hash for which this layer was made.
|
||||
rootHash() common.Hash
|
||||
|
||||
|
@ -130,17 +148,18 @@ var Defaults = &Config{
|
|||
// ReadOnly is the config in order to open database in read only mode.
|
||||
var ReadOnly = &Config{ReadOnly: true}
|
||||
|
||||
// Database is a multiple-layered structure for maintaining in-memory trie nodes.
|
||||
// It consists of one persistent base layer backed by a key-value store, on top
|
||||
// of which arbitrarily many in-memory diff layers are stacked. The memory diffs
|
||||
// can form a tree with branching, but the disk layer is singleton and common to
|
||||
// all. If a reorg goes deeper than the disk layer, a batch of reverse diffs can
|
||||
// be applied to rollback. The deepest reorg that can be handled depends on the
|
||||
// amount of state histories tracked in the disk.
|
||||
// Database is a multiple-layered structure for maintaining in-memory states
|
||||
// along with its dirty trie nodes. It consists of one persistent base layer
|
||||
// backed by a key-value store, on top of which arbitrarily many in-memory diff
|
||||
// layers are stacked. The memory diffs can form a tree with branching, but the
|
||||
// disk layer is singleton and common to all. If a reorg goes deeper than the
|
||||
// disk layer, a batch of reverse diffs can be applied to rollback. The deepest
|
||||
// reorg that can be handled depends on the amount of state histories tracked
|
||||
// in the disk.
|
||||
//
|
||||
// At most one readable and writable database can be opened at the same time in
|
||||
// the whole system which ensures that only one database writer can operate disk
|
||||
// state. Unexpected open operations can cause the system to panic.
|
||||
// the whole system which ensures that only one database writer can operate the
|
||||
// persistent state. Unexpected open operations can cause the system to panic.
|
||||
type Database struct {
|
||||
// readOnly is the flag whether the mutation is allowed to be applied.
|
||||
// It will be set automatically when the database is journaled during
|
||||
|
@ -358,7 +377,7 @@ func (db *Database) Enable(root common.Hash) error {
|
|||
}
|
||||
// Re-construct a new disk layer backed by persistent state
|
||||
// with **empty clean cache and node buffer**.
|
||||
db.tree.reset(newDiskLayer(root, 0, db, nil, newBuffer(db.config.WriteBufferSize, nil, 0)))
|
||||
db.tree.reset(newDiskLayer(root, 0, db, nil, newBuffer(db.config.WriteBufferSize, nil, nil, 0)))
|
||||
|
||||
// Re-enable the database as the final step.
|
||||
db.waitSync = false
|
||||
|
|
|
@ -309,7 +309,18 @@ func (t *tester) generate(parent common.Hash) (common.Hash, *trienode.MergedNode
|
|||
delete(t.storages, addrHash)
|
||||
}
|
||||
}
|
||||
return root, ctx.nodes, NewStateSetWithOrigin(ctx.accountOrigin, ctx.storageOrigin)
|
||||
var (
|
||||
accounts = make(map[common.Hash][]byte)
|
||||
destructs = make(map[common.Hash]struct{})
|
||||
)
|
||||
for addrHash, data := range ctx.accounts {
|
||||
if len(data) == 0 {
|
||||
destructs[addrHash] = struct{}{}
|
||||
} else {
|
||||
accounts[addrHash] = data
|
||||
}
|
||||
}
|
||||
return root, ctx.nodes, NewStateSetWithOrigin(destructs, accounts, ctx.storages, ctx.accountOrigin, ctx.storageOrigin)
|
||||
}
|
||||
|
||||
// lastHash returns the latest root hash, or empty if nothing is cached.
|
||||
|
|
|
@ -52,6 +52,7 @@ func newDiffLayer(parent layer, root common.Hash, id uint64, block uint64, nodes
|
|||
states: states,
|
||||
}
|
||||
dirtyNodeWriteMeter.Mark(int64(nodes.size))
|
||||
dirtyStateWriteMeter.Mark(int64(states.size))
|
||||
log.Debug("Created new diff layer", "id", id, "block", block, "nodesize", common.StorageSize(nodes.size), "statesize", common.StorageSize(states.size))
|
||||
return dl
|
||||
}
|
||||
|
@ -96,6 +97,58 @@ func (dl *diffLayer) node(owner common.Hash, path []byte, depth int) ([]byte, co
|
|||
return dl.parent.node(owner, path, depth+1)
|
||||
}
|
||||
|
||||
// account directly retrieves the account RLP associated with a particular
|
||||
// hash in the slim data format.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (dl *diffLayer) account(hash common.Hash, depth int) ([]byte, error) {
|
||||
// Hold the lock, ensure the parent won't be changed during the
|
||||
// state accessing.
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if blob, found := dl.states.account(hash); found {
|
||||
dirtyStateHitMeter.Mark(1)
|
||||
dirtyStateHitDepthHist.Update(int64(depth))
|
||||
dirtyStateReadMeter.Mark(int64(len(blob)))
|
||||
|
||||
if len(blob) == 0 {
|
||||
stateAccountMissMeter.Mark(1)
|
||||
} else {
|
||||
stateAccountHitMeter.Mark(1)
|
||||
}
|
||||
return blob, nil
|
||||
}
|
||||
// Account is unknown to this layer, resolve from parent
|
||||
return dl.parent.account(hash, depth+1)
|
||||
}
|
||||
|
||||
// storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (dl *diffLayer) storage(accountHash, storageHash common.Hash, depth int) ([]byte, error) {
|
||||
// Hold the lock, ensure the parent won't be changed during the
|
||||
// state accessing.
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if blob, found := dl.states.storage(accountHash, storageHash); found {
|
||||
dirtyStateHitMeter.Mark(1)
|
||||
dirtyStateHitDepthHist.Update(int64(depth))
|
||||
dirtyStateReadMeter.Mark(int64(len(blob)))
|
||||
|
||||
if len(blob) == 0 {
|
||||
stateStorageMissMeter.Mark(1)
|
||||
} else {
|
||||
stateStorageHitMeter.Mark(1)
|
||||
}
|
||||
return blob, nil
|
||||
}
|
||||
// storage slot is unknown to this layer, resolve from parent
|
||||
return dl.parent.storage(accountHash, storageHash, depth+1)
|
||||
}
|
||||
|
||||
// update implements the layer interface, creating a new layer on top of the
|
||||
// existing layer tree with the specified data items.
|
||||
func (dl *diffLayer) update(root common.Hash, id uint64, block uint64, nodes *nodeSet, states *StateSetWithOrigin) *diffLayer {
|
||||
|
|
|
@ -30,7 +30,7 @@ import (
|
|||
func emptyLayer() *diskLayer {
|
||||
return &diskLayer{
|
||||
db: New(rawdb.NewMemoryDatabase(), nil, false),
|
||||
buffer: newBuffer(defaultBufferSize, nil, 0),
|
||||
buffer: newBuffer(defaultBufferSize, nil, nil, 0),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package pathdb
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
|
@ -33,7 +34,7 @@ type diskLayer struct {
|
|||
id uint64 // Immutable, corresponding state id
|
||||
db *Database // Path-based trie database
|
||||
nodes *fastcache.Cache // GC friendly memory cache of clean nodes
|
||||
buffer *buffer // Dirty buffer to aggregate writes of nodes
|
||||
buffer *buffer // Dirty buffer to aggregate writes of nodes and states
|
||||
stale bool // Signals that the layer became stale (state progressed)
|
||||
lock sync.RWMutex // Lock used to protect stale flag
|
||||
}
|
||||
|
@ -140,6 +141,75 @@ func (dl *diskLayer) node(owner common.Hash, path []byte, depth int) ([]byte, co
|
|||
return blob, h.hash(blob), &nodeLoc{loc: locDiskLayer, depth: depth}, nil
|
||||
}
|
||||
|
||||
// account directly retrieves the account RLP associated with a particular
|
||||
// hash in the slim data format.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (dl *diskLayer) account(hash common.Hash, depth int) ([]byte, error) {
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if dl.stale {
|
||||
return nil, errSnapshotStale
|
||||
}
|
||||
// Try to retrieve the account from the not-yet-written
|
||||
// node buffer first. Note the buffer is lock free since
|
||||
// it's impossible to mutate the buffer before tagging the
|
||||
// layer as stale.
|
||||
blob, found := dl.buffer.account(hash)
|
||||
if found {
|
||||
dirtyStateHitMeter.Mark(1)
|
||||
dirtyStateReadMeter.Mark(int64(len(blob)))
|
||||
dirtyStateHitDepthHist.Update(int64(depth))
|
||||
|
||||
if len(blob) == 0 {
|
||||
stateAccountMissMeter.Mark(1)
|
||||
} else {
|
||||
stateAccountHitMeter.Mark(1)
|
||||
}
|
||||
return blob, nil
|
||||
}
|
||||
dirtyStateMissMeter.Mark(1)
|
||||
|
||||
// TODO(rjl493456442) support persistent state retrieval
|
||||
return nil, errors.New("not supported")
|
||||
}
|
||||
|
||||
// storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account.
|
||||
//
|
||||
// Note the returned account is not a copy, please don't modify it.
|
||||
func (dl *diskLayer) storage(accountHash, storageHash common.Hash, depth int) ([]byte, error) {
|
||||
// Hold the lock, ensure the parent won't be changed during the
|
||||
// state accessing.
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if dl.stale {
|
||||
return nil, errSnapshotStale
|
||||
}
|
||||
// Try to retrieve the storage slot from the not-yet-written
|
||||
// node buffer first. Note the buffer is lock free since
|
||||
// it's impossible to mutate the buffer before tagging the
|
||||
// layer as stale.
|
||||
if blob, found := dl.buffer.storage(accountHash, storageHash); found {
|
||||
dirtyStateHitMeter.Mark(1)
|
||||
dirtyStateReadMeter.Mark(int64(len(blob)))
|
||||
dirtyStateHitDepthHist.Update(int64(depth))
|
||||
|
||||
if len(blob) == 0 {
|
||||
stateStorageMissMeter.Mark(1)
|
||||
} else {
|
||||
stateStorageHitMeter.Mark(1)
|
||||
}
|
||||
return blob, nil
|
||||
}
|
||||
dirtyStateMissMeter.Mark(1)
|
||||
|
||||
// TODO(rjl493456442) support persistent state retrieval
|
||||
return nil, errors.New("not supported")
|
||||
}
|
||||
|
||||
// update implements the layer interface, returning a new diff layer on top
|
||||
// with the given state set.
|
||||
func (dl *diskLayer) update(root common.Hash, id uint64, block uint64, nodes *nodeSet, states *StateSetWithOrigin) *diffLayer {
|
||||
|
@ -190,14 +260,14 @@ func (dl *diskLayer) commit(bottom *diffLayer, force bool) (*diskLayer, error) {
|
|||
|
||||
// In a unique scenario where the ID of the oldest history object (after tail
|
||||
// truncation) surpasses the persisted state ID, we take the necessary action
|
||||
// of forcibly committing the cached dirty nodes to ensure that the persisted
|
||||
// of forcibly committing the cached dirty states to ensure that the persisted
|
||||
// state ID remains higher.
|
||||
if !force && rawdb.ReadPersistentStateID(dl.db.diskdb) < oldest {
|
||||
force = true
|
||||
}
|
||||
// Merge the trie nodes of the bottom-most diff layer into the buffer as the
|
||||
// combined layer.
|
||||
combined := dl.buffer.commit(bottom.nodes)
|
||||
// Merge the trie nodes and flat states of the bottom-most diff layer into the
|
||||
// buffer as the combined layer.
|
||||
combined := dl.buffer.commit(bottom.nodes, bottom.states.stateSet)
|
||||
if combined.full() || force {
|
||||
if err := combined.flush(dl.db.diskdb, dl.db.freezer, dl.nodes, bottom.stateID()); err != nil {
|
||||
return nil, err
|
||||
|
@ -225,6 +295,17 @@ func (dl *diskLayer) revert(h *history) (*diskLayer, error) {
|
|||
if dl.id == 0 {
|
||||
return nil, fmt.Errorf("%w: zero state id", errStateUnrecoverable)
|
||||
}
|
||||
var (
|
||||
buff = crypto.NewKeccakState()
|
||||
accounts = make(map[common.Hash][]byte)
|
||||
storages = make(map[common.Hash]map[common.Hash][]byte)
|
||||
)
|
||||
for addr, blob := range h.accounts {
|
||||
accounts[crypto.HashData(buff, addr.Bytes())] = blob
|
||||
}
|
||||
for addr, storage := range h.storages {
|
||||
storages[crypto.HashData(buff, addr.Bytes())] = storage
|
||||
}
|
||||
// Apply the reverse state changes upon the current state. This must
|
||||
// be done before holding the lock in order to access state in "this"
|
||||
// layer.
|
||||
|
@ -244,7 +325,7 @@ func (dl *diskLayer) revert(h *history) (*diskLayer, error) {
|
|||
// needs to be reverted is not yet flushed and cached in node
|
||||
// buffer, otherwise, manipulate persistent state directly.
|
||||
if !dl.buffer.empty() {
|
||||
err := dl.buffer.revert(dl.db.diskdb, nodes)
|
||||
err := dl.buffer.revert(dl.db.diskdb, nodes, accounts, storages)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -45,7 +45,8 @@ var (
|
|||
//
|
||||
// - Version 0: initial version
|
||||
// - Version 1: storage.Incomplete field is removed
|
||||
const journalVersion uint64 = 1
|
||||
// - Version 2: add post-modification state values
|
||||
const journalVersion uint64 = 2
|
||||
|
||||
// loadJournal tries to parse the layer journal from the disk.
|
||||
func (db *Database) loadJournal(diskRoot common.Hash) (layer, error) {
|
||||
|
@ -108,7 +109,7 @@ func (db *Database) loadLayers() layer {
|
|||
log.Info("Failed to load journal, discard it", "err", err)
|
||||
}
|
||||
// Return single layer with persistent state.
|
||||
return newDiskLayer(root, rawdb.ReadPersistentStateID(db.diskdb), db, nil, newBuffer(db.config.WriteBufferSize, nil, 0))
|
||||
return newDiskLayer(root, rawdb.ReadPersistentStateID(db.diskdb), db, nil, newBuffer(db.config.WriteBufferSize, nil, nil, 0))
|
||||
}
|
||||
|
||||
// loadDiskLayer reads the binary blob from the layer journal, reconstructing
|
||||
|
@ -135,7 +136,12 @@ func (db *Database) loadDiskLayer(r *rlp.Stream) (layer, error) {
|
|||
if err := nodes.decode(r); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return newDiskLayer(root, id, db, nil, newBuffer(db.config.WriteBufferSize, &nodes, id-stored)), nil
|
||||
// Resolve flat state sets in aggregated buffer
|
||||
var states stateSet
|
||||
if err := states.decode(r); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return newDiskLayer(root, id, db, nil, newBuffer(db.config.WriteBufferSize, &nodes, &states, id-stored)), nil
|
||||
}
|
||||
|
||||
// loadDiffLayer reads the next sections of a layer journal, reconstructing a new
|
||||
|
@ -189,6 +195,10 @@ func (dl *diskLayer) journal(w io.Writer) error {
|
|||
if err := dl.buffer.nodes.encode(w); err != nil {
|
||||
return err
|
||||
}
|
||||
// Step four, write the accumulated flat states into the journal
|
||||
if err := dl.buffer.states.encode(w); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("Journaled pathdb disk layer", "root", dl.root)
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -24,12 +24,23 @@ var (
|
|||
cleanNodeReadMeter = metrics.NewRegisteredMeter("pathdb/clean/node/read", nil)
|
||||
cleanNodeWriteMeter = metrics.NewRegisteredMeter("pathdb/clean/node/write", nil)
|
||||
|
||||
stateAccountMissMeter = metrics.NewRegisteredMeter("pathdb/state/account/miss/total", nil)
|
||||
stateAccountHitMeter = metrics.NewRegisteredMeter("pathdb/state/account/hit/total", nil)
|
||||
stateStorageMissMeter = metrics.NewRegisteredMeter("pathdb/state/storage/miss/total", nil)
|
||||
stateStorageHitMeter = metrics.NewRegisteredMeter("pathdb/state/storage/hit/total", nil)
|
||||
|
||||
dirtyNodeHitMeter = metrics.NewRegisteredMeter("pathdb/dirty/node/hit", nil)
|
||||
dirtyNodeMissMeter = metrics.NewRegisteredMeter("pathdb/dirty/node/miss", nil)
|
||||
dirtyNodeReadMeter = metrics.NewRegisteredMeter("pathdb/dirty/node/read", nil)
|
||||
dirtyNodeWriteMeter = metrics.NewRegisteredMeter("pathdb/dirty/node/write", nil)
|
||||
dirtyNodeHitDepthHist = metrics.NewRegisteredHistogram("pathdb/dirty/node/depth", nil, metrics.NewExpDecaySample(1028, 0.015))
|
||||
|
||||
dirtyStateHitMeter = metrics.NewRegisteredMeter("pathdb/dirty/state/hit", nil)
|
||||
dirtyStateMissMeter = metrics.NewRegisteredMeter("pathdb/dirty/state/miss", nil)
|
||||
dirtyStateReadMeter = metrics.NewRegisteredMeter("pathdb/dirty/state/read", nil)
|
||||
dirtyStateWriteMeter = metrics.NewRegisteredMeter("pathdb/dirty/state/write", nil)
|
||||
dirtyStateHitDepthHist = metrics.NewRegisteredHistogram("pathdb/dirty/state/depth", nil, metrics.NewExpDecaySample(1028, 0.015))
|
||||
|
||||
cleanFalseMeter = metrics.NewRegisteredMeter("pathdb/clean/false", nil)
|
||||
dirtyFalseMeter = metrics.NewRegisteredMeter("pathdb/dirty/false", nil)
|
||||
diskFalseMeter = metrics.NewRegisteredMeter("pathdb/disk/false", nil)
|
||||
|
@ -41,6 +52,10 @@ var (
|
|||
|
||||
gcTrieNodeMeter = metrics.NewRegisteredMeter("pathdb/gc/node/count", nil)
|
||||
gcTrieNodeBytesMeter = metrics.NewRegisteredMeter("pathdb/gc/node/bytes", nil)
|
||||
gcAccountMeter = metrics.NewRegisteredMeter("pathdb/gc/account/count", nil)
|
||||
gcAccountBytesMeter = metrics.NewRegisteredMeter("pathdb/gc/account/bytes", nil)
|
||||
gcStorageMeter = metrics.NewRegisteredMeter("pathdb/gc/storage/count", nil)
|
||||
gcStorageBytesMeter = metrics.NewRegisteredMeter("pathdb/gc/storage/bytes", nil)
|
||||
|
||||
historyBuildTimeMeter = metrics.NewRegisteredTimer("pathdb/history/time", nil)
|
||||
historyDataBytesMeter = metrics.NewRegisteredMeter("pathdb/history/bytes/data", nil)
|
||||
|
|
|
@ -21,7 +21,9 @@ import (
|
|||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/triedb/database"
|
||||
)
|
||||
|
||||
|
@ -84,6 +86,39 @@ func (r *reader) Node(owner common.Hash, path []byte, hash common.Hash) ([]byte,
|
|||
return blob, nil
|
||||
}
|
||||
|
||||
// Account directly retrieves the account associated with a particular hash in
|
||||
// the slim data format. An error will be returned if the read operation exits
|
||||
// abnormally. Specifically, if the layer is already stale.
|
||||
//
|
||||
// Note:
|
||||
// - the returned account object is safe to modify
|
||||
// - no error will be returned if the requested account is not found in database
|
||||
func (r *reader) Account(hash common.Hash) (*types.SlimAccount, error) {
|
||||
blob, err := r.layer.account(hash, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(blob) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
account := new(types.SlimAccount)
|
||||
if err := rlp.DecodeBytes(blob, account); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return account, nil
|
||||
}
|
||||
|
||||
// Storage directly retrieves the storage data associated with a particular hash,
|
||||
// within a particular account. An error will be returned if the read operation
|
||||
// exits abnormally. Specifically, if the layer is already stale.
|
||||
//
|
||||
// Note:
|
||||
// - the returned storage data is not a copy, please don't modify it
|
||||
// - no error will be returned if the requested slot is not found in database
|
||||
func (r *reader) Storage(accountHash, storageHash common.Hash) ([]byte, error) {
|
||||
return r.layer.storage(accountHash, storageHash, 0)
|
||||
}
|
||||
|
||||
// NodeReader retrieves a layer belonging to the given state root.
|
||||
func (db *Database) NodeReader(root common.Hash) (database.NodeReader, error) {
|
||||
layer := db.tree.get(root)
|
||||
|
@ -92,3 +127,13 @@ func (db *Database) NodeReader(root common.Hash) (database.NodeReader, error) {
|
|||
}
|
||||
return &reader{layer: layer, noHashCheck: db.isVerkle}, nil
|
||||
}
|
||||
|
||||
// StateReader returns a reader that allows access to the state data associated
|
||||
// with the specified state.
|
||||
func (db *Database) StateReader(root common.Hash) (database.StateReader, error) {
|
||||
layer := db.tree.get(root)
|
||||
if layer == nil {
|
||||
return nil, fmt.Errorf("state %#x is not available", root)
|
||||
}
|
||||
return &reader{layer: layer}, nil
|
||||
}
|
||||
|
|
|
@ -17,12 +17,18 @@
|
|||
package pathdb
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"slices"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"golang.org/x/exp/maps"
|
||||
)
|
||||
|
||||
// counter helps in tracking items and their corresponding sizes.
|
||||
|
@ -43,9 +49,528 @@ func (c *counter) report(count metrics.Meter, size metrics.Meter) {
|
|||
size.Mark(int64(c.size))
|
||||
}
|
||||
|
||||
// destruct represents the record of destruct set modification.
|
||||
type destruct struct {
|
||||
Hash common.Hash
|
||||
Exist bool
|
||||
}
|
||||
|
||||
// journal contains the list of modifications applied for destruct set.
|
||||
type journal struct {
|
||||
destructs [][]destruct
|
||||
}
|
||||
|
||||
func (j *journal) add(entries []destruct) {
|
||||
j.destructs = append(j.destructs, entries)
|
||||
}
|
||||
|
||||
func (j *journal) pop() ([]destruct, error) {
|
||||
if len(j.destructs) == 0 {
|
||||
return nil, errors.New("destruct journal is not available")
|
||||
}
|
||||
last := j.destructs[len(j.destructs)-1]
|
||||
j.destructs = j.destructs[:len(j.destructs)-1]
|
||||
return last, nil
|
||||
}
|
||||
|
||||
func (j *journal) reset() {
|
||||
j.destructs = nil
|
||||
}
|
||||
|
||||
func (j *journal) encode(w io.Writer) error {
|
||||
return rlp.Encode(w, j.destructs)
|
||||
}
|
||||
|
||||
func (j *journal) decode(r *rlp.Stream) error {
|
||||
var dec [][]destruct
|
||||
if err := r.Decode(&dec); err != nil {
|
||||
return err
|
||||
}
|
||||
j.destructs = dec
|
||||
return nil
|
||||
}
|
||||
|
||||
// stateSet represents a collection of state modifications belonging to a
|
||||
// transition(a block execution) or several aggregated transitions.
|
||||
type stateSet struct {
|
||||
// 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.,
|
||||
// accountData and 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 deleted
|
||||
// - in the tx_2, account A is 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
|
||||
accountData map[common.Hash][]byte // Keyed accounts for direct retrieval (nil is not expected)
|
||||
storageData map[common.Hash]map[common.Hash][]byte // Keyed storage slots for direct retrieval. one per account (nil means deleted)
|
||||
size uint64 // Memory size of the state data (destructSet, accountData and storageData)
|
||||
|
||||
journal *journal // Track the modifications to destructSet, used for reversal
|
||||
accountListSorted []common.Hash // List of account for iteration. If it exists, it's sorted, otherwise it's nil
|
||||
storageListSorted map[common.Hash][]common.Hash // List of storage slots for iterated retrievals, one per account. Any existing lists are sorted if non-nil
|
||||
lock sync.RWMutex // Lock for guarding the two lists above
|
||||
}
|
||||
|
||||
// newStates constructs the state set with the provided data.
|
||||
func newStates(destructs map[common.Hash]struct{}, accounts map[common.Hash][]byte, storages map[common.Hash]map[common.Hash][]byte) *stateSet {
|
||||
// Don't panic for the lazy callers, initialize the nil maps instead.
|
||||
if destructs == nil {
|
||||
destructs = make(map[common.Hash]struct{})
|
||||
}
|
||||
if accounts == nil {
|
||||
accounts = make(map[common.Hash][]byte)
|
||||
}
|
||||
if storages == nil {
|
||||
storages = make(map[common.Hash]map[common.Hash][]byte)
|
||||
}
|
||||
s := &stateSet{
|
||||
destructSet: destructs,
|
||||
accountData: accounts,
|
||||
storageData: storages,
|
||||
journal: &journal{},
|
||||
storageListSorted: make(map[common.Hash][]common.Hash),
|
||||
}
|
||||
s.size = s.check()
|
||||
return s
|
||||
}
|
||||
|
||||
// account returns the account data associated with the specified address hash.
|
||||
func (s *stateSet) account(hash common.Hash) ([]byte, bool) {
|
||||
// If the account is known locally, return it
|
||||
if data, ok := s.accountData[hash]; ok {
|
||||
return data, true
|
||||
}
|
||||
// If the account is known locally, but deleted, return it
|
||||
if _, ok := s.destructSet[hash]; ok {
|
||||
return nil, true
|
||||
}
|
||||
return nil, false // account is unknown in this set
|
||||
}
|
||||
|
||||
// storage returns the storage slot associated with the specified address hash
|
||||
// and storage key hash.
|
||||
func (s *stateSet) storage(accountHash, storageHash common.Hash) ([]byte, bool) {
|
||||
// If the account is known locally, try to resolve the slot locally
|
||||
if storage, ok := s.storageData[accountHash]; ok {
|
||||
if data, ok := storage[storageHash]; ok {
|
||||
return data, true
|
||||
}
|
||||
}
|
||||
// If the account is known locally, but deleted, return an empty slot
|
||||
if _, ok := s.destructSet[accountHash]; ok {
|
||||
return nil, true
|
||||
}
|
||||
return nil, false // storage is unknown in this set
|
||||
}
|
||||
|
||||
// check sanitizes accounts and storage slots to ensure the data validity.
|
||||
// Additionally, it computes the total memory size occupied by the maps.
|
||||
func (s *stateSet) check() uint64 {
|
||||
size := len(s.destructSet) * common.HashLength
|
||||
for accountHash, blob := range s.accountData {
|
||||
if blob == nil {
|
||||
panic(fmt.Sprintf("account %#x nil", accountHash)) // nil account blob is not permitted
|
||||
}
|
||||
size += common.HashLength + len(blob)
|
||||
}
|
||||
for accountHash, slots := range s.storageData {
|
||||
if slots == nil {
|
||||
panic(fmt.Sprintf("storage %#x nil", accountHash)) // nil slots is not permitted
|
||||
}
|
||||
for _, val := range slots {
|
||||
size += 2*common.HashLength + len(val)
|
||||
}
|
||||
}
|
||||
return uint64(size)
|
||||
}
|
||||
|
||||
// accountList returns a sorted list of all accounts in this state set, including
|
||||
// the deleted ones.
|
||||
//
|
||||
// Note, the returned slice is not a copy, so do not modify it.
|
||||
//
|
||||
//nolint:unused
|
||||
func (s *stateSet) accountList() []common.Hash {
|
||||
// If an old list already exists, return it
|
||||
s.lock.RLock()
|
||||
list := s.accountListSorted
|
||||
s.lock.RUnlock()
|
||||
|
||||
if list != nil {
|
||||
return list
|
||||
}
|
||||
// No old sorted account list exists, generate a new one
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
s.accountListSorted = make([]common.Hash, 0, len(s.destructSet)+len(s.accountData))
|
||||
for hash := range s.accountData {
|
||||
s.accountListSorted = append(s.accountListSorted, hash)
|
||||
}
|
||||
for hash := range s.destructSet {
|
||||
if _, ok := s.accountData[hash]; !ok {
|
||||
s.accountListSorted = append(s.accountListSorted, hash)
|
||||
}
|
||||
}
|
||||
slices.SortFunc(s.accountListSorted, common.Hash.Cmp)
|
||||
return s.accountListSorted
|
||||
}
|
||||
|
||||
// StorageList returns a sorted list of all storage slot hashes in this state set
|
||||
// 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.
|
||||
//
|
||||
//nolint:unused
|
||||
func (s *stateSet) storageList(accountHash common.Hash) ([]common.Hash, bool) {
|
||||
s.lock.RLock()
|
||||
_, destructed := s.destructSet[accountHash]
|
||||
if _, ok := s.storageData[accountHash]; !ok {
|
||||
// Account not tracked by this layer
|
||||
s.lock.RUnlock()
|
||||
return nil, destructed
|
||||
}
|
||||
// If an old list already exists, return it
|
||||
if list, exist := s.storageListSorted[accountHash]; exist {
|
||||
s.lock.RUnlock()
|
||||
return list, destructed // the cached list can't be nil
|
||||
}
|
||||
s.lock.RUnlock()
|
||||
|
||||
// No old sorted account list exists, generate a new one
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
storageList := maps.Keys(s.storageData[accountHash])
|
||||
slices.SortFunc(storageList, common.Hash.Cmp)
|
||||
s.storageListSorted[accountHash] = storageList
|
||||
return storageList, destructed
|
||||
}
|
||||
|
||||
// clearCache invalidates the cached account list and storage lists.
|
||||
func (s *stateSet) clearCache() {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
s.accountListSorted = nil
|
||||
s.storageListSorted = make(map[common.Hash][]common.Hash)
|
||||
}
|
||||
|
||||
// merge integrates the accounts and storages from the external set into the
|
||||
// local set, ensuring the combined set reflects the combined state of both.
|
||||
//
|
||||
// The provided state set will remain unchanged, as it may still be referenced
|
||||
// by other layers.
|
||||
func (s *stateSet) merge(set *stateSet) {
|
||||
var (
|
||||
delta int
|
||||
accountOverwrites counter
|
||||
storageOverwrites counter
|
||||
destructs []destruct
|
||||
)
|
||||
// Apply account deletion markers and discard any previously cached data if exists
|
||||
for accountHash := range set.destructSet {
|
||||
if origin, ok := s.accountData[accountHash]; ok {
|
||||
delta -= common.HashLength + len(origin)
|
||||
accountOverwrites.add(common.HashLength + len(origin))
|
||||
delete(s.accountData, accountHash)
|
||||
}
|
||||
if _, ok := s.storageData[accountHash]; ok {
|
||||
// Looping through the nested map may cause slight performance degradation.
|
||||
// However, since account destruction is no longer possible after the cancun
|
||||
// fork, this overhead is considered acceptable.
|
||||
for _, val := range s.storageData[accountHash] {
|
||||
delta -= 2*common.HashLength + len(val)
|
||||
storageOverwrites.add(2*common.HashLength + len(val))
|
||||
}
|
||||
delete(s.storageData, accountHash)
|
||||
}
|
||||
// Keep track of whether the account has already been marked as destructed.
|
||||
// This additional marker is useful for undoing the merge operation.
|
||||
_, exist := s.destructSet[accountHash]
|
||||
destructs = append(destructs, destruct{
|
||||
Hash: accountHash,
|
||||
Exist: exist,
|
||||
})
|
||||
if exist {
|
||||
continue
|
||||
}
|
||||
delta += common.HashLength
|
||||
s.destructSet[accountHash] = struct{}{}
|
||||
}
|
||||
s.journal.add(destructs)
|
||||
|
||||
// Apply the updated account data
|
||||
for accountHash, data := range set.accountData {
|
||||
if origin, ok := s.accountData[accountHash]; ok {
|
||||
delta += len(data) - len(origin)
|
||||
accountOverwrites.add(common.HashLength + len(origin))
|
||||
} else {
|
||||
delta += common.HashLength + len(data)
|
||||
}
|
||||
s.accountData[accountHash] = data
|
||||
}
|
||||
// Apply all the updated storage slots (individually)
|
||||
for accountHash, storage := range set.storageData {
|
||||
// If storage didn't exist (or was deleted) in the set, overwrite blindly
|
||||
if _, ok := s.storageData[accountHash]; !ok {
|
||||
// To prevent potential concurrent map read/write issues, allocate a
|
||||
// new map for the storage instead of claiming it directly from the
|
||||
// passed external set. Even after merging, the slots belonging to the
|
||||
// external state set remain accessible, so ownership of the map should
|
||||
// not be taken, and any mutation on it should be avoided.
|
||||
slots := make(map[common.Hash][]byte)
|
||||
for storageHash, data := range storage {
|
||||
slots[storageHash] = data
|
||||
delta += 2*common.HashLength + len(data)
|
||||
}
|
||||
s.storageData[accountHash] = slots
|
||||
continue
|
||||
}
|
||||
// Storage exists in both local and external set, merge the slots
|
||||
slots := s.storageData[accountHash]
|
||||
for storageHash, data := range storage {
|
||||
if origin, ok := slots[storageHash]; ok {
|
||||
delta += len(data) - len(origin)
|
||||
storageOverwrites.add(2*common.HashLength + len(origin))
|
||||
} else {
|
||||
delta += 2*common.HashLength + len(data)
|
||||
}
|
||||
slots[storageHash] = data
|
||||
}
|
||||
}
|
||||
accountOverwrites.report(gcAccountMeter, gcAccountBytesMeter)
|
||||
storageOverwrites.report(gcStorageMeter, gcStorageBytesMeter)
|
||||
s.clearCache()
|
||||
s.updateSize(delta)
|
||||
}
|
||||
|
||||
// revert takes the original value of accounts and storages as input and reverts
|
||||
// the latest state transition applied on the state set.
|
||||
func (s *stateSet) revert(accountOrigin map[common.Hash][]byte, storageOrigin map[common.Hash]map[common.Hash][]byte) {
|
||||
// Load the destruct journal whose availability is always expected
|
||||
destructs, err := s.journal.pop()
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to revert state, %v", err))
|
||||
}
|
||||
// Revert the modifications to the destruct set by journal
|
||||
var delta int
|
||||
for _, entry := range destructs {
|
||||
if entry.Exist {
|
||||
continue
|
||||
}
|
||||
delete(s.destructSet, entry.Hash)
|
||||
delta -= common.HashLength
|
||||
}
|
||||
// Overwrite the account data with original value blindly
|
||||
for addrHash, blob := range accountOrigin {
|
||||
if len(blob) == 0 {
|
||||
if data, ok := s.accountData[addrHash]; ok {
|
||||
delta -= common.HashLength + len(data)
|
||||
} else {
|
||||
panic(fmt.Sprintf("non-existent account for deleting, %x", addrHash))
|
||||
}
|
||||
delete(s.accountData, addrHash)
|
||||
} else {
|
||||
if data, ok := s.accountData[addrHash]; ok {
|
||||
delta += len(blob) - len(data)
|
||||
} else {
|
||||
delta += len(blob) + common.HashLength
|
||||
}
|
||||
s.accountData[addrHash] = blob
|
||||
}
|
||||
}
|
||||
// Overwrite the storage data with original value blindly
|
||||
for addrHash, storage := range storageOrigin {
|
||||
// It might be possible that the storage set is not existent because
|
||||
// the whole storage is deleted.
|
||||
slots := s.storageData[addrHash]
|
||||
if len(slots) == 0 {
|
||||
slots = make(map[common.Hash][]byte)
|
||||
}
|
||||
for storageHash, blob := range storage {
|
||||
if len(blob) == 0 {
|
||||
if data, ok := slots[storageHash]; ok {
|
||||
delta -= 2*common.HashLength + len(data)
|
||||
} else {
|
||||
panic(fmt.Sprintf("non-existent storage slot for deleting, %x %x", addrHash, storageHash))
|
||||
}
|
||||
delete(slots, storageHash)
|
||||
} else {
|
||||
if data, ok := slots[storageHash]; ok {
|
||||
delta += len(blob) - len(data)
|
||||
} else {
|
||||
delta += 2*common.HashLength + len(blob)
|
||||
}
|
||||
slots[storageHash] = blob
|
||||
}
|
||||
}
|
||||
if len(slots) == 0 {
|
||||
delete(s.storageData, addrHash)
|
||||
} else {
|
||||
s.storageData[addrHash] = slots
|
||||
}
|
||||
}
|
||||
s.clearCache()
|
||||
s.updateSize(delta)
|
||||
}
|
||||
|
||||
// updateSize updates the total cache size by the given delta.
|
||||
func (s *stateSet) updateSize(delta int) {
|
||||
size := int64(s.size) + int64(delta)
|
||||
if size >= 0 {
|
||||
s.size = uint64(size)
|
||||
return
|
||||
}
|
||||
log.Error("Stateset size underflow", "prev", common.StorageSize(s.size), "delta", common.StorageSize(delta))
|
||||
s.size = 0
|
||||
}
|
||||
|
||||
// encode serializes the content of state set into the provided writer.
|
||||
func (s *stateSet) encode(w io.Writer) error {
|
||||
// Encode destructs
|
||||
destructs := make([]common.Hash, 0, len(s.destructSet))
|
||||
for hash := range s.destructSet {
|
||||
destructs = append(destructs, hash)
|
||||
}
|
||||
if err := rlp.Encode(w, destructs); err != nil {
|
||||
return err
|
||||
}
|
||||
// Encode accounts
|
||||
type Account struct {
|
||||
Hash common.Hash
|
||||
Blob []byte
|
||||
}
|
||||
accounts := make([]Account, 0, len(s.accountData))
|
||||
for hash, blob := range s.accountData {
|
||||
accounts = append(accounts, Account{Hash: hash, Blob: blob})
|
||||
}
|
||||
if err := rlp.Encode(w, accounts); err != nil {
|
||||
return err
|
||||
}
|
||||
// Encode storages
|
||||
type Storage struct {
|
||||
Hash common.Hash
|
||||
Keys []common.Hash
|
||||
Blobs [][]byte
|
||||
}
|
||||
storages := make([]Storage, 0, len(s.storageData))
|
||||
for accountHash, slots := range s.storageData {
|
||||
keys := make([]common.Hash, 0, len(slots))
|
||||
vals := make([][]byte, 0, len(slots))
|
||||
for key, val := range slots {
|
||||
keys = append(keys, key)
|
||||
vals = append(vals, val)
|
||||
}
|
||||
storages = append(storages, Storage{Hash: accountHash, Keys: keys, Blobs: vals})
|
||||
}
|
||||
if err := rlp.Encode(w, storages); err != nil {
|
||||
return err
|
||||
}
|
||||
// Encode journal
|
||||
return s.journal.encode(w)
|
||||
}
|
||||
|
||||
// decode deserializes the content from the rlp stream into the state set.
|
||||
func (s *stateSet) decode(r *rlp.Stream) error {
|
||||
// Decode destructs
|
||||
var (
|
||||
destructs []common.Hash
|
||||
destructSet = make(map[common.Hash]struct{})
|
||||
)
|
||||
if err := r.Decode(&destructs); err != nil {
|
||||
return fmt.Errorf("load diff destructs: %v", err)
|
||||
}
|
||||
for _, hash := range destructs {
|
||||
destructSet[hash] = struct{}{}
|
||||
}
|
||||
s.destructSet = destructSet
|
||||
|
||||
// Decode accounts
|
||||
type Account struct {
|
||||
Hash common.Hash
|
||||
Blob []byte
|
||||
}
|
||||
var (
|
||||
accounts []Account
|
||||
accountSet = make(map[common.Hash][]byte)
|
||||
)
|
||||
if err := r.Decode(&accounts); err != nil {
|
||||
return fmt.Errorf("load diff accounts: %v", err)
|
||||
}
|
||||
for _, account := range accounts {
|
||||
accountSet[account.Hash] = account.Blob
|
||||
}
|
||||
s.accountData = accountSet
|
||||
|
||||
// Decode storages
|
||||
type Storage struct {
|
||||
AccountHash common.Hash
|
||||
Keys []common.Hash
|
||||
Vals [][]byte
|
||||
}
|
||||
var (
|
||||
storages []Storage
|
||||
storageSet = make(map[common.Hash]map[common.Hash][]byte)
|
||||
)
|
||||
if err := r.Decode(&storages); err != nil {
|
||||
return fmt.Errorf("load diff storage: %v", err)
|
||||
}
|
||||
for _, entry := range storages {
|
||||
storageSet[entry.AccountHash] = make(map[common.Hash][]byte)
|
||||
for i := 0; i < len(entry.Keys); i++ {
|
||||
storageSet[entry.AccountHash][entry.Keys[i]] = entry.Vals[i]
|
||||
}
|
||||
}
|
||||
s.storageData = storageSet
|
||||
s.storageListSorted = make(map[common.Hash][]common.Hash)
|
||||
|
||||
// Decode journal
|
||||
s.journal = &journal{}
|
||||
if err := s.journal.decode(r); err != nil {
|
||||
return err
|
||||
}
|
||||
s.size = s.check()
|
||||
return nil
|
||||
}
|
||||
|
||||
// reset clears all cached state data, including any optional sorted lists that
|
||||
// may have been generated.
|
||||
func (s *stateSet) reset() {
|
||||
s.destructSet = make(map[common.Hash]struct{})
|
||||
s.accountData = make(map[common.Hash][]byte)
|
||||
s.storageData = make(map[common.Hash]map[common.Hash][]byte)
|
||||
s.size = 0
|
||||
s.journal.reset()
|
||||
s.accountListSorted = nil
|
||||
s.storageListSorted = make(map[common.Hash][]common.Hash)
|
||||
}
|
||||
|
||||
// dbsize returns the approximate size for db write.
|
||||
//
|
||||
//nolint:unused
|
||||
func (s *stateSet) dbsize() int {
|
||||
m := (len(s.destructSet) + len(s.accountData)) * len(rawdb.SnapshotAccountPrefix)
|
||||
for _, slots := range s.storageData {
|
||||
m += len(slots) * len(rawdb.SnapshotStoragePrefix)
|
||||
}
|
||||
return m + int(s.size)
|
||||
}
|
||||
|
||||
// StateSetWithOrigin wraps the state set with additional original values of the
|
||||
// mutated states.
|
||||
type StateSetWithOrigin struct {
|
||||
*stateSet
|
||||
|
||||
// AccountOrigin represents the account data before the state transition,
|
||||
// corresponding to both the accountData and destructSet. It's keyed by the
|
||||
// account address. The nil value means the account was not present before.
|
||||
|
@ -62,7 +587,7 @@ type StateSetWithOrigin struct {
|
|||
}
|
||||
|
||||
// NewStateSetWithOrigin constructs the state set with the provided data.
|
||||
func NewStateSetWithOrigin(accountOrigin map[common.Address][]byte, storageOrigin map[common.Address]map[common.Hash][]byte) *StateSetWithOrigin {
|
||||
func NewStateSetWithOrigin(destructs map[common.Hash]struct{}, accounts map[common.Hash][]byte, storages map[common.Hash]map[common.Hash][]byte, accountOrigin map[common.Address][]byte, storageOrigin map[common.Address]map[common.Hash][]byte) *StateSetWithOrigin {
|
||||
// Don't panic for the lazy callers, initialize the nil maps instead.
|
||||
if accountOrigin == nil {
|
||||
accountOrigin = make(map[common.Address][]byte)
|
||||
|
@ -82,15 +607,21 @@ func NewStateSetWithOrigin(accountOrigin map[common.Address][]byte, storageOrigi
|
|||
size += 2*common.HashLength + len(data)
|
||||
}
|
||||
}
|
||||
set := newStates(destructs, accounts, storages)
|
||||
return &StateSetWithOrigin{
|
||||
stateSet: set,
|
||||
accountOrigin: accountOrigin,
|
||||
storageOrigin: storageOrigin,
|
||||
size: uint64(size),
|
||||
size: set.size + uint64(size),
|
||||
}
|
||||
}
|
||||
|
||||
// encode serializes the content of state set into the provided writer.
|
||||
func (s *StateSetWithOrigin) encode(w io.Writer) error {
|
||||
// Encode state set
|
||||
if err := s.stateSet.encode(w); err != nil {
|
||||
return err
|
||||
}
|
||||
// Encode accounts
|
||||
type Accounts struct {
|
||||
Addresses []common.Address
|
||||
|
@ -125,6 +656,12 @@ func (s *StateSetWithOrigin) encode(w io.Writer) error {
|
|||
|
||||
// decode deserializes the content from the rlp stream into the state set.
|
||||
func (s *StateSetWithOrigin) decode(r *rlp.Stream) error {
|
||||
if s.stateSet == nil {
|
||||
s.stateSet = &stateSet{}
|
||||
}
|
||||
if err := s.stateSet.decode(r); err != nil {
|
||||
return err
|
||||
}
|
||||
// Decode account origin
|
||||
type Accounts struct {
|
||||
Addresses []common.Address
|
||||
|
|
|
@ -0,0 +1,459 @@
|
|||
// Copyright 2024 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>
|
||||
|
||||
package pathdb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
|
||||
func TestStatesMerge(t *testing.T) {
|
||||
a := newStates(
|
||||
nil,
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0},
|
||||
common.Hash{0xb}: {0xb0},
|
||||
common.Hash{0xc}: {0xc0},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
common.Hash{0x2}: {0x20},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
},
|
||||
)
|
||||
b := newStates(
|
||||
map[common.Hash]struct{}{
|
||||
common.Hash{0xa}: {},
|
||||
common.Hash{0xc}: {},
|
||||
},
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa1},
|
||||
common.Hash{0xb}: {0xb1},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x11},
|
||||
common.Hash{0x3}: {0x31},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x11},
|
||||
},
|
||||
},
|
||||
)
|
||||
a.merge(b)
|
||||
|
||||
blob, exist := a.account(common.Hash{0xa})
|
||||
if !exist || !bytes.Equal(blob, []byte{0xa1}) {
|
||||
t.Error("Unexpected value for account a")
|
||||
}
|
||||
blob, exist = a.account(common.Hash{0xb})
|
||||
if !exist || !bytes.Equal(blob, []byte{0xb1}) {
|
||||
t.Error("Unexpected value for account b")
|
||||
}
|
||||
blob, exist = a.account(common.Hash{0xc})
|
||||
if !exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for account c")
|
||||
}
|
||||
// unknown account
|
||||
blob, exist = a.account(common.Hash{0xd})
|
||||
if exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for account d")
|
||||
}
|
||||
|
||||
blob, exist = a.storage(common.Hash{0xa}, common.Hash{0x1})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x11}) {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xa}, common.Hash{0x2})
|
||||
if !exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xa}, common.Hash{0x3})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x31}) {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xb}, common.Hash{0x1})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x11}) {
|
||||
t.Error("Unexpected value for b's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xc}, common.Hash{0x1})
|
||||
if !exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for c's storage")
|
||||
}
|
||||
|
||||
// unknown storage slots
|
||||
blob, exist = a.storage(common.Hash{0xd}, common.Hash{0x1})
|
||||
if exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for d's storage")
|
||||
}
|
||||
}
|
||||
|
||||
func TestStatesRevert(t *testing.T) {
|
||||
a := newStates(
|
||||
nil,
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0},
|
||||
common.Hash{0xb}: {0xb0},
|
||||
common.Hash{0xc}: {0xc0},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
common.Hash{0x2}: {0x20},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
},
|
||||
)
|
||||
b := newStates(
|
||||
map[common.Hash]struct{}{
|
||||
common.Hash{0xa}: {},
|
||||
common.Hash{0xc}: {},
|
||||
},
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa1},
|
||||
common.Hash{0xb}: {0xb1},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x11},
|
||||
common.Hash{0x3}: {0x31},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x11},
|
||||
},
|
||||
},
|
||||
)
|
||||
a.merge(b)
|
||||
a.revert(
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0},
|
||||
common.Hash{0xb}: {0xb0},
|
||||
common.Hash{0xc}: {0xc0},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
common.Hash{0x2}: {0x20},
|
||||
common.Hash{0x3}: {},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
},
|
||||
)
|
||||
|
||||
blob, exist := a.account(common.Hash{0xa})
|
||||
if !exist || !bytes.Equal(blob, []byte{0xa0}) {
|
||||
t.Error("Unexpected value for account a")
|
||||
}
|
||||
blob, exist = a.account(common.Hash{0xb})
|
||||
if !exist || !bytes.Equal(blob, []byte{0xb0}) {
|
||||
t.Error("Unexpected value for account b")
|
||||
}
|
||||
blob, exist = a.account(common.Hash{0xc})
|
||||
if !exist || !bytes.Equal(blob, []byte{0xc0}) {
|
||||
t.Error("Unexpected value for account c")
|
||||
}
|
||||
// unknown account
|
||||
blob, exist = a.account(common.Hash{0xd})
|
||||
if exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for account d")
|
||||
}
|
||||
|
||||
blob, exist = a.storage(common.Hash{0xa}, common.Hash{0x1})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x10}) {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xa}, common.Hash{0x2})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x20}) {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
_, exist = a.storage(common.Hash{0xa}, common.Hash{0x3})
|
||||
if exist {
|
||||
t.Error("Unexpected value for a's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xb}, common.Hash{0x1})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x10}) {
|
||||
t.Error("Unexpected value for b's storage")
|
||||
}
|
||||
blob, exist = a.storage(common.Hash{0xc}, common.Hash{0x1})
|
||||
if !exist || !bytes.Equal(blob, []byte{0x10}) {
|
||||
t.Error("Unexpected value for c's storage")
|
||||
}
|
||||
// unknown storage slots
|
||||
blob, exist = a.storage(common.Hash{0xd}, common.Hash{0x1})
|
||||
if exist || len(blob) != 0 {
|
||||
t.Error("Unexpected value for d's storage")
|
||||
}
|
||||
}
|
||||
|
||||
func TestDestructJournalEncode(t *testing.T) {
|
||||
var enc journal
|
||||
enc.add(nil) // nil
|
||||
enc.add([]destruct{}) // zero size destructs
|
||||
enc.add([]destruct{
|
||||
{Hash: common.HexToHash("0xdeadbeef"), Exist: true},
|
||||
{Hash: common.HexToHash("0xcafebabe"), Exist: false},
|
||||
})
|
||||
var buf bytes.Buffer
|
||||
enc.encode(&buf)
|
||||
|
||||
var dec journal
|
||||
if err := dec.decode(rlp.NewStream(&buf, 0)); err != nil {
|
||||
t.Fatalf("Failed to decode journal, %v", err)
|
||||
}
|
||||
if len(enc.destructs) != len(dec.destructs) {
|
||||
t.Fatalf("Unexpected destruct journal length, want: %d, got: %d", len(enc.destructs), len(dec.destructs))
|
||||
}
|
||||
for i := 0; i < len(enc.destructs); i++ {
|
||||
want := enc.destructs[i]
|
||||
got := dec.destructs[i]
|
||||
if len(want) == 0 && len(got) == 0 {
|
||||
continue
|
||||
}
|
||||
if !reflect.DeepEqual(want, got) {
|
||||
t.Fatalf("Unexpected destruct, want: %v, got: %v", want, got)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestStatesEncode(t *testing.T) {
|
||||
s := newStates(
|
||||
map[common.Hash]struct{}{
|
||||
common.Hash{0x1}: {},
|
||||
},
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0x1}: {0x1},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0x1}: {
|
||||
common.Hash{0x1}: {0x1},
|
||||
},
|
||||
},
|
||||
)
|
||||
buf := bytes.NewBuffer(nil)
|
||||
if err := s.encode(buf); err != nil {
|
||||
t.Fatalf("Failed to encode states, %v", err)
|
||||
}
|
||||
var dec stateSet
|
||||
if err := dec.decode(rlp.NewStream(buf, 0)); err != nil {
|
||||
t.Fatalf("Failed to decode states, %v", err)
|
||||
}
|
||||
if !reflect.DeepEqual(s.destructSet, dec.destructSet) {
|
||||
t.Fatal("Unexpected destruct set")
|
||||
}
|
||||
if !reflect.DeepEqual(s.accountData, dec.accountData) {
|
||||
t.Fatal("Unexpected account data")
|
||||
}
|
||||
if !reflect.DeepEqual(s.storageData, dec.storageData) {
|
||||
t.Fatal("Unexpected storage data")
|
||||
}
|
||||
}
|
||||
|
||||
func TestStateWithOriginEncode(t *testing.T) {
|
||||
s := NewStateSetWithOrigin(
|
||||
map[common.Hash]struct{}{
|
||||
common.Hash{0x1}: {},
|
||||
},
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0x1}: {0x1},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0x1}: {
|
||||
common.Hash{0x1}: {0x1},
|
||||
},
|
||||
},
|
||||
map[common.Address][]byte{
|
||||
common.Address{0x1}: {0x1},
|
||||
},
|
||||
map[common.Address]map[common.Hash][]byte{
|
||||
common.Address{0x1}: {
|
||||
common.Hash{0x1}: {0x1},
|
||||
},
|
||||
},
|
||||
)
|
||||
buf := bytes.NewBuffer(nil)
|
||||
if err := s.encode(buf); err != nil {
|
||||
t.Fatalf("Failed to encode states, %v", err)
|
||||
}
|
||||
var dec StateSetWithOrigin
|
||||
if err := dec.decode(rlp.NewStream(buf, 0)); err != nil {
|
||||
t.Fatalf("Failed to decode states, %v", err)
|
||||
}
|
||||
if !reflect.DeepEqual(s.destructSet, dec.destructSet) {
|
||||
t.Fatal("Unexpected destruct set")
|
||||
}
|
||||
if !reflect.DeepEqual(s.accountData, dec.accountData) {
|
||||
t.Fatal("Unexpected account data")
|
||||
}
|
||||
if !reflect.DeepEqual(s.storageData, dec.storageData) {
|
||||
t.Fatal("Unexpected storage data")
|
||||
}
|
||||
if !reflect.DeepEqual(s.accountOrigin, dec.accountOrigin) {
|
||||
t.Fatal("Unexpected account origin data")
|
||||
}
|
||||
if !reflect.DeepEqual(s.storageOrigin, dec.storageOrigin) {
|
||||
t.Fatal("Unexpected storage origin data")
|
||||
}
|
||||
}
|
||||
|
||||
func TestStateSizeTracking(t *testing.T) {
|
||||
expSizeA := 3*(common.HashLength+1) + /* account data */
|
||||
2*(2*common.HashLength+1) + /* storage data of 0xa */
|
||||
2*common.HashLength + 3 + /* storage data of 0xb */
|
||||
2*common.HashLength + 1 /* storage data of 0xc */
|
||||
|
||||
a := newStates(
|
||||
nil,
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0}, // common.HashLength+1
|
||||
common.Hash{0xb}: {0xb0}, // common.HashLength+1
|
||||
common.Hash{0xc}: {0xc0}, // common.HashLength+1
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10}, // 2*common.HashLength+1
|
||||
common.Hash{0x2}: {0x20}, // 2*common.HashLength+1
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10, 0x11, 0x12}, // 2*common.HashLength+3
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10}, // 2*common.HashLength+1
|
||||
},
|
||||
},
|
||||
)
|
||||
if a.size != uint64(expSizeA) {
|
||||
t.Fatalf("Unexpected size, want: %d, got: %d", expSizeA, a.size)
|
||||
}
|
||||
|
||||
expSizeB := 2*common.HashLength + /* destruct set data */
|
||||
common.HashLength + 2 + common.HashLength + 3 + /* account data */
|
||||
2*common.HashLength + 3 + 2*common.HashLength + 2 + /* storage data of 0xa */
|
||||
2*common.HashLength + 2 + 2*common.HashLength + 2 /* storage data of 0xb */
|
||||
b := newStates(
|
||||
map[common.Hash]struct{}{
|
||||
common.Hash{0xa}: {}, // common.HashLength
|
||||
common.Hash{0xc}: {}, // common.HashLength
|
||||
},
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa1, 0xa1}, // common.HashLength+2
|
||||
common.Hash{0xb}: {0xb1, 0xb1, 0xb1}, // common.HashLength+3
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x11, 0x11, 0x11}, // 2*common.HashLength+3
|
||||
common.Hash{0x3}: {0x31, 0x31}, // 2*common.HashLength+1
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x11, 0x11}, // 2*common.HashLength+2
|
||||
common.Hash{0x2}: {0x22, 0x22}, // 2*common.HashLength+2
|
||||
},
|
||||
},
|
||||
)
|
||||
if b.size != uint64(expSizeB) {
|
||||
t.Fatalf("Unexpected size, want: %d, got: %d", expSizeB, b.size)
|
||||
}
|
||||
|
||||
a.merge(b)
|
||||
mergeSize := expSizeA + 2*common.HashLength /* destruct set data */
|
||||
mergeSize += 1 /* account a data change */ + 2 /* account b data change */
|
||||
mergeSize -= common.HashLength + 1 /* account data removal of 0xc */
|
||||
mergeSize += 2 + 1 /* storage a change */
|
||||
mergeSize += 2*common.HashLength + 2 - 1 /* storage b change */
|
||||
mergeSize -= 2*common.HashLength + 1 /* storage data removal of 0xc */
|
||||
|
||||
if a.size != uint64(mergeSize) {
|
||||
t.Fatalf("Unexpected size, want: %d, got: %d", mergeSize, a.size)
|
||||
}
|
||||
|
||||
// Revert the set to original status
|
||||
a.revert(
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0},
|
||||
common.Hash{0xb}: {0xb0},
|
||||
common.Hash{0xc}: {0xc0},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
common.Hash{0x2}: {0x20},
|
||||
common.Hash{0x3}: {},
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10, 0x11, 0x12},
|
||||
common.Hash{0x2}: {},
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
},
|
||||
)
|
||||
if a.size != uint64(expSizeA) {
|
||||
t.Fatalf("Unexpected size, want: %d, got: %d", expSizeA, a.size)
|
||||
}
|
||||
|
||||
// Revert state set a again, this time with additional slots which were
|
||||
// deleted in account destruction and re-created because of resurrection.
|
||||
a.merge(b)
|
||||
a.revert(
|
||||
map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {0xa0},
|
||||
common.Hash{0xb}: {0xb0},
|
||||
common.Hash{0xc}: {0xc0},
|
||||
},
|
||||
map[common.Hash]map[common.Hash][]byte{
|
||||
common.Hash{0xa}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
common.Hash{0x2}: {0x20},
|
||||
common.Hash{0x3}: {},
|
||||
common.Hash{0x4}: {0x40}, // this slot was not in the set a, but resurrected because of revert
|
||||
common.Hash{0x5}: {0x50, 0x51}, // this slot was not in the set a, but resurrected because of revert
|
||||
},
|
||||
common.Hash{0xb}: {
|
||||
common.Hash{0x1}: {0x10, 0x11, 0x12},
|
||||
common.Hash{0x2}: {},
|
||||
},
|
||||
common.Hash{0xc}: {
|
||||
common.Hash{0x1}: {0x10},
|
||||
},
|
||||
},
|
||||
)
|
||||
expSize := expSizeA + common.HashLength*2 + 1 + /* slot 4 */ +common.HashLength*2 + 2 /* slot 5 */
|
||||
if a.size != uint64(expSize) {
|
||||
t.Fatalf("Unexpected size, want: %d, got: %d", expSize, a.size)
|
||||
}
|
||||
}
|
|
@ -47,5 +47,5 @@ func (set *StateSet) internal() *pathdb.StateSetWithOrigin {
|
|||
if set == nil {
|
||||
return nil
|
||||
}
|
||||
return pathdb.NewStateSetWithOrigin(set.AccountsOrigin, set.StoragesOrigin)
|
||||
return pathdb.NewStateSetWithOrigin(set.Destructs, set.Accounts, set.Storages, set.AccountsOrigin, set.StoragesOrigin)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue