mirror of https://github.com/ethereum/go-ethereum
all: implement path-based state scheme (#25963)
* all: implement path-based state scheme * all: edits from review * core/rawdb, trie/triedb/pathdb: review changes * core, light, trie, eth, tests: reimplement pbss history * core, trie/triedb/pathdb: track block number in state history * trie/triedb/pathdb: add history documentation * core, trie/triedb/pathdb: address comments from Peter's review Important changes to list: - Cache trie nodes by path in clean cache - Remove root->id mappings when history is truncated * trie/triedb/pathdb: fallback to disk if unexpect node in clean cache * core/rawdb: fix tests * trie/triedb/pathdb: rename metrics, change clean cache key * trie/triedb: manage the clean cache inside of disk layer * trie/triedb/pathdb: move journal function * trie/triedb/path: fix tests * trie/triedb/pathdb: fix journal * trie/triedb/pathdb: fix history * trie/triedb/pathdb: try to fix tests on windows * core, trie: address comments * trie/triedb/pathdb: fix test issues --------- Co-authored-by: Felix Lange <fjl@twurst.com> Co-authored-by: Martin Holst Swende <martin@swende.se>pull/27833/head
parent
9d744f0ca8
commit
7de748d3f6
@ -0,0 +1,61 @@ |
||||
// Copyright 2023 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 testutil |
||||
|
||||
import ( |
||||
crand "crypto/rand" |
||||
"encoding/binary" |
||||
mrand "math/rand" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
) |
||||
|
||||
// Prng is a pseudo random number generator seeded by strong randomness.
|
||||
// The randomness is printed on startup in order to make failures reproducible.
|
||||
var prng = initRand() |
||||
|
||||
func initRand() *mrand.Rand { |
||||
var seed [8]byte |
||||
crand.Read(seed[:]) |
||||
rnd := mrand.New(mrand.NewSource(int64(binary.LittleEndian.Uint64(seed[:])))) |
||||
return rnd |
||||
} |
||||
|
||||
// RandBytes generates a random byte slice with specified length.
|
||||
func RandBytes(n int) []byte { |
||||
r := make([]byte, n) |
||||
prng.Read(r) |
||||
return r |
||||
} |
||||
|
||||
// RandomHash generates a random blob of data and returns it as a hash.
|
||||
func RandomHash() common.Hash { |
||||
return common.BytesToHash(RandBytes(common.HashLength)) |
||||
} |
||||
|
||||
// RandomAddress generates a random blob of data and returns it as an address.
|
||||
func RandomAddress() common.Address { |
||||
return common.BytesToAddress(RandBytes(common.AddressLength)) |
||||
} |
||||
|
||||
// RandomNode generates a random node.
|
||||
func RandomNode() *trienode.Node { |
||||
val := RandBytes(100) |
||||
return trienode.New(crypto.Keccak256Hash(val), val) |
||||
} |
@ -0,0 +1,392 @@ |
||||
// Copyright 2022 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 ( |
||||
"errors" |
||||
"fmt" |
||||
"io" |
||||
"sync" |
||||
"time" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/ethdb" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/params" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
// maxDiffLayers is the maximum diff layers allowed in the layer tree.
|
||||
const maxDiffLayers = 128 |
||||
|
||||
// layer is the interface implemented by all state layers which includes some
|
||||
// public methods and some additional methods for internal usage.
|
||||
type layer interface { |
||||
// Node retrieves the trie node with the node info. An error will be returned
|
||||
// if the read operation exits abnormally. For example, if the layer is already
|
||||
// stale, or the associated state is regarded as corrupted. Notably, no error
|
||||
// will be returned if the requested node is not found in database.
|
||||
Node(owner common.Hash, path []byte, hash common.Hash) ([]byte, error) |
||||
|
||||
// rootHash returns the root hash for which this layer was made.
|
||||
rootHash() common.Hash |
||||
|
||||
// stateID returns the associated state id of layer.
|
||||
stateID() uint64 |
||||
|
||||
// parentLayer returns the subsequent layer of it, or nil if the disk was reached.
|
||||
parentLayer() layer |
||||
|
||||
// update creates a new layer on top of the existing layer diff tree with
|
||||
// the provided dirty trie nodes along with the state change set.
|
||||
//
|
||||
// Note, the maps are retained by the method to avoid copying everything.
|
||||
update(root common.Hash, id uint64, block uint64, nodes map[common.Hash]map[string]*trienode.Node, states *triestate.Set) *diffLayer |
||||
|
||||
// journal commits an entire diff hierarchy to disk into a single journal entry.
|
||||
// This is meant to be used during shutdown to persist the layer without
|
||||
// flattening everything down (bad for reorgs).
|
||||
journal(w io.Writer) error |
||||
} |
||||
|
||||
// Config contains the settings for database.
|
||||
type Config struct { |
||||
StateLimit uint64 // Number of recent blocks to maintain state history for
|
||||
CleanSize int // Maximum memory allowance (in bytes) for caching clean nodes
|
||||
DirtySize int // Maximum memory allowance (in bytes) for caching dirty nodes
|
||||
ReadOnly bool // Flag whether the database is opened in read only mode.
|
||||
} |
||||
|
||||
var ( |
||||
// defaultCleanSize is the default memory allowance of clean cache.
|
||||
defaultCleanSize = 16 * 1024 * 1024 |
||||
|
||||
// defaultBufferSize is the default memory allowance of node buffer
|
||||
// that aggregates the writes from above until it's flushed into the
|
||||
// disk. Do not increase the buffer size arbitrarily, otherwise the
|
||||
// system pause time will increase when the database writes happen.
|
||||
defaultBufferSize = 128 * 1024 * 1024 |
||||
) |
||||
|
||||
// Defaults contains default settings for Ethereum mainnet.
|
||||
var Defaults = &Config{ |
||||
StateLimit: params.FullImmutabilityThreshold, |
||||
CleanSize: defaultCleanSize, |
||||
DirtySize: defaultBufferSize, |
||||
} |
||||
|
||||
// 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.
|
||||
//
|
||||
// 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.
|
||||
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
|
||||
// the shutdown to reject all following unexpected mutations.
|
||||
readOnly bool // Indicator if database is opened in read only mode
|
||||
bufferSize int // Memory allowance (in bytes) for caching dirty nodes
|
||||
config *Config // Configuration for database
|
||||
diskdb ethdb.Database // Persistent storage for matured trie nodes
|
||||
tree *layerTree // The group for all known layers
|
||||
freezer *rawdb.ResettableFreezer // Freezer for storing trie histories, nil possible in tests
|
||||
lock sync.RWMutex // Lock to prevent mutations from happening at the same time
|
||||
} |
||||
|
||||
// New attempts to load an already existing layer from a persistent key-value
|
||||
// store (with a number of memory layers from a journal). If the journal is not
|
||||
// matched with the base persistent layer, all the recorded diff layers are discarded.
|
||||
func New(diskdb ethdb.Database, config *Config) *Database { |
||||
if config == nil { |
||||
config = Defaults |
||||
} |
||||
db := &Database{ |
||||
readOnly: config.ReadOnly, |
||||
bufferSize: config.DirtySize, |
||||
config: config, |
||||
diskdb: diskdb, |
||||
} |
||||
// Construct the layer tree by resolving the in-disk singleton state
|
||||
// and in-memory layer journal.
|
||||
db.tree = newLayerTree(db.loadLayers()) |
||||
|
||||
// Open the freezer for state history if the passed database contains an
|
||||
// ancient store. Otherwise, all the relevant functionalities are disabled.
|
||||
//
|
||||
// Because the freezer can only be opened once at the same time, this
|
||||
// mechanism also ensures that at most one **non-readOnly** database
|
||||
// is opened at the same time to prevent accidental mutation.
|
||||
if ancient, err := diskdb.AncientDatadir(); err == nil && ancient != "" && !db.readOnly { |
||||
freezer, err := rawdb.NewStateHistoryFreezer(ancient, false) |
||||
if err != nil { |
||||
log.Crit("Failed to open state history freezer", "err", err) |
||||
} |
||||
db.freezer = freezer |
||||
|
||||
// Truncate the extra state histories above in freezer in case
|
||||
// it's not aligned with the disk layer.
|
||||
pruned, err := truncateFromHead(db.diskdb, freezer, db.tree.bottom().stateID()) |
||||
if err != nil { |
||||
log.Crit("Failed to truncate extra state histories", "err", err) |
||||
} |
||||
if pruned != 0 { |
||||
log.Warn("Truncated extra state histories", "number", pruned) |
||||
} |
||||
} |
||||
log.Warn("Path-based state scheme is an experimental feature") |
||||
return db |
||||
} |
||||
|
||||
// Reader retrieves a layer belonging to the given state root.
|
||||
func (db *Database) Reader(root common.Hash) (layer, error) { |
||||
l := db.tree.get(root) |
||||
if l == nil { |
||||
return nil, fmt.Errorf("state %#x is not available", root) |
||||
} |
||||
return l, nil |
||||
} |
||||
|
||||
// Update adds a new layer into the tree, if that can be linked to an existing
|
||||
// old parent. It is disallowed to insert a disk layer (the origin of all). Apart
|
||||
// from that this function will flatten the extra diff layers at bottom into disk
|
||||
// to only keep 128 diff layers in memory by default.
|
||||
//
|
||||
// The passed in maps(nodes, states) will be retained to avoid copying everything.
|
||||
// Therefore, these maps must not be changed afterwards.
|
||||
func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint64, nodes *trienode.MergedNodeSet, states *triestate.Set) error { |
||||
// Hold the lock to prevent concurrent mutations.
|
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
// Short circuit if the database is in read only mode.
|
||||
if db.readOnly { |
||||
return errSnapshotReadOnly |
||||
} |
||||
if err := db.tree.add(root, parentRoot, block, nodes, states); err != nil { |
||||
return err |
||||
} |
||||
// Keep 128 diff layers in the memory, persistent layer is 129th.
|
||||
// - head layer is paired with HEAD state
|
||||
// - head-1 layer is paired with HEAD-1 state
|
||||
// - head-127 layer(bottom-most diff layer) is paired with HEAD-127 state
|
||||
// - head-128 layer(disk layer) is paired with HEAD-128 state
|
||||
return db.tree.cap(root, maxDiffLayers) |
||||
} |
||||
|
||||
// Commit traverses downwards the layer tree from a specified layer with the
|
||||
// provided state root and all the layers below are flattened downwards. It
|
||||
// can be used alone and mostly for test purposes.
|
||||
func (db *Database) Commit(root common.Hash, report bool) error { |
||||
// Hold the lock to prevent concurrent mutations.
|
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
// Short circuit if the database is in read only mode.
|
||||
if db.readOnly { |
||||
return errSnapshotReadOnly |
||||
} |
||||
return db.tree.cap(root, 0) |
||||
} |
||||
|
||||
// Reset rebuilds the database with the specified state as the base.
|
||||
//
|
||||
// - if target state is empty, clear the stored state and all layers on top
|
||||
// - if target state is non-empty, ensure the stored state matches with it
|
||||
// and clear all other layers on top.
|
||||
func (db *Database) Reset(root common.Hash) error { |
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
// Short circuit if the database is in read only mode.
|
||||
if db.readOnly { |
||||
return errSnapshotReadOnly |
||||
} |
||||
batch := db.diskdb.NewBatch() |
||||
root = types.TrieRootHash(root) |
||||
if root == types.EmptyRootHash { |
||||
// Empty state is requested as the target, nuke out
|
||||
// the root node and leave all others as dangling.
|
||||
rawdb.DeleteAccountTrieNode(batch, nil) |
||||
} else { |
||||
// Ensure the requested state is existent before any
|
||||
// action is applied.
|
||||
_, hash := rawdb.ReadAccountTrieNode(db.diskdb, nil) |
||||
if hash != root { |
||||
return fmt.Errorf("state is mismatched, local: %x, target: %x", hash, root) |
||||
} |
||||
} |
||||
// Mark the disk layer as stale before applying any mutation.
|
||||
db.tree.bottom().markStale() |
||||
|
||||
// Drop the stale state journal in persistent database and
|
||||
// reset the persistent state id back to zero.
|
||||
rawdb.DeleteTrieJournal(batch) |
||||
rawdb.WritePersistentStateID(batch, 0) |
||||
if err := batch.Write(); err != nil { |
||||
return err |
||||
} |
||||
// Clean up all state histories in freezer. Theoretically
|
||||
// all root->id mappings should be removed as well. Since
|
||||
// mappings can be huge and might take a while to clear
|
||||
// them, just leave them in disk and wait for overwriting.
|
||||
if db.freezer != nil { |
||||
if err := db.freezer.Reset(); err != nil { |
||||
return err |
||||
} |
||||
} |
||||
// Re-construct a new disk layer backed by persistent state
|
||||
// with **empty clean cache and node buffer**.
|
||||
dl := newDiskLayer(root, 0, db, nil, newNodeBuffer(db.bufferSize, nil, 0)) |
||||
db.tree.reset(dl) |
||||
log.Info("Rebuilt trie database", "root", root) |
||||
return nil |
||||
} |
||||
|
||||
// Recover rollbacks the database to a specified historical point.
|
||||
// The state is supported as the rollback destination only if it's
|
||||
// canonical state and the corresponding trie histories are existent.
|
||||
func (db *Database) Recover(root common.Hash, loader triestate.TrieLoader) error { |
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
// Short circuit if rollback operation is not supported.
|
||||
if db.readOnly || db.freezer == nil { |
||||
return errors.New("state rollback is non-supported") |
||||
} |
||||
// Short circuit if the target state is not recoverable.
|
||||
root = types.TrieRootHash(root) |
||||
if !db.Recoverable(root) { |
||||
return errStateUnrecoverable |
||||
} |
||||
// Apply the state histories upon the disk layer in order.
|
||||
var ( |
||||
start = time.Now() |
||||
dl = db.tree.bottom() |
||||
) |
||||
for dl.rootHash() != root { |
||||
h, err := readHistory(db.freezer, dl.stateID()) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
dl, err = dl.revert(h, loader) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
// reset layer with newly created disk layer. It must be
|
||||
// done after each revert operation, otherwise the new
|
||||
// disk layer won't be accessible from outside.
|
||||
db.tree.reset(dl) |
||||
} |
||||
rawdb.DeleteTrieJournal(db.diskdb) |
||||
_, err := truncateFromHead(db.diskdb, db.freezer, dl.stateID()) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
log.Debug("Recovered state", "root", root, "elapsed", common.PrettyDuration(time.Since(start))) |
||||
return nil |
||||
} |
||||
|
||||
// Recoverable returns the indicator if the specified state is recoverable.
|
||||
func (db *Database) Recoverable(root common.Hash) bool { |
||||
// Ensure the requested state is a known state.
|
||||
root = types.TrieRootHash(root) |
||||
id := rawdb.ReadStateID(db.diskdb, root) |
||||
if id == nil { |
||||
return false |
||||
} |
||||
// Recoverable state must below the disk layer. The recoverable
|
||||
// state only refers the state that is currently not available,
|
||||
// but can be restored by applying state history.
|
||||
dl := db.tree.bottom() |
||||
if *id >= dl.stateID() { |
||||
return false |
||||
} |
||||
// Ensure the requested state is a canonical state and all state
|
||||
// histories in range [id+1, disklayer.ID] are present and complete.
|
||||
parent := root |
||||
return checkHistories(db.freezer, *id+1, dl.stateID()-*id, func(m *meta) error { |
||||
if m.parent != parent { |
||||
return errors.New("unexpected state history") |
||||
} |
||||
if len(m.incomplete) > 0 { |
||||
return errors.New("incomplete state history") |
||||
} |
||||
parent = m.root |
||||
return nil |
||||
}) == nil |
||||
} |
||||
|
||||
// Close closes the trie database and the held freezer.
|
||||
func (db *Database) Close() error { |
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
db.readOnly = true |
||||
if db.freezer == nil { |
||||
return nil |
||||
} |
||||
return db.freezer.Close() |
||||
} |
||||
|
||||
// Size returns the current storage size of the memory cache in front of the
|
||||
// persistent database layer.
|
||||
func (db *Database) Size() (size common.StorageSize) { |
||||
db.tree.forEach(func(layer layer) { |
||||
if diff, ok := layer.(*diffLayer); ok { |
||||
size += common.StorageSize(diff.memory) |
||||
} |
||||
if disk, ok := layer.(*diskLayer); ok { |
||||
size += disk.size() |
||||
} |
||||
}) |
||||
return size |
||||
} |
||||
|
||||
// Initialized returns an indicator if the state data is already
|
||||
// initialized in path-based scheme.
|
||||
func (db *Database) Initialized(genesisRoot common.Hash) bool { |
||||
var inited bool |
||||
db.tree.forEach(func(layer layer) { |
||||
if layer.rootHash() != types.EmptyRootHash { |
||||
inited = true |
||||
} |
||||
}) |
||||
return inited |
||||
} |
||||
|
||||
// SetBufferSize sets the node buffer size to the provided value(in bytes).
|
||||
func (db *Database) SetBufferSize(size int) error { |
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
db.bufferSize = size |
||||
return db.tree.bottom().setBufferSize(db.bufferSize) |
||||
} |
||||
|
||||
// Scheme returns the node scheme used in the database.
|
||||
func (db *Database) Scheme() string { |
||||
return rawdb.PathScheme |
||||
} |
@ -0,0 +1,573 @@ |
||||
// Copyright 2022 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" |
||||
"errors" |
||||
"fmt" |
||||
"math/big" |
||||
"math/rand" |
||||
"testing" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/rlp" |
||||
"github.com/ethereum/go-ethereum/trie/testutil" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
func updateTrie(addrHash common.Hash, root common.Hash, dirties, cleans map[common.Hash][]byte) (common.Hash, *trienode.NodeSet) { |
||||
h, err := newTestHasher(addrHash, root, cleans) |
||||
if err != nil { |
||||
panic(fmt.Errorf("failed to create hasher, err: %w", err)) |
||||
} |
||||
for key, val := range dirties { |
||||
if len(val) == 0 { |
||||
h.Delete(key.Bytes()) |
||||
} else { |
||||
h.Update(key.Bytes(), val) |
||||
} |
||||
} |
||||
return h.Commit(false) |
||||
} |
||||
|
||||
func generateAccount(storageRoot common.Hash) types.StateAccount { |
||||
return types.StateAccount{ |
||||
Nonce: uint64(rand.Intn(100)), |
||||
Balance: big.NewInt(rand.Int63()), |
||||
CodeHash: testutil.RandBytes(32), |
||||
Root: storageRoot, |
||||
} |
||||
} |
||||
|
||||
const ( |
||||
createAccountOp int = iota |
||||
modifyAccountOp |
||||
deleteAccountOp |
||||
opLen |
||||
) |
||||
|
||||
type genctx 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 |
||||
nodes *trienode.MergedNodeSet |
||||
} |
||||
|
||||
func newCtx() *genctx { |
||||
return &genctx{ |
||||
accounts: make(map[common.Hash][]byte), |
||||
storages: make(map[common.Hash]map[common.Hash][]byte), |
||||
accountOrigin: make(map[common.Address][]byte), |
||||
storageOrigin: make(map[common.Address]map[common.Hash][]byte), |
||||
nodes: trienode.NewMergedNodeSet(), |
||||
} |
||||
} |
||||
|
||||
type tester struct { |
||||
db *Database |
||||
roots []common.Hash |
||||
preimages map[common.Hash]common.Address |
||||
accounts map[common.Hash][]byte |
||||
storages map[common.Hash]map[common.Hash][]byte |
||||
|
||||
// state snapshots
|
||||
snapAccounts map[common.Hash]map[common.Hash][]byte |
||||
snapStorages map[common.Hash]map[common.Hash]map[common.Hash][]byte |
||||
} |
||||
|
||||
func newTester(t *testing.T) *tester { |
||||
var ( |
||||
disk, _ = rawdb.NewDatabaseWithFreezer(rawdb.NewMemoryDatabase(), t.TempDir(), "", false) |
||||
db = New(disk, &Config{CleanSize: 256 * 1024, DirtySize: 256 * 1024}) |
||||
obj = &tester{ |
||||
db: db, |
||||
preimages: make(map[common.Hash]common.Address), |
||||
accounts: make(map[common.Hash][]byte), |
||||
storages: make(map[common.Hash]map[common.Hash][]byte), |
||||
snapAccounts: make(map[common.Hash]map[common.Hash][]byte), |
||||
snapStorages: make(map[common.Hash]map[common.Hash]map[common.Hash][]byte), |
||||
} |
||||
) |
||||
for i := 0; i < 2*128; i++ { |
||||
var parent = types.EmptyRootHash |
||||
if len(obj.roots) != 0 { |
||||
parent = obj.roots[len(obj.roots)-1] |
||||
} |
||||
root, nodes, states := obj.generate(parent) |
||||
if err := db.Update(root, parent, uint64(i), nodes, states); err != nil { |
||||
panic(fmt.Errorf("failed to update state changes, err: %w", err)) |
||||
} |
||||
obj.roots = append(obj.roots, root) |
||||
} |
||||
return obj |
||||
} |
||||
|
||||
func (t *tester) release() { |
||||
t.db.Close() |
||||
t.db.diskdb.Close() |
||||
} |
||||
|
||||
func (t *tester) randAccount() (common.Address, []byte) { |
||||
for addrHash, account := range t.accounts { |
||||
return t.preimages[addrHash], account |
||||
} |
||||
return common.Address{}, nil |
||||
} |
||||
|
||||
func (t *tester) generateStorage(ctx *genctx, addr common.Address) common.Hash { |
||||
var ( |
||||
addrHash = crypto.Keccak256Hash(addr.Bytes()) |
||||
storage = make(map[common.Hash][]byte) |
||||
origin = make(map[common.Hash][]byte) |
||||
) |
||||
for i := 0; i < 10; i++ { |
||||
v, _ := rlp.EncodeToBytes(common.TrimLeftZeroes(testutil.RandBytes(32))) |
||||
hash := testutil.RandomHash() |
||||
|
||||
storage[hash] = v |
||||
origin[hash] = nil |
||||
} |
||||
root, set := updateTrie(addrHash, types.EmptyRootHash, storage, nil) |
||||
|
||||
ctx.storages[addrHash] = storage |
||||
ctx.storageOrigin[addr] = origin |
||||
ctx.nodes.Merge(set) |
||||
return root |
||||
} |
||||
|
||||
func (t *tester) mutateStorage(ctx *genctx, addr common.Address, root common.Hash) common.Hash { |
||||
var ( |
||||
addrHash = crypto.Keccak256Hash(addr.Bytes()) |
||||
storage = make(map[common.Hash][]byte) |
||||
origin = make(map[common.Hash][]byte) |
||||
) |
||||
for hash, val := range t.storages[addrHash] { |
||||
origin[hash] = val |
||||
storage[hash] = nil |
||||
|
||||
if len(origin) == 3 { |
||||
break |
||||
} |
||||
} |
||||
for i := 0; i < 3; i++ { |
||||
v, _ := rlp.EncodeToBytes(common.TrimLeftZeroes(testutil.RandBytes(32))) |
||||
hash := testutil.RandomHash() |
||||
|
||||
storage[hash] = v |
||||
origin[hash] = nil |
||||
} |
||||
root, set := updateTrie(crypto.Keccak256Hash(addr.Bytes()), root, storage, t.storages[addrHash]) |
||||
|
||||
ctx.storages[addrHash] = storage |
||||
ctx.storageOrigin[addr] = origin |
||||
ctx.nodes.Merge(set) |
||||
return root |
||||
} |
||||
|
||||
func (t *tester) clearStorage(ctx *genctx, addr common.Address, root common.Hash) common.Hash { |
||||
var ( |
||||
addrHash = crypto.Keccak256Hash(addr.Bytes()) |
||||
storage = make(map[common.Hash][]byte) |
||||
origin = make(map[common.Hash][]byte) |
||||
) |
||||
for hash, val := range t.storages[addrHash] { |
||||
origin[hash] = val |
||||
storage[hash] = nil |
||||
} |
||||
root, set := updateTrie(addrHash, root, storage, t.storages[addrHash]) |
||||
if root != types.EmptyRootHash { |
||||
panic("failed to clear storage trie") |
||||
} |
||||
ctx.storages[addrHash] = storage |
||||
ctx.storageOrigin[addr] = origin |
||||
ctx.nodes.Merge(set) |
||||
return root |
||||
} |
||||
|
||||
func (t *tester) generate(parent common.Hash) (common.Hash, *trienode.MergedNodeSet, *triestate.Set) { |
||||
var ( |
||||
ctx = newCtx() |
||||
dirties = make(map[common.Hash]struct{}) |
||||
) |
||||
for i := 0; i < 20; i++ { |
||||
switch rand.Intn(opLen) { |
||||
case createAccountOp: |
||||
// account creation
|
||||
addr := testutil.RandomAddress() |
||||
addrHash := crypto.Keccak256Hash(addr.Bytes()) |
||||
if _, ok := t.accounts[addrHash]; ok { |
||||
continue |
||||
} |
||||
if _, ok := dirties[addrHash]; ok { |
||||
continue |
||||
} |
||||
dirties[addrHash] = struct{}{} |
||||
|
||||
root := t.generateStorage(ctx, addr) |
||||
ctx.accounts[addrHash] = types.SlimAccountRLP(generateAccount(root)) |
||||
ctx.accountOrigin[addr] = nil |
||||
t.preimages[addrHash] = addr |
||||
|
||||
case modifyAccountOp: |
||||
// account mutation
|
||||
addr, account := t.randAccount() |
||||
if addr == (common.Address{}) { |
||||
continue |
||||
} |
||||
addrHash := crypto.Keccak256Hash(addr.Bytes()) |
||||
if _, ok := dirties[addrHash]; ok { |
||||
continue |
||||
} |
||||
dirties[addrHash] = struct{}{} |
||||
|
||||
acct, _ := types.FullAccount(account) |
||||
stRoot := t.mutateStorage(ctx, addr, acct.Root) |
||||
newAccount := types.SlimAccountRLP(generateAccount(stRoot)) |
||||
|
||||
ctx.accounts[addrHash] = newAccount |
||||
ctx.accountOrigin[addr] = account |
||||
|
||||
case deleteAccountOp: |
||||
// account deletion
|
||||
addr, account := t.randAccount() |
||||
if addr == (common.Address{}) { |
||||
continue |
||||
} |
||||
addrHash := crypto.Keccak256Hash(addr.Bytes()) |
||||
if _, ok := dirties[addrHash]; ok { |
||||
continue |
||||
} |
||||
dirties[addrHash] = struct{}{} |
||||
|
||||
acct, _ := types.FullAccount(account) |
||||
if acct.Root != types.EmptyRootHash { |
||||
t.clearStorage(ctx, addr, acct.Root) |
||||
} |
||||
ctx.accounts[addrHash] = nil |
||||
ctx.accountOrigin[addr] = account |
||||
} |
||||
} |
||||
root, set := updateTrie(common.Hash{}, parent, ctx.accounts, t.accounts) |
||||
ctx.nodes.Merge(set) |
||||
|
||||
// Save state snapshot before commit
|
||||
t.snapAccounts[parent] = copyAccounts(t.accounts) |
||||
t.snapStorages[parent] = copyStorages(t.storages) |
||||
|
||||
// Commit all changes to live state set
|
||||
for addrHash, account := range ctx.accounts { |
||||
if len(account) == 0 { |
||||
delete(t.accounts, addrHash) |
||||
} else { |
||||
t.accounts[addrHash] = account |
||||
} |
||||
} |
||||
for addrHash, slots := range ctx.storages { |
||||
if _, ok := t.storages[addrHash]; !ok { |
||||
t.storages[addrHash] = make(map[common.Hash][]byte) |
||||
} |
||||
for sHash, slot := range slots { |
||||
if len(slot) == 0 { |
||||
delete(t.storages[addrHash], sHash) |
||||
} else { |
||||
t.storages[addrHash][sHash] = slot |
||||
} |
||||
} |
||||
} |
||||
return root, ctx.nodes, triestate.New(ctx.accountOrigin, ctx.storageOrigin, nil) |
||||
} |
||||
|
||||
// lastRoot returns the latest root hash, or empty if nothing is cached.
|
||||
func (t *tester) lastHash() common.Hash { |
||||
if len(t.roots) == 0 { |
||||
return common.Hash{} |
||||
} |
||||
return t.roots[len(t.roots)-1] |
||||
} |
||||
|
||||
func (t *tester) verifyState(root common.Hash) error { |
||||
reader, err := t.db.Reader(root) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
_, err = reader.Node(common.Hash{}, nil, root) |
||||
if err != nil { |
||||
return errors.New("root node is not available") |
||||
} |
||||
for addrHash, account := range t.snapAccounts[root] { |
||||
blob, err := reader.Node(common.Hash{}, addrHash.Bytes(), crypto.Keccak256Hash(account)) |
||||
if err != nil || !bytes.Equal(blob, account) { |
||||
return fmt.Errorf("account is mismatched: %w", err) |
||||
} |
||||
} |
||||
for addrHash, slots := range t.snapStorages[root] { |
||||
for hash, slot := range slots { |
||||
blob, err := reader.Node(addrHash, hash.Bytes(), crypto.Keccak256Hash(slot)) |
||||
if err != nil || !bytes.Equal(blob, slot) { |
||||
return fmt.Errorf("slot is mismatched: %w", err) |
||||
} |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
func (t *tester) verifyHistory() error { |
||||
bottom := t.bottomIndex() |
||||
for i, root := range t.roots { |
||||
// The state history related to the state above disk layer should not exist.
|
||||
if i > bottom { |
||||
_, err := readHistory(t.db.freezer, uint64(i+1)) |
||||
if err == nil { |
||||
return errors.New("unexpected state history") |
||||
} |
||||
continue |
||||
} |
||||
// The state history related to the state below or equal to the disk layer
|
||||
// should exist.
|
||||
obj, err := readHistory(t.db.freezer, uint64(i+1)) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
parent := types.EmptyRootHash |
||||
if i != 0 { |
||||
parent = t.roots[i-1] |
||||
} |
||||
if obj.meta.parent != parent { |
||||
return fmt.Errorf("unexpected parent, want: %x, got: %x", parent, obj.meta.parent) |
||||
} |
||||
if obj.meta.root != root { |
||||
return fmt.Errorf("unexpected root, want: %x, got: %x", root, obj.meta.root) |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// bottomIndex returns the index of current disk layer.
|
||||
func (t *tester) bottomIndex() int { |
||||
bottom := t.db.tree.bottom() |
||||
for i := 0; i < len(t.roots); i++ { |
||||
if t.roots[i] == bottom.rootHash() { |
||||
return i |
||||
} |
||||
} |
||||
return -1 |
||||
} |
||||
|
||||
func TestDatabaseRollback(t *testing.T) { |
||||
// Verify state histories
|
||||
tester := newTester(t) |
||||
defer tester.release() |
||||
|
||||
if err := tester.verifyHistory(); err != nil { |
||||
t.Fatalf("Invalid state history, err: %v", err) |
||||
} |
||||
// Revert database from top to bottom
|
||||
for i := tester.bottomIndex(); i >= 0; i-- { |
||||
root := tester.roots[i] |
||||
parent := types.EmptyRootHash |
||||
if i > 0 { |
||||
parent = tester.roots[i-1] |
||||
} |
||||
loader := newHashLoader(tester.snapAccounts[root], tester.snapStorages[root]) |
||||
if err := tester.db.Recover(parent, loader); err != nil { |
||||
t.Fatalf("Failed to revert db, err: %v", err) |
||||
} |
||||
tester.verifyState(parent) |
||||
} |
||||
if tester.db.tree.len() != 1 { |
||||
t.Fatal("Only disk layer is expected") |
||||
} |
||||
} |
||||
|
||||
func TestDatabaseRecoverable(t *testing.T) { |
||||
var ( |
||||
tester = newTester(t) |
||||
index = tester.bottomIndex() |
||||
) |
||||
defer tester.release() |
||||
|
||||
var cases = []struct { |
||||
root common.Hash |
||||
expect bool |
||||
}{ |
||||
// Unknown state should be unrecoverable
|
||||
{common.Hash{0x1}, false}, |
||||
|
||||
// Initial state should be recoverable
|
||||
{types.EmptyRootHash, true}, |
||||
|
||||
// Initial state should be recoverable
|
||||
{common.Hash{}, true}, |
||||
|
||||
// Layers below current disk layer are recoverable
|
||||
{tester.roots[index-1], true}, |
||||
|
||||
// Disklayer itself is not recoverable, since it's
|
||||
// available for accessing.
|
||||
{tester.roots[index], false}, |
||||
|
||||
// Layers above current disk layer are not recoverable
|
||||
// since they are available for accessing.
|
||||
{tester.roots[index+1], false}, |
||||
} |
||||
for i, c := range cases { |
||||
result := tester.db.Recoverable(c.root) |
||||
if result != c.expect { |
||||
t.Fatalf("case: %d, unexpected result, want %t, got %t", i, c.expect, result) |
||||
} |
||||
} |
||||
} |
||||
|
||||
func TestReset(t *testing.T) { |
||||
var ( |
||||
tester = newTester(t) |
||||
index = tester.bottomIndex() |
||||
) |
||||
defer tester.release() |
||||
|
||||
// Reset database to unknown target, should reject it
|
||||
if err := tester.db.Reset(testutil.RandomHash()); err == nil { |
||||
t.Fatal("Failed to reject invalid reset") |
||||
} |
||||
// Reset database to state persisted in the disk
|
||||
if err := tester.db.Reset(types.EmptyRootHash); err != nil { |
||||
t.Fatalf("Failed to reset database %v", err) |
||||
} |
||||
// Ensure journal is deleted from disk
|
||||
if blob := rawdb.ReadTrieJournal(tester.db.diskdb); len(blob) != 0 { |
||||
t.Fatal("Failed to clean journal") |
||||
} |
||||
// Ensure all trie histories are removed
|
||||
for i := 0; i <= index; i++ { |
||||
_, err := readHistory(tester.db.freezer, uint64(i+1)) |
||||
if err == nil { |
||||
t.Fatalf("Failed to clean state history, index %d", i+1) |
||||
} |
||||
} |
||||
// Verify layer tree structure, single disk layer is expected
|
||||
if tester.db.tree.len() != 1 { |
||||
t.Fatalf("Extra layer kept %d", tester.db.tree.len()) |
||||
} |
||||
if tester.db.tree.bottom().rootHash() != types.EmptyRootHash { |
||||
t.Fatalf("Root hash is not matched exp %x got %x", types.EmptyRootHash, tester.db.tree.bottom().rootHash()) |
||||
} |
||||
} |
||||
|
||||
func TestCommit(t *testing.T) { |
||||
tester := newTester(t) |
||||
defer tester.release() |
||||
|
||||
if err := tester.db.Commit(tester.lastHash(), false); err != nil { |
||||
t.Fatalf("Failed to cap database, err: %v", err) |
||||
} |
||||
// Verify layer tree structure, single disk layer is expected
|
||||
if tester.db.tree.len() != 1 { |
||||
t.Fatal("Layer tree structure is invalid") |
||||
} |
||||
if tester.db.tree.bottom().rootHash() != tester.lastHash() { |
||||
t.Fatal("Layer tree structure is invalid") |
||||
} |
||||
// Verify states
|
||||
if err := tester.verifyState(tester.lastHash()); err != nil { |
||||
t.Fatalf("State is invalid, err: %v", err) |
||||
} |
||||
// Verify state histories
|
||||
if err := tester.verifyHistory(); err != nil { |
||||
t.Fatalf("State history is invalid, err: %v", err) |
||||
} |
||||
} |
||||
|
||||
func TestJournal(t *testing.T) { |
||||
tester := newTester(t) |
||||
defer tester.release() |
||||
|
||||
if err := tester.db.Journal(tester.lastHash()); err != nil { |
||||
t.Errorf("Failed to journal, err: %v", err) |
||||
} |
||||
tester.db.Close() |
||||
tester.db = New(tester.db.diskdb, nil) |
||||
|
||||
// Verify states including disk layer and all diff on top.
|
||||
for i := 0; i < len(tester.roots); i++ { |
||||
if i >= tester.bottomIndex() { |
||||
if err := tester.verifyState(tester.roots[i]); err != nil { |
||||
t.Fatalf("Invalid state, err: %v", err) |
||||
} |
||||
continue |
||||
} |
||||
if err := tester.verifyState(tester.roots[i]); err == nil { |
||||
t.Fatal("Unexpected state") |
||||
} |
||||
} |
||||
} |
||||
|
||||
func TestCorruptedJournal(t *testing.T) { |
||||
tester := newTester(t) |
||||
defer tester.release() |
||||
|
||||
if err := tester.db.Journal(tester.lastHash()); err != nil { |
||||
t.Errorf("Failed to journal, err: %v", err) |
||||
} |
||||
tester.db.Close() |
||||
_, root := rawdb.ReadAccountTrieNode(tester.db.diskdb, nil) |
||||
|
||||
// Mutate the journal in disk, it should be regarded as invalid
|
||||
blob := rawdb.ReadTrieJournal(tester.db.diskdb) |
||||
blob[0] = 1 |
||||
rawdb.WriteTrieJournal(tester.db.diskdb, blob) |
||||
|
||||
// Verify states, all not-yet-written states should be discarded
|
||||
tester.db = New(tester.db.diskdb, nil) |
||||
for i := 0; i < len(tester.roots); i++ { |
||||
if tester.roots[i] == root { |
||||
if err := tester.verifyState(root); err != nil { |
||||
t.Fatalf("Disk state is corrupted, err: %v", err) |
||||
} |
||||
continue |
||||
} |
||||
if err := tester.verifyState(tester.roots[i]); err == nil { |
||||
t.Fatal("Unexpected state") |
||||
} |
||||
} |
||||
} |
||||
|
||||
// copyAccounts returns a deep-copied account set of the provided one.
|
||||
func copyAccounts(set map[common.Hash][]byte) map[common.Hash][]byte { |
||||
copied := make(map[common.Hash][]byte, len(set)) |
||||
for key, val := range set { |
||||
copied[key] = common.CopyBytes(val) |
||||
} |
||||
return copied |
||||
} |
||||
|
||||
// copyStorages returns a deep-copied storage set of the provided one.
|
||||
func copyStorages(set map[common.Hash]map[common.Hash][]byte) map[common.Hash]map[common.Hash][]byte { |
||||
copied := make(map[common.Hash]map[common.Hash][]byte, len(set)) |
||||
for addrHash, subset := range set { |
||||
copied[addrHash] = make(map[common.Hash][]byte, len(subset)) |
||||
for key, val := range subset { |
||||
copied[addrHash][key] = common.CopyBytes(val) |
||||
} |
||||
} |
||||
return copied |
||||
} |
@ -0,0 +1,174 @@ |
||||
// Copyright 2022 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 ( |
||||
"fmt" |
||||
"sync" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
// diffLayer represents a collection of modifications made to the in-memory tries
|
||||
// along with associated state changes after running a block on top.
|
||||
//
|
||||
// The goal of a diff layer is to act as a journal, tracking recent modifications
|
||||
// made to the state, that have not yet graduated into a semi-immutable state.
|
||||
type diffLayer struct { |
||||
// Immutables
|
||||
root common.Hash // Root hash to which this layer diff belongs to
|
||||
id uint64 // Corresponding state id
|
||||
block uint64 // Associated block number
|
||||
nodes map[common.Hash]map[string]*trienode.Node // Cached trie nodes indexed by owner and path
|
||||
states *triestate.Set // Associated state change set for building history
|
||||
memory uint64 // Approximate guess as to how much memory we use
|
||||
|
||||
parent layer // Parent layer modified by this one, never nil, **can be changed**
|
||||
lock sync.RWMutex // Lock used to protect parent
|
||||
} |
||||
|
||||
// newDiffLayer creates a new diff layer on top of an existing layer.
|
||||
func newDiffLayer(parent layer, root common.Hash, id uint64, block uint64, nodes map[common.Hash]map[string]*trienode.Node, states *triestate.Set) *diffLayer { |
||||
var ( |
||||
size int64 |
||||
count int |
||||
) |
||||
dl := &diffLayer{ |
||||
root: root, |
||||
id: id, |
||||
block: block, |
||||
nodes: nodes, |
||||
states: states, |
||||
parent: parent, |
||||
} |
||||
for _, subset := range nodes { |
||||
for path, n := range subset { |
||||
dl.memory += uint64(n.Size() + len(path)) |
||||
size += int64(len(n.Blob) + len(path)) |
||||
} |
||||
count += len(subset) |
||||
} |
||||
if states != nil { |
||||
dl.memory += uint64(states.Size()) |
||||
} |
||||
dirtyWriteMeter.Mark(size) |
||||
diffLayerNodesMeter.Mark(int64(count)) |
||||
diffLayerBytesMeter.Mark(int64(dl.memory)) |
||||
log.Debug("Created new diff layer", "id", id, "block", block, "nodes", count, "size", common.StorageSize(dl.memory)) |
||||
return dl |
||||
} |
||||
|
||||
// rootHash implements the layer interface, returning the root hash of
|
||||
// corresponding state.
|
||||
func (dl *diffLayer) rootHash() common.Hash { |
||||
return dl.root |
||||
} |
||||
|
||||
// stateID implements the layer interface, returning the state id of the layer.
|
||||
func (dl *diffLayer) stateID() uint64 { |
||||
return dl.id |
||||
} |
||||
|
||||
// parentLayer implements the layer interface, returning the subsequent
|
||||
// layer of the diff layer.
|
||||
func (dl *diffLayer) parentLayer() layer { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
return dl.parent |
||||
} |
||||
|
||||
// node retrieves the node with provided node information. It's the internal
|
||||
// version of Node function with additional accessed layer tracked. No error
|
||||
// will be returned if node is not found.
|
||||
func (dl *diffLayer) node(owner common.Hash, path []byte, 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 the trie node is known locally, return it
|
||||
subset, ok := dl.nodes[owner] |
||||
if ok { |
||||
n, ok := subset[string(path)] |
||||
if ok { |
||||
// If the trie node is not hash matched, or marked as removed,
|
||||
// bubble up an error here. It shouldn't happen at all.
|
||||
if n.Hash != hash { |
||||
dirtyFalseMeter.Mark(1) |
||||
log.Error("Unexpected trie node in diff layer", "owner", owner, "path", path, "expect", hash, "got", n.Hash) |
||||
return nil, newUnexpectedNodeError("diff", hash, n.Hash, owner, path) |
||||
} |
||||
dirtyHitMeter.Mark(1) |
||||
dirtyNodeHitDepthHist.Update(int64(depth)) |
||||
dirtyReadMeter.Mark(int64(len(n.Blob))) |
||||
return n.Blob, nil |
||||
} |
||||
} |
||||
// Trie node unknown to this layer, resolve from parent
|
||||
if diff, ok := dl.parent.(*diffLayer); ok { |
||||
return diff.node(owner, path, hash, depth+1) |
||||
} |
||||
// Failed to resolve through diff layers, fallback to disk layer
|
||||
return dl.parent.Node(owner, path, hash) |
||||
} |
||||
|
||||
// Node implements the layer interface, retrieving the trie node blob with the
|
||||
// provided node information. No error will be returned if the node is not found.
|
||||
func (dl *diffLayer) Node(owner common.Hash, path []byte, hash common.Hash) ([]byte, error) { |
||||
return dl.node(owner, path, hash, 0) |
||||
} |
||||
|
||||
// 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 map[common.Hash]map[string]*trienode.Node, states *triestate.Set) *diffLayer { |
||||
return newDiffLayer(dl, root, id, block, nodes, states) |
||||
} |
||||
|
||||
// persist flushes the diff layer and all its parent layers to disk layer.
|
||||
func (dl *diffLayer) persist(force bool) (layer, error) { |
||||
if parent, ok := dl.parentLayer().(*diffLayer); ok { |
||||
// Hold the lock to prevent any read operation until the new
|
||||
// parent is linked correctly.
|
||||
dl.lock.Lock() |
||||
|
||||
// The merging of diff layers starts at the bottom-most layer,
|
||||
// therefore we recurse down here, flattening on the way up
|
||||
// (diffToDisk).
|
||||
result, err := parent.persist(force) |
||||
if err != nil { |
||||
dl.lock.Unlock() |
||||
return nil, err |
||||
} |
||||
dl.parent = result |
||||
dl.lock.Unlock() |
||||
} |
||||
return diffToDisk(dl, force) |
||||
} |
||||
|
||||
// diffToDisk merges a bottom-most diff into the persistent disk layer underneath
|
||||
// it. The method will panic if called onto a non-bottom-most diff layer.
|
||||
func diffToDisk(layer *diffLayer, force bool) (layer, error) { |
||||
disk, ok := layer.parentLayer().(*diskLayer) |
||||
if !ok { |
||||
panic(fmt.Sprintf("unknown layer type: %T", layer.parentLayer())) |
||||
} |
||||
return disk.commit(layer, force) |
||||
} |
@ -0,0 +1,170 @@ |
||||
// Copyright 2019 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" |
||||
"testing" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/trie/testutil" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
) |
||||
|
||||
func emptyLayer() *diskLayer { |
||||
return &diskLayer{ |
||||
db: New(rawdb.NewMemoryDatabase(), nil), |
||||
buffer: newNodeBuffer(defaultBufferSize, nil, 0), |
||||
} |
||||
} |
||||
|
||||
// goos: darwin
|
||||
// goarch: arm64
|
||||
// pkg: github.com/ethereum/go-ethereum/trie
|
||||
// BenchmarkSearch128Layers
|
||||
// BenchmarkSearch128Layers-8 243826 4755 ns/op
|
||||
func BenchmarkSearch128Layers(b *testing.B) { benchmarkSearch(b, 0, 128) } |
||||
|
||||
// goos: darwin
|
||||
// goarch: arm64
|
||||
// pkg: github.com/ethereum/go-ethereum/trie
|
||||
// BenchmarkSearch512Layers
|
||||
// BenchmarkSearch512Layers-8 49686 24256 ns/op
|
||||
func BenchmarkSearch512Layers(b *testing.B) { benchmarkSearch(b, 0, 512) } |
||||
|
||||
// goos: darwin
|
||||
// goarch: arm64
|
||||
// pkg: github.com/ethereum/go-ethereum/trie
|
||||
// BenchmarkSearch1Layer
|
||||
// BenchmarkSearch1Layer-8 14062725 88.40 ns/op
|
||||
func BenchmarkSearch1Layer(b *testing.B) { benchmarkSearch(b, 127, 128) } |
||||
|
||||
func benchmarkSearch(b *testing.B, depth int, total int) { |
||||
var ( |
||||
npath []byte |
||||
nhash common.Hash |
||||
nblob []byte |
||||
) |
||||
// First, we set up 128 diff layers, with 3K items each
|
||||
fill := func(parent layer, index int) *diffLayer { |
||||
nodes := make(map[common.Hash]map[string]*trienode.Node) |
||||
nodes[common.Hash{}] = make(map[string]*trienode.Node) |
||||
for i := 0; i < 3000; i++ { |
||||
var ( |
||||
path = testutil.RandBytes(32) |
||||
node = testutil.RandomNode() |
||||
) |
||||
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob) |
||||
if npath == nil && depth == index { |
||||
npath = common.CopyBytes(path) |
||||
nblob = common.CopyBytes(node.Blob) |
||||
nhash = node.Hash |
||||
} |
||||
} |
||||
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil) |
||||
} |
||||
var layer layer |
||||
layer = emptyLayer() |
||||
for i := 0; i < total; i++ { |
||||
layer = fill(layer, i) |
||||
} |
||||
b.ResetTimer() |
||||
|
||||
var ( |
||||
have []byte |
||||
err error |
||||
) |
||||
for i := 0; i < b.N; i++ { |
||||
have, err = layer.Node(common.Hash{}, npath, nhash) |
||||
if err != nil { |
||||
b.Fatal(err) |
||||
} |
||||
} |
||||
if !bytes.Equal(have, nblob) { |
||||
b.Fatalf("have %x want %x", have, nblob) |
||||
} |
||||
} |
||||
|
||||
// goos: darwin
|
||||
// goarch: arm64
|
||||
// pkg: github.com/ethereum/go-ethereum/trie
|
||||
// BenchmarkPersist
|
||||
// BenchmarkPersist-8 10 111252975 ns/op
|
||||
func BenchmarkPersist(b *testing.B) { |
||||
// First, we set up 128 diff layers, with 3K items each
|
||||
fill := func(parent layer) *diffLayer { |
||||
nodes := make(map[common.Hash]map[string]*trienode.Node) |
||||
nodes[common.Hash{}] = make(map[string]*trienode.Node) |
||||
for i := 0; i < 3000; i++ { |
||||
var ( |
||||
path = testutil.RandBytes(32) |
||||
node = testutil.RandomNode() |
||||
) |
||||
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob) |
||||
} |
||||
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil) |
||||
} |
||||
for i := 0; i < b.N; i++ { |
||||
b.StopTimer() |
||||
var layer layer |
||||
layer = emptyLayer() |
||||
for i := 1; i < 128; i++ { |
||||
layer = fill(layer) |
||||
} |
||||
b.StartTimer() |
||||
|
||||
dl, ok := layer.(*diffLayer) |
||||
if !ok { |
||||
break |
||||
} |
||||
dl.persist(false) |
||||
} |
||||
} |
||||
|
||||
// BenchmarkJournal benchmarks the performance for journaling the layers.
|
||||
//
|
||||
// BenchmarkJournal
|
||||
// BenchmarkJournal-8 10 110969279 ns/op
|
||||
func BenchmarkJournal(b *testing.B) { |
||||
b.SkipNow() |
||||
|
||||
// First, we set up 128 diff layers, with 3K items each
|
||||
fill := func(parent layer) *diffLayer { |
||||
nodes := make(map[common.Hash]map[string]*trienode.Node) |
||||
nodes[common.Hash{}] = make(map[string]*trienode.Node) |
||||
for i := 0; i < 3000; i++ { |
||||
var ( |
||||
path = testutil.RandBytes(32) |
||||
node = testutil.RandomNode() |
||||
) |
||||
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob) |
||||
} |
||||
// TODO(rjl493456442) a non-nil state set is expected.
|
||||
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil) |
||||
} |
||||
var layer layer |
||||
layer = emptyLayer() |
||||
for i := 0; i < 128; i++ { |
||||
layer = fill(layer) |
||||
} |
||||
b.ResetTimer() |
||||
|
||||
for i := 0; i < b.N; i++ { |
||||
layer.journal(new(bytes.Buffer)) |
||||
} |
||||
} |
@ -0,0 +1,296 @@ |
||||
// Copyright 2022 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 ( |
||||
"errors" |
||||
"fmt" |
||||
"sync" |
||||
|
||||
"github.com/VictoriaMetrics/fastcache" |
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
"golang.org/x/crypto/sha3" |
||||
) |
||||
|
||||
// diskLayer is a low level persistent layer built on top of a key-value store.
|
||||
type diskLayer struct { |
||||
root common.Hash // Immutable, root hash to which this layer was made for
|
||||
id uint64 // Immutable, corresponding state id
|
||||
db *Database // Path-based trie database
|
||||
cleans *fastcache.Cache // GC friendly memory cache of clean node RLPs
|
||||
buffer *nodebuffer // Node buffer to aggregate writes
|
||||
stale bool // Signals that the layer became stale (state progressed)
|
||||
lock sync.RWMutex // Lock used to protect stale flag
|
||||
} |
||||
|
||||
// newDiskLayer creates a new disk layer based on the passing arguments.
|
||||
func newDiskLayer(root common.Hash, id uint64, db *Database, cleans *fastcache.Cache, buffer *nodebuffer) *diskLayer { |
||||
// Initialize a clean cache if the memory allowance is not zero
|
||||
// or reuse the provided cache if it is not nil (inherited from
|
||||
// the original disk layer).
|
||||
if cleans == nil && db.config.CleanSize != 0 { |
||||
cleans = fastcache.New(db.config.CleanSize) |
||||
} |
||||
return &diskLayer{ |
||||
root: root, |
||||
id: id, |
||||
db: db, |
||||
cleans: cleans, |
||||
buffer: buffer, |
||||
} |
||||
} |
||||
|
||||
// root implements the layer interface, returning root hash of corresponding state.
|
||||
func (dl *diskLayer) rootHash() common.Hash { |
||||
return dl.root |
||||
} |
||||
|
||||
// stateID implements the layer interface, returning the state id of disk layer.
|
||||
func (dl *diskLayer) stateID() uint64 { |
||||
return dl.id |
||||
} |
||||
|
||||
// parent implements the layer interface, returning nil as there's no layer
|
||||
// below the disk.
|
||||
func (dl *diskLayer) parentLayer() layer { |
||||
return nil |
||||
} |
||||
|
||||
// isStale return whether this layer has become stale (was flattened across) or if
|
||||
// it's still live.
|
||||
func (dl *diskLayer) isStale() bool { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
return dl.stale |
||||
} |
||||
|
||||
// markStale sets the stale flag as true.
|
||||
func (dl *diskLayer) markStale() { |
||||
dl.lock.Lock() |
||||
defer dl.lock.Unlock() |
||||
|
||||
if dl.stale { |
||||
panic("triedb disk layer is stale") // we've committed into the same base from two children, boom
|
||||
} |
||||
dl.stale = true |
||||
} |
||||
|
||||
// Node implements the layer interface, retrieving the trie node with the
|
||||
// provided node info. No error will be returned if the node is not found.
|
||||
func (dl *diskLayer) Node(owner common.Hash, path []byte, hash common.Hash) ([]byte, error) { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
if dl.stale { |
||||
return nil, errSnapshotStale |
||||
} |
||||
// Try to retrieve the trie node 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.
|
||||
n, err := dl.buffer.node(owner, path, hash) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
if n != nil { |
||||
dirtyHitMeter.Mark(1) |
||||
dirtyReadMeter.Mark(int64(len(n.Blob))) |
||||
return n.Blob, nil |
||||
} |
||||
dirtyMissMeter.Mark(1) |
||||
|
||||
// Try to retrieve the trie node from the clean memory cache
|
||||
key := cacheKey(owner, path) |
||||
if dl.cleans != nil { |
||||
if blob := dl.cleans.Get(nil, key); len(blob) > 0 { |
||||
h := newHasher() |
||||
defer h.release() |
||||
|
||||
got := h.hash(blob) |
||||
if got == hash { |
||||
cleanHitMeter.Mark(1) |
||||
cleanReadMeter.Mark(int64(len(blob))) |
||||
return blob, nil |
||||
} |
||||
cleanFalseMeter.Mark(1) |
||||
log.Error("Unexpected trie node in clean cache", "owner", owner, "path", path, "expect", hash, "got", got) |
||||
} |
||||
cleanMissMeter.Mark(1) |
||||
} |
||||
// Try to retrieve the trie node from the disk.
|
||||
var ( |
||||
nBlob []byte |
||||
nHash common.Hash |
||||
) |
||||
if owner == (common.Hash{}) { |
||||
nBlob, nHash = rawdb.ReadAccountTrieNode(dl.db.diskdb, path) |
||||
} else { |
||||
nBlob, nHash = rawdb.ReadStorageTrieNode(dl.db.diskdb, owner, path) |
||||
} |
||||
if nHash != hash { |
||||
diskFalseMeter.Mark(1) |
||||
log.Error("Unexpected trie node in disk", "owner", owner, "path", path, "expect", hash, "got", nHash) |
||||
return nil, newUnexpectedNodeError("disk", hash, nHash, owner, path) |
||||
} |
||||
if dl.cleans != nil && len(nBlob) > 0 { |
||||
dl.cleans.Set(key, nBlob) |
||||
cleanWriteMeter.Mark(int64(len(nBlob))) |
||||
} |
||||
return nBlob, nil |
||||
} |
||||
|
||||
// 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 map[common.Hash]map[string]*trienode.Node, states *triestate.Set) *diffLayer { |
||||
return newDiffLayer(dl, root, id, block, nodes, states) |
||||
} |
||||
|
||||
// commit merges the given bottom-most diff layer into the node buffer
|
||||
// and returns a newly constructed disk layer. Note the current disk
|
||||
// layer must be tagged as stale first to prevent re-access.
|
||||
func (dl *diskLayer) commit(bottom *diffLayer, force bool) (*diskLayer, error) { |
||||
dl.lock.Lock() |
||||
defer dl.lock.Unlock() |
||||
|
||||
// Construct and store the state history first. If crash happens
|
||||
// after storing the state history but without flushing the
|
||||
// corresponding states(journal), the stored state history will
|
||||
// be truncated in the next restart.
|
||||
if dl.db.freezer != nil { |
||||
err := writeHistory(dl.db.diskdb, dl.db.freezer, bottom, dl.db.config.StateLimit) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
} |
||||
// Mark the diskLayer as stale before applying any mutations on top.
|
||||
dl.stale = true |
||||
|
||||
// Store the root->id lookup afterwards. All stored lookups are
|
||||
// identified by the **unique** state root. It's impossible that
|
||||
// in the same chain blocks are not adjacent but have the same
|
||||
// root.
|
||||
if dl.id == 0 { |
||||
rawdb.WriteStateID(dl.db.diskdb, dl.root, 0) |
||||
} |
||||
rawdb.WriteStateID(dl.db.diskdb, bottom.rootHash(), bottom.stateID()) |
||||
|
||||
// Construct a new disk layer by merging the nodes from the provided
|
||||
// diff layer, and flush the content in disk layer if there are too
|
||||
// many nodes cached. The clean cache is inherited from the original
|
||||
// disk layer for reusing.
|
||||
ndl := newDiskLayer(bottom.root, bottom.stateID(), dl.db, dl.cleans, dl.buffer.commit(bottom.nodes)) |
||||
err := ndl.buffer.flush(ndl.db.diskdb, ndl.cleans, ndl.id, force) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
return ndl, nil |
||||
} |
||||
|
||||
// revert applies the given state history and return a reverted disk layer.
|
||||
func (dl *diskLayer) revert(h *history, loader triestate.TrieLoader) (*diskLayer, error) { |
||||
if h.meta.root != dl.rootHash() { |
||||
return nil, errUnexpectedHistory |
||||
} |
||||
// Reject if the provided state history is incomplete. It's due to
|
||||
// a large construct SELF-DESTRUCT which can't be handled because
|
||||
// of memory limitation.
|
||||
if len(h.meta.incomplete) > 0 { |
||||
return nil, errors.New("incomplete state history") |
||||
} |
||||
if dl.id == 0 { |
||||
return nil, fmt.Errorf("%w: zero state id", errStateUnrecoverable) |
||||
} |
||||
// 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.
|
||||
nodes, err := triestate.Apply(h.meta.parent, h.meta.root, h.accounts, h.storages, loader) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
// Mark the diskLayer as stale before applying any mutations on top.
|
||||
dl.lock.Lock() |
||||
defer dl.lock.Unlock() |
||||
|
||||
dl.stale = true |
||||
|
||||
// State change may be applied to node buffer, or the persistent
|
||||
// state, depends on if node buffer is empty or not. If the node
|
||||
// buffer is not empty, it means that the state transition that
|
||||
// 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) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
} else { |
||||
batch := dl.db.diskdb.NewBatch() |
||||
writeNodes(batch, nodes, dl.cleans) |
||||
rawdb.WritePersistentStateID(batch, dl.id-1) |
||||
if err := batch.Write(); err != nil { |
||||
log.Crit("Failed to write states", "err", err) |
||||
} |
||||
} |
||||
return newDiskLayer(h.meta.parent, dl.id-1, dl.db, dl.cleans, dl.buffer), nil |
||||
} |
||||
|
||||
// setBufferSize sets the node buffer size to the provided value.
|
||||
func (dl *diskLayer) setBufferSize(size int) error { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
if dl.stale { |
||||
return errSnapshotStale |
||||
} |
||||
return dl.buffer.setSize(size, dl.db.diskdb, dl.cleans, dl.id) |
||||
} |
||||
|
||||
// size returns the approximate size of cached nodes in the disk layer.
|
||||
func (dl *diskLayer) size() common.StorageSize { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
if dl.stale { |
||||
return 0 |
||||
} |
||||
return common.StorageSize(dl.buffer.size) |
||||
} |
||||
|
||||
// hasher is used to compute the sha256 hash of the provided data.
|
||||
type hasher struct{ sha crypto.KeccakState } |
||||
|
||||
var hasherPool = sync.Pool{ |
||||
New: func() interface{} { return &hasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} }, |
||||
} |
||||
|
||||
func newHasher() *hasher { |
||||
return hasherPool.Get().(*hasher) |
||||
} |
||||
|
||||
func (h *hasher) hash(data []byte) common.Hash { |
||||
return crypto.HashData(h.sha, data) |
||||
} |
||||
|
||||
func (h *hasher) release() { |
||||
hasherPool.Put(h) |
||||
} |
@ -0,0 +1,51 @@ |
||||
// Copyright 2023 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 ( |
||||
"errors" |
||||
"fmt" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
) |
||||
|
||||
var ( |
||||
// errSnapshotReadOnly is returned if the database is opened in read only mode
|
||||
// and mutation is requested.
|
||||
errSnapshotReadOnly = errors.New("read only") |
||||
|
||||
// errSnapshotStale is returned from data accessors if the underlying layer
|
||||
// layer had been invalidated due to the chain progressing forward far enough
|
||||
// to not maintain the layer's original state.
|
||||
errSnapshotStale = errors.New("layer stale") |
||||
|
||||
// errUnexpectedHistory is returned if an unmatched state history is applied
|
||||
// to the database for state rollback.
|
||||
errUnexpectedHistory = errors.New("unexpected state history") |
||||
|
||||
// errStateUnrecoverable is returned if state is required to be reverted to
|
||||
// a destination without associated state history available.
|
||||
errStateUnrecoverable = errors.New("state is unrecoverable") |
||||
|
||||
// errUnexpectedNode is returned if the requested node with specified path is
|
||||
// not hash matched with expectation.
|
||||
errUnexpectedNode = errors.New("unexpected node") |
||||
) |
||||
|
||||
func newUnexpectedNodeError(loc string, expHash common.Hash, gotHash common.Hash, owner common.Hash, path []byte) error { |
||||
return fmt.Errorf("%w, loc: %s, node: (%x %v), %x!=%x", errUnexpectedNode, loc, owner, path, expHash, gotHash) |
||||
} |
@ -0,0 +1,641 @@ |
||||
// Copyright 2022 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" |
||||
"encoding/binary" |
||||
"errors" |
||||
"fmt" |
||||
"time" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/ethdb" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
"golang.org/x/exp/slices" |
||||
) |
||||
|
||||
// State history records the state changes involved in executing a block. The
|
||||
// state can be reverted to the previous version by applying the associated
|
||||
// history object (state reverse diff). State history objects are kept to
|
||||
// guarantee that the system can perform state rollbacks in case of deep reorg.
|
||||
//
|
||||
// Each state transition will generate a state history object. Note that not
|
||||
// every block has a corresponding state history object. If a block performs
|
||||
// no state changes whatsoever, no state is created for it. Each state history
|
||||
// will have a sequentially increasing number acting as its unique identifier.
|
||||
//
|
||||
// The state history is written to disk (ancient store) when the corresponding
|
||||
// diff layer is merged into the disk layer. At the same time, system can prune
|
||||
// the oldest histories according to config.
|
||||
//
|
||||
// Disk State
|
||||
// ^
|
||||
// |
|
||||
// +------------+ +---------+ +---------+ +---------+
|
||||
// | Init State |---->| State 1 |---->| ... |---->| State n |
|
||||
// +------------+ +---------+ +---------+ +---------+
|
||||
//
|
||||
// +-----------+ +------+ +-----------+
|
||||
// | History 1 |----> | ... |---->| History n |
|
||||
// +-----------+ +------+ +-----------+
|
||||
//
|
||||
// # Rollback
|
||||
//
|
||||
// If the system wants to roll back to a previous state n, it needs to ensure
|
||||
// all history objects from n+1 up to the current disk layer are existent. The
|
||||
// history objects are applied to the state in reverse order, starting from the
|
||||
// current disk layer.
|
||||
|
||||
const ( |
||||
accountIndexSize = common.AddressLength + 13 // The length of encoded account index
|
||||
slotIndexSize = common.HashLength + 5 // The length of encoded slot index
|
||||
historyMetaSize = 9 + 2*common.HashLength // The length of fixed size part of meta object
|
||||
|
||||
stateHistoryVersion = uint8(0) // initial version of state history structure.
|
||||
) |
||||
|
||||
// Each state history entry is consisted of five elements:
|
||||
//
|
||||
// # metadata
|
||||
// This object contains a few meta fields, such as the associated state root,
|
||||
// block number, version tag and so on. This object may contain an extra
|
||||
// accountHash list which means the storage changes belong to these accounts
|
||||
// are not complete due to large contract destruction. The incomplete history
|
||||
// can not be used for rollback and serving archive state request.
|
||||
//
|
||||
// # account index
|
||||
// This object contains some index information of account. For example, offset
|
||||
// and length indicate the location of the data belonging to the account. Besides,
|
||||
// storageOffset and storageSlots indicate the storage modification location
|
||||
// belonging to the account.
|
||||
//
|
||||
// The size of each account index is *fixed*, and all indexes are sorted
|
||||
// lexicographically. Thus binary search can be performed to quickly locate a
|
||||
// specific account.
|
||||
//
|
||||
// # account data
|
||||
// Account data is a concatenated byte stream composed of all account data.
|
||||
// The account data can be solved by the offset and length info indicated
|
||||
// by corresponding account index.
|
||||
//
|
||||
// fixed size
|
||||
// ^ ^
|
||||
// / \
|
||||
// +-----------------+-----------------+----------------+-----------------+
|
||||
// | Account index 1 | Account index 2 | ... | Account index N |
|
||||
// +-----------------+-----------------+----------------+-----------------+
|
||||
// |
|
||||
// | length
|
||||
// offset |----------------+
|
||||
// v v
|
||||
// +----------------+----------------+----------------+----------------+
|
||||
// | Account data 1 | Account data 2 | ... | Account data N |
|
||||
// +----------------+----------------+----------------+----------------+
|
||||
//
|
||||
// # storage index
|
||||
// This object is similar with account index. It's also fixed size and contains
|
||||
// the location info of storage slot data.
|
||||
//
|
||||
// # storage data
|
||||
// Storage data is a concatenated byte stream composed of all storage slot data.
|
||||
// The storage slot data can be solved by the location info indicated by
|
||||
// corresponding account index and storage slot index.
|
||||
//
|
||||
// fixed size
|
||||
// ^ ^
|
||||
// / \
|
||||
// +-----------------+-----------------+----------------+-----------------+
|
||||
// | Account index 1 | Account index 2 | ... | Account index N |
|
||||
// +-----------------+-----------------+----------------+-----------------+
|
||||
// |
|
||||
// | storage slots
|
||||
// storage offset |-----------------------------------------------------+
|
||||
// v v
|
||||
// +-----------------+-----------------+-----------------+
|
||||
// | storage index 1 | storage index 2 | storage index 3 |
|
||||
// +-----------------+-----------------+-----------------+
|
||||
// | length
|
||||
// offset |-------------+
|
||||
// v v
|
||||
// +-------------+
|
||||
// | slot data 1 |
|
||||
// +-------------+
|
||||
|
||||
// accountIndex describes the metadata belonging to an account.
|
||||
type accountIndex struct { |
||||
address common.Address // The address of account
|
||||
length uint8 // The length of account data, size limited by 255
|
||||
offset uint32 // The offset of item in account data table
|
||||
storageOffset uint32 // The offset of storage index in storage index table
|
||||
storageSlots uint32 // The number of mutated storage slots belonging to the account
|
||||
} |
||||
|
||||
// encode packs account index into byte stream.
|
||||
func (i *accountIndex) encode() []byte { |
||||
var buf [accountIndexSize]byte |
||||
copy(buf[:], i.address.Bytes()) |
||||
buf[common.AddressLength] = i.length |
||||
binary.BigEndian.PutUint32(buf[common.AddressLength+1:], i.offset) |
||||
binary.BigEndian.PutUint32(buf[common.AddressLength+5:], i.storageOffset) |
||||
binary.BigEndian.PutUint32(buf[common.AddressLength+9:], i.storageSlots) |
||||
return buf[:] |
||||
} |
||||
|
||||
// decode unpacks account index from byte stream.
|
||||
func (i *accountIndex) decode(blob []byte) { |
||||
i.address = common.BytesToAddress(blob[:common.AddressLength]) |
||||
i.length = blob[common.AddressLength] |
||||
i.offset = binary.BigEndian.Uint32(blob[common.AddressLength+1:]) |
||||
i.storageOffset = binary.BigEndian.Uint32(blob[common.AddressLength+5:]) |
||||
i.storageSlots = binary.BigEndian.Uint32(blob[common.AddressLength+9:]) |
||||
} |
||||
|
||||
// slotIndex describes the metadata belonging to a storage slot.
|
||||
type slotIndex struct { |
||||
hash common.Hash // The hash of slot key
|
||||
length uint8 // The length of storage slot, up to 32 bytes defined in protocol
|
||||
offset uint32 // The offset of item in storage slot data table
|
||||
} |
||||
|
||||
// encode packs slot index into byte stream.
|
||||
func (i *slotIndex) encode() []byte { |
||||
var buf [slotIndexSize]byte |
||||
copy(buf[:common.HashLength], i.hash.Bytes()) |
||||
buf[common.HashLength] = i.length |
||||
binary.BigEndian.PutUint32(buf[common.HashLength+1:], i.offset) |
||||
return buf[:] |
||||
} |
||||
|
||||
// decode unpack slot index from the byte stream.
|
||||
func (i *slotIndex) decode(blob []byte) { |
||||
i.hash = common.BytesToHash(blob[:common.HashLength]) |
||||
i.length = blob[common.HashLength] |
||||
i.offset = binary.BigEndian.Uint32(blob[common.HashLength+1:]) |
||||
} |
||||
|
||||
// meta describes the meta data of state history object.
|
||||
type meta struct { |
||||
version uint8 // version tag of history object
|
||||
parent common.Hash // prev-state root before the state transition
|
||||
root common.Hash // post-state root after the state transition
|
||||
block uint64 // associated block number
|
||||
incomplete []common.Address // list of address whose storage set is incomplete
|
||||
} |
||||
|
||||
// encode packs the meta object into byte stream.
|
||||
func (m *meta) encode() []byte { |
||||
buf := make([]byte, historyMetaSize+len(m.incomplete)*common.AddressLength) |
||||
buf[0] = m.version |
||||
copy(buf[1:1+common.HashLength], m.parent.Bytes()) |
||||
copy(buf[1+common.HashLength:1+2*common.HashLength], m.root.Bytes()) |
||||
binary.BigEndian.PutUint64(buf[1+2*common.HashLength:historyMetaSize], m.block) |
||||
for i, h := range m.incomplete { |
||||
copy(buf[i*common.AddressLength+historyMetaSize:], h.Bytes()) |
||||
} |
||||
return buf[:] |
||||
} |
||||
|
||||
// decode unpacks the meta object from byte stream.
|
||||
func (m *meta) decode(blob []byte) error { |
||||
if len(blob) < 1 { |
||||
return fmt.Errorf("no version tag") |
||||
} |
||||
switch blob[0] { |
||||
case stateHistoryVersion: |
||||
if len(blob) < historyMetaSize { |
||||
return fmt.Errorf("invalid state history meta, len: %d", len(blob)) |
||||
} |
||||
if (len(blob)-historyMetaSize)%common.AddressLength != 0 { |
||||
return fmt.Errorf("corrupted state history meta, len: %d", len(blob)) |
||||
} |
||||
m.version = blob[0] |
||||
m.parent = common.BytesToHash(blob[1 : 1+common.HashLength]) |
||||
m.root = common.BytesToHash(blob[1+common.HashLength : 1+2*common.HashLength]) |
||||
m.block = binary.BigEndian.Uint64(blob[1+2*common.HashLength : historyMetaSize]) |
||||
for pos := historyMetaSize; pos < len(blob); { |
||||
m.incomplete = append(m.incomplete, common.BytesToAddress(blob[pos:pos+common.AddressLength])) |
||||
pos += common.AddressLength |
||||
} |
||||
return nil |
||||
default: |
||||
return fmt.Errorf("unknown version %d", blob[0]) |
||||
} |
||||
} |
||||
|
||||
// history represents a set of state changes belong to a block along with
|
||||
// the metadata including the state roots involved in the state transition.
|
||||
// State history objects in disk are linked with each other by a unique id
|
||||
// (8-bytes integer), the oldest state history object can be pruned on demand
|
||||
// in order to control the storage size.
|
||||
type history struct { |
||||
meta *meta // Meta data of history
|
||||
accounts map[common.Address][]byte // Account data keyed by its address hash
|
||||
accountList []common.Address // Sorted account hash list
|
||||
storages map[common.Address]map[common.Hash][]byte // Storage data keyed by its address hash and slot hash
|
||||
storageList map[common.Address][]common.Hash // Sorted slot hash list
|
||||
} |
||||
|
||||
// newHistory constructs the state history object with provided state change set.
|
||||
func newHistory(root common.Hash, parent common.Hash, block uint64, states *triestate.Set) *history { |
||||
var ( |
||||
accountList []common.Address |
||||
storageList = make(map[common.Address][]common.Hash) |
||||
incomplete []common.Address |
||||
) |
||||
for addr := range states.Accounts { |
||||
accountList = append(accountList, addr) |
||||
} |
||||
slices.SortFunc(accountList, func(a, b common.Address) bool { return a.Less(b) }) |
||||
|
||||
for addr, slots := range states.Storages { |
||||
slist := make([]common.Hash, 0, len(slots)) |
||||
for slotHash := range slots { |
||||
slist = append(slist, slotHash) |
||||
} |
||||
slices.SortFunc(slist, func(a, b common.Hash) bool { return a.Less(b) }) |
||||
storageList[addr] = slist |
||||
} |
||||
for addr := range states.Incomplete { |
||||
incomplete = append(incomplete, addr) |
||||
} |
||||
slices.SortFunc(incomplete, func(a, b common.Address) bool { return a.Less(b) }) |
||||
|
||||
return &history{ |
||||
meta: &meta{ |
||||
version: stateHistoryVersion, |
||||
parent: parent, |
||||
root: root, |
||||
block: block, |
||||
incomplete: incomplete, |
||||
}, |
||||
accounts: states.Accounts, |
||||
accountList: accountList, |
||||
storages: states.Storages, |
||||
storageList: storageList, |
||||
} |
||||
} |
||||
|
||||
// encode serializes the state history and returns four byte streams represent
|
||||
// concatenated account/storage data, account/storage indexes respectively.
|
||||
func (h *history) encode() ([]byte, []byte, []byte, []byte) { |
||||
var ( |
||||
slotNumber uint32 // the number of processed slots
|
||||
accountData []byte // the buffer for concatenated account data
|
||||
storageData []byte // the buffer for concatenated storage data
|
||||
accountIndexes []byte // the buffer for concatenated account index
|
||||
storageIndexes []byte // the buffer for concatenated storage index
|
||||
) |
||||
for _, addr := range h.accountList { |
||||
accIndex := accountIndex{ |
||||
address: addr, |
||||
length: uint8(len(h.accounts[addr])), |
||||
offset: uint32(len(accountData)), |
||||
} |
||||
slots, exist := h.storages[addr] |
||||
if exist { |
||||
// Encode storage slots in order
|
||||
for _, slotHash := range h.storageList[addr] { |
||||
sIndex := slotIndex{ |
||||
hash: slotHash, |
||||
length: uint8(len(slots[slotHash])), |
||||
offset: uint32(len(storageData)), |
||||
} |
||||
storageData = append(storageData, slots[slotHash]...) |
||||
storageIndexes = append(storageIndexes, sIndex.encode()...) |
||||
} |
||||
// Fill up the storage meta in account index
|
||||
accIndex.storageOffset = slotNumber |
||||
accIndex.storageSlots = uint32(len(slots)) |
||||
slotNumber += uint32(len(slots)) |
||||
} |
||||
accountData = append(accountData, h.accounts[addr]...) |
||||
accountIndexes = append(accountIndexes, accIndex.encode()...) |
||||
} |
||||
return accountData, storageData, accountIndexes, storageIndexes |
||||
} |
||||
|
||||
// decoder wraps the byte streams for decoding with extra meta fields.
|
||||
type decoder struct { |
||||
accountData []byte // the buffer for concatenated account data
|
||||
storageData []byte // the buffer for concatenated storage data
|
||||
accountIndexes []byte // the buffer for concatenated account index
|
||||
storageIndexes []byte // the buffer for concatenated storage index
|
||||
|
||||
lastAccount *common.Address // the address of last resolved account
|
||||
lastAccountRead uint32 // the read-cursor position of account data
|
||||
lastSlotIndexRead uint32 // the read-cursor position of storage slot index
|
||||
lastSlotDataRead uint32 // the read-cursor position of storage slot data
|
||||
} |
||||
|
||||
// verify validates the provided byte streams for decoding state history. A few
|
||||
// checks will be performed to quickly detect data corruption. The byte stream
|
||||
// is regarded as corrupted if:
|
||||
//
|
||||
// - account indexes buffer is empty(empty state set is invalid)
|
||||
// - account indexes/storage indexer buffer is not aligned
|
||||
//
|
||||
// note, these situations are allowed:
|
||||
//
|
||||
// - empty account data: all accounts were not present
|
||||
// - empty storage set: no slots are modified
|
||||
func (r *decoder) verify() error { |
||||
if len(r.accountIndexes)%accountIndexSize != 0 || len(r.accountIndexes) == 0 { |
||||
return fmt.Errorf("invalid account index, len: %d", len(r.accountIndexes)) |
||||
} |
||||
if len(r.storageIndexes)%slotIndexSize != 0 { |
||||
return fmt.Errorf("invalid storage index, len: %d", len(r.storageIndexes)) |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// readAccount parses the account from the byte stream with specified position.
|
||||
func (r *decoder) readAccount(pos int) (accountIndex, []byte, error) { |
||||
// Decode account index from the index byte stream.
|
||||
var index accountIndex |
||||
if (pos+1)*accountIndexSize > len(r.accountIndexes) { |
||||
return accountIndex{}, nil, errors.New("account data buffer is corrupted") |
||||
} |
||||
index.decode(r.accountIndexes[pos*accountIndexSize : (pos+1)*accountIndexSize]) |
||||
|
||||
// Perform validation before parsing account data, ensure
|
||||
// - account is sorted in order in byte stream
|
||||
// - account data is strictly encoded with no gap inside
|
||||
// - account data is not out-of-slice
|
||||
if r.lastAccount != nil { // zero address is possible
|
||||
if bytes.Compare(r.lastAccount.Bytes(), index.address.Bytes()) >= 0 { |
||||
return accountIndex{}, nil, errors.New("account is not in order") |
||||
} |
||||
} |
||||
if index.offset != r.lastAccountRead { |
||||
return accountIndex{}, nil, errors.New("account data buffer is gaped") |
||||
} |
||||
last := index.offset + uint32(index.length) |
||||
if uint32(len(r.accountData)) < last { |
||||
return accountIndex{}, nil, errors.New("account data buffer is corrupted") |
||||
} |
||||
data := r.accountData[index.offset:last] |
||||
|
||||
r.lastAccount = &index.address |
||||
r.lastAccountRead = last |
||||
|
||||
return index, data, nil |
||||
} |
||||
|
||||
// readStorage parses the storage slots from the byte stream with specified account.
|
||||
func (r *decoder) readStorage(accIndex accountIndex) ([]common.Hash, map[common.Hash][]byte, error) { |
||||
var ( |
||||
last common.Hash |
||||
list []common.Hash |
||||
storage = make(map[common.Hash][]byte) |
||||
) |
||||
for j := 0; j < int(accIndex.storageSlots); j++ { |
||||
var ( |
||||
index slotIndex |
||||
start = (accIndex.storageOffset + uint32(j)) * uint32(slotIndexSize) |
||||
end = (accIndex.storageOffset + uint32(j+1)) * uint32(slotIndexSize) |
||||
) |
||||
// Perform validation before parsing storage slot data, ensure
|
||||
// - slot index is not out-of-slice
|
||||
// - slot data is not out-of-slice
|
||||
// - slot is sorted in order in byte stream
|
||||
// - slot indexes is strictly encoded with no gap inside
|
||||
// - slot data is strictly encoded with no gap inside
|
||||
if start != r.lastSlotIndexRead { |
||||
return nil, nil, errors.New("storage index buffer is gapped") |
||||
} |
||||
if uint32(len(r.storageIndexes)) < end { |
||||
return nil, nil, errors.New("storage index buffer is corrupted") |
||||
} |
||||
index.decode(r.storageIndexes[start:end]) |
||||
|
||||
if bytes.Compare(last.Bytes(), index.hash.Bytes()) >= 0 { |
||||
return nil, nil, errors.New("storage slot is not in order") |
||||
} |
||||
if index.offset != r.lastSlotDataRead { |
||||
return nil, nil, errors.New("storage data buffer is gapped") |
||||
} |
||||
sEnd := index.offset + uint32(index.length) |
||||
if uint32(len(r.storageData)) < sEnd { |
||||
return nil, nil, errors.New("storage data buffer is corrupted") |
||||
} |
||||
storage[index.hash] = r.storageData[r.lastSlotDataRead:sEnd] |
||||
list = append(list, index.hash) |
||||
|
||||
last = index.hash |
||||
r.lastSlotIndexRead = end |
||||
r.lastSlotDataRead = sEnd |
||||
} |
||||
return list, storage, nil |
||||
} |
||||
|
||||
// decode deserializes the account and storage data from the provided byte stream.
|
||||
func (h *history) decode(accountData, storageData, accountIndexes, storageIndexes []byte) error { |
||||
var ( |
||||
accounts = make(map[common.Address][]byte) |
||||
storages = make(map[common.Address]map[common.Hash][]byte) |
||||
accountList []common.Address |
||||
storageList = make(map[common.Address][]common.Hash) |
||||
|
||||
r = &decoder{ |
||||
accountData: accountData, |
||||
storageData: storageData, |
||||
accountIndexes: accountIndexes, |
||||
storageIndexes: storageIndexes, |
||||
} |
||||
) |
||||
if err := r.verify(); err != nil { |
||||
return err |
||||
} |
||||
for i := 0; i < len(accountIndexes)/accountIndexSize; i++ { |
||||
// Resolve account first
|
||||
accIndex, accData, err := r.readAccount(i) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
accounts[accIndex.address] = accData |
||||
accountList = append(accountList, accIndex.address) |
||||
|
||||
// Resolve storage slots
|
||||
slotList, slotData, err := r.readStorage(accIndex) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
if len(slotList) > 0 { |
||||
storageList[accIndex.address] = slotList |
||||
storages[accIndex.address] = slotData |
||||
} |
||||
} |
||||
h.accounts = accounts |
||||
h.accountList = accountList |
||||
h.storages = storages |
||||
h.storageList = storageList |
||||
return nil |
||||
} |
||||
|
||||
// readHistory reads and decodes the state history object by the given id.
|
||||
func readHistory(freezer *rawdb.ResettableFreezer, id uint64) (*history, error) { |
||||
blob := rawdb.ReadStateHistoryMeta(freezer, id) |
||||
if len(blob) == 0 { |
||||
return nil, fmt.Errorf("state history not found %d", id) |
||||
} |
||||
var m meta |
||||
if err := m.decode(blob); err != nil { |
||||
return nil, err |
||||
} |
||||
var ( |
||||
dec = history{meta: &m} |
||||
accountData = rawdb.ReadStateAccountHistory(freezer, id) |
||||
storageData = rawdb.ReadStateStorageHistory(freezer, id) |
||||
accountIndexes = rawdb.ReadStateAccountIndex(freezer, id) |
||||
storageIndexes = rawdb.ReadStateStorageIndex(freezer, id) |
||||
) |
||||
if err := dec.decode(accountData, storageData, accountIndexes, storageIndexes); err != nil { |
||||
return nil, err |
||||
} |
||||
return &dec, nil |
||||
} |
||||
|
||||
// writeHistory writes the state history with provided state set. After
|
||||
// storing the corresponding state history, it will also prune the stale
|
||||
// histories from the disk with the given threshold.
|
||||
func writeHistory(db ethdb.KeyValueStore, freezer *rawdb.ResettableFreezer, dl *diffLayer, limit uint64) error { |
||||
// Short circuit if state set is not available.
|
||||
if dl.states == nil { |
||||
return errors.New("state change set is not available") |
||||
} |
||||
var ( |
||||
err error |
||||
n int |
||||
start = time.Now() |
||||
h = newHistory(dl.rootHash(), dl.parentLayer().rootHash(), dl.block, dl.states) |
||||
) |
||||
accountData, storageData, accountIndex, storageIndex := h.encode() |
||||
dataSize := common.StorageSize(len(accountData) + len(storageData)) |
||||
indexSize := common.StorageSize(len(accountIndex) + len(storageIndex)) |
||||
|
||||
// Write history data into five freezer table respectively.
|
||||
rawdb.WriteStateHistory(freezer, dl.stateID(), h.meta.encode(), accountIndex, storageIndex, accountData, storageData) |
||||
|
||||
// Prune stale state histories based on the config.
|
||||
if limit != 0 && dl.stateID() > limit { |
||||
n, err = truncateFromTail(db, freezer, dl.stateID()-limit) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
} |
||||
historyDataBytesMeter.Mark(int64(dataSize)) |
||||
historyIndexBytesMeter.Mark(int64(indexSize)) |
||||
historyBuildTimeMeter.UpdateSince(start) |
||||
log.Debug("Stored state history", "id", dl.stateID(), "block", dl.block, "data", dataSize, "index", indexSize, "pruned", n, "elapsed", common.PrettyDuration(time.Since(start))) |
||||
return nil |
||||
} |
||||
|
||||
// checkHistories retrieves a batch of meta objects with the specified range
|
||||
// and performs the callback on each item.
|
||||
func checkHistories(freezer *rawdb.ResettableFreezer, start, count uint64, check func(*meta) error) error { |
||||
for count > 0 { |
||||
number := count |
||||
if number > 10000 { |
||||
number = 10000 // split the big read into small chunks
|
||||
} |
||||
blobs, err := rawdb.ReadStateHistoryMetaList(freezer, start, number) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
for _, blob := range blobs { |
||||
var dec meta |
||||
if err := dec.decode(blob); err != nil { |
||||
return err |
||||
} |
||||
if err := check(&dec); err != nil { |
||||
return err |
||||
} |
||||
} |
||||
count -= uint64(len(blobs)) |
||||
start += uint64(len(blobs)) |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// truncateFromHead removes the extra state histories from the head with the given
|
||||
// parameters. It returns the number of items removed from the head.
|
||||
func truncateFromHead(db ethdb.Batcher, freezer *rawdb.ResettableFreezer, nhead uint64) (int, error) { |
||||
ohead, err := freezer.Ancients() |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
if ohead <= nhead { |
||||
return 0, nil |
||||
} |
||||
// Load the meta objects in range [nhead+1, ohead]
|
||||
blobs, err := rawdb.ReadStateHistoryMetaList(freezer, nhead+1, ohead-nhead) |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
batch := db.NewBatch() |
||||
for _, blob := range blobs { |
||||
var m meta |
||||
if err := m.decode(blob); err != nil { |
||||
return 0, err |
||||
} |
||||
rawdb.DeleteStateID(batch, m.root) |
||||
} |
||||
if err := batch.Write(); err != nil { |
||||
return 0, err |
||||
} |
||||
ohead, err = freezer.TruncateHead(nhead) |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
return int(ohead - nhead), nil |
||||
} |
||||
|
||||
// truncateFromTail removes the extra state histories from the tail with the given
|
||||
// parameters. It returns the number of items removed from the tail.
|
||||
func truncateFromTail(db ethdb.Batcher, freezer *rawdb.ResettableFreezer, ntail uint64) (int, error) { |
||||
otail, err := freezer.Tail() |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
if otail >= ntail { |
||||
return 0, nil |
||||
} |
||||
// Load the meta objects in range [otail+1, ntail]
|
||||
blobs, err := rawdb.ReadStateHistoryMetaList(freezer, otail+1, ntail-otail) |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
batch := db.NewBatch() |
||||
for _, blob := range blobs { |
||||
var m meta |
||||
if err := m.decode(blob); err != nil { |
||||
return 0, err |
||||
} |
||||
rawdb.DeleteStateID(batch, m.root) |
||||
} |
||||
if err := batch.Write(); err != nil { |
||||
return 0, err |
||||
} |
||||
otail, err = freezer.TruncateTail(ntail) |
||||
if err != nil { |
||||
return 0, err |
||||
} |
||||
return int(ntail - otail), nil |
||||
} |
@ -0,0 +1,290 @@ |
||||
// Copyright 2022 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" |
||||
"fmt" |
||||
"reflect" |
||||
"testing" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/ethdb" |
||||
"github.com/ethereum/go-ethereum/rlp" |
||||
"github.com/ethereum/go-ethereum/trie/testutil" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
// randomStateSet generates a random state change set.
|
||||
func randomStateSet(n int) *triestate.Set { |
||||
var ( |
||||
accounts = make(map[common.Address][]byte) |
||||
storages = make(map[common.Address]map[common.Hash][]byte) |
||||
) |
||||
for i := 0; i < n; i++ { |
||||
addr := testutil.RandomAddress() |
||||
storages[addr] = make(map[common.Hash][]byte) |
||||
for j := 0; j < 3; j++ { |
||||
v, _ := rlp.EncodeToBytes(common.TrimLeftZeroes(testutil.RandBytes(32))) |
||||
storages[addr][testutil.RandomHash()] = v |
||||
} |
||||
account := generateAccount(types.EmptyRootHash) |
||||
accounts[addr] = types.SlimAccountRLP(account) |
||||
} |
||||
return triestate.New(accounts, storages, nil) |
||||
} |
||||
|
||||
func makeHistory() *history { |
||||
return newHistory(testutil.RandomHash(), types.EmptyRootHash, 0, randomStateSet(3)) |
||||
} |
||||
|
||||
func makeHistories(n int) []*history { |
||||
var ( |
||||
parent = types.EmptyRootHash |
||||
result []*history |
||||
) |
||||
for i := 0; i < n; i++ { |
||||
root := testutil.RandomHash() |
||||
h := newHistory(root, parent, uint64(i), randomStateSet(3)) |
||||
parent = root |
||||
result = append(result, h) |
||||
} |
||||
return result |
||||
} |
||||
|
||||
func TestEncodeDecodeHistory(t *testing.T) { |
||||
var ( |
||||
m meta |
||||
dec history |
||||
obj = makeHistory() |
||||
) |
||||
// check if meta data can be correctly encode/decode
|
||||
blob := obj.meta.encode() |
||||
if err := m.decode(blob); err != nil { |
||||
t.Fatalf("Failed to decode %v", err) |
||||
} |
||||
if !reflect.DeepEqual(&m, obj.meta) { |
||||
t.Fatal("meta is mismatched") |
||||
} |
||||
|
||||
// check if account/storage data can be correctly encode/decode
|
||||
accountData, storageData, accountIndexes, storageIndexes := obj.encode() |
||||
if err := dec.decode(accountData, storageData, accountIndexes, storageIndexes); err != nil { |
||||
t.Fatalf("Failed to decode, err: %v", err) |
||||
} |
||||
if !compareSet(dec.accounts, obj.accounts) { |
||||
t.Fatal("account data is mismatched") |
||||
} |
||||
if !compareStorages(dec.storages, obj.storages) { |
||||
t.Fatal("storage data is mismatched") |
||||
} |
||||
if !compareList(dec.accountList, obj.accountList) { |
||||
t.Fatal("account list is mismatched") |
||||
} |
||||
if !compareStorageList(dec.storageList, obj.storageList) { |
||||
t.Fatal("storage list is mismatched") |
||||
} |
||||
} |
||||
|
||||
func checkHistory(t *testing.T, db ethdb.KeyValueReader, freezer *rawdb.ResettableFreezer, id uint64, root common.Hash, exist bool) { |
||||
blob := rawdb.ReadStateHistoryMeta(freezer, id) |
||||
if exist && len(blob) == 0 { |
||||
t.Fatalf("Failed to load trie history, %d", id) |
||||
} |
||||
if !exist && len(blob) != 0 { |
||||
t.Fatalf("Unexpected trie history, %d", id) |
||||
} |
||||
if exist && rawdb.ReadStateID(db, root) == nil { |
||||
t.Fatalf("Root->ID mapping is not found, %d", id) |
||||
} |
||||
if !exist && rawdb.ReadStateID(db, root) != nil { |
||||
t.Fatalf("Unexpected root->ID mapping, %d", id) |
||||
} |
||||
} |
||||
|
||||
func checkHistoriesInRange(t *testing.T, db ethdb.KeyValueReader, freezer *rawdb.ResettableFreezer, from, to uint64, roots []common.Hash, exist bool) { |
||||
for i, j := from, 0; i <= to; i, j = i+1, j+1 { |
||||
checkHistory(t, db, freezer, i, roots[j], exist) |
||||
} |
||||
} |
||||
|
||||
func TestTruncateHeadHistory(t *testing.T) { |
||||
var ( |
||||
roots []common.Hash |
||||
hs = makeHistories(10) |
||||
db = rawdb.NewMemoryDatabase() |
||||
freezer, _ = openFreezer(t.TempDir(), false) |
||||
) |
||||
defer freezer.Close() |
||||
|
||||
for i := 0; i < len(hs); i++ { |
||||
accountData, storageData, accountIndex, storageIndex := hs[i].encode() |
||||
rawdb.WriteStateHistory(freezer, uint64(i+1), hs[i].meta.encode(), accountIndex, storageIndex, accountData, storageData) |
||||
rawdb.WriteStateID(db, hs[i].meta.root, uint64(i+1)) |
||||
roots = append(roots, hs[i].meta.root) |
||||
} |
||||
for size := len(hs); size > 0; size-- { |
||||
pruned, err := truncateFromHead(db, freezer, uint64(size-1)) |
||||
if err != nil { |
||||
t.Fatalf("Failed to truncate from head %v", err) |
||||
} |
||||
if pruned != 1 { |
||||
t.Error("Unexpected pruned items", "want", 1, "got", pruned) |
||||
} |
||||
checkHistoriesInRange(t, db, freezer, uint64(size), uint64(10), roots[size-1:], false) |
||||
checkHistoriesInRange(t, db, freezer, uint64(1), uint64(size-1), roots[:size-1], true) |
||||
} |
||||
} |
||||
|
||||
func TestTruncateTailHistory(t *testing.T) { |
||||
var ( |
||||
roots []common.Hash |
||||
hs = makeHistories(10) |
||||
db = rawdb.NewMemoryDatabase() |
||||
freezer, _ = openFreezer(t.TempDir(), false) |
||||
) |
||||
defer freezer.Close() |
||||
|
||||
for i := 0; i < len(hs); i++ { |
||||
accountData, storageData, accountIndex, storageIndex := hs[i].encode() |
||||
rawdb.WriteStateHistory(freezer, uint64(i+1), hs[i].meta.encode(), accountIndex, storageIndex, accountData, storageData) |
||||
rawdb.WriteStateID(db, hs[i].meta.root, uint64(i+1)) |
||||
roots = append(roots, hs[i].meta.root) |
||||
} |
||||
for newTail := 1; newTail < len(hs); newTail++ { |
||||
pruned, _ := truncateFromTail(db, freezer, uint64(newTail)) |
||||
if pruned != 1 { |
||||
t.Error("Unexpected pruned items", "want", 1, "got", pruned) |
||||
} |
||||
checkHistoriesInRange(t, db, freezer, uint64(1), uint64(newTail), roots[:newTail], false) |
||||
checkHistoriesInRange(t, db, freezer, uint64(newTail+1), uint64(10), roots[newTail:], true) |
||||
} |
||||
} |
||||
|
||||
func TestTruncateTailHistories(t *testing.T) { |
||||
var cases = []struct { |
||||
limit uint64 |
||||
expPruned int |
||||
maxPruned uint64 |
||||
minUnpruned uint64 |
||||
empty bool |
||||
}{ |
||||
{ |
||||
1, 9, 9, 10, false, |
||||
}, |
||||
{ |
||||
0, 10, 10, 0 /* no meaning */, true, |
||||
}, |
||||
{ |
||||
10, 0, 0, 1, false, |
||||
}, |
||||
} |
||||
for i, c := range cases { |
||||
var ( |
||||
roots []common.Hash |
||||
hs = makeHistories(10) |
||||
db = rawdb.NewMemoryDatabase() |
||||
freezer, _ = openFreezer(t.TempDir()+fmt.Sprintf("%d", i), false) |
||||
) |
||||
defer freezer.Close() |
||||
|
||||
for i := 0; i < len(hs); i++ { |
||||
accountData, storageData, accountIndex, storageIndex := hs[i].encode() |
||||
rawdb.WriteStateHistory(freezer, uint64(i+1), hs[i].meta.encode(), accountIndex, storageIndex, accountData, storageData) |
||||
rawdb.WriteStateID(db, hs[i].meta.root, uint64(i+1)) |
||||
roots = append(roots, hs[i].meta.root) |
||||
} |
||||
pruned, _ := truncateFromTail(db, freezer, uint64(10)-c.limit) |
||||
if pruned != c.expPruned { |
||||
t.Error("Unexpected pruned items", "want", c.expPruned, "got", pruned) |
||||
} |
||||
if c.empty { |
||||
checkHistoriesInRange(t, db, freezer, uint64(1), uint64(10), roots, false) |
||||
} else { |
||||
tail := 10 - int(c.limit) |
||||
checkHistoriesInRange(t, db, freezer, uint64(1), c.maxPruned, roots[:tail], false) |
||||
checkHistoriesInRange(t, db, freezer, c.minUnpruned, uint64(10), roots[tail:], true) |
||||
} |
||||
} |
||||
} |
||||
|
||||
// openFreezer initializes the freezer instance for storing state histories.
|
||||
func openFreezer(datadir string, readOnly bool) (*rawdb.ResettableFreezer, error) { |
||||
return rawdb.NewStateHistoryFreezer(datadir, readOnly) |
||||
} |
||||
|
||||
func compareSet[k comparable](a, b map[k][]byte) bool { |
||||
if len(a) != len(b) { |
||||
return false |
||||
} |
||||
for key, valA := range a { |
||||
valB, ok := b[key] |
||||
if !ok { |
||||
return false |
||||
} |
||||
if !bytes.Equal(valA, valB) { |
||||
return false |
||||
} |
||||
} |
||||
return true |
||||
} |
||||
|
||||
func compareList[k comparable](a, b []k) bool { |
||||
if len(a) != len(b) { |
||||
return false |
||||
} |
||||
for i := 0; i < len(a); i++ { |
||||
if a[i] != b[i] { |
||||
return false |
||||
} |
||||
} |
||||
return true |
||||
} |
||||
|
||||
func compareStorages(a, b map[common.Address]map[common.Hash][]byte) bool { |
||||
if len(a) != len(b) { |
||||
return false |
||||
} |
||||
for h, subA := range a { |
||||
subB, ok := b[h] |
||||
if !ok { |
||||
return false |
||||
} |
||||
if !compareSet(subA, subB) { |
||||
return false |
||||
} |
||||
} |
||||
return true |
||||
} |
||||
|
||||
func compareStorageList(a, b map[common.Address][]common.Hash) bool { |
||||
if len(a) != len(b) { |
||||
return false |
||||
} |
||||
for h, la := range a { |
||||
lb, ok := b[h] |
||||
if !ok { |
||||
return false |
||||
} |
||||
if !compareList(la, lb) { |
||||
return false |
||||
} |
||||
} |
||||
return true |
||||
} |
@ -0,0 +1,378 @@ |
||||
// Copyright 2022 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" |
||||
"errors" |
||||
"fmt" |
||||
"io" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/rlp" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
var ( |
||||
errMissJournal = errors.New("journal not found") |
||||
errMissVersion = errors.New("version not found") |
||||
errUnexpectedVersion = errors.New("unexpected journal version") |
||||
errMissDiskRoot = errors.New("disk layer root not found") |
||||
errUnmatchedJournal = errors.New("unmatched journal") |
||||
) |
||||
|
||||
const journalVersion uint64 = 0 |
||||
|
||||
// journalNode represents a trie node persisted in the journal.
|
||||
type journalNode struct { |
||||
Path []byte // Path of the node in the trie
|
||||
Blob []byte // RLP-encoded trie node blob, nil means the node is deleted
|
||||
} |
||||
|
||||
// journalNodes represents a list trie nodes belong to a single account
|
||||
// or the main account trie.
|
||||
type journalNodes struct { |
||||
Owner common.Hash |
||||
Nodes []journalNode |
||||
} |
||||
|
||||
// journalAccounts represents a list accounts belong to the layer.
|
||||
type journalAccounts struct { |
||||
Addresses []common.Address |
||||
Accounts [][]byte |
||||
} |
||||
|
||||
// journalStorage represents a list of storage slots belong to an account.
|
||||
type journalStorage struct { |
||||
Incomplete bool |
||||
Account common.Address |
||||
Hashes []common.Hash |
||||
Slots [][]byte |
||||
} |
||||
|
||||
// loadJournal tries to parse the layer journal from the disk.
|
||||
func (db *Database) loadJournal(diskRoot common.Hash) (layer, error) { |
||||
journal := rawdb.ReadTrieJournal(db.diskdb) |
||||
if len(journal) == 0 { |
||||
return nil, errMissJournal |
||||
} |
||||
r := rlp.NewStream(bytes.NewReader(journal), 0) |
||||
|
||||
// Firstly, resolve the first element as the journal version
|
||||
version, err := r.Uint64() |
||||
if err != nil { |
||||
return nil, errMissVersion |
||||
} |
||||
if version != journalVersion { |
||||
return nil, fmt.Errorf("%w want %d got %d", errUnexpectedVersion, journalVersion, version) |
||||
} |
||||
// Secondly, resolve the disk layer root, ensure it's continuous
|
||||
// with disk layer. Note now we can ensure it's the layer journal
|
||||
// correct version, so we expect everything can be resolved properly.
|
||||
var root common.Hash |
||||
if err := r.Decode(&root); err != nil { |
||||
return nil, errMissDiskRoot |
||||
} |
||||
// The journal is not matched with persistent state, discard them.
|
||||
// It can happen that geth crashes without persisting the journal.
|
||||
if !bytes.Equal(root.Bytes(), diskRoot.Bytes()) { |
||||
return nil, fmt.Errorf("%w want %x got %x", errUnmatchedJournal, root, diskRoot) |
||||
} |
||||
// Load the disk layer from the journal
|
||||
base, err := db.loadDiskLayer(r) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
// Load all the diff layers from the journal
|
||||
head, err := db.loadDiffLayer(base, r) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
log.Debug("Loaded layer journal", "diskroot", diskRoot, "diffhead", head.rootHash()) |
||||
return head, nil |
||||
} |
||||
|
||||
// loadLayers loads a pre-existing state layer backed by a key-value store.
|
||||
func (db *Database) loadLayers() layer { |
||||
// Retrieve the root node of persistent state.
|
||||
_, root := rawdb.ReadAccountTrieNode(db.diskdb, nil) |
||||
root = types.TrieRootHash(root) |
||||
|
||||
// Load the layers by resolving the journal
|
||||
head, err := db.loadJournal(root) |
||||
if err == nil { |
||||
return head |
||||
} |
||||
// journal is not matched(or missing) with the persistent state, discard
|
||||
// it. Display log for discarding journal, but try to avoid showing
|
||||
// useless information when the db is created from scratch.
|
||||
if !(root == types.EmptyRootHash && errors.Is(err, errMissJournal)) { |
||||
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, newNodeBuffer(db.bufferSize, nil, 0)) |
||||
} |
||||
|
||||
// loadDiskLayer reads the binary blob from the layer journal, reconstructing
|
||||
// a new disk layer on it.
|
||||
func (db *Database) loadDiskLayer(r *rlp.Stream) (layer, error) { |
||||
// Resolve disk layer root
|
||||
var root common.Hash |
||||
if err := r.Decode(&root); err != nil { |
||||
return nil, fmt.Errorf("load disk root: %v", err) |
||||
} |
||||
// Resolve the state id of disk layer, it can be different
|
||||
// with the persistent id tracked in disk, the id distance
|
||||
// is the number of transitions aggregated in disk layer.
|
||||
var id uint64 |
||||
if err := r.Decode(&id); err != nil { |
||||
return nil, fmt.Errorf("load state id: %v", err) |
||||
} |
||||
stored := rawdb.ReadPersistentStateID(db.diskdb) |
||||
if stored > id { |
||||
return nil, fmt.Errorf("invalid state id: stored %d resolved %d", stored, id) |
||||
} |
||||
// Resolve nodes cached in node buffer
|
||||
var encoded []journalNodes |
||||
if err := r.Decode(&encoded); err != nil { |
||||
return nil, fmt.Errorf("load disk nodes: %v", err) |
||||
} |
||||
nodes := make(map[common.Hash]map[string]*trienode.Node) |
||||
for _, entry := range encoded { |
||||
subset := make(map[string]*trienode.Node) |
||||
for _, n := range entry.Nodes { |
||||
if len(n.Blob) > 0 { |
||||
subset[string(n.Path)] = trienode.New(crypto.Keccak256Hash(n.Blob), n.Blob) |
||||
} else { |
||||
subset[string(n.Path)] = trienode.NewDeleted() |
||||
} |
||||
} |
||||
nodes[entry.Owner] = subset |
||||
} |
||||
// Calculate the internal state transitions by id difference.
|
||||
base := newDiskLayer(root, id, db, nil, newNodeBuffer(db.bufferSize, nodes, id-stored)) |
||||
return base, nil |
||||
} |
||||
|
||||
// loadDiffLayer reads the next sections of a layer journal, reconstructing a new
|
||||
// diff and verifying that it can be linked to the requested parent.
|
||||
func (db *Database) loadDiffLayer(parent layer, r *rlp.Stream) (layer, error) { |
||||
// Read the next diff journal entry
|
||||
var root common.Hash |
||||
if err := r.Decode(&root); err != nil { |
||||
// The first read may fail with EOF, marking the end of the journal
|
||||
if err == io.EOF { |
||||
return parent, nil |
||||
} |
||||
return nil, fmt.Errorf("load diff root: %v", err) |
||||
} |
||||
var block uint64 |
||||
if err := r.Decode(&block); err != nil { |
||||
return nil, fmt.Errorf("load block number: %v", err) |
||||
} |
||||
// Read in-memory trie nodes from journal
|
||||
var encoded []journalNodes |
||||
if err := r.Decode(&encoded); err != nil { |
||||
return nil, fmt.Errorf("load diff nodes: %v", err) |
||||
} |
||||
nodes := make(map[common.Hash]map[string]*trienode.Node) |
||||
for _, entry := range encoded { |
||||
subset := make(map[string]*trienode.Node) |
||||
for _, n := range entry.Nodes { |
||||
if len(n.Blob) > 0 { |
||||
subset[string(n.Path)] = trienode.New(crypto.Keccak256Hash(n.Blob), n.Blob) |
||||
} else { |
||||
subset[string(n.Path)] = trienode.NewDeleted() |
||||
} |
||||
} |
||||
nodes[entry.Owner] = subset |
||||
} |
||||
// Read state changes from journal
|
||||
var ( |
||||
jaccounts journalAccounts |
||||
jstorages []journalStorage |
||||
accounts = make(map[common.Address][]byte) |
||||
storages = make(map[common.Address]map[common.Hash][]byte) |
||||
incomplete = make(map[common.Address]struct{}) |
||||
) |
||||
if err := r.Decode(&jaccounts); err != nil { |
||||
return nil, fmt.Errorf("load diff accounts: %v", err) |
||||
} |
||||
for i, addr := range jaccounts.Addresses { |
||||
accounts[addr] = jaccounts.Accounts[i] |
||||
} |
||||
if err := r.Decode(&jstorages); err != nil { |
||||
return nil, fmt.Errorf("load diff storages: %v", err) |
||||
} |
||||
for _, entry := range jstorages { |
||||
set := make(map[common.Hash][]byte) |
||||
for i, h := range entry.Hashes { |
||||
if len(entry.Slots[i]) > 0 { |
||||
set[h] = entry.Slots[i] |
||||
} else { |
||||
set[h] = nil |
||||
} |
||||
} |
||||
if entry.Incomplete { |
||||
incomplete[entry.Account] = struct{}{} |
||||
} |
||||
storages[entry.Account] = set |
||||
} |
||||
return db.loadDiffLayer(newDiffLayer(parent, root, parent.stateID()+1, block, nodes, triestate.New(accounts, storages, incomplete)), r) |
||||
} |
||||
|
||||
// journal implements the layer interface, marshaling the un-flushed trie nodes
|
||||
// along with layer meta data into provided byte buffer.
|
||||
func (dl *diskLayer) journal(w io.Writer) error { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
// Ensure the layer didn't get stale
|
||||
if dl.stale { |
||||
return errSnapshotStale |
||||
} |
||||
// Step one, write the disk root into the journal.
|
||||
if err := rlp.Encode(w, dl.root); err != nil { |
||||
return err |
||||
} |
||||
// Step two, write the corresponding state id into the journal
|
||||
if err := rlp.Encode(w, dl.id); err != nil { |
||||
return err |
||||
} |
||||
// Step three, write all unwritten nodes into the journal
|
||||
nodes := make([]journalNodes, 0, len(dl.buffer.nodes)) |
||||
for owner, subset := range dl.buffer.nodes { |
||||
entry := journalNodes{Owner: owner} |
||||
for path, node := range subset { |
||||
entry.Nodes = append(entry.Nodes, journalNode{Path: []byte(path), Blob: node.Blob}) |
||||
} |
||||
nodes = append(nodes, entry) |
||||
} |
||||
if err := rlp.Encode(w, nodes); err != nil { |
||||
return err |
||||
} |
||||
log.Debug("Journaled pathdb disk layer", "root", dl.root, "nodes", len(dl.buffer.nodes)) |
||||
return nil |
||||
} |
||||
|
||||
// journal implements the layer interface, writing the memory layer contents
|
||||
// into a buffer to be stored in the database as the layer journal.
|
||||
func (dl *diffLayer) journal(w io.Writer) error { |
||||
dl.lock.RLock() |
||||
defer dl.lock.RUnlock() |
||||
|
||||
// journal the parent first
|
||||
if err := dl.parent.journal(w); err != nil { |
||||
return err |
||||
} |
||||
// Everything below was journaled, persist this layer too
|
||||
if err := rlp.Encode(w, dl.root); err != nil { |
||||
return err |
||||
} |
||||
if err := rlp.Encode(w, dl.block); err != nil { |
||||
return err |
||||
} |
||||
// Write the accumulated trie nodes into buffer
|
||||
nodes := make([]journalNodes, 0, len(dl.nodes)) |
||||
for owner, subset := range dl.nodes { |
||||
entry := journalNodes{Owner: owner} |
||||
for path, node := range subset { |
||||
entry.Nodes = append(entry.Nodes, journalNode{Path: []byte(path), Blob: node.Blob}) |
||||
} |
||||
nodes = append(nodes, entry) |
||||
} |
||||
if err := rlp.Encode(w, nodes); err != nil { |
||||
return err |
||||
} |
||||
// Write the accumulated state changes into buffer
|
||||
var jacct journalAccounts |
||||
for addr, account := range dl.states.Accounts { |
||||
jacct.Addresses = append(jacct.Addresses, addr) |
||||
jacct.Accounts = append(jacct.Accounts, account) |
||||
} |
||||
if err := rlp.Encode(w, jacct); err != nil { |
||||
return err |
||||
} |
||||
storage := make([]journalStorage, 0, len(dl.states.Storages)) |
||||
for addr, slots := range dl.states.Storages { |
||||
entry := journalStorage{Account: addr} |
||||
if _, ok := dl.states.Incomplete[addr]; ok { |
||||
entry.Incomplete = true |
||||
} |
||||
for slotHash, slot := range slots { |
||||
entry.Hashes = append(entry.Hashes, slotHash) |
||||
entry.Slots = append(entry.Slots, slot) |
||||
} |
||||
storage = append(storage, entry) |
||||
} |
||||
if err := rlp.Encode(w, storage); err != nil { |
||||
return err |
||||
} |
||||
log.Debug("Journaled pathdb diff layer", "root", dl.root, "parent", dl.parent.rootHash(), "id", dl.stateID(), "block", dl.block, "nodes", len(dl.nodes)) |
||||
return nil |
||||
} |
||||
|
||||
// Journal commits an entire diff hierarchy to disk into a single journal entry.
|
||||
// This is meant to be used during shutdown to persist the layer without
|
||||
// flattening everything down (bad for reorgs). And this function will mark the
|
||||
// database as read-only to prevent all following mutation to disk.
|
||||
func (db *Database) Journal(root common.Hash) error { |
||||
// Retrieve the head layer to journal from.
|
||||
l := db.tree.get(root) |
||||
if l == nil { |
||||
return fmt.Errorf("triedb layer [%#x] missing", root) |
||||
} |
||||
// Run the journaling
|
||||
db.lock.Lock() |
||||
defer db.lock.Unlock() |
||||
|
||||
// Short circuit if the database is in read only mode.
|
||||
if db.readOnly { |
||||
return errSnapshotReadOnly |
||||
} |
||||
// Firstly write out the metadata of journal
|
||||
journal := new(bytes.Buffer) |
||||
if err := rlp.Encode(journal, journalVersion); err != nil { |
||||
return err |
||||
} |
||||
// The stored state in disk might be empty, convert the
|
||||
// root to emptyRoot in this case.
|
||||
_, diskroot := rawdb.ReadAccountTrieNode(db.diskdb, nil) |
||||
diskroot = types.TrieRootHash(diskroot) |
||||
|
||||
// Secondly write out the state root in disk, ensure all layers
|
||||
// on top are continuous with disk.
|
||||
if err := rlp.Encode(journal, diskroot); err != nil { |
||||
return err |
||||
} |
||||
// Finally write out the journal of each layer in reverse order.
|
||||
if err := l.journal(journal); err != nil { |
||||
return err |
||||
} |
||||
// Store the journal into the database and return
|
||||
rawdb.WriteTrieJournal(db.diskdb, journal.Bytes()) |
||||
|
||||
// Set the db in read only mode to reject all following mutations
|
||||
db.readOnly = true |
||||
log.Info("Stored journal in triedb", "disk", diskroot, "size", common.StorageSize(journal.Len())) |
||||
return nil |
||||
} |
@ -0,0 +1,214 @@ |
||||
// Copyright 2022 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 ( |
||||
"errors" |
||||
"fmt" |
||||
"sync" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
) |
||||
|
||||
// layerTree is a group of state layers identified by the state root.
|
||||
// This structure defines a few basic operations for manipulating
|
||||
// state layers linked with each other in a tree structure. It's
|
||||
// thread-safe to use. However, callers need to ensure the thread-safety
|
||||
// of the referenced layer by themselves.
|
||||
type layerTree struct { |
||||
lock sync.RWMutex |
||||
layers map[common.Hash]layer |
||||
} |
||||
|
||||
// newLayerTree constructs the layerTree with the given head layer.
|
||||
func newLayerTree(head layer) *layerTree { |
||||
tree := new(layerTree) |
||||
tree.reset(head) |
||||
return tree |
||||
} |
||||
|
||||
// reset initializes the layerTree by the given head layer.
|
||||
// All the ancestors will be iterated out and linked in the tree.
|
||||
func (tree *layerTree) reset(head layer) { |
||||
tree.lock.Lock() |
||||
defer tree.lock.Unlock() |
||||
|
||||
var layers = make(map[common.Hash]layer) |
||||
for head != nil { |
||||
layers[head.rootHash()] = head |
||||
head = head.parentLayer() |
||||
} |
||||
tree.layers = layers |
||||
} |
||||
|
||||
// get retrieves a layer belonging to the given state root.
|
||||
func (tree *layerTree) get(root common.Hash) layer { |
||||
tree.lock.RLock() |
||||
defer tree.lock.RUnlock() |
||||
|
||||
return tree.layers[types.TrieRootHash(root)] |
||||
} |
||||
|
||||
// forEach iterates the stored layers inside and applies the
|
||||
// given callback on them.
|
||||
func (tree *layerTree) forEach(onLayer func(layer)) { |
||||
tree.lock.RLock() |
||||
defer tree.lock.RUnlock() |
||||
|
||||
for _, layer := range tree.layers { |
||||
onLayer(layer) |
||||
} |
||||
} |
||||
|
||||
// len returns the number of layers cached.
|
||||
func (tree *layerTree) len() int { |
||||
tree.lock.RLock() |
||||
defer tree.lock.RUnlock() |
||||
|
||||
return len(tree.layers) |
||||
} |
||||
|
||||
// add inserts a new layer into the tree if it can be linked to an existing old parent.
|
||||
func (tree *layerTree) add(root common.Hash, parentRoot common.Hash, block uint64, nodes *trienode.MergedNodeSet, states *triestate.Set) error { |
||||
// Reject noop updates to avoid self-loops. This is a special case that can
|
||||
// happen for clique networks and proof-of-stake networks where empty blocks
|
||||
// don't modify the state (0 block subsidy).
|
||||
//
|
||||
// Although we could silently ignore this internally, it should be the caller's
|
||||
// responsibility to avoid even attempting to insert such a layer.
|
||||
root, parentRoot = types.TrieRootHash(root), types.TrieRootHash(parentRoot) |
||||
if root == parentRoot { |
||||
return errors.New("layer cycle") |
||||
} |
||||
parent := tree.get(parentRoot) |
||||
if parent == nil { |
||||
return fmt.Errorf("triedb parent [%#x] layer missing", parentRoot) |
||||
} |
||||
l := parent.update(root, parent.stateID()+1, block, nodes.Flatten(), states) |
||||
|
||||
tree.lock.Lock() |
||||
tree.layers[l.rootHash()] = l |
||||
tree.lock.Unlock() |
||||
return nil |
||||
} |
||||
|
||||
// cap traverses downwards the diff tree until the number of allowed diff layers
|
||||
// are crossed. All diffs beyond the permitted number are flattened downwards.
|
||||
func (tree *layerTree) cap(root common.Hash, layers int) error { |
||||
// Retrieve the head layer to cap from
|
||||
root = types.TrieRootHash(root) |
||||
l := tree.get(root) |
||||
if l == nil { |
||||
return fmt.Errorf("triedb layer [%#x] missing", root) |
||||
} |
||||
diff, ok := l.(*diffLayer) |
||||
if !ok { |
||||
return fmt.Errorf("triedb layer [%#x] is disk layer", root) |
||||
} |
||||
tree.lock.Lock() |
||||
defer tree.lock.Unlock() |
||||
|
||||
// If full commit was requested, flatten the diffs and merge onto disk
|
||||
if layers == 0 { |
||||
base, err := diff.persist(true) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
// Replace the entire layer tree with the flat base
|
||||
tree.layers = map[common.Hash]layer{base.rootHash(): base} |
||||
return nil |
||||
} |
||||
// Dive until we run out of layers or reach the persistent database
|
||||
for i := 0; i < layers-1; i++ { |
||||
// If we still have diff layers below, continue down
|
||||
if parent, ok := diff.parentLayer().(*diffLayer); ok { |
||||
diff = parent |
||||
} else { |
||||
// Diff stack too shallow, return without modifications
|
||||
return nil |
||||
} |
||||
} |
||||
// We're out of layers, flatten anything below, stopping if it's the disk or if
|
||||
// the memory limit is not yet exceeded.
|
||||
switch parent := diff.parentLayer().(type) { |
||||
case *diskLayer: |
||||
return nil |
||||
|
||||
case *diffLayer: |
||||
// Hold the lock to prevent any read operations until the new
|
||||
// parent is linked correctly.
|
||||
diff.lock.Lock() |
||||
|
||||
base, err := parent.persist(false) |
||||
if err != nil { |
||||
diff.lock.Unlock() |
||||
return err |
||||
} |
||||
tree.layers[base.rootHash()] = base |
||||
diff.parent = base |
||||
|
||||
diff.lock.Unlock() |
||||
|
||||
default: |
||||
panic(fmt.Sprintf("unknown data layer in triedb: %T", parent)) |
||||
} |
||||
// Remove any layer that is stale or links into a stale layer
|
||||
children := make(map[common.Hash][]common.Hash) |
||||
for root, layer := range tree.layers { |
||||
if dl, ok := layer.(*diffLayer); ok { |
||||
parent := dl.parentLayer().rootHash() |
||||
children[parent] = append(children[parent], root) |
||||
} |
||||
} |
||||
var remove func(root common.Hash) |
||||
remove = func(root common.Hash) { |
||||
delete(tree.layers, root) |
||||
for _, child := range children[root] { |
||||
remove(child) |
||||
} |
||||
delete(children, root) |
||||
} |
||||
for root, layer := range tree.layers { |
||||
if dl, ok := layer.(*diskLayer); ok && dl.isStale() { |
||||
remove(root) |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// bottom returns the bottom-most disk layer in this tree.
|
||||
func (tree *layerTree) bottom() *diskLayer { |
||||
tree.lock.RLock() |
||||
defer tree.lock.RUnlock() |
||||
|
||||
if len(tree.layers) == 0 { |
||||
return nil // Shouldn't happen, empty tree
|
||||
} |
||||
// pick a random one as the entry point
|
||||
var current layer |
||||
for _, layer := range tree.layers { |
||||
current = layer |
||||
break |
||||
} |
||||
for current.parentLayer() != nil { |
||||
current = current.parentLayer() |
||||
} |
||||
return current.(*diskLayer) |
||||
} |
@ -0,0 +1,50 @@ |
||||
// Copyright 2022 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 "github.com/ethereum/go-ethereum/metrics" |
||||
|
||||
var ( |
||||
cleanHitMeter = metrics.NewRegisteredMeter("pathdb/clean/hit", nil) |
||||
cleanMissMeter = metrics.NewRegisteredMeter("pathdb/clean/miss", nil) |
||||
cleanReadMeter = metrics.NewRegisteredMeter("pathdb/clean/read", nil) |
||||
cleanWriteMeter = metrics.NewRegisteredMeter("pathdb/clean/write", nil) |
||||
|
||||
dirtyHitMeter = metrics.NewRegisteredMeter("pathdb/dirty/hit", nil) |
||||
dirtyMissMeter = metrics.NewRegisteredMeter("pathdb/dirty/miss", nil) |
||||
dirtyReadMeter = metrics.NewRegisteredMeter("pathdb/dirty/read", nil) |
||||
dirtyWriteMeter = metrics.NewRegisteredMeter("pathdb/dirty/write", nil) |
||||
dirtyNodeHitDepthHist = metrics.NewRegisteredHistogram("pathdb/dirty/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) |
||||
|
||||
commitTimeTimer = metrics.NewRegisteredTimer("pathdb/commit/time", nil) |
||||
commitNodesMeter = metrics.NewRegisteredMeter("pathdb/commit/nodes", nil) |
||||
commitBytesMeter = metrics.NewRegisteredMeter("pathdb/commit/bytes", nil) |
||||
|
||||
gcNodesMeter = metrics.NewRegisteredMeter("pathdb/gc/nodes", nil) |
||||
gcBytesMeter = metrics.NewRegisteredMeter("pathdb/gc/bytes", nil) |
||||
|
||||
diffLayerBytesMeter = metrics.NewRegisteredMeter("pathdb/diff/bytes", nil) |
||||
diffLayerNodesMeter = metrics.NewRegisteredMeter("pathdb/diff/nodes", nil) |
||||
|
||||
historyBuildTimeMeter = metrics.NewRegisteredTimer("pathdb/history/time", nil) |
||||
historyDataBytesMeter = metrics.NewRegisteredMeter("pathdb/history/bytes/data", nil) |
||||
historyIndexBytesMeter = metrics.NewRegisteredMeter("pathdb/history/bytes/index", nil) |
||||
) |
@ -0,0 +1,275 @@ |
||||
// Copyright 2022 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 ( |
||||
"fmt" |
||||
"time" |
||||
|
||||
"github.com/VictoriaMetrics/fastcache" |
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/rawdb" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/ethdb" |
||||
"github.com/ethereum/go-ethereum/log" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
) |
||||
|
||||
// nodebuffer is a collection of modified trie nodes to aggregate the disk
|
||||
// write. The content of the nodebuffer must be checked before diving into
|
||||
// disk (since it basically is not-yet-written data).
|
||||
type nodebuffer struct { |
||||
layers uint64 // The number of diff layers aggregated inside
|
||||
size uint64 // The size of aggregated writes
|
||||
limit uint64 // The maximum memory allowance in bytes
|
||||
nodes map[common.Hash]map[string]*trienode.Node // The dirty node set, mapped by owner and path
|
||||
} |
||||
|
||||
// newNodeBuffer initializes the node buffer with the provided nodes.
|
||||
func newNodeBuffer(limit int, nodes map[common.Hash]map[string]*trienode.Node, layers uint64) *nodebuffer { |
||||
if nodes == nil { |
||||
nodes = make(map[common.Hash]map[string]*trienode.Node) |
||||
} |
||||
var size uint64 |
||||
for _, subset := range nodes { |
||||
for path, n := range subset { |
||||
size += uint64(len(n.Blob) + len(path)) |
||||
} |
||||
} |
||||
return &nodebuffer{ |
||||
layers: layers, |
||||
nodes: nodes, |
||||
size: size, |
||||
limit: uint64(limit), |
||||
} |
||||
} |
||||
|
||||
// node retrieves the trie node with given node info.
|
||||
func (b *nodebuffer) node(owner common.Hash, path []byte, hash common.Hash) (*trienode.Node, error) { |
||||
subset, ok := b.nodes[owner] |
||||
if !ok { |
||||
return nil, nil |
||||
} |
||||
n, ok := subset[string(path)] |
||||
if !ok { |
||||
return nil, nil |
||||
} |
||||
if n.Hash != hash { |
||||
dirtyFalseMeter.Mark(1) |
||||
log.Error("Unexpected trie node in node buffer", "owner", owner, "path", path, "expect", hash, "got", n.Hash) |
||||
return nil, newUnexpectedNodeError("dirty", hash, n.Hash, owner, path) |
||||
} |
||||
return n, nil |
||||
} |
||||
|
||||
// commit merges the dirty nodes into the nodebuffer. This operation won't take
|
||||
// the ownership of the nodes map which belongs to the bottom-most diff layer.
|
||||
// It will just hold the node references from the given map which are safe to
|
||||
// copy.
|
||||
func (b *nodebuffer) commit(nodes map[common.Hash]map[string]*trienode.Node) *nodebuffer { |
||||
var ( |
||||
delta int64 |
||||
overwrite int64 |
||||
overwriteSize int64 |
||||
) |
||||
for owner, subset := range nodes { |
||||
current, exist := b.nodes[owner] |
||||
if !exist { |
||||
// Allocate a new map for the subset instead of claiming it directly
|
||||
// from the passed map to avoid potential concurrent map read/write.
|
||||
// The nodes belong to original diff layer are still accessible even
|
||||
// after merging, thus the ownership of nodes map should still belong
|
||||
// to original layer and any mutation on it should be prevented.
|
||||
current = make(map[string]*trienode.Node) |
||||
for path, n := range subset { |
||||
current[path] = n |
||||
delta += int64(len(n.Blob) + len(path)) |
||||
} |
||||
b.nodes[owner] = current |
||||
continue |
||||
} |
||||
for path, n := range subset { |
||||
if orig, exist := current[path]; !exist { |
||||
delta += int64(len(n.Blob) + len(path)) |
||||
} else { |
||||
delta += int64(len(n.Blob) - len(orig.Blob)) |
||||
overwrite++ |
||||
overwriteSize += int64(len(orig.Blob) + len(path)) |
||||
} |
||||
current[path] = n |
||||
} |
||||
b.nodes[owner] = current |
||||
} |
||||
b.updateSize(delta) |
||||
b.layers++ |
||||
gcNodesMeter.Mark(overwrite) |
||||
gcBytesMeter.Mark(overwriteSize) |
||||
return b |
||||
} |
||||
|
||||
// revert is the reverse operation of commit. It also merges the provided nodes
|
||||
// into the nodebuffer, the difference is that the provided node set should
|
||||
// revert the changes made by the last state transition.
|
||||
func (b *nodebuffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[string]*trienode.Node) error { |
||||
// Short circuit if no embedded state transition to revert.
|
||||
if b.layers == 0 { |
||||
return errStateUnrecoverable |
||||
} |
||||
b.layers-- |
||||
|
||||
// Reset the entire buffer if only a single transition left.
|
||||
if b.layers == 0 { |
||||
b.reset() |
||||
return nil |
||||
} |
||||
var delta int64 |
||||
for owner, subset := range nodes { |
||||
current, ok := b.nodes[owner] |
||||
if !ok { |
||||
panic(fmt.Sprintf("non-existent subset (%x)", owner)) |
||||
} |
||||
for path, n := range subset { |
||||
orig, ok := current[path] |
||||
if !ok { |
||||
// There is a special case in MPT that one child is removed from
|
||||
// a fullNode which only has two children, and then a new child
|
||||
// with different position is immediately inserted into the fullNode.
|
||||
// In this case, the clean child of the fullNode will also be
|
||||
// marked as dirty because of node collapse and expansion.
|
||||
//
|
||||
// In case of database rollback, don't panic if this "clean"
|
||||
// node occurs which is not present in buffer.
|
||||
var nhash common.Hash |
||||
if owner == (common.Hash{}) { |
||||
_, nhash = rawdb.ReadAccountTrieNode(db, []byte(path)) |
||||
} else { |
||||
_, nhash = rawdb.ReadStorageTrieNode(db, owner, []byte(path)) |
||||
} |
||||
// Ignore the clean node in the case described above.
|
||||
if nhash == n.Hash { |
||||
continue |
||||
} |
||||
panic(fmt.Sprintf("non-existent node (%x %v) blob: %v", owner, path, crypto.Keccak256Hash(n.Blob).Hex())) |
||||
} |
||||
current[path] = n |
||||
delta += int64(len(n.Blob)) - int64(len(orig.Blob)) |
||||
} |
||||
} |
||||
b.updateSize(delta) |
||||
return nil |
||||
} |
||||
|
||||
// updateSize updates the total cache size by the given delta.
|
||||
func (b *nodebuffer) updateSize(delta int64) { |
||||
size := int64(b.size) + delta |
||||
if size >= 0 { |
||||
b.size = uint64(size) |
||||
return |
||||
} |
||||
s := b.size |
||||
b.size = 0 |
||||
log.Error("Invalid pathdb buffer size", "prev", common.StorageSize(s), "delta", common.StorageSize(delta)) |
||||
} |
||||
|
||||
// reset cleans up the disk cache.
|
||||
func (b *nodebuffer) reset() { |
||||
b.layers = 0 |
||||
b.size = 0 |
||||
b.nodes = make(map[common.Hash]map[string]*trienode.Node) |
||||
} |
||||
|
||||
// empty returns an indicator if nodebuffer contains any state transition inside.
|
||||
func (b *nodebuffer) empty() bool { |
||||
return b.layers == 0 |
||||
} |
||||
|
||||
// setSize sets the buffer size to the provided number, and invokes a flush
|
||||
// operation if the current memory usage exceeds the new limit.
|
||||
func (b *nodebuffer) setSize(size int, db ethdb.KeyValueStore, clean *fastcache.Cache, id uint64) error { |
||||
b.limit = uint64(size) |
||||
return b.flush(db, clean, id, false) |
||||
} |
||||
|
||||
// flush persists the in-memory dirty trie node into the disk if the configured
|
||||
// memory threshold is reached. Note, all data must be written atomically.
|
||||
func (b *nodebuffer) flush(db ethdb.KeyValueStore, clean *fastcache.Cache, id uint64, force bool) error { |
||||
if b.size <= b.limit && !force { |
||||
return nil |
||||
} |
||||
// Ensure the target state id is aligned with the internal counter.
|
||||
head := rawdb.ReadPersistentStateID(db) |
||||
if head+b.layers != id { |
||||
return fmt.Errorf("buffer layers (%d) cannot be applied on top of persisted state id (%d) to reach requested state id (%d)", b.layers, head, id) |
||||
} |
||||
var ( |
||||
start = time.Now() |
||||
batch = db.NewBatchWithSize(int(b.size)) |
||||
) |
||||
nodes := writeNodes(batch, b.nodes, clean) |
||||
rawdb.WritePersistentStateID(batch, id) |
||||
|
||||
// Flush all mutations in a single batch
|
||||
size := batch.ValueSize() |
||||
if err := batch.Write(); err != nil { |
||||
return err |
||||
} |
||||
commitBytesMeter.Mark(int64(size)) |
||||
commitNodesMeter.Mark(int64(nodes)) |
||||
commitTimeTimer.UpdateSince(start) |
||||
log.Debug("Persisted pathdb nodes", "nodes", len(b.nodes), "bytes", common.StorageSize(size), "elapsed", common.PrettyDuration(time.Since(start))) |
||||
b.reset() |
||||
return nil |
||||
} |
||||
|
||||
// writeNodes writes the trie nodes into the provided database batch.
|
||||
// Note this function will also inject all the newly written nodes
|
||||
// into clean cache.
|
||||
func writeNodes(batch ethdb.Batch, nodes map[common.Hash]map[string]*trienode.Node, clean *fastcache.Cache) (total int) { |
||||
for owner, subset := range nodes { |
||||
for path, n := range subset { |
||||
if n.IsDeleted() { |
||||
if owner == (common.Hash{}) { |
||||
rawdb.DeleteAccountTrieNode(batch, []byte(path)) |
||||
} else { |
||||
rawdb.DeleteStorageTrieNode(batch, owner, []byte(path)) |
||||
} |
||||
if clean != nil { |
||||
clean.Del(cacheKey(owner, []byte(path))) |
||||
} |
||||
} else { |
||||
if owner == (common.Hash{}) { |
||||
rawdb.WriteAccountTrieNode(batch, []byte(path), n.Blob) |
||||
} else { |
||||
rawdb.WriteStorageTrieNode(batch, owner, []byte(path), n.Blob) |
||||
} |
||||
if clean != nil { |
||||
clean.Set(cacheKey(owner, []byte(path)), n.Blob) |
||||
} |
||||
} |
||||
} |
||||
total += len(subset) |
||||
} |
||||
return total |
||||
} |
||||
|
||||
// cacheKey constructs the unique key of clean cache.
|
||||
func cacheKey(owner common.Hash, path []byte) []byte { |
||||
if owner == (common.Hash{}) { |
||||
return path |
||||
} |
||||
return append(owner.Bytes(), path...) |
||||
} |
@ -0,0 +1,156 @@ |
||||
// Copyright 2023 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" |
||||
"fmt" |
||||
|
||||
"github.com/ethereum/go-ethereum/common" |
||||
"github.com/ethereum/go-ethereum/core/types" |
||||
"github.com/ethereum/go-ethereum/crypto" |
||||
"github.com/ethereum/go-ethereum/trie/trienode" |
||||
"github.com/ethereum/go-ethereum/trie/triestate" |
||||
"golang.org/x/exp/slices" |
||||
) |
||||
|
||||
// testHasher is a test utility for computing root hash of a batch of state
|
||||
// elements. The hash algorithm is to sort all the elements in lexicographical
|
||||
// order, concat the key and value in turn, and perform hash calculation on
|
||||
// the concatenated bytes. Except the root hash, a nodeset will be returned
|
||||
// once Commit is called, which contains all the changes made to hasher.
|
||||
type testHasher struct { |
||||
owner common.Hash // owner identifier
|
||||
root common.Hash // original root
|
||||
dirties map[common.Hash][]byte // dirty states
|
||||
cleans map[common.Hash][]byte // clean states
|
||||
} |
||||
|
||||
// newTestHasher constructs a hasher object with provided states.
|
||||
func newTestHasher(owner common.Hash, root common.Hash, cleans map[common.Hash][]byte) (*testHasher, error) { |
||||
if cleans == nil { |
||||
cleans = make(map[common.Hash][]byte) |
||||
} |
||||
if got, _ := hash(cleans); got != root { |
||||
return nil, fmt.Errorf("state root mismatched, want: %x, got: %x", root, got) |
||||
} |
||||
return &testHasher{ |
||||
owner: owner, |
||||
root: root, |
||||
dirties: make(map[common.Hash][]byte), |
||||
cleans: cleans, |
||||
}, nil |
||||
} |
||||
|
||||
// Get returns the value for key stored in the trie.
|
||||
func (h *testHasher) Get(key []byte) ([]byte, error) { |
||||
hash := common.BytesToHash(key) |
||||
val, ok := h.dirties[hash] |
||||
if ok { |
||||
return val, nil |
||||
} |
||||
return h.cleans[hash], nil |
||||
} |
||||
|
||||
// Update associates key with value in the trie.
|
||||
func (h *testHasher) Update(key, value []byte) error { |
||||
h.dirties[common.BytesToHash(key)] = common.CopyBytes(value) |
||||
return nil |
||||
} |
||||
|
||||
// Delete removes any existing value for key from the trie.
|
||||
func (h *testHasher) Delete(key []byte) error { |
||||
h.dirties[common.BytesToHash(key)] = nil |
||||
return nil |
||||
} |
||||
|
||||
// Commit computes the new hash of the states and returns the set with all
|
||||
// state changes.
|
||||
func (h *testHasher) Commit(collectLeaf bool) (common.Hash, *trienode.NodeSet) { |
||||
var ( |
||||
nodes = make(map[common.Hash][]byte) |
||||
set = trienode.NewNodeSet(h.owner) |
||||
) |
||||
for hash, val := range h.cleans { |
||||
nodes[hash] = val |
||||
} |
||||
for hash, val := range h.dirties { |
||||
nodes[hash] = val |
||||
if bytes.Equal(val, h.cleans[hash]) { |
||||
continue |
||||
} |
||||
if len(val) == 0 { |
||||
set.AddNode(hash.Bytes(), trienode.NewDeleted()) |
||||
} else { |
||||
set.AddNode(hash.Bytes(), trienode.New(crypto.Keccak256Hash(val), val)) |
||||
} |
||||
} |
||||
root, blob := hash(nodes) |
||||
|
||||
// Include the dirty root node as well.
|
||||
if root != types.EmptyRootHash && root != h.root { |
||||
set.AddNode(nil, trienode.New(root, blob)) |
||||
} |
||||
if root == types.EmptyRootHash && h.root != types.EmptyRootHash { |
||||
set.AddNode(nil, trienode.NewDeleted()) |
||||
} |
||||
return root, set |
||||
} |
||||
|
||||
// hash performs the hash computation upon the provided states.
|
||||
func hash(states map[common.Hash][]byte) (common.Hash, []byte) { |
||||
var hs []common.Hash |
||||
for hash := range states { |
||||
hs = append(hs, hash) |
||||
} |
||||
slices.SortFunc(hs, func(a, b common.Hash) bool { return a.Less(b) }) |
||||
|
||||
var input []byte |
||||
for _, hash := range hs { |
||||
if len(states[hash]) == 0 { |
||||
continue |
||||
} |
||||
input = append(input, hash.Bytes()...) |
||||
input = append(input, states[hash]...) |
||||
} |
||||
if len(input) == 0 { |
||||
return types.EmptyRootHash, nil |
||||
} |
||||
return crypto.Keccak256Hash(input), input |
||||
} |
||||
|
||||
type hashLoader struct { |
||||
accounts map[common.Hash][]byte |
||||
storages map[common.Hash]map[common.Hash][]byte |
||||
} |
||||
|
||||
func newHashLoader(accounts map[common.Hash][]byte, storages map[common.Hash]map[common.Hash][]byte) *hashLoader { |
||||
return &hashLoader{ |
||||
accounts: accounts, |
||||
storages: storages, |
||||
} |
||||
} |
||||
|
||||
// OpenTrie opens the main account trie.
|
||||
func (l *hashLoader) OpenTrie(root common.Hash) (triestate.Trie, error) { |
||||
return newTestHasher(common.Hash{}, root, l.accounts) |
||||
} |
||||
|
||||
// OpenStorageTrie opens the storage trie of an account.
|
||||
func (l *hashLoader) OpenStorageTrie(stateRoot common.Hash, addrHash, root common.Hash) (triestate.Trie, error) { |
||||
return newTestHasher(addrHash, root, l.storages[addrHash]) |
||||
} |
Loading…
Reference in new issue