all: implement EIP-compliant verkle trees
verkle: Implement Trie, NodeIterator and Database ifs Fix crash in TestDump Fix TestDump Fix TrieCopy remove unnecessary traces fix: Error() returned errIteratorEnd in verkle node iterator rewrite the iterator and change the signature of OpenStorageTrie add the adapter to reuse the account trie for storage don't try to deserialize a storage leaf into an account Fix statedb unit tests (#14) * debug code * Fix more unit tests * remove traces * Go back to the full range One tree to rule them all remove updateRoot, there is no root to update store code inside the account leaf fix build save current state for Sina Update go-verkle to latest Charge WITNESS_*_COST gas on storage loads Add witness costs for SSTORE as well Charge witness gas in the case of code execution corresponding code deletion add a --verkle flag to separate verkle experiments from regular geth operations use the snapshot to get data stateless execution from block witness AccessWitness functions Add block generation test + genesis snapshot generation test stateless block execution (#18) * test stateless block execution * Force tree resolution before generating the proof increased coverage in stateless test execution (#19) * test stateless block execution * Force tree resolution before generating the proof * increase coverage in stateless test execution ensure geth compiles fix issues in tests with verkle trees deactivated Ensure stateless data is available when executing statelessly (#20) * Ensure stateless data is available when executing statelessly * Actual execution of a statless block * bugfixes in stateless block execution * code cleanup - Reduce PR footprint by reverting NewEVM to its original signature - Move the access witness to the block context - prepare for a change in AW semantics Need to store the initial values. - Use the touch helper function, DRY * revert the signature of MustCommit to its original form (#21) fix leaf proofs in stateless execution (#22) * Fixes in witness pre-state * Add the recipient's nonce to the witness * reduce PR footprint and investigate issue in root state calculation * quick build fix cleanup: Remove extra parameter in ToBlock revert ToBlock to its older signature fix import cycle in vm tests fix linter issue fix appveyor build fix nil pointers in tests Add indices, yis and Cis to the block's Verkle proof upgrade geth dependency to drop geth's common dep fix cmd/devp2p tests fix rebase issues quell an appveyor warning fix address touching in SLOAD and SSTORE fix access witness for code size touch target account data before calling make sure the proper locations get touched in (ext)codecopy touch all code pages in execution add pushdata to witness remove useless code in genesis snapshot generation testnet: fix some of the rebase/drift issues Fix verkle proof generation in block fix an issue occuring when chunking past the code size fix: ensure the code copy doesn't extend past the code size
This commit is contained in:
@ -277,6 +277,7 @@ type Config struct {
|
||||
Cache int // Memory allowance (MB) to use for caching trie nodes in memory
|
||||
Journal string // Journal of clean cache to survive node restarts
|
||||
Preimages bool // Flag whether the preimage of trie key is recorded
|
||||
UseVerkle bool // Flag whether the data is stored in a verkle trie
|
||||
}
|
||||
|
||||
// NewDatabase creates a new trie database to store ephemeral trie content before
|
||||
|
@ -217,3 +217,7 @@ func (t *SecureTrie) getSecKeyCache() map[string][]byte {
|
||||
}
|
||||
return t.secKeyCache
|
||||
}
|
||||
|
||||
func (t *SecureTrie) IsVerkle() bool {
|
||||
return false
|
||||
}
|
||||
|
103
trie/utils/verkle.go
Normal file
103
trie/utils/verkle.go
Normal file
@ -0,0 +1,103 @@
|
||||
// Copyright 2021 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 utils
|
||||
|
||||
import (
|
||||
"crypto/sha256"
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
)
|
||||
|
||||
const (
|
||||
VersionLeafKey = 0
|
||||
BalanceLeafKey = 1
|
||||
NonceLeafKey = 2
|
||||
CodeKeccakLeafKey = 3
|
||||
CodeSizeLeafKey = 4
|
||||
)
|
||||
|
||||
var (
|
||||
zero = uint256.NewInt(0)
|
||||
HeaderStorageOffset = uint256.NewInt(64)
|
||||
CodeOffset = uint256.NewInt(128)
|
||||
MainStorageOffset = new(uint256.Int).Lsh(uint256.NewInt(256), 31)
|
||||
VerkleNodeWidth = uint256.NewInt(8)
|
||||
codeStorageDelta = uint256.NewInt(0).Sub(HeaderStorageOffset, CodeOffset)
|
||||
)
|
||||
|
||||
func GetTreeKey(address []byte, treeIndex *uint256.Int, subIndex byte) []byte {
|
||||
digest := sha256.New()
|
||||
digest.Write(address)
|
||||
treeIndexBytes := treeIndex.Bytes()
|
||||
var payload [32]byte
|
||||
copy(payload[:len(treeIndexBytes)], treeIndexBytes)
|
||||
digest.Write(payload[:])
|
||||
h := digest.Sum(nil)
|
||||
h[31] = subIndex
|
||||
return h
|
||||
}
|
||||
|
||||
func GetTreeKeyAccountLeaf(address []byte, leaf byte) []byte {
|
||||
return GetTreeKey(address, zero, leaf)
|
||||
}
|
||||
|
||||
func GetTreeKeyVersion(address []byte) []byte {
|
||||
return GetTreeKey(address, zero, VersionLeafKey)
|
||||
}
|
||||
|
||||
func GetTreeKeyBalance(address []byte) []byte {
|
||||
return GetTreeKey(address, zero, BalanceLeafKey)
|
||||
}
|
||||
|
||||
func GetTreeKeyNonce(address []byte) []byte {
|
||||
return GetTreeKey(address, zero, NonceLeafKey)
|
||||
}
|
||||
|
||||
func GetTreeKeyCodeKeccak(address []byte) []byte {
|
||||
return GetTreeKey(address, zero, CodeKeccakLeafKey)
|
||||
}
|
||||
|
||||
func GetTreeKeyCodeSize(address []byte) []byte {
|
||||
return GetTreeKey(address, zero, CodeSizeLeafKey)
|
||||
}
|
||||
|
||||
func GetTreeKeyCodeChunk(address []byte, chunk *uint256.Int) []byte {
|
||||
chunkOffset := new(uint256.Int).Add(CodeOffset, chunk)
|
||||
treeIndex := new(uint256.Int).Div(chunkOffset, VerkleNodeWidth)
|
||||
subIndexMod := new(uint256.Int).Mod(chunkOffset, VerkleNodeWidth).Bytes()
|
||||
var subIndex byte
|
||||
if len(subIndexMod) != 0 {
|
||||
subIndex = subIndexMod[0]
|
||||
}
|
||||
return GetTreeKey(address, treeIndex, subIndex)
|
||||
}
|
||||
|
||||
func GetTreeKeyStorageSlot(address []byte, storageKey *uint256.Int) []byte {
|
||||
treeIndex := storageKey.Clone()
|
||||
if storageKey.Cmp(codeStorageDelta) < 0 {
|
||||
treeIndex.Add(HeaderStorageOffset, storageKey)
|
||||
} else {
|
||||
treeIndex.Add(MainStorageOffset, storageKey)
|
||||
}
|
||||
treeIndex.Div(treeIndex, VerkleNodeWidth)
|
||||
subIndexMod := new(uint256.Int).Mod(treeIndex, VerkleNodeWidth).Bytes()
|
||||
var subIndex byte
|
||||
if len(subIndexMod) != 0 {
|
||||
subIndex = subIndexMod[0]
|
||||
}
|
||||
return GetTreeKey(address, treeIndex, subIndex)
|
||||
}
|
261
trie/verkle.go
Normal file
261
trie/verkle.go
Normal file
@ -0,0 +1,261 @@
|
||||
// Copyright 2021 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 (
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/trie/utils"
|
||||
"github.com/gballet/go-verkle"
|
||||
"github.com/protolambda/go-kzg/bls"
|
||||
)
|
||||
|
||||
// VerkleTrie is a wrapper around VerkleNode that implements the trie.Trie
|
||||
// interface so that Verkle trees can be reused verbatim.
|
||||
type VerkleTrie struct {
|
||||
root verkle.VerkleNode
|
||||
db *Database
|
||||
}
|
||||
|
||||
//func (vt *VerkleTrie) ToDot() string {
|
||||
//return verkle.ToDot(vt.root)
|
||||
//}
|
||||
|
||||
func NewVerkleTrie(root verkle.VerkleNode, db *Database) *VerkleTrie {
|
||||
return &VerkleTrie{
|
||||
root: root,
|
||||
db: db,
|
||||
}
|
||||
}
|
||||
|
||||
var errInvalidProof = errors.New("invalid proof")
|
||||
|
||||
// GetKey returns the sha3 preimage of a hashed key that was previously used
|
||||
// to store a value.
|
||||
func (trie *VerkleTrie) GetKey(key []byte) []byte {
|
||||
return key
|
||||
}
|
||||
|
||||
// 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
|
||||
// trie.MissingNodeError is returned.
|
||||
func (trie *VerkleTrie) TryGet(key []byte) ([]byte, error) {
|
||||
return trie.root.Get(key, trie.db.DiskDB().Get)
|
||||
}
|
||||
|
||||
func (t *VerkleTrie) TryUpdateAccount(key []byte, acc *types.StateAccount) error {
|
||||
|
||||
var err error
|
||||
if err = t.TryUpdate(utils.GetTreeKeyVersion(key), []byte{0}); err != nil {
|
||||
return fmt.Errorf("updateStateObject (%x) error: %v", key, err)
|
||||
}
|
||||
var nonce [32]byte
|
||||
binary.BigEndian.PutUint64(nonce[:], acc.Nonce)
|
||||
if err = t.TryUpdate(utils.GetTreeKeyNonce(key), nonce[:]); err != nil {
|
||||
return fmt.Errorf("updateStateObject (%x) error: %v", key, err)
|
||||
}
|
||||
if err = t.TryUpdate(utils.GetTreeKeyBalance(key), acc.Balance.Bytes()); err != nil {
|
||||
return fmt.Errorf("updateStateObject (%x) error: %v", key, err)
|
||||
}
|
||||
if err = t.TryUpdate(utils.GetTreeKeyCodeKeccak(key), acc.CodeHash); err != nil {
|
||||
return fmt.Errorf("updateStateObject (%x) error: %v", key, err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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
|
||||
// by the caller while they are stored in the trie. If a node was not found in the
|
||||
// database, a trie.MissingNodeError is returned.
|
||||
func (trie *VerkleTrie) TryUpdate(key, value []byte) error {
|
||||
return trie.root.Insert(key, value, func(h []byte) ([]byte, error) {
|
||||
return trie.db.DiskDB().Get(h)
|
||||
})
|
||||
}
|
||||
|
||||
// TryDelete removes any existing value for key from the trie. If a node was not
|
||||
// found in the database, a trie.MissingNodeError is returned.
|
||||
func (trie *VerkleTrie) TryDelete(key []byte) error {
|
||||
return trie.root.Delete(key)
|
||||
}
|
||||
|
||||
// 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 (trie *VerkleTrie) Hash() common.Hash {
|
||||
// TODO cache this value
|
||||
rootC := trie.root.ComputeCommitment()
|
||||
return bls.FrTo32(rootC)
|
||||
}
|
||||
|
||||
func nodeToDBKey(n verkle.VerkleNode) []byte {
|
||||
ret := bls.FrTo32(n.ComputeCommitment())
|
||||
return ret[:]
|
||||
}
|
||||
|
||||
// Commit writes all nodes to the trie's memory database, tracking the internal
|
||||
// and external (for account tries) references.
|
||||
func (trie *VerkleTrie) Commit(onleaf LeafCallback) (common.Hash, int, error) {
|
||||
flush := make(chan verkle.VerkleNode)
|
||||
go func() {
|
||||
trie.root.(*verkle.InternalNode).Flush(func(n verkle.VerkleNode) {
|
||||
flush <- n
|
||||
})
|
||||
close(flush)
|
||||
}()
|
||||
var commitCount int
|
||||
for n := range flush {
|
||||
commitCount += 1
|
||||
value, err := n.Serialize()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err := trie.db.DiskDB().Put(nodeToDBKey(n), value); err != nil {
|
||||
return common.Hash{}, commitCount, err
|
||||
}
|
||||
}
|
||||
|
||||
return trie.Hash(), commitCount, nil
|
||||
}
|
||||
|
||||
// NodeIterator returns an iterator that returns nodes of the trie. Iteration
|
||||
// starts at the key after the given start key.
|
||||
func (trie *VerkleTrie) NodeIterator(startKey []byte) NodeIterator {
|
||||
return newVerkleNodeIterator(trie, nil)
|
||||
}
|
||||
|
||||
// Prove constructs a Merkle proof for key. The result contains all encoded nodes
|
||||
// on the path to the value at key. The value itself is also included in the last
|
||||
// node and can be retrieved by verifying the proof.
|
||||
//
|
||||
// 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), ending
|
||||
// with the node that proves the absence of the key.
|
||||
func (trie *VerkleTrie) Prove(key []byte, fromLevel uint, proofDb ethdb.KeyValueWriter) error {
|
||||
panic("not implemented")
|
||||
}
|
||||
|
||||
func (trie *VerkleTrie) Copy(db *Database) *VerkleTrie {
|
||||
return &VerkleTrie{
|
||||
root: trie.root.Copy(),
|
||||
db: db,
|
||||
}
|
||||
}
|
||||
func (trie *VerkleTrie) IsVerkle() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
type KeyValuePair struct {
|
||||
Key []byte
|
||||
Value []byte
|
||||
}
|
||||
|
||||
type verkleproof struct {
|
||||
D *bls.G1Point
|
||||
Y *bls.Fr
|
||||
Σ *bls.G1Point
|
||||
|
||||
Cis []*bls.G1Point
|
||||
Indices []uint
|
||||
Yis []*bls.Fr
|
||||
|
||||
Leaves []KeyValuePair
|
||||
}
|
||||
|
||||
func (trie *VerkleTrie) ProveAndSerialize(keys [][]byte, kv map[common.Hash][]byte) ([]byte, error) {
|
||||
d, y, σ, cis, indices, yis := verkle.MakeVerkleMultiProof(trie.root, keys)
|
||||
vp := verkleproof{
|
||||
D: d,
|
||||
Y: y,
|
||||
Σ: σ,
|
||||
Cis: cis,
|
||||
Indices: indices,
|
||||
Yis: yis,
|
||||
}
|
||||
for key, val := range kv {
|
||||
var k [32]byte
|
||||
copy(k[:], key[:])
|
||||
vp.Leaves = append(vp.Leaves, KeyValuePair{
|
||||
Key: k[:],
|
||||
Value: val,
|
||||
})
|
||||
}
|
||||
return rlp.EncodeToBytes(vp)
|
||||
}
|
||||
|
||||
func DeserializeAndVerifyVerkleProof(proof []byte) (*bls.G1Point, *bls.Fr, *bls.G1Point, map[common.Hash]common.Hash, error) {
|
||||
d, y, σ, cis, indices, yis, leaves, err := deserializeVerkleProof(proof)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, fmt.Errorf("could not deserialize proof: %w", err)
|
||||
}
|
||||
if !verkle.VerifyVerkleProof(d, σ, y, cis, indices, yis, verkle.GetKZGConfig()) {
|
||||
return nil, nil, nil, nil, errInvalidProof
|
||||
}
|
||||
|
||||
return d, y, σ, leaves, nil
|
||||
}
|
||||
|
||||
func deserializeVerkleProof(proof []byte) (*bls.G1Point, *bls.Fr, *bls.G1Point, []*bls.G1Point, []uint, []*bls.Fr, map[common.Hash]common.Hash, error) {
|
||||
var vp verkleproof
|
||||
err := rlp.DecodeBytes(proof, &vp)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, nil, nil, nil, fmt.Errorf("verkle proof deserialization error: %w", err)
|
||||
}
|
||||
leaves := make(map[common.Hash]common.Hash, len(vp.Leaves))
|
||||
for _, kvp := range vp.Leaves {
|
||||
leaves[common.BytesToHash(kvp.Key)] = common.BytesToHash(kvp.Value)
|
||||
}
|
||||
return vp.D, vp.Y, vp.Σ, vp.Cis, vp.Indices, vp.Yis, leaves, nil
|
||||
}
|
||||
|
||||
// Copy the values here so as to avoid an import cycle
|
||||
const (
|
||||
PUSH1 = 0x60
|
||||
PUSH32 = 0x71
|
||||
)
|
||||
|
||||
func ChunkifyCode(addr common.Address, code []byte) ([][32]byte, error) {
|
||||
lastOffset := byte(0)
|
||||
chunkCount := len(code) / 31
|
||||
if len(code)%31 != 0 {
|
||||
chunkCount++
|
||||
}
|
||||
chunks := make([][32]byte, chunkCount)
|
||||
for i, chunk := range chunks {
|
||||
end := 31 * (i + 1)
|
||||
if len(code) < end {
|
||||
end = len(code)
|
||||
}
|
||||
copy(chunk[1:], code[31*i:end])
|
||||
for j := lastOffset; int(j) < len(code[31*i:end]); j++ {
|
||||
if code[j] >= byte(PUSH1) && code[j] <= byte(PUSH32) {
|
||||
j += code[j] - byte(PUSH1) + 1
|
||||
lastOffset = (j + 1) % 31
|
||||
}
|
||||
}
|
||||
chunk[0] = lastOffset
|
||||
}
|
||||
|
||||
return chunks, nil
|
||||
}
|
251
trie/verkle_iterator.go
Normal file
251
trie/verkle_iterator.go
Normal file
@ -0,0 +1,251 @@
|
||||
// Copyright 2021 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 (
|
||||
"fmt"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/protolambda/go-kzg/bls"
|
||||
|
||||
//"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/gballet/go-verkle"
|
||||
)
|
||||
|
||||
type verkleNodeIteratorState struct {
|
||||
Node verkle.VerkleNode
|
||||
Index int
|
||||
}
|
||||
|
||||
type verkleNodeIterator struct {
|
||||
trie *VerkleTrie
|
||||
current verkle.VerkleNode
|
||||
lastErr error
|
||||
|
||||
stack []verkleNodeIteratorState
|
||||
}
|
||||
|
||||
func newVerkleNodeIterator(trie *VerkleTrie, start []byte) NodeIterator {
|
||||
if trie.Hash() == emptyState {
|
||||
return new(nodeIterator)
|
||||
}
|
||||
it := &verkleNodeIterator{trie: trie, current: trie.root}
|
||||
//it.err = it.seek(start)
|
||||
return it
|
||||
}
|
||||
|
||||
// Next moves the iterator to the next node. If the parameter is false, any child
|
||||
// nodes will be skipped.
|
||||
func (it *verkleNodeIterator) Next(descend bool) bool {
|
||||
if it.lastErr == errIteratorEnd {
|
||||
it.lastErr = errIteratorEnd
|
||||
return false
|
||||
}
|
||||
|
||||
if len(it.stack) == 0 {
|
||||
it.stack = append(it.stack, verkleNodeIteratorState{Node: it.trie.root, Index: 0})
|
||||
it.current = it.trie.root
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
switch node := it.current.(type) {
|
||||
case *verkle.InternalNode:
|
||||
context := &it.stack[len(it.stack)-1]
|
||||
|
||||
// Look for the next non-empty child
|
||||
children := node.Children()
|
||||
for ; context.Index < len(children); context.Index++ {
|
||||
if _, ok := children[context.Index].(verkle.Empty); !ok {
|
||||
it.stack = append(it.stack, verkleNodeIteratorState{Node: children[context.Index], Index: 0})
|
||||
it.current = children[context.Index]
|
||||
return it.Next(descend)
|
||||
}
|
||||
}
|
||||
|
||||
// Reached the end of this node, go back to the parent, if
|
||||
// this isn't root.
|
||||
if len(it.stack) == 1 {
|
||||
it.lastErr = errIteratorEnd
|
||||
return false
|
||||
}
|
||||
it.stack = it.stack[:len(it.stack)-1]
|
||||
it.current = it.stack[len(it.stack)-1].Node
|
||||
it.stack[len(it.stack)-1].Index++
|
||||
return it.Next(descend)
|
||||
case *verkle.LeafNode:
|
||||
// Look for the next non-empty value
|
||||
for i := it.stack[len(it.stack)-1].Index + 1; i < 256; i++ {
|
||||
if node.Value(i) != nil {
|
||||
it.stack[len(it.stack)-1].Index = i
|
||||
return true
|
||||
}
|
||||
}
|
||||
// go back to parent to get the next leaf
|
||||
it.stack = it.stack[:len(it.stack)-1]
|
||||
it.current = it.stack[len(it.stack)-1].Node
|
||||
it.stack[len(it.stack)-1].Index++
|
||||
return it.Next(descend)
|
||||
case *verkle.HashedNode:
|
||||
// resolve the node
|
||||
data, err := it.trie.db.diskdb.Get(nodeToDBKey(node))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
it.current, err = verkle.ParseNode(data, len(it.stack)-1)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// update the stack and parent with the resolved node
|
||||
it.stack[len(it.stack)-1].Node = it.current
|
||||
parent := &it.stack[len(it.stack)-2]
|
||||
parent.Node.(*verkle.InternalNode).SetChild(parent.Index, it.current)
|
||||
return true
|
||||
default:
|
||||
fmt.Println(node)
|
||||
panic("invalid node type")
|
||||
}
|
||||
}
|
||||
|
||||
// Error returns the error status of the iterator.
|
||||
func (it *verkleNodeIterator) Error() error {
|
||||
if it.lastErr == errIteratorEnd {
|
||||
return nil
|
||||
}
|
||||
return it.lastErr
|
||||
}
|
||||
|
||||
// Hash returns the hash of the current node.
|
||||
func (it *verkleNodeIterator) Hash() common.Hash {
|
||||
return bls.FrTo32(it.current.ComputeCommitment())
|
||||
}
|
||||
|
||||
// Parent returns the hash of the parent of the current node. The hash may be the one
|
||||
// grandparent if the immediate parent is an internal node with no hash.
|
||||
func (it *verkleNodeIterator) Parent() common.Hash {
|
||||
return bls.FrTo32(it.stack[len(it.stack)-1].Node.ComputeCommitment())
|
||||
}
|
||||
|
||||
// Path returns the hex-encoded path to the current node.
|
||||
// Callers must not retain references to the return value after calling Next.
|
||||
// For leaf nodes, the last element of the path is the 'terminator symbol' 0x10.
|
||||
func (it *verkleNodeIterator) Path() []byte {
|
||||
|
||||
panic("not completely implemented")
|
||||
}
|
||||
|
||||
// Leaf returns true iff the current node is a leaf node.
|
||||
func (it *verkleNodeIterator) Leaf() bool {
|
||||
_, ok := it.current.(*verkle.LeafNode)
|
||||
return ok
|
||||
}
|
||||
|
||||
// LeafKey returns the key of the leaf. The method panics if the iterator is not
|
||||
// positioned at a leaf. Callers must not retain references to the value after
|
||||
// calling Next.
|
||||
func (it *verkleNodeIterator) LeafKey() []byte {
|
||||
leaf, ok := it.current.(*verkle.LeafNode)
|
||||
if !ok {
|
||||
panic("Leaf() called on an verkle node iterator not at a leaf location")
|
||||
}
|
||||
|
||||
return leaf.Key(it.stack[len(it.stack)-1].Index)
|
||||
}
|
||||
|
||||
// LeafBlob returns the content of the leaf. The method panics if the iterator
|
||||
// is not positioned at a leaf. Callers must not retain references to the value
|
||||
// after calling Next.
|
||||
func (it *verkleNodeIterator) LeafBlob() []byte {
|
||||
leaf, ok := it.current.(*verkle.LeafNode)
|
||||
if !ok {
|
||||
panic("LeafBlob() called on an verkle node iterator not at a leaf location")
|
||||
}
|
||||
|
||||
return leaf.Value(it.stack[len(it.stack)-1].Index)
|
||||
}
|
||||
|
||||
// LeafProof returns the Merkle proof of the leaf. The method panics if the
|
||||
// iterator is not positioned at a leaf. Callers must not retain references
|
||||
// to the value after calling Next.
|
||||
func (it *verkleNodeIterator) LeafProof() [][]byte {
|
||||
_, ok := it.current.(*verkle.LeafNode)
|
||||
if !ok {
|
||||
panic("LeafProof() called on an verkle node iterator not at a leaf location")
|
||||
}
|
||||
|
||||
//return it.trie.Prove(leaf.Key())
|
||||
panic("not completely implemented")
|
||||
}
|
||||
|
||||
// AddResolver sets an intermediate database to use for looking up trie nodes
|
||||
// before reaching into the real persistent layer.
|
||||
//
|
||||
// This is not required for normal operation, rather is an optimization for
|
||||
// cases where trie nodes can be recovered from some external mechanism without
|
||||
// reading from disk. In those cases, this resolver allows short circuiting
|
||||
// accesses and returning them from memory.
|
||||
//
|
||||
// Before adding a similar mechanism to any other place in Geth, consider
|
||||
// making trie.Database an interface and wrapping at that level. It's a huge
|
||||
// refactor, but it could be worth it if another occurrence arises.
|
||||
func (it *verkleNodeIterator) AddResolver(ethdb.KeyValueStore) {
|
||||
panic("not completely implemented")
|
||||
}
|
||||
|
||||
type dummy struct{}
|
||||
|
||||
func (it dummy) Next(descend bool) bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (it dummy) Error() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it dummy) Hash() common.Hash {
|
||||
panic("should not be called")
|
||||
}
|
||||
|
||||
func (it dummy) Leaf() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (it dummy) LeafKey() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it dummy) LeafProof() [][]byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it dummy) LeafBlob() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it dummy) Parent() common.Hash {
|
||||
return common.Hash{}
|
||||
}
|
||||
|
||||
func (it dummy) Path() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it dummy) AddResolver(ethdb.KeyValueStore) {
|
||||
panic("not completely implemented")
|
||||
}
|
Reference in New Issue
Block a user