core/trie: remove trie tracer (#26665)
This PR contains a small portion of the full pbss PR, namely Remove the tracer from trie (and comitter), and instead using an accessList. Related changes to the Nodeset. --------- Co-authored-by: Gary Rong <garyrong0905@gmail.com>
This commit is contained in:
parent
15e5e6176b
commit
7c749c947a
@ -23,8 +23,6 @@ var (
|
||||
storageUpdatedMeter = metrics.NewRegisteredMeter("state/update/storage", nil)
|
||||
accountDeletedMeter = metrics.NewRegisteredMeter("state/delete/account", nil)
|
||||
storageDeletedMeter = metrics.NewRegisteredMeter("state/delete/storage", nil)
|
||||
accountTrieUpdatedMeter = metrics.NewRegisteredMeter("state/update/accountnodes", nil)
|
||||
storageTriesUpdatedMeter = metrics.NewRegisteredMeter("state/update/storagenodes", nil)
|
||||
accountTrieDeletedMeter = metrics.NewRegisteredMeter("state/delete/accountnodes", nil)
|
||||
storageTriesDeletedMeter = metrics.NewRegisteredMeter("state/delete/storagenodes", nil)
|
||||
accountTrieNodesMeter = metrics.NewRegisteredMeter("state/trie/account", nil)
|
||||
storageTriesNodesMeter = metrics.NewRegisteredMeter("state/trie/storage", nil)
|
||||
)
|
||||
|
@ -970,13 +970,11 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
|
||||
|
||||
// Commit objects to the trie, measuring the elapsed time
|
||||
var (
|
||||
accountTrieNodesUpdated int
|
||||
accountTrieNodesDeleted int
|
||||
storageTrieNodesUpdated int
|
||||
storageTrieNodesDeleted int
|
||||
accountTrieNodes int
|
||||
storageTrieNodes 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
|
||||
@ -994,17 +992,9 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
|
||||
if err := nodes.Merge(set); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
updates, deleted := set.Size()
|
||||
storageTrieNodesUpdated += updates
|
||||
storageTrieNodesDeleted += deleted
|
||||
storageTrieNodes += set.Size()
|
||||
}
|
||||
}
|
||||
// If the contract is destructed, the storage is still left in the
|
||||
// database as dangling data. Theoretically it's should be wiped from
|
||||
// database as well, but in hash-based-scheme it's extremely hard to
|
||||
// determine that if the trie nodes are also referenced by other storage,
|
||||
// and in path-based-scheme some technical challenges are still unsolved.
|
||||
// Although it won't affect the correctness but please fix it TODO(rjl493456442).
|
||||
}
|
||||
if len(s.stateObjectsDirty) > 0 {
|
||||
s.stateObjectsDirty = make(map[common.Address]struct{})
|
||||
@ -1025,7 +1015,7 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
|
||||
if err := nodes.Merge(set); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
accountTrieNodesUpdated, accountTrieNodesDeleted = set.Size()
|
||||
accountTrieNodes = set.Size()
|
||||
}
|
||||
if metrics.EnabledExpensive {
|
||||
s.AccountCommits += time.Since(start)
|
||||
@ -1034,10 +1024,9 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
|
||||
storageUpdatedMeter.Mark(int64(s.StorageUpdated))
|
||||
accountDeletedMeter.Mark(int64(s.AccountDeleted))
|
||||
storageDeletedMeter.Mark(int64(s.StorageDeleted))
|
||||
accountTrieUpdatedMeter.Mark(int64(accountTrieNodesUpdated))
|
||||
accountTrieDeletedMeter.Mark(int64(accountTrieNodesDeleted))
|
||||
storageTriesUpdatedMeter.Mark(int64(storageTrieNodesUpdated))
|
||||
storageTriesDeletedMeter.Mark(int64(storageTrieNodesDeleted))
|
||||
accountTrieNodesMeter.Mark(int64(accountTrieNodes))
|
||||
storageTriesNodesMeter.Mark(int64(storageTrieNodes))
|
||||
|
||||
s.AccountUpdated, s.AccountDeleted = 0, 0
|
||||
s.StorageUpdated, s.StorageDeleted = 0, 0
|
||||
}
|
||||
|
@ -33,29 +33,22 @@ 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(nodes *NodeSet, collectLeaf bool) *committer {
|
||||
return &committer{
|
||||
nodes: NewNodeSet(owner),
|
||||
tracer: tracer,
|
||||
nodes: nodes,
|
||||
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) {
|
||||
func (c *committer) Commit(n node) hashNode {
|
||||
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
|
||||
return h.(hashNode)
|
||||
}
|
||||
|
||||
// commit collapses a node down into a hash node and returns it.
|
||||
@ -85,12 +78,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 +88,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
|
||||
@ -169,7 +150,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
|
||||
|
@ -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(false, 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.
|
||||
|
114
trie/nodeset.go
114
trie/nodeset.go
@ -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,14 +65,20 @@ 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
|
||||
}
|
||||
|
||||
// unwrap returns the internal memoryNode object.
|
||||
// nolint:unused
|
||||
// nolint: unused
|
||||
func (n *nodeWithPrev) unwrap() *memoryNode {
|
||||
return n.memoryNode
|
||||
}
|
||||
@ -79,64 +86,69 @@ 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
|
||||
nodes map[string]*memoryNode // the set of dirty nodes(inserted, updated, deleted)
|
||||
leaves []*leaf // the list of dirty leaves
|
||||
accessList map[string][]byte // The list of accessed nodes, which records the original node value
|
||||
}
|
||||
|
||||
// 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 {
|
||||
// for a specific account or storage trie. The owner is zero for the account trie
|
||||
// and the owning account address hash for storage tries. The provided accessList
|
||||
// represents the original value of accessed nodes, it can be optional but would
|
||||
// be beneficial for speeding up the construction of trie history.
|
||||
func NewNodeSet(owner common.Hash, accessList map[string][]byte) *NodeSet {
|
||||
// Don't panic for lazy users
|
||||
if accessList == nil {
|
||||
accessList = make(map[string][]byte)
|
||||
}
|
||||
return &NodeSet{
|
||||
owner: owner,
|
||||
updates: &nodesWithOrder{
|
||||
nodes: make(map[string]*nodeWithPrev),
|
||||
},
|
||||
deletes: make(map[string][]byte),
|
||||
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 specified order.
|
||||
// If topToBottom is true:
|
||||
//
|
||||
// then the order of iteration is top to bottom, left to right.
|
||||
//
|
||||
// If topToBottom is false:
|
||||
//
|
||||
// then the order of iteration is bottom to top, right to left.
|
||||
func (set *NodeSet) forEachWithOrder(topToBottom bool, callback func(path string, n *memoryNode)) {
|
||||
var paths sort.StringSlice
|
||||
for path := range set.nodes {
|
||||
paths = append(paths, path)
|
||||
}
|
||||
if topToBottom {
|
||||
paths.Sort()
|
||||
} else {
|
||||
sort.Sort(sort.Reverse(paths))
|
||||
}
|
||||
for _, path := range paths {
|
||||
callback(path, set.nodes[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,
|
||||
}
|
||||
func (set *NodeSet) markUpdated(path []byte, node *memoryNode) {
|
||||
set.nodes[string(path)] = node
|
||||
}
|
||||
|
||||
// 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
|
||||
// nolint: unused
|
||||
func (set *NodeSet) markDeleted(path []byte) {
|
||||
set.nodes[string(path)] = &memoryNode{}
|
||||
}
|
||||
|
||||
// addLeaf collects the provided leaf node into set.
|
||||
@ -144,16 +156,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.
|
||||
func (set *NodeSet) Size() (int, int) {
|
||||
return len(set.updates.order), len(set.deletes)
|
||||
// Size returns the number of dirty nodes contained in the set.
|
||||
func (set *NodeSet) Size() int {
|
||||
return len(set.nodes)
|
||||
}
|
||||
|
||||
// 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 +175,17 @@ 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\n", path)
|
||||
continue
|
||||
}
|
||||
// Update
|
||||
fmt.Fprintf(out, " [+]: %x -> %v\n", path, n.hash)
|
||||
}
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
59
trie/trie.go
59
trie/trie.go
@ -51,12 +51,11 @@ type Trie struct {
|
||||
// actually unhashed nodes.
|
||||
unhashed int
|
||||
|
||||
// accessList tracks the loaded nodes from database.
|
||||
accessList map[string][]byte
|
||||
|
||||
// reader is the handler trie can retrieve nodes from.
|
||||
reader *trieReader
|
||||
|
||||
// tracer is the tool to track the trie changes.
|
||||
// It will be reset after each commit operation.
|
||||
tracer *tracer
|
||||
}
|
||||
|
||||
// newFlag returns the cache flag value for a newly created node.
|
||||
@ -66,12 +65,16 @@ func (t *Trie) newFlag() nodeFlag {
|
||||
|
||||
// Copy returns a copy of Trie.
|
||||
func (t *Trie) Copy() *Trie {
|
||||
accessList := make(map[string][]byte)
|
||||
for path, blob := range t.accessList {
|
||||
accessList[path] = common.CopyBytes(blob)
|
||||
}
|
||||
return &Trie{
|
||||
root: t.root,
|
||||
owner: t.owner,
|
||||
unhashed: t.unhashed,
|
||||
reader: t.reader,
|
||||
tracer: t.tracer.copy(),
|
||||
accessList: accessList,
|
||||
}
|
||||
}
|
||||
|
||||
@ -89,7 +92,7 @@ func New(id *ID, db NodeReader) (*Trie, error) {
|
||||
trie := &Trie{
|
||||
owner: id.Owner,
|
||||
reader: reader,
|
||||
//tracer: newTracer(),
|
||||
accessList: make(map[string][]byte),
|
||||
}
|
||||
if id.Root != (common.Hash{}) && id.Root != emptyRoot {
|
||||
rootnode, err := trie.resolveAndTrack(id.Root[:], nil)
|
||||
@ -326,11 +329,6 @@ func (t *Trie) insert(n node, prefix, key []byte, value node) (bool, node, error
|
||||
if matchlen == 0 {
|
||||
return true, branch, nil
|
||||
}
|
||||
// New branch node is created as a child of the original short node.
|
||||
// Track the newly inserted node in the tracer. The node identifier
|
||||
// passed is the path from the root node.
|
||||
t.tracer.onInsert(append(prefix, key[:matchlen]...))
|
||||
|
||||
// Replace it with a short node leading up to the branch.
|
||||
return true, &shortNode{key[:matchlen], branch, t.newFlag()}, nil
|
||||
|
||||
@ -345,11 +343,6 @@ func (t *Trie) insert(n node, prefix, key []byte, value node) (bool, node, error
|
||||
return true, n, nil
|
||||
|
||||
case nil:
|
||||
// New short node is created and track it in the tracer. The node identifier
|
||||
// passed is the path from the root node. Note the valueNode won't be tracked
|
||||
// since it's always embedded in its parent.
|
||||
t.tracer.onInsert(prefix)
|
||||
|
||||
return true, &shortNode{key, value, t.newFlag()}, nil
|
||||
|
||||
case hashNode:
|
||||
@ -402,11 +395,6 @@ func (t *Trie) delete(n node, prefix, key []byte) (bool, node, error) {
|
||||
return false, n, nil // don't replace n on mismatch
|
||||
}
|
||||
if matchlen == len(key) {
|
||||
// The matched short node is deleted entirely and track
|
||||
// it in the deletion set. The same the valueNode doesn't
|
||||
// need to be tracked at all since it's always embedded.
|
||||
t.tracer.onDelete(prefix)
|
||||
|
||||
return true, nil, nil // remove n entirely for whole matches
|
||||
}
|
||||
// The key is longer than n.Key. Remove the remaining suffix
|
||||
@ -419,10 +407,6 @@ func (t *Trie) delete(n node, prefix, key []byte) (bool, node, error) {
|
||||
}
|
||||
switch child := child.(type) {
|
||||
case *shortNode:
|
||||
// The child shortNode is merged into its parent, track
|
||||
// is deleted as well.
|
||||
t.tracer.onDelete(append(prefix, n.Key...))
|
||||
|
||||
// Deleting from the subtrie reduced it to another
|
||||
// short node. Merge the nodes to avoid creating a
|
||||
// shortNode{..., shortNode{...}}. Use concat (which
|
||||
@ -484,11 +468,6 @@ func (t *Trie) delete(n node, prefix, key []byte) (bool, node, error) {
|
||||
return false, nil, err
|
||||
}
|
||||
if cnode, ok := cnode.(*shortNode); ok {
|
||||
// Replace the entire full node with the short node.
|
||||
// Mark the original short node as deleted since the
|
||||
// value is embedded into the parent now.
|
||||
t.tracer.onDelete(append(prefix, byte(pos)))
|
||||
|
||||
k := append([]byte{byte(pos)}, cnode.Key...)
|
||||
return true, &shortNode{k, cnode.Val, t.newFlag()}, nil
|
||||
}
|
||||
@ -548,7 +527,7 @@ func (t *Trie) resolveAndTrack(n hashNode, prefix []byte) (node, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
t.tracer.onRead(prefix, blob)
|
||||
t.accessList[string(prefix)] = blob
|
||||
return mustDecodeNode(n, blob), nil
|
||||
}
|
||||
|
||||
@ -567,16 +546,15 @@ func (t *Trie) Hash() common.Hash {
|
||||
// Once the trie is committed, it's not usable anymore. A new trie must
|
||||
// be created with new root and updated trie database for following usage
|
||||
func (t *Trie) Commit(collectLeaf bool) (common.Hash, *NodeSet) {
|
||||
defer t.tracer.reset()
|
||||
|
||||
// Reset accessList at the end of commit
|
||||
defer func() {
|
||||
t.accessList = make(map[string][]byte)
|
||||
}()
|
||||
// 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 emptyRoot, set
|
||||
return emptyRoot, nil
|
||||
}
|
||||
// Derive the hash for all dirty nodes first. We hold the assumption
|
||||
// in the following procedure that all nodes are hashed.
|
||||
@ -590,8 +568,9 @@ 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)
|
||||
nodes := NewNodeSet(t.owner, t.accessList)
|
||||
h := newCommitter(nodes, collectLeaf)
|
||||
newRoot := h.Commit(t.root)
|
||||
t.root = newRoot
|
||||
return rootHash, nodes
|
||||
}
|
||||
@ -614,5 +593,5 @@ func (t *Trie) Reset() {
|
||||
t.root = nil
|
||||
t.owner = common.Hash{}
|
||||
t.unhashed = 0
|
||||
t.tracer.reset()
|
||||
t.accessList = make(map[string][]byte)
|
||||
}
|
||||
|
@ -410,7 +410,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",
|
||||
@ -449,21 +448,14 @@ func runRandTest(rt randTest) bool {
|
||||
root, nodes := tr.Commit(true)
|
||||
// Validity the returned nodeset
|
||||
if nodes != nil {
|
||||
for path, node := range nodes.updates.nodes {
|
||||
for path := range nodes.nodes {
|
||||
blob, _, _ := origTrie.TryGetNode(hexToCompact([]byte(path)))
|
||||
got := node.prev
|
||||
got := nodes.accessList[path]
|
||||
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))
|
||||
@ -477,7 +469,6 @@ func runRandTest(rt randTest) bool {
|
||||
|
||||
// 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()
|
||||
@ -490,59 +481,6 @@ func runRandTest(rt randTest) bool {
|
||||
if tr.Hash() != checktr.Hash() {
|
||||
rt[i].err = fmt.Errorf("hash mismatch in opItercheckhash")
|
||||
}
|
||||
case opNodeDiff:
|
||||
var (
|
||||
inserted = tr.tracer.insertList()
|
||||
deleted = tr.tracer.deleteList()
|
||||
origIter = origTrie.NodeIterator(nil)
|
||||
curIter = tr.NodeIterator(nil)
|
||||
origSeen = make(map[string]struct{})
|
||||
curSeen = make(map[string]struct{})
|
||||
)
|
||||
for origIter.Next(true) {
|
||||
if origIter.Leaf() {
|
||||
continue
|
||||
}
|
||||
origSeen[string(origIter.Path())] = struct{}{}
|
||||
}
|
||||
for curIter.Next(true) {
|
||||
if curIter.Leaf() {
|
||||
continue
|
||||
}
|
||||
curSeen[string(curIter.Path())] = struct{}{}
|
||||
}
|
||||
var (
|
||||
insertExp = make(map[string]struct{})
|
||||
deleteExp = make(map[string]struct{})
|
||||
)
|
||||
for path := range curSeen {
|
||||
_, present := origSeen[path]
|
||||
if !present {
|
||||
insertExp[path] = struct{}{}
|
||||
}
|
||||
}
|
||||
for path := range origSeen {
|
||||
_, present := curSeen[path]
|
||||
if !present {
|
||||
deleteExp[path] = struct{}{}
|
||||
}
|
||||
}
|
||||
if len(insertExp) != len(inserted) {
|
||||
rt[i].err = fmt.Errorf("insert set mismatch")
|
||||
}
|
||||
if len(deleteExp) != len(deleted) {
|
||||
rt[i].err = fmt.Errorf("delete set mismatch")
|
||||
}
|
||||
for _, insert := range inserted {
|
||||
if _, present := insertExp[string(insert)]; !present {
|
||||
rt[i].err = fmt.Errorf("missing inserted node")
|
||||
}
|
||||
}
|
||||
for _, del := range deleted {
|
||||
if _, present := deleteExp[string(del)]; !present {
|
||||
rt[i].err = fmt.Errorf("missing deleted node")
|
||||
}
|
||||
}
|
||||
}
|
||||
// Abort the test on error.
|
||||
if rt[i].err != nil {
|
||||
|
@ -1,304 +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"
|
||||
)
|
||||
|
||||
// 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 != emptyRoot {
|
||||
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")
|
||||
}
|
||||
}
|
199
trie/utils.go
199
trie/utils.go
@ -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)
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue
Block a user