core/state, trie: port changes from PBSS (#26763)

This commit is contained in:
rjl493456442 2023-03-14 16:50:53 +08:00 committed by GitHub
parent 94ff721911
commit c8a6b7100c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 646 additions and 646 deletions

View File

@ -970,8 +970,8 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
storageTrieNodesUpdated int
storageTrieNodesDeleted int
nodes = trie.NewMergedNodeSet()
codeWriter = s.db.DiskDB().NewBatch()
)
codeWriter := s.db.DiskDB().NewBatch()
for addr := range s.stateObjectsDirty {
if obj := s.stateObjects[addr]; !obj.deleted {
// Write any contract code associated with the state object

View File

@ -33,29 +33,20 @@ type leaf struct {
// insertion order.
type committer struct {
nodes *NodeSet
tracer *tracer
collectLeaf bool
}
// newCommitter creates a new committer or picks one from the pool.
func newCommitter(owner common.Hash, tracer *tracer, collectLeaf bool) *committer {
func newCommitter(nodeset *NodeSet, collectLeaf bool) *committer {
return &committer{
nodes: NewNodeSet(owner),
tracer: tracer,
nodes: nodeset,
collectLeaf: collectLeaf,
}
}
// Commit collapses a node down into a hash node and returns it along with
// the modified nodeset.
func (c *committer) Commit(n node) (hashNode, *NodeSet) {
h := c.commit(nil, n)
// Some nodes can be deleted from trie which can't be captured
// by committer itself. Iterate all deleted nodes tracked by
// tracer and marked them as deleted only if they are present
// in database previously.
c.tracer.markDeletions(c.nodes)
return h.(hashNode), c.nodes
// Commit collapses a node down into a hash node.
func (c *committer) Commit(n node) hashNode {
return c.commit(nil, n).(hashNode)
}
// commit collapses a node down into a hash node and returns it.
@ -74,9 +65,7 @@ func (c *committer) commit(path []byte, n node) node {
// If the child is fullNode, recursively commit,
// otherwise it can only be hashNode or valueNode.
if _, ok := cn.Val.(*fullNode); ok {
childV := c.commit(append(path, cn.Key...), cn.Val)
collapsed.Val = childV
collapsed.Val = c.commit(append(path, cn.Key...), cn.Val)
}
// The key needs to be copied, since we're adding it to the
// modified nodeset.
@ -85,12 +74,6 @@ func (c *committer) commit(path []byte, n node) node {
if hn, ok := hashedNode.(hashNode); ok {
return hn
}
// The short node now is embedded in its parent. Mark the node as
// deleted if it's present in database previously. It's equivalent
// as deletion from database's perspective.
if prev := c.tracer.getPrev(path); len(prev) != 0 {
c.nodes.markDeleted(path, prev)
}
return collapsed
case *fullNode:
hashedKids := c.commitChildren(path, cn)
@ -101,12 +84,6 @@ func (c *committer) commit(path []byte, n node) node {
if hn, ok := hashedNode.(hashNode); ok {
return hn
}
// The full node now is embedded in its parent. Mark the node as
// deleted if it's present in database previously. It's equivalent
// as deletion from database's perspective.
if prev := c.tracer.getPrev(path); len(prev) != 0 {
c.nodes.markDeleted(path, prev)
}
return collapsed
case hashNode:
return cn
@ -134,8 +111,7 @@ func (c *committer) commitChildren(path []byte, n *fullNode) [17]node {
// Commit the child recursively and store the "hashed" value.
// Note the returned node can be some embedded nodes, so it's
// possible the type is not hashNode.
hashed := c.commit(append(path, byte(i)), child)
children[i] = hashed
children[i] = c.commit(append(path, byte(i)), child)
}
// For the 17th child, it's possible the type is valuenode.
if n.Children[16] != nil {
@ -155,6 +131,12 @@ func (c *committer) store(path []byte, n node) node {
// usually is leaf node). But small value (less than 32bytes) is not
// our target (leaves in account trie only).
if hash == nil {
// 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.
if _, ok := c.nodes.accessList[string(path)]; ok {
c.nodes.markDeleted(path)
}
return n
}
// We have the hash already, estimate the RLP encoding-size of the node.
@ -169,7 +151,7 @@ func (c *committer) store(path []byte, n node) node {
}
)
// Collect the dirty node to nodeset for return.
c.nodes.markUpdated(path, mnode, c.tracer.getPrev(path))
c.nodes.markUpdated(path, mnode)
// 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

View File

@ -792,13 +792,12 @@ func (db *Database) Update(nodes *MergedNodeSet) error {
}
for _, owner := range order {
subset := nodes.sets[owner]
for _, path := range subset.updates.order {
n, ok := subset.updates.nodes[path]
if !ok {
return fmt.Errorf("missing node %x %v", owner, path)
subset.forEachWithOrder(func(path string, n *memoryNode) {
if n.isDeleted() {
return // ignore deletion
}
db.insert(n.hash, int(n.size), n.node)
}
})
}
// Link up the account trie and storage trie if the node points
// to an account trie leaf.

View File

@ -19,6 +19,7 @@ package trie
import (
"fmt"
"reflect"
"sort"
"strings"
"github.com/ethereum/go-ethereum/common"
@ -40,8 +41,8 @@ var memoryNodeSize = int(reflect.TypeOf(memoryNode{}).Size())
// memorySize returns the total memory size used by this node.
// nolint:unused
func (n *memoryNode) memorySize(key int) int {
return int(n.size) + memoryNodeSize + key
func (n *memoryNode) memorySize(pathlen int) int {
return int(n.size) + memoryNodeSize + pathlen
}
// rlp returns the raw rlp encoded blob of the cached trie node, either directly
@ -64,7 +65,13 @@ func (n *memoryNode) obj() node {
return expandNode(n.hash[:], n.node)
}
// isDeleted returns the indicator if the node is marked as deleted.
func (n *memoryNode) isDeleted() bool {
return n.hash == (common.Hash{})
}
// nodeWithPrev wraps the memoryNode with the previous node value.
// nolint: unused
type nodeWithPrev struct {
*memoryNode
prev []byte // RLP-encoded previous value, nil means it's non-existent
@ -79,64 +86,60 @@ func (n *nodeWithPrev) unwrap() *memoryNode {
// memorySize returns the total memory size used by this node. It overloads
// the function in memoryNode by counting the size of previous value as well.
// nolint: unused
func (n *nodeWithPrev) memorySize(key int) int {
return n.memoryNode.memorySize(key) + len(n.prev)
}
// nodesWithOrder represents a collection of dirty nodes which includes
// newly-inserted and updated nodes. The modification order of all nodes
// is represented by order list.
type nodesWithOrder struct {
order []string // the path list of dirty nodes, sort by insertion order
nodes map[string]*nodeWithPrev // the map of dirty nodes, keyed by node path
func (n *nodeWithPrev) memorySize(pathlen int) int {
return n.memoryNode.memorySize(pathlen) + len(n.prev)
}
// NodeSet contains all dirty nodes collected during the commit operation.
// Each node is keyed by path. It's not thread-safe to use.
type NodeSet struct {
owner common.Hash // the identifier of the trie
updates *nodesWithOrder // the set of updated nodes(newly inserted, updated)
deletes map[string][]byte // the map of deleted nodes, keyed by node
leaves []*leaf // the list of dirty leaves
owner common.Hash // the identifier of the trie
nodes map[string]*memoryNode // the set of dirty nodes(inserted, updated, deleted)
leaves []*leaf // the list of dirty leaves
updates int // the count of updated and inserted nodes
deletes int // the count of deleted nodes
// The list of accessed nodes, which records the original node value.
// The origin value is expected to be nil for newly inserted node
// and is expected to be non-nil for other types(updated, deleted).
accessList map[string][]byte
}
// NewNodeSet initializes an empty node set to be used for tracking dirty nodes
// from a specific account or storage trie. The owner is zero for the account
// trie and the owning account address hash for storage tries.
func NewNodeSet(owner common.Hash) *NodeSet {
func NewNodeSet(owner common.Hash, accessList map[string][]byte) *NodeSet {
return &NodeSet{
owner: owner,
updates: &nodesWithOrder{
nodes: make(map[string]*nodeWithPrev),
},
deletes: make(map[string][]byte),
owner: owner,
nodes: make(map[string]*memoryNode),
accessList: accessList,
}
}
/*
// NewNodeSetWithDeletion initializes the nodeset with provided deletion set.
func NewNodeSetWithDeletion(owner common.Hash, paths [][]byte, prev [][]byte) *NodeSet {
set := NewNodeSet(owner)
for i, path := range paths {
set.markDeleted(path, prev[i])
// forEachWithOrder iterates the dirty nodes with the order from bottom to top,
// right to left, nodes with the longest path will be iterated first.
func (set *NodeSet) forEachWithOrder(callback func(path string, n *memoryNode)) {
var paths sort.StringSlice
for path := range set.nodes {
paths = append(paths, path)
}
return set
}
*/
// markUpdated marks the node as dirty(newly-inserted or updated) with provided
// node path, node object along with its previous value.
func (set *NodeSet) markUpdated(path []byte, node *memoryNode, prev []byte) {
set.updates.order = append(set.updates.order, string(path))
set.updates.nodes[string(path)] = &nodeWithPrev{
memoryNode: node,
prev: prev,
// Bottom-up, longest path first
sort.Sort(sort.Reverse(paths))
for _, path := range paths {
callback(path, set.nodes[path])
}
}
// markDeleted marks the node as deleted with provided path and previous value.
func (set *NodeSet) markDeleted(path []byte, prev []byte) {
set.deletes[string(path)] = prev
// markUpdated marks the node as dirty(newly-inserted or updated).
func (set *NodeSet) markUpdated(path []byte, node *memoryNode) {
set.nodes[string(path)] = node
set.updates += 1
}
// markDeleted marks the node as deleted.
func (set *NodeSet) markDeleted(path []byte) {
set.nodes[string(path)] = &memoryNode{}
set.deletes += 1
}
// addLeaf collects the provided leaf node into set.
@ -144,16 +147,16 @@ func (set *NodeSet) addLeaf(node *leaf) {
set.leaves = append(set.leaves, node)
}
// Size returns the number of updated and deleted nodes contained in the set.
// Size returns the number of dirty nodes in set.
func (set *NodeSet) Size() (int, int) {
return len(set.updates.order), len(set.deletes)
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 {
var ret []common.Hash
for _, node := range set.updates.nodes {
for _, node := range set.nodes {
ret = append(ret, node.hash)
}
return ret
@ -163,19 +166,23 @@ func (set *NodeSet) Hashes() []common.Hash {
func (set *NodeSet) Summary() string {
var out = new(strings.Builder)
fmt.Fprintf(out, "nodeset owner: %v\n", set.owner)
if set.updates != nil {
for _, key := range set.updates.order {
updated := set.updates.nodes[key]
if updated.prev != nil {
fmt.Fprintf(out, " [*]: %x -> %v prev: %x\n", key, updated.hash, updated.prev)
} else {
fmt.Fprintf(out, " [+]: %x -> %v\n", key, updated.hash)
if set.nodes != nil {
for path, n := range set.nodes {
// Deletion
if n.isDeleted() {
fmt.Fprintf(out, " [-]: %x prev: %x\n", path, set.accessList[path])
continue
}
// Insertion
origin, ok := set.accessList[path]
if !ok {
fmt.Fprintf(out, " [+]: %x -> %v\n", path, n.hash)
continue
}
// Update
fmt.Fprintf(out, " [*]: %x -> %v prev: %x\n", path, n.hash, origin)
}
}
for k, n := range set.deletes {
fmt.Fprintf(out, " [-]: %x -> %x\n", k, n)
}
for _, n := range set.leaves {
fmt.Fprintf(out, "[leaf]: %v\n", n)
}

View File

@ -563,7 +563,7 @@ func VerifyRangeProof(rootHash common.Hash, firstKey []byte, lastKey []byte, key
}
// Rebuild the trie with the leaf stream, the shape of trie
// should be same with the original one.
tr := &Trie{root: root, reader: newEmptyReader()}
tr := &Trie{root: root, reader: newEmptyReader(), tracer: newTracer()}
if empty {
tr.root = nil
}

125
trie/tracer.go Normal file
View File

@ -0,0 +1,125 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import "github.com/ethereum/go-ethereum/common"
// tracer tracks the changes of trie nodes. During the trie operations,
// some nodes can be deleted from the trie, while these deleted nodes
// won't be captured by trie.Hasher or trie.Committer. Thus, these deleted
// nodes won't be removed from the disk at all. Tracer is an auxiliary tool
// used to track all insert and delete operations of trie and capture all
// deleted nodes eventually.
//
// The changed nodes can be mainly divided into two categories: the leaf
// node and intermediate node. The former is inserted/deleted by callers
// while the latter is inserted/deleted in order to follow the rule of trie.
// This tool can track all of them no matter the node is embedded in its
// parent or not, but valueNode is never tracked.
//
// Besides, it's also used for recording the original value of the nodes
// when they are resolved from the disk. The pre-value of the nodes will
// be used to construct trie history in the future.
//
// Note tracer is not thread-safe, callers should be responsible for handling
// the concurrency issues by themselves.
type tracer struct {
inserts map[string]struct{}
deletes map[string]struct{}
accessList map[string][]byte
}
// newTracer initializes the tracer for capturing trie changes.
func newTracer() *tracer {
return &tracer{
inserts: make(map[string]struct{}),
deletes: make(map[string]struct{}),
accessList: make(map[string][]byte),
}
}
// onRead tracks the newly loaded trie node and caches the rlp-encoded
// blob internally. Don't change the value outside of function since
// it's not deep-copied.
func (t *tracer) onRead(path []byte, val []byte) {
t.accessList[string(path)] = val
}
// onInsert tracks the newly inserted trie node. If it's already
// in the deletion set (resurrected node), then just wipe it from
// the deletion set as it's "untouched".
func (t *tracer) onInsert(path []byte) {
if _, present := t.deletes[string(path)]; present {
delete(t.deletes, string(path))
return
}
t.inserts[string(path)] = struct{}{}
}
// onDelete tracks the newly deleted trie node. If it's already
// in the addition set, then just wipe it from the addition set
// as it's untouched.
func (t *tracer) onDelete(path []byte) {
if _, present := t.inserts[string(path)]; present {
delete(t.inserts, string(path))
return
}
t.deletes[string(path)] = struct{}{}
}
// reset clears the content tracked by tracer.
func (t *tracer) reset() {
t.inserts = make(map[string]struct{})
t.deletes = make(map[string]struct{})
t.accessList = make(map[string][]byte)
}
// copy returns a deep copied tracer instance.
func (t *tracer) copy() *tracer {
var (
inserts = make(map[string]struct{})
deletes = make(map[string]struct{})
accessList = make(map[string][]byte)
)
for path := range t.inserts {
inserts[path] = struct{}{}
}
for path := range t.deletes {
deletes[path] = struct{}{}
}
for path, blob := range t.accessList {
accessList[path] = common.CopyBytes(blob)
}
return &tracer{
inserts: inserts,
deletes: deletes,
accessList: accessList,
}
}
// markDeletions puts all tracked deletions into the provided nodeset.
func (t *tracer) markDeletions(set *NodeSet) {
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.
if _, ok := set.accessList[path]; !ok {
continue
}
set.markDeleted([]byte(path))
}
}

368
trie/tracer_test.go Normal file
View File

@ -0,0 +1,368 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import (
"bytes"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
)
var (
tiny = []struct{ k, v string }{
{"k1", "v1"},
{"k2", "v2"},
{"k3", "v3"},
}
nonAligned = []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"dog", "puppy"},
{"somethingveryoddindeedthis is", "myothernodedata"},
}
standard = []struct{ k, v string }{
{string(randBytes(32)), "verb"},
{string(randBytes(32)), "wookiedoo"},
{string(randBytes(32)), "stallion"},
{string(randBytes(32)), "horse"},
{string(randBytes(32)), "coin"},
{string(randBytes(32)), "puppy"},
{string(randBytes(32)), "myothernodedata"},
}
)
func TestTrieTracer(t *testing.T) {
testTrieTracer(t, tiny)
testTrieTracer(t, nonAligned)
testTrieTracer(t, standard)
}
// Tests if the trie diffs are tracked correctly. Tracer should capture
// all non-leaf dirty nodes, no matter the node is embedded or not.
func testTrieTracer(t *testing.T, vals []struct{ k, v string }) {
db := NewDatabase(rawdb.NewMemoryDatabase())
trie := NewEmpty(db)
// Determine all new nodes are tracked
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
insertSet := copySet(trie.tracer.inserts) // copy before commit
deleteSet := copySet(trie.tracer.deletes) // copy before commit
root, nodes := trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
seen := setKeys(iterNodes(db, root))
if !compareSet(insertSet, seen) {
t.Fatal("Unexpected insertion set")
}
if !compareSet(deleteSet, nil) {
t.Fatal("Unexpected deletion set")
}
// Determine all deletions are tracked
trie, _ = New(TrieID(root), db)
for _, val := range vals {
trie.Delete([]byte(val.k))
}
insertSet, deleteSet = copySet(trie.tracer.inserts), copySet(trie.tracer.deletes)
if !compareSet(insertSet, nil) {
t.Fatal("Unexpected insertion set")
}
if !compareSet(deleteSet, seen) {
t.Fatal("Unexpected deletion set")
}
}
// Test that after inserting a new batch of nodes and deleting them immediately,
// the trie tracer should be cleared normally as no operation happened.
func TestTrieTracerNoop(t *testing.T) {
testTrieTracerNoop(t, tiny)
testTrieTracerNoop(t, nonAligned)
testTrieTracerNoop(t, standard)
}
func testTrieTracerNoop(t *testing.T, vals []struct{ k, v string }) {
trie := NewEmpty(NewDatabase(rawdb.NewMemoryDatabase()))
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
for _, val := range vals {
trie.Delete([]byte(val.k))
}
if len(trie.tracer.inserts) != 0 {
t.Fatal("Unexpected insertion set")
}
if len(trie.tracer.deletes) != 0 {
t.Fatal("Unexpected deletion set")
}
}
// Tests if the accessList is correctly tracked.
func TestAccessList(t *testing.T) {
testAccessList(t, tiny)
testAccessList(t, nonAligned)
testAccessList(t, standard)
}
func testAccessList(t *testing.T, vals []struct{ k, v string }) {
var (
db = NewDatabase(rawdb.NewMemoryDatabase())
trie = NewEmpty(db)
orig = trie.Copy()
)
// Create trie from scratch
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
root, nodes := trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, nodes); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
// Update trie
trie, _ = New(TrieID(root), db)
orig = trie.Copy()
for _, val := range vals {
trie.Update([]byte(val.k), randBytes(32))
}
root, nodes = trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, nodes); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
// Add more new nodes
trie, _ = New(TrieID(root), db)
orig = trie.Copy()
var keys []string
for i := 0; i < 30; i++ {
key := randBytes(32)
keys = append(keys, string(key))
trie.Update(key, randBytes(32))
}
root, nodes = trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, nodes); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
// Partial deletions
trie, _ = New(TrieID(root), db)
orig = trie.Copy()
for _, key := range keys {
trie.Update([]byte(key), nil)
}
root, nodes = trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, nodes); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
// Delete all
trie, _ = New(TrieID(root), db)
orig = trie.Copy()
for _, val := range vals {
trie.Update([]byte(val.k), nil)
}
root, nodes = trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, nodes); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
}
// Tests origin values won't be tracked in Iterator or Prover
func TestAccessListLeak(t *testing.T) {
var (
db = NewDatabase(rawdb.NewMemoryDatabase())
trie = NewEmpty(db)
)
// Create trie from scratch
for _, val := range standard {
trie.Update([]byte(val.k), []byte(val.v))
}
root, nodes := trie.Commit(false)
db.Update(NewWithNodeSet(nodes))
var cases = []struct {
op func(tr *Trie)
}{
{
func(tr *Trie) {
it := tr.NodeIterator(nil)
for it.Next(true) {
}
},
},
{
func(tr *Trie) {
it := NewIterator(tr.NodeIterator(nil))
for it.Next() {
}
},
},
{
func(tr *Trie) {
for _, val := range standard {
tr.Prove([]byte(val.k), 0, rawdb.NewMemoryDatabase())
}
},
},
}
for _, c := range cases {
trie, _ = New(TrieID(root), db)
n1 := len(trie.tracer.accessList)
c.op(trie)
n2 := len(trie.tracer.accessList)
if n1 != n2 {
t.Fatalf("AccessList is leaked, prev %d after %d", n1, n2)
}
}
}
// Tests whether the original tree node is correctly deleted after being embedded
// in its parent due to the smaller size of the original tree node.
func TestTinyTree(t *testing.T) {
var (
db = NewDatabase(rawdb.NewMemoryDatabase())
trie = NewEmpty(db)
)
for _, val := range tiny {
trie.Update([]byte(val.k), randBytes(32))
}
root, set := trie.Commit(false)
db.Update(NewWithNodeSet(set))
trie, _ = New(TrieID(root), db)
orig := trie.Copy()
for _, val := range tiny {
trie.Update([]byte(val.k), []byte(val.v))
}
root, set = trie.Commit(false)
db.Update(NewWithNodeSet(set))
trie, _ = New(TrieID(root), db)
if err := verifyAccessList(orig, trie, set); err != nil {
t.Fatalf("Invalid accessList %v", err)
}
}
func compareSet(setA, setB map[string]struct{}) bool {
if len(setA) != len(setB) {
return false
}
for key := range setA {
if _, ok := setB[key]; !ok {
return false
}
}
return true
}
func forNodes(tr *Trie) map[string][]byte {
var (
it = tr.NodeIterator(nil)
nodes = make(map[string][]byte)
)
for it.Next(true) {
if it.Leaf() {
continue
}
nodes[string(it.Path())] = common.CopyBytes(it.NodeBlob())
}
return nodes
}
func iterNodes(db *Database, root common.Hash) map[string][]byte {
tr, _ := New(TrieID(root), db)
return forNodes(tr)
}
func forHashedNodes(tr *Trie) map[string][]byte {
var (
it = tr.NodeIterator(nil)
nodes = make(map[string][]byte)
)
for it.Next(true) {
if it.Hash() == (common.Hash{}) {
continue
}
nodes[string(it.Path())] = common.CopyBytes(it.NodeBlob())
}
return nodes
}
func diffTries(trieA, trieB *Trie) (map[string][]byte, map[string][]byte, map[string][]byte) {
var (
nodesA = forHashedNodes(trieA)
nodesB = forHashedNodes(trieB)
inA = make(map[string][]byte) // hashed nodes in trie a but not b
inB = make(map[string][]byte) // hashed nodes in trie b but not a
both = make(map[string][]byte) // hashed nodes in both tries but different value
)
for path, blobA := range nodesA {
if blobB, ok := nodesB[path]; ok {
if bytes.Equal(blobA, blobB) {
continue
}
both[path] = blobA
continue
}
inA[path] = blobA
}
for path, blobB := range nodesB {
if _, ok := nodesA[path]; ok {
continue
}
inB[path] = blobB
}
return inA, inB, both
}
func setKeys(set map[string][]byte) map[string]struct{} {
keys := make(map[string]struct{})
for k := range set {
keys[k] = struct{}{}
}
return keys
}
func copySet(set map[string]struct{}) map[string]struct{} {
copied := make(map[string]struct{})
for k := range set {
copied[k] = struct{}{}
}
return copied
}

View File

@ -81,7 +81,7 @@ func New(id *ID, db NodeReader) (*Trie, error) {
trie := &Trie{
owner: id.Owner,
reader: reader,
//tracer: newTracer(),
tracer: newTracer(),
}
if id.Root != (common.Hash{}) && id.Root != types.EmptyRootHash {
rootnode, err := trie.resolveAndTrack(id.Root[:], nil)
@ -547,7 +547,7 @@ func (t *Trie) resolveAndTrack(n hashNode, prefix []byte) (node, error) {
// Hash returns the root hash of the trie. It does not write to the
// database and can be used even if the trie doesn't have one.
func (t *Trie) Hash() common.Hash {
hash, cached, _ := t.hashRoot()
hash, cached := t.hashRoot()
t.root = cached
return common.BytesToHash(hash.(hashNode))
}
@ -561,14 +561,14 @@ func (t *Trie) Hash() common.Hash {
func (t *Trie) Commit(collectLeaf bool) (common.Hash, *NodeSet) {
defer t.tracer.reset()
nodes := NewNodeSet(t.owner, t.tracer.accessList)
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
if t.root == nil {
// Wrap tracked deletions as the return
set := NewNodeSet(t.owner)
t.tracer.markDeletions(set)
return types.EmptyRootHash, set
return types.EmptyRootHash, nodes
}
// Derive the hash for all dirty nodes first. We hold the assumption
// in the following procedure that all nodes are hashed.
@ -582,23 +582,23 @@ func (t *Trie) Commit(collectLeaf bool) (common.Hash, *NodeSet) {
t.root = hashedNode
return rootHash, nil
}
h := newCommitter(t.owner, t.tracer, collectLeaf)
newRoot, nodes := h.Commit(t.root)
t.root = newRoot
t.root = newCommitter(nodes, collectLeaf).Commit(t.root)
return rootHash, nodes
}
// hashRoot calculates the root hash of the given trie
func (t *Trie) hashRoot() (node, node, error) {
func (t *Trie) hashRoot() (node, node) {
if t.root == nil {
return hashNode(types.EmptyRootHash.Bytes()), nil, nil
return hashNode(types.EmptyRootHash.Bytes()), nil
}
// If the number of changes is below 100, we let one thread handle it
h := newHasher(t.unhashed >= 100)
defer returnHasherToPool(h)
defer func() {
returnHasherToPool(h)
t.unhashed = 0
}()
hashed, cached := h.hash(t.root, true)
t.unhashed = 0
return hashed, cached, nil
return hashed, cached
}
// Reset drops the referenced root node and cleans all internal state.

View File

@ -403,6 +403,51 @@ func (randTest) Generate(r *rand.Rand, size int) reflect.Value {
return reflect.ValueOf(steps)
}
func verifyAccessList(old *Trie, new *Trie, set *NodeSet) error {
deletes, inserts, updates := diffTries(old, new)
// Check insertion set
for path := range inserts {
n, ok := set.nodes[path]
if !ok || n.isDeleted() {
return errors.New("expect new node")
}
_, ok = set.accessList[path]
if ok {
return errors.New("unexpected origin value")
}
}
// Check deletion set
for path, blob := range deletes {
n, ok := set.nodes[path]
if !ok || !n.isDeleted() {
return errors.New("expect deleted node")
}
v, ok := set.accessList[path]
if !ok {
return errors.New("expect origin value")
}
if !bytes.Equal(v, blob) {
return errors.New("invalid origin value")
}
}
// Check update set
for path, blob := range updates {
n, ok := set.nodes[path]
if !ok || n.isDeleted() {
return errors.New("expect updated node")
}
v, ok := set.accessList[path]
if !ok {
return errors.New("expect origin value")
}
if !bytes.Equal(v, blob) {
return errors.New("invalid origin value")
}
}
return nil
}
func runRandTest(rt randTest) bool {
var (
triedb = NewDatabase(rawdb.NewMemoryDatabase())
@ -410,8 +455,6 @@ func runRandTest(rt randTest) bool {
values = make(map[string]string) // tracks content of the trie
origTrie = NewEmpty(triedb)
)
tr.tracer = newTracer()
for i, step := range rt {
// fmt.Printf("{op: %d, key: common.Hex2Bytes(\"%x\"), value: common.Hex2Bytes(\"%x\")}, // step %d\n",
// step.op, step.key, step.value, i)
@ -447,24 +490,6 @@ func runRandTest(rt randTest) bool {
tr.Hash()
case opCommit:
root, nodes := tr.Commit(true)
// Validity the returned nodeset
if nodes != nil {
for path, node := range nodes.updates.nodes {
blob, _, _ := origTrie.TryGetNode(hexToCompact([]byte(path)))
got := node.prev
if !bytes.Equal(blob, got) {
rt[i].err = fmt.Errorf("prevalue mismatch for 0x%x, got 0x%x want 0x%x", path, got, blob)
panic(rt[i].err)
}
}
for path, prev := range nodes.deletes {
blob, _, _ := origTrie.TryGetNode(hexToCompact([]byte(path)))
if !bytes.Equal(blob, prev) {
rt[i].err = fmt.Errorf("prevalue mismatch for 0x%x, got 0x%x want 0x%x", path, prev, blob)
return false
}
}
}
if nodes != nil {
triedb.Update(NewWithNodeSet(nodes))
}
@ -473,13 +498,13 @@ func runRandTest(rt randTest) bool {
rt[i].err = err
return false
}
if nodes != nil {
if err := verifyAccessList(origTrie, newtr, nodes); err != nil {
rt[i].err = err
return false
}
}
tr = newtr
// Enable node tracing. Resolve the root node again explicitly
// since it's not captured at the beginning.
tr.tracer = newTracer()
tr.resolveAndTrack(root.Bytes(), nil)
origTrie = tr.Copy()
case opItercheckhash:
checktr := NewEmpty(triedb)
@ -492,8 +517,6 @@ func runRandTest(rt randTest) bool {
}
case opNodeDiff:
var (
inserted = tr.tracer.insertList()
deleted = tr.tracer.deleteList()
origIter = origTrie.NodeIterator(nil)
curIter = tr.NodeIterator(nil)
origSeen = make(map[string]struct{})
@ -527,19 +550,19 @@ func runRandTest(rt randTest) bool {
deleteExp[path] = struct{}{}
}
}
if len(insertExp) != len(inserted) {
if len(insertExp) != len(tr.tracer.inserts) {
rt[i].err = fmt.Errorf("insert set mismatch")
}
if len(deleteExp) != len(deleted) {
if len(deleteExp) != len(tr.tracer.deletes) {
rt[i].err = fmt.Errorf("delete set mismatch")
}
for _, insert := range inserted {
if _, present := insertExp[string(insert)]; !present {
for insert := range tr.tracer.inserts {
if _, present := insertExp[insert]; !present {
rt[i].err = fmt.Errorf("missing inserted node")
}
}
for _, del := range deleted {
if _, present := deleteExp[string(del)]; !present {
for del := range tr.tracer.deletes {
if _, present := deleteExp[del]; !present {
rt[i].err = fmt.Errorf("missing deleted node")
}
}

View File

@ -1,305 +0,0 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import (
"bytes"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
)
// Tests if the trie diffs are tracked correctly.
func TestTrieTracer(t *testing.T) {
db := NewDatabase(rawdb.NewMemoryDatabase())
trie := NewEmpty(db)
trie.tracer = newTracer()
// Insert a batch of entries, all the nodes should be marked as inserted
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"dog", "puppy"},
{"somethingveryoddindeedthis is", "myothernodedata"},
}
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
trie.Hash()
seen := make(map[string]struct{})
it := trie.NodeIterator(nil)
for it.Next(true) {
if it.Leaf() {
continue
}
seen[string(it.Path())] = struct{}{}
}
inserted := trie.tracer.insertList()
if len(inserted) != len(seen) {
t.Fatalf("Unexpected inserted node tracked want %d got %d", len(seen), len(inserted))
}
for _, k := range inserted {
_, ok := seen[string(k)]
if !ok {
t.Fatalf("Unexpected inserted node")
}
}
deleted := trie.tracer.deleteList()
if len(deleted) != 0 {
t.Fatalf("Unexpected deleted node tracked %d", len(deleted))
}
// Commit the changes and re-create with new root
root, nodes := trie.Commit(false)
if err := db.Update(NewWithNodeSet(nodes)); err != nil {
t.Fatal(err)
}
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
// Delete all the elements, check deletion set
for _, val := range vals {
trie.Delete([]byte(val.k))
}
trie.Hash()
inserted = trie.tracer.insertList()
if len(inserted) != 0 {
t.Fatalf("Unexpected inserted node tracked %d", len(inserted))
}
deleted = trie.tracer.deleteList()
if len(deleted) != len(seen) {
t.Fatalf("Unexpected deleted node tracked want %d got %d", len(seen), len(deleted))
}
for _, k := range deleted {
_, ok := seen[string(k)]
if !ok {
t.Fatalf("Unexpected inserted node")
}
}
}
func TestTrieTracerNoop(t *testing.T) {
trie := NewEmpty(NewDatabase(rawdb.NewMemoryDatabase()))
trie.tracer = newTracer()
// Insert a batch of entries, all the nodes should be marked as inserted
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"dog", "puppy"},
{"somethingveryoddindeedthis is", "myothernodedata"},
}
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
for _, val := range vals {
trie.Delete([]byte(val.k))
}
if len(trie.tracer.insertList()) != 0 {
t.Fatalf("Unexpected inserted node tracked %d", len(trie.tracer.insertList()))
}
if len(trie.tracer.deleteList()) != 0 {
t.Fatalf("Unexpected deleted node tracked %d", len(trie.tracer.deleteList()))
}
}
func TestTrieTracePrevValue(t *testing.T) {
db := NewDatabase(rawdb.NewMemoryDatabase())
trie := NewEmpty(db)
trie.tracer = newTracer()
paths, blobs := trie.tracer.prevList()
if len(paths) != 0 || len(blobs) != 0 {
t.Fatalf("Nothing should be tracked")
}
// Insert a batch of entries, all the nodes should be marked as inserted
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"dog", "puppy"},
{"somethingveryoddindeedthis is", "myothernodedata"},
}
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
paths, blobs = trie.tracer.prevList()
if len(paths) != 0 || len(blobs) != 0 {
t.Fatalf("Nothing should be tracked")
}
// Commit the changes and re-create with new root
root, nodes := trie.Commit(false)
if err := db.Update(NewWithNodeSet(nodes)); err != nil {
t.Fatal(err)
}
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
trie.resolveAndTrack(root.Bytes(), nil)
// Load all nodes in trie
for _, val := range vals {
trie.TryGet([]byte(val.k))
}
// Ensure all nodes are tracked by tracer with correct prev-values
iter := trie.NodeIterator(nil)
seen := make(map[string][]byte)
for iter.Next(true) {
// Embedded nodes are ignored since they are not present in
// database.
if iter.Hash() == (common.Hash{}) {
continue
}
seen[string(iter.Path())] = common.CopyBytes(iter.NodeBlob())
}
paths, blobs = trie.tracer.prevList()
if len(paths) != len(seen) || len(blobs) != len(seen) {
t.Fatalf("Unexpected tracked values")
}
for i, path := range paths {
blob := blobs[i]
prev, ok := seen[string(path)]
if !ok {
t.Fatalf("Missing node %v", path)
}
if !bytes.Equal(blob, prev) {
t.Fatalf("Unexpected value path: %v, want: %v, got: %v", path, prev, blob)
}
}
// Re-open the trie and iterate the trie, ensure nothing will be tracked.
// Iterator will not link any loaded nodes to trie.
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
iter = trie.NodeIterator(nil)
for iter.Next(true) {
}
paths, blobs = trie.tracer.prevList()
if len(paths) != 0 || len(blobs) != 0 {
t.Fatalf("Nothing should be tracked")
}
// Re-open the trie and generate proof for entries, ensure nothing will
// be tracked. Prover will not link any loaded nodes to trie.
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
for _, val := range vals {
trie.Prove([]byte(val.k), 0, rawdb.NewMemoryDatabase())
}
paths, blobs = trie.tracer.prevList()
if len(paths) != 0 || len(blobs) != 0 {
t.Fatalf("Nothing should be tracked")
}
// Delete entries from trie, ensure all previous values are correct.
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
trie.resolveAndTrack(root.Bytes(), nil)
for _, val := range vals {
trie.TryDelete([]byte(val.k))
}
paths, blobs = trie.tracer.prevList()
if len(paths) != len(seen) || len(blobs) != len(seen) {
t.Fatalf("Unexpected tracked values")
}
for i, path := range paths {
blob := blobs[i]
prev, ok := seen[string(path)]
if !ok {
t.Fatalf("Missing node %v", path)
}
if !bytes.Equal(blob, prev) {
t.Fatalf("Unexpected value path: %v, want: %v, got: %v", path, prev, blob)
}
}
}
func TestDeleteAll(t *testing.T) {
db := NewDatabase(rawdb.NewMemoryDatabase())
trie := NewEmpty(db)
trie.tracer = newTracer()
// Insert a batch of entries, all the nodes should be marked as inserted
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"dog", "puppy"},
{"somethingveryoddindeedthis is", "myothernodedata"},
}
for _, val := range vals {
trie.Update([]byte(val.k), []byte(val.v))
}
root, set := trie.Commit(false)
if err := db.Update(NewWithNodeSet(set)); err != nil {
t.Fatal(err)
}
// Delete entries from trie, ensure all values are detected
trie, _ = New(TrieID(root), db)
trie.tracer = newTracer()
trie.resolveAndTrack(root.Bytes(), nil)
// Iterate all existent nodes
var (
it = trie.NodeIterator(nil)
nodes = make(map[string][]byte)
)
for it.Next(true) {
if it.Hash() != (common.Hash{}) {
nodes[string(it.Path())] = common.CopyBytes(it.NodeBlob())
}
}
// Perform deletion to purge the entire trie
for _, val := range vals {
trie.Delete([]byte(val.k))
}
root, set = trie.Commit(false)
if root != types.EmptyRootHash {
t.Fatalf("Invalid trie root %v", root)
}
for path, blob := range set.deletes {
prev, ok := nodes[path]
if !ok {
t.Fatalf("Extra node deleted %v", []byte(path))
}
if !bytes.Equal(prev, blob) {
t.Fatalf("Unexpected previous value %v", []byte(path))
}
}
if len(set.deletes) != len(nodes) {
t.Fatalf("Unexpected deletion set")
}
}

View File

@ -1,199 +0,0 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
// tracer tracks the changes of trie nodes. During the trie operations,
// some nodes can be deleted from the trie, while these deleted nodes
// won't be captured by trie.Hasher or trie.Committer. Thus, these deleted
// nodes won't be removed from the disk at all. Tracer is an auxiliary tool
// used to track all insert and delete operations of trie and capture all
// deleted nodes eventually.
//
// The changed nodes can be mainly divided into two categories: the leaf
// node and intermediate node. The former is inserted/deleted by callers
// while the latter is inserted/deleted in order to follow the rule of trie.
// This tool can track all of them no matter the node is embedded in its
// parent or not, but valueNode is never tracked.
//
// Besides, it's also used for recording the original value of the nodes
// when they are resolved from the disk. The pre-value of the nodes will
// be used to construct reverse-diffs in the future.
//
// Note tracer is not thread-safe, callers should be responsible for handling
// the concurrency issues by themselves.
type tracer struct {
insert map[string]struct{}
delete map[string]struct{}
origin map[string][]byte
}
// newTracer initializes the tracer for capturing trie changes.
func newTracer() *tracer {
return &tracer{
insert: make(map[string]struct{}),
delete: make(map[string]struct{}),
origin: make(map[string][]byte),
}
}
// onRead tracks the newly loaded trie node and caches the rlp-encoded blob internally.
// Don't change the value outside of function since it's not deep-copied.
func (t *tracer) onRead(path []byte, val []byte) {
// Tracer isn't used right now, remove this check later.
if t == nil {
return
}
t.origin[string(path)] = val
}
// onInsert tracks the newly inserted trie node. If it's already in the deletion set
// (resurrected node), then just wipe it from the deletion set as the "untouched".
func (t *tracer) onInsert(path []byte) {
// Tracer isn't used right now, remove this check later.
if t == nil {
return
}
if _, present := t.delete[string(path)]; present {
delete(t.delete, string(path))
return
}
t.insert[string(path)] = struct{}{}
}
// onDelete tracks the newly deleted trie node. If it's already
// in the addition set, then just wipe it from the addition set
// as it's untouched.
func (t *tracer) onDelete(path []byte) {
// Tracer isn't used right now, remove this check later.
if t == nil {
return
}
if _, present := t.insert[string(path)]; present {
delete(t.insert, string(path))
return
}
t.delete[string(path)] = struct{}{}
}
// insertList returns the tracked inserted trie nodes in list format.
func (t *tracer) insertList() [][]byte {
// Tracer isn't used right now, remove this check later.
if t == nil {
return nil
}
var ret [][]byte
for path := range t.insert {
ret = append(ret, []byte(path))
}
return ret
}
// deleteList returns the tracked deleted trie nodes in list format.
func (t *tracer) deleteList() [][]byte {
// Tracer isn't used right now, remove this check later.
if t == nil {
return nil
}
var ret [][]byte
for path := range t.delete {
ret = append(ret, []byte(path))
}
return ret
}
// prevList returns the tracked node blobs in list format.
func (t *tracer) prevList() ([][]byte, [][]byte) {
// Tracer isn't used right now, remove this check later.
if t == nil {
return nil, nil
}
var (
paths [][]byte
blobs [][]byte
)
for path, blob := range t.origin {
paths = append(paths, []byte(path))
blobs = append(blobs, blob)
}
return paths, blobs
}
// getPrev returns the cached original value of the specified node.
func (t *tracer) getPrev(path []byte) []byte {
// Tracer isn't used right now, remove this check later.
if t == nil {
return nil
}
return t.origin[string(path)]
}
// reset clears the content tracked by tracer.
func (t *tracer) reset() {
// Tracer isn't used right now, remove this check later.
if t == nil {
return
}
t.insert = make(map[string]struct{})
t.delete = make(map[string]struct{})
t.origin = make(map[string][]byte)
}
// copy returns a deep copied tracer instance.
func (t *tracer) copy() *tracer {
// Tracer isn't used right now, remove this check later.
if t == nil {
return nil
}
var (
insert = make(map[string]struct{})
delete = make(map[string]struct{})
origin = make(map[string][]byte)
)
for key := range t.insert {
insert[key] = struct{}{}
}
for key := range t.delete {
delete[key] = struct{}{}
}
for key, val := range t.origin {
origin[key] = val
}
return &tracer{
insert: insert,
delete: delete,
origin: origin,
}
}
// markDeletions puts all tracked deletions into the provided nodeset.
func (t *tracer) markDeletions(set *NodeSet) {
// Tracer isn't used right now, remove this check later.
if t == nil {
return
}
for _, path := range t.deleteList() {
// There are a few possibilities for this scenario(the node is deleted
// but not present in database previously), for example the node was
// embedded in the parent and now deleted from the trie. In this case
// it's noop from database's perspective.
val := t.getPrev(path)
if len(val) == 0 {
continue
}
set.markDeleted(path, val)
}
}