core/rawdb, trie: improve db APIs for accessing trie nodes (#29362)

* core/rawdb, trie: improve db APIs for accessing trie nodes

* triedb/pathdb: fix
pull/29692/head
rjl493456442 5 months ago committed by GitHub
parent f8820f170c
commit 9f96e07c1c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 5
      cmd/devp2p/internal/ethtest/snap.go
  2. 10
      cmd/geth/dbcmd.go
  3. 2
      core/genesis_test.go
  4. 125
      core/rawdb/accessors_trie.go
  5. 7
      core/rawdb/ancient_utils.go
  6. 3
      eth/handler.go
  7. 4
      eth/protocols/snap/gentrie.go
  8. 7
      eth/protocols/snap/sync.go
  9. 4
      eth/protocols/snap/sync_test.go
  10. 3
      trie/hasher.go
  11. 5
      trie/stacktrie_fuzzer_test.go
  12. 34
      trie/sync.go
  13. 4
      trie/trie_test.go
  14. 10
      trie/trienode/node.go
  15. 3
      trie/triestate/state.go
  16. 9
      triedb/database.go
  17. 3
      triedb/database/database.go
  18. 5
      triedb/hashdb/database.go
  19. 12
      triedb/pathdb/database.go
  20. 4
      triedb/pathdb/database_test.go
  21. 8
      triedb/pathdb/difflayer_test.go
  22. 27
      triedb/pathdb/disklayer.go
  23. 18
      triedb/pathdb/journal.go
  24. 9
      triedb/pathdb/nodebuffer.go

@ -32,7 +32,6 @@ import (
"github.com/ethereum/go-ethereum/internal/utesting"
"github.com/ethereum/go-ethereum/trie"
"github.com/ethereum/go-ethereum/trie/trienode"
"golang.org/x/crypto/sha3"
)
func (c *Conn) snapRequest(code uint64, msg any) (any, error) {
@ -905,7 +904,7 @@ func (s *Suite) snapGetByteCodes(t *utesting.T, tc *byteCodesTest) error {
// that the serving node is missing
var (
bytecodes = res.Codes
hasher = sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher = crypto.NewKeccakState()
hash = make([]byte, 32)
codes = make([][]byte, len(req.Hashes))
)
@ -964,7 +963,7 @@ func (s *Suite) snapGetTrieNodes(t *utesting.T, tc *trieNodesTest) error {
// Cross reference the requested trienodes with the response to find gaps
// that the serving node is missing
hasher := sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher := crypto.NewKeccakState()
hash := make([]byte, 32)
trienodes := res.Nodes
if got, want := len(trienodes), len(tc.expHashes); got != want {

@ -246,11 +246,17 @@ func removeDB(ctx *cli.Context) error {
ancientDir = config.Node.ResolvePath(ancientDir)
}
// Delete state data
statePaths := []string{rootDir, filepath.Join(ancientDir, rawdb.StateFreezerName)}
statePaths := []string{
rootDir,
filepath.Join(ancientDir, rawdb.StateFreezerName),
}
confirmAndRemoveDB(statePaths, "state data", ctx, removeStateDataFlag.Name)
// Delete ancient chain
chainPaths := []string{filepath.Join(ancientDir, rawdb.ChainFreezerName)}
chainPaths := []string{filepath.Join(
ancientDir,
rawdb.ChainFreezerName,
)}
confirmAndRemoveDB(chainPaths, "ancient chain", ctx, removeChainDataFlag.Name)
return nil
}

@ -322,7 +322,7 @@ func TestVerkleGenesisCommit(t *testing.T) {
t.Fatalf("expected trie to be verkle")
}
if !rawdb.ExistsAccountTrieNode(db, nil) {
if !rawdb.HasAccountTrieNode(db, nil) {
t.Fatal("could not find node")
}
}

@ -24,7 +24,6 @@ import (
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"golang.org/x/crypto/sha3"
)
// HashScheme is the legacy hash-based state scheme with which trie nodes are
@ -50,7 +49,7 @@ const PathScheme = "path"
type hasher struct{ sha crypto.KeccakState }
var hasherPool = sync.Pool{
New: func() interface{} { return &hasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} },
New: func() interface{} { return &hasher{sha: crypto.NewKeccakState()} },
}
func newHasher() *hasher {
@ -65,33 +64,15 @@ func (h *hasher) release() {
hasherPool.Put(h)
}
// ReadAccountTrieNode retrieves the account trie node and the associated node
// hash with the specified node path.
func ReadAccountTrieNode(db ethdb.KeyValueReader, path []byte) ([]byte, common.Hash) {
data, err := db.Get(accountTrieNodeKey(path))
if err != nil {
return nil, common.Hash{}
}
h := newHasher()
defer h.release()
return data, h.hash(data)
}
// HasAccountTrieNode checks the account trie node presence with the specified
// node path and the associated node hash.
func HasAccountTrieNode(db ethdb.KeyValueReader, path []byte, hash common.Hash) bool {
data, err := db.Get(accountTrieNodeKey(path))
if err != nil {
return false
}
h := newHasher()
defer h.release()
return h.hash(data) == hash
// ReadAccountTrieNode retrieves the account trie node with the specified node path.
func ReadAccountTrieNode(db ethdb.KeyValueReader, path []byte) []byte {
data, _ := db.Get(accountTrieNodeKey(path))
return data
}
// ExistsAccountTrieNode checks the presence of the account trie node with the
// HasAccountTrieNode checks the presence of the account trie node with the
// specified node path, regardless of the node hash.
func ExistsAccountTrieNode(db ethdb.KeyValueReader, path []byte) bool {
func HasAccountTrieNode(db ethdb.KeyValueReader, path []byte) bool {
has, err := db.Has(accountTrieNodeKey(path))
if err != nil {
return false
@ -113,33 +94,15 @@ func DeleteAccountTrieNode(db ethdb.KeyValueWriter, path []byte) {
}
}
// ReadStorageTrieNode retrieves the storage trie node and the associated node
// hash with the specified node path.
func ReadStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path []byte) ([]byte, common.Hash) {
data, err := db.Get(storageTrieNodeKey(accountHash, path))
if err != nil {
return nil, common.Hash{}
}
h := newHasher()
defer h.release()
return data, h.hash(data)
}
// HasStorageTrieNode checks the storage trie node presence with the provided
// node path and the associated node hash.
func HasStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path []byte, hash common.Hash) bool {
data, err := db.Get(storageTrieNodeKey(accountHash, path))
if err != nil {
return false
}
h := newHasher()
defer h.release()
return h.hash(data) == hash
// ReadStorageTrieNode retrieves the storage trie node with the specified node path.
func ReadStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path []byte) []byte {
data, _ := db.Get(storageTrieNodeKey(accountHash, path))
return data
}
// ExistsStorageTrieNode checks the presence of the storage trie node with the
// HasStorageTrieNode checks the presence of the storage trie node with the
// specified account hash and node path, regardless of the node hash.
func ExistsStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path []byte) bool {
func HasStorageTrieNode(db ethdb.KeyValueReader, accountHash common.Hash, path []byte) bool {
has, err := db.Has(storageTrieNodeKey(accountHash, path))
if err != nil {
return false
@ -198,10 +161,18 @@ func HasTrieNode(db ethdb.KeyValueReader, owner common.Hash, path []byte, hash c
case HashScheme:
return HasLegacyTrieNode(db, hash)
case PathScheme:
var blob []byte
if owner == (common.Hash{}) {
return HasAccountTrieNode(db, path, hash)
blob = ReadAccountTrieNode(db, path)
} else {
blob = ReadStorageTrieNode(db, owner, path)
}
return HasStorageTrieNode(db, owner, path, hash)
if len(blob) == 0 {
return false
}
h := newHasher()
defer h.release()
return h.hash(blob) == hash // exists but not match
default:
panic(fmt.Sprintf("Unknown scheme %v", scheme))
}
@ -209,43 +180,35 @@ func HasTrieNode(db ethdb.KeyValueReader, owner common.Hash, path []byte, hash c
// ReadTrieNode retrieves the trie node from database with the provided node info
// and associated node hash.
// hashScheme-based lookup requires the following:
// - hash
//
// pathScheme-based lookup requires the following:
// - owner
// - path
func ReadTrieNode(db ethdb.KeyValueReader, owner common.Hash, path []byte, hash common.Hash, scheme string) []byte {
switch scheme {
case HashScheme:
return ReadLegacyTrieNode(db, hash)
case PathScheme:
var (
blob []byte
nHash common.Hash
)
var blob []byte
if owner == (common.Hash{}) {
blob, nHash = ReadAccountTrieNode(db, path)
blob = ReadAccountTrieNode(db, path)
} else {
blob, nHash = ReadStorageTrieNode(db, owner, path)
blob = ReadStorageTrieNode(db, owner, path)
}
if nHash != hash {
if len(blob) == 0 {
return nil
}
h := newHasher()
defer h.release()
if h.hash(blob) != hash {
return nil // exists but not match
}
return blob
default:
panic(fmt.Sprintf("Unknown scheme %v", scheme))
}
}
// WriteTrieNode writes the trie node into database with the provided node info
// and associated node hash.
// hashScheme-based lookup requires the following:
// - hash
// WriteTrieNode writes the trie node into database with the provided node info.
//
// pathScheme-based lookup requires the following:
// - owner
// - path
// hash-scheme requires the node hash as the identifier.
// path-scheme requires the node owner and path as the identifier.
func WriteTrieNode(db ethdb.KeyValueWriter, owner common.Hash, path []byte, hash common.Hash, node []byte, scheme string) {
switch scheme {
case HashScheme:
@ -261,14 +224,10 @@ func WriteTrieNode(db ethdb.KeyValueWriter, owner common.Hash, path []byte, hash
}
}
// DeleteTrieNode deletes the trie node from database with the provided node info
// and associated node hash.
// hashScheme-based lookup requires the following:
// - hash
// DeleteTrieNode deletes the trie node from database with the provided node info.
//
// pathScheme-based lookup requires the following:
// - owner
// - path
// hash-scheme requires the node hash as the identifier.
// path-scheme requires the node owner and path as the identifier.
func DeleteTrieNode(db ethdb.KeyValueWriter, owner common.Hash, path []byte, hash common.Hash, scheme string) {
switch scheme {
case HashScheme:
@ -287,9 +246,8 @@ func DeleteTrieNode(db ethdb.KeyValueWriter, owner common.Hash, path []byte, has
// ReadStateScheme reads the state scheme of persistent state, or none
// if the state is not present in database.
func ReadStateScheme(db ethdb.Reader) string {
// Check if state in path-based scheme is present
blob, _ := ReadAccountTrieNode(db, nil)
if len(blob) != 0 {
// Check if state in path-based scheme is present.
if HasAccountTrieNode(db, nil) {
return PathScheme
}
// The root node might be deleted during the initial snap sync, check
@ -304,8 +262,7 @@ func ReadStateScheme(db ethdb.Reader) string {
if header == nil {
return "" // empty datadir
}
blob = ReadLegacyTrieNode(db, header.Root)
if len(blob) == 0 {
if !HasLegacyTrieNode(db, header.Root) {
return "" // no state in disk
}
return HashScheme

@ -89,20 +89,17 @@ func inspectFreezers(db ethdb.Database) ([]freezerInfo, error) {
infos = append(infos, info)
case StateFreezerName:
if ReadStateScheme(db) != PathScheme {
continue
}
datadir, err := db.AncientDatadir()
if err != nil {
return nil, err
}
f, err := NewStateFreezer(datadir, true)
if err != nil {
return nil, err
continue // might be possible the state freezer is not existent
}
defer f.Close()
info, err := inspect(StateFreezerName, stateFreezerNoSnappy, f)
info, err := inspect(freezer, stateFreezerNoSnappy, f)
if err != nil {
return nil, err
}

@ -42,7 +42,6 @@ import (
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/triedb/pathdb"
"golang.org/x/crypto/sha3"
)
const (
@ -480,7 +479,7 @@ func (h *handler) BroadcastTransactions(txs types.Transactions) {
var (
signer = types.LatestSignerForChainID(h.chain.Config().ChainID) // Don't care about chain status, we just need *a* sender
hasher = sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher = crypto.NewKeccakState()
hash = make([]byte, 32)
)
for _, tx := range txs {

@ -164,7 +164,7 @@ func (t *pathTrie) deleteAccountNode(path []byte, inner bool) {
} else {
accountOuterLookupGauge.Inc(1)
}
if !rawdb.ExistsAccountTrieNode(t.db, path) {
if !rawdb.HasAccountTrieNode(t.db, path) {
return
}
if inner {
@ -181,7 +181,7 @@ func (t *pathTrie) deleteStorageNode(path []byte, inner bool) {
} else {
storageOuterLookupGauge.Inc(1)
}
if !rawdb.ExistsStorageTrieNode(t.db, t.owner, path) {
if !rawdb.HasStorageTrieNode(t.db, t.owner, path) {
return
}
if inner {

@ -42,7 +42,6 @@ import (
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie"
"github.com/ethereum/go-ethereum/trie/trienode"
"golang.org/x/crypto/sha3"
)
const (
@ -2653,7 +2652,7 @@ func (s *Syncer) onByteCodes(peer SyncPeer, id uint64, bytecodes [][]byte) error
// Cross reference the requested bytecodes with the response to find gaps
// that the serving node is missing
hasher := sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher := crypto.NewKeccakState()
hash := make([]byte, 32)
codes := make([][]byte, len(req.hashes))
@ -2901,7 +2900,7 @@ func (s *Syncer) OnTrieNodes(peer SyncPeer, id uint64, trienodes [][]byte) error
// Cross reference the requested trienodes with the response to find gaps
// that the serving node is missing
var (
hasher = sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher = crypto.NewKeccakState()
hash = make([]byte, 32)
nodes = make([][]byte, len(req.hashes))
fills uint64
@ -3007,7 +3006,7 @@ func (s *Syncer) onHealByteCodes(peer SyncPeer, id uint64, bytecodes [][]byte) e
// Cross reference the requested bytecodes with the response to find gaps
// that the serving node is missing
hasher := sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher := crypto.NewKeccakState()
hash := make([]byte, 32)
codes := make([][]byte, len(req.hashes))

@ -64,7 +64,7 @@ func TestHashing(t *testing.T) {
}
}
var new = func() {
hasher := sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher := crypto.NewKeccakState()
var hash = make([]byte, 32)
for i := 0; i < len(bytecodes); i++ {
hasher.Reset()
@ -96,7 +96,7 @@ func BenchmarkHashing(b *testing.B) {
}
}
var new = func() {
hasher := sha3.NewLegacyKeccak256().(crypto.KeccakState)
hasher := crypto.NewKeccakState()
var hash = make([]byte, 32)
for i := 0; i < len(bytecodes); i++ {
hasher.Reset()

@ -21,7 +21,6 @@ import (
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/rlp"
"golang.org/x/crypto/sha3"
)
// hasher is a type used for the trie Hash operation. A hasher has some
@ -38,7 +37,7 @@ var hasherPool = sync.Pool{
New: func() interface{} {
return &hasher{
tmp: make([]byte, 0, 550), // cap is as large as a full fullNode.
sha: sha3.NewLegacyKeccak256().(crypto.KeccakState),
sha: crypto.NewKeccakState(),
encbuf: rlp.NewEncoderBuffer(nil),
}
},

@ -28,7 +28,6 @@ import (
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/trie/trienode"
"golang.org/x/crypto/sha3"
)
func FuzzStackTrie(f *testing.F) {
@ -41,10 +40,10 @@ func fuzz(data []byte, debugging bool) {
// This spongeDb is used to check the sequence of disk-db-writes
var (
input = bytes.NewReader(data)
spongeA = &spongeDb{sponge: sha3.NewLegacyKeccak256()}
spongeA = &spongeDb{sponge: crypto.NewKeccakState()}
dbA = newTestDatabase(rawdb.NewDatabase(spongeA), rawdb.HashScheme)
trieA = NewEmpty(dbA)
spongeB = &spongeDb{sponge: sha3.NewLegacyKeccak256()}
spongeB = &spongeDb{sponge: crypto.NewKeccakState()}
dbB = newTestDatabase(rawdb.NewDatabase(spongeB), rawdb.HashScheme)
trieB = NewStackTrie(func(path []byte, hash common.Hash, blob []byte) {
rawdb.WriteTrieNode(spongeB, common.Hash{}, path, hash, blob, dbB.Scheme())

@ -25,6 +25,7 @@ import (
"github.com/ethereum/go-ethereum/common/prque"
"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/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
@ -546,9 +547,9 @@ func (s *Sync) children(req *nodeRequest, object node) ([]*nodeRequest, error) {
// the performance impact negligible.
var exists bool
if owner == (common.Hash{}) {
exists = rawdb.ExistsAccountTrieNode(s.database, append(inner, key[:i]...))
exists = rawdb.HasAccountTrieNode(s.database, append(inner, key[:i]...))
} else {
exists = rawdb.ExistsStorageTrieNode(s.database, owner, append(inner, key[:i]...))
exists = rawdb.HasStorageTrieNode(s.database, owner, append(inner, key[:i]...))
}
if exists {
s.membatch.delNode(owner, append(inner, key[:i]...))
@ -691,13 +692,14 @@ func (s *Sync) hasNode(owner common.Hash, path []byte, hash common.Hash) (exists
}
// If node is running with path scheme, check the presence with node path.
var blob []byte
var dbHash common.Hash
if owner == (common.Hash{}) {
blob, dbHash = rawdb.ReadAccountTrieNode(s.database, path)
blob = rawdb.ReadAccountTrieNode(s.database, path)
} else {
blob, dbHash = rawdb.ReadStorageTrieNode(s.database, owner, path)
blob = rawdb.ReadStorageTrieNode(s.database, owner, path)
}
exists = hash == dbHash
h := newBlobHasher()
defer h.release()
exists = hash == h.hash(blob)
inconsistent = !exists && len(blob) != 0
return exists, inconsistent
}
@ -712,3 +714,23 @@ func ResolvePath(path []byte) (common.Hash, []byte) {
}
return owner, path
}
// blobHasher is used to compute the sha256 hash of the provided data.
type blobHasher struct{ state crypto.KeccakState }
// blobHasherPool is the pool for reusing pre-allocated hash state.
var blobHasherPool = sync.Pool{
New: func() interface{} { return &blobHasher{state: crypto.NewKeccakState()} },
}
func newBlobHasher() *blobHasher {
return blobHasherPool.Get().(*blobHasher)
}
func (h *blobHasher) hash(data []byte) common.Hash {
return crypto.HashData(h.state, data)
}
func (h *blobHasher) release() {
blobHasherPool.Put(h)
}

@ -886,7 +886,7 @@ func TestCommitSequence(t *testing.T) {
} {
addresses, accounts := makeAccounts(tc.count)
// This spongeDb is used to check the sequence of disk-db-writes
s := &spongeDb{sponge: sha3.NewLegacyKeccak256()}
s := &spongeDb{sponge: crypto.NewKeccakState()}
db := newTestDatabase(rawdb.NewDatabase(s), rawdb.HashScheme)
trie := NewEmpty(db)
// Fill the trie with elements
@ -917,7 +917,7 @@ func TestCommitSequenceRandomBlobs(t *testing.T) {
} {
prng := rand.New(rand.NewSource(int64(i)))
// This spongeDb is used to check the sequence of disk-db-writes
s := &spongeDb{sponge: sha3.NewLegacyKeccak256()}
s := &spongeDb{sponge: crypto.NewKeccakState()}
db := newTestDatabase(rawdb.NewDatabase(s), rawdb.HashScheme)
trie := NewEmpty(db)
// Fill the trie with elements

@ -135,16 +135,6 @@ func (set *NodeSet) Size() (int, int) {
return set.updates, set.deletes
}
// Hashes returns the hashes of all updated nodes. TODO(rjl493456442) how can
// we get rid of it?
func (set *NodeSet) Hashes() []common.Hash {
ret := make([]common.Hash, 0, len(set.Nodes))
for _, node := range set.Nodes {
ret = append(ret, node.Hash)
}
return ret
}
// Summary returns a string-representation of the NodeSet.
func (set *NodeSet) Summary() string {
var out = new(strings.Builder)

@ -26,7 +26,6 @@ import (
"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
@ -257,7 +256,7 @@ func deleteAccount(ctx *context, loader TrieLoader, addr common.Address) error {
type hasher struct{ sha crypto.KeccakState }
var hasherPool = sync.Pool{
New: func() interface{} { return &hasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} },
New: func() interface{} { return &hasher{sha: crypto.NewKeccakState()} },
}
func newHasher() *hasher {

@ -20,6 +20,7 @@ import (
"errors"
"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"
@ -48,9 +49,6 @@ var HashDefaults = &Config{
// backend defines the methods needed to access/update trie nodes in different
// state scheme.
type backend interface {
// Scheme returns the identifier of used storage scheme.
Scheme() string
// Initialized returns an indicator if the state data is already initialized
// according to the state scheme.
Initialized(genesisRoot common.Hash) bool
@ -181,7 +179,10 @@ func (db *Database) Initialized(genesisRoot common.Hash) bool {
// Scheme returns the node scheme used in the database.
func (db *Database) Scheme() string {
return db.backend.Scheme()
if db.config.PathDB != nil {
return rawdb.PathScheme
}
return rawdb.HashScheme
}
// Close flushes the dangling preimages to disk and closes the trie database.

@ -25,6 +25,9 @@ type Reader interface {
// 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.
//
// Don't modify the returned byte slice since it's not deep-copied and
// still be referenced by database.
Node(owner common.Hash, path []byte, hash common.Hash) ([]byte, error)
}

@ -623,11 +623,6 @@ func (db *Database) Close() error {
return nil
}
// Scheme returns the node scheme used in the database.
func (db *Database) Scheme() string {
return rawdb.HashScheme
}
// Reader retrieves a node reader belonging to the given state root.
// An error will be returned if the requested state is not available.
func (db *Database) Reader(root common.Hash) (*reader, error) {

@ -26,6 +26,7 @@ import (
"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/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
@ -306,8 +307,10 @@ func (db *Database) Enable(root common.Hash) error {
}
// Ensure the provided state root matches the stored one.
root = types.TrieRootHash(root)
_, stored := rawdb.ReadAccountTrieNode(db.diskdb, nil)
stored = types.TrieRootHash(stored)
stored := types.EmptyRootHash
if blob := rawdb.ReadAccountTrieNode(db.diskdb, nil); len(blob) > 0 {
stored = crypto.Keccak256Hash(blob)
}
if stored != root {
return fmt.Errorf("state root mismatch: stored %x, synced %x", stored, root)
}
@ -480,11 +483,6 @@ func (db *Database) SetBufferSize(size int) error {
return db.tree.bottom().setBufferSize(db.bufferSize)
}
// Scheme returns the node scheme used in the database.
func (db *Database) Scheme() string {
return rawdb.PathScheme
}
// modifyAllowed returns the indicator if mutation is allowed. This function
// assumes the db.lock is already held.
func (db *Database) modifyAllowed() error {

@ -474,7 +474,7 @@ func TestDisable(t *testing.T) {
tester := newTester(t, 0)
defer tester.release()
_, stored := rawdb.ReadAccountTrieNode(tester.db.diskdb, nil)
stored := crypto.Keccak256Hash(rawdb.ReadAccountTrieNode(tester.db.diskdb, nil))
if err := tester.db.Disable(); err != nil {
t.Fatalf("Failed to deactivate database: %v", err)
}
@ -580,7 +580,7 @@ func TestCorruptedJournal(t *testing.T) {
t.Errorf("Failed to journal, err: %v", err)
}
tester.db.Close()
_, root := rawdb.ReadAccountTrieNode(tester.db.diskdb, nil)
root := crypto.Keccak256Hash(rawdb.ReadAccountTrieNode(tester.db.diskdb, nil))
// Mutate the journal in disk, it should be regarded as invalid
blob := rawdb.ReadTrieJournal(tester.db.diskdb)

@ -70,10 +70,10 @@ func benchmarkSearch(b *testing.B, depth int, total int) {
blob = testrand.Bytes(100)
node = trienode.New(crypto.Keccak256Hash(blob), blob)
)
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob)
nodes[common.Hash{}][string(path)] = node
if npath == nil && depth == index {
npath = common.CopyBytes(path)
nblob = common.CopyBytes(node.Blob)
nblob = common.CopyBytes(blob)
}
}
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil)
@ -116,7 +116,7 @@ func BenchmarkPersist(b *testing.B) {
blob = testrand.Bytes(100)
node = trienode.New(crypto.Keccak256Hash(blob), blob)
)
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob)
nodes[common.Hash{}][string(path)] = node
}
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil)
}
@ -154,7 +154,7 @@ func BenchmarkJournal(b *testing.B) {
blob = testrand.Bytes(100)
node = trienode.New(crypto.Keccak256Hash(blob), blob)
)
nodes[common.Hash{}][string(path)] = trienode.New(node.Hash, node.Blob)
nodes[common.Hash{}][string(path)] = node
}
// TODO(rjl493456442) a non-nil state set is expected.
return newDiffLayer(parent, common.Hash{}, 0, 0, nodes, nil)

@ -27,7 +27,6 @@ import (
"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.
@ -117,12 +116,12 @@ func (dl *diskLayer) node(owner common.Hash, path []byte, depth int) ([]byte, co
dirtyMissMeter.Mark(1)
// Try to retrieve the trie node from the clean memory cache
h := newHasher()
defer h.release()
key := cacheKey(owner, path)
if dl.cleans != nil {
if blob := dl.cleans.Get(nil, key); len(blob) > 0 {
h := newHasher()
defer h.release()
cleanHitMeter.Mark(1)
cleanReadMeter.Mark(int64(len(blob)))
return blob, h.hash(blob), &nodeLoc{loc: locCleanCache, depth: depth}, nil
@ -130,20 +129,18 @@ func (dl *diskLayer) node(owner common.Hash, path []byte, depth int) ([]byte, co
cleanMissMeter.Mark(1)
}
// Try to retrieve the trie node from the disk.
var (
nBlob []byte
nHash common.Hash
)
var blob []byte
if owner == (common.Hash{}) {
nBlob, nHash = rawdb.ReadAccountTrieNode(dl.db.diskdb, path)
blob = rawdb.ReadAccountTrieNode(dl.db.diskdb, path)
} else {
nBlob, nHash = rawdb.ReadStorageTrieNode(dl.db.diskdb, owner, path)
blob = rawdb.ReadStorageTrieNode(dl.db.diskdb, owner, path)
}
if dl.cleans != nil && len(nBlob) > 0 {
dl.cleans.Set(key, nBlob)
cleanWriteMeter.Mark(int64(len(nBlob)))
if dl.cleans != nil && len(blob) > 0 {
dl.cleans.Set(key, blob)
cleanWriteMeter.Mark(int64(len(blob)))
}
return nBlob, nHash, &nodeLoc{loc: locDiskLayer, depth: depth}, nil
return blob, h.hash(blob), &nodeLoc{loc: locDiskLayer, depth: depth}, nil
}
// update implements the layer interface, returning a new diff layer on top
@ -303,7 +300,7 @@ func (dl *diskLayer) resetCache() {
type hasher struct{ sha crypto.KeccakState }
var hasherPool = sync.Pool{
New: func() interface{} { return &hasher{sha: sha3.NewLegacyKeccak256().(crypto.KeccakState)} },
New: func() interface{} { return &hasher{sha: crypto.NewKeccakState()} },
}
func newHasher() *hasher {

@ -120,9 +120,10 @@ func (db *Database) loadJournal(diskRoot common.Hash) (layer, error) {
// 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)
var root = types.EmptyRootHash
if blob := rawdb.ReadAccountTrieNode(db.diskdb, nil); len(blob) > 0 {
root = crypto.Keccak256Hash(blob)
}
// Load the layers by resolving the journal
head, err := db.loadJournal(root)
if err == nil {
@ -361,14 +362,13 @@ func (db *Database) Journal(root common.Hash) error {
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 {
diskRoot := types.EmptyRootHash
if blob := rawdb.ReadAccountTrieNode(db.diskdb, nil); len(blob) > 0 {
diskRoot = crypto.Keccak256Hash(blob)
}
if err := rlp.Encode(journal, diskRoot); err != nil {
return err
}
// Finally write out the journal of each layer in reverse order.

@ -17,6 +17,7 @@
package pathdb
import (
"bytes"
"fmt"
"time"
@ -148,14 +149,14 @@ func (b *nodebuffer) revert(db ethdb.KeyValueReader, nodes map[common.Hash]map[s
//
// In case of database rollback, don't panic if this "clean"
// node occurs which is not present in buffer.
var nhash common.Hash
var blob []byte
if owner == (common.Hash{}) {
_, nhash = rawdb.ReadAccountTrieNode(db, []byte(path))
blob = rawdb.ReadAccountTrieNode(db, []byte(path))
} else {
_, nhash = rawdb.ReadStorageTrieNode(db, owner, []byte(path))
blob = rawdb.ReadStorageTrieNode(db, owner, []byte(path))
}
// Ignore the clean node in the case described above.
if nhash == n.Hash {
if bytes.Equal(blob, n.Blob) {
continue
}
panic(fmt.Sprintf("non-existent node (%x %v) blob: %v", owner, path, crypto.Keccak256Hash(n.Blob).Hex()))

Loading…
Cancel
Save