Skip to content

Commit

Permalink
all: implement path-based state scheme (ethereum#25963)
Browse files Browse the repository at this point in the history
* all: implement path-based state scheme

* all: edits from review

* core/rawdb, trie/triedb/pathdb: review changes

* core, light, trie, eth, tests: reimplement pbss history

* core, trie/triedb/pathdb: track block number in state history

* trie/triedb/pathdb: add history documentation

* core, trie/triedb/pathdb: address comments from Peter's review

Important changes to list:

- Cache trie nodes by path in clean cache
- Remove root->id mappings when history is truncated

* trie/triedb/pathdb: fallback to disk if unexpect node in clean cache

* core/rawdb: fix tests

* trie/triedb/pathdb: rename metrics, change clean cache key

* trie/triedb: manage the clean cache inside of disk layer

* trie/triedb/pathdb: move journal function

* trie/triedb/path: fix tests

* trie/triedb/pathdb: fix journal

* trie/triedb/pathdb: fix history

* trie/triedb/pathdb: try to fix tests on windows

* core, trie: address comments

* trie/triedb/pathdb: fix test issues

---------

Co-authored-by: Felix Lange <fjl@twurst.com>
Co-authored-by: Martin Holst Swende <martin@swende.se>
  • Loading branch information
3 people authored and MoonShiesty committed Aug 30, 2023
1 parent 03cb2fb commit 8bd250b
Show file tree
Hide file tree
Showing 43 changed files with 4,422 additions and 230 deletions.
6 changes: 3 additions & 3 deletions core/blockchain.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down
4 changes: 2 additions & 2 deletions core/rawdb/accessors_chain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions core/rawdb/accessors_indexes_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -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 {
Expand Down
172 changes: 172 additions & 0 deletions core/rawdb/accessors_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
})
}
46 changes: 24 additions & 22 deletions core/rawdb/accessors_trie.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 *hasher) hash(data []byte) common.Hash {
return crypto.HashData(h.sha, data)
}

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) release() {
hasherPool.Put(h)
}

// ReadAccountTrieNode retrieves the account trie node and the associated node
Expand All @@ -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
Expand All @@ -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.
Expand All @@ -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
Expand All @@ -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.
Expand Down
30 changes: 29 additions & 1 deletion core/rawdb/ancient_scheme.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
}

0 comments on commit 8bd250b

Please sign in to comment.