Merge pull request #102 from vulcanize/log-table

Create a seperate table for storing logs.
This commit is contained in:
Arijit Das 2021-08-27 12:50:08 +05:30 committed by GitHub
commit 639af8b849
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 762 additions and 207 deletions

View File

@ -33,4 +33,3 @@ jobs:
run: echo ${{ secrets.GITHUB_TOKEN }} | docker login https://docker.pkg.github.com -u vulcanize --password-stdin
- name: Docker Push
run: docker push docker.pkg.github.com/vulcanize/go-ethereum/go-ethereum:${{steps.vars.outputs.sha}}

View File

@ -70,6 +70,7 @@ type Receipt struct {
BlockHash common.Hash `json:"blockHash,omitempty"`
BlockNumber *big.Int `json:"blockNumber,omitempty"`
TransactionIndex uint `json:"transactionIndex"`
LogRoot common.Hash `json:"logRoot"`
}
type receiptMarshaling struct {
@ -211,7 +212,7 @@ func (r *Receipt) DecodeRLP(s *rlp.Stream) error {
}
}
// UnmarshalBinary decodes the canonical encoding of receipts.
// UnmarshalBinary decodes the consensus encoding of receipts.
// It supports legacy RLP receipts and EIP-2718 typed receipts.
func (r *Receipt) UnmarshalBinary(b []byte) error {
if len(b) > 0 && b[0] > 0x7f {
@ -234,13 +235,13 @@ func (r *Receipt) decodeTyped(b []byte) error {
return errEmptyTypedReceipt
}
switch b[0] {
case AccessListTxType:
case DynamicFeeTxType, AccessListTxType:
var data receiptRLP
err := rlp.DecodeBytes(b[1:], &data)
if err != nil {
return err
}
r.Type = AccessListTxType
r.Type = b[0]
return r.setFromRLP(data)
default:
return ErrTxTypeNotSupported

View File

@ -29,6 +29,7 @@ import (
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp"
sdtrie "github.com/ethereum/go-ethereum/statediff/trie"
. "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
)
@ -51,28 +52,6 @@ type builder struct {
stateCache state.Database
}
func resolveNode(it trie.NodeIterator, trieDB *trie.Database) (StateNode, []interface{}, error) {
nodePath := make([]byte, len(it.Path()))
copy(nodePath, it.Path())
node, err := trieDB.Node(it.Hash())
if err != nil {
return StateNode{}, nil, err
}
var nodeElements []interface{}
if err := rlp.DecodeBytes(node, &nodeElements); err != nil {
return StateNode{}, nil, err
}
ty, err := CheckKeyType(nodeElements)
if err != nil {
return StateNode{}, nil, err
}
return StateNode{
NodeType: ty,
Path: nodePath,
NodeValue: node,
}, nodeElements, nil
}
// convenience
func stateNodeAppender(nodes *[]StateNode) StateNodeSink {
return func(node StateNode) error {
@ -127,7 +106,7 @@ func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]StateNode, []CodeAnd
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return nil, nil, err
}
@ -319,7 +298,7 @@ func (sdb *builder) createdAndUpdatedState(a, b trie.NodeIterator, watchedAddres
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return nil, nil, err
}
@ -363,7 +342,7 @@ func (sdb *builder) createdAndUpdatedStateWithIntermediateNodes(a, b trie.NodeIt
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return nil, nil, err
}
@ -415,7 +394,7 @@ func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB m
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return nil, err
}
@ -576,7 +555,7 @@ func (sdb *builder) buildStorageNodesFromTrie(it trie.NodeIterator, watchedStora
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return err
}
@ -650,7 +629,7 @@ func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return nil, err
}
@ -695,7 +674,7 @@ func (sdb *builder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffPathsAtB
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := resolveNode(it, sdb.stateCache.TrieDB())
node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB())
if err != nil {
return err
}

View File

@ -6,15 +6,11 @@ CREATE TABLE eth.receipt_cids (
mh_key TEXT NOT NULL REFERENCES public.blocks (key) ON DELETE CASCADE DEFERRABLE INITIALLY DEFERRED,
contract VARCHAR(66),
contract_hash VARCHAR(66),
topic0s VARCHAR(66)[],
topic1s VARCHAR(66)[],
topic2s VARCHAR(66)[],
topic3s VARCHAR(66)[],
log_contracts VARCHAR(66)[],
post_state VARCHAR(66),
post_status INTEGER,
log_root VARCHAR(66),
UNIQUE (tx_id)
);
-- +goose Down
DROP TABLE eth.receipt_cids;
DROP TABLE eth.receipt_cids;

View File

@ -36,16 +36,6 @@ CREATE INDEX rct_contract_index ON eth.receipt_cids USING btree (contract);
CREATE INDEX rct_contract_hash_index ON eth.receipt_cids USING btree (contract_hash);
CREATE INDEX rct_topic0_index ON eth.receipt_cids USING gin (topic0s);
CREATE INDEX rct_topic1_index ON eth.receipt_cids USING gin (topic1s);
CREATE INDEX rct_topic2_index ON eth.receipt_cids USING gin (topic2s);
CREATE INDEX rct_topic3_index ON eth.receipt_cids USING gin (topic3s);
CREATE INDEX rct_log_contract_index ON eth.receipt_cids USING gin (log_contracts);
-- state node indexes
CREATE INDEX state_header_id_index ON eth.state_cids USING btree (header_id);
@ -93,11 +83,6 @@ DROP INDEX eth.state_leaf_key_index;
DROP INDEX eth.state_header_id_index;
-- receipt indexes
DROP INDEX eth.rct_log_contract_index;
DROP INDEX eth.rct_topic3_index;
DROP INDEX eth.rct_topic2_index;
DROP INDEX eth.rct_topic1_index;
DROP INDEX eth.rct_topic0_index;
DROP INDEX eth.rct_contract_hash_index;
DROP INDEX eth.rct_contract_index;
DROP INDEX eth.rct_mh_index;
@ -118,4 +103,4 @@ DROP INDEX eth.state_root_index;
DROP INDEX eth.header_mh_index;
DROP INDEX eth.header_cid_index;
DROP INDEX eth.block_hash_index;
DROP INDEX eth.block_number_index;
DROP INDEX eth.block_number_index;

View File

@ -0,0 +1,60 @@
-- +goose Up
CREATE TABLE eth.log_cids (
id SERIAL PRIMARY KEY,
leaf_cid TEXT NOT NULL,
leaf_mh_key TEXT NOT NULL REFERENCES public.blocks (key) ON DELETE CASCADE DEFERRABLE INITIALLY DEFERRED,
receipt_id INTEGER NOT NULL REFERENCES eth.receipt_cids (id) ON DELETE CASCADE DEFERRABLE INITIALLY DEFERRED,
address VARCHAR(66),
log_data BYTEA,
index INTEGER NOT NULL,
topic0 VARCHAR(66),
topic1 VARCHAR(66),
topic2 VARCHAR(66),
topic3 VARCHAR(66),
UNIQUE (receipt_id, index)
);
CREATE INDEX log_mh_index ON eth.log_cids USING btree (leaf_mh_key);
CREATE INDEX log_cid_index ON eth.log_cids USING btree (leaf_cid);
CREATE INDEX log_rct_id_index ON eth.log_cids USING btree (receipt_id);
--
-- Name: log_topic0_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX log_topic0_index ON eth.log_cids USING btree (topic0);
--
-- Name: log_topic1_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX log_topic1_index ON eth.log_cids USING btree (topic1);
--
-- Name: log_topic2_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX log_topic2_index ON eth.log_cids USING btree (topic2);
--
-- Name: log_topic3_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX log_topic3_index ON eth.log_cids USING btree (topic3);
-- +goose Down
-- log indexes
DROP INDEX eth.log_mh_index;
DROP INDEX eth.log_cid_index;
DROP INDEX eth.log_rct_id_index;
DROP INDEX eth.log_topic0_index;
DROP INDEX eth.log_topic1_index;
DROP INDEX eth.log_topic2_index;
DROP INDEX eth.log_topic3_index;
DROP TABLE eth.log_cids;

View File

@ -998,34 +998,6 @@ CREATE INDEX rct_log_contract_index ON eth.receipt_cids USING gin (log_contracts
CREATE INDEX rct_mh_index ON eth.receipt_cids USING btree (mh_key);
--
-- Name: rct_topic0_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX rct_topic0_index ON eth.receipt_cids USING gin (topic0s);
--
-- Name: rct_topic1_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX rct_topic1_index ON eth.receipt_cids USING gin (topic1s);
--
-- Name: rct_topic2_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX rct_topic2_index ON eth.receipt_cids USING gin (topic2s);
--
-- Name: rct_topic3_index; Type: INDEX; Schema: eth; Owner: -
--
CREATE INDEX rct_topic3_index ON eth.receipt_cids USING gin (topic3s);
--
-- Name: rct_tx_id_index; Type: INDEX; Schema: eth; Owner: -
--
@ -1330,4 +1302,3 @@ ALTER TABLE ONLY eth.uncle_cids
--
-- PostgreSQL database dump complete
--

View File

@ -20,11 +20,8 @@
package statediff
import (
"fmt"
"sort"
"strings"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
func sortKeys(data AccountMap) []string {
@ -74,25 +71,3 @@ func findIntersection(a, b []string) []string {
}
}
// CheckKeyType checks what type of key we have
func CheckKeyType(elements []interface{}) (sdtypes.NodeType, error) {
if len(elements) > 2 {
return sdtypes.Branch, nil
}
if len(elements) < 2 {
return sdtypes.Unknown, fmt.Errorf("node cannot be less than two elements in length")
}
switch elements[0].([]byte)[0] / 16 {
case '\x00':
return sdtypes.Extension, nil
case '\x01':
return sdtypes.Extension, nil
case '\x02':
return sdtypes.Leaf, nil
case '\x03':
return sdtypes.Leaf, nil
default:
return sdtypes.Unknown, fmt.Errorf("unknown hex prefix")
}
}

View File

@ -36,7 +36,7 @@ import (
"github.com/ethereum/go-ethereum/statediff/indexer/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ipfs/go-cid"
node "github.com/ipfs/go-ipld-format"
"github.com/jmoiron/sqlx"
"github.com/multiformats/go-multihash"
@ -108,14 +108,17 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, transactions); err != nil {
return nil, err
}
// Generate the block iplds
headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, err := ipld.FromBlockAndReceipts(block, receipts)
headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, logTrieNodes, logLeafNodeCIDs, err := ipld.FromBlockAndReceipts(block, receipts)
if err != nil {
return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err)
}
if len(txNodes) != len(txTrieNodes) && len(rctNodes) != len(rctTrieNodes) && len(txNodes) != len(rctNodes) {
return nil, fmt.Errorf("expected number of transactions (%d), transaction trie nodes (%d), receipts (%d), and receipt trie nodes (%d)to be equal", len(txNodes), len(txTrieNodes), len(rctNodes), len(rctTrieNodes))
}
// Calculate reward
var reward *big.Int
// in PoA networks block reward is 0
@ -189,14 +192,16 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
t = time.Now()
// Publish and index receipts and txs
err = sdi.processReceiptsAndTxs(tx, processArgs{
headerID: headerID,
blockNumber: block.Number(),
receipts: receipts,
txs: transactions,
rctNodes: rctNodes,
rctTrieNodes: rctTrieNodes,
txNodes: txNodes,
txTrieNodes: txTrieNodes,
headerID: headerID,
blockNumber: block.Number(),
receipts: receipts,
txs: transactions,
rctNodes: rctNodes,
rctTrieNodes: rctTrieNodes,
txNodes: txNodes,
txTrieNodes: txTrieNodes,
logTrieNodes: logTrieNodes,
logLeafNodeCIDs: logLeafNodeCIDs,
})
if err != nil {
return nil, err
@ -273,14 +278,16 @@ func (sdi *StateDiffIndexer) processUncles(tx *sqlx.Tx, headerID int64, blockNum
// processArgs bundles arguments to processReceiptsAndTxs
type processArgs struct {
headerID int64
blockNumber *big.Int
receipts types.Receipts
txs types.Transactions
rctNodes []*ipld.EthReceipt
rctTrieNodes []*ipld.EthRctTrie
txNodes []*ipld.EthTx
txTrieNodes []*ipld.EthTxTrie
headerID int64
blockNumber *big.Int
receipts types.Receipts
txs types.Transactions
rctNodes []*ipld.EthReceipt
rctTrieNodes []*ipld.EthRctTrie
txNodes []*ipld.EthTx
txTrieNodes []*ipld.EthTxTrie
logTrieNodes [][]*ipld.EthLogTrie
logLeafNodeCIDs [][]cid.Cid
}
// processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres
@ -295,14 +302,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs
return fmt.Errorf("error deriving tx sender: %v", err)
}
// Publishing
// publish trie nodes, these aren't indexed directly
if err := shared.PublishIPLD(tx, args.txTrieNodes[i]); err != nil {
return fmt.Errorf("error publishing tx trie node IPLD: %v", err)
}
if err := shared.PublishIPLD(tx, args.rctTrieNodes[i]); err != nil {
return fmt.Errorf("error publishing rct trie node IPLD: %v", err)
for _, trie := range args.logTrieNodes[i] {
if err = shared.PublishIPLD(tx, trie); err != nil {
return fmt.Errorf("error publishing log trie node IPLD: %w", err)
}
}
// publish the txs and receipts
txNode, rctNode := args.txNodes[i], args.rctNodes[i]
if err := shared.PublishIPLD(tx, txNode); err != nil {
@ -314,13 +319,31 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs
// Indexing
// extract topic and contract data from the receipt for indexing
topicSets := make([][]string, 4)
mappedContracts := make(map[string]bool) // use map to avoid duplicate addresses
for _, log := range receipt.Logs {
for i, topic := range log.Topics {
topicSets[i] = append(topicSets[i], topic.Hex())
logDataSet := make([]*models.LogsModel, len(receipt.Logs))
for idx, l := range receipt.Logs {
topicSet := make([]string, 4)
for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex()
}
if !args.logLeafNodeCIDs[i][idx].Defined() {
return fmt.Errorf("invalid log cid")
}
mappedContracts[l.Address.String()] = true
logDataSet[idx] = &models.LogsModel{
ID: 0,
Address: l.Address.String(),
Index: int64(l.Index),
Data: l.Data,
LeafCID: args.logLeafNodeCIDs[i][idx].String(),
LeafMhKey: shared.MultihashKeyFromCID(args.logLeafNodeCIDs[i][idx]),
Topic0: topicSet[0],
Topic1: topicSet[1],
Topic2: topicSet[2],
Topic3: topicSet[3],
}
mappedContracts[log.Address.String()] = true
}
// these are the contracts seen in the logs
logContracts := make([]string, 0, len(mappedContracts))
@ -368,26 +391,42 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs
}
}
// index the receipt
rctModel := models.ReceiptModel{
Topic0s: topicSets[0],
Topic1s: topicSets[1],
Topic2s: topicSets[2],
Topic3s: topicSets[3],
rctModel := &models.ReceiptModel{
Contract: contract,
ContractHash: contractHash,
LogContracts: logContracts,
CID: rctNode.Cid().String(),
MhKey: shared.MultihashKeyFromCID(rctNode.Cid()),
LogRoot: rctNode.LogRoot.String(),
}
if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status
} else {
rctModel.PostState = common.Bytes2Hex(receipt.PostState)
}
if err := sdi.dbWriter.upsertReceiptCID(tx, rctModel, txID); err != nil {
receiptID, err := sdi.dbWriter.upsertReceiptCID(tx, rctModel, txID)
if err != nil {
return err
}
if err = sdi.dbWriter.upsertLogCID(tx, logDataSet, receiptID); err != nil {
return err
}
}
// publish trie nodes, these aren't indexed directly
for _, n := range args.txTrieNodes {
if err := shared.PublishIPLD(tx, n); err != nil {
return fmt.Errorf("error publishing tx trie node IPLD: %w", err)
}
}
for _, n := range args.rctTrieNodes {
if err := shared.PublishIPLD(tx, n); err != nil {
return fmt.Errorf("error publishing rct trie node IPLD: %w", err)
}
}
return nil
}

View File

@ -24,17 +24,18 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer"
"github.com/ethereum/go-ethereum/statediff/indexer/ipfs/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ipfs/go-cid"
blockstore "github.com/ipfs/go-ipfs-blockstore"
dshelp "github.com/ipfs/go-ipfs-ds-help"
"github.com/multiformats/go-multihash"
"github.com/stretchr/testify/require"
)
var (
@ -309,6 +310,43 @@ func TestPublishAndIndexer(t *testing.T) {
}
})
t.Run("Publish and index log IPLDs for single receipt", func(t *testing.T) {
setup(t)
defer tearDown(t)
type logIPLD struct {
Index int `db:"index"`
Address string `db:"address"`
Data []byte `db:"data"`
Topic0 string `db:"topic0"`
Topic1 string `db:"topic1"`
}
results := make([]logIPLD, 0)
pgStr := `SELECT log_cids.index, log_cids.address, log_cids.Topic0, log_cids.Topic1, data FROM eth.log_cids
INNER JOIN eth.receipt_cids ON (log_cids.receipt_id = receipt_cids.id)
INNER JOIN public.blocks ON (log_cids.leaf_mh_key = blocks.key)
WHERE receipt_cids.cid = $1 ORDER BY eth.log_cids.index ASC`
err = db.Select(&results, pgStr, rct4CID.String())
require.NoError(t, err)
// expecting MockLog1 and MockLog2 for mockReceipt4
expectedLogs := mocks.MockReceipts[3].Logs
shared.ExpectEqual(t, len(results), len(expectedLogs))
var nodeElements []interface{}
for idx, r := range results {
// Decode the log leaf node.
err = rlp.DecodeBytes(r.Data, &nodeElements)
require.NoError(t, err)
logRaw, err := rlp.EncodeToBytes(expectedLogs[idx])
require.NoError(t, err)
// 2nd element of the leaf node contains the encoded log data.
shared.ExpectEqual(t, logRaw, nodeElements[1].([]byte))
}
})
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
setup(t)
defer tearDown(t)

View File

@ -9,11 +9,10 @@ import (
"strconv"
"testing"
"github.com/ethereum/go-ethereum/core/types"
block "github.com/ipfs/go-block-format"
node "github.com/ipfs/go-ipld-format"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types"
)
func TestBlockBodyRlpParsing(t *testing.T) {

View File

@ -0,0 +1,157 @@
package ipld
import (
"fmt"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ipfs/go-cid"
node "github.com/ipfs/go-ipld-format"
mh "github.com/multiformats/go-multihash"
)
// EthLog (eth-log, codec 0x9a), represents an ethereum block header
type EthLog struct {
*types.Log
rawData []byte
cid cid.Cid
}
// Static (compile time) check that EthLog satisfies the node.Node interface.
var _ node.Node = (*EthLog)(nil)
// NewLog create a new EthLog IPLD node
func NewLog(log *types.Log) (*EthLog, error) {
logRaw, err := rlp.EncodeToBytes(log)
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthLog, logRaw, mh.KECCAK_256)
if err != nil {
return nil, err
}
return &EthLog{
Log: log,
cid: c,
rawData: logRaw,
}, nil
}
// DecodeEthLogs takes a cid and its raw binary data
func DecodeEthLogs(c cid.Cid, b []byte) (*EthLog, error) {
l := new(types.Log)
if err := rlp.DecodeBytes(b, l); err != nil {
return nil, err
}
return &EthLog{
Log: l,
cid: c,
rawData: b,
}, nil
}
/*
Block INTERFACE
*/
// RawData returns the binary of the RLP encode of the log.
func (l *EthLog) RawData() []byte {
return l.rawData
}
// Cid returns the cid of the receipt log.
func (l *EthLog) Cid() cid.Cid {
return l.cid
}
// String is a helper for output
func (l *EthLog) String() string {
return fmt.Sprintf("<EthereumLog %s>", l.cid)
}
// Loggable returns in a map the type of IPLD Link.
func (l *EthLog) Loggable() map[string]interface{} {
return map[string]interface{}{
"type": "eth-log",
}
}
// Resolve resolves a path through this node, stopping at any link boundary
// and returning the object found as well as the remaining path to traverse
func (l *EthLog) Resolve(p []string) (interface{}, []string, error) {
if len(p) == 0 {
return l, nil, nil
}
if len(p) > 1 {
return nil, nil, fmt.Errorf("unexpected path elements past %s", p[0])
}
switch p[0] {
case "address":
return l.Address, nil, nil
case "data":
// This is a []byte. By default they are marshalled into Base64.
return fmt.Sprintf("0x%x", l.Data), nil, nil
case "topics":
return l.Topics, nil, nil
case "logIndex":
return l.Index, nil, nil
case "removed":
return l.Removed, nil, nil
default:
return nil, nil, ErrInvalidLink
}
}
// Tree lists all paths within the object under 'path', and up to the given depth.
// To list the entire object (similar to `find .`) pass "" and -1
func (l *EthLog) Tree(p string, depth int) []string {
if p != "" || depth == 0 {
return nil
}
return []string{
"address",
"data",
"topics",
"logIndex",
"removed",
}
}
// ResolveLink is a helper function that calls resolve and asserts the
// output is a link
func (l *EthLog) ResolveLink(p []string) (*node.Link, []string, error) {
obj, rest, err := l.Resolve(p)
if err != nil {
return nil, nil, err
}
if lnk, ok := obj.(*node.Link); ok {
return lnk, rest, nil
}
return nil, nil, fmt.Errorf("resolved item was not a link")
}
// Copy will go away. It is here to comply with the Node interface.
func (l *EthLog) Copy() node.Node {
panic("implement me")
}
// Links is a helper function that returns all links within this object
func (l *EthLog) Links() []*node.Link {
return nil
}
// Stat will go away. It is here to comply with the interface.
func (l *EthLog) Stat() (*node.NodeStat, error) {
return &node.NodeStat{}, nil
}
// Size will go away. It is here to comply with the interface.
func (l *EthLog) Size() (uint64, error) {
return 0, nil
}

View File

@ -0,0 +1,148 @@
package ipld
import (
"fmt"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash"
)
// EthLogTrie (eth-tx-trie codec 0x9p) represents
// a node from the transaction trie in ethereum.
type EthLogTrie struct {
*TrieNode
}
/*
OUTPUT
*/
// DecodeEthLogTrie returns an EthLogTrie object from its cid and rawdata.
func DecodeEthLogTrie(c cid.Cid, b []byte) (*EthLogTrie, error) {
tn, err := decodeTrieNode(c, b, decodeEthLogTrieLeaf)
if err != nil {
return nil, err
}
return &EthLogTrie{TrieNode: tn}, nil
}
// decodeEthLogTrieLeaf parses a eth-log-trie leaf
// from decoded RLP elements
func decodeEthLogTrieLeaf(i []interface{}) ([]interface{}, error) {
l := new(types.Log)
if err := rlp.DecodeBytes(i[1].([]byte), l); err != nil {
return nil, err
}
c, err := RawdataToCid(MEthLogTrie, i[1].([]byte), multihash.KECCAK_256)
if err != nil {
return nil, err
}
return []interface{}{
i[0].([]byte),
&EthLog{
Log: l,
cid: c,
rawData: i[1].([]byte),
},
}, nil
}
/*
Block INTERFACE
*/
// RawData returns the binary of the RLP encode of the transaction.
func (t *EthLogTrie) RawData() []byte {
return t.rawdata
}
// Cid returns the cid of the transaction.
func (t *EthLogTrie) Cid() cid.Cid {
return t.cid
}
// String is a helper for output
func (t *EthLogTrie) String() string {
return fmt.Sprintf("<EthereumLogTrie %s>", t.cid)
}
// Loggable returns in a map the type of IPLD Link.
func (t *EthLogTrie) Loggable() map[string]interface{} {
return map[string]interface{}{
"type": "eth-log-trie",
}
}
// logTrie wraps a localTrie for use on the receipt trie.
type logTrie struct {
*localTrie
}
// newLogTrie initializes and returns a logTrie.
func newLogTrie() *logTrie {
return &logTrie{
localTrie: newLocalTrie(),
}
}
// getNodes invokes the localTrie, which computes the root hash of the
// log trie and returns its database keys, to return a slice
// of EthLogTrie nodes.
func (rt *logTrie) getNodes() ([]*EthLogTrie, error) {
keys, err := rt.getKeys()
if err != nil {
return nil, err
}
out := make([]*EthLogTrie, 0, len(keys))
for _, k := range keys {
n, err := rt.getNodeFromDB(k)
if err != nil {
return nil, err
}
out = append(out, n)
}
return out, nil
}
func (rt *logTrie) getNodeFromDB(key []byte) (*EthLogTrie, error) {
rawdata, err := rt.db.Get(key)
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthLogTrie, rawdata, multihash.KECCAK_256)
if err != nil {
return nil, err
}
tn := &TrieNode{
cid: c,
rawdata: rawdata,
}
return &EthLogTrie{TrieNode: tn}, nil
}
// getLeafNodes invokes the localTrie, which returns a slice
// of EthLogTrie leaf nodes.
func (rt *logTrie) getLeafNodes() ([]*EthLogTrie, []*nodeKey, error) {
keys, err := rt.getLeafKeys()
if err != nil {
return nil, nil, err
}
out := make([]*EthLogTrie, 0, len(keys))
for _, k := range keys {
n, err := rt.getNodeFromDB(k.dbKey)
if err != nil {
return nil, nil, err
}
out = append(out, n)
}
return out, keys, nil
}

