core: use TryGetAccount to read what TryUpdateAccount has written (#25458)

* core: use TryGetAccount to read where TryUpdateAccount has been used to write

* Gary's review feedback

* implement Gary's suggestion

* fix bug + rename NewSecure into NewStateTrie

* trie: add backwards-compatibility aliases for SecureTrie

* Update database.go

* make the linter happy

Co-authored-by: Felix Lange <fjl@twurst.com>
Co-authored-by: rjl493456442 <garyrong0905@gmail.com>
This commit is contained in:
Guillaume Ballet 2022-08-04 16:13:18 +02:00 committed by GitHub
parent 8b53b92eb4
commit f67e54c92f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 142 additions and 88 deletions

View File

@ -271,7 +271,7 @@ func traverseState(ctx *cli.Context) error {
log.Info("Start traversing the state", "root", root, "number", headBlock.NumberU64())
}
triedb := trie.NewDatabase(chaindb)
t, err := trie.NewSecure(common.Hash{}, root, triedb)
t, err := trie.NewStateTrie(common.Hash{}, root, triedb)
if err != nil {
log.Error("Failed to open trie", "root", root, "err", err)
return err
@ -292,7 +292,7 @@ func traverseState(ctx *cli.Context) error {
return err
}
if acc.Root != emptyRoot {
storageTrie, err := trie.NewSecure(common.BytesToHash(accIter.Key), acc.Root, triedb)
storageTrie, err := trie.NewStateTrie(common.BytesToHash(accIter.Key), acc.Root, triedb)
if err != nil {
log.Error("Failed to open storage trie", "root", acc.Root, "err", err)
return err
@ -360,7 +360,7 @@ func traverseRawState(ctx *cli.Context) error {
log.Info("Start traversing the state", "root", root, "number", headBlock.NumberU64())
}
triedb := trie.NewDatabase(chaindb)
t, err := trie.NewSecure(common.Hash{}, root, triedb)
t, err := trie.NewStateTrie(common.Hash{}, root, triedb)
if err != nil {
log.Error("Failed to open trie", "root", root, "err", err)
return err
@ -406,7 +406,7 @@ func traverseRawState(ctx *cli.Context) error {
return errors.New("invalid account")
}
if acc.Root != emptyRoot {
storageTrie, err := trie.NewSecure(common.BytesToHash(accIter.LeafKey()), acc.Root, triedb)
storageTrie, err := trie.NewStateTrie(common.BytesToHash(accIter.LeafKey()), acc.Root, triedb)
if err != nil {
log.Error("Failed to open storage trie", "root", acc.Root, "err", err)
return errors.New("missing storage trie")

View File

@ -706,7 +706,7 @@ func (bc *BlockChain) SnapSyncCommitHead(hash common.Hash) error {
if block == nil {
return fmt.Errorf("non existent block [%x..]", hash[:4])
}
if _, err := trie.NewSecure(common.Hash{}, block.Root(), bc.stateCache.TrieDB()); err != nil {
if _, err := trie.NewStateTrie(common.Hash{}, block.Root(), bc.stateCache.TrieDB()); err != nil {
return err
}

View File

@ -63,7 +63,7 @@ type Trie interface {
// GetKey returns the sha3 preimage of a hashed key that was previously used
// to store a value.
//
// TODO(fjl): remove this when SecureTrie is removed
// TODO(fjl): remove this when StateTrie is removed
GetKey([]byte) []byte
// TryGet returns the value for key stored in the trie. The value bytes must
@ -71,8 +71,8 @@ type Trie interface {
// trie.MissingNodeError is returned.
TryGet(key []byte) ([]byte, error)
// TryUpdateAccount abstract an account write in the trie.
TryUpdateAccount(key []byte, account *types.StateAccount) error
// TryGetAccount abstract an account read from the trie.
TryGetAccount(key []byte) (*types.StateAccount, error)
// TryUpdate associates key with value in the trie. If value has length zero, any
// existing value is deleted from the trie. The value bytes must not be modified
@ -80,6 +80,9 @@ type Trie interface {
// database, a trie.MissingNodeError is returned.
TryUpdate(key, value []byte) error
// TryUpdateAccount abstract an account write to the trie.
TryUpdateAccount(key []byte, account *types.StateAccount) error
// TryDelete removes any existing value for key from the trie. If a node was not
// found in the database, a trie.MissingNodeError is returned.
TryDelete(key []byte) error
@ -137,7 +140,7 @@ type cachingDB struct {
// OpenTrie opens the main account trie at a specific root hash.
func (db *cachingDB) OpenTrie(root common.Hash) (Trie, error) {
tr, err := trie.NewSecure(common.Hash{}, root, db.db)
tr, err := trie.NewStateTrie(common.Hash{}, root, db.db)
if err != nil {
return nil, err
}
@ -146,7 +149,7 @@ func (db *cachingDB) OpenTrie(root common.Hash) (Trie, error) {
// OpenStorageTrie opens the storage trie of an account.
func (db *cachingDB) OpenStorageTrie(addrHash, root common.Hash) (Trie, error) {
tr, err := trie.NewSecure(addrHash, root, db.db)
tr, err := trie.NewStateTrie(addrHash, root, db.db)
if err != nil {
return nil, err
}
@ -156,7 +159,7 @@ func (db *cachingDB) OpenStorageTrie(addrHash, root common.Hash) (Trie, error) {
// CopyTrie returns an independent copy of the given trie.
func (db *cachingDB) CopyTrie(t Trie) Trie {
switch t := t.(type) {
case *trie.SecureTrie:
case *trie.StateTrie:
return t.Copy()
default:
panic(fmt.Errorf("unknown trie type %T", t))

View File

@ -410,7 +410,7 @@ func extractGenesis(db ethdb.Database, stateBloom *stateBloom) error {
if genesis == nil {
return errors.New("missing genesis block")
}
t, err := trie.NewSecure(common.Hash{}, genesis.Root(), trie.NewDatabase(db))
t, err := trie.NewStateTrie(common.Hash{}, genesis.Root(), trie.NewDatabase(db))
if err != nil {
return err
}
@ -430,7 +430,7 @@ func extractGenesis(db ethdb.Database, stateBloom *stateBloom) error {
return err
}
if acc.Root != emptyRoot {
storageTrie, err := trie.NewSecure(common.BytesToHash(accIter.LeafKey()), acc.Root, trie.NewDatabase(db))
storageTrie, err := trie.NewStateTrie(common.BytesToHash(accIter.LeafKey()), acc.Root, trie.NewDatabase(db))
if err != nil {
return err
}

View File

@ -142,14 +142,14 @@ func checkSnapRoot(t *testing.T, snap *diskLayer, trieRoot common.Hash) {
type testHelper struct {
diskdb ethdb.Database
triedb *trie.Database
accTrie *trie.SecureTrie
accTrie *trie.StateTrie
nodes *trie.MergedNodeSet
}
func newHelper() *testHelper {
diskdb := rawdb.NewMemoryDatabase()
triedb := trie.NewDatabase(diskdb)
accTrie, _ := trie.NewSecure(common.Hash{}, common.Hash{}, triedb)
accTrie, _ := trie.NewStateTrie(common.Hash{}, common.Hash{}, triedb)
return &testHelper{
diskdb: diskdb,
triedb: triedb,
@ -182,7 +182,7 @@ func (t *testHelper) addSnapStorage(accKey string, keys []string, vals []string)
}
func (t *testHelper) makeStorageTrie(stateRoot, owner common.Hash, keys []string, vals []string, commit bool) []byte {
stTrie, _ := trie.NewSecure(owner, common.Hash{}, t.triedb)
stTrie, _ := trie.NewStateTrie(owner, common.Hash{}, t.triedb)
for i, k := range keys {
stTrie.Update([]byte(k), []byte(vals[i]))
}

View File

@ -537,20 +537,16 @@ func (s *StateDB) getDeletedStateObject(addr common.Address) *stateObject {
// If snapshot unavailable or reading from it failed, load from the database
if data == nil {
start := time.Now()
enc, err := s.trie.TryGet(addr.Bytes())
var err error
data, err = s.trie.TryGetAccount(addr.Bytes())
if metrics.EnabledExpensive {
s.AccountReads += time.Since(start)
}
if err != nil {
s.setError(fmt.Errorf("getDeleteStateObject (%x) error: %v", addr.Bytes(), err))
s.setError(fmt.Errorf("getDeleteStateObject (%x) error: %w", addr.Bytes(), err))
return nil
}
if len(enc) == 0 {
return nil
}
data = new(types.StateAccount)
if err := rlp.DecodeBytes(enc, data); err != nil {
log.Error("Failed to decode state object", "addr", addr, "err", err)
if data == nil {
return nil
}
}

View File

@ -331,7 +331,11 @@ func (sf *subfetcher) loop() {
if _, ok := sf.seen[string(task)]; ok {
sf.dups++
} else {
sf.trie.TryGet(task)
if len(task) == len(common.Address{}) {
sf.trie.TryGetAccount(task)
} else {
sf.trie.TryGet(task)
}
sf.seen[string(task)] = struct{}{}
}
}

View File

@ -506,11 +506,11 @@ func (api *DebugAPI) getModifiedAccounts(startBlock, endBlock *types.Block) ([]c
}
triedb := api.eth.BlockChain().StateCache().TrieDB()
oldTrie, err := trie.NewSecure(common.Hash{}, startBlock.Root(), triedb)
oldTrie, err := trie.NewStateTrie(common.Hash{}, startBlock.Root(), triedb)
if err != nil {
return nil, err
}
newTrie, err := trie.NewSecure(common.Hash{}, endBlock.Root(), triedb)
newTrie, err := trie.NewStateTrie(common.Hash{}, endBlock.Root(), triedb)
if err != nil {
return nil, err
}

View File

@ -23,14 +23,12 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/light"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie"
)
@ -415,15 +413,15 @@ func ServiceGetStorageRangesQuery(chain *core.BlockChain, req *GetStorageRangesP
if origin != (common.Hash{}) || (abort && len(storage) > 0) {
// Request started at a non-zero hash or was capped prematurely, add
// the endpoint Merkle proofs
accTrie, err := trie.New(common.Hash{}, req.Root, chain.StateCache().TrieDB())
accTrie, err := trie.NewStateTrie(common.Hash{}, req.Root, chain.StateCache().TrieDB())
if err != nil {
return nil, nil
}
var acc types.StateAccount
if err := rlp.DecodeBytes(accTrie.Get(account[:]), &acc); err != nil {
acc, err := accTrie.TryGetAccountWithPreHashedKey(account[:])
if err != nil || acc == nil {
return nil, nil
}
stTrie, err := trie.New(account, acc.Root, chain.StateCache().TrieDB())
stTrie, err := trie.NewStateTrie(account, acc.Root, chain.StateCache().TrieDB())
if err != nil {
return nil, nil
}
@ -489,7 +487,7 @@ func ServiceGetTrieNodesQuery(chain *core.BlockChain, req *GetTrieNodesPacket, s
// Make sure we have the state associated with the request
triedb := chain.StateCache().TrieDB()
accTrie, err := trie.NewSecure(common.Hash{}, req.Root, triedb)
accTrie, err := trie.NewStateTrie(common.Hash{}, req.Root, triedb)
if err != nil {
// We don't have the requested state available, bail out
return nil, nil
@ -531,7 +529,7 @@ func ServiceGetTrieNodesQuery(chain *core.BlockChain, req *GetTrieNodesPacket, s
if err != nil || account == nil {
break
}
stTrie, err := trie.NewSecure(common.BytesToHash(pathset[0]), common.BytesToHash(account.Root), triedb)
stTrie, err := trie.NewStateTrie(common.BytesToHash(pathset[0]), common.BytesToHash(account.Root), triedb)
loads++ // always account database reads, even for failures
if err != nil {
break

View File

@ -1658,7 +1658,7 @@ func verifyTrie(db ethdb.KeyValueStore, root common.Hash, t *testing.T) {
}
accounts++
if acc.Root != emptyRoot {
storeTrie, err := trie.NewSecure(common.BytesToHash(accIt.Key), acc.Root, triedb)
storeTrie, err := trie.NewStateTrie(common.BytesToHash(accIt.Key), acc.Root, triedb)
if err != nil {
t.Fatal(err)
}

View File

@ -229,7 +229,7 @@ func (dl *downloadTester) CurrentFastBlock() *types.Block {
func (dl *downloadTester) FastSyncCommitHead(hash common.Hash) error {
// For now only check that the state trie is correct
if block := dl.GetBlockByHash(hash); block != nil {
_, err := trie.NewSecure(common.Hash{}, block.Root(), trie.NewDatabase(dl.stateDb))
_, err := trie.NewStateTrie(common.Hash{}, block.Root(), trie.NewDatabase(dl.stateDb))
return err
}
return fmt.Errorf("non existent block: %x", hash[:4])

View File

@ -112,6 +112,22 @@ func (t *odrTrie) TryGet(key []byte) ([]byte, error) {
return res, err
}
func (t *odrTrie) TryGetAccount(key []byte) (*types.StateAccount, error) {
key = crypto.Keccak256(key)
var res types.StateAccount
err := t.do(key, func() (err error) {
value, err := t.trie.TryGet(key)
if err != nil {
return err
}
if value == nil {
return nil
}
return rlp.DecodeBytes(value, &res)
})
return &res, err
}
func (t *odrTrie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
key = crypto.Keccak256(key)
value, err := rlp.EncodeToBytes(acc)

View File

@ -529,11 +529,11 @@ func (l *loggingDb) Close() error {
}
// makeLargeTestTrie create a sample test trie
func makeLargeTestTrie() (*Database, *SecureTrie, *loggingDb) {
func makeLargeTestTrie() (*Database, *StateTrie, *loggingDb) {
// Create an empty trie
logDb := &loggingDb{0, memorydb.New()}
triedb := NewDatabase(logDb)
trie, _ := NewSecure(common.Hash{}, common.Hash{}, triedb)
trie, _ := NewStateTrie(common.Hash{}, common.Hash{}, triedb)
// Fill it with some arbitrary data
for i := 0; i < 10000; i++ {

View File

@ -100,7 +100,7 @@ func (t *Trie) Prove(key []byte, fromLevel uint, proofDb ethdb.KeyValueWriter) e
// If the trie does not contain a value for key, the returned proof contains all
// nodes of the longest existing prefix of the key (at least the root node), ending
// with the node that proves the absence of the key.
func (t *SecureTrie) Prove(key []byte, fromLevel uint, proofDb ethdb.KeyValueWriter) error {
func (t *StateTrie) Prove(key []byte, fromLevel uint, proofDb ethdb.KeyValueWriter) error {
return t.trie.Prove(key, fromLevel, proofDb)
}

View File

@ -25,25 +25,35 @@ import (
"github.com/ethereum/go-ethereum/rlp"
)
// SecureTrie wraps a trie with key hashing. In a secure trie, all
// SecureTrie is the old name of StateTrie.
// Deprecated: use StateTrie.
type SecureTrie = StateTrie
// NewSecure creates a new StateTrie.
// Deprecated: use NewStateTrie.
func NewSecure(owner common.Hash, root common.Hash, db *Database) (*SecureTrie, error) {
return NewStateTrie(owner, root, db)
}
// StateTrie wraps a trie with key hashing. In a secure trie, all
// access operations hash the key using keccak256. This prevents
// calling code from creating long chains of nodes that
// increase the access time.
//
// Contrary to a regular trie, a SecureTrie can only be created with
// Contrary to a regular trie, a StateTrie can only be created with
// New and must have an attached database. The database also stores
// the preimage of each key.
//
// SecureTrie is not safe for concurrent use.
type SecureTrie struct {
// StateTrie is not safe for concurrent use.
type StateTrie struct {
trie Trie
preimages *preimageStore
hashKeyBuf [common.HashLength]byte
secKeyCache map[string][]byte
secKeyCacheOwner *SecureTrie // Pointer to self, replace the key cache on mismatch
secKeyCacheOwner *StateTrie // Pointer to self, replace the key cache on mismatch
}
// NewSecure creates a trie with an existing root node from a backing database
// NewStateTrie creates a trie with an existing root node from a backing database
// and optional intermediate in-memory node pool.
//
// If root is the zero hash or the sha3 hash of an empty string, the
@ -54,7 +64,7 @@ type SecureTrie struct {
// Loaded nodes are kept around until their 'cache generation' expires.
// A new cache generation is created by each call to Commit.
// cachelimit sets the number of past cache generations to keep.
func NewSecure(owner common.Hash, root common.Hash, db *Database) (*SecureTrie, error) {
func NewStateTrie(owner common.Hash, root common.Hash, db *Database) (*StateTrie, error) {
if db == nil {
panic("trie.NewSecure called without a database")
}
@ -62,12 +72,12 @@ func NewSecure(owner common.Hash, root common.Hash, db *Database) (*SecureTrie,
if err != nil {
return nil, err
}
return &SecureTrie{trie: *trie, preimages: db.preimages}, nil
return &StateTrie{trie: *trie, preimages: db.preimages}, nil
}
// Get returns the value for key stored in the trie.
// The value bytes must not be modified by the caller.
func (t *SecureTrie) Get(key []byte) []byte {
func (t *StateTrie) Get(key []byte) []byte {
res, err := t.TryGet(key)
if err != nil {
log.Error(fmt.Sprintf("Unhandled trie error: %v", err))
@ -78,19 +88,50 @@ func (t *SecureTrie) Get(key []byte) []byte {
// TryGet returns the value for key stored in the trie.
// The value bytes must not be modified by the caller.
// If a node was not found in the database, a MissingNodeError is returned.
func (t *SecureTrie) TryGet(key []byte) ([]byte, error) {
func (t *StateTrie) TryGet(key []byte) ([]byte, error) {
return t.trie.TryGet(t.hashKey(key))
}
func (t *StateTrie) TryGetAccount(key []byte) (*types.StateAccount, error) {
var ret types.StateAccount
res, err := t.trie.TryGet(t.hashKey(key))
if err != nil {
log.Error(fmt.Sprintf("Unhandled trie error: %v", err))
return &ret, err
}
if res == nil {
return nil, nil
}
err = rlp.DecodeBytes(res, &ret)
return &ret, err
}
// TryGetAccountWithPreHashedKey does the same thing as TryGetAccount, however
// it expects a key that is already hashed. This constitutes an abstraction leak,
// since the client code needs to know the key format.
func (t *StateTrie) TryGetAccountWithPreHashedKey(key []byte) (*types.StateAccount, error) {
var ret types.StateAccount
res, err := t.trie.TryGet(key)
if err != nil {
log.Error(fmt.Sprintf("Unhandled trie error: %v", err))
return &ret, err
}
if res == nil {
return nil, nil
}
err = rlp.DecodeBytes(res, &ret)
return &ret, err
}
// TryGetNode attempts to retrieve a trie node by compact-encoded path. It is not
// possible to use keybyte-encoding as the path might contain odd nibbles.
func (t *SecureTrie) TryGetNode(path []byte) ([]byte, int, error) {
func (t *StateTrie) TryGetNode(path []byte) ([]byte, int, error) {
return t.trie.TryGetNode(path)
}
// TryUpdateAccount account will abstract the write of an account to the
// secure trie.
func (t *SecureTrie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
func (t *StateTrie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
hk := t.hashKey(key)
data, err := rlp.EncodeToBytes(acc)
if err != nil {
@ -109,7 +150,7 @@ func (t *SecureTrie) TryUpdateAccount(key []byte, acc *types.StateAccount) error
//
// The value bytes must not be modified by the caller while they are
// stored in the trie.
func (t *SecureTrie) Update(key, value []byte) {
func (t *StateTrie) Update(key, value []byte) {
if err := t.TryUpdate(key, value); err != nil {
log.Error(fmt.Sprintf("Unhandled trie error: %v", err))
}
@ -123,7 +164,7 @@ func (t *SecureTrie) Update(key, value []byte) {
// stored in the trie.
//
// If a node was not found in the database, a MissingNodeError is returned.
func (t *SecureTrie) TryUpdate(key, value []byte) error {
func (t *StateTrie) TryUpdate(key, value []byte) error {
hk := t.hashKey(key)
err := t.trie.TryUpdate(hk, value)
if err != nil {
@ -134,7 +175,7 @@ func (t *SecureTrie) TryUpdate(key, value []byte) error {
}
// Delete removes any existing value for key from the trie.
func (t *SecureTrie) Delete(key []byte) {
func (t *StateTrie) Delete(key []byte) {
if err := t.TryDelete(key); err != nil {
log.Error(fmt.Sprintf("Unhandled trie error: %v", err))
}
@ -142,7 +183,7 @@ func (t *SecureTrie) Delete(key []byte) {
// TryDelete removes any existing value for key from the trie.
// If a node was not found in the database, a MissingNodeError is returned.
func (t *SecureTrie) TryDelete(key []byte) error {
func (t *StateTrie) TryDelete(key []byte) error {
hk := t.hashKey(key)
delete(t.getSecKeyCache(), string(hk))
return t.trie.TryDelete(hk)
@ -150,7 +191,7 @@ func (t *SecureTrie) TryDelete(key []byte) error {
// GetKey returns the sha3 preimage of a hashed key that was
// previously used to store a value.
func (t *SecureTrie) GetKey(shaKey []byte) []byte {
func (t *StateTrie) GetKey(shaKey []byte) []byte {
if key, ok := t.getSecKeyCache()[string(shaKey)]; ok {
return key
}
@ -167,7 +208,7 @@ func (t *SecureTrie) GetKey(shaKey []byte) []byte {
// All cached preimages will be also flushed if preimages recording is enabled.
// 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 *SecureTrie) Commit(collectLeaf bool) (common.Hash, *NodeSet, error) {
func (t *StateTrie) Commit(collectLeaf bool) (common.Hash, *NodeSet, error) {
// Write all the pre-images to the actual disk database
if len(t.getSecKeyCache()) > 0 {
if t.preimages != nil {
@ -183,15 +224,15 @@ func (t *SecureTrie) Commit(collectLeaf bool) (common.Hash, *NodeSet, error) {
return t.trie.Commit(collectLeaf)
}
// Hash returns the root hash of SecureTrie. It does not write to the
// Hash returns the root hash of StateTrie. It does not write to the
// database and can be used even if the trie doesn't have one.
func (t *SecureTrie) Hash() common.Hash {
func (t *StateTrie) Hash() common.Hash {
return t.trie.Hash()
}
// Copy returns a copy of SecureTrie.
func (t *SecureTrie) Copy() *SecureTrie {
return &SecureTrie{
// Copy returns a copy of StateTrie.
func (t *StateTrie) Copy() *StateTrie {
return &StateTrie{
trie: *t.trie.Copy(),
preimages: t.preimages,
secKeyCache: t.secKeyCache,
@ -200,14 +241,14 @@ func (t *SecureTrie) Copy() *SecureTrie {
// NodeIterator returns an iterator that returns nodes of the underlying trie. Iteration
// starts at the key after the given start key.
func (t *SecureTrie) NodeIterator(start []byte) NodeIterator {
func (t *StateTrie) NodeIterator(start []byte) NodeIterator {
return t.trie.NodeIterator(start)
}
// hashKey returns the hash of key as an ephemeral buffer.
// The caller must not hold onto the return value because it will become
// invalid on the next call to hashKey or secKey.
func (t *SecureTrie) hashKey(key []byte) []byte {
func (t *StateTrie) hashKey(key []byte) []byte {
h := newHasher(false)
h.sha.Reset()
h.sha.Write(key)
@ -219,7 +260,7 @@ func (t *SecureTrie) hashKey(key []byte) []byte {
// getSecKeyCache returns the current secure key cache, creating a new one if
// ownership changed (i.e. the current secure trie is a copy of another owning
// the actual cache).
func (t *SecureTrie) getSecKeyCache() map[string][]byte {
func (t *StateTrie) getSecKeyCache() map[string][]byte {
if t != t.secKeyCacheOwner {
t.secKeyCacheOwner = t
t.secKeyCache = make(map[string][]byte)

View File

@ -28,16 +28,16 @@ import (
"github.com/ethereum/go-ethereum/ethdb/memorydb"
)
func newEmptySecure() *SecureTrie {
trie, _ := NewSecure(common.Hash{}, common.Hash{}, NewDatabase(memorydb.New()))
func newEmptySecure() *StateTrie {
trie, _ := NewStateTrie(common.Hash{}, common.Hash{}, NewDatabase(memorydb.New()))
return trie
}
// makeTestSecureTrie creates a large enough secure trie for testing.
func makeTestSecureTrie() (*Database, *SecureTrie, map[string][]byte) {
// makeTestStateTrie creates a large enough secure trie for testing.
func makeTestStateTrie() (*Database, *StateTrie, map[string][]byte) {
// Create an empty trie
triedb := NewDatabase(memorydb.New())
trie, _ := NewSecure(common.Hash{}, common.Hash{}, triedb)
trie, _ := NewStateTrie(common.Hash{}, common.Hash{}, triedb)
// Fill it with some arbitrary data
content := make(map[string][]byte)
@ -112,12 +112,12 @@ func TestSecureGetKey(t *testing.T) {
}
}
func TestSecureTrieConcurrency(t *testing.T) {
func TestStateTrieConcurrency(t *testing.T) {
// Create an initial trie and copy if for concurrent access
_, trie, _ := makeTestSecureTrie()
_, trie, _ := makeTestStateTrie()
threads := runtime.NumCPU()
tries := make([]*SecureTrie, threads)
tries := make([]*StateTrie, threads)
for i := 0; i < threads; i++ {
tries[i] = trie.Copy()
}

View File

@ -27,10 +27,10 @@ import (
)
// makeTestTrie create a sample test trie to test node-wise reconstruction.
func makeTestTrie() (*Database, *SecureTrie, map[string][]byte) {
func makeTestTrie() (*Database, *StateTrie, map[string][]byte) {
// Create an empty trie
triedb := NewDatabase(memorydb.New())
trie, _ := NewSecure(common.Hash{}, common.Hash{}, triedb)
trie, _ := NewStateTrie(common.Hash{}, common.Hash{}, triedb)
// Fill it with some arbitrary data
content := make(map[string][]byte)
@ -67,7 +67,7 @@ func makeTestTrie() (*Database, *SecureTrie, map[string][]byte) {
// content map.
func checkTrieContents(t *testing.T, db *Database, root []byte, content map[string][]byte) {
// Check root availability and trie contents
trie, err := NewSecure(common.Hash{}, common.BytesToHash(root), db)
trie, err := NewStateTrie(common.Hash{}, common.BytesToHash(root), db)
if err != nil {
t.Fatalf("failed to create trie at %x: %v", root, err)
}
@ -84,7 +84,7 @@ func checkTrieContents(t *testing.T, db *Database, root []byte, content map[stri
// checkTrieConsistency checks that all nodes in a trie are indeed present.
func checkTrieConsistency(db *Database, root common.Hash) error {
// Create and iterate a trie rooted in a subnode
trie, err := NewSecure(common.Hash{}, root, db)
trie, err := NewStateTrie(common.Hash{}, root, db)
if err != nil {
return nil // Consider a non existent state consistent
}

View File

@ -23,10 +23,8 @@ import (
"fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp"
)
var (
@ -275,14 +273,6 @@ func (t *Trie) Update(key, value []byte) {
}
}
func (t *Trie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
data, err := rlp.EncodeToBytes(acc)
if err != nil {
return fmt.Errorf("can't encode object at %x: %w", key[:], err)
}
return t.TryUpdate(key, data)
}
// TryUpdate associates key with value in the trie. Subsequent calls to
// Get will return value. If value has length zero, any existing value
// is deleted from the trie and calls to Get will return nil.
@ -292,6 +282,12 @@ func (t *Trie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
//
// If a node was not found in the database, a MissingNodeError is returned.
func (t *Trie) TryUpdate(key, value []byte) error {
return t.tryUpdate(key, value)
}
// tryUpdate expects an RLP-encoded value and performs the core function
// for TryUpdate and TryUpdateAccount.
func (t *Trie) tryUpdate(key, value []byte) error {
t.unhashed++
k := keybytesToHex(key)
if len(value) != 0 {