diff --git a/core/blockchain.go b/core/blockchain.go
index 2f549806c..3952c31b6 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -713,7 +713,7 @@ func (bc *BlockChain) setHeadBeyondRoot(head uint64, time uint64, root common.Ha
if num+1 <= frozen {
// Truncate all relative data(header, total difficulty, body, receipt
// and canonical hash) from ancient store.
- if err := bc.db.TruncateHead(num); err != nil {
+ if _, err := bc.db.TruncateHead(num); err != nil {
log.Crit("Failed to truncate ancient data", "number", num, "err", err)
}
// Remove the hash <-> number mapping from the active store.
@@ -1136,7 +1136,7 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
size += int64(batch.ValueSize())
if err = batch.Write(); err != nil {
snapBlock := bc.CurrentSnapBlock().Number.Uint64()
- if err := bc.db.TruncateHead(snapBlock + 1); err != nil {
+ if _, err := bc.db.TruncateHead(snapBlock + 1); err != nil {
log.Error("Can't truncate ancient store after failed insert", "err", err)
}
return 0, err
@@ -1154,7 +1154,7 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
if !updateHead(blockChain[len(blockChain)-1]) {
// We end up here if the header chain has reorg'ed, and the blocks/receipts
// don't match the canonical chain.
- if err := bc.db.TruncateHead(previousSnapBlock + 1); err != nil {
+ if _, err := bc.db.TruncateHead(previousSnapBlock + 1); err != nil {
log.Error("Can't truncate ancient store after failed insert", "err", err)
}
return 0, errSideChainReceipts
diff --git a/core/rawdb/accessors_chain_test.go b/core/rawdb/accessors_chain_test.go
index 32e38a81c..beeec9f5a 100644
--- a/core/rawdb/accessors_chain_test.go
+++ b/core/rawdb/accessors_chain_test.go
@@ -85,7 +85,7 @@ func TestBodyStorage(t *testing.T) {
WriteBody(db, hash, 0, body)
if entry := ReadBody(db, hash, 0); entry == nil {
t.Fatalf("Stored body not found")
- } else if types.DeriveSha(types.Transactions(entry.Transactions), newHasher()) != types.DeriveSha(types.Transactions(body.Transactions), newHasher()) || types.CalcUncleHash(entry.Uncles) != types.CalcUncleHash(body.Uncles) {
+ } else if types.DeriveSha(types.Transactions(entry.Transactions), newTestHasher()) != types.DeriveSha(types.Transactions(body.Transactions), newTestHasher()) || types.CalcUncleHash(entry.Uncles) != types.CalcUncleHash(body.Uncles) {
t.Fatalf("Retrieved body mismatch: have %v, want %v", entry, body)
}
if entry := ReadBodyRLP(db, hash, 0); entry == nil {
@@ -139,7 +139,7 @@ func TestBlockStorage(t *testing.T) {
}
if entry := ReadBody(db, block.Hash(), block.NumberU64()); entry == nil {
t.Fatalf("Stored body not found")
- } else if types.DeriveSha(types.Transactions(entry.Transactions), newHasher()) != types.DeriveSha(block.Transactions(), newHasher()) || types.CalcUncleHash(entry.Uncles) != types.CalcUncleHash(block.Uncles()) {
+ } else if types.DeriveSha(types.Transactions(entry.Transactions), newTestHasher()) != types.DeriveSha(block.Transactions(), newTestHasher()) || types.CalcUncleHash(entry.Uncles) != types.CalcUncleHash(block.Uncles()) {
t.Fatalf("Retrieved body mismatch: have %v, want %v", entry, block.Body())
}
// Delete the block and verify the execution
diff --git a/core/rawdb/accessors_indexes_test.go b/core/rawdb/accessors_indexes_test.go
index 020075525..124389ba7 100644
--- a/core/rawdb/accessors_indexes_test.go
+++ b/core/rawdb/accessors_indexes_test.go
@@ -29,7 +29,7 @@ import (
"github.com/ethereum/go-ethereum/rlp"
)
-var newHasher = blocktest.NewHasher
+var newTestHasher = blocktest.NewHasher
// Tests that positional lookup metadata can be stored and retrieved.
func TestLookupStorage(t *testing.T) {
@@ -76,7 +76,7 @@ func TestLookupStorage(t *testing.T) {
tx3 := types.NewTransaction(3, common.BytesToAddress([]byte{0x33}), big.NewInt(333), 3333, big.NewInt(33333), []byte{0x33, 0x33, 0x33})
txs := []*types.Transaction{tx1, tx2, tx3}
- block := types.NewBlock(&types.Header{Number: big.NewInt(314)}, txs, nil, nil, newHasher())
+ block := types.NewBlock(&types.Header{Number: big.NewInt(314)}, txs, nil, nil, newTestHasher())
// Check that no transactions entries are in a pristine database
for i, tx := range txs {
diff --git a/core/rawdb/accessors_state.go b/core/rawdb/accessors_state.go
index 39900df23..9ce58e7d2 100644
--- a/core/rawdb/accessors_state.go
+++ b/core/rawdb/accessors_state.go
@@ -17,6 +17,8 @@
package rawdb
import (
+ "encoding/binary"
+
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
@@ -92,3 +94,173 @@ func DeleteCode(db ethdb.KeyValueWriter, hash common.Hash) {
log.Crit("Failed to delete contract code", "err", err)
}
}
+
+// ReadStateID retrieves the state id with the provided state root.
+func ReadStateID(db ethdb.KeyValueReader, root common.Hash) *uint64 {
+ data, err := db.Get(stateIDKey(root))
+ if err != nil || len(data) == 0 {
+ return nil
+ }
+ number := binary.BigEndian.Uint64(data)
+ return &number
+}
+
+// WriteStateID writes the provided state lookup to database.
+func WriteStateID(db ethdb.KeyValueWriter, root common.Hash, id uint64) {
+ var buff [8]byte
+ binary.BigEndian.PutUint64(buff[:], id)
+ if err := db.Put(stateIDKey(root), buff[:]); err != nil {
+ log.Crit("Failed to store state ID", "err", err)
+ }
+}
+
+// DeleteStateID deletes the specified state lookup from the database.
+func DeleteStateID(db ethdb.KeyValueWriter, root common.Hash) {
+ if err := db.Delete(stateIDKey(root)); err != nil {
+ log.Crit("Failed to delete state ID", "err", err)
+ }
+}
+
+// ReadPersistentStateID retrieves the id of the persistent state from the database.
+func ReadPersistentStateID(db ethdb.KeyValueReader) uint64 {
+ data, _ := db.Get(persistentStateIDKey)
+ if len(data) != 8 {
+ return 0
+ }
+ return binary.BigEndian.Uint64(data)
+}
+
+// WritePersistentStateID stores the id of the persistent state into database.
+func WritePersistentStateID(db ethdb.KeyValueWriter, number uint64) {
+ if err := db.Put(persistentStateIDKey, encodeBlockNumber(number)); err != nil {
+ log.Crit("Failed to store the persistent state ID", "err", err)
+ }
+}
+
+// ReadTrieJournal retrieves the serialized in-memory trie nodes of layers saved at
+// the last shutdown.
+func ReadTrieJournal(db ethdb.KeyValueReader) []byte {
+ data, _ := db.Get(trieJournalKey)
+ return data
+}
+
+// WriteTrieJournal stores the serialized in-memory trie nodes of layers to save at
+// shutdown.
+func WriteTrieJournal(db ethdb.KeyValueWriter, journal []byte) {
+ if err := db.Put(trieJournalKey, journal); err != nil {
+ log.Crit("Failed to store tries journal", "err", err)
+ }
+}
+
+// DeleteTrieJournal deletes the serialized in-memory trie nodes of layers saved at
+// the last shutdown.
+func DeleteTrieJournal(db ethdb.KeyValueWriter) {
+ if err := db.Delete(trieJournalKey); err != nil {
+ log.Crit("Failed to remove tries journal", "err", err)
+ }
+}
+
+// ReadStateHistoryMeta retrieves the metadata corresponding to the specified
+// state history. Compute the position of state history in freezer by minus
+// one since the id of first state history starts from one(zero for initial
+// state).
+func ReadStateHistoryMeta(db ethdb.AncientReaderOp, id uint64) []byte {
+ blob, err := db.Ancient(stateHistoryMeta, id-1)
+ if err != nil {
+ return nil
+ }
+ return blob
+}
+
+// ReadStateHistoryMetaList retrieves a batch of meta objects with the specified
+// start position and count. Compute the position of state history in freezer by
+// minus one since the id of first state history starts from one(zero for initial
+// state).
+func ReadStateHistoryMetaList(db ethdb.AncientReaderOp, start uint64, count uint64) ([][]byte, error) {
+ return db.AncientRange(stateHistoryMeta, start-1, count, 0)
+}
+
+// ReadStateAccountIndex retrieves the state root corresponding to the specified
+// state history. Compute the position of state history in freezer by minus one
+// since the id of first state history starts from one(zero for initial state).
+func ReadStateAccountIndex(db ethdb.AncientReaderOp, id uint64) []byte {
+ blob, err := db.Ancient(stateHistoryAccountIndex, id-1)
+ if err != nil {
+ return nil
+ }
+ return blob
+}
+
+// ReadStateStorageIndex retrieves the state root corresponding to the specified
+// state history. Compute the position of state history in freezer by minus one
+// since the id of first state history starts from one(zero for initial state).
+func ReadStateStorageIndex(db ethdb.AncientReaderOp, id uint64) []byte {
+ blob, err := db.Ancient(stateHistoryStorageIndex, id-1)
+ if err != nil {
+ return nil
+ }
+ return blob
+}
+
+// ReadStateAccountHistory retrieves the state root corresponding to the specified
+// state history. Compute the position of state history in freezer by minus one
+// since the id of first state history starts from one(zero for initial state).
+func ReadStateAccountHistory(db ethdb.AncientReaderOp, id uint64) []byte {
+ blob, err := db.Ancient(stateHistoryAccountData, id-1)
+ if err != nil {
+ return nil
+ }
+ return blob
+}
+
+// ReadStateStorageHistory retrieves the state root corresponding to the specified
+// state history. Compute the position of state history in freezer by minus one
+// since the id of first state history starts from one(zero for initial state).
+func ReadStateStorageHistory(db ethdb.AncientReaderOp, id uint64) []byte {
+ blob, err := db.Ancient(stateHistoryStorageData, id-1)
+ if err != nil {
+ return nil
+ }
+ return blob
+}
+
+// ReadStateHistory retrieves the state history from database with provided id.
+// Compute the position of state history in freezer by minus one since the id
+// of first state history starts from one(zero for initial state).
+func ReadStateHistory(db ethdb.AncientReaderOp, id uint64) ([]byte, []byte, []byte, []byte, []byte, error) {
+ meta, err := db.Ancient(stateHistoryMeta, id-1)
+ if err != nil {
+ return nil, nil, nil, nil, nil, err
+ }
+ accountIndex, err := db.Ancient(stateHistoryAccountIndex, id-1)
+ if err != nil {
+ return nil, nil, nil, nil, nil, err
+ }
+ storageIndex, err := db.Ancient(stateHistoryStorageIndex, id-1)
+ if err != nil {
+ return nil, nil, nil, nil, nil, err
+ }
+ accountData, err := db.Ancient(stateHistoryAccountData, id-1)
+ if err != nil {
+ return nil, nil, nil, nil, nil, err
+ }
+ storageData, err := db.Ancient(stateHistoryStorageData, id-1)
+ if err != nil {
+ return nil, nil, nil, nil, nil, err
+ }
+ return meta, accountIndex, storageIndex, accountData, storageData, nil
+}
+
+// WriteStateHistory writes the provided state history to database. Compute the
+// position of state history in freezer by minus one since the id of first state
+// history starts from one(zero for initial state).
+func WriteStateHistory(db ethdb.AncientWriter, id uint64, meta []byte, accountIndex []byte, storageIndex []byte, accounts []byte, storages []byte) {
+ db.ModifyAncients(func(op ethdb.AncientWriteOp) error {
+ op.AppendRaw(stateHistoryMeta, id-1, meta)
+ op.AppendRaw(stateHistoryAccountIndex, id-1, accountIndex)
+ op.AppendRaw(stateHistoryStorageIndex, id-1, storageIndex)
+ op.AppendRaw(stateHistoryAccountData, id-1, accounts)
+ op.AppendRaw(stateHistoryStorageData, id-1, storages)
+ return nil
+ })
+}
diff --git a/core/rawdb/accessors_trie.go b/core/rawdb/accessors_trie.go
index e24021302..12f1ecdf8 100644
--- a/core/rawdb/accessors_trie.go
+++ b/core/rawdb/accessors_trie.go
@@ -46,21 +46,23 @@ const HashScheme = "hashScheme"
// on extra state diffs to survive deep reorg.
const PathScheme = "pathScheme"
-// nodeHasher used to derive the hash of trie node.
-type nodeHasher struct{ sha crypto.KeccakState }
+// 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 &nodeHasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} },
+ New: func() interface{} { return &hasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} },
}
-func newNodeHasher() *nodeHasher { return hasherPool.Get().(*nodeHasher) }
-func returnHasherToPool(h *nodeHasher) { hasherPool.Put(h) }
+func newHasher() *hasher {
+ return hasherPool.Get().(*hasher)
+}
-func (h *nodeHasher) hashData(data []byte) (n common.Hash) {
- h.sha.Reset()
- h.sha.Write(data)
- h.sha.Read(n[:])
- return n
+func (h *hasher) hash(data []byte) common.Hash {
+ return crypto.HashData(h.sha, data)
+}
+
+func (h *hasher) release() {
+ hasherPool.Put(h)
}
// ReadAccountTrieNode retrieves the account trie node and the associated node
@@ -70,9 +72,9 @@ func ReadAccountTrieNode(db ethdb.KeyValueReader, path []byte) ([]byte, common.H
if err != nil {
return nil, common.Hash{}
}
- hasher := newNodeHasher()
- defer returnHasherToPool(hasher)
- return data, hasher.hashData(data)
+ h := newHasher()
+ defer h.release()
+ return data, h.hash(data)
}
// HasAccountTrieNode checks the account trie node presence with the specified
@@ -82,9 +84,9 @@ func HasAccountTrieNode(db ethdb.KeyValueReader, path []byte, hash common.Hash)
if err != nil {
return false
}
- hasher := newNodeHasher()
- defer returnHasherToPool(hasher)
- return hasher.hashData(data) == hash
+ h := newHasher()
+ defer h.release()
+ return h.hash(data) == hash
}
// WriteAccountTrieNode writes the provided account trie node into database.
@@ -108,9 +110,9 @@ func ReadStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path
if err != nil {
return nil, common.Hash{}
}
- hasher := newNodeHasher()
- defer returnHasherToPool(hasher)
- return data, hasher.hashData(data)
+ h := newHasher()
+ defer h.release()
+ return data, h.hash(data)
}
// HasStorageTrieNode checks the storage trie node presence with the provided
@@ -120,9 +122,9 @@ func HasStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path [
if err != nil {
return false
}
- hasher := newNodeHasher()
- defer returnHasherToPool(hasher)
- return hasher.hashData(data) == hash
+ h := newHasher()
+ defer h.release()
+ return h.hash(data) == hash
}
// WriteStorageTrieNode writes the provided storage trie node into database.
diff --git a/core/rawdb/ancient_scheme.go b/core/rawdb/ancient_scheme.go
index b0428c5f5..c1cd7fda1 100644
--- a/core/rawdb/ancient_scheme.go
+++ b/core/rawdb/ancient_scheme.go
@@ -16,6 +16,8 @@
package rawdb
+import "path/filepath"
+
// The list of table names of chain freezer.
const (
// ChainFreezerHeaderTable indicates the name of the freezer header table.
@@ -44,10 +46,36 @@ var chainFreezerNoSnappy = map[string]bool{
ChainFreezerDifficultyTable: true,
}
+const (
+ // stateHistoryTableSize defines the maximum size of freezer data files.
+ stateHistoryTableSize = 2 * 1000 * 1000 * 1000
+
+ // stateHistoryAccountIndex indicates the name of the freezer state history table.
+ stateHistoryMeta = "history.meta"
+ stateHistoryAccountIndex = "account.index"
+ stateHistoryStorageIndex = "storage.index"
+ stateHistoryAccountData = "account.data"
+ stateHistoryStorageData = "storage.data"
+)
+
+var stateHistoryFreezerNoSnappy = map[string]bool{
+ stateHistoryMeta: true,
+ stateHistoryAccountIndex: false,
+ stateHistoryStorageIndex: false,
+ stateHistoryAccountData: false,
+ stateHistoryStorageData: false,
+}
+
// The list of identifiers of ancient stores.
var (
chainFreezerName = "chain" // the folder name of chain segment ancient store.
+ stateFreezerName = "state" // the folder name of reverse diff ancient store.
)
// freezers the collections of all builtin freezers.
-var freezers = []string{chainFreezerName}
+var freezers = []string{chainFreezerName, stateFreezerName}
+
+// NewStateHistoryFreezer initializes the freezer for state history.
+func NewStateHistoryFreezer(ancientDir string, readOnly bool) (*ResettableFreezer, error) {
+ return NewResettableFreezer(filepath.Join(ancientDir, stateFreezerName), "eth/db/state", readOnly, stateHistoryTableSize, stateHistoryFreezerNoSnappy)
+}
diff --git a/core/rawdb/chain_iterator_test.go b/core/rawdb/chain_iterator_test.go
index fd405e9d6..9e18c8605 100644
--- a/core/rawdb/chain_iterator_test.go
+++ b/core/rawdb/chain_iterator_test.go
@@ -34,7 +34,7 @@ func TestChainIterator(t *testing.T) {
var block *types.Block
var txs []*types.Transaction
to := common.BytesToAddress([]byte{0x11})
- block = types.NewBlock(&types.Header{Number: big.NewInt(int64(0))}, nil, nil, nil, newHasher()) // Empty genesis block
+ block = types.NewBlock(&types.Header{Number: big.NewInt(int64(0))}, nil, nil, nil, newTestHasher()) // Empty genesis block
WriteBlock(chainDb, block)
WriteCanonicalHash(chainDb, block.Hash(), block.NumberU64())
for i := uint64(1); i <= 10; i++ {
@@ -60,7 +60,7 @@ func TestChainIterator(t *testing.T) {
})
}
txs = append(txs, tx)
- block = types.NewBlock(&types.Header{Number: big.NewInt(int64(i))}, []*types.Transaction{tx}, nil, nil, newHasher())
+ block = types.NewBlock(&types.Header{Number: big.NewInt(int64(i))}, []*types.Transaction{tx}, nil, nil, newTestHasher())
WriteBlock(chainDb, block)
WriteCanonicalHash(chainDb, block.Hash(), block.NumberU64())
}
@@ -113,7 +113,7 @@ func TestIndexTransactions(t *testing.T) {
to := common.BytesToAddress([]byte{0x11})
// Write empty genesis block
- block = types.NewBlock(&types.Header{Number: big.NewInt(int64(0))}, nil, nil, nil, newHasher())
+ block = types.NewBlock(&types.Header{Number: big.NewInt(int64(0))}, nil, nil, nil, newTestHasher())
WriteBlock(chainDb, block)
WriteCanonicalHash(chainDb, block.Hash(), block.NumberU64())
@@ -140,7 +140,7 @@ func TestIndexTransactions(t *testing.T) {
})
}
txs = append(txs, tx)
- block = types.NewBlock(&types.Header{Number: big.NewInt(int64(i))}, []*types.Transaction{tx}, nil, nil, newHasher())
+ block = types.NewBlock(&types.Header{Number: big.NewInt(int64(i))}, []*types.Transaction{tx}, nil, nil, newTestHasher())
WriteBlock(chainDb, block)
WriteCanonicalHash(chainDb, block.Hash(), block.NumberU64())
}
diff --git a/core/rawdb/database.go b/core/rawdb/database.go
index e864bcb2e..7bc910906 100644
--- a/core/rawdb/database.go
+++ b/core/rawdb/database.go
@@ -123,13 +123,13 @@ func (db *nofreezedb) ModifyAncients(func(ethdb.AncientWriteOp) error) (int64, e
}
// TruncateHead returns an error as we don't have a backing chain freezer.
-func (db *nofreezedb) TruncateHead(items uint64) error {
- return errNotSupported
+func (db *nofreezedb) TruncateHead(items uint64) (uint64, error) {
+ return 0, errNotSupported
}
// TruncateTail returns an error as we don't have a backing chain freezer.
-func (db *nofreezedb) TruncateTail(items uint64) error {
- return errNotSupported
+func (db *nofreezedb) TruncateTail(items uint64) (uint64, error) {
+ return 0, errNotSupported
}
// Sync returns an error as we don't have a backing chain freezer.
diff --git a/core/rawdb/freezer.go b/core/rawdb/freezer.go
index 2846858e0..a9fe23432 100644
--- a/core/rawdb/freezer.go
+++ b/core/rawdb/freezer.go
@@ -275,43 +275,46 @@ func (f *Freezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (writeSize
}
// TruncateHead discards any recent data above the provided threshold number.
-func (f *Freezer) TruncateHead(items uint64) error {
+// It returns the previous head number.
+func (f *Freezer) TruncateHead(items uint64) (uint64, error) {
if f.readonly {
- return errReadOnly
+ return 0, errReadOnly
}
f.writeLock.Lock()
defer f.writeLock.Unlock()
- if f.frozen.Load() <= items {
- return nil
+ oitems := f.frozen.Load()
+ if oitems <= items {
+ return oitems, nil
}
for _, table := range f.tables {
if err := table.truncateHead(items); err != nil {
- return err
+ return 0, err
}
}
f.frozen.Store(items)
- return nil
+ return oitems, nil
}
// TruncateTail discards any recent data below the provided threshold number.
-func (f *Freezer) TruncateTail(tail uint64) error {
+func (f *Freezer) TruncateTail(tail uint64) (uint64, error) {
if f.readonly {
- return errReadOnly
+ return 0, errReadOnly
}
f.writeLock.Lock()
defer f.writeLock.Unlock()
- if f.tail.Load() >= tail {
- return nil
+ old := f.tail.Load()
+ if old >= tail {
+ return old, nil
}
for _, table := range f.tables {
if err := table.truncateTail(tail); err != nil {
- return err
+ return 0, err
}
}
f.tail.Store(tail)
- return nil
+ return old, nil
}
// Sync flushes all data tables to disk.
diff --git a/core/rawdb/freezer_resettable.go b/core/rawdb/freezer_resettable.go
index f9a56c6de..0a3892bcd 100644
--- a/core/rawdb/freezer_resettable.go
+++ b/core/rawdb/freezer_resettable.go
@@ -170,7 +170,8 @@ func (f *ResettableFreezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error)
}
// TruncateHead discards any recent data above the provided threshold number.
-func (f *ResettableFreezer) TruncateHead(items uint64) error {
+// It returns the previous head number.
+func (f *ResettableFreezer) TruncateHead(items uint64) (uint64, error) {
f.lock.RLock()
defer f.lock.RUnlock()
@@ -178,7 +179,8 @@ func (f *ResettableFreezer) TruncateHead(items uint64) error {
}
// TruncateTail discards any recent data below the provided threshold number.
-func (f *ResettableFreezer) TruncateTail(tail uint64) error {
+// It returns the previous value
+func (f *ResettableFreezer) TruncateTail(tail uint64) (uint64, error) {
f.lock.RLock()
defer f.lock.RUnlock()
diff --git a/core/rawdb/freezer_test.go b/core/rawdb/freezer_test.go
index 630c9029b..96d24cc94 100644
--- a/core/rawdb/freezer_test.go
+++ b/core/rawdb/freezer_test.go
@@ -192,7 +192,7 @@ func TestFreezerConcurrentModifyTruncate(t *testing.T) {
for i := 0; i < 10; i++ {
// First reset and write 100 items.
- if err := f.TruncateHead(0); err != nil {
+ if _, err := f.TruncateHead(0); err != nil {
t.Fatal("truncate failed:", err)
}
_, err := f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
@@ -227,7 +227,7 @@ func TestFreezerConcurrentModifyTruncate(t *testing.T) {
wg.Done()
}()
go func() {
- truncateErr = f.TruncateHead(10)
+ _, truncateErr = f.TruncateHead(10)
wg.Done()
}()
go func() {
diff --git a/core/rawdb/schema.go b/core/rawdb/schema.go
index 18722ed5d..940ce0154 100644
--- a/core/rawdb/schema.go
+++ b/core/rawdb/schema.go
@@ -43,6 +43,9 @@ var (
// headFinalizedBlockKey tracks the latest known finalized block hash.
headFinalizedBlockKey = []byte("LastFinalized")
+ // persistentStateIDKey tracks the id of latest stored state(for path-based only).
+ persistentStateIDKey = []byte("LastStateID")
+
// lastPivotKey tracks the last pivot block used by fast sync (to reenable on sethead).
lastPivotKey = []byte("LastPivot")
@@ -70,6 +73,9 @@ var (
// skeletonSyncStatusKey tracks the skeleton sync status across restarts.
skeletonSyncStatusKey = []byte("SkeletonSyncStatus")
+ // trieJournalKey tracks the in-memory trie node layers across restarts.
+ trieJournalKey = []byte("TrieJournal")
+
// txIndexTailKey tracks the oldest block whose transactions have been indexed.
txIndexTailKey = []byte("TransactionIndexTail")
@@ -104,6 +110,7 @@ var (
// Path-based storage scheme of merkle patricia trie.
trieNodeAccountPrefix = []byte("A") // trieNodeAccountPrefix + hexPath -> trie node
trieNodeStoragePrefix = []byte("O") // trieNodeStoragePrefix + accountHash + hexPath -> trie node
+ stateIDPrefix = []byte("L") // stateIDPrefix + state root -> state id
PreimagePrefix = []byte("secure-key-") // PreimagePrefix + hash -> preimage
configPrefix = []byte("ethereum-config-") // config prefix for the db
@@ -240,6 +247,11 @@ func genesisStateSpecKey(hash common.Hash) []byte {
return append(genesisPrefix, hash.Bytes()...)
}
+// stateIDKey = stateIDPrefix + root (32 bytes)
+func stateIDKey(root common.Hash) []byte {
+ return append(stateIDPrefix, root.Bytes()...)
+}
+
// accountTrieNodeKey = trieNodeAccountPrefix + nodePath.
func accountTrieNodeKey(path []byte) []byte {
return append(trieNodeAccountPrefix, path...)
diff --git a/core/rawdb/table.go b/core/rawdb/table.go
index 6d6fa0555..1895f61da 100644
--- a/core/rawdb/table.go
+++ b/core/rawdb/table.go
@@ -97,13 +97,13 @@ func (t *table) ReadAncients(fn func(reader ethdb.AncientReaderOp) error) (err e
// TruncateHead is a noop passthrough that just forwards the request to the underlying
// database.
-func (t *table) TruncateHead(items uint64) error {
+func (t *table) TruncateHead(items uint64) (uint64, error) {
return t.db.TruncateHead(items)
}
// TruncateTail is a noop passthrough that just forwards the request to the underlying
// database.
-func (t *table) TruncateTail(items uint64) error {
+func (t *table) TruncateTail(items uint64) (uint64, error) {
return t.db.TruncateTail(items)
}
diff --git a/core/state/statedb.go b/core/state/statedb.go
index 8321128dc..fdaeacc6b 100644
--- a/core/state/statedb.go
+++ b/core/state/statedb.go
@@ -1054,8 +1054,8 @@ func (s *StateDB) deleteStorage(addr common.Address, addrHash common.Hash, root
if it.Hash() == (common.Hash{}) {
continue
}
- nodeSize += common.StorageSize(len(it.Path()) + len(it.NodeBlob()))
- set.AddNode(it.Path(), trienode.NewWithPrev(common.Hash{}, nil, it.NodeBlob()))
+ nodeSize += common.StorageSize(len(it.Path()))
+ set.AddNode(it.Path(), trienode.NewDeleted())
}
if err := it.Error(); err != nil {
return false, nil, nil, err
@@ -1274,12 +1274,7 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool) (common.Hash, er
}
if root != origin {
start := time.Now()
- set := &triestate.Set{
- Accounts: s.accountsOrigin,
- Storages: s.storagesOrigin,
- Incomplete: incomplete,
- }
- if err := s.db.TrieDB().Update(root, origin, block, nodes, set); err != nil {
+ if err := s.db.TrieDB().Update(root, origin, block, nodes, triestate.New(s.accountsOrigin, s.storagesOrigin, incomplete)); err != nil {
return common.Hash{}, err
}
s.originalRoot = root
diff --git a/core/types/hashes.go b/core/types/hashes.go
index 3bad430be..3a787aa13 100644
--- a/core/types/hashes.go
+++ b/core/types/hashes.go
@@ -19,6 +19,7 @@ package types
import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto"
+ "github.com/ethereum/go-ethereum/log"
)
var (
@@ -40,3 +41,13 @@ var (
// EmptyWithdrawalsHash is the known hash of the empty withdrawal set.
EmptyWithdrawalsHash = common.HexToHash("56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421")
)
+
+// TrieRootHash returns the hash itself if it's non-empty or the predefined
+// emptyHash one instead.
+func TrieRootHash(hash common.Hash) common.Hash {
+ if hash == (common.Hash{}) {
+ log.Error("Zero trie root hash!")
+ return EmptyRootHash
+ }
+ return hash
+}
diff --git a/ethdb/database.go b/ethdb/database.go
index 3fe6dfa73..4d4817daf 100644
--- a/ethdb/database.go
+++ b/ethdb/database.go
@@ -114,14 +114,14 @@ type AncientWriter interface {
// TruncateHead discards all but the first n ancient data from the ancient store.
// After the truncation, the latest item can be accessed it item_n-1(start from 0).
- TruncateHead(n uint64) error
+ TruncateHead(n uint64) (uint64, error)
// TruncateTail discards the first n ancient data from the ancient store. The already
// deleted items are ignored. After the truncation, the earliest item can be accessed
// is item_n(start from 0). The deleted items may not be removed from the ancient store
// immediately, but only when the accumulated deleted data reach the threshold then
// will be removed all together.
- TruncateTail(n uint64) error
+ TruncateTail(n uint64) (uint64, error)
// Sync flushes all in-memory ancient store data to disk.
Sync() error
diff --git a/ethdb/remotedb/remotedb.go b/ethdb/remotedb/remotedb.go
index 9ce657d78..c1c803caf 100644
--- a/ethdb/remotedb/remotedb.go
+++ b/ethdb/remotedb/remotedb.go
@@ -98,11 +98,11 @@ func (db *Database) ModifyAncients(f func(ethdb.AncientWriteOp) error) (int64, e
panic("not supported")
}
-func (db *Database) TruncateHead(n uint64) error {
+func (db *Database) TruncateHead(n uint64) (uint64, error) {
panic("not supported")
}
-func (db *Database) TruncateTail(n uint64) error {
+func (db *Database) TruncateTail(n uint64) (uint64, error) {
panic("not supported")
}
diff --git a/trie/committer.go b/trie/committer.go
index e825287fd..92163cdb3 100644
--- a/trie/committer.go
+++ b/trie/committer.go
@@ -131,22 +131,15 @@ func (c *committer) store(path []byte, n node) node {
// The node is embedded in its parent, in other words, this node
// will not be stored in the database independently, mark it as
// deleted only if the node was existent in database before.
- prev, ok := c.tracer.accessList[string(path)]
+ _, ok := c.tracer.accessList[string(path)]
if ok {
- c.nodes.AddNode(path, trienode.NewWithPrev(common.Hash{}, nil, prev))
+ c.nodes.AddNode(path, trienode.NewDeleted())
}
return n
}
// Collect the dirty node to nodeset for return.
- var (
- nhash = common.BytesToHash(hash)
- node = trienode.NewWithPrev(
- nhash,
- nodeToBytes(n),
- c.tracer.accessList[string(path)],
- )
- )
- c.nodes.AddNode(path, node)
+ nhash := common.BytesToHash(hash)
+ c.nodes.AddNode(path, trienode.New(nhash, nodeToBytes(n)))
// Collect the corresponding leaf node if it's required. We don't check
// full node since it's impossible to store value in fullNode. The key
diff --git a/trie/database.go b/trie/database.go
index 08ef5d07e..49a884fd7 100644
--- a/trie/database.go
+++ b/trie/database.go
@@ -19,18 +19,19 @@ package trie
import (
"errors"
- "github.com/VictoriaMetrics/fastcache"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/trie/triedb/hashdb"
+ "github.com/ethereum/go-ethereum/trie/triedb/pathdb"
"github.com/ethereum/go-ethereum/trie/trienode"
"github.com/ethereum/go-ethereum/trie/triestate"
)
// Config defines all necessary options for database.
type Config struct {
- Cache int // Memory allowance (MB) to use for caching trie nodes in memory
- Preimages bool // Flag whether the preimage of trie key is recorded
+ Cache int // Memory allowance (MB) to use for caching trie nodes in memory
+ Preimages bool // Flag whether the preimage of trie key is recorded
+ PathDB *pathdb.Config // Configs for experimental path-based scheme, not used yet.
// Testing hooks
OnCommit func(states *triestate.Set) // Hook invoked when commit is performed
@@ -53,7 +54,10 @@ type backend interface {
// Update performs a state transition by committing dirty nodes contained
// in the given set in order to update state from the specified parent to
// the specified root.
- Update(root common.Hash, parent common.Hash, nodes *trienode.MergedNodeSet) error
+ //
+ // The passed in maps(nodes, states) will be retained to avoid copying
+ // everything. Therefore, these maps must not be changed afterwards.
+ Update(root common.Hash, parent common.Hash, block uint64, nodes *trienode.MergedNodeSet, states *triestate.Set) error
// Commit writes all relevant trie nodes belonging to the specified state
// to disk. Report specifies whether logs will be displayed in info level.
@@ -67,20 +71,15 @@ type backend interface {
// types of node backend as an entrypoint. It's responsible for all interactions
// relevant with trie nodes and node preimages.
type Database struct {
- config *Config // Configuration for trie database
- diskdb ethdb.Database // Persistent database to store the snapshot
- cleans *fastcache.Cache // Megabytes permitted using for read caches
- preimages *preimageStore // The store for caching preimages
- backend backend // The backend for managing trie nodes
+ config *Config // Configuration for trie database
+ diskdb ethdb.Database // Persistent database to store the snapshot
+ preimages *preimageStore // The store for caching preimages
+ backend backend // The backend for managing trie nodes
}
// prepare initializes the database with provided configs, but the
// database backend is still left as nil.
func prepare(diskdb ethdb.Database, config *Config) *Database {
- var cleans *fastcache.Cache
- if config != nil && config.Cache > 0 {
- cleans = fastcache.New(config.Cache * 1024 * 1024)
- }
var preimages *preimageStore
if config != nil && config.Preimages {
preimages = newPreimageStore(diskdb)
@@ -88,7 +87,6 @@ func prepare(diskdb ethdb.Database, config *Config) *Database {
return &Database{
config: config,
diskdb: diskdb,
- cleans: cleans,
preimages: preimages,
}
}
@@ -103,21 +101,34 @@ func NewDatabase(diskdb ethdb.Database) *Database {
// The path-based scheme is not activated yet, always initialized with legacy
// hash-based scheme by default.
func NewDatabaseWithConfig(diskdb ethdb.Database, config *Config) *Database {
+ var cleans int
+ if config != nil && config.Cache != 0 {
+ cleans = config.Cache * 1024 * 1024
+ }
db := prepare(diskdb, config)
- db.backend = hashdb.New(diskdb, db.cleans, mptResolver{})
+ db.backend = hashdb.New(diskdb, cleans, mptResolver{})
return db
}
// Reader returns a reader for accessing all trie nodes with provided state root.
// An error will be returned if the requested state is not available.
func (db *Database) Reader(blockRoot common.Hash) (Reader, error) {
- return db.backend.(*hashdb.Database).Reader(blockRoot)
+ switch b := db.backend.(type) {
+ case *hashdb.Database:
+ return b.Reader(blockRoot)
+ case *pathdb.Database:
+ return b.Reader(blockRoot)
+ }
+ return nil, errors.New("unknown backend")
}
// Update performs a state transition by committing dirty nodes contained in the
// given set in order to update state from the specified parent to the specified
// root. The held pre-images accumulated up to this point will be flushed in case
// the size exceeds the threshold.
+//
+// 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, parent common.Hash, block uint64, nodes *trienode.MergedNodeSet, states *triestate.Set) error {
if db.config != nil && db.config.OnCommit != nil {
db.config.OnCommit(states)
@@ -125,7 +136,7 @@ func (db *Database) Update(root common.Hash, parent common.Hash, block uint64, n
if db.preimages != nil {
db.preimages.commit(false)
}
- return db.backend.Update(root, parent, nodes)
+ return db.backend.Update(root, parent, block, nodes, states)
}
// Commit iterates over all the children of a particular node, writes them out
diff --git a/trie/database_test.go b/trie/database_test.go
index cad462f73..ed43a81e5 100644
--- a/trie/database_test.go
+++ b/trie/database_test.go
@@ -20,16 +20,16 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/trie/triedb/hashdb"
+ "github.com/ethereum/go-ethereum/trie/triedb/pathdb"
)
// newTestDatabase initializes the trie database with specified scheme.
func newTestDatabase(diskdb ethdb.Database, scheme string) *Database {
db := prepare(diskdb, nil)
if scheme == rawdb.HashScheme {
- db.backend = hashdb.New(diskdb, db.cleans, mptResolver{})
+ db.backend = hashdb.New(diskdb, 0, mptResolver{})
+ } else {
+ db.backend = pathdb.New(diskdb, &pathdb.Config{}) // disable clean/dirty cache
}
- //} else {
- // db.backend = snap.New(diskdb, db.cleans, nil)
- //}
return db
}
diff --git a/trie/iterator_test.go b/trie/iterator_test.go
index abe9cf1b1..bf20c00c6 100644
--- a/trie/iterator_test.go
+++ b/trie/iterator_test.go
@@ -130,7 +130,7 @@ type iterationElement struct {
// Tests that the node iterator indeed walks over the entire database contents.
func TestNodeIteratorCoverage(t *testing.T) {
testNodeIteratorCoverage(t, rawdb.HashScheme)
- //testNodeIteratorCoverage(t, rawdb.PathScheme)
+ testNodeIteratorCoverage(t, rawdb.PathScheme)
}
func testNodeIteratorCoverage(t *testing.T, scheme string) {
@@ -355,8 +355,8 @@ func TestIteratorNoDups(t *testing.T) {
func TestIteratorContinueAfterError(t *testing.T) {
testIteratorContinueAfterError(t, false, rawdb.HashScheme)
testIteratorContinueAfterError(t, true, rawdb.HashScheme)
- // testIteratorContinueAfterError(t, false, rawdb.PathScheme)
- // testIteratorContinueAfterError(t, true, rawdb.PathScheme)
+ testIteratorContinueAfterError(t, false, rawdb.PathScheme)
+ testIteratorContinueAfterError(t, true, rawdb.PathScheme)
}
func testIteratorContinueAfterError(t *testing.T, memonly bool, scheme string) {
@@ -461,8 +461,8 @@ func testIteratorContinueAfterError(t *testing.T, memonly bool, scheme string) {
func TestIteratorContinueAfterSeekError(t *testing.T) {
testIteratorContinueAfterSeekError(t, false, rawdb.HashScheme)
testIteratorContinueAfterSeekError(t, true, rawdb.HashScheme)
- // testIteratorContinueAfterSeekError(t, false, rawdb.PathScheme)
- // testIteratorContinueAfterSeekError(t, true, rawdb.PathScheme)
+ testIteratorContinueAfterSeekError(t, false, rawdb.PathScheme)
+ testIteratorContinueAfterSeekError(t, true, rawdb.PathScheme)
}
func testIteratorContinueAfterSeekError(t *testing.T, memonly bool, scheme string) {
@@ -534,7 +534,7 @@ func checkIteratorNoDups(t *testing.T, it NodeIterator, seen map[string]bool) in
func TestIteratorNodeBlob(t *testing.T) {
testIteratorNodeBlob(t, rawdb.HashScheme)
- //testIteratorNodeBlob(t, rawdb.PathScheme)
+ testIteratorNodeBlob(t, rawdb.PathScheme)
}
type loggingDb struct {
diff --git a/trie/sync_test.go b/trie/sync_test.go
index 589e2858f..b6fe8d84a 100644
--- a/trie/sync_test.go
+++ b/trie/sync_test.go
@@ -111,16 +111,16 @@ type trieElement struct {
func TestEmptySync(t *testing.T) {
dbA := NewDatabase(rawdb.NewMemoryDatabase())
dbB := NewDatabase(rawdb.NewMemoryDatabase())
- //dbC := newTestDatabase(rawdb.NewMemoryDatabase(), rawdb.PathScheme)
- //dbD := newTestDatabase(rawdb.NewMemoryDatabase(), rawdb.PathScheme)
+ dbC := newTestDatabase(rawdb.NewMemoryDatabase(), rawdb.PathScheme)
+ dbD := newTestDatabase(rawdb.NewMemoryDatabase(), rawdb.PathScheme)
emptyA := NewEmpty(dbA)
emptyB, _ := New(TrieID(types.EmptyRootHash), dbB)
- //emptyC := NewEmpty(dbC)
- //emptyD, _ := New(TrieID(types.EmptyRootHash), dbD)
+ emptyC := NewEmpty(dbC)
+ emptyD, _ := New(TrieID(types.EmptyRootHash), dbD)
- for i, trie := range []*Trie{emptyA, emptyB /*emptyC, emptyD*/} {
- sync := NewSync(trie.Hash(), memorydb.New(), nil, []*Database{dbA, dbB /*dbC, dbD*/}[i].Scheme())
+ for i, trie := range []*Trie{emptyA, emptyB, emptyC, emptyD} {
+ sync := NewSync(trie.Hash(), memorydb.New(), nil, []*Database{dbA, dbB, dbC, dbD}[i].Scheme())
if paths, nodes, codes := sync.Missing(1); len(paths) != 0 || len(nodes) != 0 || len(codes) != 0 {
t.Errorf("test %d: content requested for empty trie: %v, %v, %v", i, paths, nodes, codes)
}
@@ -134,10 +134,10 @@ func TestIterativeSync(t *testing.T) {
testIterativeSync(t, 100, false, rawdb.HashScheme)
testIterativeSync(t, 1, true, rawdb.HashScheme)
testIterativeSync(t, 100, true, rawdb.HashScheme)
- // testIterativeSync(t, 1, false, rawdb.PathScheme)
- // testIterativeSync(t, 100, false, rawdb.PathScheme)
- // testIterativeSync(t, 1, true, rawdb.PathScheme)
- // testIterativeSync(t, 100, true, rawdb.PathScheme)
+ testIterativeSync(t, 1, false, rawdb.PathScheme)
+ testIterativeSync(t, 100, false, rawdb.PathScheme)
+ testIterativeSync(t, 1, true, rawdb.PathScheme)
+ testIterativeSync(t, 100, true, rawdb.PathScheme)
}
func testIterativeSync(t *testing.T, count int, bypath bool, scheme string) {
@@ -212,7 +212,7 @@ func testIterativeSync(t *testing.T, count int, bypath bool, scheme string) {
// partial results are returned, and the others sent only later.
func TestIterativeDelayedSync(t *testing.T) {
testIterativeDelayedSync(t, rawdb.HashScheme)
- //testIterativeDelayedSync(t, rawdb.PathScheme)
+ testIterativeDelayedSync(t, rawdb.PathScheme)
}
func testIterativeDelayedSync(t *testing.T, scheme string) {
@@ -280,8 +280,8 @@ func testIterativeDelayedSync(t *testing.T, scheme string) {
func TestIterativeRandomSyncIndividual(t *testing.T) {
testIterativeRandomSync(t, 1, rawdb.HashScheme)
testIterativeRandomSync(t, 100, rawdb.HashScheme)
- // testIterativeRandomSync(t, 1, rawdb.PathScheme)
- // testIterativeRandomSync(t, 100, rawdb.PathScheme)
+ testIterativeRandomSync(t, 1, rawdb.PathScheme)
+ testIterativeRandomSync(t, 100, rawdb.PathScheme)
}
func testIterativeRandomSync(t *testing.T, count int, scheme string) {
@@ -348,7 +348,7 @@ func testIterativeRandomSync(t *testing.T, count int, scheme string) {
// partial results are returned (Even those randomly), others sent only later.
func TestIterativeRandomDelayedSync(t *testing.T) {
testIterativeRandomDelayedSync(t, rawdb.HashScheme)
- // testIterativeRandomDelayedSync(t, rawdb.PathScheme)
+ testIterativeRandomDelayedSync(t, rawdb.PathScheme)
}
func testIterativeRandomDelayedSync(t *testing.T, scheme string) {
@@ -420,7 +420,7 @@ func testIterativeRandomDelayedSync(t *testing.T, scheme string) {
// have such references.
func TestDuplicateAvoidanceSync(t *testing.T) {
testDuplicateAvoidanceSync(t, rawdb.HashScheme)
- // testDuplicateAvoidanceSync(t, rawdb.PathScheme)
+ testDuplicateAvoidanceSync(t, rawdb.PathScheme)
}
func testDuplicateAvoidanceSync(t *testing.T, scheme string) {
@@ -491,12 +491,10 @@ func testDuplicateAvoidanceSync(t *testing.T, scheme string) {
// the database.
func TestIncompleteSyncHash(t *testing.T) {
testIncompleteSync(t, rawdb.HashScheme)
- // testIncompleteSync(t, rawdb.PathScheme)
+ testIncompleteSync(t, rawdb.PathScheme)
}
func testIncompleteSync(t *testing.T, scheme string) {
- t.Parallel()
-
// Create a random trie to copy
_, srcDb, srcTrie, _ := makeTestTrie(scheme)
@@ -582,7 +580,7 @@ func testIncompleteSync(t *testing.T, scheme string) {
// depth.
func TestSyncOrdering(t *testing.T) {
testSyncOrdering(t, rawdb.HashScheme)
- // testSyncOrdering(t, rawdb.PathScheme)
+ testSyncOrdering(t, rawdb.PathScheme)
}
func testSyncOrdering(t *testing.T, scheme string) {
@@ -716,7 +714,7 @@ func syncWith(t *testing.T, root common.Hash, db ethdb.Database, srcDb *Database
// states synced in the last cycle.
func TestSyncMovingTarget(t *testing.T) {
testSyncMovingTarget(t, rawdb.HashScheme)
- // testSyncMovingTarget(t, rawdb.PathScheme)
+ testSyncMovingTarget(t, rawdb.PathScheme)
}
func testSyncMovingTarget(t *testing.T, scheme string) {
diff --git a/trie/testutil/utils.go b/trie/testutil/utils.go
new file mode 100644
index 000000000..a75d0431b
--- /dev/null
+++ b/trie/testutil/utils.go
@@ -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 .
+
+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)
+}
diff --git a/trie/tracer.go b/trie/tracer.go
index 2b5de8ec4..5786af4d3 100644
--- a/trie/tracer.go
+++ b/trie/tracer.go
@@ -18,7 +18,6 @@ package trie
import (
"github.com/ethereum/go-ethereum/common"
- "github.com/ethereum/go-ethereum/trie/trienode"
)
// tracer tracks the changes of trie nodes. During the trie operations,
@@ -114,16 +113,18 @@ func (t *tracer) copy() *tracer {
}
}
-// markDeletions puts all tracked deletions into the provided nodeset.
-func (t *tracer) markDeletions(set *trienode.NodeSet) {
+// deletedNodes returns a list of node paths which are deleted from the trie.
+func (t *tracer) deletedNodes() []string {
+ var paths []string
for path := range t.deletes {
// It's possible a few deleted nodes were embedded
// in their parent before, the deletions can be no
// effect by deleting nothing, filter them out.
- prev, ok := t.accessList[path]
+ _, ok := t.accessList[path]
if !ok {
continue
}
- set.AddNode([]byte(path), trienode.NewWithPrev(common.Hash{}, nil, prev))
+ paths = append(paths, path)
}
+ return paths
}
diff --git a/trie/trie.go b/trie/trie.go
index 8fbd4e64a..07467ac69 100644
--- a/trie/trie.go
+++ b/trie/trie.go
@@ -612,14 +612,20 @@ func (t *Trie) Commit(collectLeaf bool) (common.Hash, *trienode.NodeSet, error)
defer func() {
t.committed = true
}()
- nodes := trienode.NewNodeSet(t.owner)
- t.tracer.markDeletions(nodes)
-
// Trie is empty and can be classified into two types of situations:
- // - The trie was empty and no update happens
- // - The trie was non-empty and all nodes are dropped
+ // (a) The trie was empty and no update happens => return nil
+ // (b) The trie was non-empty and all nodes are dropped => return
+ // the node set includes all deleted nodes
if t.root == nil {
- return types.EmptyRootHash, nodes, nil
+ paths := t.tracer.deletedNodes()
+ if len(paths) == 0 {
+ return types.EmptyRootHash, nil, nil // case (a)
+ }
+ nodes := trienode.NewNodeSet(t.owner)
+ for _, path := range paths {
+ nodes.AddNode([]byte(path), trienode.NewDeleted())
+ }
+ return types.EmptyRootHash, nodes, nil // case (b)
}
// Derive the hash for all dirty nodes first. We hold the assumption
// in the following procedure that all nodes are hashed.
@@ -633,6 +639,10 @@ func (t *Trie) Commit(collectLeaf bool) (common.Hash, *trienode.NodeSet, error)
t.root = hashedNode
return rootHash, nil, nil
}
+ nodes := trienode.NewNodeSet(t.owner)
+ for _, path := range t.tracer.deletedNodes() {
+ nodes.AddNode([]byte(path), trienode.NewDeleted())
+ }
t.root = newCommitter(nodes, t.tracer, collectLeaf).Commit(t.root)
return rootHash, nodes, nil
}
diff --git a/trie/trie_reader.go b/trie/trie_reader.go
index d42adad2c..1c63ff454 100644
--- a/trie/trie_reader.go
+++ b/trie/trie_reader.go
@@ -24,9 +24,14 @@ import (
// Reader wraps the Node method of a backing trie store.
type Reader interface {
- // Node retrieves the RLP-encoded trie node blob with the provided trie
- // identifier, node path and the corresponding node hash. No error will
- // be returned if the node is not found.
+ // Node retrieves the trie node blob with the provided trie identifier, node path and
+ // the corresponding node hash. No error will be returned if the node is not found.
+ //
+ // When looking up nodes in the account trie, 'owner' is the zero hash. For contract
+ // storage trie nodes, 'owner' is the hash of the account address that containing the
+ // storage.
+ //
+ // TODO(rjl493456442): remove the 'hash' parameter, it's redundant in PBSS.
Node(owner common.Hash, path []byte, hash common.Hash) ([]byte, error)
}
diff --git a/trie/trie_test.go b/trie/trie_test.go
index cc86e7aee..3cb21c195 100644
--- a/trie/trie_test.go
+++ b/trie/trie_test.go
@@ -76,9 +76,9 @@ func TestMissingRoot(t *testing.T) {
func TestMissingNode(t *testing.T) {
testMissingNode(t, false, rawdb.HashScheme)
- //testMissingNode(t, false, rawdb.PathScheme)
+ testMissingNode(t, false, rawdb.PathScheme)
testMissingNode(t, true, rawdb.HashScheme)
- //testMissingNode(t, true, rawdb.PathScheme)
+ testMissingNode(t, true, rawdb.PathScheme)
}
func testMissingNode(t *testing.T, memonly bool, scheme string) {
@@ -422,44 +422,44 @@ func verifyAccessList(old *Trie, new *Trie, set *trienode.NodeSet) error {
if !ok || n.IsDeleted() {
return errors.New("expect new node")
}
- if len(n.Prev) > 0 {
- return errors.New("unexpected origin value")
- }
+ //if len(n.Prev) > 0 {
+ // return errors.New("unexpected origin value")
+ //}
}
// Check deletion set
- for path, blob := range deletes {
+ for path := range deletes {
n, ok := set.Nodes[path]
if !ok || !n.IsDeleted() {
return errors.New("expect deleted node")
}
- if len(n.Prev) == 0 {
- return errors.New("expect origin value")
- }
- if !bytes.Equal(n.Prev, blob) {
- return errors.New("invalid origin value")
- }
+ //if len(n.Prev) == 0 {
+ // return errors.New("expect origin value")
+ //}
+ //if !bytes.Equal(n.Prev, blob) {
+ // return errors.New("invalid origin value")
+ //}
}
// Check update set
- for path, blob := range updates {
+ for path := range updates {
n, ok := set.Nodes[path]
if !ok || n.IsDeleted() {
return errors.New("expect updated node")
}
- if len(n.Prev) == 0 {
- return errors.New("expect origin value")
- }
- if !bytes.Equal(n.Prev, blob) {
- return errors.New("invalid origin value")
- }
+ //if len(n.Prev) == 0 {
+ // return errors.New("expect origin value")
+ //}
+ //if !bytes.Equal(n.Prev, blob) {
+ // return errors.New("invalid origin value")
+ //}
}
return nil
}
func runRandTest(rt randTest) bool {
var scheme = rawdb.HashScheme
- //if rand.Intn(2) == 0 {
- // scheme = rawdb.PathScheme
- //}
+ if rand.Intn(2) == 0 {
+ scheme = rawdb.PathScheme
+ }
var (
origin = types.EmptyRootHash
triedb = newTestDatabase(rawdb.NewMemoryDatabase(), scheme)
diff --git a/trie/triedb/hashdb/database.go b/trie/triedb/hashdb/database.go
index ed20ff9cd..4441f2a38 100644
--- a/trie/triedb/hashdb/database.go
+++ b/trie/triedb/hashdb/database.go
@@ -32,30 +32,31 @@ import (
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie/trienode"
+ "github.com/ethereum/go-ethereum/trie/triestate"
)
var (
- memcacheCleanHitMeter = metrics.NewRegisteredMeter("trie/memcache/clean/hit", nil)
- memcacheCleanMissMeter = metrics.NewRegisteredMeter("trie/memcache/clean/miss", nil)
- memcacheCleanReadMeter = metrics.NewRegisteredMeter("trie/memcache/clean/read", nil)
- memcacheCleanWriteMeter = metrics.NewRegisteredMeter("trie/memcache/clean/write", nil)
+ memcacheCleanHitMeter = metrics.NewRegisteredMeter("hashdb/memcache/clean/hit", nil)
+ memcacheCleanMissMeter = metrics.NewRegisteredMeter("hashdb/memcache/clean/miss", nil)
+ memcacheCleanReadMeter = metrics.NewRegisteredMeter("hashdb/memcache/clean/read", nil)
+ memcacheCleanWriteMeter = metrics.NewRegisteredMeter("hashdb/memcache/clean/write", nil)
- memcacheDirtyHitMeter = metrics.NewRegisteredMeter("trie/memcache/dirty/hit", nil)
- memcacheDirtyMissMeter = metrics.NewRegisteredMeter("trie/memcache/dirty/miss", nil)
- memcacheDirtyReadMeter = metrics.NewRegisteredMeter("trie/memcache/dirty/read", nil)
- memcacheDirtyWriteMeter = metrics.NewRegisteredMeter("trie/memcache/dirty/write", nil)
+ memcacheDirtyHitMeter = metrics.NewRegisteredMeter("hashdb/memcache/dirty/hit", nil)
+ memcacheDirtyMissMeter = metrics.NewRegisteredMeter("hashdb/memcache/dirty/miss", nil)
+ memcacheDirtyReadMeter = metrics.NewRegisteredMeter("hashdb/memcache/dirty/read", nil)
+ memcacheDirtyWriteMeter = metrics.NewRegisteredMeter("hashdb/memcache/dirty/write", nil)
- memcacheFlushTimeTimer = metrics.NewRegisteredResettingTimer("trie/memcache/flush/time", nil)
- memcacheFlushNodesMeter = metrics.NewRegisteredMeter("trie/memcache/flush/nodes", nil)
- memcacheFlushSizeMeter = metrics.NewRegisteredMeter("trie/memcache/flush/size", nil)
+ memcacheFlushTimeTimer = metrics.NewRegisteredResettingTimer("hashdb/memcache/flush/time", nil)
+ memcacheFlushNodesMeter = metrics.NewRegisteredMeter("hashdb/memcache/flush/nodes", nil)
+ memcacheFlushBytesMeter = metrics.NewRegisteredMeter("hashdb/memcache/flush/bytes", nil)
- memcacheGCTimeTimer = metrics.NewRegisteredResettingTimer("trie/memcache/gc/time", nil)
- memcacheGCNodesMeter = metrics.NewRegisteredMeter("trie/memcache/gc/nodes", nil)
- memcacheGCSizeMeter = metrics.NewRegisteredMeter("trie/memcache/gc/size", nil)
+ memcacheGCTimeTimer = metrics.NewRegisteredResettingTimer("hashdb/memcache/gc/time", nil)
+ memcacheGCNodesMeter = metrics.NewRegisteredMeter("hashdb/memcache/gc/nodes", nil)
+ memcacheGCBytesMeter = metrics.NewRegisteredMeter("hashdb/memcache/gc/bytes", nil)
- memcacheCommitTimeTimer = metrics.NewRegisteredResettingTimer("trie/memcache/commit/time", nil)
- memcacheCommitNodesMeter = metrics.NewRegisteredMeter("trie/memcache/commit/nodes", nil)
- memcacheCommitSizeMeter = metrics.NewRegisteredMeter("trie/memcache/commit/size", nil)
+ memcacheCommitTimeTimer = metrics.NewRegisteredResettingTimer("hashdb/memcache/commit/time", nil)
+ memcacheCommitNodesMeter = metrics.NewRegisteredMeter("hashdb/memcache/commit/nodes", nil)
+ memcacheCommitBytesMeter = metrics.NewRegisteredMeter("hashdb/memcache/commit/bytes", nil)
)
// ChildResolver defines the required method to decode the provided
@@ -121,7 +122,13 @@ func (n *cachedNode) forChildren(resolver ChildResolver, onChild func(hash commo
}
// New initializes the hash-based node database.
-func New(diskdb ethdb.Database, cleans *fastcache.Cache, resolver ChildResolver) *Database {
+func New(diskdb ethdb.Database, size int, resolver ChildResolver) *Database {
+ // Initialize the clean cache if the specified cache allowance
+ // is non-zero. Note, the size is in bytes.
+ var cleans *fastcache.Cache
+ if size > 0 {
+ cleans = fastcache.New(size)
+ }
return &Database{
diskdb: diskdb,
resolver: resolver,
@@ -269,7 +276,7 @@ func (db *Database) Dereference(root common.Hash) {
db.gctime += time.Since(start)
memcacheGCTimeTimer.Update(time.Since(start))
- memcacheGCSizeMeter.Mark(int64(storage - db.dirtiesSize))
+ memcacheGCBytesMeter.Mark(int64(storage - db.dirtiesSize))
memcacheGCNodesMeter.Mark(int64(nodes - len(db.dirties)))
log.Debug("Dereferenced trie from memory database", "nodes", nodes-len(db.dirties), "size", storage-db.dirtiesSize, "time", time.Since(start),
@@ -390,7 +397,7 @@ func (db *Database) Cap(limit common.StorageSize) error {
db.flushtime += time.Since(start)
memcacheFlushTimeTimer.Update(time.Since(start))
- memcacheFlushSizeMeter.Mark(int64(storage - db.dirtiesSize))
+ memcacheFlushBytesMeter.Mark(int64(storage - db.dirtiesSize))
memcacheFlushNodesMeter.Mark(int64(nodes - len(db.dirties)))
log.Debug("Persisted nodes from memory database", "nodes", nodes-len(db.dirties), "size", storage-db.dirtiesSize, "time", time.Since(start),
@@ -436,7 +443,7 @@ func (db *Database) Commit(node common.Hash, report bool) error {
// Reset the storage counters and bumped metrics
memcacheCommitTimeTimer.Update(time.Since(start))
- memcacheCommitSizeMeter.Mark(int64(storage - db.dirtiesSize))
+ memcacheCommitBytesMeter.Mark(int64(storage - db.dirtiesSize))
memcacheCommitNodesMeter.Mark(int64(nodes - len(db.dirties)))
logger := log.Info
@@ -549,7 +556,7 @@ func (db *Database) Initialized(genesisRoot common.Hash) bool {
// Update inserts the dirty nodes in provided nodeset into database and link the
// account trie with multiple storage tries if necessary.
-func (db *Database) Update(root common.Hash, parent common.Hash, nodes *trienode.MergedNodeSet) error {
+func (db *Database) Update(root common.Hash, parent common.Hash, block uint64, nodes *trienode.MergedNodeSet, states *triestate.Set) error {
// Ensure the parent state is present and signal a warning if not.
if parent != types.EmptyRootHash {
if blob, _ := db.Node(parent); len(blob) == 0 {
diff --git a/trie/triedb/pathdb/database.go b/trie/triedb/pathdb/database.go
new file mode 100644
index 000000000..29f6b5e10
--- /dev/null
+++ b/trie/triedb/pathdb/database.go
@@ -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 .
+
+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
+}
diff --git a/trie/triedb/pathdb/database_test.go b/trie/triedb/pathdb/database_test.go
new file mode 100644
index 000000000..bcc37e59c
--- /dev/null
+++ b/trie/triedb/pathdb/database_test.go
@@ -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 .
+
+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
+}
diff --git a/trie/triedb/pathdb/difflayer.go b/trie/triedb/pathdb/difflayer.go
new file mode 100644
index 000000000..d25ac1c60
--- /dev/null
+++ b/trie/triedb/pathdb/difflayer.go
@@ -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 .
+
+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)
+}
diff --git a/trie/triedb/pathdb/difflayer_test.go b/trie/triedb/pathdb/difflayer_test.go
new file mode 100644
index 000000000..77c4cd572
--- /dev/null
+++ b/trie/triedb/pathdb/difflayer_test.go
@@ -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 .
+
+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))
+ }
+}
diff --git a/trie/triedb/pathdb/disklayer.go b/trie/triedb/pathdb/disklayer.go
new file mode 100644
index 000000000..b526b3b7d
--- /dev/null
+++ b/trie/triedb/pathdb/disklayer.go
@@ -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 .
+
+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)
+}
diff --git a/trie/triedb/pathdb/errors.go b/trie/triedb/pathdb/errors.go
new file mode 100644
index 000000000..f503a9c49
--- /dev/null
+++ b/trie/triedb/pathdb/errors.go
@@ -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
+
+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)
+}
diff --git a/trie/triedb/pathdb/history.go b/trie/triedb/pathdb/history.go
new file mode 100644
index 000000000..c12fc5518
--- /dev/null
+++ b/trie/triedb/pathdb/history.go
@@ -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
+
+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
+}
diff --git a/trie/triedb/pathdb/history_test.go b/trie/triedb/pathdb/history_test.go
new file mode 100644
index 000000000..6c250c259
--- /dev/null
+++ b/trie/triedb/pathdb/history_test.go
@@ -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
+
+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
+}
diff --git a/trie/triedb/pathdb/journal.go b/trie/triedb/pathdb/journal.go
new file mode 100644
index 000000000..d8c7d39fb
--- /dev/null
+++ b/trie/triedb/pathdb/journal.go
@@ -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 .
+
+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
+}
diff --git a/trie/triedb/pathdb/layertree.go b/trie/triedb/pathdb/layertree.go
new file mode 100644
index 000000000..d31477991
--- /dev/null
+++ b/trie/triedb/pathdb/layertree.go
@@ -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
+
+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)
+}
diff --git a/trie/triedb/pathdb/metrics.go b/trie/triedb/pathdb/metrics.go
new file mode 100644
index 000000000..9e2b1dcbf
--- /dev/null
+++ b/trie/triedb/pathdb/metrics.go
@@ -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
+
+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)
+)
diff --git a/trie/triedb/pathdb/nodebuffer.go b/trie/triedb/pathdb/nodebuffer.go
new file mode 100644
index 000000000..67de225b0
--- /dev/null
+++ b/trie/triedb/pathdb/nodebuffer.go
@@ -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 .
+
+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...)
+}
diff --git a/trie/triedb/pathdb/testutils.go b/trie/triedb/pathdb/testutils.go
new file mode 100644
index 000000000..cb3a240cc
--- /dev/null
+++ b/trie/triedb/pathdb/testutils.go
@@ -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 .
+
+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])
+}
diff --git a/trie/trienode/node.go b/trie/trienode/node.go
index d99f04bd5..8998bcba0 100644
--- a/trie/trienode/node.go
+++ b/trie/trienode/node.go
@@ -25,8 +25,8 @@ import (
)
// Node is a wrapper which contains the encoded blob of the trie node and its
-// unique hash identifier. It is general enough that can be used to represent
-// trie nodes corresponding to different trie implementations.
+// node hash. It is general enough that can be used to represent trie node
+// corresponding to different trie implementations.
type Node struct {
Hash common.Hash // Node hash, empty for deleted node
Blob []byte // Encoded node blob, nil for the deleted node
@@ -42,35 +42,13 @@ func (n *Node) IsDeleted() bool {
return n.Hash == (common.Hash{})
}
-// WithPrev wraps the Node with the previous node value attached.
-type WithPrev struct {
- *Node
- Prev []byte // Encoded original value, nil means it's non-existent
-}
-
-// Unwrap returns the internal Node object.
-func (n *WithPrev) Unwrap() *Node {
- return n.Node
-}
-
-// Size returns the total memory size used by this node. It overloads
-// the function in Node by counting the size of previous value as well.
-func (n *WithPrev) Size() int {
- return n.Node.Size() + len(n.Prev)
-}
-
// New constructs a node with provided node information.
func New(hash common.Hash, blob []byte) *Node {
return &Node{Hash: hash, Blob: blob}
}
-// NewWithPrev constructs a node with provided node information.
-func NewWithPrev(hash common.Hash, blob []byte, prev []byte) *WithPrev {
- return &WithPrev{
- Node: New(hash, blob),
- Prev: prev,
- }
-}
+// NewDeleted constructs a node which is deleted.
+func NewDeleted() *Node { return New(common.Hash{}, nil) }
// leaf represents a trie leaf node
type leaf struct {
@@ -83,7 +61,7 @@ type leaf struct {
type NodeSet struct {
Owner common.Hash
Leaves []*leaf
- Nodes map[string]*WithPrev
+ Nodes map[string]*Node
updates int // the count of updated and inserted nodes
deletes int // the count of deleted nodes
}
@@ -93,7 +71,7 @@ type NodeSet struct {
func NewNodeSet(owner common.Hash) *NodeSet {
return &NodeSet{
Owner: owner,
- Nodes: make(map[string]*WithPrev),
+ Nodes: make(map[string]*Node),
}
}
@@ -104,17 +82,17 @@ func (set *NodeSet) ForEachWithOrder(callback func(path string, n *Node)) {
for path := range set.Nodes {
paths = append(paths, path)
}
- // Bottom-up, longest path first
+ // Bottom-up, the longest path first
slices.SortFunc(paths, func(a, b string) bool {
return a > b // Sort in reverse order
})
for _, path := range paths {
- callback(path, set.Nodes[path].Unwrap())
+ callback(path, set.Nodes[path])
}
}
// AddNode adds the provided node into set.
-func (set *NodeSet) AddNode(path []byte, n *WithPrev) {
+func (set *NodeSet) AddNode(path []byte, n *Node) {
if n.IsDeleted() {
set.deletes += 1
} else {
@@ -124,7 +102,7 @@ func (set *NodeSet) AddNode(path []byte, n *WithPrev) {
}
// Merge adds a set of nodes into the set.
-func (set *NodeSet) Merge(owner common.Hash, nodes map[string]*WithPrev) error {
+func (set *NodeSet) Merge(owner common.Hash, nodes map[string]*Node) error {
if set.Owner != owner {
return fmt.Errorf("nodesets belong to different owner are not mergeable %x-%x", set.Owner, owner)
}
@@ -172,16 +150,11 @@ func (set *NodeSet) Summary() string {
for path, n := range set.Nodes {
// Deletion
if n.IsDeleted() {
- fmt.Fprintf(out, " [-]: %x prev: %x\n", path, n.Prev)
+ fmt.Fprintf(out, " [-]: %x\n", path)
continue
}
- // Insertion
- if len(n.Prev) == 0 {
- fmt.Fprintf(out, " [+]: %x -> %v\n", path, n.Hash)
- continue
- }
- // Update
- fmt.Fprintf(out, " [*]: %x -> %v prev: %x\n", path, n.Hash, n.Prev)
+ // Insertion or update
+ fmt.Fprintf(out, " [+/*]: %x -> %v \n", path, n.Hash)
}
}
for _, n := range set.Leaves {
@@ -217,3 +190,12 @@ func (set *MergedNodeSet) Merge(other *NodeSet) error {
set.Sets[other.Owner] = other
return nil
}
+
+// Flatten returns a two-dimensional map for internal nodes.
+func (set *MergedNodeSet) Flatten() map[common.Hash]map[string]*Node {
+ nodes := make(map[common.Hash]map[string]*Node)
+ for owner, set := range set.Sets {
+ nodes[owner] = set.Nodes
+ }
+ return nodes
+}
diff --git a/trie/triestate/state.go b/trie/triestate/state.go
index 68fee26d5..cb3611baf 100644
--- a/trie/triestate/state.go
+++ b/trie/triestate/state.go
@@ -16,7 +16,44 @@
package triestate
-import "github.com/ethereum/go-ethereum/common"
+import (
+ "errors"
+ "fmt"
+ "sync"
+
+ "github.com/ethereum/go-ethereum/common"
+ "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/trienode"
+ "golang.org/x/crypto/sha3"
+)
+
+// Trie is an Ethereum state trie, can be implemented by Ethereum Merkle Patricia
+// tree or Verkle tree.
+type Trie interface {
+ // Get returns the value for key stored in the trie.
+ Get(key []byte) ([]byte, error)
+
+ // Update associates key with value in the trie.
+ Update(key, value []byte) error
+
+ // Delete removes any existing value for key from the trie.
+ Delete(key []byte) error
+
+ // Commit the trie and returns a set of dirty nodes generated along with
+ // the new root hash.
+ Commit(collectLeaf bool) (common.Hash, *trienode.NodeSet)
+}
+
+// TrieLoader wraps functions to load tries.
+type TrieLoader interface {
+ // OpenTrie opens the main account trie.
+ OpenTrie(root common.Hash) (Trie, error)
+
+ // OpenStorageTrie opens the storage trie of an account.
+ OpenStorageTrie(stateRoot common.Hash, addrHash, root common.Hash) (Trie, error)
+}
// Set represents a collection of mutated states during a state transition.
// The value refers to the original content of state before the transition
@@ -24,5 +61,207 @@ import "github.com/ethereum/go-ethereum/common"
type Set struct {
Accounts map[common.Address][]byte // Mutated account set, nil means the account was not present
Storages map[common.Address]map[common.Hash][]byte // Mutated storage set, nil means the slot was not present
- Incomplete map[common.Address]struct{} // Indicator whether the storage slot is incomplete due to large deletion
+ Incomplete map[common.Address]struct{} // Indicator whether the storage is incomplete due to large deletion
+ size common.StorageSize // Approximate size of set
+}
+
+// New constructs the state set with provided data.
+func New(accounts map[common.Address][]byte, storages map[common.Address]map[common.Hash][]byte, incomplete map[common.Address]struct{}) *Set {
+ return &Set{
+ Accounts: accounts,
+ Storages: storages,
+ Incomplete: incomplete,
+ }
+}
+
+// Size returns the approximate memory size occupied by the set.
+func (s *Set) Size() common.StorageSize {
+ if s.size != 0 {
+ return s.size
+ }
+ for _, account := range s.Accounts {
+ s.size += common.StorageSize(common.AddressLength + len(account))
+ }
+ for _, slots := range s.Storages {
+ for _, val := range slots {
+ s.size += common.StorageSize(common.HashLength + len(val))
+ }
+ s.size += common.StorageSize(common.AddressLength)
+ }
+ s.size += common.StorageSize(common.AddressLength * len(s.Incomplete))
+ return s.size
+}
+
+// context wraps all fields for executing state diffs.
+type context struct {
+ prevRoot common.Hash
+ postRoot common.Hash
+ accounts map[common.Address][]byte
+ storages map[common.Address]map[common.Hash][]byte
+ accountTrie Trie
+ nodes *trienode.MergedNodeSet
+}
+
+// Apply traverses the provided state diffs, apply them in the associated
+// post-state and return the generated dirty trie nodes. The state can be
+// loaded via the provided trie loader.
+func Apply(prevRoot common.Hash, postRoot common.Hash, accounts map[common.Address][]byte, storages map[common.Address]map[common.Hash][]byte, loader TrieLoader) (map[common.Hash]map[string]*trienode.Node, error) {
+ tr, err := loader.OpenTrie(postRoot)
+ if err != nil {
+ return nil, err
+ }
+ ctx := &context{
+ prevRoot: prevRoot,
+ postRoot: postRoot,
+ accounts: accounts,
+ storages: storages,
+ accountTrie: tr,
+ nodes: trienode.NewMergedNodeSet(),
+ }
+ for addr, account := range accounts {
+ var err error
+ if len(account) == 0 {
+ err = deleteAccount(ctx, loader, addr)
+ } else {
+ err = updateAccount(ctx, loader, addr)
+ }
+ if err != nil {
+ return nil, fmt.Errorf("failed to revert state, err: %w", err)
+ }
+ }
+ root, result := tr.Commit(false)
+ if root != prevRoot {
+ return nil, fmt.Errorf("failed to revert state, want %#x, got %#x", prevRoot, root)
+ }
+ if err := ctx.nodes.Merge(result); err != nil {
+ return nil, err
+ }
+ return ctx.nodes.Flatten(), nil
+}
+
+// updateAccount the account was present in prev-state, and may or may not
+// existent in post-state. Apply the reverse diff and verify if the storage
+// root matches the one in prev-state account.
+func updateAccount(ctx *context, loader TrieLoader, addr common.Address) error {
+ // The account was present in prev-state, decode it from the
+ // 'slim-rlp' format bytes.
+ h := newHasher()
+ defer h.release()
+
+ addrHash := h.hash(addr.Bytes())
+ prev, err := types.FullAccount(ctx.accounts[addr])
+ if err != nil {
+ return err
+ }
+ // The account may or may not existent in post-state, try to
+ // load it and decode if it's found.
+ blob, err := ctx.accountTrie.Get(addrHash.Bytes())
+ if err != nil {
+ return err
+ }
+ post := types.NewEmptyStateAccount()
+ if len(blob) != 0 {
+ if err := rlp.DecodeBytes(blob, &post); err != nil {
+ return err
+ }
+ }
+ // Apply all storage changes into the post-state storage trie.
+ st, err := loader.OpenStorageTrie(ctx.postRoot, addrHash, post.Root)
+ if err != nil {
+ return err
+ }
+ for key, val := range ctx.storages[addr] {
+ var err error
+ if len(val) == 0 {
+ err = st.Delete(key.Bytes())
+ } else {
+ err = st.Update(key.Bytes(), val)
+ }
+ if err != nil {
+ return err
+ }
+ }
+ root, result := st.Commit(false)
+ if root != prev.Root {
+ return errors.New("failed to reset storage trie")
+ }
+ // The returned set can be nil if storage trie is not changed
+ // at all.
+ if result != nil {
+ if err := ctx.nodes.Merge(result); err != nil {
+ return err
+ }
+ }
+ // Write the prev-state account into the main trie
+ full, err := rlp.EncodeToBytes(prev)
+ if err != nil {
+ return err
+ }
+ return ctx.accountTrie.Update(addrHash.Bytes(), full)
+}
+
+// deleteAccount the account was not present in prev-state, and is expected
+// to be existent in post-state. Apply the reverse diff and verify if the
+// account and storage is wiped out correctly.
+func deleteAccount(ctx *context, loader TrieLoader, addr common.Address) error {
+ // The account must be existent in post-state, load the account.
+ h := newHasher()
+ defer h.release()
+
+ addrHash := h.hash(addr.Bytes())
+ blob, err := ctx.accountTrie.Get(addrHash.Bytes())
+ if err != nil {
+ return err
+ }
+ if len(blob) == 0 {
+ return fmt.Errorf("account is non-existent %#x", addrHash)
+ }
+ var post types.StateAccount
+ if err := rlp.DecodeBytes(blob, &post); err != nil {
+ return err
+ }
+ st, err := loader.OpenStorageTrie(ctx.postRoot, addrHash, post.Root)
+ if err != nil {
+ return err
+ }
+ for key, val := range ctx.storages[addr] {
+ if len(val) != 0 {
+ return errors.New("expect storage deletion")
+ }
+ if err := st.Delete(key.Bytes()); err != nil {
+ return err
+ }
+ }
+ root, result := st.Commit(false)
+ if root != types.EmptyRootHash {
+ return errors.New("failed to clear storage trie")
+ }
+ // The returned set can be nil if storage trie is not changed
+ // at all.
+ if result != nil {
+ if err := ctx.nodes.Merge(result); err != nil {
+ return err
+ }
+ }
+ // Delete the post-state account from the main trie.
+ return ctx.accountTrie.Delete(addrHash.Bytes())
+}
+
+// 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)
}