View File

@ -23,10 +23,11 @@ import (
"io"
"io/ioutil"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash"
)
// FromBlockRLP takes an RLP message representing
@ -122,31 +123,35 @@ func FromBlockJSON(r io.Reader) (*EthHeader, []*EthTx, []*EthTxTrie, error) {
// FromBlockAndReceipts takes a block and processes it
// to return it a set of IPLD nodes for further processing.
func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) (*EthHeader, []*EthHeader, []*EthTx, []*EthTxTrie, []*EthReceipt, []*EthRctTrie, error) {
func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) (*EthHeader, []*EthHeader, []*EthTx, []*EthTxTrie, []*EthReceipt, []*EthRctTrie, [][]*EthLogTrie, [][]cid.Cid, error) {
// Process the header
headerNode, err := NewEthHeader(block.Header())
if err != nil {
return nil, nil, nil, nil, nil, nil, err
return nil, nil, nil, nil, nil, nil, nil, nil, err
}
// Process the uncles
uncleNodes := make([]*EthHeader, len(block.Uncles()))
for i, uncle := range block.Uncles() {
uncleNode, err := NewEthHeader(uncle)
if err != nil {
return nil, nil, nil, nil, nil, nil, err
return nil, nil, nil, nil, nil, nil, nil, nil, err
}
uncleNodes[i] = uncleNode
}
// Process the txs
ethTxNodes, ethTxTrieNodes, err := processTransactions(block.Transactions(),
txNodes, txTrieNodes, err := processTransactions(block.Transactions(),
block.Header().TxHash[:])
if err != nil {
return nil, nil, nil, nil, nil, nil, err
return nil, nil, nil, nil, nil, nil, nil, nil, err
}
// Process the receipts
ethRctNodes, ethRctTrieNodes, err := processReceipts(receipts,
// Process the receipts and logs
rctNodes, tctTrieNodes, logTrieNodes, logLeafNodeCIDs, err := processReceiptsAndLogs(receipts,
block.Header().ReceiptHash[:])
return headerNode, uncleNodes, ethTxNodes, ethTxTrieNodes, ethRctNodes, ethRctTrieNodes, err
return headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, tctTrieNodes, logTrieNodes, logLeafNodeCIDs, err
}
// processTransactions will take the found transactions in a parsed block body
@ -173,26 +178,77 @@ func processTransactions(txs []*types.Transaction, expectedTxRoot []byte) ([]*Et
return ethTxNodes, txTrieNodes, err
}
// processReceipts will take in receipts
// to return IPLD node slices for eth-rct and eth-rct-trie
func processReceipts(rcts []*types.Receipt, expectedRctRoot []byte) ([]*EthReceipt, []*EthRctTrie, error) {
var ethRctNodes []*EthReceipt
// processReceiptsAndLogs will take in receipts
// to return IPLD node slices for eth-rct, eth-rct-trie, eth-log, eth-log-trie
func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*EthReceipt, []*EthRctTrie, [][]*EthLogTrie, [][]cid.Cid, error) {
// Pre allocating memory.
ethRctNodes := make([]*EthReceipt, 0, len(rcts))
ethLogleafNodeCids := make([][]cid.Cid, 0, len(rcts))
ethLogTrieNodes := make([][]*EthLogTrie, 0, len(rcts))
receiptTrie := newRctTrie()
for idx, rct := range rcts {
// Process logs for each receipt.
logTrieNodes, leafNodeCids, logTrieHash, err := processLogs(rct.Logs)
if err != nil {
return nil, nil, nil, nil, err
}
rct.LogRoot = logTrieHash
ethLogTrieNodes = append(ethLogTrieNodes, logTrieNodes)
ethLogleafNodeCids = append(ethLogleafNodeCids, leafNodeCids)
ethRct, err := NewReceipt(rct)
if err != nil {
return nil, nil, err
return nil, nil, nil, nil, err
}
ethRctNodes = append(ethRctNodes, ethRct)
if err := receiptTrie.add(idx, ethRct.RawData()); err != nil {
return nil, nil, err
if err = receiptTrie.add(idx, ethRct.RawData()); err != nil {
return nil, nil, nil, nil, err
}
}
if !bytes.Equal(receiptTrie.rootHash(), expectedRctRoot) {
return nil, nil, fmt.Errorf("wrong receipt hash computed")
return nil, nil, nil, nil, fmt.Errorf("wrong receipt hash computed")
}
rctTrieNodes, err := receiptTrie.getNodes()
return ethRctNodes, rctTrieNodes, err
return ethRctNodes, rctTrieNodes, ethLogTrieNodes, ethLogleafNodeCids, err
}
func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, error) {
logTr := newLogTrie()
for idx, log := range logs {
ethLog, err := NewLog(log)
if err != nil {
return nil, nil, common.Hash{}, err
}
if err = logTr.add(idx, ethLog.RawData()); err != nil {
return nil, nil, common.Hash{}, err
}
}
logTrieNodes, err := logTr.getNodes()
if err != nil {
return nil, nil, common.Hash{}, err
}
leafNodes, keys, err := logTr.getLeafNodes()
if err != nil {
return nil, nil, common.Hash{}, err
}
leafNodeCids := make([]cid.Cid, len(leafNodes))
for i, ln := range leafNodes {
var idx uint
r := bytes.NewReader(keys[i].trieKey)
err = rlp.Decode(r, &idx)
if err != nil {
return nil, nil, common.Hash{}, err
}
leafNodeCids[idx] = ln.Cid()
}
return logTrieNodes, leafNodeCids, common.BytesToHash(logTr.rootHash()), err
}

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program 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 Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package ipld
import (
@ -7,6 +23,8 @@ import (
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/stretchr/testify/require"
)
type kind string
@ -73,9 +91,17 @@ func loadBlockData(t *testing.T) []testCase {
func TestFromBlockAndReceipts(t *testing.T) {
testCases := loadBlockData(t)
for _, tc := range testCases {
_, _, _, _, _, _, err := FromBlockAndReceipts(tc.block, tc.receipts)
_, _, _, _, _, _, _, _, err := FromBlockAndReceipts(tc.block, tc.receipts)
if err != nil {
t.Fatalf("error generating IPLDs from block and receipts, err %v, kind %s, block hash %s", err, tc.kind, tc.block.Hash())
}
}
}
func TestProcessLogs(t *testing.T) {
logs := []*types.Log{mocks.MockLog1, mocks.MockLog2}
nodes, cids, _, err := processLogs(logs)
require.NoError(t, err)
require.GreaterOrEqual(t, len(nodes), len(logs))
require.Equal(t, len(logs), len(cids))
}

View File

@ -80,12 +80,14 @@ func DecodeEthReceipt(c cid.Cid, b []byte) (*EthReceipt, error) {
Block INTERFACE
*/
func (node *EthReceipt) RawData() []byte {
return node.rawdata
// RawData returns the binary of the RLP encode of the receipt.
func (r *EthReceipt) RawData() []byte {
return r.rawdata
}
func (node *EthReceipt) Cid() cid.Cid {
return node.cid
// Cid returns the cid of the receipt.
func (r *EthReceipt) Cid() cid.Cid {
return r.cid
}
// String is a helper for output
@ -107,12 +109,14 @@ func (r *EthReceipt) Resolve(p []string) (interface{}, []string, error) {
return r, nil, nil
}
if len(p) > 1 {
return nil, nil, fmt.Errorf("unexpected path elements past %s", p[0])
first, rest := p[0], p[1:]
if first != "logs" && len(p) != 1 {
return nil, nil, fmt.Errorf("unexpected path elements past %s", first)
}
switch p[0] {
switch first {
case "logs":
return &node.Link{Cid: commonHashToCid(MEthLog, r.LogRoot)}, rest, nil
case "root":
return r.PostState, nil, nil
case "status":
@ -121,14 +125,14 @@ func (r *EthReceipt) Resolve(p []string) (interface{}, []string, error) {
return r.CumulativeGasUsed, nil, nil
case "logsBloom":
return r.Bloom, nil, nil
case "logs":
return r.Logs, nil, nil
case "transactionHash":
return r.TxHash, nil, nil
case "contractAddress":
return r.ContractAddress, nil, nil
case "gasUsed":
return r.GasUsed, nil, nil
case "type":
return r.Type, nil, nil
default:
return nil, nil, ErrInvalidLink
}
@ -140,7 +144,7 @@ func (r *EthReceipt) Tree(p string, depth int) []string {
if p != "" || depth == 0 {
return nil
}
return []string{"root", "status", "cumulativeGasUsed", "logsBloom", "logs", "transactionHash", "contractAddress", "gasUsed"}
return []string{"type", "root", "status", "cumulativeGasUsed", "logsBloom", "logs", "transactionHash", "contractAddress", "gasUsed"}
}
// ResolveLink is a helper function that calls resolve and asserts the
@ -159,13 +163,15 @@ func (r *EthReceipt) ResolveLink(p []string) (*node.Link, []string, error) {
}
// Copy will go away. It is here to comply with the Node interface.
func (*EthReceipt) Copy() node.Node {
func (r *EthReceipt) Copy() node.Node {
panic("implement me")
}
// Links is a helper function that returns all links within this object
func (*EthReceipt) Links() []*node.Link {
return nil
func (r *EthReceipt) Links() []*node.Link {
return []*node.Link{
{Cid: commonHashToCid(MEthLog, r.LogRoot)},
}
}
// Stat will go away. It is here to comply with the interface.

View File

@ -121,7 +121,8 @@ func (t *EthTx) Resolve(p []string) (interface{}, []string, error) {
}
switch p[0] {
case "type":
return t.Type(), nil, nil
case "gas":
return t.Gas(), nil, nil
case "gasPrice":
@ -154,7 +155,7 @@ func (t *EthTx) Tree(p string, depth int) []string {
if p != "" || depth == 0 {
return nil
}
return []string{"gas", "gasPrice", "input", "nonce", "r", "s", "toAddress", "v", "value"}
return []string{"type", "gas", "gasPrice", "input", "nonce", "r", "s", "toAddress", "v", "value"}
}
// ResolveLink is a helper function that calls resolve and asserts the

View File

@ -213,6 +213,7 @@ func TestEthTxTree(t *testing.T) {
// Good cases
tree = tx.Tree("", 1)
lookupElements := map[string]interface{}{
"type": nil,
"gas": nil,
"gasPrice": nil,
"input": nil,

View File

@ -20,14 +20,15 @@ import (
"bytes"
"errors"
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/rlp"
sdtrie "github.com/ethereum/go-ethereum/statediff/trie"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
)
// IPLD Codecs for Ethereum
@ -44,6 +45,8 @@ const (
MEthStateTrie = 0x96
MEthAccountSnapshot = 0x97
MEthStorageTrie = 0x98
MEthLogTrie = 0x99
MEthLog = 0x9a
)
var (
@ -150,6 +153,41 @@ func (lt *localTrie) getKeys() ([][]byte, error) {
return keyBytes, nil
}
type nodeKey struct {
dbKey []byte
trieKey []byte
}
// getLeafKeys returns the stored leaf keys from the memory database
// of the localTrie for further processing.
func (lt *localTrie) getLeafKeys() ([]*nodeKey, error) {
it := lt.trie.NodeIterator([]byte{})
leafKeys := make([]*nodeKey, 0)
for it.Next(true) {
if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) {
continue
}
node, nodeElements, err := sdtrie.ResolveNode(it, lt.trieDB)
if err != nil {
return nil, err
}
if node.NodeType != sdtypes.Leaf {
continue
}
partialPath := trie.CompactToHex(nodeElements[0].([]byte))
valueNodePath := append(node.Path, partialPath...)
encodedPath := trie.HexToCompact(valueNodePath)
leafKey := encodedPath[1:]
leafKeys = append(leafKeys, &nodeKey{dbKey: it.Hash().Bytes(), trieKey: leafKey})
}
return leafKeys, nil
}
// getRLP encodes the given object to RLP returning its bytes.
func getRLP(object interface{}) []byte {
buf := new(bytes.Buffer)

View File

@ -381,7 +381,7 @@ func createTransactionsAndReceipts(config *params.ChainConfig, blockNumber *big.
PostState: common.HexToHash("0x3").Bytes(),
Status: types.ReceiptStatusSuccessful,
CumulativeGasUsed: 175,
Logs: []*types.Log{},
Logs: []*types.Log{MockLog1, MockLog2},
TxHash: signedTrx4.Hash(),
}
mockReceipt5 := &types.Receipt{

View File

@ -75,19 +75,15 @@ type AccessListElementModel struct {
// ReceiptModel is the db model for eth.receipt_cids
type ReceiptModel struct {
ID int64 `db:"id"`
TxID int64 `db:"tx_id"`
CID string `db:"cid"`
MhKey string `db:"mh_key"`
PostStatus uint64 `db:"post_status"`
PostState string `db:"post_state"`
Contract string `db:"contract"`
ContractHash string `db:"contract_hash"`
LogContracts pq.StringArray `db:"log_contracts"`
Topic0s pq.StringArray `db:"topic0s"`
Topic1s pq.StringArray `db:"topic1s"`
Topic2s pq.StringArray `db:"topic2s"`
Topic3s pq.StringArray `db:"topic3s"`
ID int64 `db:"id"`
TxID int64 `db:"tx_id"`
CID string `db:"cid"`
MhKey string `db:"mh_key"`
PostStatus uint64 `db:"post_status"`
PostState string `db:"post_state"`
Contract string `db:"contract"`
ContractHash string `db:"contract_hash"`
LogRoot string `db:"log_root"`
}
// StateNodeModel is the db model for eth.state_cids
@ -136,3 +132,18 @@ type StateAccountModel struct {
CodeHash []byte `db:"code_hash"`
StorageRoot string `db:"storage_root"`
}
// LogsModel is the db model for eth.logs
type LogsModel struct {
ID int64 `db:"id"`
LeafCID string `db:"leaf_cid"`
LeafMhKey string `db:"leaf_mh_key"`
ReceiptID int64 `db:"receipt_id"`
Address string `db:"address"`
Index int64 `db:"index"`
Data []byte `db:"log_data"`
Topic0 string `db:"topic0"`
Topic1 string `db:"topic1"`
Topic2 string `db:"topic2"`
Topic3 string `db:"topic3"`
}

View File

@ -91,14 +91,29 @@ func (in *PostgresCIDWriter) upsertAccessListElement(tx *sqlx.Tx, accessListElem
return nil
}
func (in *PostgresCIDWriter) upsertReceiptCID(tx *sqlx.Tx, rct models.ReceiptModel, txID int64) error {
_, err := tx.Exec(`INSERT INTO eth.receipt_cids (tx_id, cid, contract, contract_hash, topic0s, topic1s, topic2s, topic3s, log_contracts, mh_key, post_state, post_status) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12)
ON CONFLICT (tx_id) DO UPDATE SET (cid, contract, contract_hash, topic0s, topic1s, topic2s, topic3s, log_contracts, mh_key, post_state, post_status) = ($2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12)`,
txID, rct.CID, rct.Contract, rct.ContractHash, rct.Topic0s, rct.Topic1s, rct.Topic2s, rct.Topic3s, rct.LogContracts, rct.MhKey, rct.PostState, rct.PostStatus)
func (in *PostgresCIDWriter) upsertReceiptCID(tx *sqlx.Tx, rct *models.ReceiptModel, txID int64) (int64, error) {
var receiptID int64
err := tx.QueryRowx(`INSERT INTO eth.receipt_cids (tx_id, cid, contract, contract_hash, mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8)
ON CONFLICT (tx_id) DO UPDATE SET (cid, contract, contract_hash, mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8)
RETURNING id`,
txID, rct.CID, rct.Contract, rct.ContractHash, rct.MhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID)
if err != nil {
return fmt.Errorf("error upserting receipt_cids entry: %v", err)
return 0, fmt.Errorf("error upserting receipt_cids entry: %w", err)
}
indexerMetrics.receipts.Inc(1)
return receiptID, nil
}
func (in *PostgresCIDWriter) upsertLogCID(tx *sqlx.Tx, logs []*models.LogsModel, receiptID int64) error {
for _, log := range logs {
_, err := tx.Exec(`INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, receipt_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)
ON CONFLICT (receipt_id, index) DO UPDATE SET (leaf_cid, leaf_mh_key ,address, topic0, topic1, topic2, topic3,log_data ) = ($1, $2, $4, $6, $7, $8, $9, $10)`,
log.LeafCID, log.LeafMhKey, receiptID, log.Address, log.Index, log.Topic0, log.Topic1, log.Topic2, log.Topic3, log.Data)
if err != nil {
return fmt.Errorf("error upserting logs entry: %w", err)
}
}
// TODO: Add metrics for logs.
return nil
}

54
statediff/trie/node.go Normal file
View File

@ -0,0 +1,54 @@
package trie
import (
"fmt"
"github.com/ethereum/go-ethereum/rlp"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
)
// CheckKeyType checks what type of key we have
func CheckKeyType(elements []interface{}) (sdtypes.NodeType, error) {
if len(elements) > 2 {
return sdtypes.Branch, nil
}
if len(elements) < 2 {
return sdtypes.Unknown, fmt.Errorf("node cannot be less than two elements in length")
}
switch elements[0].([]byte)[0] / 16 {
case '\x00':
return sdtypes.Extension, nil
case '\x01':
return sdtypes.Extension, nil
case '\x02':
return sdtypes.Leaf, nil
case '\x03':
return sdtypes.Leaf, nil
default:
return sdtypes.Unknown, fmt.Errorf("unknown hex prefix")
}
}
// ResolveNode return the state diff node pointed by the iterator.
func ResolveNode(it trie.NodeIterator, trieDB *trie.Database) (sdtypes.StateNode, []interface{}, error) {
nodePath := make([]byte, len(it.Path()))
copy(nodePath, it.Path())
node, err := trieDB.Node(it.Hash())
if err != nil {
return sdtypes.StateNode{}, nil, err
}
var nodeElements []interface{}
if err = rlp.DecodeBytes(node, &nodeElements); err != nil {
return sdtypes.StateNode{}, nil, err
}
ty, err := CheckKeyType(nodeElements)
if err != nil {
return sdtypes.StateNode{}, nil, err
}
return sdtypes.StateNode{
NodeType: ty,
Path: nodePath,
NodeValue: node,
}, nodeElements, nil
}