core/state: introduce stateupdate structure (#29530)

* core/state: introduce stateUpate structure

* core/state: remove outdated function description

* core/state: address comments
pull/29738/head
rjl493456442 4 months ago committed by GitHub
parent c9e0b3105b
commit d38b88a5a1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 4
      core/genesis_test.go
  2. 268
      core/state/state_object.go
  3. 391
      core/state/statedb.go
  4. 30
      core/state/statedb_fuzz_test.go
  5. 133
      core/state/stateupdate.go

@ -304,7 +304,7 @@ func TestVerkleGenesisCommit(t *testing.T) {
}, },
} }
expected := common.Hex2Bytes("14398d42be3394ff8d50681816a4b7bf8d8283306f577faba2d5bc57498de23b") expected := common.FromHex("14398d42be3394ff8d50681816a4b7bf8d8283306f577faba2d5bc57498de23b")
got := genesis.ToBlock().Root().Bytes() got := genesis.ToBlock().Root().Bytes()
if !bytes.Equal(got, expected) { if !bytes.Equal(got, expected) {
t.Fatalf("invalid genesis state root, expected %x, got %x", expected, got) t.Fatalf("invalid genesis state root, expected %x, got %x", expected, got)
@ -314,7 +314,7 @@ func TestVerkleGenesisCommit(t *testing.T) {
triedb := triedb.NewDatabase(db, &triedb.Config{IsVerkle: true, PathDB: pathdb.Defaults}) triedb := triedb.NewDatabase(db, &triedb.Config{IsVerkle: true, PathDB: pathdb.Defaults})
block := genesis.MustCommit(db, triedb) block := genesis.MustCommit(db, triedb)
if !bytes.Equal(block.Root().Bytes(), expected) { if !bytes.Equal(block.Root().Bytes(), expected) {
t.Fatalf("invalid genesis state root, expected %x, got %x", expected, got) t.Fatalf("invalid genesis state root, expected %x, got %x", expected, block.Root())
} }
// Test that the trie is verkle // Test that the trie is verkle

@ -19,9 +19,7 @@ package state
import ( import (
"bytes" "bytes"
"fmt" "fmt"
"io"
"maps" "maps"
"sync"
"time" "time"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
@ -34,14 +32,6 @@ import (
"github.com/holiman/uint256" "github.com/holiman/uint256"
) )
// hasherPool holds a pool of hashers used by state objects during concurrent
// trie updates.
var hasherPool = sync.Pool{
New: func() interface{} {
return crypto.NewKeccakState()
},
}
type Storage map[common.Hash]common.Hash type Storage map[common.Hash]common.Hash
func (s Storage) Copy() Storage { func (s Storage) Copy() Storage {
@ -65,9 +55,20 @@ type stateObject struct {
trie Trie // storage trie, which becomes non-nil on first access trie Trie // storage trie, which becomes non-nil on first access
code []byte // contract bytecode, which gets set when code is loaded code []byte // contract bytecode, which gets set when code is loaded
originStorage Storage // Storage cache of original entries to dedup rewrites originStorage Storage // Storage entries that have been accessed within the current block
pendingStorage Storage // Storage entries that need to be flushed to disk, at the end of an entire block dirtyStorage Storage // Storage entries that have been modified within the current transaction
dirtyStorage Storage // Storage entries that have been modified in the current transaction execution, reset for every transaction pendingStorage Storage // Storage entries that have been modified within the current block
// uncommittedStorage tracks a set of storage entries that have been modified
// but not yet committed since the "last commit operation", along with their
// original values before mutation.
//
// Specifically, the commit will be performed after each transaction before
// the byzantium fork, therefore the map is already reset at the transaction
// boundary; however post the byzantium fork, the commit will only be performed
// at the end of block, this set essentially tracks all the modifications
// made within the block.
uncommittedStorage Storage
// Cache flags. // Cache flags.
dirtyCode bool // true if the code was updated dirtyCode bool // true if the code was updated
@ -96,22 +97,18 @@ func newObject(db *StateDB, address common.Address, acct *types.StateAccount) *s
acct = types.NewEmptyStateAccount() acct = types.NewEmptyStateAccount()
} }
return &stateObject{ return &stateObject{
db: db, db: db,
address: address, address: address,
addrHash: crypto.Keccak256Hash(address[:]), addrHash: crypto.Keccak256Hash(address[:]),
origin: origin, origin: origin,
data: *acct, data: *acct,
originStorage: make(Storage), originStorage: make(Storage),
pendingStorage: make(Storage), dirtyStorage: make(Storage),
dirtyStorage: make(Storage), pendingStorage: make(Storage),
uncommittedStorage: make(Storage),
} }
} }
// EncodeRLP implements rlp.Encoder.
func (s *stateObject) EncodeRLP(w io.Writer) error {
return rlp.Encode(w, &s.data)
}
func (s *stateObject) markSelfdestructed() { func (s *stateObject) markSelfdestructed() {
s.selfDestructed = true s.selfDestructed = true
} }
@ -160,7 +157,7 @@ func (s *stateObject) getPrefetchedTrie() Trie {
return s.db.prefetcher.trie(s.addrHash, s.data.Root) return s.db.prefetcher.trie(s.addrHash, s.data.Root)
} }
// GetState retrieves a value from the account storage trie. // GetState retrieves a value associated with the given storage key.
func (s *stateObject) GetState(key common.Hash) common.Hash { func (s *stateObject) GetState(key common.Hash) common.Hash {
value, _ := s.getState(key) value, _ := s.getState(key)
return value return value
@ -177,7 +174,8 @@ func (s *stateObject) getState(key common.Hash) (common.Hash, common.Hash) {
return origin, origin return origin, origin
} }
// GetCommittedState retrieves a value from the committed account storage trie. // GetCommittedState retrieves the value associated with the specific key
// without any mutations caused in the current execution.
func (s *stateObject) GetCommittedState(key common.Hash) common.Hash { func (s *stateObject) GetCommittedState(key common.Hash) common.Hash {
// If we have a pending write or clean cached, return that // If we have a pending write or clean cached, return that
if value, pending := s.pendingStorage[key]; pending { if value, pending := s.pendingStorage[key]; pending {
@ -193,6 +191,7 @@ func (s *stateObject) GetCommittedState(key common.Hash) common.Hash {
// have been handles via pendingStorage above. // have been handles via pendingStorage above.
// 2) we don't have new values, and can deliver empty response back // 2) we don't have new values, and can deliver empty response back
if _, destructed := s.db.stateObjectsDestruct[s.address]; destructed { if _, destructed := s.db.stateObjectsDestruct[s.address]; destructed {
s.originStorage[key] = common.Hash{} // track the empty slot as origin value
return common.Hash{} return common.Hash{}
} }
// If no live objects are available, attempt to use snapshots // If no live objects are available, attempt to use snapshots
@ -272,17 +271,26 @@ func (s *stateObject) setState(key common.Hash, value common.Hash, origin common
func (s *stateObject) finalise() { func (s *stateObject) finalise() {
slotsToPrefetch := make([][]byte, 0, len(s.dirtyStorage)) slotsToPrefetch := make([][]byte, 0, len(s.dirtyStorage))
for key, value := range s.dirtyStorage { for key, value := range s.dirtyStorage {
// If the slot is different from its original value, move it into the if origin, exist := s.uncommittedStorage[key]; exist && origin == value {
// pending area to be committed at the end of the block (and prefetch // The slot is reverted to its original value, delete the entry
// the pathways). // to avoid thrashing the data structures.
if value != s.originStorage[key] { delete(s.uncommittedStorage, key)
s.pendingStorage[key] = value } else if exist {
slotsToPrefetch = append(slotsToPrefetch, common.CopyBytes(key[:])) // Copy needed for closure // The slot is modified to another value and the slot has been
// tracked for commit, do nothing here.
} else { } else {
// Otherwise, the slot was reverted to its original value, remove it // The slot is different from its original value and hasn't been
// from the pending area to avoid thrashing the data structure. // tracked for commit yet.
delete(s.pendingStorage, key) s.uncommittedStorage[key] = s.GetCommittedState(key)
slotsToPrefetch = append(slotsToPrefetch, common.CopyBytes(key[:])) // Copy needed for closure
} }
// Aggregate the dirty storage slots into the pending area. It might
// be possible that the value of tracked slot here is same with the
// one in originStorage (e.g. the slot was modified in tx_a and then
// modified back in tx_b). We can't blindly remove it from pending
// map as the dirty slot might have been committed already (before the
// byzantium fork) and entry is necessary to modify the value back.
s.pendingStorage[key] = value
} }
if s.db.prefetcher != nil && len(slotsToPrefetch) > 0 && s.data.Root != types.EmptyRootHash { if s.db.prefetcher != nil && len(slotsToPrefetch) > 0 && s.data.Root != types.EmptyRootHash {
if err := s.db.prefetcher.prefetch(s.addrHash, s.data.Root, s.address, slotsToPrefetch); err != nil { if err := s.db.prefetcher.prefetch(s.addrHash, s.data.Root, s.address, slotsToPrefetch); err != nil {
@ -308,7 +316,7 @@ func (s *stateObject) finalise() {
// It assumes all the dirty storage slots have been finalized before. // It assumes all the dirty storage slots have been finalized before.
func (s *stateObject) updateTrie() (Trie, error) { func (s *stateObject) updateTrie() (Trie, error) {
// Short circuit if nothing changed, don't bother with hashing anything // Short circuit if nothing changed, don't bother with hashing anything
if len(s.pendingStorage) == 0 { if len(s.uncommittedStorage) == 0 {
return s.trie, nil return s.trie, nil
} }
// Retrieve a pretecher populated trie, or fall back to the database // Retrieve a pretecher populated trie, or fall back to the database
@ -325,20 +333,8 @@ func (s *stateObject) updateTrie() (Trie, error) {
return nil, err return nil, err
} }
} }
// Perform trie updates before deletions. This prevents resolution of unnecessary trie nodes
// The snapshot storage map for the object // in circumstances similar to the following:
var (
storage map[common.Hash][]byte
origin map[common.Hash][]byte
)
// Insert all the pending storage updates into the trie
usedStorage := make([][]byte, 0, len(s.pendingStorage))
hasher := hasherPool.Get().(crypto.KeccakState)
defer hasherPool.Put(hasher)
// Perform trie updates before deletions. This prevents resolution of unnecessary trie nodes
// in circumstances similar to the following:
// //
// Consider nodes `A` and `B` who share the same full node parent `P` and have no other siblings. // Consider nodes `A` and `B` who share the same full node parent `P` and have no other siblings.
// During the execution of a block: // During the execution of a block:
@ -347,21 +343,23 @@ func (s *stateObject) updateTrie() (Trie, error) {
// If the deletion is handled first, then `P` would be left with only one child, thus collapsed // If the deletion is handled first, then `P` would be left with only one child, thus collapsed
// into a shortnode. This requires `B` to be resolved from disk. // into a shortnode. This requires `B` to be resolved from disk.
// Whereas if the created node is handled first, then the collapse is avoided, and `B` is not resolved. // Whereas if the created node is handled first, then the collapse is avoided, and `B` is not resolved.
var deletions []common.Hash var (
for key, value := range s.pendingStorage { deletions []common.Hash
used = make([][]byte, 0, len(s.uncommittedStorage))
)
for key, origin := range s.uncommittedStorage {
// Skip noop changes, persist actual changes // Skip noop changes, persist actual changes
if value == s.originStorage[key] { value, exist := s.pendingStorage[key]
if value == origin {
log.Error("Storage update was noop", "address", s.address, "slot", key)
continue
}
if !exist {
log.Error("Storage slot is not found in pending area", s.address, "slot", key)
continue continue
} }
prev := s.originStorage[key]
s.originStorage[key] = value
var encoded []byte // rlp-encoded value to be used by the snapshot
if (value != common.Hash{}) { if (value != common.Hash{}) {
// Encoding []byte cannot fail, ok to ignore the error. if err := tr.UpdateStorage(s.address, key[:], common.TrimLeftZeroes(value[:])); err != nil {
trimmed := common.TrimLeftZeroes(value[:])
encoded, _ = rlp.EncodeToBytes(trimmed)
if err := tr.UpdateStorage(s.address, key[:], trimmed); err != nil {
s.db.setError(err) s.db.setError(err)
return nil, err return nil, err
} }
@ -369,39 +367,8 @@ func (s *stateObject) updateTrie() (Trie, error) {
} else { } else {
deletions = append(deletions, key) deletions = append(deletions, key)
} }
// Cache the mutated storage slots until commit
if storage == nil {
s.db.storagesLock.Lock()
if storage = s.db.storages[s.addrHash]; storage == nil {
storage = make(map[common.Hash][]byte)
s.db.storages[s.addrHash] = storage
}
s.db.storagesLock.Unlock()
}
khash := crypto.HashData(hasher, key[:])
storage[khash] = encoded // encoded will be nil if it's deleted
// Cache the original value of mutated storage slots
if origin == nil {
s.db.storagesLock.Lock()
if origin = s.db.storagesOrigin[s.address]; origin == nil {
origin = make(map[common.Hash][]byte)
s.db.storagesOrigin[s.address] = origin
}
s.db.storagesLock.Unlock()
}
// Track the original value of slot only if it's mutated first time
if _, ok := origin[khash]; !ok {
if prev == (common.Hash{}) {
origin[khash] = nil // nil if it was not present previously
} else {
// Encoding []byte cannot fail, ok to ignore the error.
b, _ := rlp.EncodeToBytes(common.TrimLeftZeroes(prev[:]))
origin[khash] = b
}
}
// Cache the items for preloading // Cache the items for preloading
usedStorage = append(usedStorage, common.CopyBytes(key[:])) // Copy needed for closure used = append(used, common.CopyBytes(key[:])) // Copy needed for closure
} }
for _, key := range deletions { for _, key := range deletions {
if err := tr.DeleteStorage(s.address, key[:]); err != nil { if err := tr.DeleteStorage(s.address, key[:]); err != nil {
@ -410,15 +377,10 @@ func (s *stateObject) updateTrie() (Trie, error) {
} }
s.db.StorageDeleted.Add(1) s.db.StorageDeleted.Add(1)
} }
// If no slots were touched, issue a warning as we shouldn't have done all
// the above work in the first place
if len(usedStorage) == 0 {
log.Error("State object update was noop", "addr", s.address, "slots", len(s.pendingStorage))
}
if s.db.prefetcher != nil { if s.db.prefetcher != nil {
s.db.prefetcher.used(s.addrHash, s.data.Root, usedStorage) s.db.prefetcher.used(s.addrHash, s.data.Root, used)
} }
s.pendingStorage = make(Storage) // reset pending map s.uncommittedStorage = make(Storage) // empties the commit markers
return tr, nil return tr, nil
} }
@ -434,30 +396,79 @@ func (s *stateObject) updateRoot() {
s.data.Root = tr.Hash() s.data.Root = tr.Hash()
} }
// commit obtains a set of dirty storage trie nodes and updates the account data. // commitStorage overwrites the clean storage with the storage changes and
// The returned set can be nil if nothing to commit. This function assumes all // fulfills the storage diffs into the given accountUpdate struct.
// storage mutations have already been flushed into trie by updateRoot. func (s *stateObject) commitStorage(op *accountUpdate) {
var (
buf = crypto.NewKeccakState()
encode = func(val common.Hash) []byte {
if val == (common.Hash{}) {
return nil
}
blob, _ := rlp.EncodeToBytes(common.TrimLeftZeroes(val[:]))
return blob
}
)
for key, val := range s.pendingStorage {
// Skip the noop storage changes, it might be possible the value
// of tracked slot is same in originStorage and pendingStorage
// map, e.g. the storage slot is modified in tx_a and then reset
// back in tx_b.
if val == s.originStorage[key] {
continue
}
hash := crypto.HashData(buf, key[:])
if op.storages == nil {
op.storages = make(map[common.Hash][]byte)
}
op.storages[hash] = encode(val)
if op.storagesOrigin == nil {
op.storagesOrigin = make(map[common.Hash][]byte)
}
op.storagesOrigin[hash] = encode(s.originStorage[key])
// Overwrite the clean value of storage slots
s.originStorage[key] = val
}
s.pendingStorage = make(Storage)
}
// commit obtains the account changes (metadata, storage slots, code) caused by
// state execution along with the dirty storage trie nodes.
// //
// Note, commit may run concurrently across all the state objects. Do not assume // Note, commit may run concurrently across all the state objects. Do not assume
// thread-safe access to the statedb. // thread-safe access to the statedb.
func (s *stateObject) commit() (*trienode.NodeSet, error) { func (s *stateObject) commit() (*accountUpdate, *trienode.NodeSet, error) {
// Short circuit if trie is not even loaded, don't bother with committing anything // commit the account metadata changes
if s.trie == nil { op := &accountUpdate{
address: s.address,
data: types.SlimAccountRLP(s.data),
}
if s.origin != nil {
op.origin = types.SlimAccountRLP(*s.origin)
}
// commit the contract code if it's modified
if s.dirtyCode {
op.code = &contractCode{
hash: common.BytesToHash(s.CodeHash()),
blob: s.code,
}
s.dirtyCode = false // reset the dirty flag
}
// Commit storage changes and the associated storage trie
s.commitStorage(op)
if len(op.storages) == 0 {
// nothing changed, don't bother to commit the trie
s.origin = s.data.Copy() s.origin = s.data.Copy()
return nil, nil return op, nil, nil
} }
// The trie is currently in an open state and could potentially contain
// cached mutations. Call commit to acquire a set of nodes that have been
// modified, the set can be nil if nothing to commit.
root, nodes, err := s.trie.Commit(false) root, nodes, err := s.trie.Commit(false)
if err != nil { if err != nil {
return nil, err return nil, nil, err
} }
s.data.Root = root s.data.Root = root
// Update original account data after commit
s.origin = s.data.Copy() s.origin = s.data.Copy()
return nodes, nil return op, nodes, nil
} }
// AddBalance adds amount to s's balance. // AddBalance adds amount to s's balance.
@ -500,18 +511,19 @@ func (s *stateObject) setBalance(amount *uint256.Int) {
func (s *stateObject) deepCopy(db *StateDB) *stateObject { func (s *stateObject) deepCopy(db *StateDB) *stateObject {
obj := &stateObject{ obj := &stateObject{
db: db, db: db,
address: s.address, address: s.address,
addrHash: s.addrHash, addrHash: s.addrHash,
origin: s.origin, origin: s.origin,
data: s.data, data: s.data,
code: s.code, code: s.code,
originStorage: s.originStorage.Copy(), originStorage: s.originStorage.Copy(),
pendingStorage: s.pendingStorage.Copy(), pendingStorage: s.pendingStorage.Copy(),
dirtyStorage: s.dirtyStorage.Copy(), dirtyStorage: s.dirtyStorage.Copy(),
dirtyCode: s.dirtyCode, uncommittedStorage: s.uncommittedStorage.Copy(),
selfDestructed: s.selfDestructed, dirtyCode: s.dirtyCode,
newContract: s.newContract, selfDestructed: s.selfDestructed,
newContract: s.newContract,
} }
if s.trie != nil { if s.trie != nil {
obj.trie = db.db.CopyTrie(s.trie) obj.trie = db.db.CopyTrie(s.trie)

@ -18,6 +18,7 @@
package state package state
import ( import (
"errors"
"fmt" "fmt"
"maps" "maps"
"math/big" "math/big"
@ -95,15 +96,6 @@ type StateDB struct {
// It will be updated when the Commit is called. // It will be updated when the Commit is called.
originalRoot common.Hash originalRoot common.Hash
// These maps hold the state changes (including the corresponding
// original value) that occurred in this **block**.
accounts map[common.Hash][]byte // The mutated accounts in 'slim RLP' encoding
accountsOrigin map[common.Address][]byte // The original value of mutated accounts in 'slim RLP' encoding
storages map[common.Hash]map[common.Hash][]byte // The mutated slots in prefix-zero trimmed rlp format
storagesOrigin map[common.Address]map[common.Hash][]byte // The original value of mutated slots in prefix-zero trimmed rlp format
storagesLock sync.Mutex // Mutex protecting the maps during concurrent updates/commits
// This map holds 'live' objects, which will get modified while // This map holds 'live' objects, which will get modified while
// processing a state transition. // processing a state transition.
stateObjects map[common.Address]*stateObject stateObjects map[common.Address]*stateObject
@ -171,9 +163,6 @@ type StateDB struct {
StorageUpdated atomic.Int64 StorageUpdated atomic.Int64
AccountDeleted int AccountDeleted int
StorageDeleted atomic.Int64 StorageDeleted atomic.Int64
// Testing hooks
onCommit func(states *triestate.Set) // Hook invoked when commit is performed
} }
// New creates a new state from a given trie. // New creates a new state from a given trie.
@ -187,10 +176,6 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error)
trie: tr, trie: tr,
originalRoot: root, originalRoot: root,
snaps: snaps, snaps: snaps,
accounts: make(map[common.Hash][]byte),
storages: make(map[common.Hash]map[common.Hash][]byte),
accountsOrigin: make(map[common.Address][]byte),
storagesOrigin: make(map[common.Address]map[common.Hash][]byte),
stateObjects: make(map[common.Address]*stateObject), stateObjects: make(map[common.Address]*stateObject),
stateObjectsDestruct: make(map[common.Address]*types.StateAccount), stateObjectsDestruct: make(map[common.Address]*types.StateAccount),
mutations: make(map[common.Address]*mutation), mutations: make(map[common.Address]*mutation),
@ -380,7 +365,7 @@ func (s *StateDB) GetCodeHash(addr common.Address) common.Hash {
return common.Hash{} return common.Hash{}
} }
// GetState retrieves a value from the given account's storage trie. // GetState retrieves the value associated with the specific key.
func (s *StateDB) GetState(addr common.Address, hash common.Hash) common.Hash { func (s *StateDB) GetState(addr common.Address, hash common.Hash) common.Hash {
stateObject := s.getStateObject(addr) stateObject := s.getStateObject(addr)
if stateObject != nil { if stateObject != nil {
@ -389,7 +374,8 @@ func (s *StateDB) GetState(addr common.Address, hash common.Hash) common.Hash {
return common.Hash{} return common.Hash{}
} }
// GetCommittedState retrieves a value from the given account's committed storage trie. // GetCommittedState retrieves the value associated with the specific key
// without any mutations caused in the current execution.
func (s *StateDB) GetCommittedState(addr common.Address, hash common.Hash) common.Hash { func (s *StateDB) GetCommittedState(addr common.Address, hash common.Hash) common.Hash {
stateObject := s.getStateObject(addr) stateObject := s.getStateObject(addr)
if stateObject != nil { if stateObject != nil {
@ -557,22 +543,6 @@ func (s *StateDB) updateStateObject(obj *stateObject) {
if obj.dirtyCode { if obj.dirtyCode {
s.trie.UpdateContractCode(obj.Address(), common.BytesToHash(obj.CodeHash()), obj.code) s.trie.UpdateContractCode(obj.Address(), common.BytesToHash(obj.CodeHash()), obj.code)
} }
// Cache the data until commit. Note, this update mechanism is not symmetric
// to the deletion, because whereas it is enough to track account updates
// at commit time, deletions need tracking at transaction boundary level to
// ensure we capture state clearing.
s.accounts[obj.addrHash] = types.SlimAccountRLP(obj.data)
// Track the original value of mutated account, nil means it was not present.
// Skip if it has been tracked (because updateStateObject may be called
// multiple times in a block).
if _, ok := s.accountsOrigin[obj.address]; !ok {
if obj.origin == nil {
s.accountsOrigin[obj.address] = nil
} else {
s.accountsOrigin[obj.address] = types.SlimAccountRLP(*obj.origin)
}
}
} }
// deleteStateObject removes the given object from the state trie. // deleteStateObject removes the given object from the state trie.
@ -691,10 +661,6 @@ func (s *StateDB) Copy() *StateDB {
trie: s.db.CopyTrie(s.trie), trie: s.db.CopyTrie(s.trie),
hasher: crypto.NewKeccakState(), hasher: crypto.NewKeccakState(),
originalRoot: s.originalRoot, originalRoot: s.originalRoot,
accounts: copySet(s.accounts),
storages: copy2DSet(s.storages),
accountsOrigin: copySet(s.accountsOrigin),
storagesOrigin: copy2DSet(s.storagesOrigin),
stateObjects: make(map[common.Address]*stateObject, len(s.stateObjects)), stateObjects: make(map[common.Address]*stateObject, len(s.stateObjects)),
stateObjectsDestruct: maps.Clone(s.stateObjectsDestruct), stateObjectsDestruct: maps.Clone(s.stateObjectsDestruct),
mutations: make(map[common.Address]*mutation, len(s.mutations)), mutations: make(map[common.Address]*mutation, len(s.mutations)),
@ -803,13 +769,6 @@ func (s *StateDB) Finalise(deleteEmptyObjects bool) {
if _, ok := s.stateObjectsDestruct[obj.address]; !ok { if _, ok := s.stateObjectsDestruct[obj.address]; !ok {
s.stateObjectsDestruct[obj.address] = obj.origin s.stateObjectsDestruct[obj.address] = obj.origin
} }
// Note, we can't do this only at the end of a block because multiple
// transactions within the same block might self destruct and then
// resurrect an account; but the snapshotter needs both events.
delete(s.accounts, obj.addrHash) // Clear out any previously updated account data (may be recreated via a resurrect)
delete(s.storages, obj.addrHash) // Clear out any previously updated storage data (may be recreated via a resurrect)
delete(s.accountsOrigin, obj.address) // Clear out any previously updated account data (may be recreated via a resurrect)
delete(s.storagesOrigin, obj.address) // Clear out any previously updated storage data (may be recreated via a resurrect)
} else { } else {
obj.finalise() obj.finalise()
s.markUpdate(addr) s.markUpdate(addr)
@ -1020,10 +979,9 @@ func (s *StateDB) slowDeleteStorage(addr common.Address, addrHash common.Hash, r
} }
// deleteStorage is designed to delete the storage trie of a designated account. // deleteStorage is designed to delete the storage trie of a designated account.
// It could potentially be terminated if the storage size is excessively large, // The function will make an attempt to utilize an efficient strategy if the
// potentially leading to an out-of-memory panic. The function will make an attempt // associated state snapshot is reachable; otherwise, it will resort to a less
// to utilize an efficient strategy if the associated state snapshot is reachable; // efficient approach.
// otherwise, it will resort to a less-efficient approach.
func (s *StateDB) deleteStorage(addr common.Address, addrHash common.Hash, root common.Hash) (map[common.Hash][]byte, *trienode.NodeSet, error) { func (s *StateDB) deleteStorage(addr common.Address, addrHash common.Hash, root common.Hash) (map[common.Hash][]byte, *trienode.NodeSet, error) {
var ( var (
start = time.Now() start = time.Now()
@ -1058,75 +1016,61 @@ func (s *StateDB) deleteStorage(addr common.Address, addrHash common.Hash, root
} }
// handleDestruction processes all destruction markers and deletes the account // handleDestruction processes all destruction markers and deletes the account
// and associated storage slots if necessary. There are four possible situations // and associated storage slots if necessary. There are four potential scenarios
// here: // as following:
//
// - the account was not existent and be marked as destructed
//
// - the account was not existent and be marked as destructed,
// however, it's resurrected later in the same block.
// //
// - the account was existent and be marked as destructed // (a) the account was not existent and be marked as destructed
// // (b) the account was not existent and be marked as destructed,
// - the account was existent and be marked as destructed, // however, it's resurrected later in the same block.
// however it's resurrected later in the same block. // (c) the account was existent and be marked as destructed
// (d) the account was existent and be marked as destructed,
// however it's resurrected later in the same block.
// //
// In case (a), nothing needs be deleted, nil to nil transition can be ignored. // In case (a), nothing needs be deleted, nil to nil transition can be ignored.
//
// In case (b), nothing needs be deleted, nil is used as the original value for // In case (b), nothing needs be deleted, nil is used as the original value for
// newly created account and storages // newly created account and storages
//
// In case (c), **original** account along with its storages should be deleted, // In case (c), **original** account along with its storages should be deleted,
// with their values be tracked as original value. // with their values be tracked as original value.
//
// In case (d), **original** account along with its storages should be deleted, // In case (d), **original** account along with its storages should be deleted,
// with their values be tracked as original value. // with their values be tracked as original value.
func (s *StateDB) handleDestruction(nodes *trienode.MergedNodeSet) error { func (s *StateDB) handleDestruction() (map[common.Hash]*accountDelete, []*trienode.NodeSet, error) {
// Short circuit if geth is running with hash mode. This procedure can consume var (
// considerable time and storage deletion isn't supported in hash mode, thus nodes []*trienode.NodeSet
// preemptively avoiding unnecessary expenses. buf = crypto.NewKeccakState()
if s.db.TrieDB().Scheme() == rawdb.HashScheme { deletes = make(map[common.Hash]*accountDelete)
return nil )
}
for addr, prev := range s.stateObjectsDestruct { for addr, prev := range s.stateObjectsDestruct {
// The original account was non-existing, and it's marked as destructed // The account was non-existent, and it's marked as destructed in the scope
// in the scope of block. It can be case (a) or (b). // of block. It can be either case (a) or (b) and will be interpreted as
// - for (a), skip it without doing anything. // null->null state transition.
// - for (b), track account's original value as nil. It may overwrite // - for (a), skip it without doing anything
// the data cached in s.accountsOrigin set by 'updateStateObject'. // - for (b), the resurrected account with nil as original will be handled afterwards
addrHash := crypto.Keccak256Hash(addr[:])
if prev == nil { if prev == nil {
if _, ok := s.accounts[addrHash]; ok {
s.accountsOrigin[addr] = nil // case (b)
}
continue continue
} }
// It can overwrite the data in s.accountsOrigin set by 'updateStateObject'. // The account was existent, it can be either case (c) or (d).
s.accountsOrigin[addr] = types.SlimAccountRLP(*prev) // case (c) or (d) addrHash := crypto.HashData(buf, addr.Bytes())
op := &accountDelete{
address: addr,
origin: types.SlimAccountRLP(*prev),
}
deletes[addrHash] = op
// Short circuit if the storage was empty. // Short circuit if the origin storage was empty.
if prev.Root == types.EmptyRootHash { if prev.Root == types.EmptyRootHash {
continue continue
} }
// Remove storage slots belong to the account. // Remove storage slots belonging to the account.
slots, set, err := s.deleteStorage(addr, addrHash, prev.Root) slots, set, err := s.deleteStorage(addr, addrHash, prev.Root)
if err != nil { if err != nil {
return fmt.Errorf("failed to delete storage, err: %w", err) return nil, nil, fmt.Errorf("failed to delete storage, err: %w", err)
}
if s.storagesOrigin[addr] == nil {
s.storagesOrigin[addr] = slots
} else {
// It can overwrite the data in s.storagesOrigin[addrHash] set by
// 'object.updateTrie'.
for key, val := range slots {
s.storagesOrigin[addr][key] = val
}
}
if err := nodes.Merge(set); err != nil {
return err
} }
op.storagesOrigin = slots
// Aggregate the associated trie node changes.
nodes = append(nodes, set)
} }
return nil return deletes, nodes, nil
} }
// GetTrie returns the account trie. // GetTrie returns the account trie.
@ -1134,18 +1078,12 @@ func (s *StateDB) GetTrie() Trie {
return s.trie return s.trie
} }
// Commit writes the state to the underlying in-memory trie database. // commit gathers the state mutations accumulated along with the associated
// Once the state is committed, tries cached in stateDB (including account // trie changes, resetting all internal flags with the new state as the base.
// trie, storage tries) will no longer be functional. A new state instance func (s *StateDB) commit(deleteEmptyObjects bool) (*stateUpdate, error) {
// must be created with new root and updated database for accessing post-
// commit states.
//
// The associated block number of the state transition is also provided
// for more chain context.
func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, error) {
// Short circuit in case any database failure occurred earlier. // Short circuit in case any database failure occurred earlier.
if s.dbErr != nil { if s.dbErr != nil {
return common.Hash{}, fmt.Errorf("commit aborted due to earlier error: %v", s.dbErr) return nil, fmt.Errorf("commit aborted due to earlier error: %v", s.dbErr)
} }
// Finalize any pending changes and merge everything into the tries // Finalize any pending changes and merge everything into the tries
s.IntermediateRoot(deleteEmptyObjects) s.IntermediateRoot(deleteEmptyObjects)
@ -1156,19 +1094,56 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, er
accountTrieNodesDeleted int accountTrieNodesDeleted int
storageTrieNodesUpdated int storageTrieNodesUpdated int
storageTrieNodesDeleted int storageTrieNodesDeleted int
nodes = trienode.NewMergedNodeSet()
lock sync.Mutex // protect two maps below
nodes = trienode.NewMergedNodeSet() // aggregated trie nodes
updates = make(map[common.Hash]*accountUpdate, len(s.mutations)) // aggregated account updates
// merge aggregates the dirty trie nodes into the global set.
//
// Given that some accounts may be destroyed and then recreated within
// the same block, it's possible that a node set with the same owner
// may already exists. In such cases, these two sets are combined, with
// the later one overwriting the previous one if any nodes are modified
// or deleted in both sets.
//
// merge run concurrently across all the state objects and account trie.
merge = func(set *trienode.NodeSet) error {
if set == nil {
return nil
}
lock.Lock()
defer lock.Unlock()
updates, deletes := set.Size()
if set.Owner == (common.Hash{}) {
accountTrieNodesUpdated += updates
accountTrieNodesDeleted += deletes
} else {
storageTrieNodesUpdated += updates
storageTrieNodesDeleted += deletes
}
return nodes.Merge(set)
}
) )
// Handle all state deletions first // Given that some accounts could be destroyed and then recreated within
if err := s.handleDestruction(nodes); err != nil { // the same block, account deletions must be processed first. This ensures
return common.Hash{}, err // that the storage trie nodes deleted during destruction and recreated
// during subsequent resurrection can be combined correctly.
deletes, delNodes, err := s.handleDestruction()
if err != nil {
return nil, err
}
for _, set := range delNodes {
if err := merge(set); err != nil {
return nil, err
}
} }
// Handle all state updates afterwards, concurrently to one another to shave // Handle all state updates afterwards, concurrently to one another to shave
// off some milliseconds from the commit operation. Also accumulate the code // off some milliseconds from the commit operation. Also accumulate the code
// writes to run in parallel with the computations. // writes to run in parallel with the computations.
start := time.Now()
var ( var (
code = s.db.DiskDB().NewBatch() start = time.Now()
lock sync.Mutex
root common.Hash root common.Hash
workers errgroup.Group workers errgroup.Group
) )
@ -1189,15 +1164,8 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, er
} }
root = newroot root = newroot
// Merge the dirty nodes of account trie into global set if err := merge(set); err != nil {
lock.Lock() return err
defer lock.Unlock()
if set != nil {
if err = nodes.Merge(set); err != nil {
return err
}
accountTrieNodesUpdated, accountTrieNodesDeleted = set.Size()
} }
s.AccountCommits = time.Since(start) s.AccountCommits = time.Since(start)
return nil return nil
@ -1215,49 +1183,29 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, er
} }
// Write any contract code associated with the state object // Write any contract code associated with the state object
obj := s.stateObjects[addr] obj := s.stateObjects[addr]
if obj.code != nil && obj.dirtyCode { if obj == nil {
rawdb.WriteCode(code, common.BytesToHash(obj.CodeHash()), obj.code) return nil, errors.New("missing state object")
obj.dirtyCode = false
} }
// Run the storage updates concurrently to one another // Run the storage updates concurrently to one another
workers.Go(func() error { workers.Go(func() error {
// Write any storage changes in the state object to its storage trie // Write any storage changes in the state object to its storage trie
set, err := obj.commit() update, set, err := obj.commit()
if err != nil { if err != nil {
return err return err
} }
// Merge the dirty nodes of storage trie into global set. It is possible if err := merge(set); err != nil {
// that the account was destructed and then resurrected in the same block. return err
// In this case, the node set is shared by both accounts.
lock.Lock()
defer lock.Unlock()
if set != nil {
if err = nodes.Merge(set); err != nil {
return err
}
updates, deleted := set.Size()
storageTrieNodesUpdated += updates
storageTrieNodesDeleted += deleted
} }
lock.Lock()
updates[obj.addrHash] = update
lock.Unlock()
s.StorageCommits = time.Since(start) // overwrite with the longest storage commit runtime s.StorageCommits = time.Since(start) // overwrite with the longest storage commit runtime
return nil return nil
}) })
} }
// Schedule the code commits to run concurrently too. This shouldn't really
// take much since we don't often commit code, but since it's disk access,
// it's always yolo.
workers.Go(func() error {
if code.ValueSize() > 0 {
if err := code.Write(); err != nil {
log.Crit("Failed to commit dirty codes", "error", err)
}
}
return nil
})
// Wait for everything to finish and update the metrics // Wait for everything to finish and update the metrics
if err := workers.Wait(); err != nil { if err := workers.Wait(); err != nil {
return common.Hash{}, err return nil, err
} }
accountUpdatedMeter.Mark(int64(s.AccountUpdated)) accountUpdatedMeter.Mark(int64(s.AccountUpdated))
storageUpdatedMeter.Mark(s.StorageUpdated.Load()) storageUpdatedMeter.Mark(s.StorageUpdated.Load())
@ -1271,53 +1219,78 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, er
s.StorageUpdated.Store(0) s.StorageUpdated.Store(0)
s.StorageDeleted.Store(0) s.StorageDeleted.Store(0)
// If snapshotting is enabled, update the snapshot tree with this new version // Clear all internal flags and update state root at the end.
if s.snap != nil { s.mutations = make(map[common.Address]*mutation)
start = time.Now() s.stateObjectsDestruct = make(map[common.Address]*types.StateAccount)
// Only update if there's a state transition (skip empty Clique blocks)
if parent := s.snap.Root(); parent != root { origin := s.originalRoot
if err := s.snaps.Update(root, parent, s.convertAccountSet(s.stateObjectsDestruct), s.accounts, s.storages); err != nil { s.originalRoot = root
log.Warn("Failed to update snapshot tree", "from", parent, "to", root, "err", err) return newStateUpdate(origin, root, deletes, updates, nodes), nil
}
// commitAndFlush is a wrapper of commit which also commits the state mutations
// to the configured data stores.
func (s *StateDB) commitAndFlush(block uint64, deleteEmptyObjects bool) (*stateUpdate, error) {
ret, err := s.commit(deleteEmptyObjects)
if err != nil {
return nil, err
}
// Commit dirty contract code if any exists
if db := s.db.DiskDB(); db != nil && len(ret.codes) > 0 {
batch := db.NewBatch()
for _, code := range ret.codes {
rawdb.WriteCode(batch, code.hash, code.blob)
}
if err := batch.Write(); err != nil {
return nil, err
}
}
if !ret.empty() {
// If snapshotting is enabled, update the snapshot tree with this new version
if s.snap != nil {
s.snap = nil
start := time.Now()
if err := s.snaps.Update(ret.root, ret.originRoot, ret.destructs, ret.accounts, ret.storages); err != nil {
log.Warn("Failed to update snapshot tree", "from", ret.originRoot, "to", ret.root, "err", err)
} }
// Keep TriesInMemory diff layers in the memory, persistent layer is 129th. // Keep 128 diff layers in the memory, persistent layer is 129th.
// - head layer is paired with HEAD state // - head layer is paired with HEAD state
// - head-1 layer is paired with HEAD-1 state // - head-1 layer is paired with HEAD-1 state
// - head-127 layer(bottom-most diff layer) is paired with HEAD-127 state // - head-127 layer(bottom-most diff layer) is paired with HEAD-127 state
if err := s.snaps.Cap(root, TriesInMemory); err != nil { if err := s.snaps.Cap(ret.root, TriesInMemory); err != nil {
log.Warn("Failed to cap snapshot tree", "root", root, "layers", TriesInMemory, "err", err) log.Warn("Failed to cap snapshot tree", "root", ret.root, "layers", TriesInMemory, "err", err)
} }
s.SnapshotCommits += time.Since(start)
} }
s.SnapshotCommits += time.Since(start) // If trie database is enabled, commit the state update as a new layer
s.snap = nil if db := s.db.TrieDB(); db != nil {
} start := time.Now()
if root == (common.Hash{}) { set := triestate.New(ret.accountsOrigin, ret.storagesOrigin)
root = types.EmptyRootHash if err := db.Update(ret.root, ret.originRoot, block, ret.nodes, set); err != nil {
} return nil, err
origin := s.originalRoot }
if origin == (common.Hash{}) { s.TrieDBCommits += time.Since(start)
origin = types.EmptyRootHash
}
if root != origin {
start = time.Now()
set := triestate.New(s.accountsOrigin, s.storagesOrigin)
if err := s.db.TrieDB().Update(root, origin, block, nodes, set); err != nil {
return common.Hash{}, err
} }
s.originalRoot = root }
s.TrieDBCommits += time.Since(start) return ret, err
}
if s.onCommit != nil { // Commit writes the state mutations into the configured data stores.
s.onCommit(set) //
} // Once the state is committed, tries cached in stateDB (including account
// trie, storage tries) will no longer be functional. A new state instance
// must be created with new root and updated database for accessing post-
// commit states.
//
// The associated block number of the state transition is also provided
// for more chain context.
func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, error) {
ret, err := s.commitAndFlush(block, deleteEmptyObjects)
if err != nil {
return common.Hash{}, err
} }
// Clear all internal flags at the end of commit operation. return ret.root, nil
s.accounts = make(map[common.Hash][]byte)
s.storages = make(map[common.Hash]map[common.Hash][]byte)
s.accountsOrigin = make(map[common.Address][]byte)
s.storagesOrigin = make(map[common.Address]map[common.Hash][]byte)
s.mutations = make(map[common.Address]*mutation)
s.stateObjectsDestruct = make(map[common.Address]*types.StateAccount)
return root, nil
} }
// Prepare handles the preparatory steps for executing a state transition with. // Prepare handles the preparatory steps for executing a state transition with.
@ -1399,41 +1372,9 @@ func (s *StateDB) SlotInAccessList(addr common.Address, slot common.Hash) (addre
return s.accessList.Contains(addr, slot) return s.accessList.Contains(addr, slot)
} }
// convertAccountSet converts a provided account set from address keyed to hash keyed. // markDelete is invoked when an account is deleted but the deletion is
func (s *StateDB) convertAccountSet(set map[common.Address]*types.StateAccount) map[common.Hash]struct{} { // not yet committed. The pending mutation is cached and will be applied
ret := make(map[common.Hash]struct{}, len(set)) // all together
for addr := range set {
obj, exist := s.stateObjects[addr]
if !exist {
ret[crypto.Keccak256Hash(addr[:])] = struct{}{}
} else {
ret[obj.addrHash] = struct{}{}
}
}
return ret
}
// copySet returns a deep-copied set.
func copySet[k comparable](set map[k][]byte) map[k][]byte {
copied := make(map[k][]byte, len(set))
for key, val := range set {
copied[key] = common.CopyBytes(val)
}
return copied
}
// copy2DSet returns a two-dimensional deep-copied set.
func copy2DSet[k comparable](set map[k]map[common.Hash][]byte) map[k]map[common.Hash][]byte {
copied := make(map[k]map[common.Hash][]byte, len(set))
for addr, subset := range set {
copied[addr] = make(map[common.Hash][]byte, len(subset))
for key, val := range subset {
copied[addr][key] = common.CopyBytes(val)
}
}
return copied
}
func (s *StateDB) markDelete(addr common.Address) { func (s *StateDB) markDelete(addr common.Address) {
if _, ok := s.mutations[addr]; !ok { if _, ok := s.mutations[addr]; !ok {
s.mutations[addr] = &mutation{} s.mutations[addr] = &mutation{}

@ -36,7 +36,6 @@ import (
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
"github.com/ethereum/go-ethereum/trie/triestate"
"github.com/ethereum/go-ethereum/triedb" "github.com/ethereum/go-ethereum/triedb"
"github.com/ethereum/go-ethereum/triedb/pathdb" "github.com/ethereum/go-ethereum/triedb/pathdb"
"github.com/holiman/uint256" "github.com/holiman/uint256"
@ -180,9 +179,21 @@ func (test *stateTest) run() bool {
roots []common.Hash roots []common.Hash
accountList []map[common.Address][]byte accountList []map[common.Address][]byte
storageList []map[common.Address]map[common.Hash][]byte storageList []map[common.Address]map[common.Hash][]byte
onCommit = func(states *triestate.Set) { copyUpdate = func(update *stateUpdate) {
accountList = append(accountList, copySet(states.Accounts)) accounts := make(map[common.Address][]byte, len(update.accountsOrigin))
storageList = append(storageList, copy2DSet(states.Storages)) for key, val := range update.accountsOrigin {
accounts[key] = common.CopyBytes(val)
}
accountList = append(accountList, accounts)
storages := make(map[common.Address]map[common.Hash][]byte, len(update.storagesOrigin))
for addr, subset := range update.storagesOrigin {
storages[addr] = make(map[common.Hash][]byte, len(subset))
for key, val := range subset {
storages[addr][key] = common.CopyBytes(val)
}
}
storageList = append(storageList, storages)
} }
disk = rawdb.NewMemoryDatabase() disk = rawdb.NewMemoryDatabase()
tdb = triedb.NewDatabase(disk, &triedb.Config{PathDB: pathdb.Defaults}) tdb = triedb.NewDatabase(disk, &triedb.Config{PathDB: pathdb.Defaults})
@ -210,8 +221,6 @@ func (test *stateTest) run() bool {
if err != nil { if err != nil {
panic(err) panic(err)
} }
state.onCommit = onCommit
for i, action := range actions { for i, action := range actions {
if i%test.chunk == 0 && i != 0 { if i%test.chunk == 0 && i != 0 {
if byzantium { if byzantium {
@ -227,14 +236,15 @@ func (test *stateTest) run() bool {
} else { } else {
state.IntermediateRoot(true) // call intermediateRoot at the transaction boundary state.IntermediateRoot(true) // call intermediateRoot at the transaction boundary
} }
nroot, err := state.Commit(0, true) // call commit at the block boundary ret, err := state.commitAndFlush(0, true) // call commit at the block boundary
if err != nil { if err != nil {
panic(err) panic(err)
} }
if nroot == root { if ret.empty() {
return true // filter out non-change state transition return true
} }
roots = append(roots, nroot) copyUpdate(ret)
roots = append(roots, ret.root)
} }
for i := 0; i < len(test.actions); i++ { for i := 0; i < len(test.actions); i++ {
root := types.EmptyRootHash root := types.EmptyRootHash

@ -0,0 +1,133 @@
// 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 state
import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/trie/trienode"
)
// contractCode represents a contract code with associated metadata.
type contractCode struct {
hash common.Hash // hash is the cryptographic hash of the contract code.
blob []byte // blob is the binary representation of the contract code.
}
// accountDelete represents an operation for deleting an Ethereum account.
type accountDelete struct {
address common.Address // address is the unique account identifier
origin []byte // origin is the original value of account data in slim-RLP encoding.
storagesOrigin map[common.Hash][]byte // storagesOrigin stores the original values of mutated slots in prefix-zero-trimmed RLP format.
}
// accountUpdate represents an operation for updating an Ethereum account.
type accountUpdate struct {
address common.Address // address is the unique account identifier
data []byte // data is the slim-RLP encoded account data.
origin []byte // origin is the original value of account data in slim-RLP encoding.
code *contractCode // code represents mutated contract code; nil means it's not modified.
storages map[common.Hash][]byte // storages stores mutated slots in prefix-zero-trimmed RLP format.
storagesOrigin map[common.Hash][]byte // storagesOrigin stores the original values of mutated slots in prefix-zero-trimmed RLP format.
}
// stateUpdate represents the difference between two states resulting from state
// execution. It contains information about mutated contract codes, accounts,
// and storage slots, along with their original values.
type stateUpdate struct {
originRoot common.Hash // hash of the state before applying mutation
root common.Hash // hash of the state after applying mutation
destructs map[common.Hash]struct{} // destructs contains the list of destructed accounts
accounts map[common.Hash][]byte // accounts stores mutated accounts in 'slim RLP' encoding
accountsOrigin map[common.Address][]byte // accountsOrigin stores the original values of mutated accounts in 'slim RLP' encoding
storages map[common.Hash]map[common.Hash][]byte // storages stores mutated slots in 'prefix-zero-trimmed' RLP format
storagesOrigin map[common.Address]map[common.Hash][]byte // storagesOrigin stores the original values of mutated slots in 'prefix-zero-trimmed' RLP format
codes map[common.Address]contractCode // codes contains the set of dirty codes
nodes *trienode.MergedNodeSet // Aggregated dirty nodes caused by state changes
}
// empty returns a flag indicating the state transition is empty or not.
func (sc *stateUpdate) empty() bool {
return sc.originRoot == sc.root
}
// newStateUpdate constructs a state update object, representing the differences
// between two states by performing state execution. It aggregates the given
// account deletions and account updates to form a comprehensive state update.
func newStateUpdate(originRoot common.Hash, root common.Hash, deletes map[common.Hash]*accountDelete, updates map[common.Hash]*accountUpdate, nodes *trienode.MergedNodeSet) *stateUpdate {
var (
destructs = make(map[common.Hash]struct{})
accounts = make(map[common.Hash][]byte)
accountsOrigin = make(map[common.Address][]byte)
storages = make(map[common.Hash]map[common.Hash][]byte)
storagesOrigin = make(map[common.Address]map[common.Hash][]byte)
codes = make(map[common.Address]contractCode)
)
// Due to the fact that some accounts could be destructed and resurrected
// within the same block, the deletions must be aggregated first.
for addrHash, op := range deletes {
addr := op.address
destructs[addrHash] = struct{}{}
accountsOrigin[addr] = op.origin
if len(op.storagesOrigin) > 0 {
storagesOrigin[addr] = op.storagesOrigin
}
}
// Aggregate account updates then.
for addrHash, op := range updates {
// Aggregate dirty contract codes if they are available.
addr := op.address
if op.code != nil {
codes[addr] = *op.code
}
// Aggregate the account changes. The original account value will only
// be tracked if it's not present yet.
accounts[addrHash] = op.data
if _, found := accountsOrigin[addr]; !found {
accountsOrigin[addr] = op.origin
}
// Aggregate the storage changes. The original storage slot value will
// only be tracked if it's not present yet.
if len(op.storages) > 0 {
storages[addrHash] = op.storages
}
if len(op.storagesOrigin) > 0 {
origin := storagesOrigin[addr]
if origin == nil {
storagesOrigin[addr] = op.storagesOrigin
continue
}
for key, slot := range op.storagesOrigin {
if _, found := origin[key]; !found {
origin[key] = slot
}
}
storagesOrigin[addr] = origin
}
}
return &stateUpdate{
originRoot: types.TrieRootHash(originRoot),
root: types.TrieRootHash(root),
destructs: destructs,
accounts: accounts,
accountsOrigin: accountsOrigin,
storages: storages,
storagesOrigin: storagesOrigin,
codes: codes,
nodes: nodes,
}
}
Loading…
Cancel
Save