Use iterator based approach for getSlice
This commit is contained in:
parent
a338bd1a59
commit
2a194a9488
@ -839,10 +839,7 @@ func (pea *PublicEthAPI) localGetProof(ctx context.Context, address common.Addre
|
||||
|
||||
// GetSlice returns a slice of state or storage nodes from a provided root to a provided path and past it to a certain depth
|
||||
func (pea *PublicEthAPI) GetSlice(ctx context.Context, path string, depth int, root common.Hash, storage bool) (*GetSliceResponse, error) {
|
||||
if storage {
|
||||
return pea.B.GetStorageSlice(path, depth, root)
|
||||
}
|
||||
return pea.B.GetStateSlice(path, depth, root)
|
||||
return pea.B.GetSlice(path, depth, root, storage)
|
||||
}
|
||||
|
||||
// revertError is an API error that encompassas an EVM revertal with JSON error
|
||||
|
@ -42,13 +42,11 @@ import (
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
ethServerShared "github.com/ethereum/go-ethereum/statediff/indexer/shared"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
"github.com/ipfs/go-cid"
|
||||
"github.com/jmoiron/sqlx"
|
||||
log "github.com/sirupsen/logrus"
|
||||
|
||||
ethServerShared "github.com/ethereum/go-ethereum/statediff/indexer/shared"
|
||||
|
||||
"github.com/cerc-io/ipld-eth-server/v4/pkg/shared"
|
||||
)
|
||||
|
||||
@ -97,33 +95,6 @@ const (
|
||||
ORDER BY header_cids.block_number DESC
|
||||
LIMIT 1`
|
||||
RetrieveCodeByMhKey = `SELECT data FROM public.blocks WHERE key = $1`
|
||||
RetrieveBlockNumberForStateRoot = `SELECT block_number
|
||||
FROM eth.header_cids
|
||||
WHERE state_root = $1
|
||||
AND header_cids.block_hash = (SELECT canonical_header_hash(header_cids.block_number))
|
||||
ORDER BY block_number DESC
|
||||
LIMIT 1`
|
||||
RetrieveBlockNumberAndStateLeafKeyForStorageRoot = `SELECT state_accounts.block_number, state_leaf_key
|
||||
FROM eth.state_cids, eth.state_accounts
|
||||
WHERE state_accounts.storage_root = $1
|
||||
AND state_cids.state_path = state_accounts.state_path
|
||||
AND state_cids.header_id = state_accounts.header_id
|
||||
AND state_cids.block_number = state_accounts.block_number
|
||||
AND state_cids.node_type != 3
|
||||
AND state_accounts.header_id = (SELECT canonical_header_hash(state_accounts.block_number))
|
||||
ORDER BY state_accounts.block_number DESC
|
||||
LIMIT 1`
|
||||
RetrieveAccountByStateCID = `SELECT state_leaf_key, storage_root, code_hash
|
||||
FROM eth.state_cids
|
||||
INNER JOIN eth.state_accounts ON (
|
||||
state_cids.state_path = state_accounts.state_path
|
||||
AND state_cids.header_id = state_accounts.header_id
|
||||
AND state_cids.block_number = state_accounts.block_number
|
||||
)
|
||||
WHERE state_cids.cid = $1
|
||||
AND state_cids.block_number <= $2
|
||||
ORDER BY state_cids.block_number DESC
|
||||
LIMIT 1`
|
||||
)
|
||||
|
||||
const (
|
||||
@ -916,267 +887,132 @@ func (b *Backend) GetStorageByHash(ctx context.Context, address common.Address,
|
||||
return storageRlp, err
|
||||
}
|
||||
|
||||
func (b *Backend) GetStateSlice(path string, depth int, root common.Hash) (*GetSliceResponse, error) {
|
||||
func (b *Backend) GetSlice(path string, depth int, root common.Hash, storage bool) (*GetSliceResponse, error) {
|
||||
response := new(GetSliceResponse)
|
||||
response.init(path, depth, root)
|
||||
|
||||
// Start a timer
|
||||
trieLoadingStart := makeTimestamp()
|
||||
t, _ := b.StateDatabase.OpenTrie(root)
|
||||
headPath := common.FromHex(path)
|
||||
|
||||
// Get the block height for the input state root
|
||||
blockHeight, err := b.getBlockHeightForStateRoot(root)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice blockheight lookup error: %s", err.Error())
|
||||
}
|
||||
// Metadata fields
|
||||
metaData := metaDataFields{}
|
||||
|
||||
// TODO: Use an iterator instead of doing an exhausitive database search for all possible paths at the given depth
|
||||
// Get all the paths
|
||||
headPath, stemPaths, slicePaths, err := getPaths(path, depth)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice path generation error: %s", err.Error())
|
||||
}
|
||||
response.MetaData.TimeStats["00-trie-loading"] = strconv.Itoa(int(makeTimestamp() - trieLoadingStart))
|
||||
|
||||
// Begin tx
|
||||
tx, err := b.DB.Beginx()
|
||||
// Get Stem nodes
|
||||
err := b.getSliceStem(headPath, t, response, &metaData, storage)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer func() {
|
||||
if p := recover(); p != nil {
|
||||
shared.Rollback(tx)
|
||||
panic(p)
|
||||
} else if err != nil {
|
||||
shared.Rollback(tx)
|
||||
} else {
|
||||
err = tx.Commit()
|
||||
}
|
||||
}()
|
||||
|
||||
// Fetch stem nodes
|
||||
// some of the "stem" nodes can be leaf nodes (but not value nodes)
|
||||
stemNodes, stemLeafCIDs, _, timeSpent, err := b.getStateNodesByPathsAndBlockNumber(tx, stemPaths, blockHeight)
|
||||
// Get Head node
|
||||
err = b.getSliceHead(headPath, t, response, &metaData, storage)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice stem node lookup error: %s", err.Error())
|
||||
return nil, err
|
||||
}
|
||||
response.TrieNodes.Stem = stemNodes
|
||||
response.MetaData.TimeStats["01-fetch-stem-keys"] = timeSpent
|
||||
|
||||
// Fetch slice nodes
|
||||
sliceNodes, sliceLeafCIDs, deepestPath, timeSpent, err := b.getStateNodesByPathsAndBlockNumber(tx, slicePaths, blockHeight)
|
||||
if depth > 0 {
|
||||
// Get Slice nodes
|
||||
err = b.getSliceTrie(headPath, t, response, &metaData, depth, storage)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice slice node lookup error: %s", err.Error())
|
||||
}
|
||||
response.TrieNodes.Slice = sliceNodes
|
||||
response.MetaData.TimeStats["02-fetch-slice-keys"] = timeSpent
|
||||
|
||||
// Fetch head node
|
||||
headNode, headLeafCID, _, _, err := b.getStateNodesByPathsAndBlockNumber(tx, [][]byte{headPath}, blockHeight)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice head node lookup error: %s", err.Error())
|
||||
}
|
||||
response.TrieNodes.Head = headNode
|
||||
|
||||
// Fetch leaf contract data and fill in remaining metadata
|
||||
leafFetchStart := makeTimestamp()
|
||||
leafNodes := make([]cid.Cid, 0, len(stemLeafCIDs)+len(sliceLeafCIDs)+len(headLeafCID))
|
||||
leafNodes = append(leafNodes, stemLeafCIDs...)
|
||||
leafNodes = append(leafNodes, sliceLeafCIDs...)
|
||||
leafNodes = append(leafNodes, headLeafCID...)
|
||||
|
||||
for _, leafNodeCID := range leafNodes {
|
||||
stateLeafKey, storageRoot, code, err := b.getAccountByStateCID(tx, leafNodeCID.String(), blockHeight)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStateSlice account lookup error: %s", err.Error())
|
||||
}
|
||||
|
||||
if len(code) > 0 {
|
||||
response.Leaves[stateLeafKey] = GetSliceResponseAccount{
|
||||
StorageRoot: storageRoot,
|
||||
EVMCode: common.Bytes2Hex(code),
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
response.MetaData.TimeStats["03-fetch-leaves-info"] = strconv.Itoa(int(makeTimestamp() - leafFetchStart))
|
||||
|
||||
maxDepth := deepestPath - len(headPath)
|
||||
if maxDepth < 0 {
|
||||
maxDepth = 0
|
||||
}
|
||||
response.MetaData.NodeStats["01-max-depth"] = strconv.Itoa(maxDepth)
|
||||
|
||||
response.MetaData.NodeStats["02-total-trie-nodes"] = strconv.Itoa(len(response.TrieNodes.Stem) + len(response.TrieNodes.Head) + len(response.TrieNodes.Slice))
|
||||
response.MetaData.NodeStats["03-leaves"] = strconv.Itoa(len(leafNodes))
|
||||
response.MetaData.NodeStats["04-smart-contracts"] = strconv.Itoa(len(response.Leaves))
|
||||
response.MetaData.NodeStats["00-stem-and-head-nodes"] = strconv.Itoa(len(response.TrieNodes.Stem) + len(response.TrieNodes.Head))
|
||||
response.populateMetaData(metaData)
|
||||
|
||||
return response, nil
|
||||
}
|
||||
|
||||
func (b *Backend) GetStorageSlice(path string, depth int, root common.Hash) (*GetSliceResponse, error) {
|
||||
response := new(GetSliceResponse)
|
||||
response.init(path, depth, root)
|
||||
func (b *Backend) getSliceStem(headPath []byte, t state.Trie, response *GetSliceResponse, metaData *metaDataFields, storage bool) error {
|
||||
for i := 0; i < len(headPath); i++ {
|
||||
// Create path for each node along the stem
|
||||
startPath := make([]byte, len(headPath[:i]))
|
||||
copy(startPath, headPath[:i])
|
||||
|
||||
// Start a timer
|
||||
trieLoadingStart := makeTimestamp()
|
||||
// Create an iterator initialized at startPath
|
||||
it, timeTaken := getIteratorAtPath(t, startPath)
|
||||
metaData.trieLoadingTime += timeTaken
|
||||
|
||||
// Get the block height and state leaf key for the input storage root
|
||||
blockHeight, stateLeafKey, err := b.getBlockHeightAndStateLeafKeyForStorageRoot(root)
|
||||
sliceNodeMetrics, err := fillSliceNodeData(b.EthDB, b.StateDatabase.TrieDB(), response.TrieNodes.Stem, response.Leaves, it, storage)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStorageSlice blockheight and state key lookup error: %s", err.Error())
|
||||
return err
|
||||
}
|
||||
|
||||
// Get all the paths
|
||||
headPath, stemPaths, slicePaths, err := getPaths(path, depth)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStorageSlice path generation error: %s", err.Error())
|
||||
// Update metadata
|
||||
if (sliceNodeMetrics.pathLen - len(headPath)) > metaData.maxDepth {
|
||||
metaData.maxDepth = sliceNodeMetrics.pathLen
|
||||
}
|
||||
if sliceNodeMetrics.isLeaf {
|
||||
metaData.leafCount++
|
||||
}
|
||||
metaData.stemNodesFetchTime += sliceNodeMetrics.nodeFetchTime
|
||||
metaData.leavesFetchTime += sliceNodeMetrics.leafFetchTime
|
||||
}
|
||||
response.MetaData.TimeStats["00-trie-loading"] = strconv.Itoa(int(makeTimestamp() - trieLoadingStart))
|
||||
|
||||
// Begin tx
|
||||
tx, err := b.DB.Beginx()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *Backend) getSliceHead(headPath []byte, t state.Trie, response *GetSliceResponse, metaData *metaDataFields, storage bool) error {
|
||||
// Create an iterator initialized at headPath
|
||||
it, timeTaken := getIteratorAtPath(t, headPath)
|
||||
metaData.trieLoadingTime += timeTaken
|
||||
|
||||
sliceNodeMetrics, err := fillSliceNodeData(b.EthDB, b.StateDatabase.TrieDB(), response.TrieNodes.Head, response.Leaves, it, storage)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
defer func() {
|
||||
if p := recover(); p != nil {
|
||||
shared.Rollback(tx)
|
||||
panic(p)
|
||||
} else if err != nil {
|
||||
shared.Rollback(tx)
|
||||
|
||||
// Update metadata
|
||||
if (sliceNodeMetrics.pathLen - len(headPath)) > metaData.maxDepth {
|
||||
metaData.maxDepth = sliceNodeMetrics.pathLen
|
||||
}
|
||||
if sliceNodeMetrics.isLeaf {
|
||||
metaData.leafCount++
|
||||
}
|
||||
metaData.stemNodesFetchTime += sliceNodeMetrics.nodeFetchTime
|
||||
metaData.leavesFetchTime += sliceNodeMetrics.leafFetchTime
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *Backend) getSliceTrie(headPath []byte, t state.Trie, response *GetSliceResponse, metaData *metaDataFields, depth int, storage bool) error {
|
||||
it, timeTaken := getIteratorAtPath(t, headPath)
|
||||
metaData.trieLoadingTime += timeTaken
|
||||
|
||||
headPathLen := len(headPath)
|
||||
maxPathLen := headPathLen + depth
|
||||
descend := true
|
||||
for it.Next(descend) {
|
||||
pathLen := len(it.Path())
|
||||
|
||||
// End iteration on coming out of subtrie
|
||||
if pathLen <= headPathLen {
|
||||
break
|
||||
}
|
||||
|
||||
// Avoid descending further if max depth reached
|
||||
if pathLen >= maxPathLen {
|
||||
descend = false
|
||||
} else {
|
||||
err = tx.Commit()
|
||||
descend = true
|
||||
}
|
||||
}()
|
||||
|
||||
// Fetch stem nodes
|
||||
// some of the "stem" nodes can be leaf nodes (but not value nodes)
|
||||
stemNodes, stemLeafCIDs, _, timeSpent, err := b.getStorageNodesByStateLeafKeyAndPathsAndBlockNumber(tx, stateLeafKey, stemPaths, blockHeight)
|
||||
sliceNodeMetrics, err := fillSliceNodeData(b.EthDB, b.StateDatabase.TrieDB(), response.TrieNodes.Slice, response.Leaves, it, storage)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStorageSlice stem node lookup error: %s", err.Error())
|
||||
}
|
||||
response.TrieNodes.Stem = stemNodes
|
||||
response.MetaData.TimeStats["01-fetch-stem-keys"] = timeSpent
|
||||
|
||||
// Fetch slice nodes
|
||||
sliceNodes, sliceLeafCIDs, deepestPath, timeSpent, err := b.getStorageNodesByStateLeafKeyAndPathsAndBlockNumber(tx, stateLeafKey, slicePaths, blockHeight)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStorageSlice slice node lookup error: %s", err.Error())
|
||||
}
|
||||
response.TrieNodes.Slice = sliceNodes
|
||||
response.MetaData.TimeStats["02-fetch-slice-keys"] = timeSpent
|
||||
|
||||
// Fetch head node
|
||||
headNode, headLeafCID, _, _, err := b.getStorageNodesByStateLeafKeyAndPathsAndBlockNumber(tx, stateLeafKey, [][]byte{headPath}, blockHeight)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("GetStorageSlice head node lookup error: %s", err.Error())
|
||||
}
|
||||
response.TrieNodes.Head = headNode
|
||||
|
||||
// Fill in metadata
|
||||
maxDepth := deepestPath - len(headPath)
|
||||
if maxDepth < 0 {
|
||||
maxDepth = 0
|
||||
}
|
||||
response.MetaData.NodeStats["01-max-depth"] = strconv.Itoa(maxDepth)
|
||||
|
||||
response.MetaData.NodeStats["02-total-trie-nodes"] = strconv.Itoa(len(response.TrieNodes.Stem) + len(response.TrieNodes.Slice) + 1)
|
||||
response.MetaData.NodeStats["03-leaves"] = strconv.Itoa(len(stemLeafCIDs) + len(sliceLeafCIDs) + len(headLeafCID))
|
||||
response.MetaData.NodeStats["00-stem-and-head-nodes"] = strconv.Itoa(len(response.TrieNodes.Stem) + 1)
|
||||
response.MetaData.TimeStats["03-fetch-leaves-info"] = strconv.Itoa(0)
|
||||
|
||||
return response, nil
|
||||
}
|
||||
|
||||
func (b *Backend) getBlockHeightForStateRoot(root common.Hash) (uint64, error) {
|
||||
var blockHeight uint64
|
||||
return blockHeight, b.DB.Get(&blockHeight, RetrieveBlockNumberForStateRoot, root.String())
|
||||
}
|
||||
|
||||
func (b *Backend) getBlockHeightAndStateLeafKeyForStorageRoot(root common.Hash) (uint64, string, error) {
|
||||
var res struct {
|
||||
BlockNumber uint64 `db:"block_number"`
|
||||
StateLeafKey string `db:"state_leaf_key"`
|
||||
return err
|
||||
}
|
||||
|
||||
return res.BlockNumber, res.StateLeafKey, b.DB.Get(&res, RetrieveBlockNumberAndStateLeafKeyForStorageRoot, root.String())
|
||||
}
|
||||
|
||||
func (b *Backend) getStateNodesByPathsAndBlockNumber(tx *sqlx.Tx, paths [][]byte, blockHeight uint64) (map[string]string, []cid.Cid, int, string, error) {
|
||||
nodes := make(map[string]string)
|
||||
fetchStart := makeTimestamp()
|
||||
|
||||
// Get CIDs for all nodes at the provided paths
|
||||
leafCIDs, leafIPLDs, intermediateCIDs, intermediateIPLDs, deepestPath, err := b.IPLDRetriever.RetrieveStatesByPathsAndBlockNumber(tx, paths, blockHeight)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
// Update metadata
|
||||
if (sliceNodeMetrics.pathLen - len(headPath)) > metaData.maxDepth {
|
||||
metaData.maxDepth = sliceNodeMetrics.pathLen
|
||||
}
|
||||
if sliceNodeMetrics.isLeaf {
|
||||
metaData.leafCount++
|
||||
}
|
||||
metaData.sliceNodesFetchTime += sliceNodeMetrics.nodeFetchTime
|
||||
metaData.leavesFetchTime += sliceNodeMetrics.leafFetchTime
|
||||
}
|
||||
|
||||
// Populate the nodes map for leaf nodes
|
||||
err = populateNodesMap(nodes, leafCIDs, leafIPLDs)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
}
|
||||
|
||||
// Populate the nodes map for intermediate nodes
|
||||
err = populateNodesMap(nodes, intermediateCIDs, intermediateIPLDs)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
}
|
||||
|
||||
return nodes, leafCIDs, deepestPath, strconv.Itoa(int(makeTimestamp() - fetchStart)), nil
|
||||
}
|
||||
|
||||
func (b *Backend) getAccountByStateCID(tx *sqlx.Tx, cid string, blockHeight uint64) (string, string, []byte, error) {
|
||||
var err error
|
||||
var res struct {
|
||||
StateLeafKey string `db:"state_leaf_key"`
|
||||
StorageRoot string `db:"storage_root"`
|
||||
CodeHash []byte `db:"code_hash"`
|
||||
}
|
||||
|
||||
if err = tx.Get(&res, RetrieveAccountByStateCID, cid, blockHeight); err != nil {
|
||||
return "", "", nil, err
|
||||
}
|
||||
|
||||
mhKey, err := ethServerShared.MultihashKeyFromKeccak256(common.BytesToHash(res.CodeHash))
|
||||
if err != nil {
|
||||
return "", "", nil, err
|
||||
}
|
||||
|
||||
code := make([]byte, 0)
|
||||
err = tx.Get(&code, RetrieveCodeByMhKey, mhKey)
|
||||
if err != nil {
|
||||
return "", "", nil, err
|
||||
}
|
||||
|
||||
return res.StateLeafKey, res.StorageRoot, code, nil
|
||||
}
|
||||
|
||||
func (b *Backend) getStorageNodesByStateLeafKeyAndPathsAndBlockNumber(tx *sqlx.Tx, stateLeafKey string, paths [][]byte, blockHeight uint64) (map[string]string, []cid.Cid, int, string, error) {
|
||||
nodes := make(map[string]string)
|
||||
fetchStart := makeTimestamp()
|
||||
|
||||
// Get CIDs for all nodes at the provided paths
|
||||
leafCIDs, leafIPLDs, intermediateCIDs, intermediateIPLDs, deepestPath, err := b.IPLDRetriever.RetrieveStorageByStateLeafKeyAndPathsAndBlockNumber(tx, stateLeafKey, paths, blockHeight)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
}
|
||||
|
||||
// Populate the nodes map for leaf nodes
|
||||
err = populateNodesMap(nodes, leafCIDs, leafIPLDs)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
}
|
||||
|
||||
// Populate the nodes map for intermediate nodes
|
||||
err = populateNodesMap(nodes, intermediateCIDs, intermediateIPLDs)
|
||||
if err != nil {
|
||||
return nil, nil, 0, "", err
|
||||
}
|
||||
|
||||
return nodes, leafCIDs, deepestPath, strconv.Itoa(int(makeTimestamp() - fetchStart)), nil
|
||||
return nil
|
||||
}
|
||||
|
||||
// Engine satisfied the ChainContext interface
|
||||
|
@ -17,20 +17,32 @@
|
||||
package eth
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math/big"
|
||||
|
||||
nodeiter "github.com/cerc-io/go-eth-state-node-iterator"
|
||||
"github.com/ethereum/go-ethereum"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/common/math"
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
sdtrie "github.com/ethereum/go-ethereum/statediff/trie_helpers"
|
||||
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
var nullHashBytes = common.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000")
|
||||
var emptyCodeHash = crypto.Keccak256([]byte{})
|
||||
|
||||
// RPCMarshalHeader converts the given header to the RPC output.
|
||||
// This function is eth/internal so we have to make our own version here...
|
||||
func RPCMarshalHeader(head *types.Header) map[string]interface{} {
|
||||
@ -300,3 +312,108 @@ func toBlockNumArg(number *big.Int) string {
|
||||
}
|
||||
return hexutil.EncodeBig(number)
|
||||
}
|
||||
|
||||
func getIteratorAtPath(t state.Trie, startKey []byte) (trie.NodeIterator, int64) {
|
||||
startTime := makeTimestamp()
|
||||
var it trie.NodeIterator
|
||||
|
||||
if len(startKey)%2 != 0 {
|
||||
// Zero-pad for odd-length keys, required by HexToKeyBytes()
|
||||
startKey = append(startKey, 0)
|
||||
it = t.NodeIterator(nodeiter.HexToKeyBytes(startKey))
|
||||
} else {
|
||||
it = t.NodeIterator(nodeiter.HexToKeyBytes(startKey))
|
||||
// Step to the required node (not required if original startKey was odd-length)
|
||||
it.Next(true)
|
||||
}
|
||||
|
||||
return it, makeTimestamp() - startTime
|
||||
}
|
||||
|
||||
type SliceNodeMetrics struct {
|
||||
pathLen int
|
||||
isLeaf bool
|
||||
nodeFetchTime int64
|
||||
leafFetchTime int64
|
||||
}
|
||||
|
||||
func fillSliceNodeData(
|
||||
ethDB ethdb.KeyValueReader,
|
||||
trieDB *trie.Database,
|
||||
nodesMap map[string]string,
|
||||
leavesMap map[string]GetSliceResponseAccount,
|
||||
it trie.NodeIterator,
|
||||
storage bool,
|
||||
) (SliceNodeMetrics, error) {
|
||||
// Skip value nodes
|
||||
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
|
||||
return SliceNodeMetrics{}, nil
|
||||
}
|
||||
|
||||
nodeStartTime := makeTimestamp()
|
||||
|
||||
node, nodeElements, err := sdtrie.ResolveNode(it, trieDB)
|
||||
if err != nil {
|
||||
return SliceNodeMetrics{}, err
|
||||
}
|
||||
|
||||
sliceNodeMetrics := SliceNodeMetrics{
|
||||
pathLen: len(it.Path()),
|
||||
isLeaf: node.NodeType == sdtypes.Leaf,
|
||||
}
|
||||
|
||||
// Populate the nodes map
|
||||
nodeVal := node.NodeValue
|
||||
nodeValHash := crypto.Keccak256Hash(nodeVal)
|
||||
nodesMap[common.Bytes2Hex(nodeValHash.Bytes())] = common.Bytes2Hex(nodeVal)
|
||||
|
||||
sliceNodeMetrics.nodeFetchTime = makeTimestamp() - nodeStartTime
|
||||
|
||||
// Extract account data if it's a Leaf node
|
||||
if node.NodeType == sdtypes.Leaf && !storage {
|
||||
leafStartTime := makeTimestamp()
|
||||
|
||||
stateLeafKey, storageRoot, code, err := extractContractAccountInfo(ethDB, node, nodeElements)
|
||||
if err != nil {
|
||||
return SliceNodeMetrics{}, fmt.Errorf("GetSlice account lookup error: %s", err.Error())
|
||||
}
|
||||
|
||||
if len(code) > 0 {
|
||||
// Populate the leaves map
|
||||
leavesMap[stateLeafKey] = GetSliceResponseAccount{
|
||||
StorageRoot: storageRoot,
|
||||
EVMCode: common.Bytes2Hex(code),
|
||||
}
|
||||
}
|
||||
|
||||
sliceNodeMetrics.leafFetchTime = makeTimestamp() - leafStartTime
|
||||
}
|
||||
|
||||
return sliceNodeMetrics, nil
|
||||
}
|
||||
|
||||
func extractContractAccountInfo(ethDB ethdb.KeyValueReader, node sdtypes.StateNode, nodeElements []interface{}) (string, string, []byte, error) {
|
||||
var account types.StateAccount
|
||||
if err := rlp.DecodeBytes(nodeElements[1].([]byte), &account); err != nil {
|
||||
return "", "", nil, fmt.Errorf("error decoding account for leaf node at path %x nerror: %v", node.Path, err)
|
||||
}
|
||||
|
||||
if bytes.Equal(account.CodeHash, emptyCodeHash) {
|
||||
return "", "", nil, nil
|
||||
}
|
||||
|
||||
// Extract state leaf key
|
||||
partialPath := trie.CompactToHex(nodeElements[0].([]byte))
|
||||
valueNodePath := append(node.Path, partialPath...)
|
||||
encodedPath := trie.HexToCompact(valueNodePath)
|
||||
leafKey := encodedPath[1:]
|
||||
stateLeafKeyString := common.BytesToHash(leafKey).String()
|
||||
|
||||
storageRootString := account.Root.String()
|
||||
|
||||
// Extract codeHash and get code
|
||||
codeHash := common.BytesToHash(account.CodeHash)
|
||||
codeBytes := rawdb.ReadCode(ethDB, codeHash)
|
||||
|
||||
return stateLeafKeyString, storageRootString, codeBytes, nil
|
||||
}
|
||||
|
@ -17,16 +17,9 @@
|
||||
package eth
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
|
||||
"github.com/ipfs/go-cid"
|
||||
"github.com/multiformats/go-multihash"
|
||||
)
|
||||
|
||||
func ResolveToNodeType(nodeType int) sdtypes.NodeType {
|
||||
@ -44,86 +37,7 @@ func ResolveToNodeType(nodeType int) sdtypes.NodeType {
|
||||
}
|
||||
}
|
||||
|
||||
var pathSteps = []byte{'\x00', '\x01', '\x02', '\x03', '\x04', '\x05', '\x06', '\x07', '\x08', '\x09', '\x0a', '\x0b', '\x0c', '\x0d', '\x0e', '\x0f'}
|
||||
|
||||
// Return head, stem, and slice byte paths for the given head path and depth
|
||||
func getPaths(path string, depth int) ([]byte, [][]byte, [][]byte, error) {
|
||||
// Convert the head hex path to a decoded byte path
|
||||
headPath := common.FromHex(path)
|
||||
|
||||
pathLen := len(headPath)
|
||||
if pathLen > 64 { // max path len is 64
|
||||
return nil, nil, nil, fmt.Errorf("path length cannot exceed 64; got %d", pathLen)
|
||||
}
|
||||
|
||||
maxDepth := 64 - pathLen
|
||||
if depth > maxDepth {
|
||||
return nil, nil, nil, fmt.Errorf("max depth for path %s is %d; got %d", path, maxDepth, depth)
|
||||
}
|
||||
|
||||
// Collect all of the stem paths
|
||||
stemPaths := make([][]byte, 0, pathLen)
|
||||
for i := 0; i < pathLen; i++ {
|
||||
stemPaths = append(stemPaths, headPath[:i])
|
||||
}
|
||||
|
||||
// Generate all of the slice paths
|
||||
slicePaths := make([][]byte, 0, int(math.Pow(16, float64(depth))))
|
||||
makeSlicePaths(headPath, depth, &slicePaths)
|
||||
|
||||
return headPath, stemPaths, slicePaths, nil
|
||||
}
|
||||
|
||||
// An iterative function to generate the set of slice paths
|
||||
func makeSlicePaths(path []byte, depth int, slicePaths *[][]byte) {
|
||||
// return if depth has reached 0
|
||||
if depth <= 0 {
|
||||
return
|
||||
}
|
||||
depth--
|
||||
|
||||
// slice to hold the next 16 paths
|
||||
nextPaths := make([][]byte, 0, 16)
|
||||
for _, step := range pathSteps {
|
||||
// create next paths by adding steps to current path
|
||||
nextPath := make([]byte, len(path))
|
||||
copy(nextPath, path)
|
||||
nextPath = append(nextPath, step)
|
||||
|
||||
nextPaths = append(nextPaths, nextPath)
|
||||
|
||||
// also add the next path to the collection of all slice paths
|
||||
dst := make([]byte, len(nextPath))
|
||||
copy(dst, nextPath)
|
||||
*slicePaths = append(*slicePaths, dst)
|
||||
}
|
||||
|
||||
// iterate over the next paths to repeat the process if not
|
||||
for _, nextPath := range nextPaths {
|
||||
makeSlicePaths(nextPath, depth, slicePaths)
|
||||
}
|
||||
}
|
||||
|
||||
// Timestamp in milliseconds
|
||||
func makeTimestamp() int64 {
|
||||
return time.Now().UnixNano() / int64(time.Millisecond)
|
||||
}
|
||||
|
||||
func populateNodesMap(nodes map[string]string, cids []cid.Cid, iplds [][]byte) error {
|
||||
for i, cid := range cids {
|
||||
decodedMh, err := multihash.Decode(cid.Hash())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
data := iplds[i]
|
||||
hash := crypto.Keccak256Hash(data)
|
||||
if !bytes.Equal(hash.Bytes(), decodedMh.Digest) {
|
||||
return fmt.Errorf("multihash digest should equal keccak of raw data")
|
||||
}
|
||||
|
||||
nodes[common.Bytes2Hex(decodedMh.Digest)] = common.Bytes2Hex(data)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -17,14 +17,12 @@
|
||||
package eth
|
||||
|
||||
import (
|
||||
"database/sql"
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"github.com/cerc-io/ipld-eth-server/v4/pkg/shared"
|
||||
"github.com/ethereum/go-ethereum/statediff/trie_helpers"
|
||||
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
|
||||
"github.com/ipfs/go-cid"
|
||||
"github.com/jmoiron/sqlx"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
@ -236,34 +234,6 @@ const (
|
||||
)
|
||||
WHERE tx_hash = $1
|
||||
AND transaction_cids.header_id = (SELECT canonical_header_hash(transaction_cids.block_number))`
|
||||
RetrieveStateByPathAndBlockNumberPgStr = `SELECT cid, data, node_type
|
||||
FROM eth.state_cids
|
||||
INNER JOIN public.blocks ON (
|
||||
state_cids.mh_key = blocks.key
|
||||
AND state_cids.block_number = blocks.block_number
|
||||
)
|
||||
WHERE state_path = $1
|
||||
AND state_cids.block_number <= $2
|
||||
AND state_cids.header_id = (SELECT canonical_header_hash(state_cids.block_number))
|
||||
ORDER BY state_cids.block_number DESC
|
||||
LIMIT 1`
|
||||
RetrieveStorageByStateLeafKeyAndPathAndBlockNumberPgStr = `SELECT storage_cids.cid, data, storage_cids.node_type
|
||||
FROM eth.storage_cids
|
||||
INNER JOIN eth.state_cids ON (
|
||||
storage_cids.state_path = state_cids.state_path
|
||||
AND storage_cids.header_id = state_cids.header_id
|
||||
AND storage_cids.block_number = state_cids.block_number
|
||||
)
|
||||
INNER JOIN public.blocks ON (
|
||||
storage_cids.mh_key = blocks.key
|
||||
AND storage_cids.block_number = blocks.block_number
|
||||
)
|
||||
WHERE state_leaf_key = $1
|
||||
AND storage_path = $2
|
||||
AND storage_cids.block_number <= $3
|
||||
AND storage_cids.header_id = (SELECT canonical_header_hash(storage_cids.block_number))
|
||||
ORDER BY storage_cids.block_number DESC
|
||||
LIMIT 1`
|
||||
RetrieveAccountByLeafKeyAndBlockHashPgStr = `SELECT state_cids.cid, state_cids.mh_key, state_cids.block_number, state_cids.node_type
|
||||
FROM eth.state_cids
|
||||
INNER JOIN eth.header_cids ON (
|
||||
@ -749,103 +719,3 @@ func (r *IPLDRetriever) RetrieveStorageAtByAddressAndStorageKeyAndBlockNumber(ad
|
||||
}
|
||||
return storageResult.CID, i[1].([]byte), nil
|
||||
}
|
||||
|
||||
// RetrieveStatesByPathsAndBlockNumber returns the cid and rlp bytes for the state nodes corresponding to the provided state paths and block number
|
||||
func (r *IPLDRetriever) RetrieveStatesByPathsAndBlockNumber(tx *sqlx.Tx, paths [][]byte, number uint64) ([]cid.Cid, [][]byte, []cid.Cid, [][]byte, int, error) {
|
||||
deepestPath := 0
|
||||
|
||||
leafNodeCIDs := make([]cid.Cid, 0)
|
||||
intermediateNodeCIDs := make([]cid.Cid, 0)
|
||||
|
||||
leafNodeIPLDs := make([][]byte, 0)
|
||||
intermediateNodeIPLDs := make([][]byte, 0)
|
||||
|
||||
// TODO: fetch all nodes in a single query
|
||||
for _, path := range paths {
|
||||
// Create a result object, select: cid, data, node_type
|
||||
res := new(nodeInfo)
|
||||
if err := tx.Get(res, RetrieveStateByPathAndBlockNumberPgStr, path, number); err != nil {
|
||||
// Skip if node not found for a path
|
||||
if err == sql.ErrNoRows {
|
||||
continue
|
||||
}
|
||||
|
||||
return nil, nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
// Skip if node is of removed type
|
||||
if res.NodeType == sdtypes.Removed.Int() {
|
||||
continue
|
||||
}
|
||||
|
||||
pathLen := len(path)
|
||||
if pathLen > deepestPath {
|
||||
deepestPath = pathLen
|
||||
}
|
||||
|
||||
cid, err := cid.Decode(res.CID)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
if res.NodeType == sdtypes.Leaf.Int() {
|
||||
leafNodeCIDs = append(leafNodeCIDs, cid)
|
||||
leafNodeIPLDs = append(leafNodeIPLDs, res.Data)
|
||||
} else {
|
||||
intermediateNodeCIDs = append(intermediateNodeCIDs, cid)
|
||||
intermediateNodeIPLDs = append(intermediateNodeIPLDs, res.Data)
|
||||
}
|
||||
}
|
||||
|
||||
return leafNodeCIDs, leafNodeIPLDs, intermediateNodeCIDs, intermediateNodeIPLDs, deepestPath, nil
|
||||
}
|
||||
|
||||
// RetrieveStorageByStateLeafKeyAndPathsAndBlockNumber returns the cid and rlp bytes for the storage nodes corresponding to the provided state leaf key, storage paths and block number
|
||||
func (r *IPLDRetriever) RetrieveStorageByStateLeafKeyAndPathsAndBlockNumber(tx *sqlx.Tx, stateLeafKey string, paths [][]byte, number uint64) ([]cid.Cid, [][]byte, []cid.Cid, [][]byte, int, error) {
|
||||
deepestPath := 0
|
||||
|
||||
leafNodeCIDs := make([]cid.Cid, 0)
|
||||
intermediateNodeCIDs := make([]cid.Cid, 0)
|
||||
|
||||
leafNodeIPLDs := make([][]byte, 0)
|
||||
intermediateNodeIPLDs := make([][]byte, 0)
|
||||
|
||||
// TODO: fetch all nodes in a single query
|
||||
for _, path := range paths {
|
||||
// Create a result object, select: cid, data, node_type
|
||||
res := new(nodeInfo)
|
||||
if err := tx.Get(res, RetrieveStorageByStateLeafKeyAndPathAndBlockNumberPgStr, stateLeafKey, path, number); err != nil {
|
||||
// Skip if node not found for a path
|
||||
if err == sql.ErrNoRows {
|
||||
continue
|
||||
}
|
||||
|
||||
return nil, nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
// Skip if node is of removed type
|
||||
if res.NodeType == sdtypes.Removed.Int() {
|
||||
continue
|
||||
}
|
||||
|
||||
pathLen := len(path)
|
||||
if pathLen > deepestPath {
|
||||
deepestPath = pathLen
|
||||
}
|
||||
|
||||
cid, err := cid.Decode(res.CID)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
if res.NodeType == sdtypes.Leaf.Int() {
|
||||
leafNodeCIDs = append(leafNodeCIDs, cid)
|
||||
leafNodeIPLDs = append(leafNodeIPLDs, res.Data)
|
||||
} else {
|
||||
intermediateNodeCIDs = append(intermediateNodeCIDs, cid)
|
||||
intermediateNodeIPLDs = append(intermediateNodeIPLDs, res.Data)
|
||||
}
|
||||
}
|
||||
|
||||
return leafNodeCIDs, leafNodeIPLDs, intermediateNodeCIDs, intermediateNodeIPLDs, deepestPath, nil
|
||||
}
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/big"
|
||||
"strconv"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
@ -288,6 +289,19 @@ func (sr *GetSliceResponse) init(path string, depth int, root common.Hash) {
|
||||
}
|
||||
}
|
||||
|
||||
func (sr *GetSliceResponse) populateMetaData(metaData metaDataFields) {
|
||||
sr.MetaData.NodeStats["00-stem-and-head-nodes"] = strconv.Itoa(len(sr.TrieNodes.Stem) + len(sr.TrieNodes.Head))
|
||||
sr.MetaData.NodeStats["01-max-depth"] = strconv.Itoa(metaData.maxDepth)
|
||||
sr.MetaData.NodeStats["02-total-trie-nodes"] = strconv.Itoa(len(sr.TrieNodes.Stem) + len(sr.TrieNodes.Head) + len(sr.TrieNodes.Slice))
|
||||
sr.MetaData.NodeStats["03-leaves"] = strconv.Itoa(metaData.leafCount)
|
||||
sr.MetaData.NodeStats["04-smart-contracts"] = strconv.Itoa(len(sr.Leaves))
|
||||
|
||||
sr.MetaData.TimeStats["00-trie-loading"] = strconv.FormatInt(metaData.trieLoadingTime, 10)
|
||||
sr.MetaData.TimeStats["01-fetch-stem-keys"] = strconv.FormatInt(metaData.stemNodesFetchTime, 10)
|
||||
sr.MetaData.TimeStats["02-fetch-slice-keys"] = strconv.FormatInt(metaData.sliceNodesFetchTime, 10)
|
||||
sr.MetaData.TimeStats["03-fetch-leaves-info"] = strconv.FormatInt(metaData.leavesFetchTime, 10)
|
||||
}
|
||||
|
||||
type GetSliceResponseMetadata struct {
|
||||
TimeStats map[string]string `json:"timeStats"` // stem, state, storage (one by one)
|
||||
NodeStats map[string]string `json:"nodeStats"` // total, leaves, smart contracts
|
||||
@ -303,3 +317,12 @@ type GetSliceResponseAccount struct {
|
||||
StorageRoot string `json:"storageRoot"`
|
||||
EVMCode string `json:"evmCode"`
|
||||
}
|
||||
|
||||
type metaDataFields struct {
|
||||
maxDepth int
|
||||
leafCount int
|
||||
trieLoadingTime int64
|
||||
stemNodesFetchTime int64
|
||||
sliceNodesFetchTime int64
|
||||
leavesFetchTime int64
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user