From dcaaa40067b078988d2f50456cea0397824fda5e Mon Sep 17 00:00:00 2001 From: i-norden Date: Mon, 8 Nov 2021 18:30:35 -0600 Subject: [PATCH 01/33] batch inserts to public.blocks --- statediff/indexer/batch_tx.go | 91 ++++++++++++++++++ statediff/indexer/indexer.go | 116 ++++++++++------------- statediff/indexer/indexer_legacy_test.go | 2 +- statediff/indexer/indexer_test.go | 2 +- statediff/indexer/models/batch.go | 100 +++++++++++++++++++ statediff/indexer/models/models.go | 6 ++ statediff/indexer/shared/functions.go | 63 +----------- statediff/indexer/writer.go | 4 +- statediff/service.go | 2 +- 9 files changed, 257 insertions(+), 129 deletions(-) create mode 100644 statediff/indexer/batch_tx.go create mode 100644 statediff/indexer/models/batch.go diff --git a/statediff/indexer/batch_tx.go b/statediff/indexer/batch_tx.go new file mode 100644 index 000000000..170d6ab51 --- /dev/null +++ b/statediff/indexer/batch_tx.go @@ -0,0 +1,91 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package indexer + +import ( + blockstore "github.com/ipfs/go-ipfs-blockstore" + dshelp "github.com/ipfs/go-ipfs-ds-help" + node "github.com/ipfs/go-ipld-format" + "github.com/jmoiron/sqlx" + "github.com/lib/pq" + + "github.com/ethereum/go-ethereum/statediff/indexer/ipfs/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/models" +) + +const ipldBatchInsertPgStr string = `INSERT INTO public.blocks (key, data) VALUES (unnest($1::TEXT[]), unnest($2::BYTEA[])) ON CONFLICT (key) DO NOTHING` + +// BlockTx wraps a Postgres tx with the state necessary for building the Postgres tx concurrently during trie difference iteration +type BlockTx struct { + dbtx *sqlx.Tx + BlockNumber uint64 + headerID int64 + Close func(blockTx *BlockTx, err error) error + + quit chan struct{} + iplds chan models.IPLDModel + ipldCache models.IPLDBatch +} + +func (tx *BlockTx) flush() error { + _, err := tx.dbtx.Exec(ipldBatchInsertPgStr, pq.Array(tx.ipldCache.Keys), pq.Array(tx.ipldCache.Values)) + if err != nil { + return err + } + tx.ipldCache = models.IPLDBatch{} + return nil +} + +// run in background goroutine to synchronize concurrent appends to the ipldCache +func (tx *BlockTx) cache() { + for { + select { + case i := <-tx.iplds: + tx.ipldCache.Keys = append(tx.ipldCache.Keys, i.Key) + tx.ipldCache.Values = append(tx.ipldCache.Values, i.Data) + case <-tx.quit: + return + } + } +} + +func (tx *BlockTx) cacheDirect(key string, value []byte) { + tx.iplds <- models.IPLDModel{ + Key: key, + Data: value, + } +} + +func (tx *BlockTx) cacheIPLD(i node.Node) { + tx.iplds <- models.IPLDModel{ + Key: blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(i.Cid().Hash()).String(), + Data: i.RawData(), + } +} + +func (tx *BlockTx) cacheRaw(codec, mh uint64, raw []byte) (string, string, error) { + c, err := ipld.RawdataToCid(codec, raw, mh) + if err != nil { + return "", "", err + } + prefixedKey := blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(c.Hash()).String() + tx.iplds <- models.IPLDModel{ + Key: prefixedKey, + Data: raw, + } + return c.String(), prefixedKey, err +} diff --git a/statediff/indexer/indexer.go b/statediff/indexer/indexer.go index bab5fa938..f8ffee429 100644 --- a/statediff/indexer/indexer.go +++ b/statediff/indexer/indexer.go @@ -25,7 +25,6 @@ import ( "github.com/ipfs/go-cid" node "github.com/ipfs/go-ipld-format" - "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/ethereum/go-ethereum/common" @@ -65,7 +64,6 @@ type Indexer interface { type StateDiffIndexer struct { chainConfig *params.ChainConfig dbWriter *PostgresCIDWriter - init bool } // NewStateDiffIndexer creates a pointer to a new PayloadConverter which satisfies the PayloadConverter interface @@ -80,13 +78,6 @@ func NewStateDiffIndexer(chainConfig *params.ChainConfig, db *postgres.DB) (*Sta }, nil } -type BlockTx struct { - dbtx *sqlx.Tx - BlockNumber uint64 - headerID int64 - Close func(err error) error -} - // ReportDBMetrics is a reporting function to run as goroutine func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bool) { if !metrics.Enabled { @@ -127,7 +118,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } if len(txNodes) != len(rctNodes) || len(rctNodes) != len(rctLeafNodeCIDs) { - return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d), and receipt trie leaf nodes (%d)to be equal", len(txNodes), len(rctNodes), len(rctLeafNodeCIDs)) + return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d), and receipt trie leaf nodes (%d) to be equal", len(txNodes), len(rctNodes), len(rctLeafNodeCIDs)) + } + if len(txTrieNodes) != len(rctTrieNodes) { + return nil, fmt.Errorf("expected number of tx trie (%d) and rct trie (%d) nodes to be equal", len(txTrieNodes), len(rctTrieNodes)) } // Calculate reward @@ -139,6 +133,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip reward = CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) } t = time.Now() + // Begin new db tx for everything tx, err := sdi.dbWriter.db.Beginx() if err != nil { @@ -153,9 +148,14 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } }() blockTx := &BlockTx{ - dbtx: tx, + iplds: make(chan models.IPLDModel), + quit: make(chan struct{}), + ipldCache: models.IPLDBatch{}, + dbtx: tx, // handle transaction commit or rollback for any return case - Close: func(err error) error { + Close: func(self *BlockTx, err error) error { + close(self.quit) + close(self.iplds) if p := recover(); p != nil { shared.Rollback(tx) panic(p) @@ -166,6 +166,12 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) t = time.Now() + if err := self.flush(); err != nil { + shared.Rollback(tx) + traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) + log.Debug(traceMsg) + return err + } err = tx.Commit() tDiff = time.Since(t) indexerMetrics.tPostgresCommit.Update(tDiff) @@ -176,6 +182,8 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip return err }, } + go blockTx.cache() + tDiff := time.Since(t) indexerMetrics.tFreePostgres.Update(tDiff) @@ -184,7 +192,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // Publish and index header, collect headerID var headerID int64 - headerID, err = sdi.processHeader(tx, block.Header(), headerNode, reward, totalDifficulty) + headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) if err != nil { return nil, err } @@ -193,7 +201,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) t = time.Now() // Publish and index uncles - err = sdi.processUncles(tx, headerID, height, uncleNodes) + err = sdi.processUncles(blockTx, headerID, height, uncleNodes) if err != nil { return nil, err } @@ -202,7 +210,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String()) t = time.Now() // Publish and index receipts and txs - err = sdi.processReceiptsAndTxs(tx, processArgs{ + err = sdi.processReceiptsAndTxs(blockTx, processArgs{ headerID: headerID, blockNumber: block.Number(), receipts: receipts, @@ -230,11 +238,8 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // processHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *sqlx.Tx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { - // publish header - if err := shared.PublishIPLD(tx, headerNode); err != nil { - return 0, fmt.Errorf("error publishing header IPLD: %v", err) - } +func (sdi *StateDiffIndexer) processHeader(tx *BlockTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { + tx.cacheIPLD(headerNode) var baseFee *int64 if header.BaseFee != nil { @@ -243,7 +248,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *sqlx.Tx, header *types.Header, he } // index header - return sdi.dbWriter.upsertHeaderCID(tx, models.HeaderModel{ + return sdi.dbWriter.upsertHeaderCID(tx.dbtx, models.HeaderModel{ CID: headerNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), ParentHash: header.ParentHash.String(), @@ -262,12 +267,10 @@ func (sdi *StateDiffIndexer) processHeader(tx *sqlx.Tx, header *types.Header, he } // processUncles publishes and indexes uncle IPLDs in Postgres -func (sdi *StateDiffIndexer) processUncles(tx *sqlx.Tx, headerID int64, blockNumber uint64, uncleNodes []*ipld.EthHeader) error { +func (sdi *StateDiffIndexer) processUncles(tx *BlockTx, headerID int64, blockNumber uint64, uncleNodes []*ipld.EthHeader) error { // publish and index uncles for _, uncleNode := range uncleNodes { - if err := shared.PublishIPLD(tx, uncleNode); err != nil { - return fmt.Errorf("error publishing uncle IPLD: %v", err) - } + tx.cacheIPLD(uncleNode) var uncleReward *big.Int // in PoA networks uncle reward is 0 if sdi.chainConfig.Clique != nil { @@ -282,7 +285,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *sqlx.Tx, headerID int64, blockNum BlockHash: uncleNode.Hash().String(), Reward: uncleReward.String(), } - if err := sdi.dbWriter.upsertUncleCID(tx, uncle, headerID); err != nil { + if err := sdi.dbWriter.upsertUncleCID(tx.dbtx, uncle, headerID); err != nil { return err } } @@ -305,28 +308,15 @@ type processArgs struct { } // processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres -func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs) error { +func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BlockTx, args processArgs) error { // Process receipts and txs signer := types.MakeSigner(sdi.chainConfig, args.blockNumber) for i, receipt := range args.receipts { - // tx that corresponds with this receipt - trx := args.txs[i] - from, err := types.Sender(signer, trx) - if err != nil { - return fmt.Errorf("error deriving tx sender: %v", err) + for _, logTrieNode := range args.logTrieNodes[i] { + tx.cacheIPLD(logTrieNode) } - - 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 := args.txNodes[i] - if err := shared.PublishIPLD(tx, txNode); err != nil { - return fmt.Errorf("error publishing tx IPLD: %v", err) - } + tx.cacheIPLD(txNode) // Indexing // extract topic and contract data from the receipt for indexing @@ -344,7 +334,6 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs mappedContracts[l.Address.String()] = true logDataSet[idx] = &models.LogsModel{ - ID: 0, Address: l.Address.String(), Index: int64(l.Index), Data: l.Data, @@ -368,6 +357,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() } // index tx first so that the receipt can reference it by FK + trx := args.txs[i] + // derive sender for the tx that corresponds with this receipt + from, err := types.Sender(signer, trx) + if err != nil { + return fmt.Errorf("error deriving tx sender: %v", err) + } txModel := models.TxModel{ Dst: shared.HandleZeroAddrPointer(trx.To()), Src: shared.HandleZeroAddr(from), @@ -381,7 +376,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs if txType != types.LegacyTxType { txModel.Type = &txType } - txID, err := sdi.dbWriter.upsertTransactionCID(tx, txModel, args.headerID) + txID, err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel, args.headerID) if err != nil { return err } @@ -397,7 +392,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs Address: accessListElement.Address.Hex(), StorageKeys: storageKeys, } - if err := sdi.dbWriter.upsertAccessListElement(tx, accessListElementModel, txID); err != nil { + if err := sdi.dbWriter.upsertAccessListElement(tx.dbtx, accessListElementModel, txID); err != nil { return err } } @@ -420,27 +415,20 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *sqlx.Tx, args processArgs rctModel.PostState = common.Bytes2Hex(receipt.PostState) } - receiptID, err := sdi.dbWriter.upsertReceiptCID(tx, rctModel, txID) + receiptID, err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel, txID) if err != nil { return err } - if err = sdi.dbWriter.upsertLogCID(tx, logDataSet, receiptID); err != nil { + if err = sdi.dbWriter.upsertLogCID(tx.dbtx, 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) - } + for i, n := range args.txTrieNodes { + tx.cacheIPLD(n) + tx.cacheIPLD(args.rctTrieNodes[i]) } return nil @@ -462,9 +450,9 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN _, err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel, tx.headerID) return err } - stateCIDStr, stateMhKey, err := shared.PublishRaw(tx.dbtx, ipld.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) + stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) if err != nil { - return fmt.Errorf("error publishing state node IPLD: %v", err) + return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) } stateModel := models.StateNodeModel{ Path: stateNode.Path, @@ -518,9 +506,9 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN } continue } - storageCIDStr, storageMhKey, err := shared.PublishRaw(tx.dbtx, ipld.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) + storageCIDStr, storageMhKey, err := tx.cacheRaw(ipld.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) if err != nil { - return fmt.Errorf("error publishing storage node IPLD: %v", err) + return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) } storageModel := models.StorageNodeModel{ Path: storageNode.Path, @@ -544,8 +532,6 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(tx *BlockTx, codeAndCodeHash sd if err != nil { return fmt.Errorf("error deriving multihash key from codehash: %v", err) } - if err := shared.PublishDirect(tx.dbtx, mhKey, codeAndCodeHash.Code); err != nil { - return fmt.Errorf("error publishing code IPLD: %v", err) - } + tx.cacheDirect(mhKey, codeAndCodeHash.Code) return nil } diff --git a/statediff/indexer/indexer_legacy_test.go b/statediff/indexer/indexer_legacy_test.go index 4b1563190..7c01f567f 100644 --- a/statediff/indexer/indexer_legacy_test.go +++ b/statediff/indexer/indexer_legacy_test.go @@ -51,7 +51,7 @@ func setupLegacy(t *testing.T) { legacyData.MockBlock.Difficulty()) require.NoError(t, err) - defer tx.Close(err) + defer tx.Close(tx, err) for _, node := range legacyData.StateDiffs { err = ind.PushStateNode(tx, node) require.NoError(t, err) diff --git a/statediff/indexer/indexer_test.go b/statediff/indexer/indexer_test.go index 67645a12d..d1962dad2 100644 --- a/statediff/indexer/indexer_test.go +++ b/statediff/indexer/indexer_test.go @@ -149,7 +149,7 @@ func setup(t *testing.T) { if err != nil { t.Fatal(err) } - defer tx.Close(err) + defer tx.Close(tx, err) for _, node := range mocks.StateDiffs { err = ind.PushStateNode(tx, node) if err != nil { diff --git a/statediff/indexer/models/batch.go b/statediff/indexer/models/batch.go new file mode 100644 index 000000000..f780a9b0d --- /dev/null +++ b/statediff/indexer/models/batch.go @@ -0,0 +1,100 @@ +package models + +import "github.com/lib/pq" + +// IPLDBatch holds the arguments for a batch insert of IPLD data +type IPLDBatch struct { + Keys []string + Values [][]byte +} + +// UncleBatch holds the arguments for a batch insert of uncle data +type UncleBatch struct { + HeaderID []int64 + BlockHashes []string + ParentHashes []string + CIDs []string + MhKeys []string + Rewards []string +} + +// TxBatch holds the arguments for a batch insert of tx data +type TxBatch struct { + HeaderID int64 + Indexes []int64 + TxHashes []string + CIDs []string + MhKeys []string + Dsts []string + Srcs []string + Datas [][]byte + Types []*uint8 +} + +// AccessListBatch holds the arguments for a batch insert of access list data +type AccessListBatch struct { + Indexes []int64 + TxIDs []int64 + Addresses []string + StorageKeysSets []pq.StringArray +} + +// ReceiptBatch holds the arguments for a batch insert of receipt data +type ReceiptBatch struct { + TxIDs []int64 + LeafCIDs []string + LeafMhKeys []string + PostStatuses []uint64 + PostStates []string + Contracts []string + ContractHashes []string + LogRoots []string +} + +// LogBatch holds the arguments for a batch insert of log data +type LogBatch struct { + LeafCIDs []string + LeafMhKeys []string + ReceiptIDs []int64 + Addresses []string + Indexes []int64 + Datas [][]byte + Topic0s []string + Topic1s []string + Topic2s []string + Topic3s []string +} + +// StateBatch holds the arguments for a batch insert of state data +type StateBatch struct { + ID int64 + HeaderID int64 + Path []byte + StateKey string + NodeType int + CID string + MhKey string + Diff bool +} + +// AccountBatch holds the arguments for a batch insert of account data +type AccountBatch struct { + ID int64 + StateID int64 + Balance string + Nonce uint64 + CodeHash []byte + StorageRoot string +} + +// StorageBatch holds the arguments for a batch insert of storage data +type StorageBatch struct { + ID int64 + StateID int64 + Path []byte + StorageKey string + NodeType int + CID string + MhKey string + Diff bool +} diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index cb9404385..5837488f8 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -18,6 +18,12 @@ package models import "github.com/lib/pq" +// IPLDModel is the db model for public.blocks +type IPLDModel struct { + Key string + Data []byte +} + // HeaderModel is the db model for eth.header_cids type HeaderModel struct { ID int64 `db:"id"` diff --git a/statediff/indexer/shared/functions.go b/statediff/indexer/shared/functions.go index cb2ca6cea..7823d8f78 100644 --- a/statediff/indexer/shared/functions.go +++ b/statediff/indexer/shared/functions.go @@ -19,17 +19,18 @@ package shared import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/statediff/indexer/ipfs/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/postgres" "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" - format "github.com/ipfs/go-ipld-format" "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" ) +// IPLDInsertPgStr is the postgres statement string for IPLDs inserting into public.blocks +const IPLDInsertPgStr = `INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING` + // HandleZeroAddrPointer will return an empty string for a nil address pointer func HandleZeroAddrPointer(to *common.Address) string { if to == nil { @@ -53,62 +54,12 @@ func Rollback(tx *sqlx.Tx) { } } -// PublishIPLD is used to insert an IPLD into Postgres blockstore with the provided tx -func PublishIPLD(tx *sqlx.Tx, i format.Node) error { - dbKey := dshelp.MultihashToDsKey(i.Cid().Hash()) - prefixedKey := blockstore.BlockPrefix.String() + dbKey.String() - raw := i.RawData() - _, err := tx.Exec(`INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING`, prefixedKey, raw) - return err -} - -// FetchIPLD is used to retrieve an ipld from Postgres blockstore with the provided tx and cid string -func FetchIPLD(tx *sqlx.Tx, cid string) ([]byte, error) { - mhKey, err := MultihashKeyFromCIDString(cid) - if err != nil { - return nil, err - } - pgStr := `SELECT data FROM public.blocks WHERE key = $1` - var block []byte - return block, tx.Get(&block, pgStr, mhKey) -} - -// FetchIPLDByMhKey is used to retrieve an ipld from Postgres blockstore with the provided tx and mhkey string -func FetchIPLDByMhKey(tx *sqlx.Tx, mhKey string) ([]byte, error) { - pgStr := `SELECT data FROM public.blocks WHERE key = $1` - var block []byte - return block, tx.Get(&block, pgStr, mhKey) -} - // MultihashKeyFromCID converts a cid into a blockstore-prefixed multihash db key string func MultihashKeyFromCID(c cid.Cid) string { dbKey := dshelp.MultihashToDsKey(c.Hash()) return blockstore.BlockPrefix.String() + dbKey.String() } -// MultihashKeyFromCIDString converts a cid string into a blockstore-prefixed multihash db key string -func MultihashKeyFromCIDString(c string) (string, error) { - dc, err := cid.Decode(c) - if err != nil { - return "", err - } - dbKey := dshelp.MultihashToDsKey(dc.Hash()) - return blockstore.BlockPrefix.String() + dbKey.String(), nil -} - -// PublishRaw derives a cid from raw bytes and provided codec and multihash type, and writes it to the db tx -// returns the CID and blockstore prefixed multihash key -func PublishRaw(tx *sqlx.Tx, codec, mh uint64, raw []byte) (string, string, error) { - c, err := ipld.RawdataToCid(codec, raw, mh) - if err != nil { - return "", "", err - } - dbKey := dshelp.MultihashToDsKey(c.Hash()) - prefixedKey := blockstore.BlockPrefix.String() + dbKey.String() - _, err = tx.Exec(`INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING`, prefixedKey, raw) - return c.String(), prefixedKey, err -} - // MultihashKeyFromKeccak256 converts keccak256 hash bytes into a blockstore-prefixed multihash db key string func MultihashKeyFromKeccak256(hash common.Hash) (string, error) { mh, err := multihash.Encode(hash.Bytes(), multihash.KECCAK_256) @@ -119,14 +70,8 @@ func MultihashKeyFromKeccak256(hash common.Hash) (string, error) { return blockstore.BlockPrefix.String() + dbKey.String(), nil } -// PublishDirect diretly writes a previously derived mhkey => value pair to the ipld database in the provided tx -func PublishDirect(tx *sqlx.Tx, key string, value []byte) error { - _, err := tx.Exec(`INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING`, key, value) - return err -} - // PublishDirectWithDB diretly writes a previously derived mhkey => value pair to the ipld database func PublishDirectWithDB(db *postgres.DB, key string, value []byte) error { - _, err := db.Exec(`INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING`, key, value) + _, err := db.Exec(IPLDInsertPgStr, key, value) return err } diff --git a/statediff/indexer/writer.go b/statediff/indexer/writer.go index 62b36ca58..cbc058896 100644 --- a/statediff/indexer/writer.go +++ b/statediff/indexer/writer.go @@ -35,7 +35,7 @@ type PostgresCIDWriter struct { db *postgres.DB } -// NewPostgresCIDWriter creates a new pointer to a Indexer which satisfies the PostgresCIDWriter interface +// NewPostgresCIDWriter creates a new pointer to a PostgresCIDWriter func NewPostgresCIDWriter(db *postgres.DB) *PostgresCIDWriter { return &PostgresCIDWriter{ db: db, @@ -107,7 +107,7 @@ func (in *PostgresCIDWriter) upsertReceiptCID(tx *sqlx.Tx, rct *models.ReceiptMo 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)`, + 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) diff --git a/statediff/service.go b/statediff/service.go index 1154e4750..de6e84a65 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -672,7 +672,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p return err } // defer handling of commit/rollback for any return case - defer tx.Close(err) + defer tx.Close(tx, err) output := func(node StateNode) error { return sds.indexer.PushStateNode(tx, node) } -- 2.45.2 From b3add308eeb385871d82dd04bb537781b62d9f3d Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 10 Nov 2021 17:02:26 -0600 Subject: [PATCH 02/33] major refactor --- cmd/geth/config.go | 44 +- cmd/geth/main.go | 12 +- cmd/geth/usage.go | 12 +- cmd/utils/flags.go | 53 +- go.mod | 8 +- go.sum | 157 +++- statediff/README.md | 20 +- statediff/api.go | 6 +- statediff/builder.go | 182 ++--- statediff/builder_test.go | 699 +++++++++--------- statediff/config.go | 58 ++ statediff/indexer/constructor.go | 66 ++ .../indexer/{ => database/dump}/batch_tx.go | 46 +- statediff/indexer/database/dump/config.go | 31 + statediff/indexer/database/dump/indexer.go | 490 ++++++++++++ statediff/indexer/database/dump/metrics.go | 94 +++ statediff/indexer/database/sql/batch_tx.go | 106 +++ .../indexer/database/sql/batch_writer.go | 216 ++++++ .../indexer/{ => database/sql}/indexer.go | 120 +-- .../{ => database/sql}/indexer_legacy_test.go | 30 +- .../{ => database/sql}/indexer_test.go | 213 +++--- statediff/indexer/database/sql/interfaces.go | 88 +++ .../indexer/{ => database/sql}/metrics.go | 39 +- .../indexer/database/sql/postgres/config.go | 81 ++ .../indexer/database/sql/postgres/database.go | 112 +++ .../{ => database/sql}/postgres/errors.go | 0 .../indexer/database/sql/postgres/pgx.go | 255 +++++++ .../indexer/database/sql/postgres/pgx_test.go | 122 +++ .../sql}/postgres/postgres_suite_test.go | 0 .../indexer/database/sql/postgres/sqlx.go | 197 +++++ .../sql/postgres/sqlx_test.go} | 48 +- .../database/sql/postgres/test_helpers.go | 42 ++ .../{ => database/sql}/test_helpers.go | 24 +- statediff/indexer/database/sql/writer.go | 141 ++++ statediff/indexer/helpers.go | 45 -- statediff/indexer/interfaces/interfaces.go | 46 ++ .../ipld/eip2930_test_data/eth-block-12252078 | Bin .../ipld/eip2930_test_data/eth-block-12365585 | Bin .../ipld/eip2930_test_data/eth-block-12365586 | Bin .../eip2930_test_data/eth-receipts-12252078 | Bin .../eip2930_test_data/eth-receipts-12365585 | Bin .../eip2930_test_data/eth-receipts-12365586 | Bin .../indexer/{ipfs => }/ipld/eth_account.go | 0 .../{ipfs => }/ipld/eth_account_test.go | 0 .../indexer/{ipfs => }/ipld/eth_header.go | 0 .../{ipfs => }/ipld/eth_header_test.go | 0 statediff/indexer/{ipfs => }/ipld/eth_log.go | 0 .../indexer/{ipfs => }/ipld/eth_log_trie.go | 2 +- .../indexer/{ipfs => }/ipld/eth_parser.go | 0 .../{ipfs => }/ipld/eth_parser_test.go | 0 .../indexer/{ipfs => }/ipld/eth_receipt.go | 0 .../{ipfs => }/ipld/eth_receipt_trie.go | 2 +- .../indexer/{ipfs => }/ipld/eth_state.go | 0 .../indexer/{ipfs => }/ipld/eth_state_test.go | 0 .../indexer/{ipfs => }/ipld/eth_storage.go | 0 .../{ipfs => }/ipld/eth_storage_test.go | 0 statediff/indexer/{ipfs => }/ipld/eth_tx.go | 0 .../indexer/{ipfs => }/ipld/eth_tx_test.go | 0 .../indexer/{ipfs => }/ipld/eth_tx_trie.go | 2 +- .../{ipfs => }/ipld/eth_tx_trie_test.go | 0 statediff/indexer/{ipfs => }/ipld/shared.go | 6 +- .../error-tx-eth-block-body-json-999999 | 0 .../ipld/test_data/eth-block-body-json-0 | 0 .../test_data/eth-block-body-json-4139497 | 0 .../ipld/test_data/eth-block-body-json-997522 | 0 .../ipld/test_data/eth-block-body-json-999998 | 0 .../ipld/test_data/eth-block-body-json-999999 | 0 .../ipld/test_data/eth-block-body-rlp-997522 | Bin .../ipld/test_data/eth-block-body-rlp-999999 | Bin .../test_data/eth-block-header-rlp-999996 | Bin .../test_data/eth-block-header-rlp-999997 | Bin .../test_data/eth-block-header-rlp-999999 | Bin .../ipld/test_data/eth-state-trie-rlp-0e8b34 | Bin .../ipld/test_data/eth-state-trie-rlp-56864f | 0 .../ipld/test_data/eth-state-trie-rlp-6fc2d7 | 0 .../ipld/test_data/eth-state-trie-rlp-727994 | Bin .../ipld/test_data/eth-state-trie-rlp-c9070d | Bin .../ipld/test_data/eth-state-trie-rlp-d5be90 | Bin .../ipld/test_data/eth-state-trie-rlp-d7f897 | Bin .../ipld/test_data/eth-state-trie-rlp-eb2f5f | Bin .../test_data/eth-storage-trie-rlp-000dd0 | Bin .../test_data/eth-storage-trie-rlp-113049 | 0 .../test_data/eth-storage-trie-rlp-9d1860 | 0 .../test_data/eth-storage-trie-rlp-ffbcad | Bin .../test_data/eth-storage-trie-rlp-ffc25c | 0 .../ipld/test_data/eth-uncle-json-997522-0 | 0 .../ipld/test_data/eth-uncle-json-997522-1 | 0 .../indexer/{ipfs => }/ipld/test_data/tx_data | Bin .../indexer/{ipfs => }/ipld/trie_node.go | 0 statediff/indexer/mocks/test_data.go | 8 +- statediff/indexer/models/batch.go | 16 + statediff/indexer/models/models.go | 4 +- statediff/indexer/postgres/config.go | 59 -- statediff/indexer/postgres/postgres.go | 76 -- statediff/indexer/shared/chain_type.go | 78 -- statediff/indexer/shared/constants.go | 7 +- statediff/indexer/shared/data_type.go | 102 --- .../{ipfs/models.go => shared/db_kind.go} | 14 +- statediff/indexer/shared/functions.go | 20 - statediff/indexer/{ => shared}/reward.go | 2 +- statediff/indexer/shared/types.go | 44 -- .../{shared => test_helpers}/test_helpers.go | 33 +- statediff/indexer/writer.go | 158 ---- statediff/mainnet_tests/builder_test.go | 34 +- statediff/metrics.go | 16 + statediff/payload.go | 57 ++ statediff/service.go | 91 +-- statediff/service_test.go | 6 +- statediff/test_helpers/constant.go | 33 + .../{testhelpers => test_helpers}/helpers.go | 2 +- .../mocks/blockchain.go | 0 .../mocks/builder.go | 14 +- .../mocks/service.go | 0 .../mocks/service_test.go | 24 +- .../test_data.go | 2 +- statediff/testhelpers/constant.go | 17 - statediff/trie/node.go | 54 -- statediff/{ => trie_helpers}/helpers.go | 59 +- statediff/types.go | 113 --- statediff/types/types.go | 35 +- 120 files changed, 3448 insertions(+), 1645 deletions(-) create mode 100644 statediff/config.go create mode 100644 statediff/indexer/constructor.go rename statediff/indexer/{ => database/dump}/batch_tx.go (67%) create mode 100644 statediff/indexer/database/dump/config.go create mode 100644 statediff/indexer/database/dump/indexer.go create mode 100644 statediff/indexer/database/dump/metrics.go create mode 100644 statediff/indexer/database/sql/batch_tx.go create mode 100644 statediff/indexer/database/sql/batch_writer.go rename statediff/indexer/{ => database/sql}/indexer.go (83%) rename statediff/indexer/{ => database/sql}/indexer_legacy_test.go (73%) rename statediff/indexer/{ => database/sql}/indexer_test.go (72%) create mode 100644 statediff/indexer/database/sql/interfaces.go rename statediff/indexer/{ => database/sql}/metrics.go (79%) create mode 100644 statediff/indexer/database/sql/postgres/config.go create mode 100644 statediff/indexer/database/sql/postgres/database.go rename statediff/indexer/{ => database/sql}/postgres/errors.go (100%) create mode 100644 statediff/indexer/database/sql/postgres/pgx.go create mode 100644 statediff/indexer/database/sql/postgres/pgx_test.go rename statediff/indexer/{ => database/sql}/postgres/postgres_suite_test.go (100%) create mode 100644 statediff/indexer/database/sql/postgres/sqlx.go rename statediff/indexer/{postgres/postgres_test.go => database/sql/postgres/sqlx_test.go} (66%) create mode 100644 statediff/indexer/database/sql/postgres/test_helpers.go rename statediff/indexer/{ => database/sql}/test_helpers.go (69%) create mode 100644 statediff/indexer/database/sql/writer.go delete mode 100644 statediff/indexer/helpers.go create mode 100644 statediff/indexer/interfaces/interfaces.go rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-block-12252078 (100%) rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-block-12365585 (100%) rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-block-12365586 (100%) rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-receipts-12252078 (100%) rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-receipts-12365585 (100%) rename statediff/indexer/{ipfs => }/ipld/eip2930_test_data/eth-receipts-12365586 (100%) rename statediff/indexer/{ipfs => }/ipld/eth_account.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_account_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_header.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_header_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_log.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_log_trie.go (98%) rename statediff/indexer/{ipfs => }/ipld/eth_parser.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_parser_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_receipt.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_receipt_trie.go (98%) rename statediff/indexer/{ipfs => }/ipld/eth_state.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_state_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_storage.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_storage_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_tx.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_tx_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/eth_tx_trie.go (98%) rename statediff/indexer/{ipfs => }/ipld/eth_tx_trie_test.go (100%) rename statediff/indexer/{ipfs => }/ipld/shared.go (96%) rename statediff/indexer/{ipfs => }/ipld/test_data/error-tx-eth-block-body-json-999999 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-json-0 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-json-4139497 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-json-997522 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-json-999998 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-json-999999 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-rlp-997522 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-body-rlp-999999 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-header-rlp-999996 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-header-rlp-999997 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-block-header-rlp-999999 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-0e8b34 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-56864f (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-6fc2d7 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-727994 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-c9070d (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-d5be90 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-d7f897 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-state-trie-rlp-eb2f5f (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-storage-trie-rlp-000dd0 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-storage-trie-rlp-113049 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-storage-trie-rlp-9d1860 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-storage-trie-rlp-ffbcad (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-storage-trie-rlp-ffc25c (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-uncle-json-997522-0 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/eth-uncle-json-997522-1 (100%) rename statediff/indexer/{ipfs => }/ipld/test_data/tx_data (100%) rename statediff/indexer/{ipfs => }/ipld/trie_node.go (100%) delete mode 100644 statediff/indexer/postgres/config.go delete mode 100644 statediff/indexer/postgres/postgres.go delete mode 100644 statediff/indexer/shared/chain_type.go delete mode 100644 statediff/indexer/shared/data_type.go rename statediff/indexer/{ipfs/models.go => shared/db_kind.go} (83%) rename statediff/indexer/{ => shared}/reward.go (99%) delete mode 100644 statediff/indexer/shared/types.go rename statediff/indexer/{shared => test_helpers}/test_helpers.go (59%) delete mode 100644 statediff/indexer/writer.go create mode 100644 statediff/payload.go create mode 100644 statediff/test_helpers/constant.go rename statediff/{testhelpers => test_helpers}/helpers.go (99%) rename statediff/{testhelpers => test_helpers}/mocks/blockchain.go (100%) rename statediff/{testhelpers => test_helpers}/mocks/builder.go (80%) rename statediff/{testhelpers => test_helpers}/mocks/service.go (100%) rename statediff/{testhelpers => test_helpers}/mocks/service_test.go (93%) rename statediff/{testhelpers => test_helpers}/test_data.go (99%) delete mode 100644 statediff/testhelpers/constant.go delete mode 100644 statediff/trie/node.go rename statediff/{ => trie_helpers}/helpers.go (53%) delete mode 100644 statediff/types.go diff --git a/cmd/geth/config.go b/cmd/geth/config.go index 1a8ee4025..cfc43d2ab 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -23,8 +23,11 @@ import ( "math/big" "os" "reflect" + "time" "unicode" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/eth/downloader" "github.com/ethereum/go-ethereum/statediff" "gopkg.in/urfave/cli.v1" @@ -182,27 +185,48 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) { } if ctx.GlobalBool(utils.StateDiffFlag.Name) { - var dbParams *statediff.DBParams - if ctx.GlobalIsSet(utils.StateDiffDBFlag.Name) { - dbParams = new(statediff.DBParams) - dbParams.ConnectionURL = ctx.GlobalString(utils.StateDiffDBFlag.Name) + var dbConfig *sql.Config + if ctx.GlobalIsSet(utils.StateDiffWritingFlag.Name) { + dbConfig = new(sql.Config) + dbConfig.Hostname = ctx.GlobalString(utils.StateDiffDBHostFlag.Name) + dbConfig.Port = ctx.GlobalInt(utils.StateDiffDBPortFlag.Name) + dbConfig.DatabaseName = ctx.GlobalString(utils.StateDiffDBNameFlag.Name) + dbConfig.Username = ctx.GlobalString(utils.StateDiffDBUserFlag.Name) + dbConfig.Password = ctx.GlobalString(utils.StateDiffDBPasswordFlag.Name) + if ctx.GlobalIsSet(utils.StateDiffDBNodeIDFlag.Name) { - dbParams.ID = ctx.GlobalString(utils.StateDiffDBNodeIDFlag.Name) + dbConfig.ID = ctx.GlobalString(utils.StateDiffDBNodeIDFlag.Name) } else { utils.Fatalf("Must specify node ID for statediff DB output") } + if ctx.GlobalIsSet(utils.StateDiffDBClientNameFlag.Name) { - dbParams.ClientName = ctx.GlobalString(utils.StateDiffDBClientNameFlag.Name) + dbConfig.ClientName = ctx.GlobalString(utils.StateDiffDBClientNameFlag.Name) } else { utils.Fatalf("Must specify client name for statediff DB output") } - } else { - if ctx.GlobalBool(utils.StateDiffWritingFlag.Name) { - utils.Fatalf("Must pass DB parameters if enabling statediff write loop") + + if ctx.GlobalIsSet(utils.StateDiffDBMinConns.Name) { + dbConfig.MinConns = ctx.GlobalInt(utils.StateDiffDBMinConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConns.Name) { + dbConfig.MaxConns = ctx.GlobalInt(utils.StateDiffDBMaxConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxIdleConns.Name) { + dbConfig.MaxIdle = ctx.GlobalInt(utils.StateDiffDBMaxIdleConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConnLifetime.Name) { + dbConfig.MaxConnLifetime = ctx.GlobalDuration(utils.StateDiffDBMaxConnLifetime.Name) * time.Second + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConnIdleTime.Name) { + dbConfig.MaxConnIdleTime = ctx.GlobalDuration(utils.StateDiffDBMaxConnIdleTime.Name) * time.Second + } + if ctx.GlobalIsSet(utils.StateDiffDBConnTimeout.Name) { + dbConfig.ConnTimeout = ctx.GlobalDuration(utils.StateDiffDBConnTimeout.Name) * time.Second } } p := statediff.ServiceParams{ - DBParams: dbParams, + DBParams: dbConfig, EnableWriteLoop: ctx.GlobalBool(utils.StateDiffWritingFlag.Name), NumWorkers: ctx.GlobalUint(utils.StateDiffWorkersFlag.Name), } diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 8432b8005..9c8dbdcfd 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -149,7 +149,17 @@ var ( utils.GpoIgnoreGasPriceFlag, utils.MinerNotifyFullFlag, utils.StateDiffFlag, - utils.StateDiffDBFlag, + utils.StateDiffDBNameFlag, + utils.StateDiffDBPasswordFlag, + utils.StateDiffDBUserFlag, + utils.StateDiffDBHostFlag, + utils.StateDiffDBPortFlag, + utils.StateDiffDBMaxConnLifetime, + utils.StateDiffDBMaxConnIdleTime, + utils.StateDiffDBMaxConns, + utils.StateDiffDBMinConns, + utils.StateDiffDBMaxIdleConns, + utils.StateDiffDBConnTimeout, utils.StateDiffDBNodeIDFlag, utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, diff --git a/cmd/geth/usage.go b/cmd/geth/usage.go index e61d2927c..1ad1b8557 100644 --- a/cmd/geth/usage.go +++ b/cmd/geth/usage.go @@ -225,7 +225,17 @@ var AppHelpFlagGroups = []flags.FlagGroup{ Name: "STATE DIFF", Flags: []cli.Flag{ utils.StateDiffFlag, - utils.StateDiffDBFlag, + utils.StateDiffDBNameFlag, + utils.StateDiffDBPasswordFlag, + utils.StateDiffDBUserFlag, + utils.StateDiffDBHostFlag, + utils.StateDiffDBPortFlag, + utils.StateDiffDBMaxConnLifetime, + utils.StateDiffDBMaxConnIdleTime, + utils.StateDiffDBMaxConns, + utils.StateDiffDBMinConns, + utils.StateDiffDBMaxIdleConns, + utils.StateDiffDBConnTimeout, utils.StateDiffDBNodeIDFlag, utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index d7a0b7a6a..8dfb92a7e 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -786,16 +786,59 @@ var ( Name: "statediff", Usage: "Enables the processing of state diffs between each block", } - StateDiffDBFlag = cli.StringFlag{ - Name: "statediff.db", - Usage: "PostgreSQL database connection string for writing state diffs", + StateDiffDBHostFlag = cli.StringFlag{ + Name: "statediff.db.host", + Usage: "Statediff database hostname/ip", + Value: "localhost", + } + StateDiffDBPortFlag = cli.IntFlag{ + Name: "statediff.db.port", + Usage: "Statediff database port", + Value: 5432, + } + StateDiffDBNameFlag = cli.StringFlag{ + Name: "statediff.db.name", + Usage: "Statediff database name", + } + StateDiffDBPasswordFlag = cli.StringFlag{ + Name: "statediff.db.password", + Usage: "Statediff database password", + } + StateDiffDBUserFlag = cli.StringFlag{ + Name: "statediff.db.user", + Usage: "Statediff database username", + Value: "postgres", + } + StateDiffDBMaxConnLifetime = cli.DurationFlag{ + Name: "statediff.db.maxconnlifetime", + Usage: "Statediff database maximum connection lifetime (in seconds)", + } + StateDiffDBMaxConnIdleTime = cli.DurationFlag{ + Name: "statediff.db.maxconnidletime", + Usage: "Statediff database maximum connection idle time (in seconds)", + } + StateDiffDBMaxConns = cli.IntFlag{ + Name: "statediff.db.maxconns", + Usage: "Statediff database maximum connections", + } + StateDiffDBMinConns = cli.IntFlag{ + Name: "statediff.db.minconns", + Usage: "Statediff database minimum connections", + } + StateDiffDBMaxIdleConns = cli.IntFlag{ + Name: "statediff.db.maxidleconns", + Usage: "Statediff database maximum idle connections", + } + StateDiffDBConnTimeout = cli.DurationFlag{ + Name: "statediff.db.conntimeout", + Usage: "Statediff database connection timeout (in seconds)", } StateDiffDBNodeIDFlag = cli.StringFlag{ - Name: "statediff.dbnodeid", + Name: "statediff.db.nodeid", Usage: "Node ID to use when writing state diffs to database", } StateDiffDBClientNameFlag = cli.StringFlag{ - Name: "statediff.dbclientname", + Name: "statediff.db.clientname", Usage: "Client name to use when writing state diffs to database", } StateDiffWritingFlag = cli.BoolFlag{ diff --git a/go.mod b/go.mod index 0f94c2611..da726b7c4 100644 --- a/go.mod +++ b/go.mod @@ -25,6 +25,7 @@ require ( github.com/fatih/color v1.7.0 github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff + github.com/georgysavva/scany v0.2.9 github.com/go-ole/go-ole v1.2.1 // indirect github.com/go-stack/stack v1.8.0 github.com/golang/protobuf v1.4.3 @@ -46,6 +47,10 @@ require ( github.com/ipfs/go-ipfs-blockstore v1.0.1 github.com/ipfs/go-ipfs-ds-help v1.0.0 github.com/ipfs/go-ipld-format v0.2.0 + github.com/jackc/fake v0.0.0-20150926172116-812a484cc733 // indirect + github.com/jackc/pgconn v1.10.0 + github.com/jackc/pgx v3.6.2+incompatible + github.com/jackc/pgx/v4 v4.13.0 github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458 github.com/jedisct1/go-minisign v0.0.0-20190909160543-45766022959e github.com/jmoiron/sqlx v1.2.0 @@ -55,7 +60,6 @@ require ( github.com/lib/pq v1.10.2 github.com/mattn/go-colorable v0.1.8 github.com/mattn/go-isatty v0.0.12 - github.com/mattn/go-sqlite3 v1.14.7 // indirect github.com/multiformats/go-multihash v0.0.14 github.com/naoina/go-stringutil v0.1.0 // indirect github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416 @@ -70,7 +74,7 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 github.com/tklauser/go-sysconf v0.3.5 // indirect github.com/tyler-smith/go-bip39 v1.0.1-0.20181017060643-dbb3b84ba2ef - golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a + golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912 golang.org/x/text v0.3.6 diff --git a/go.sum b/go.sum index 0097b96f7..cf5e867db 100644 --- a/go.sum +++ b/go.sum @@ -42,6 +42,8 @@ github.com/Azure/go-autorest/tracing v0.5.0/go.mod h1:r/s2XiOKccPW3HrqB+W0TQzfbt github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= +github.com/Masterminds/semver/v3 v3.1.1 h1:hLg3sBzpNErnxhQtUy/mmLR2I9foDujNK030IGemrRc= +github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 h1:fLjPD/aNc3UIOA6tDi6QXUemppXK3P9BI7mr2hd6gx8= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= @@ -100,10 +102,17 @@ github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMn github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/cloudflare-go v0.14.0 h1:gFqGlGl/5f9UGXAaKapCGUfaTCgRKKnzu2VvzMZlOFA= github.com/cloudflare/cloudflare-go v0.14.0/go.mod h1:EnwdgGMaFOruiPZRFSgn+TsQ3hQ7C/YWzIGLeu5c304= +github.com/cockroachdb/apd v1.1.0 h1:3LFP3629v+1aKXU5Q37mxmRxX/pIu1nijXydLShEq5I= +github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= +github.com/cockroachdb/cockroach-go/v2 v2.0.3 h1:ZA346ACHIZctef6trOTwBAEvPVm1k0uLm/bb2Atc+S8= +github.com/cockroachdb/cockroach-go/v2 v2.0.3/go.mod h1:hAuDgiVgDVkfirP9JnhXEfcXEPRKBpYdGz+l7mvYSzw= github.com/consensys/bavard v0.1.8-0.20210406032232-f3452dc9b572/go.mod h1:Bpd0/3mZuaj6Sj+PqrmIquiOKy397AKGThQPaGzNXAQ= github.com/consensys/gnark-crypto v0.4.1-0.20210426202927-39ac3d4b3f1f h1:C43yEtQ6NIf4ftFXD/V55gnGFgPbMQobd//YlnLjUJ8= github.com/consensys/gnark-crypto v0.4.1-0.20210426202927-39ac3d4b3f1f/go.mod h1:815PAHg3wvysy0SyIqanF8gZ0Y1wjk/hrDHD/iT88+Q= +github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cyberdelia/templates v0.0.0-20141128023046-ca7fffd4298c/go.mod h1:GyV+0YP4qX0UQ7r2MoYZ+AvYDp12OF5yg4q8rGnyNh4= github.com/dave/jennifer v1.2.0/go.mod h1:fIb+770HOpJ2fmN9EPPKOqm1vMGhB+TwXKMZhrIygKg= @@ -116,6 +125,7 @@ github.com/deckarep/golang-set v0.0.0-20180603214616-504e848d77ea/go.mod h1:93vs github.com/deepmap/oapi-codegen v1.6.0/go.mod h1:ryDa9AgbELGeB+YEXE1dR53yAjHwFvE9iAUlWl9Al3M= github.com/deepmap/oapi-codegen v1.8.2 h1:SegyeYGcdi0jLLrpbCMoJxnUUn8GBXHsvr4rbzjuhfU= github.com/deepmap/oapi-codegen v1.8.2/go.mod h1:YLgSKSDv/bZQB7N4ws6luhozi3cEdRktEqrX88CvjIw= +github.com/denisenkom/go-mssqldb v0.0.0-20191124224453-732737034ffd/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8/go.mod h1:VMaSuZ+SZcx/wljOQKvp5srsbCiKDEb6K2wC4+PiBmQ= @@ -132,6 +142,7 @@ github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw= github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0= github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 h1:FtmdgXiUlNeRsoNMFlKLDt+S+6hbjVMEW6RGQ7aUf7c= @@ -142,6 +153,8 @@ github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWo github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff h1:tY80oXqGNY4FhTFhk+o9oFHGINQ/+vhlm8HFzi6znCI= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= +github.com/georgysavva/scany v0.2.9 h1:Xt6rjYpHnMClTm/g+oZTnoSxUwiln5GqMNU+QeLNHQU= +github.com/georgysavva/scany v0.2.9/go.mod h1:yeOeC1BdIdl6hOwy8uefL2WNSlseFzbhlG/frrh65SA= github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= github.com/getkin/kin-openapi v0.61.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -152,9 +165,11 @@ github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0 h1:Wz+5lgoB0kkuqLEc6NVmwRknTKP6dTGbSqvhZtBI/j0= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.1 h1:2lOsA72HgjxAuMlKpFiCbHTvu44PIVkZ5hqm3RSdI/E= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= github.com/go-openapi/jsonpointer v0.19.5/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= @@ -166,11 +181,15 @@ github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZp github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= +github.com/gofrs/uuid v4.0.0+incompatible h1:1SD/1F5pU8p29ybwgQSwpQk+mwdRrXCYuPhW6m+TnJw= +github.com/gofrs/uuid v4.0.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -281,6 +300,80 @@ github.com/ipfs/go-log v0.0.1 h1:9XTUN/rW64BCG1YhPK9Hoy3q8nr4gOmHHBpgFdfw6Lc= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= +github.com/jackc/chunkreader v1.0.0 h1:4s39bBR8ByfqH+DKm8rQA3E1LHZWB9XWcrz8fqaZbe0= +github.com/jackc/chunkreader v1.0.0/go.mod h1:RT6O25fNZIuasFJRyZ4R/Y2BbhasbmZXF9QQ7T3kePo= +github.com/jackc/chunkreader/v2 v2.0.0/go.mod h1:odVSm741yZoC3dpHEUXIqA9tQRhFrgOHwnPIn9lDKlk= +github.com/jackc/chunkreader/v2 v2.0.1 h1:i+RDz65UE+mmpjTfyz0MoVTnzeYxroil2G82ki7MGG8= +github.com/jackc/chunkreader/v2 v2.0.1/go.mod h1:odVSm741yZoC3dpHEUXIqA9tQRhFrgOHwnPIn9lDKlk= +github.com/jackc/fake v0.0.0-20150926172116-812a484cc733 h1:vr3AYkKovP8uR8AvSGGUK1IDqRa5lAAvEkZG1LKaCRc= +github.com/jackc/fake v0.0.0-20150926172116-812a484cc733/go.mod h1:WrMFNQdiFJ80sQsxDoMokWK1W5TQtxBFNpzWTD84ibQ= +github.com/jackc/pgconn v0.0.0-20190420214824-7e0022ef6ba3/go.mod h1:jkELnwuX+w9qN5YIfX0fl88Ehu4XC3keFuOJJk9pcnA= +github.com/jackc/pgconn v0.0.0-20190824142844-760dd75542eb/go.mod h1:lLjNuW/+OfW9/pnVKPazfWOgNfH2aPem8YQ7ilXGvJE= +github.com/jackc/pgconn v0.0.0-20190831204454-2fabfa3c18b7/go.mod h1:ZJKsE/KZfsUgOEh9hBm+xYTstcNHg7UPMVJqRfQxq4s= +github.com/jackc/pgconn v1.4.0/go.mod h1:Y2O3ZDF0q4mMacyWV3AstPJpeHXWGEetiFttmq5lahk= +github.com/jackc/pgconn v1.5.0/go.mod h1:QeD3lBfpTFe8WUnPZWN5KY/mB8FGMIYRdd8P8Jr0fAI= +github.com/jackc/pgconn v1.5.1-0.20200601181101-fa742c524853/go.mod h1:QeD3lBfpTFe8WUnPZWN5KY/mB8FGMIYRdd8P8Jr0fAI= +github.com/jackc/pgconn v1.6.4/go.mod h1:w2pne1C2tZgP+TvjqLpOigGzNqjBgQW9dUw/4Chex78= +github.com/jackc/pgconn v1.7.0/go.mod h1:sF/lPpNEMEOp+IYhyQGdAvrG20gWf6A1tKlr0v7JMeA= +github.com/jackc/pgconn v1.8.0/go.mod h1:1C2Pb36bGIP9QHGBYCjnyhqu7Rv3sGshaQUvmfGIB/o= +github.com/jackc/pgconn v1.9.0/go.mod h1:YctiPyvzfU11JFxoXokUOOKQXQmDMoJL9vJzHH8/2JY= +github.com/jackc/pgconn v1.9.1-0.20210724152538-d89c8390a530/go.mod h1:4z2w8XhRbP1hYxkpTuBjTS3ne3J48K83+u0zoyvg2pI= +github.com/jackc/pgconn v1.10.0 h1:4EYhlDVEMsJ30nNj0mmgwIUXoq7e9sMJrVC2ED6QlCU= +github.com/jackc/pgconn v1.10.0/go.mod h1:4z2w8XhRbP1hYxkpTuBjTS3ne3J48K83+u0zoyvg2pI= +github.com/jackc/pgio v1.0.0 h1:g12B9UwVnzGhueNavwioyEEpAmqMe1E/BN9ES+8ovkE= +github.com/jackc/pgio v1.0.0/go.mod h1:oP+2QK2wFfUWgr+gxjoBH9KGBb31Eio69xUb0w5bYf8= +github.com/jackc/pgmock v0.0.0-20190831213851-13a1b77aafa2/go.mod h1:fGZlG77KXmcq05nJLRkk0+p82V8B8Dw8KN2/V9c/OAE= +github.com/jackc/pgmock v0.0.0-20201204152224-4fe30f7445fd/go.mod h1:hrBW0Enj2AZTNpt/7Y5rr2xe/9Mn757Wtb2xeBzPv2c= +github.com/jackc/pgmock v0.0.0-20210724152146-4ad1a8207f65 h1:DadwsjnMwFjfWc9y5Wi/+Zz7xoE5ALHsRQlOctkOiHc= +github.com/jackc/pgmock v0.0.0-20210724152146-4ad1a8207f65/go.mod h1:5R2h2EEX+qri8jOWMbJCtaPWkrrNc7OHwsp2TCqp7ak= +github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM= +github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= +github.com/jackc/pgproto3 v1.1.0 h1:FYYE4yRw+AgI8wXIinMlNjBbp/UitDJwfj5LqqewP1A= +github.com/jackc/pgproto3 v1.1.0/go.mod h1:eR5FA3leWg7p9aeAqi37XOTgTIbkABlvcPB3E5rlc78= +github.com/jackc/pgproto3/v2 v2.0.0-alpha1.0.20190420180111-c116219b62db/go.mod h1:bhq50y+xrl9n5mRYyCBFKkpRVTLYJVWeCc+mEAI3yXA= +github.com/jackc/pgproto3/v2 v2.0.0-alpha1.0.20190609003834-432c2951c711/go.mod h1:uH0AWtUmuShn0bcesswc4aBTWGvw0cAxIJp+6OB//Wg= +github.com/jackc/pgproto3/v2 v2.0.0-rc3/go.mod h1:ryONWYqW6dqSg1Lw6vXNMXoBJhpzvWKnT95C46ckYeM= +github.com/jackc/pgproto3/v2 v2.0.0-rc3.0.20190831210041-4c03ce451f29/go.mod h1:ryONWYqW6dqSg1Lw6vXNMXoBJhpzvWKnT95C46ckYeM= +github.com/jackc/pgproto3/v2 v2.0.1/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA= +github.com/jackc/pgproto3/v2 v2.0.2/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA= +github.com/jackc/pgproto3/v2 v2.0.5/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA= +github.com/jackc/pgproto3/v2 v2.0.6/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA= +github.com/jackc/pgproto3/v2 v2.1.1 h1:7PQ/4gLoqnl87ZxL7xjO0DR5gYuviDCZxQJsUlFW1eI= +github.com/jackc/pgproto3/v2 v2.1.1/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA= +github.com/jackc/pgservicefile v0.0.0-20200307190119-3430c5407db8/go.mod h1:vsD4gTJCa9TptPL8sPkXrLZ+hDuNrZCnj29CQpr4X1E= +github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b h1:C8S2+VttkHFdOOCXJe+YGfa4vHYwlt4Zx+IVXQ97jYg= +github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b/go.mod h1:vsD4gTJCa9TptPL8sPkXrLZ+hDuNrZCnj29CQpr4X1E= +github.com/jackc/pgtype v0.0.0-20190421001408-4ed0de4755e0/go.mod h1:hdSHsc1V01CGwFsrv11mJRHWJ6aifDLfdV3aVjFF0zg= +github.com/jackc/pgtype v0.0.0-20190824184912-ab885b375b90/go.mod h1:KcahbBH1nCMSo2DXpzsoWOAfFkdEtEJpPbVLq8eE+mc= +github.com/jackc/pgtype v0.0.0-20190828014616-a8802b16cc59/go.mod h1:MWlu30kVJrUS8lot6TQqcg7mtthZ9T0EoIBFiJcmcyw= +github.com/jackc/pgtype v1.2.0/go.mod h1:5m2OfMh1wTK7x+Fk952IDmI4nw3nPrvtQdM0ZT4WpC0= +github.com/jackc/pgtype v1.3.0/go.mod h1:b0JqxHvPmljG+HQ5IsvQ0yqeSi4nGcDTVjFoiLDb0Ik= +github.com/jackc/pgtype v1.3.1-0.20200510190516-8cd94a14c75a/go.mod h1:vaogEUkALtxZMCH411K+tKzNpwzCKU+AnPzBKZ+I+Po= +github.com/jackc/pgtype v1.3.1-0.20200606141011-f6355165a91c/go.mod h1:cvk9Bgu/VzJ9/lxTO5R5sf80p0DiucVtN7ZxvaC4GmQ= +github.com/jackc/pgtype v1.4.2/go.mod h1:JCULISAZBFGrHaOXIIFiyfzW5VY0GRitRr8NeJsrdig= +github.com/jackc/pgtype v1.8.1-0.20210724151600-32e20a603178/go.mod h1:C516IlIV9NKqfsMCXTdChteoXmwgUceqaLfjg2e3NlM= +github.com/jackc/pgtype v1.8.1 h1:9k0IXtdJXHJbyAWQgbWr1lU+MEhPXZz6RIXxfR5oxXs= +github.com/jackc/pgtype v1.8.1/go.mod h1:LUMuVrfsFfdKGLw+AFFVv6KtHOFMwRgDDzBt76IqCA4= +github.com/jackc/pgx v3.6.2+incompatible h1:2zP5OD7kiyR3xzRYMhOcXVvkDZsImVXfj+yIyTQf3/o= +github.com/jackc/pgx v3.6.2+incompatible/go.mod h1:0ZGrqGqkRlliWnWB4zKnWtjbSWbGkVEFm4TeybAXq+I= +github.com/jackc/pgx/v4 v4.0.0-20190420224344-cc3461e65d96/go.mod h1:mdxmSJJuR08CZQyj1PVQBHy9XOp5p8/SHH6a0psbY9Y= +github.com/jackc/pgx/v4 v4.0.0-20190421002000-1b8f0016e912/go.mod h1:no/Y67Jkk/9WuGR0JG/JseM9irFbnEPbuWV2EELPNuM= +github.com/jackc/pgx/v4 v4.0.0-pre1.0.20190824185557-6972a5742186/go.mod h1:X+GQnOEnf1dqHGpw7JmHqHc1NxDoalibchSk9/RWuDc= +github.com/jackc/pgx/v4 v4.5.0/go.mod h1:EpAKPLdnTorwmPUUsqrPxy5fphV18j9q3wrfRXgo+kA= +github.com/jackc/pgx/v4 v4.6.0/go.mod h1:vPh43ZzxijXUVJ+t/EmXBtFmbFVO72cuneCT9oAlxAg= +github.com/jackc/pgx/v4 v4.6.1-0.20200510190926-94ba730bb1e9/go.mod h1:t3/cdRQl6fOLDxqtlyhe9UWgfIi9R8+8v8GKV5TRA/o= +github.com/jackc/pgx/v4 v4.6.1-0.20200606145419-4e5062306904/go.mod h1:ZDaNWkt9sW1JMiNn0kdYBaLelIhw7Pg4qd+Vk6tw7Hg= +github.com/jackc/pgx/v4 v4.8.1/go.mod h1:4HOLxrl8wToZJReD04/yB20GDwf4KBYETvlHciCnwW0= +github.com/jackc/pgx/v4 v4.12.1-0.20210724153913-640aa07df17c/go.mod h1:1QD0+tgSXP7iUjYm9C1NxKhny7lq6ee99u/z+IHFcgs= +github.com/jackc/pgx/v4 v4.13.0 h1:JCjhT5vmhMAf/YwBHLvrBn4OGdIQBiFG6ym8Zmdx570= +github.com/jackc/pgx/v4 v4.13.0/go.mod h1:9P4X524sErlaxj0XSGZk7s+LD0eOyu1ZDUrrpznYDF0= +github.com/jackc/puddle v0.0.0-20190413234325-e4ced69a3a2b/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= +github.com/jackc/puddle v0.0.0-20190608224051-11cab39313c9/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= +github.com/jackc/puddle v1.1.0/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= +github.com/jackc/puddle v1.1.1/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= +github.com/jackc/puddle v1.1.2/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= +github.com/jackc/puddle v1.1.3 h1:JnPg/5Q9xVJGfjsO5CPUOjnJps1JaRUm8I9FXVCFK94= +github.com/jackc/puddle v1.1.3/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458 h1:6OvNmYgJyexcZ3pYbTI9jWx5tHo1Dee/tWbLMfPe2TA= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8 h1:bspPhN+oKYFk5fcGNuQzp6IGzYQSenLEgH3s6jkXrWw= @@ -288,6 +381,9 @@ github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8/go.mod h1:Ly/wlsj github.com/jedisct1/go-minisign v0.0.0-20190909160543-45766022959e h1:UvSe12bq+Uj2hWd8aOlwPmoZ+CITRFrdit+sDGfAg8U= github.com/jedisct1/go-minisign v0.0.0-20190909160543-45766022959e/go.mod h1:G1CVv03EnqU1wYL2dFwXxW2An0az9JTl/ZsqXQeBlkU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= +github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= +github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= @@ -313,12 +409,13 @@ github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM52 github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= +github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/pty v1.1.8/go.mod h1:O1sed60cT9XZ5uDucP5qwvh+TE3NnUj51EiZO/lmSfw= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= @@ -329,6 +426,11 @@ github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.1.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.4.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/lib/pq v1.10.2 h1:AqzbZs4ZoCBp+GtejcpCpcxM3zlSMx29dXbUSeVtJb8= github.com/lib/pq v1.10.2/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= @@ -338,6 +440,7 @@ github.com/matryer/moq v0.0.0-20190312154309-6cfb0558e1bd/go.mod h1:9ELz6aaclSIG github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.8 h1:c1ghPdyEDarC70ftn0y+A/Ee++9zz8ljHG1b13eJ0s8= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= @@ -346,6 +449,7 @@ github.com/mattn/go-ieproxy v0.0.0-20190702010315-6dee0af9227d h1:oNAwILwmgWKFpu github.com/mattn/go-ieproxy v0.0.0-20190702010315-6dee0af9227d/go.mod h1:31jz6HNzdxOmlERGGEc4v/dMssOfmp2p5bT/okiKFFc= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= @@ -355,8 +459,8 @@ github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/Qd github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/mattn/go-sqlite3 v1.14.7 h1:fxWBnXkxfM6sRiuH3bqJ4CfzZojMOLVc0UTsTglEghA= -github.com/mattn/go-sqlite3 v1.14.7/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= +github.com/mattn/go-sqlite3 v2.0.1+incompatible h1:xQ15muvnzGBHpIpdrNi1DA5x0+TcBZzsIDwmw9uTHzw= +github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= @@ -441,14 +545,25 @@ github.com/rjeczalik/notify v0.9.1/go.mod h1:rKwnCoCGeuQnwBtTSPL9Dad03Vh2n40ePRr github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= +github.com/rs/zerolog v1.13.0/go.mod h1:YbFCdg8HfsridGWAh22vktObvhZbQsZXe4/zB0OKkWU= +github.com/rs/zerolog v1.15.0/go.mod h1:xYTKnLHcpfU2225ny5qZjxnj9NvkumZYjJHlAThCjNc= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/segmentio/kafka-go v0.2.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= +github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shopspring/decimal v0.0.0-20200419222939-1884f454f8ea/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ= +github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -461,6 +576,8 @@ github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 h1:Gb2Tyox57N github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4/go.mod h1:RZLeN1LMWmRsyYjvAu+I6Dm9QmlDaIIt+Y+4Kd7Tp+Q= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= +github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.0/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -487,29 +604,45 @@ github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPyS github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= +go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190411191339-88737f569e3a/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190909091759-094676da4a83/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190911031432-227b76d455e7/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201203163018-be400aefbc4c/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a h1:kr2P4QFmQr29mSLA43kwrOcgcReGTfbE9N577tCTuBc= -golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= +golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97 h1:/UOmuWzQfxxo9UtlXMwuQU8CMgg1eZXqTRwkSQJWKOI= +golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -555,6 +688,7 @@ golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= @@ -589,13 +723,16 @@ golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190219092855-153ac476189d/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -618,6 +755,7 @@ golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210420205809-ac73e9fd8988/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912 h1:uCLL3g5wH2xjxVREVuAbP9JM5PPKjRbXKRa6IBjkzmU= golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= @@ -648,25 +786,31 @@ golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3 golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190425163242-31fd60d6bfdc/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190823170909-c4a336ef6a2f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200108203644-89082a384178/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.2 h1:kRBLX7v7Af8W7Gdbbc908OJcdgtK8bOz9Uaj8/F1ACA= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -725,6 +869,7 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s= gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce h1:+JknDZhAj8YMt7GC73Ei8pv4MzjDUNPHgQWJdtMAaDU= gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce/go.mod h1:5AcXVHNjg+BDxry382+8OKon8SEWiKktQR07RKPsv1c= gopkg.in/olebedev/go-duktape.v3 v3.0.0-20200619000410-60c24ae608a6 h1:a6cXbcDDUkSBlpnkWV1bJ+vv3mOgQEltEJ2rPxroVu0= diff --git a/statediff/README.md b/statediff/README.md index 74c82f2d2..dd2eaed7f 100644 --- a/statediff/README.md +++ b/statediff/README.md @@ -73,23 +73,31 @@ type Payload struct { ## Usage This state diffing service runs as an auxiliary service concurrent to the regular syncing process of the geth node. - ### CLI configuration This service introduces a CLI flag namespace `statediff` `--statediff` flag is used to turn on the service `--statediff.writing` is used to tell the service to write state diff objects it produces from synced ChainEvents directly to a configured Postgres database `--statediff.workers` is used to set the number of concurrent workers to process state diff objects and write them into the database -`--statediff.db` is the connection string for the Postgres database to write to -`--statediff.db.init` indicates whether we need to initialize a new database; set true if its the first time running the process on a given database -`--statediff.dbnodeid` is the node id to use in the Postgres database -`--statediff.dbclientname` is the client name to use in the Postgres database +`--statediff.db.host` is the hostname/ip to dial to connect to the database +`--statediff.db.port` is the port to dial to connect to the database +`--statediff.db.name` is the name of the database to connect to +`--statediff.db.user` is the user to connect to the database as +`--statediff.db.password` is the password to use to connect to the database +`--statediff.db.conntimeout` is the connection timeout (in seconds) +`--statediff.db.maxconns` is the maximum number of database connections +`--statediff.db.minconns` is the minimum number of database connections +`--statediff.db.maxidleconns` is the maximum number of idle connections +`--statediff.db.maxconnidletime` is the maximum lifetime for an idle connection (in seconds) +`--statediff.db.maxconnlifetime` is the maximum lifetime for a connection (in seconds) +`--statediff.db.nodeid` is the node id to use in the Postgres database +`--statediff.db.clientname` is the client name to use in the Postgres database The service can only operate in full sync mode (`--syncmode=full`), but only the historical RPC endpoints require an archive node (`--gcmode=archive`) e.g. ` -./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db=postgres://localhost:5432/vulcanize_testing?sslmode=disable --statediff.db.init=true --statediff.dbnodeid={nodeId} --statediff.dbclientname={dbClientName} +./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db=postgres://localhost:5432/vulcanize_testing?sslmode=disable --statediff.dbnodeid={nodeId} --statediff.dbclientname={dbClientName} ` ### RPC endpoints diff --git a/statediff/api.go b/statediff/api.go index 923a0073f..5c534cddb 100644 --- a/statediff/api.go +++ b/statediff/api.go @@ -19,11 +19,11 @@ package statediff import ( "context" - "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rpc" - . "github.com/ethereum/go-ethereum/statediff/types" ) // APIName is the namespace used for the state diffing service API @@ -117,7 +117,7 @@ func (api *PublicStateDiffAPI) StreamCodeAndCodeHash(ctx context.Context, blockN // create subscription and start waiting for events rpcSub := notifier.CreateSubscription() - payloadChan := make(chan CodeAndCodeHash, chainEventChanSize) + payloadChan := make(chan types.CodeAndCodeHash, chainEventChanSize) quitChan := make(chan bool) api.sds.StreamCodeAndCodeHash(blockNumber, payloadChan, quitChan) go func() { diff --git a/statediff/builder.go b/statediff/builder.go index 7befb6b3c..eacfeca15 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -23,14 +23,16 @@ import ( "bytes" "fmt" + "github.com/ethereum/go-ethereum/statediff/trie_helpers" + + types2 "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/common" "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/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" ) @@ -43,9 +45,9 @@ var ( // Builder interface exposes the method for building a state diff between two blocks type Builder interface { - BuildStateDiffObject(args Args, params Params) (StateObject, error) - BuildStateTrieObject(current *types.Block) (StateObject, error) - WriteStateDiffObject(args StateRoots, params Params, output StateNodeSink, codeOutput CodeSink) error + BuildStateDiffObject(args Args, params Params) (types2.StateObject, error) + BuildStateTrieObject(current *types.Block) (types2.StateObject, error) + WriteStateDiffObject(args types2.StateRoots, params Params, output types2.StateNodeSink, codeOutput types2.CodeSink) error } type builder struct { @@ -53,20 +55,20 @@ type builder struct { } // convenience -func stateNodeAppender(nodes *[]StateNode) StateNodeSink { - return func(node StateNode) error { +func stateNodeAppender(nodes *[]types2.StateNode) types2.StateNodeSink { + return func(node types2.StateNode) error { *nodes = append(*nodes, node) return nil } } -func storageNodeAppender(nodes *[]StorageNode) StorageNodeSink { - return func(node StorageNode) error { +func storageNodeAppender(nodes *[]types2.StorageNode) types2.StorageNodeSink { + return func(node types2.StorageNode) error { *nodes = append(*nodes, node) return nil } } -func codeMappingAppender(codeAndCodeHashes *[]CodeAndCodeHash) CodeSink { - return func(c CodeAndCodeHash) error { +func codeMappingAppender(codeAndCodeHashes *[]types2.CodeAndCodeHash) types2.CodeSink { + return func(c types2.CodeAndCodeHash) error { *codeAndCodeHashes = append(*codeAndCodeHashes, c) return nil } @@ -80,17 +82,17 @@ func NewBuilder(stateCache state.Database) Builder { } // BuildStateTrieObject builds a state trie object from the provided block -func (sdb *builder) BuildStateTrieObject(current *types.Block) (StateObject, error) { +func (sdb *builder) BuildStateTrieObject(current *types.Block) (types2.StateObject, error) { currentTrie, err := sdb.stateCache.OpenTrie(current.Root()) if err != nil { - return StateObject{}, fmt.Errorf("error creating trie for block %d: %v", current.Number(), err) + return types2.StateObject{}, fmt.Errorf("error creating trie for block %d: %v", current.Number(), err) } it := currentTrie.NodeIterator([]byte{}) stateNodes, codeAndCodeHashes, err := sdb.buildStateTrie(it) if err != nil { - return StateObject{}, fmt.Errorf("error collecting state nodes for block %d: %v", current.Number(), err) + return types2.StateObject{}, fmt.Errorf("error collecting state nodes for block %d: %v", current.Number(), err) } - return StateObject{ + return types2.StateObject{ BlockNumber: current.Number(), BlockHash: current.Hash(), Nodes: stateNodes, @@ -98,20 +100,20 @@ func (sdb *builder) BuildStateTrieObject(current *types.Block) (StateObject, err }, nil } -func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]StateNode, []CodeAndCodeHash, error) { - stateNodes := make([]StateNode, 0) - codeAndCodeHashes := make([]CodeAndCodeHash, 0) +func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]types2.StateNode, []types2.CodeAndCodeHash, error) { + stateNodes := make([]types2.StateNode, 0) + codeAndCodeHashes := make([]types2.CodeAndCodeHash, 0) for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return nil, nil, err } switch node.NodeType { - case Leaf: + case types2.Leaf: var account types.StateAccount if err := rlp.DecodeBytes(nodeElements[1].([]byte), &account); err != nil { return nil, nil, fmt.Errorf("error decoding account for leaf node at path %x nerror: %v", node.Path, err) @@ -122,7 +124,7 @@ func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]StateNode, []CodeAnd leafKey := encodedPath[1:] node.LeafKey = leafKey if !bytes.Equal(account.CodeHash, nullCodeHash) { - var storageNodes []StorageNode + var storageNodes []types2.StorageNode err := sdb.buildStorageNodesEventual(account.Root, nil, true, storageNodeAppender(&storageNodes)) if err != nil { return nil, nil, fmt.Errorf("failed building eventual storage diffs for account %+v\r\nerror: %v", account, err) @@ -134,13 +136,13 @@ func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]StateNode, []CodeAnd if err != nil { return nil, nil, fmt.Errorf("failed to retrieve code for codehash %s\r\n error: %v", codeHash.String(), err) } - codeAndCodeHashes = append(codeAndCodeHashes, CodeAndCodeHash{ + codeAndCodeHashes = append(codeAndCodeHashes, types2.CodeAndCodeHash{ Hash: codeHash, Code: code, }) } stateNodes = append(stateNodes, node) - case Extension, Branch: + case types2.Extension, types2.Branch: stateNodes = append(stateNodes, node) default: return nil, nil, fmt.Errorf("unexpected node type %s", node.NodeType) @@ -150,16 +152,16 @@ func (sdb *builder) buildStateTrie(it trie.NodeIterator) ([]StateNode, []CodeAnd } // BuildStateDiffObject builds a statediff object from two blocks and the provided parameters -func (sdb *builder) BuildStateDiffObject(args Args, params Params) (StateObject, error) { - var stateNodes []StateNode - var codeAndCodeHashes []CodeAndCodeHash +func (sdb *builder) BuildStateDiffObject(args Args, params Params) (types2.StateObject, error) { + var stateNodes []types2.StateNode + var codeAndCodeHashes []types2.CodeAndCodeHash err := sdb.WriteStateDiffObject( - StateRoots{OldStateRoot: args.OldStateRoot, NewStateRoot: args.NewStateRoot}, + types2.StateRoots{OldStateRoot: args.OldStateRoot, NewStateRoot: args.NewStateRoot}, params, stateNodeAppender(&stateNodes), codeMappingAppender(&codeAndCodeHashes)) if err != nil { - return StateObject{}, err + return types2.StateObject{}, err } - return StateObject{ + return types2.StateObject{ BlockHash: args.BlockHash, BlockNumber: args.BlockNumber, Nodes: stateNodes, @@ -168,7 +170,7 @@ func (sdb *builder) BuildStateDiffObject(args Args, params Params) (StateObject, } // Writes a statediff object to output callback -func (sdb *builder) WriteStateDiffObject(args StateRoots, params Params, output StateNodeSink, codeOutput CodeSink) error { +func (sdb *builder) WriteStateDiffObject(args types2.StateRoots, params Params, output types2.StateNodeSink, codeOutput types2.CodeSink) error { if !params.IntermediateStateNodes || len(params.WatchedAddresses) > 0 { // if we are watching only specific accounts then we are only diffing leaf nodes return sdb.buildStateDiffWithoutIntermediateStateNodes(args, params, output, codeOutput) @@ -177,7 +179,7 @@ func (sdb *builder) WriteStateDiffObject(args StateRoots, params Params, output } } -func (sdb *builder) buildStateDiffWithIntermediateStateNodes(args StateRoots, params Params, output StateNodeSink, codeOutput CodeSink) error { +func (sdb *builder) buildStateDiffWithIntermediateStateNodes(args types2.StateRoots, params Params, output types2.StateNodeSink, codeOutput types2.CodeSink) error { // Load tries for old and new states oldTrie, err := sdb.stateCache.OpenTrie(args.OldStateRoot) if err != nil { @@ -208,14 +210,14 @@ func (sdb *builder) buildStateDiffWithIntermediateStateNodes(args StateRoots, pa } // collect and sort the leafkey keys for both account mappings into a slice - createKeys := sortKeys(diffAccountsAtB) - deleteKeys := sortKeys(diffAccountsAtA) + createKeys := trie_helpers.SortKeys(diffAccountsAtB) + deleteKeys := trie_helpers.SortKeys(diffAccountsAtA) // and then find the intersection of these keys // these are the leafkeys for the accounts which exist at both A and B but are different // this also mutates the passed in createKeys and deleteKeys, removing the intersection keys // and leaving the truly created or deleted keys in place - updatedKeys := findIntersection(createKeys, deleteKeys) + updatedKeys := trie_helpers.FindIntersection(createKeys, deleteKeys) // build the diff nodes for the updated accounts using the mappings at both A and B as directed by the keys found as the intersection of the two err = sdb.buildAccountUpdates( @@ -232,7 +234,7 @@ func (sdb *builder) buildStateDiffWithIntermediateStateNodes(args StateRoots, pa return nil } -func (sdb *builder) buildStateDiffWithoutIntermediateStateNodes(args StateRoots, params Params, output StateNodeSink, codeOutput CodeSink) error { +func (sdb *builder) buildStateDiffWithoutIntermediateStateNodes(args types2.StateRoots, params Params, output types2.StateNodeSink, codeOutput types2.CodeSink) error { // Load tries for old (A) and new (B) states oldTrie, err := sdb.stateCache.OpenTrie(args.OldStateRoot) if err != nil { @@ -262,14 +264,14 @@ func (sdb *builder) buildStateDiffWithoutIntermediateStateNodes(args StateRoots, } // collect and sort the leafkeys for both account mappings into a slice - createKeys := sortKeys(diffAccountsAtB) - deleteKeys := sortKeys(diffAccountsAtA) + createKeys := trie_helpers.SortKeys(diffAccountsAtB) + deleteKeys := trie_helpers.SortKeys(diffAccountsAtA) // and then find the intersection of these keys // these are the leafkeys for the accounts which exist at both A and B but are different // this also mutates the passed in createKeys and deleteKeys, removing in intersection keys // and leaving the truly created or deleted keys in place - updatedKeys := findIntersection(createKeys, deleteKeys) + updatedKeys := trie_helpers.FindIntersection(createKeys, deleteKeys) // build the diff nodes for the updated accounts using the mappings at both A and B as directed by the keys found as the intersection of the two err = sdb.buildAccountUpdates( @@ -289,20 +291,20 @@ func (sdb *builder) buildStateDiffWithoutIntermediateStateNodes(args StateRoots, // createdAndUpdatedState returns // a mapping of their leafkeys to all the accounts that exist in a different state at B than A // and a slice of the paths for all of the nodes included in both -func (sdb *builder) createdAndUpdatedState(a, b trie.NodeIterator, watchedAddresses []common.Address) (AccountMap, map[string]bool, error) { +func (sdb *builder) createdAndUpdatedState(a, b trie.NodeIterator, watchedAddresses []common.Address) (types2.AccountMap, map[string]bool, error) { diffPathsAtB := make(map[string]bool) - diffAcountsAtB := make(AccountMap) + diffAcountsAtB := make(types2.AccountMap) it, _ := trie.NewDifferenceIterator(a, b) for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return nil, nil, err } - if node.NodeType == Leaf { + if node.NodeType == types2.Leaf { // created vs updated is important for leaf nodes since we need to diff their storage // so we need to map all changed accounts at B to their leafkey, since account can change pathes but not leafkey var account types.StateAccount @@ -314,7 +316,7 @@ func (sdb *builder) createdAndUpdatedState(a, b trie.NodeIterator, watchedAddres encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] if isWatchedAddress(watchedAddresses, leafKey) { - diffAcountsAtB[common.Bytes2Hex(leafKey)] = accountWrapper{ + diffAcountsAtB[common.Bytes2Hex(leafKey)] = types2.AccountWrapper{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -333,21 +335,21 @@ func (sdb *builder) createdAndUpdatedState(a, b trie.NodeIterator, watchedAddres // a slice of all the intermediate nodes that exist in a different state at B than A // a mapping of their leafkeys to all the accounts that exist in a different state at B than A // and a slice of the paths for all of the nodes included in both -func (sdb *builder) createdAndUpdatedStateWithIntermediateNodes(a, b trie.NodeIterator, output StateNodeSink) (AccountMap, map[string]bool, error) { +func (sdb *builder) createdAndUpdatedStateWithIntermediateNodes(a, b trie.NodeIterator, output types2.StateNodeSink) (types2.AccountMap, map[string]bool, error) { diffPathsAtB := make(map[string]bool) - diffAcountsAtB := make(AccountMap) + diffAcountsAtB := make(types2.AccountMap) it, _ := trie.NewDifferenceIterator(a, b) for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return nil, nil, err } switch node.NodeType { - case Leaf: + case types2.Leaf: // created vs updated is important for leaf nodes since we need to diff their storage // so we need to map all changed accounts at B to their leafkey, since account can change paths but not leafkey var account types.StateAccount @@ -358,17 +360,17 @@ func (sdb *builder) createdAndUpdatedStateWithIntermediateNodes(a, b trie.NodeIt valueNodePath := append(node.Path, partialPath...) encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] - diffAcountsAtB[common.Bytes2Hex(leafKey)] = accountWrapper{ + diffAcountsAtB[common.Bytes2Hex(leafKey)] = types2.AccountWrapper{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, LeafKey: leafKey, Account: &account, } - case Extension, Branch: + case types2.Extension, types2.Branch: // create a diff for any intermediate node that has changed at b // created vs updated makes no difference for intermediate nodes since we do not need to diff storage - if err := output(StateNode{ + if err := output(types2.StateNode{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -386,20 +388,20 @@ func (sdb *builder) createdAndUpdatedStateWithIntermediateNodes(a, b trie.NodeIt // deletedOrUpdatedState returns a slice of all the pathes that are emptied at B // and a mapping of their leafkeys to all the accounts that exist in a different state at A than B -func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB map[string]bool, output StateNodeSink) (AccountMap, error) { - diffAccountAtA := make(AccountMap) +func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB map[string]bool, output types2.StateNodeSink) (types2.AccountMap, error) { + diffAccountAtA := make(types2.AccountMap) it, _ := trie.NewDifferenceIterator(b, a) for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return nil, err } switch node.NodeType { - case Leaf: + case types2.Leaf: // map all different accounts at A to their leafkey var account types.StateAccount if err := rlp.DecodeBytes(nodeElements[1].([]byte), &account); err != nil { @@ -409,7 +411,7 @@ func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB m valueNodePath := append(node.Path, partialPath...) encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] - diffAccountAtA[common.Bytes2Hex(leafKey)] = accountWrapper{ + diffAccountAtA[common.Bytes2Hex(leafKey)] = types2.AccountWrapper{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -420,24 +422,24 @@ func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB m // that means the node at this path was deleted (or moved) in B // emit an empty "removed" diff to signify as such if _, ok := diffPathsAtB[common.Bytes2Hex(node.Path)]; !ok { - if err := output(StateNode{ + if err := output(types2.StateNode{ Path: node.Path, NodeValue: []byte{}, - NodeType: Removed, + NodeType: types2.Removed, LeafKey: leafKey, }); err != nil { return nil, err } } - case Extension, Branch: + case types2.Extension, types2.Branch: // if this node's path did not show up in diffPathsAtB // that means the node at this path was deleted (or moved) in B // emit an empty "removed" diff to signify as such if _, ok := diffPathsAtB[common.Bytes2Hex(node.Path)]; !ok { - if err := output(StateNode{ + if err := output(types2.StateNode{ Path: node.Path, NodeValue: []byte{}, - NodeType: Removed, + NodeType: types2.Removed, }); err != nil { return nil, err } @@ -454,13 +456,13 @@ func (sdb *builder) deletedOrUpdatedState(a, b trie.NodeIterator, diffPathsAtB m // to generate the statediff node objects for all of the accounts that existed at both A and B but in different states // needs to be called before building account creations and deletions as this mutates // those account maps to remove the accounts which were updated -func (sdb *builder) buildAccountUpdates(creations, deletions AccountMap, updatedKeys []string, - watchedStorageKeys []common.Hash, intermediateStorageNodes bool, output StateNodeSink) error { +func (sdb *builder) buildAccountUpdates(creations, deletions types2.AccountMap, updatedKeys []string, + watchedStorageKeys []common.Hash, intermediateStorageNodes bool, output types2.StateNodeSink) error { var err error for _, key := range updatedKeys { createdAcc := creations[key] deletedAcc := deletions[key] - var storageDiffs []StorageNode + var storageDiffs []types2.StorageNode if deletedAcc.Account != nil && createdAcc.Account != nil { oldSR := deletedAcc.Account.Root newSR := createdAcc.Account.Root @@ -471,7 +473,7 @@ func (sdb *builder) buildAccountUpdates(creations, deletions AccountMap, updated return fmt.Errorf("failed building incremental storage diffs for account with leafkey %s\r\nerror: %v", key, err) } } - if err = output(StateNode{ + if err = output(types2.StateNode{ NodeType: createdAcc.NodeType, Path: createdAcc.Path, NodeValue: createdAcc.NodeValue, @@ -489,9 +491,9 @@ func (sdb *builder) buildAccountUpdates(creations, deletions AccountMap, updated // buildAccountCreations returns the statediff node objects for all the accounts that exist at B but not at A // it also returns the code and codehash for created contract accounts -func (sdb *builder) buildAccountCreations(accounts AccountMap, watchedStorageKeys []common.Hash, intermediateStorageNodes bool, output StateNodeSink, codeOutput CodeSink) error { +func (sdb *builder) buildAccountCreations(accounts types2.AccountMap, watchedStorageKeys []common.Hash, intermediateStorageNodes bool, output types2.StateNodeSink, codeOutput types2.CodeSink) error { for _, val := range accounts { - diff := StateNode{ + diff := types2.StateNode{ NodeType: val.NodeType, Path: val.Path, LeafKey: val.LeafKey, @@ -499,7 +501,7 @@ func (sdb *builder) buildAccountCreations(accounts AccountMap, watchedStorageKey } if !bytes.Equal(val.Account.CodeHash, nullCodeHash) { // For contract creations, any storage node contained is a diff - var storageDiffs []StorageNode + var storageDiffs []types2.StorageNode err := sdb.buildStorageNodesEventual(val.Account.Root, watchedStorageKeys, intermediateStorageNodes, storageNodeAppender(&storageDiffs)) if err != nil { return fmt.Errorf("failed building eventual storage diffs for node %x\r\nerror: %v", val.Path, err) @@ -511,7 +513,7 @@ func (sdb *builder) buildAccountCreations(accounts AccountMap, watchedStorageKey if err != nil { return fmt.Errorf("failed to retrieve code for codehash %s\r\n error: %v", codeHash.String(), err) } - if err := codeOutput(CodeAndCodeHash{ + if err := codeOutput(types2.CodeAndCodeHash{ Hash: codeHash, Code: code, }); err != nil { @@ -528,7 +530,7 @@ func (sdb *builder) buildAccountCreations(accounts AccountMap, watchedStorageKey // buildStorageNodesEventual builds the storage diff node objects for a created account // i.e. it returns all the storage nodes at this state, since there is no previous state -func (sdb *builder) buildStorageNodesEventual(sr common.Hash, watchedStorageKeys []common.Hash, intermediateNodes bool, output StorageNodeSink) error { +func (sdb *builder) buildStorageNodesEventual(sr common.Hash, watchedStorageKeys []common.Hash, intermediateNodes bool, output types2.StorageNodeSink) error { if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) { return nil } @@ -549,24 +551,24 @@ func (sdb *builder) buildStorageNodesEventual(sr common.Hash, watchedStorageKeys // buildStorageNodesFromTrie returns all the storage diff node objects in the provided node interator // if any storage keys are provided it will only return those leaf nodes // including intermediate nodes can be turned on or off -func (sdb *builder) buildStorageNodesFromTrie(it trie.NodeIterator, watchedStorageKeys []common.Hash, intermediateNodes bool, output StorageNodeSink) error { +func (sdb *builder) buildStorageNodesFromTrie(it trie.NodeIterator, watchedStorageKeys []common.Hash, intermediateNodes bool, output types2.StorageNodeSink) error { for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return err } switch node.NodeType { - case Leaf: + case types2.Leaf: partialPath := trie.CompactToHex(nodeElements[0].([]byte)) valueNodePath := append(node.Path, partialPath...) encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] if isWatchedStorageKey(watchedStorageKeys, leafKey) { - if err := output(StorageNode{ + if err := output(types2.StorageNode{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -575,9 +577,9 @@ func (sdb *builder) buildStorageNodesFromTrie(it trie.NodeIterator, watchedStora return err } } - case Extension, Branch: + case types2.Extension, types2.Branch: if intermediateNodes { - if err := output(StorageNode{ + if err := output(types2.StorageNode{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -593,7 +595,7 @@ func (sdb *builder) buildStorageNodesFromTrie(it trie.NodeIterator, watchedStora } // buildStorageNodesIncremental builds the storage diff node objects for all nodes that exist in a different state at B than A -func (sdb *builder) buildStorageNodesIncremental(oldSR common.Hash, newSR common.Hash, watchedStorageKeys []common.Hash, intermediateNodes bool, output StorageNodeSink) error { +func (sdb *builder) buildStorageNodesIncremental(oldSR common.Hash, newSR common.Hash, watchedStorageKeys []common.Hash, intermediateNodes bool, output types2.StorageNodeSink) error { if bytes.Equal(newSR.Bytes(), oldSR.Bytes()) { return nil } @@ -621,7 +623,7 @@ func (sdb *builder) buildStorageNodesIncremental(oldSR common.Hash, newSR common return nil } -func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys []common.Hash, intermediateNodes bool, output StorageNodeSink) (map[string]bool, error) { +func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys []common.Hash, intermediateNodes bool, output types2.StorageNodeSink) (map[string]bool, error) { diffPathsAtB := make(map[string]bool) it, _ := trie.NewDifferenceIterator(a, b) for it.Next(true) { @@ -629,18 +631,18 @@ func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return nil, err } switch node.NodeType { - case Leaf: + case types2.Leaf: partialPath := trie.CompactToHex(nodeElements[0].([]byte)) valueNodePath := append(node.Path, partialPath...) encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] if isWatchedStorageKey(watchedKeys, leafKey) { - if err := output(StorageNode{ + if err := output(types2.StorageNode{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -649,9 +651,9 @@ func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys return nil, err } } - case Extension, Branch: + case types2.Extension, types2.Branch: if intermediateNodes { - if err := output(StorageNode{ + if err := output(types2.StorageNode{ NodeType: node.NodeType, Path: node.Path, NodeValue: node.NodeValue, @@ -667,14 +669,14 @@ func (sdb *builder) createdAndUpdatedStorage(a, b trie.NodeIterator, watchedKeys return diffPathsAtB, it.Error() } -func (sdb *builder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffPathsAtB map[string]bool, watchedKeys []common.Hash, intermediateNodes bool, output StorageNodeSink) error { +func (sdb *builder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffPathsAtB map[string]bool, watchedKeys []common.Hash, intermediateNodes bool, output types2.StorageNodeSink) error { it, _ := trie.NewDifferenceIterator(b, a) for it.Next(true) { // skip value nodes if it.Leaf() || bytes.Equal(nullHashBytes, it.Hash().Bytes()) { continue } - node, nodeElements, err := sdtrie.ResolveNode(it, sdb.stateCache.TrieDB()) + node, nodeElements, err := trie_helpers.ResolveNode(it, sdb.stateCache.TrieDB()) if err != nil { return err } @@ -685,14 +687,14 @@ func (sdb *builder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffPathsAtB continue } switch node.NodeType { - case Leaf: + case types2.Leaf: partialPath := trie.CompactToHex(nodeElements[0].([]byte)) valueNodePath := append(node.Path, partialPath...) encodedPath := trie.HexToCompact(valueNodePath) leafKey := encodedPath[1:] if isWatchedStorageKey(watchedKeys, leafKey) { - if err := output(StorageNode{ - NodeType: Removed, + if err := output(types2.StorageNode{ + NodeType: types2.Removed, Path: node.Path, NodeValue: []byte{}, LeafKey: leafKey, @@ -700,10 +702,10 @@ func (sdb *builder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffPathsAtB return err } } - case Extension, Branch: + case types2.Extension, types2.Branch: if intermediateNodes { - if err := output(StorageNode{ - NodeType: Removed, + if err := output(types2.StorageNode{ + NodeType: types2.Removed, Path: node.Path, NodeValue: []byte{}, }); err != nil { diff --git a/statediff/builder_test.go b/statediff/builder_test.go index 6a88bbba0..d4d67940e 100644 --- a/statediff/builder_test.go +++ b/statediff/builder_test.go @@ -24,23 +24,24 @@ import ( "sort" "testing" + types2 "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/statediff" - "github.com/ethereum/go-ethereum/statediff/testhelpers" - sdtypes "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/statediff/test_helpers" ) var ( contractLeafKey []byte - emptyDiffs = make([]sdtypes.StateNode, 0) - emptyStorage = make([]sdtypes.StorageNode, 0) + emptyDiffs = make([]types2.StateNode, 0) + emptyStorage = make([]types2.StorageNode, 0) block0, block1, block2, block3, block4, block5, block6 *types.Block builder statediff.Builder minerAddress = common.HexToAddress("0x0") - minerLeafKey = testhelpers.AddressToLeafKey(minerAddress) + minerLeafKey = test_helpers.AddressToLeafKey(minerAddress) slot0 = common.HexToHash("0") slot1 = common.HexToHash("1") @@ -122,8 +123,8 @@ var ( minerAccountAtBlock1, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(2000002625000000000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) minerAccountAtBlock1LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3380c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312a"), @@ -132,8 +133,8 @@ var ( minerAccountAtBlock2, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(4000111203461610525), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) minerAccountAtBlock2LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3380c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312a"), @@ -142,9 +143,9 @@ var ( account1AtBlock1, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, - Balance: testhelpers.Block1Account1Balance, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + Balance: test_helpers.Block1Account1Balance, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account1AtBlock1LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3926db69aaced518e9b9f0f434a473e7174109c943548bb8f23be41ca76d9ad2"), @@ -153,8 +154,8 @@ var ( account1AtBlock2, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 2, Balance: big.NewInt(999555797000009000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account1AtBlock2LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3926db69aaced518e9b9f0f434a473e7174109c943548bb8f23be41ca76d9ad2"), @@ -163,8 +164,8 @@ var ( account1AtBlock5, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 2, Balance: big.NewInt(2999566008847709960), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account1AtBlock5LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3926db69aaced518e9b9f0f434a473e7174109c943548bb8f23be41ca76d9ad2"), @@ -173,8 +174,8 @@ var ( account1AtBlock6, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 3, Balance: big.NewInt(2999537516847709960), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account1AtBlock6LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3926db69aaced518e9b9f0f434a473e7174109c943548bb8f23be41ca76d9ad2"), @@ -184,8 +185,8 @@ var ( account2AtBlock2, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(1000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account2AtBlock2LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3957f3e2f04a0764c3a0491b175f69926da61efbcc8f61fa1455fd2d2b4cdd45"), @@ -194,8 +195,8 @@ var ( account2AtBlock3, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(2000013574009435976), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account2AtBlock3LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3957f3e2f04a0764c3a0491b175f69926da61efbcc8f61fa1455fd2d2b4cdd45"), @@ -204,8 +205,8 @@ var ( account2AtBlock4, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(4000048088163070348), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account2AtBlock4LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3957f3e2f04a0764c3a0491b175f69926da61efbcc8f61fa1455fd2d2b4cdd45"), @@ -214,8 +215,8 @@ var ( account2AtBlock6, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(6000063293259748636), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) account2AtBlock6LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3957f3e2f04a0764c3a0491b175f69926da61efbcc8f61fa1455fd2d2b4cdd45"), @@ -224,33 +225,33 @@ var ( bankAccountAtBlock0, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, - Balance: big.NewInt(testhelpers.TestBankFunds.Int64()), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + Balance: big.NewInt(test_helpers.TestBankFunds.Int64()), + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock0LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("2000bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), bankAccountAtBlock0, }) - block1BankBalance = big.NewInt(testhelpers.TestBankFunds.Int64() - testhelpers.BalanceChange10000 - testhelpers.GasFees) + block1BankBalance = big.NewInt(test_helpers.TestBankFunds.Int64() - test_helpers.BalanceChange10000 - test_helpers.GasFees) bankAccountAtBlock1, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 1, Balance: block1BankBalance, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock1LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), bankAccountAtBlock1, }) - block2BankBalance = block1BankBalance.Int64() - testhelpers.BalanceChange1Ether - testhelpers.GasFees + block2BankBalance = block1BankBalance.Int64() - test_helpers.BalanceChange1Ether - test_helpers.GasFees bankAccountAtBlock2, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 2, Balance: big.NewInt(block2BankBalance), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock2LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -259,8 +260,8 @@ var ( bankAccountAtBlock3, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 3, Balance: big.NewInt(999914255999990000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock3LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -269,8 +270,8 @@ var ( bankAccountAtBlock4, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 6, Balance: big.NewInt(999826859999990000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock4LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -279,8 +280,8 @@ var ( bankAccountAtBlock5, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 7, Balance: big.NewInt(999805027999990000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock5LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -469,10 +470,10 @@ func init() { } func TestBuilder(t *testing.T) { - blocks, chain := testhelpers.MakeChain(3, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] block3 = blocks[2] @@ -482,7 +483,7 @@ func TestBuilder(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testEmptyDiff", @@ -492,7 +493,7 @@ func TestBuilder(t *testing.T) { BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -502,19 +503,19 @@ func TestBuilder(t *testing.T) { "testBlock0", //10000 transferred from testBankAddress to account1Addr statediff.Args{ - OldStateRoot: testhelpers.NullHash, + OldStateRoot: test_helpers.NullHash, NewStateRoot: block0.Root(), BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock0LeafNode, StorageNodes: emptyStorage, }, @@ -530,28 +531,28 @@ func TestBuilder(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock1LeafNode, StorageNodes: emptyStorage, }, @@ -569,46 +570,46 @@ func TestBuilder(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock2LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, @@ -616,16 +617,16 @@ func TestBuilder(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock2LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -640,26 +641,26 @@ func TestBuilder(t *testing.T) { BlockNumber: block3.Number(), BlockHash: block3.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock3LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock3LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot3StorageKey.Bytes(), NodeValue: slot3StorageLeafNode, }, @@ -667,8 +668,8 @@ func TestBuilder(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock3LeafNode, StorageNodes: emptyStorage, }, @@ -700,10 +701,10 @@ func TestBuilder(t *testing.T) { } func TestBuilderWithIntermediateNodes(t *testing.T) { - blocks, chain := testhelpers.MakeChain(3, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] block3 = blocks[2] @@ -717,7 +718,7 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testEmptyDiff", @@ -727,7 +728,7 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -737,19 +738,19 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { "testBlock0", //10000 transferred from testBankAddress to account1Addr statediff.Args{ - OldStateRoot: testhelpers.NullHash, + OldStateRoot: test_helpers.NullHash, NewStateRoot: block0.Root(), BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock0LeafNode, StorageNodes: emptyStorage, }, @@ -765,34 +766,34 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block1BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock1LeafNode, StorageNodes: emptyStorage, }, @@ -810,57 +811,57 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block2BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock2LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block2StorageBranchRootNode, }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, @@ -868,16 +869,16 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock2LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -892,37 +893,37 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { BlockNumber: block3.Number(), BlockHash: block3.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block3BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock3LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock3LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block3StorageBranchRootNode, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot3StorageKey.Bytes(), NodeValue: slot3StorageLeafNode, }, @@ -930,8 +931,8 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock3LeafNode, StorageNodes: emptyStorage, }, @@ -977,22 +978,22 @@ func TestBuilderWithIntermediateNodes(t *testing.T) { } func TestBuilderWithWatchedAddressList(t *testing.T) { - blocks, chain := testhelpers.MakeChain(3, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] block3 = blocks[2] params := statediff.Params{ - WatchedAddresses: []common.Address{testhelpers.Account1Addr, testhelpers.ContractAddr}, + WatchedAddresses: []common.Address{test_helpers.Account1Addr, test_helpers.ContractAddr}, } builder = statediff.NewBuilder(chain.StateCache()) var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testEmptyDiff", @@ -1002,7 +1003,7 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -1012,12 +1013,12 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { "testBlock0", //10000 transferred from testBankAddress to account1Addr statediff.Args{ - OldStateRoot: testhelpers.NullHash, + OldStateRoot: test_helpers.NullHash, NewStateRoot: block0.Root(), BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -1032,14 +1033,14 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock1LeafNode, StorageNodes: emptyStorage, }, @@ -1056,25 +1057,25 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock2LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, @@ -1082,16 +1083,16 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -1106,19 +1107,19 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { BlockNumber: block3.Number(), BlockHash: block3.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock3LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot3StorageKey.Bytes(), NodeValue: slot3StorageLeafNode, }, @@ -1152,15 +1153,15 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { } func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { - blocks, chain := testhelpers.MakeChain(3, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] block3 = blocks[2] params := statediff.Params{ - WatchedAddresses: []common.Address{testhelpers.Account1Addr, testhelpers.ContractAddr}, + WatchedAddresses: []common.Address{test_helpers.Account1Addr, test_helpers.ContractAddr}, WatchedStorageSlots: []common.Hash{slot1StorageKey}, } builder = statediff.NewBuilder(chain.StateCache()) @@ -1168,7 +1169,7 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testEmptyDiff", @@ -1178,7 +1179,7 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -1188,12 +1189,12 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { "testBlock0", //10000 transferred from testBankAddress to account1Addr statediff.Args{ - OldStateRoot: testhelpers.NullHash, + OldStateRoot: test_helpers.NullHash, NewStateRoot: block0.Root(), BlockNumber: block0.Number(), BlockHash: block0.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block0.Number(), BlockHash: block0.Hash(), Nodes: emptyDiffs, @@ -1208,14 +1209,14 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock1LeafNode, StorageNodes: emptyStorage, }, @@ -1232,19 +1233,19 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock2LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, @@ -1252,16 +1253,16 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -1276,13 +1277,13 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { BlockNumber: block3.Number(), BlockHash: block3.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock3LeafNode, StorageNodes: emptyStorage, @@ -1315,8 +1316,8 @@ func TestBuilderWithWatchedAddressAndStorageKeyList(t *testing.T) { } func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { - blocks, chain := testhelpers.MakeChain(6, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(6, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() block3 = blocks[2] block4 = blocks[3] @@ -1331,7 +1332,7 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ // blocks 0-3 are the same as in TestBuilderWithIntermediateNodes { @@ -1342,49 +1343,49 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { BlockNumber: block4.Number(), BlockHash: block4.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block4.Number(), BlockHash: block4.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block4BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock4LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock4LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block4StorageBranchRootNode, }, { Path: []byte{'\x04'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot2StorageKey.Bytes(), NodeValue: slot2StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot1StorageKey.Bytes(), NodeValue: []byte{}, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot3StorageKey.Bytes(), NodeValue: []byte{}, }, @@ -1392,8 +1393,8 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock4LeafNode, StorageNodes: emptyStorage, }, @@ -1408,44 +1409,44 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { BlockNumber: block5.Number(), BlockHash: block5.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block5.Number(), BlockHash: block5.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block5BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock5LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock5LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, NodeValue: slot0StorageLeafRootNode, LeafKey: slot0StorageKey.Bytes(), }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot0StorageKey.Bytes(), NodeValue: []byte{}, }, { Path: []byte{'\x04'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot2StorageKey.Bytes(), NodeValue: []byte{}, }, @@ -1453,8 +1454,8 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock5LeafNode, StorageNodes: emptyStorage, }, @@ -1469,34 +1470,34 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { BlockNumber: block6.Number(), BlockHash: block6.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block6.Number(), BlockHash: block6.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block6BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: contractLeafKey, NodeValue: []byte{}, StorageNodes: emptyStorage, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock6LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock6LeafNode, StorageNodes: emptyStorage, }, @@ -1528,8 +1529,8 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { } func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing.T) { - blocks, chain := testhelpers.MakeChain(6, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(6, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() block3 = blocks[2] block4 = blocks[3] @@ -1544,7 +1545,7 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ // blocks 0-3 are the same as in TestBuilderWithIntermediateNodes { @@ -1555,38 +1556,38 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. BlockNumber: block4.Number(), BlockHash: block4.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block4.Number(), BlockHash: block4.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock4LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock4LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x04'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot2StorageKey.Bytes(), NodeValue: slot2StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot1StorageKey.Bytes(), NodeValue: []byte{}, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot3StorageKey.Bytes(), NodeValue: []byte{}, }, @@ -1594,8 +1595,8 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock4LeafNode, StorageNodes: emptyStorage, }, @@ -1610,38 +1611,38 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. BlockNumber: block5.Number(), BlockHash: block5.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block5.Number(), BlockHash: block5.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock5LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock5LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafRootNode, }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot0StorageKey.Bytes(), NodeValue: []byte{}, }, { Path: []byte{'\x04'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: slot2StorageKey.Bytes(), NodeValue: []byte{}, }, @@ -1649,8 +1650,8 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock5LeafNode, StorageNodes: emptyStorage, }, @@ -1665,27 +1666,27 @@ func TestBuilderWithRemovedAccountAndStorageWithoutIntermediateNodes(t *testing. BlockNumber: block6.Number(), BlockHash: block6.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block6.Number(), BlockHash: block6.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x06'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: contractLeafKey, NodeValue: []byte{}, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock6LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock6LeafNode, StorageNodes: emptyStorage, }, @@ -1740,8 +1741,8 @@ var ( bankAccountAtBlock01, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 1, Balance: big.NewInt(3999629697375000000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock01LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("30bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -1750,8 +1751,8 @@ var ( bankAccountAtBlock02, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 2, Balance: big.NewInt(5999607323457344852), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) bankAccountAtBlock02LeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("2000bf49f440a1cd0527e4d06e2765654c0f56452257516d793a9b8d604dcfdf2a"), @@ -1800,10 +1801,10 @@ var ( ) func TestBuilderWithMovedAccount(t *testing.T) { - blocks, chain := testhelpers.MakeChain(2, testhelpers.Genesis, testhelpers.TestSelfDestructChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(2, test_helpers.Genesis, test_helpers.TestSelfDestructChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] params := statediff.Params{ @@ -1815,7 +1816,7 @@ func TestBuilderWithMovedAccount(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testBlock1", @@ -1825,53 +1826,53 @@ func TestBuilderWithMovedAccount(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block01BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock01LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x01'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock01LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block01StorageBranchRootNode, }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot00StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, }, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -1884,27 +1885,27 @@ func TestBuilderWithMovedAccount(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock02LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x01'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: contractLeafKey, NodeValue: []byte{}, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Removed, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Removed, + LeafKey: test_helpers.BankLeafKey, NodeValue: []byte{}, }, }, @@ -1936,10 +1937,10 @@ func TestBuilderWithMovedAccount(t *testing.T) { } func TestBuilderWithMovedAccountOnlyLeafs(t *testing.T) { - blocks, chain := testhelpers.MakeChain(2, testhelpers.Genesis, testhelpers.TestSelfDestructChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(2, test_helpers.Genesis, test_helpers.TestSelfDestructChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] block2 = blocks[1] params := statediff.Params{ @@ -1951,7 +1952,7 @@ func TestBuilderWithMovedAccountOnlyLeafs(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *types2.StateObject }{ { "testBlock1", @@ -1961,42 +1962,42 @@ func TestBuilderWithMovedAccountOnlyLeafs(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock01LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x01'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock01LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot00StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, }, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -2009,27 +2010,27 @@ func TestBuilderWithMovedAccountOnlyLeafs(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock02LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x01'}, - NodeType: sdtypes.Removed, + NodeType: types2.Removed, LeafKey: contractLeafKey, NodeValue: []byte{}, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Removed, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Removed, + LeafKey: test_helpers.BankLeafKey, NodeValue: []byte{}, }, }, @@ -2060,8 +2061,8 @@ func TestBuilderWithMovedAccountOnlyLeafs(t *testing.T) { } func TestBuildStateTrie(t *testing.T) { - blocks, chain := testhelpers.MakeChain(3, testhelpers.Genesis, testhelpers.TestChainGen) - contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) defer chain.Stop() block1 = blocks[0] block2 = blocks[1] @@ -2071,39 +2072,39 @@ func TestBuildStateTrie(t *testing.T) { var tests = []struct { name string block *types.Block - expected *statediff.StateObject + expected *types2.StateObject }{ { "testBlock1", block1, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block1BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock1LeafNode, StorageNodes: emptyStorage, }, @@ -2113,57 +2114,57 @@ func TestBuildStateTrie(t *testing.T) { { "testBlock2", block2, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block2BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock2LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block2StorageBranchRootNode, }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, @@ -2171,16 +2172,16 @@ func TestBuildStateTrie(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock2LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, @@ -2188,63 +2189,63 @@ func TestBuildStateTrie(t *testing.T) { { "testBlock3", block3, - &statediff.StateObject{ + &types2.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), - Nodes: []sdtypes.StateNode{ + Nodes: []types2.StateNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block3BranchRootNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountAtBlock3LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x05'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: minerLeafKey, NodeValue: minerAccountAtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x0e'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1AtBlock2LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x06'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: contractLeafKey, NodeValue: contractAccountAtBlock3LeafNode, - StorageNodes: []sdtypes.StorageNode{ + StorageNodes: []types2.StorageNode{ { Path: []byte{}, - NodeType: sdtypes.Branch, + NodeType: types2.Branch, NodeValue: block3StorageBranchRootNode, }, { Path: []byte{'\x02'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot0StorageKey.Bytes(), NodeValue: slot0StorageLeafNode, }, { Path: []byte{'\x0b'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot1StorageKey.Bytes(), NodeValue: slot1StorageLeafNode, }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, + NodeType: types2.Leaf, LeafKey: slot3StorageKey.Bytes(), NodeValue: slot3StorageLeafNode, }, @@ -2252,16 +2253,16 @@ func TestBuildStateTrie(t *testing.T) { }, { Path: []byte{'\x0c'}, - NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account2LeafKey, + NodeType: types2.Leaf, + LeafKey: test_helpers.Account2LeafKey, NodeValue: account2AtBlock3LeafNode, StorageNodes: emptyStorage, }, }, - CodeAndCodeHashes: []sdtypes.CodeAndCodeHash{ + CodeAndCodeHashes: []types2.CodeAndCodeHash{ { - Hash: testhelpers.CodeHash, - Code: testhelpers.ByteCodeAfterDeployment, + Hash: test_helpers.CodeHash, + Code: test_helpers.ByteCodeAfterDeployment, }, }, }, diff --git a/statediff/config.go b/statediff/config.go new file mode 100644 index 000000000..dc9da579b --- /dev/null +++ b/statediff/config.go @@ -0,0 +1,58 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package statediff + +import ( + "context" + "math/big" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" +) + +// Config contains instantiation parameters for the state diffing service +type Config struct { + IndexerConfig interfaces.Config + // A unique ID used for this service + ID string + // Name for the client this service is running + ClientName string + // Whether to enable writing state diffs directly to track blockchain head + EnableWriteLoop bool + // Size of the worker pool + NumWorkers uint + // Context + Context context.Context +} + +// Params contains config parameters for the state diff builder +type Params struct { + IntermediateStateNodes bool + IntermediateStorageNodes bool + IncludeBlock bool + IncludeReceipts bool + IncludeTD bool + IncludeCode bool + WatchedAddresses []common.Address + WatchedStorageSlots []common.Hash +} + +// Args bundles the arguments for the state diff builder +type Args struct { + OldStateRoot, NewStateRoot, BlockHash common.Hash + BlockNumber *big.Int +} diff --git a/statediff/indexer/constructor.go b/statediff/indexer/constructor.go new file mode 100644 index 000000000..7a44638d0 --- /dev/null +++ b/statediff/indexer/constructor.go @@ -0,0 +1,66 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package indexer + +import ( + "context" + "fmt" + + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/node" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" +) + +// NewStateDiffIndexer creates and returns an implementation of the StateDiffIndexer interface +func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, nodeInfo node.Info, config interfaces.Config) (interfaces.StateDiffIndexer, error) { + switch config.Type() { + case shared.POSTGRES: + pgc, ok := config.(postgres.Config) + if !ok { + return nil, fmt.Errorf("ostgres config is not the correct type: got %T, expected %T", config, postgres.Config{}) + } + var err error + var driver sql.Driver + switch pgc.Driver { + case postgres.PGX: + driver, err = postgres.NewPGXDriver(ctx, pgc, nodeInfo) + if err != nil { + return nil, err + } + case postgres.SQLX: + driver, err = postgres.NewSQLXDriver(ctx, pgc, nodeInfo) + if err != nil { + return nil, err + } + default: + return nil, fmt.Errorf("unrecongized Postgres driver type: %s", pgc.Driver) + } + return sql.NewStateDiffIndexer(ctx, chainConfig, postgres.NewPostgresDB(driver)) + case shared.DUMP: + dumpc, ok := config.(dump.Config) + if !ok { + return nil, fmt.Errorf("dump config is not the correct type: got %T, expected %T", config, dump.Config{}) + } + return dump.NewStateDiffIndexer(chainConfig, dumpc), nil + default: + return nil, fmt.Errorf("unrecognized database type: %s", config.Type()) + } +} diff --git a/statediff/indexer/batch_tx.go b/statediff/indexer/database/dump/batch_tx.go similarity index 67% rename from statediff/indexer/batch_tx.go rename to statediff/indexer/database/dump/batch_tx.go index 170d6ab51..a0021baf7 100644 --- a/statediff/indexer/batch_tx.go +++ b/statediff/indexer/database/dump/batch_tx.go @@ -14,36 +14,37 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package indexer +package dump import ( + "fmt" + "io" + + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + + "github.com/ethereum/go-ethereum/statediff/indexer/models" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" node "github.com/ipfs/go-ipld-format" - "github.com/jmoiron/sqlx" - "github.com/lib/pq" - - "github.com/ethereum/go-ethereum/statediff/indexer/ipfs/ipld" - "github.com/ethereum/go-ethereum/statediff/indexer/models" ) -const ipldBatchInsertPgStr string = `INSERT INTO public.blocks (key, data) VALUES (unnest($1::TEXT[]), unnest($2::BYTEA[])) ON CONFLICT (key) DO NOTHING` - -// BlockTx wraps a Postgres tx with the state necessary for building the Postgres tx concurrently during trie difference iteration -type BlockTx struct { - dbtx *sqlx.Tx - BlockNumber uint64 - headerID int64 - Close func(blockTx *BlockTx, err error) error - +// BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration +type BatchTx struct { + dump io.Writer quit chan struct{} iplds chan models.IPLDModel ipldCache models.IPLDBatch + + close func(blockTx *BatchTx, err error) error } -func (tx *BlockTx) flush() error { - _, err := tx.dbtx.Exec(ipldBatchInsertPgStr, pq.Array(tx.ipldCache.Keys), pq.Array(tx.ipldCache.Values)) - if err != nil { +// Submit satisfies indexer.AtomicTx +func (tx *BatchTx) Submit(err error) error { + return tx.close(tx, err) +} + +func (tx *BatchTx) flush() error { + if _, err := fmt.Fprintf(tx.dump, "%+v", tx.ipldCache); err != nil { return err } tx.ipldCache = models.IPLDBatch{} @@ -51,33 +52,34 @@ func (tx *BlockTx) flush() error { } // run in background goroutine to synchronize concurrent appends to the ipldCache -func (tx *BlockTx) cache() { +func (tx *BatchTx) cache() { for { select { case i := <-tx.iplds: tx.ipldCache.Keys = append(tx.ipldCache.Keys, i.Key) tx.ipldCache.Values = append(tx.ipldCache.Values, i.Data) case <-tx.quit: + tx.ipldCache = models.IPLDBatch{} return } } } -func (tx *BlockTx) cacheDirect(key string, value []byte) { +func (tx *BatchTx) cacheDirect(key string, value []byte) { tx.iplds <- models.IPLDModel{ Key: key, Data: value, } } -func (tx *BlockTx) cacheIPLD(i node.Node) { +func (tx *BatchTx) cacheIPLD(i node.Node) { tx.iplds <- models.IPLDModel{ Key: blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(i.Cid().Hash()).String(), Data: i.RawData(), } } -func (tx *BlockTx) cacheRaw(codec, mh uint64, raw []byte) (string, string, error) { +func (tx *BatchTx) cacheRaw(codec, mh uint64, raw []byte) (string, string, error) { c, err := ipld.RawdataToCid(codec, raw, mh) if err != nil { return "", "", err diff --git a/statediff/indexer/database/dump/config.go b/statediff/indexer/database/dump/config.go new file mode 100644 index 000000000..fb2e6a58c --- /dev/null +++ b/statediff/indexer/database/dump/config.go @@ -0,0 +1,31 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package dump + +import ( + "io" + + "github.com/ethereum/go-ethereum/statediff/indexer/shared" +) + +type Config struct { + Dump io.WriteCloser +} + +func (c Config) Type() shared.DBType { + return shared.DUMP +} diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go new file mode 100644 index 000000000..f815305b1 --- /dev/null +++ b/statediff/indexer/database/dump/indexer.go @@ -0,0 +1,490 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package dump + +import ( + "fmt" + "io" + "math/big" + "time" + + ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + + "github.com/ipfs/go-cid" + node "github.com/ipfs/go-ipld-format" + "github.com/multiformats/go-multihash" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" + sdtypes "github.com/ethereum/go-ethereum/statediff/types" +) + +var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} + +var ( + indexerMetrics = RegisterIndexerMetrics(metrics.DefaultRegistry) +) + +// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of a void +type StateDiffIndexer struct { + dump io.WriteCloser + chainConfig *params.ChainConfig +} + +// NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer +func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config) *StateDiffIndexer { + return &StateDiffIndexer{ + dump: config.Dump, + chainConfig: chainConfig, + } +} + +// ReportDBMetrics has nothing to report for dump +func (sdi *StateDiffIndexer) ReportDBMetrics(time.Duration, <-chan bool) {} + +// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts) +// Returns an initiated DB transaction which must be Closed via defer to commit or rollback +func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) { + start, t := time.Now(), time.Now() + blockHash := block.Hash() + blockHashStr := blockHash.String() + height := block.NumberU64() + traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHashStr) + transactions := block.Transactions() + // Derive any missing fields + if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, transactions); err != nil { + return nil, err + } + + // Generate the block iplds + headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := ipld2.FromBlockAndReceipts(block, receipts) + if err != nil { + return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) + } + + if len(txNodes) != len(rctNodes) || len(rctNodes) != len(rctLeafNodeCIDs) { + return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d), and receipt trie leaf nodes (%d) to be equal", len(txNodes), len(rctNodes), len(rctLeafNodeCIDs)) + } + if len(txTrieNodes) != len(rctTrieNodes) { + return nil, fmt.Errorf("expected number of tx trie (%d) and rct trie (%d) nodes to be equal", len(txTrieNodes), len(rctTrieNodes)) + } + + // Calculate reward + var reward *big.Int + // in PoA networks block reward is 0 + if sdi.chainConfig.Clique != nil { + reward = big.NewInt(0) + } else { + reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) + } + t = time.Now() + + blockTx := &BatchTx{ + dump: sdi.dump, + iplds: make(chan models.IPLDModel), + quit: make(chan struct{}), + ipldCache: models.IPLDBatch{}, + close: func(self *BatchTx, err error) error { + close(self.quit) + close(self.iplds) + tDiff := time.Since(t) + indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) + t = time.Now() + if err := self.flush(); err != nil { + traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) + log.Debug(traceMsg) + return err + } + tDiff = time.Since(t) + indexerMetrics.tPostgresCommit.Update(tDiff) + traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String()) + traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) + log.Debug(traceMsg) + return err + }, + } + go blockTx.cache() + + tDiff := time.Since(t) + indexerMetrics.tFreePostgres.Update(tDiff) + + traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String()) + t = time.Now() + + // Publish and index header, collect headerID + var headerID int64 + headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) + if err != nil { + return nil, err + } + tDiff = time.Since(t) + indexerMetrics.tHeaderProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) + t = time.Now() + // Publish and index uncles + err = sdi.processUncles(blockTx, headerID, height, uncleNodes) + if err != nil { + return nil, err + } + tDiff = time.Since(t) + indexerMetrics.tUncleProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String()) + t = time.Now() + // Publish and index receipts and txs + err = sdi.processReceiptsAndTxs(blockTx, processArgs{ + headerID: headerID, + blockNumber: block.Number(), + receipts: receipts, + txs: transactions, + rctNodes: rctNodes, + rctTrieNodes: rctTrieNodes, + txNodes: txNodes, + txTrieNodes: txTrieNodes, + logTrieNodes: logTrieNodes, + logLeafNodeCIDs: logLeafNodeCIDs, + rctLeafNodeCIDs: rctLeafNodeCIDs, + }) + if err != nil { + return nil, err + } + tDiff = time.Since(t) + indexerMetrics.tTxAndRecProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String()) + t = time.Now() + + return blockTx, err +} + +// processHeader publishes and indexes a header IPLD in Postgres +// it returns the headerID +func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { + tx.cacheIPLD(headerNode) + + var baseFee *int64 + if header.BaseFee != nil { + baseFee = new(int64) + *baseFee = header.BaseFee.Int64() + } + + mod := models.HeaderModel{ + CID: headerNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), + ParentHash: header.ParentHash.String(), + BlockNumber: header.Number.String(), + BlockHash: header.Hash().String(), + TotalDifficulty: td.String(), + Reward: reward.String(), + Bloom: header.Bloom.Bytes(), + StateRoot: header.Root.String(), + RctRoot: header.ReceiptHash.String(), + TxRoot: header.TxHash.String(), + UncleRoot: header.UncleHash.String(), + Timestamp: header.Time, + BaseFee: baseFee, + } + _, err := fmt.Fprintf(sdi.dump, "%+v", mod) + return 0, err +} + +// processUncles publishes and indexes uncle IPLDs in Postgres +func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { + // publish and index uncles + for _, uncleNode := range uncleNodes { + tx.cacheIPLD(uncleNode) + var uncleReward *big.Int + // in PoA networks uncle reward is 0 + if sdi.chainConfig.Clique != nil { + uncleReward = big.NewInt(0) + } else { + uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) + } + uncle := models.UncleModel{ + CID: uncleNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(uncleNode.Cid()), + ParentHash: uncleNode.ParentHash.String(), + BlockHash: uncleNode.Hash().String(), + Reward: uncleReward.String(), + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", uncle); err != nil { + return err + } + } + return nil +} + +// processArgs bundles arguments to processReceiptsAndTxs +type processArgs struct { + headerID int64 + blockNumber *big.Int + receipts types.Receipts + txs types.Transactions + rctNodes []*ipld2.EthReceipt + rctTrieNodes []*ipld2.EthRctTrie + txNodes []*ipld2.EthTx + txTrieNodes []*ipld2.EthTxTrie + logTrieNodes [][]*ipld2.EthLogTrie + logLeafNodeCIDs [][]cid.Cid + rctLeafNodeCIDs []cid.Cid +} + +// processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres +func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs) error { + // Process receipts and txs + signer := types.MakeSigner(sdi.chainConfig, args.blockNumber) + for i, receipt := range args.receipts { + for _, logTrieNode := range args.logTrieNodes[i] { + tx.cacheIPLD(logTrieNode) + } + txNode := args.txNodes[i] + tx.cacheIPLD(txNode) + + // Indexing + // extract topic and contract data from the receipt for indexing + mappedContracts := make(map[string]bool) // use map to avoid duplicate addresses + 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{ + 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], + } + } + // these are the contracts seen in the logs + logContracts := make([]string, 0, len(mappedContracts)) + for addr := range mappedContracts { + logContracts = append(logContracts, addr) + } + // this is the contract address if this receipt is for a contract creation tx + contract := shared.HandleZeroAddr(receipt.ContractAddress) + var contractHash string + if contract != "" { + contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() + } + // index tx first so that the receipt can reference it by FK + trx := args.txs[i] + // derive sender for the tx that corresponds with this receipt + from, err := types.Sender(signer, trx) + if err != nil { + return fmt.Errorf("error deriving tx sender: %v", err) + } + txModel := models.TxModel{ + Dst: shared.HandleZeroAddrPointer(trx.To()), + Src: shared.HandleZeroAddr(from), + TxHash: trx.Hash().String(), + Index: int64(i), + Data: trx.Data(), + CID: txNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(txNode.Cid()), + } + txType := trx.Type() + if txType != types.LegacyTxType { + txModel.Type = &txType + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", txModel); err != nil { + return err + } + + // index access list if this is one + for j, accessListElement := range trx.AccessList() { + storageKeys := make([]string, len(accessListElement.StorageKeys)) + for k, storageKey := range accessListElement.StorageKeys { + storageKeys[k] = storageKey.Hex() + } + accessListElementModel := models.AccessListElementModel{ + Index: int64(j), + Address: accessListElement.Address.Hex(), + StorageKeys: storageKeys, + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", accessListElementModel); err != nil { + return err + } + } + + // index the receipt + if !args.rctLeafNodeCIDs[i].Defined() { + return fmt.Errorf("invalid receipt leaf node cid") + } + + rctModel := &models.ReceiptModel{ + Contract: contract, + ContractHash: contractHash, + LeafCID: args.rctLeafNodeCIDs[i].String(), + LeafMhKey: shared.MultihashKeyFromCID(args.rctLeafNodeCIDs[i]), + LogRoot: args.rctNodes[i].LogRoot.String(), + } + if len(receipt.PostState) == 0 { + rctModel.PostStatus = receipt.Status + } else { + rctModel.PostState = common.Bytes2Hex(receipt.PostState) + } + + if _, err := fmt.Fprintf(sdi.dump, "%+v", rctModel); err != nil { + return err + } + + if _, err := fmt.Fprintf(sdi.dump, "%+v", logDataSet); err != nil { + return err + } + } + + // publish trie nodes, these aren't indexed directly + for i, n := range args.txTrieNodes { + tx.cacheIPLD(n) + tx.cacheIPLD(args.rctTrieNodes[i]) + } + + return nil +} + +// PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql +func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode) error { + tx, ok := batch.(*BatchTx) + if !ok { + return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } + // publish the state node + if stateNode.NodeType == sdtypes.Removed { + // short circuit if it is a Removed node + // this assumes the db has been initialized and a public.blocks entry for the Removed node is present + stateModel := models.StateNodeModel{ + Path: stateNode.Path, + StateKey: common.BytesToHash(stateNode.LeafKey).String(), + CID: shared.RemovedNodeStateCID, + MhKey: shared.RemovedNodeMhKey, + NodeType: stateNode.NodeType.Int(), + } + _, err := fmt.Fprintf(sdi.dump, "%+v", stateModel) + return err + } + stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) + if err != nil { + return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) + } + stateModel := models.StateNodeModel{ + Path: stateNode.Path, + StateKey: common.BytesToHash(stateNode.LeafKey).String(), + CID: stateCIDStr, + MhKey: stateMhKey, + NodeType: stateNode.NodeType.Int(), + } + // index the state node, collect the stateID to reference by FK + if _, err := fmt.Fprintf(sdi.dump, "%+v", stateModel); err != nil { + return err + } + // if we have a leaf, decode and index the account data + if stateNode.NodeType == sdtypes.Leaf { + var i []interface{} + if err := rlp.DecodeBytes(stateNode.NodeValue, &i); err != nil { + return fmt.Errorf("error decoding state leaf node rlp: %s", err.Error()) + } + if len(i) != 2 { + return fmt.Errorf("eth IPLDPublisher expected state leaf node rlp to decode into two elements") + } + var account types.StateAccount + if err := rlp.DecodeBytes(i[1].([]byte), &account); err != nil { + return fmt.Errorf("error decoding state account rlp: %s", err.Error()) + } + accountModel := models.StateAccountModel{ + Balance: account.Balance.String(), + Nonce: account.Nonce, + CodeHash: account.CodeHash, + StorageRoot: account.Root.String(), + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", accountModel); err != nil { + return err + } + } + // if there are any storage nodes associated with this node, publish and index them + for _, storageNode := range stateNode.StorageNodes { + if storageNode.NodeType == sdtypes.Removed { + // short circuit if it is a Removed node + // this assumes the db has been initialized and a public.blocks entry for the Removed node is present + storageModel := models.StorageNodeModel{ + Path: storageNode.Path, + StorageKey: common.BytesToHash(storageNode.LeafKey).String(), + CID: shared.RemovedNodeStorageCID, + MhKey: shared.RemovedNodeMhKey, + NodeType: storageNode.NodeType.Int(), + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", storageModel); err != nil { + return err + } + continue + } + storageCIDStr, storageMhKey, err := tx.cacheRaw(ipld2.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) + if err != nil { + return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) + } + storageModel := models.StorageNodeModel{ + Path: storageNode.Path, + StorageKey: common.BytesToHash(storageNode.LeafKey).String(), + CID: storageCIDStr, + MhKey: storageMhKey, + NodeType: storageNode.NodeType.Int(), + } + if _, err := fmt.Fprintf(sdi.dump, "%+v", storageModel); err != nil { + return err + } + } + + return nil +} + +// PushCodeAndCodeHash publishes code and codehash pairs to the ipld sql +func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAndCodeHash sdtypes.CodeAndCodeHash) error { + tx, ok := batch.(*BatchTx) + if !ok { + return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } + // codec doesn't matter since db key is multihash-based + mhKey, err := shared.MultihashKeyFromKeccak256(codeAndCodeHash.Hash) + if err != nil { + return fmt.Errorf("error deriving multihash key from codehash: %v", err) + } + tx.cacheDirect(mhKey, codeAndCodeHash.Code) + return nil +} + +// Close satisfied io.Closer +func (sdi *StateDiffIndexer) Close() error { + return sdi.dump.Close() +} diff --git a/statediff/indexer/database/dump/metrics.go b/statediff/indexer/database/dump/metrics.go new file mode 100644 index 000000000..700e42dc0 --- /dev/null +++ b/statediff/indexer/database/dump/metrics.go @@ -0,0 +1,94 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package dump + +import ( + "strings" + + "github.com/ethereum/go-ethereum/metrics" +) + +const ( + namespace = "statediff" +) + +// Build a fully qualified metric name +func metricName(subsystem, name string) string { + if name == "" { + return "" + } + parts := []string{namespace, name} + if subsystem != "" { + parts = []string{namespace, subsystem, name} + } + // Prometheus uses _ but geth metrics uses / and replaces + return strings.Join(parts, "/") +} + +type indexerMetricsHandles struct { + // The total number of processed blocks + blocks metrics.Counter + // The total number of processed transactions + transactions metrics.Counter + // The total number of processed receipts + receipts metrics.Counter + // The total number of processed logs + logs metrics.Counter + // The total number of access list entries processed + accessListEntries metrics.Counter + // Time spent waiting for free postgres tx + tFreePostgres metrics.Timer + // Postgres transaction commit duration + tPostgresCommit metrics.Timer + // Header processing time + tHeaderProcessing metrics.Timer + // Uncle processing time + tUncleProcessing metrics.Timer + // Tx and receipt processing time + tTxAndRecProcessing metrics.Timer + // State, storage, and code combined processing time + tStateStoreCodeProcessing metrics.Timer +} + +func RegisterIndexerMetrics(reg metrics.Registry) indexerMetricsHandles { + ctx := indexerMetricsHandles{ + blocks: metrics.NewCounter(), + transactions: metrics.NewCounter(), + receipts: metrics.NewCounter(), + logs: metrics.NewCounter(), + accessListEntries: metrics.NewCounter(), + tFreePostgres: metrics.NewTimer(), + tPostgresCommit: metrics.NewTimer(), + tHeaderProcessing: metrics.NewTimer(), + tUncleProcessing: metrics.NewTimer(), + tTxAndRecProcessing: metrics.NewTimer(), + tStateStoreCodeProcessing: metrics.NewTimer(), + } + subsys := "indexer" + reg.Register(metricName(subsys, "blocks"), ctx.blocks) + reg.Register(metricName(subsys, "transactions"), ctx.transactions) + reg.Register(metricName(subsys, "receipts"), ctx.receipts) + reg.Register(metricName(subsys, "logs"), ctx.logs) + reg.Register(metricName(subsys, "access_list_entries"), ctx.accessListEntries) + reg.Register(metricName(subsys, "t_free_postgres"), ctx.tFreePostgres) + reg.Register(metricName(subsys, "t_postgres_commit"), ctx.tPostgresCommit) + reg.Register(metricName(subsys, "t_header_processing"), ctx.tHeaderProcessing) + reg.Register(metricName(subsys, "t_uncle_processing"), ctx.tUncleProcessing) + reg.Register(metricName(subsys, "t_tx_receipt_processing"), ctx.tTxAndRecProcessing) + reg.Register(metricName(subsys, "t_state_store_code_processing"), ctx.tStateStoreCodeProcessing) + return ctx +} diff --git a/statediff/indexer/database/sql/batch_tx.go b/statediff/indexer/database/sql/batch_tx.go new file mode 100644 index 000000000..2041af1ed --- /dev/null +++ b/statediff/indexer/database/sql/batch_tx.go @@ -0,0 +1,106 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package sql + +import ( + "context" + + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + + blockstore "github.com/ipfs/go-ipfs-blockstore" + dshelp "github.com/ipfs/go-ipfs-ds-help" + node "github.com/ipfs/go-ipld-format" + "github.com/lib/pq" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/statediff/indexer/models" +) + +// BatchTx wraps a sql tx with the state necessary for building the tx concurrently during trie difference iteration +type BatchTx struct { + ctx context.Context + dbtx Tx + headerID int64 + stm string + quit chan struct{} + iplds chan models.IPLDModel + ipldCache models.IPLDBatch + + close func(blockTx *BatchTx, err error) error +} + +// Submit satisfies indexer.AtomicTx +func (tx *BatchTx) Submit(err error) error { + return tx.close(tx, err) +} + +func (tx *BatchTx) flush() error { + _, err := tx.dbtx.Exec(tx.ctx, tx.stm, pq.Array(tx.ipldCache.Keys), pq.Array(tx.ipldCache.Values)) + if err != nil { + return err + } + tx.ipldCache = models.IPLDBatch{} + return nil +} + +// run in background goroutine to synchronize concurrent appends to the ipldCache +func (tx *BatchTx) cache() { + for { + select { + case i := <-tx.iplds: + tx.ipldCache.Keys = append(tx.ipldCache.Keys, i.Key) + tx.ipldCache.Values = append(tx.ipldCache.Values, i.Data) + case <-tx.quit: + tx.ipldCache = models.IPLDBatch{} + return + } + } +} + +func (tx *BatchTx) cacheDirect(key string, value []byte) { + tx.iplds <- models.IPLDModel{ + Key: key, + Data: value, + } +} + +func (tx *BatchTx) cacheIPLD(i node.Node) { + tx.iplds <- models.IPLDModel{ + Key: blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(i.Cid().Hash()).String(), + Data: i.RawData(), + } +} + +func (tx *BatchTx) cacheRaw(codec, mh uint64, raw []byte) (string, string, error) { + c, err := ipld.RawdataToCid(codec, raw, mh) + if err != nil { + return "", "", err + } + prefixedKey := blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(c.Hash()).String() + tx.iplds <- models.IPLDModel{ + Key: prefixedKey, + Data: raw, + } + return c.String(), prefixedKey, err +} + +// rollback sql transaction and log any error +func rollback(ctx context.Context, tx Tx) { + if err := tx.Rollback(ctx); err != nil { + log.Error(err.Error()) + } +} diff --git a/statediff/indexer/database/sql/batch_writer.go b/statediff/indexer/database/sql/batch_writer.go new file mode 100644 index 000000000..05c882259 --- /dev/null +++ b/statediff/indexer/database/sql/batch_writer.go @@ -0,0 +1,216 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package sql + +/* +import ( + "fmt" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/jmoiron/sqlx" +) + +*/ +/* +// PG_MAX_PARAMS is the max number of placeholders+args a statement can support +// above this limit we need to split into a separate batch +const PG_MAX_PARAMS int = 32767 + +const ( + ipldInsertPgStr string = `INSERT INTO public.blocks (key, data) VALUES (unnest($1), unnest($2)) ON CONFLICT (key) DO NOTHING` + headerCIDsPgStr string = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) + VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9), unnest($10), unnest($11), unnest($12), unnest($13), unnest($14), unnest($15), unnest($16)) + ON CONFLICT (block_number, block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (excluded.parent_hash, excluded.cid, excluded.td, excluded.node_id, excluded.reward, excluded.state_root, excluded.tx_root, excluded.receipt_root, excluded.uncle_root, excluded.bloom, excluded.timestamp, excluded.mh_key, eth.header_cids.times_validated + 1, excluded.base_fee) + RETURNING id` + unclesCIDsPgStr string = `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6)) + ON CONFLICT (header_id, block_hash) DO UPDATE SET (parent_hash, cid, reward, mh_key) = (excluded.parent_hash, excluded.cid, excluded.reward, excluded.mh_key)` + txCIDsPgStr string = `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9)) + ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = (excluded.cid, excluded.dst, excluded.src, excluded.index, excluded.mh_key, excluded.tx_data, excluded.tx_type) + RETURNING id` + accessListPgStr string = `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES (unnest($1), unnest($2), unnest($3), unnest($4)) + ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = (excluded.address, excluded.storage_keys)` + rctCIDsPgStr string = `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8)) + ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = (excluded.leaf_cid, excluded.contract, excluded.contract_hash, excluded.leaf_mh_key, excluded.post_state, excluded.post_status, excluded.log_root) + RETURNING id` + logCIDsPgStr string = `INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, receipt_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9), unnest($10)) + ON CONFLICT (receipt_id, index) DO UPDATE SET (leaf_cid, leaf_mh_key, address, topic0, topic1, topic2, topic3, log_data) = (excluded.leaf_cid, excluded.leaf_mh_key, excluded.address, excluded.topic0, excluded.topic1, excluded.topic2, excluded.topic3, excluded.log_data)` + stateCIDsPgStr string = `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7)) + ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = (excluded.state_leaf_key, excluded.cid, excluded.node_type, excluded.diff, excluded.mh_key) + RETURNING id` + stateAccountsPgStr string = `INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5)) + ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = (excluded.balance, excluded.nonce, excluded.code_hash, excluded.storage_root)` + storageCIDsPgStr string = `INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7)) + ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = (excluded.storage_leaf_key, excluded.cid, excluded.node_type, excluded.diff, excluded.mh_key)` +) + +// PostgresBatchWriter is used to write statediff data to Postgres using batch inserts/upserts +type PostgresBatchWriter struct { + db *postgres.DB + + // prepared statements (prepared inside tx) + ipldsPreparedStm *sqlx.Stmt + unclesPrepared *sqlx.Stmt + txPreparedStm *sqlx.Stmt + accessListPreparedStm *sqlx.Stmt + rctPreparedStm *sqlx.Stmt + logPreparedStm *sqlx.Stmt + statePreparedStm *sqlx.Stmt + accountPreparedStm *sqlx.Stmt + storagePreparedStm *sqlx.Stmt + + // cached arguments + queuedHeaderArgs models.HeaderModel + queuedUnclesArgs models.UncleBatch + queuedTxArgs models.TxBatch + queuedAccessListArgs models.AccessListBatch + queuedRctArgs models.ReceiptBatch + queuedLogArgs models.LogBatch + queuedStateArgs models.StateBatch + queuedAccountArgs models.AccountBatch + queuedStorageArgs models.StorageBatch +} + +// NewPostgresBatchWriter creates a new pointer to a PostgresBatchWriter +func NewPostgresBatchWriter(db *postgres.DB) *PostgresBatchWriter { + return &PostgresBatchWriter{ + db: db, + } +} + +func (pbw *PostgresBatchWriter) queueHeader(header models.HeaderModel) { + pbw.queuedHeaderArgs = header +} + +func (pbw *PostgresBatchWriter) queueUncle(uncle models.UncleModel) { + pbw.queuedUnclesArgs.BlockHashes = append(pbw.queuedUnclesArgs.BlockHashes, uncle.BlockHash) + pbw.queuedUnclesArgs.ParentHashes = append(pbw.queuedUnclesArgs.ParentHashes, uncle.ParentHash) + pbw.queuedUnclesArgs.CIDs = append(pbw.queuedUnclesArgs.CIDs, uncle.CID) + pbw.queuedUnclesArgs.MhKeys = append(pbw.queuedUnclesArgs.MhKeys, uncle.MhKey) + pbw.queuedUnclesArgs.Rewards = append(pbw.queuedUnclesArgs.Rewards, uncle.Reward) +} + +func (pbw *PostgresBatchWriter) queueTransaction(tx models.TxModel) { + pbw.queuedTxArgs.Indexes = append(pbw.queuedTxArgs.Indexes, tx.Index) + pbw.queuedTxArgs.TxHashes = append(pbw.queuedTxArgs.TxHashes, tx.TxHash) + pbw.queuedTxArgs.CIDs = append(pbw.queuedTxArgs.CIDs, tx.CID) + pbw.queuedTxArgs.MhKeys = append(pbw.queuedTxArgs.MhKeys, tx.MhKey) + pbw.queuedTxArgs.Dsts = append(pbw.queuedTxArgs.Dsts, tx.Dst) + pbw.queuedTxArgs.Srcs = append(pbw.queuedTxArgs.Srcs, tx.Src) + pbw.queuedTxArgs.Datas = append(pbw.queuedTxArgs.Datas, tx.Data) + pbw.queuedTxArgs.Types = append(pbw.queuedTxArgs.Types, tx.Type) +} + +func (pbw *PostgresBatchWriter) queueAccessListElement(al models.AccessListElementModel) { + +} + +func (pbw *PostgresBatchWriter) queueReceipt(rct models.ReceiptModel) { + +} + +func (pbw *PostgresBatchWriter) upsertTransactionCID(tx *sqlx.Tx, transaction models.TxModel, headerID int64) (int64, error) { + var txID int64 + err := tx.QueryRowx(`INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) + ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = ($3, $4, $5, $6, $7, $8, $9) + RETURNING id`, + headerID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type).Scan(&txID) + if err != nil { + return 0, fmt.Errorf("error upserting transaction_cids entry: %v", err) + } + indexerMetrics.transactions.Inc(1) + return txID, nil +} + +func (pbw *PostgresBatchWriter) upsertAccessListElement(tx *sqlx.Tx, accessListElement models.AccessListElementModel, txID int64) error { + _, err := tx.Exec(`INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) + ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)`, + txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) + if err != nil { + return fmt.Errorf("error upserting access_list_element entry: %v", err) + } + indexerMetrics.accessListEntries.Inc(1) + return nil +} + +func (pbw *PostgresBatchWriter) upsertReceiptCID(tx *sqlx.Tx, rct *models.ReceiptModel, txID int64) (int64, error) { + var receiptID int64 + err := tx.QueryRowx(`INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) + ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8) + RETURNING id`, + txID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID) + if err != nil { + return 0, fmt.Errorf("error upserting receipt_cids entry: %w", err) + } + indexerMetrics.receipts.Inc(1) + return receiptID, nil +} + +func (pbw *PostgresBatchWriter) 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) + } + indexerMetrics.logs.Inc(1) + } + return nil +} + +func (pbw *PostgresBatchWriter) upsertStateCID(tx *sqlx.Tx, stateNode models.StateNodeModel, headerID int64) (int64, error) { + var stateID int64 + var stateKey string + if stateNode.StateKey != nullHash.String() { + stateKey = stateNode.StateKey + } + err := tx.QueryRowx(`INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) + ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) + RETURNING id`, + headerID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey).Scan(&stateID) + if err != nil { + return 0, fmt.Errorf("error upserting state_cids entry: %v", err) + } + return stateID, nil +} + +func (pbw *PostgresBatchWriter) upsertStateAccount(tx *sqlx.Tx, stateAccount models.StateAccountModel, stateID int64) error { + _, err := tx.Exec(`INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5) + ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = ($2, $3, $4, $5)`, + stateID, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) + if err != nil { + return fmt.Errorf("error upserting state_accounts entry: %v", err) + } + return nil +} + +func (pbw *PostgresBatchWriter) upsertStorageCID(tx *sqlx.Tx, storageCID models.StorageNodeModel, stateID int64) error { + var storageKey string + if storageCID.StorageKey != nullHash.String() { + storageKey = storageCID.StorageKey + } + _, err := tx.Exec(`INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) + ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)`, + stateID, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) + if err != nil { + return fmt.Errorf("error upserting storage_cids entry: %v", err) + } + return nil +} +*/ diff --git a/statediff/indexer/indexer.go b/statediff/indexer/database/sql/indexer.go similarity index 83% rename from statediff/indexer/indexer.go rename to statediff/indexer/database/sql/indexer.go index f8ffee429..6c35cccac 100644 --- a/statediff/indexer/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -14,15 +14,19 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -// Package indexer provides an interface for pushing and indexing IPLD objects into a Postgres database +// Package sql provides an interface for pushing and indexing IPLD objects into a sql database // Metrics for reporting processing and connection stats are defined in ./metrics.go -package indexer + +package sql import ( + "context" "fmt" "math/big" "time" + ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ipfs/go-cid" node "github.com/ipfs/go-ipld-format" "github.com/multiformats/go-multihash" @@ -34,47 +38,36 @@ import ( "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/rlp" - "github.com/ethereum/go-ethereum/statediff/indexer/ipfs/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/models" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/shared" sdtypes "github.com/ethereum/go-ethereum/statediff/types" ) +var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} + var ( indexerMetrics = RegisterIndexerMetrics(metrics.DefaultRegistry) dbMetrics = RegisterDBMetrics(metrics.DefaultRegistry) ) -const ( - RemovedNodeStorageCID = "bagmacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya" - RemovedNodeStateCID = "baglacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya" - RemovedNodeMhKey = "/blocks/DMQMLUSGAGDPOIZ4SJ7H3MW4Y4B4BZIAWZJ4VARHHN57VWAELWC2I4A" -) - -// Indexer interface to allow substitution of mocks for testing -type Indexer interface { - PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (*BlockTx, error) - PushStateNode(tx *BlockTx, stateNode sdtypes.StateNode) error - PushCodeAndCodeHash(tx *BlockTx, codeAndCodeHash sdtypes.CodeAndCodeHash) error - ReportDBMetrics(delay time.Duration, quit <-chan bool) -} - -// StateDiffIndexer satisfies the Indexer interface for ethereum statediff objects +// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of an SQL sql type StateDiffIndexer struct { + ctx context.Context chainConfig *params.ChainConfig - dbWriter *PostgresCIDWriter + dbWriter *Writer } -// NewStateDiffIndexer creates a pointer to a new PayloadConverter which satisfies the PayloadConverter interface -func NewStateDiffIndexer(chainConfig *params.ChainConfig, db *postgres.DB) (*StateDiffIndexer, error) { +// NewStateDiffIndexer creates a sql implementation of interfaces.StateDiffIndexer +func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, db Database) (*StateDiffIndexer, error) { // Write the removed node to the db on init - if err := shared.PublishDirectWithDB(db, RemovedNodeMhKey, []byte{}); err != nil { + if _, err := db.Exec(ctx, db.InsertIPLDStm(), shared.RemovedNodeMhKey, []byte{}); err != nil { return nil, err } return &StateDiffIndexer{ + ctx: ctx, chainConfig: chainConfig, - dbWriter: NewPostgresCIDWriter(db), + dbWriter: NewWriter(db), }, nil } @@ -97,9 +90,9 @@ func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bo }() } -// PushBlock pushes and indexes block data in database, except state & storage nodes (includes header, uncles, transactions & receipts) +// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts) // Returns an initiated DB transaction which must be Closed via defer to commit or rollback -func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (*BlockTx, error) { +func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) { start, t := time.Now(), time.Now() blockHash := block.Hash() blockHashStr := blockHash.String() @@ -112,7 +105,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } // Generate the block iplds - headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := ipld.FromBlockAndReceipts(block, receipts) + headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := ipld2.FromBlockAndReceipts(block, receipts) if err != nil { return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) } @@ -130,49 +123,50 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip if sdi.chainConfig.Clique != nil { reward = big.NewInt(0) } else { - reward = CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) + reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) } t = time.Now() // Begin new db tx for everything - tx, err := sdi.dbWriter.db.Beginx() + tx, err := sdi.dbWriter.db.Begin(sdi.ctx) if err != nil { return nil, err } defer func() { if p := recover(); p != nil { - shared.Rollback(tx) + rollback(sdi.ctx, tx) panic(p) } else if err != nil { - shared.Rollback(tx) + rollback(sdi.ctx, tx) } }() - blockTx := &BlockTx{ + blockTx := &BatchTx{ + stm: sdi.dbWriter.db.InsertIPLDsStm(), iplds: make(chan models.IPLDModel), quit: make(chan struct{}), ipldCache: models.IPLDBatch{}, dbtx: tx, // handle transaction commit or rollback for any return case - Close: func(self *BlockTx, err error) error { + close: func(self *BatchTx, err error) error { close(self.quit) close(self.iplds) if p := recover(); p != nil { - shared.Rollback(tx) + rollback(sdi.ctx, tx) panic(p) } else if err != nil { - shared.Rollback(tx) + rollback(sdi.ctx, tx) } else { tDiff := time.Since(t) indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) t = time.Now() if err := self.flush(); err != nil { - shared.Rollback(tx) + rollback(sdi.ctx, tx) traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) log.Debug(traceMsg) return err } - err = tx.Commit() + err = tx.Commit(sdi.ctx) tDiff = time.Since(t) indexerMetrics.tPostgresCommit.Update(tDiff) traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String()) @@ -231,14 +225,13 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String()) t = time.Now() - blockTx.BlockNumber = height blockTx.headerID = headerID return blockTx, err } // processHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *BlockTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { +func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { tx.cacheIPLD(headerNode) var baseFee *int64 @@ -267,7 +260,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BlockTx, header *types.Header, he } // processUncles publishes and indexes uncle IPLDs in Postgres -func (sdi *StateDiffIndexer) processUncles(tx *BlockTx, headerID int64, blockNumber uint64, uncleNodes []*ipld.EthHeader) error { +func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { // publish and index uncles for _, uncleNode := range uncleNodes { tx.cacheIPLD(uncleNode) @@ -276,7 +269,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BlockTx, headerID int64, blockNum if sdi.chainConfig.Clique != nil { uncleReward = big.NewInt(0) } else { - uncleReward = CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) + uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) } uncle := models.UncleModel{ CID: uncleNode.Cid().String(), @@ -298,17 +291,17 @@ type processArgs struct { blockNumber *big.Int receipts types.Receipts txs types.Transactions - rctNodes []*ipld.EthReceipt - rctTrieNodes []*ipld.EthRctTrie - txNodes []*ipld.EthTx - txTrieNodes []*ipld.EthTxTrie - logTrieNodes [][]*ipld.EthLogTrie + rctNodes []*ipld2.EthReceipt + rctTrieNodes []*ipld2.EthRctTrie + txNodes []*ipld2.EthTx + txTrieNodes []*ipld2.EthTxTrie + logTrieNodes [][]*ipld2.EthLogTrie logLeafNodeCIDs [][]cid.Cid rctLeafNodeCIDs []cid.Cid } // processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres -func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BlockTx, args processArgs) error { +func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs) error { // Process receipts and txs signer := types.MakeSigner(sdi.chainConfig, args.blockNumber) for i, receipt := range args.receipts { @@ -434,8 +427,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BlockTx, args processArgs return nil } -// PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD database -func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateNode) error { +// PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql +func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode) error { + tx, ok := batch.(*BatchTx) + if !ok { + return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } // publish the state node if stateNode.NodeType == sdtypes.Removed { // short circuit if it is a Removed node @@ -443,14 +440,14 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN stateModel := models.StateNodeModel{ Path: stateNode.Path, StateKey: common.BytesToHash(stateNode.LeafKey).String(), - CID: RemovedNodeStateCID, - MhKey: RemovedNodeMhKey, + CID: shared.RemovedNodeStateCID, + MhKey: shared.RemovedNodeMhKey, NodeType: stateNode.NodeType.Int(), } _, err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel, tx.headerID) return err } - stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) + stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) if err != nil { return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) } @@ -497,8 +494,8 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN storageModel := models.StorageNodeModel{ Path: storageNode.Path, StorageKey: common.BytesToHash(storageNode.LeafKey).String(), - CID: RemovedNodeStorageCID, - MhKey: RemovedNodeMhKey, + CID: shared.RemovedNodeStorageCID, + MhKey: shared.RemovedNodeMhKey, NodeType: storageNode.NodeType.Int(), } if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel, stateID); err != nil { @@ -506,7 +503,7 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN } continue } - storageCIDStr, storageMhKey, err := tx.cacheRaw(ipld.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) + storageCIDStr, storageMhKey, err := tx.cacheRaw(ipld2.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) if err != nil { return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) } @@ -525,8 +522,12 @@ func (sdi *StateDiffIndexer) PushStateNode(tx *BlockTx, stateNode sdtypes.StateN return nil } -// PushCodeAndCodeHash publishes code and codehash pairs to the ipld database -func (sdi *StateDiffIndexer) PushCodeAndCodeHash(tx *BlockTx, codeAndCodeHash sdtypes.CodeAndCodeHash) error { +// PushCodeAndCodeHash publishes code and codehash pairs to the ipld sql +func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAndCodeHash sdtypes.CodeAndCodeHash) error { + tx, ok := batch.(*BatchTx) + if !ok { + return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } // codec doesn't matter since db key is multihash-based mhKey, err := shared.MultihashKeyFromKeccak256(codeAndCodeHash.Hash) if err != nil { @@ -535,3 +536,8 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(tx *BlockTx, codeAndCodeHash sd tx.cacheDirect(mhKey, codeAndCodeHash.Code) return nil } + +// Close satisfied io.Closer +func (sdi *StateDiffIndexer) Close() error { + return sdi.dbWriter.db.Close() +} diff --git a/statediff/indexer/indexer_legacy_test.go b/statediff/indexer/database/sql/indexer_legacy_test.go similarity index 73% rename from statediff/indexer/indexer_legacy_test.go rename to statediff/indexer/database/sql/indexer_legacy_test.go index 7c01f567f..f2fdb0521 100644 --- a/statediff/indexer/indexer_legacy_test.go +++ b/statediff/indexer/database/sql/indexer_legacy_test.go @@ -14,19 +14,21 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package indexer_test +package sql_test import ( + "context" "testing" - "github.com/ethereum/go-ethereum/core/types" - "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/shared" "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) var ( @@ -39,12 +41,12 @@ func setupLegacy(t *testing.T) { mockLegacyBlock = legacyData.MockBlock legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256) - db, err = shared.SetupDB() + db, err = test_helpers.SetupDB() require.NoError(t, err) - ind, err = indexer.NewStateDiffIndexer(legacyData.Config, db) + ind, err = sql.NewSQLIndexer(context.Background(), legacyData.Config, db) require.NoError(t, err) - var tx *indexer.BlockTx + var tx *sql.BlockTx tx, err = ind.PushBlock( mockLegacyBlock, legacyData.MockReceipts, @@ -57,7 +59,7 @@ func setupLegacy(t *testing.T) { require.NoError(t, err) } - shared.ExpectEqual(t, tx.BlockNumber, legacyData.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.BlockNumber, legacyData.BlockNumber.Uint64()) } func TestPublishAndIndexerLegacy(t *testing.T) { @@ -76,12 +78,12 @@ func TestPublishAndIndexerLegacy(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRowx(pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) require.NoError(t, err) - shared.ExpectEqual(t, header.CID, legacyHeaderCID.String()) - shared.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) - shared.ExpectEqual(t, header.Reward, "5000000000000011250") + test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) + test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") require.Nil(t, legacyData.MockHeader.BaseFee) require.Nil(t, header.BaseFee) }) diff --git a/statediff/indexer/indexer_test.go b/statediff/indexer/database/sql/indexer_test.go similarity index 72% rename from statediff/indexer/indexer_test.go rename to statediff/indexer/database/sql/indexer_test.go index d1962dad2..91d55f094 100644 --- a/statediff/indexer/indexer_test.go +++ b/statediff/indexer/database/sql/indexer_test.go @@ -14,35 +14,37 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package indexer_test +package sql_test import ( "bytes" + "context" "fmt" "os" "testing" - "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" - "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" + + "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/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) var ( - db *postgres.DB + db sql.Database err error - ind *indexer.StateDiffIndexer + ind *interfaces.StateDiffIndexer ipfsPgGet = `SELECT data FROM public.blocks WHERE key = $1` tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte @@ -135,13 +137,13 @@ func init() { } func setup(t *testing.T) { - db, err = shared.SetupDB() + db, err = test_helpers.SetupDB() if err != nil { t.Fatal(err) } - ind, err = indexer.NewStateDiffIndexer(mocks.TestConfig, db) + ind, err = indexer.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db) require.NoError(t, err) - var tx *indexer.BlockTx + var tx *sql.BlockTx tx, err = ind.PushBlock( mockBlock, mocks.MockReceipts, @@ -157,11 +159,14 @@ func setup(t *testing.T) { } } - shared.ExpectEqual(t, tx.BlockNumber, mocks.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.BlockNumber, mocks.BlockNumber.Uint64()) } func tearDown(t *testing.T) { - indexer.TearDownDB(t, db) + sql.TearDownDB(t, db) + if err := ind.Close(); err != nil { + t.Fatal(err) + } } func TestPublishAndIndexer(t *testing.T) { @@ -180,14 +185,14 @@ func TestPublishAndIndexer(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRowx(pgStr, mocks.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).StructScan(header) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, header.CID, headerCID.String()) - shared.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) - shared.ExpectEqual(t, header.Reward, "2000000000000021250") - shared.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + test_helpers.ExpectEqual(t, header.CID, headerCID.String()) + test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) @@ -195,11 +200,11 @@ func TestPublishAndIndexer(t *testing.T) { mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() var data []byte - err = db.Get(&data, ipfsPgGet, prefixedKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, data, mocks.MockHeaderRlp) + test_helpers.ExpectEqual(t, data, mocks.MockHeaderRlp) }) t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { @@ -209,16 +214,16 @@ func TestPublishAndIndexer(t *testing.T) { trxs := make([]string, 0) pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.id) WHERE header_cids.block_number = $1` - err = db.Select(&trxs, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &trxs, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(trxs), 5) - expectTrue(t, shared.ListContainsString(trxs, trx1CID.String())) - expectTrue(t, shared.ListContainsString(trxs, trx2CID.String())) - expectTrue(t, shared.ListContainsString(trxs, trx3CID.String())) - expectTrue(t, shared.ListContainsString(trxs, trx4CID.String())) - expectTrue(t, shared.ListContainsString(trxs, trx5CID.String())) + test_helpers.ExpectEqual(t, len(trxs), 5) + expectTrue(t, test_helpers.ListContainsString(trxs, trx1CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx2CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) // and published for _, c := range trxs { dc, err := cid.Decode(c) @@ -228,16 +233,16 @@ func TestPublishAndIndexer(t *testing.T) { mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() var data []byte - err = db.Get(&data, ipfsPgGet, prefixedKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } switch c { case trx1CID.String(): - shared.ExpectEqual(t, data, tx1) + test_helpers.ExpectEqual(t, data, tx1) var txType *uint8 pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(&txType, pgStr, c) + err = db.Get(context.Background(), &txType, pgStr, c) if err != nil { t.Fatal(err) } @@ -245,10 +250,10 @@ func TestPublishAndIndexer(t *testing.T) { t.Fatalf("expected nil tx_type, got %d", *txType) } case trx2CID.String(): - shared.ExpectEqual(t, data, tx2) + test_helpers.ExpectEqual(t, data, tx2) var txType *uint8 pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(&txType, pgStr, c) + err = db.Get(context.Background(), &txType, pgStr, c) if err != nil { t.Fatal(err) } @@ -256,10 +261,10 @@ func TestPublishAndIndexer(t *testing.T) { t.Fatalf("expected nil tx_type, got %d", *txType) } case trx3CID.String(): - shared.ExpectEqual(t, data, tx3) + test_helpers.ExpectEqual(t, data, tx3) var txType *uint8 pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(&txType, pgStr, c) + err = db.Get(context.Background(), &txType, pgStr, c) if err != nil { t.Fatal(err) } @@ -267,10 +272,10 @@ func TestPublishAndIndexer(t *testing.T) { t.Fatalf("expected nil tx_type, got %d", *txType) } case trx4CID.String(): - shared.ExpectEqual(t, data, tx4) + test_helpers.ExpectEqual(t, data, tx4) var txType *uint8 pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(&txType, pgStr, c) + err = db.Get(context.Background(), &txType, pgStr, c) if err != nil { t.Fatal(err) } @@ -279,7 +284,7 @@ func TestPublishAndIndexer(t *testing.T) { } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.id) WHERE cid = $1 ORDER BY access_list_element.index ASC` - err = db.Select(&accessListElementModels, pgStr, c) + err = db.Select(context.Background(), &accessListElementModels, pgStr, c) if err != nil { t.Fatal(err) } @@ -295,13 +300,13 @@ func TestPublishAndIndexer(t *testing.T) { Address: accessListElementModels[1].Address, StorageKeys: accessListElementModels[1].StorageKeys, } - shared.ExpectEqual(t, model1, mocks.AccessListEntry1Model) - shared.ExpectEqual(t, model2, mocks.AccessListEntry2Model) + test_helpers.ExpectEqual(t, model1, mocks.AccessListEntry1Model) + test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) case trx5CID.String(): - shared.ExpectEqual(t, data, tx5) + test_helpers.ExpectEqual(t, data, tx5) var txType *uint8 pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(&txType, pgStr, c) + err = db.Get(context.Background(), &txType, pgStr, c) if err != nil { t.Fatal(err) } @@ -322,7 +327,7 @@ func TestPublishAndIndexer(t *testing.T) { AND transaction_cids.header_id = header_cids.id AND header_cids.block_number = $1 ORDER BY transaction_cids.index` - err = db.Select(&rcts, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } @@ -340,12 +345,12 @@ func TestPublishAndIndexer(t *testing.T) { 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.leaf_cid = $1 ORDER BY eth.log_cids.index ASC` - err = db.Select(&results, pgStr, rcts[i]) + err = db.Select(context.Background(), &results, pgStr, rcts[i]) require.NoError(t, err) // expecting MockLog1 and MockLog2 for mockReceipt4 expectedLogs := mocks.MockReceipts[i].Logs - shared.ExpectEqual(t, len(results), len(expectedLogs)) + test_helpers.ExpectEqual(t, len(results), len(expectedLogs)) var nodeElements []interface{} for idx, r := range results { @@ -357,7 +362,7 @@ func TestPublishAndIndexer(t *testing.T) { require.NoError(t, err) // 2nd element of the leaf node contains the encoded log data. - shared.ExpectEqual(t, logRaw, nodeElements[1].([]byte)) + test_helpers.ExpectEqual(t, logRaw, nodeElements[1].([]byte)) } } }) @@ -372,19 +377,19 @@ func TestPublishAndIndexer(t *testing.T) { WHERE receipt_cids.tx_id = transaction_cids.id AND transaction_cids.header_id = header_cids.id AND header_cids.block_number = $1 order by transaction_cids.id` - err = db.Select(&rcts, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(rcts), 5) + test_helpers.ExpectEqual(t, len(rcts), 5) for idx, rctLeafCID := range rcts { - result := make([]ipfs.BlockModel, 0) + result := make([]models.IPLDModel, 0) pgStr = `SELECT data FROM eth.receipt_cids INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) WHERE receipt_cids.leaf_cid = $1` - err = db.Select(&result, pgStr, rctLeafCID) + err = db.Select(context.Background(), &result, pgStr, rctLeafCID) if err != nil { t.Fatal(err) } @@ -397,7 +402,7 @@ func TestPublishAndIndexer(t *testing.T) { expectedRct, err := mocks.MockReceipts[idx].MarshalBinary() require.NoError(t, err) - shared.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) + test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) } // and published @@ -409,57 +414,57 @@ func TestPublishAndIndexer(t *testing.T) { mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() var data []byte - err = db.Get(&data, ipfsPgGet, prefixedKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } switch c { case rct1CID.String(): - shared.ExpectEqual(t, data, rct1) + test_helpers.ExpectEqual(t, data, rct1) var postStatus uint64 pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(&postStatus, pgStr, c) + err = db.Get(context.Background(), &postStatus, pgStr, c) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) + test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) case rct2CID.String(): - shared.ExpectEqual(t, data, rct2) + test_helpers.ExpectEqual(t, data, rct2) var postState string pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(&postState, pgStr, c) + err = db.Get(context.Background(), &postState, pgStr, c) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, postState, mocks.ExpectedPostState1) + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) case rct3CID.String(): - shared.ExpectEqual(t, data, rct3) + test_helpers.ExpectEqual(t, data, rct3) var postState string pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(&postState, pgStr, c) + err = db.Get(context.Background(), &postState, pgStr, c) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, postState, mocks.ExpectedPostState2) + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) case rct4CID.String(): - shared.ExpectEqual(t, data, rct4) + test_helpers.ExpectEqual(t, data, rct4) var postState string pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(&postState, pgStr, c) + err = db.Get(context.Background(), &postState, pgStr, c) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, postState, mocks.ExpectedPostState3) + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) case rct5CID.String(): - shared.ExpectEqual(t, data, rct5) + test_helpers.ExpectEqual(t, data, rct5) var postState string pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(&postState, pgStr, c) + err = db.Get(context.Background(), &postState, pgStr, c) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, postState, mocks.ExpectedPostState3) + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) } } }) @@ -472,11 +477,11 @@ func TestPublishAndIndexer(t *testing.T) { pgStr := `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) WHERE header_cids.block_number = $1 AND node_type != 3` - err = db.Select(&stateNodes, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(stateNodes), 2) + test_helpers.ExpectEqual(t, len(stateNodes), 2) for _, stateNode := range stateNodes { var data []byte dc, err := cid.Decode(stateNode.CID) @@ -485,22 +490,22 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() - err = db.Get(&data, ipfsPgGet, prefixedKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } pgStr = `SELECT * from eth.state_accounts WHERE state_id = $1` var account models.StateAccountModel - err = db.Get(&account, pgStr, stateNode.ID) + err = db.Get(context.Background(), &account, pgStr, stateNode.ID) if err != nil { t.Fatal(err) } if stateNode.CID == state1CID.String() { - shared.ExpectEqual(t, stateNode.NodeType, 2) - shared.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.ContractLeafKey).Hex()) - shared.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) - shared.ExpectEqual(t, data, mocks.ContractLeafNode) - shared.ExpectEqual(t, account, models.StateAccountModel{ + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.ContractLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) + test_helpers.ExpectEqual(t, data, mocks.ContractLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ ID: account.ID, StateID: stateNode.ID, Balance: "0", @@ -510,11 +515,11 @@ func TestPublishAndIndexer(t *testing.T) { }) } if stateNode.CID == state2CID.String() { - shared.ExpectEqual(t, stateNode.NodeType, 2) - shared.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.AccountLeafKey).Hex()) - shared.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) - shared.ExpectEqual(t, data, mocks.AccountLeafNode) - shared.ExpectEqual(t, account, models.StateAccountModel{ + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.AccountLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) + test_helpers.ExpectEqual(t, data, mocks.AccountLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ ID: account.ID, StateID: stateNode.ID, Balance: "1000", @@ -530,11 +535,11 @@ func TestPublishAndIndexer(t *testing.T) { pgStr = `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) WHERE header_cids.block_number = $1 AND node_type = 3` - err = db.Select(&stateNodes, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(stateNodes), 1) + test_helpers.ExpectEqual(t, len(stateNodes), 1) stateNode := stateNodes[0] var data []byte dc, err := cid.Decode(stateNode.CID) @@ -543,14 +548,14 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() - shared.ExpectEqual(t, prefixedKey, indexer.RemovedNodeMhKey) - err = db.Get(&data, ipfsPgGet, prefixedKey) + test_helpers.ExpectEqual(t, prefixedKey, sql.RemovedNodeMhKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, stateNode.CID, indexer.RemovedNodeStateCID) - shared.ExpectEqual(t, stateNode.Path, []byte{'\x02'}) - shared.ExpectEqual(t, data, []byte{}) + test_helpers.ExpectEqual(t, stateNode.CID, sql.RemovedNodeStateCID) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x02'}) + test_helpers.ExpectEqual(t, data, []byte{}) }) t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { @@ -564,12 +569,12 @@ func TestPublishAndIndexer(t *testing.T) { AND state_cids.header_id = header_cids.id AND header_cids.block_number = $1 AND storage_cids.node_type != 3` - err = db.Select(&storageNodes, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(storageNodes), 1) - shared.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ CID: storageCID.String(), NodeType: 2, StorageKey: common.BytesToHash(mocks.StorageLeafKey).Hex(), @@ -583,11 +588,11 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() - err = db.Get(&data, ipfsPgGet, prefixedKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, data, mocks.StorageLeafNode) + test_helpers.ExpectEqual(t, data, mocks.StorageLeafNode) // check that Removed storage nodes were properly indexed storageNodes = make([]models.StorageNodeWithStateKeyModel, 0) @@ -597,13 +602,13 @@ func TestPublishAndIndexer(t *testing.T) { AND state_cids.header_id = header_cids.id AND header_cids.block_number = $1 AND storage_cids.node_type = 3` - err = db.Select(&storageNodes, pgStr, mocks.BlockNumber.Uint64()) + err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, len(storageNodes), 1) - shared.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ - CID: indexer.RemovedNodeStorageCID, + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + CID: sql.RemovedNodeStorageCID, NodeType: 3, StorageKey: common.BytesToHash(mocks.RemovedLeafKey).Hex(), StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), @@ -615,11 +620,11 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey = dshelp.MultihashToDsKey(dc.Hash()) prefixedKey = blockstore.BlockPrefix.String() + mhKey.String() - shared.ExpectEqual(t, prefixedKey, indexer.RemovedNodeMhKey) - err = db.Get(&data, ipfsPgGet, prefixedKey) + test_helpers.ExpectEqual(t, prefixedKey, sql.RemovedNodeMhKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } - shared.ExpectEqual(t, data, []byte{}) + test_helpers.ExpectEqual(t, data, []byte{}) }) } diff --git a/statediff/indexer/database/sql/interfaces.go b/statediff/indexer/database/sql/interfaces.go new file mode 100644 index 000000000..755c4e156 --- /dev/null +++ b/statediff/indexer/database/sql/interfaces.go @@ -0,0 +1,88 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package sql + +import ( + "context" + "io" + "time" +) + +// Database interfaces required by the sql indexer +type Database interface { + Driver + Statements +} + +// Driver interface has all the methods required by a driver implementation to support the sql indexer +type Driver interface { + QueryRow(ctx context.Context, sql string, args ...interface{}) ScannableRow + Exec(ctx context.Context, sql string, args ...interface{}) (Result, error) + Select(ctx context.Context, dest interface{}, query string, args ...interface{}) error + Get(ctx context.Context, dest interface{}, query string, args ...interface{}) error + Begin(ctx context.Context) (Tx, error) + Stats() Stats + NodeID() int64 + Context() context.Context + io.Closer +} + +// Statements interface to accommodate different SQL query syntax +type Statements interface { + InsertHeaderStm() string + InsertUncleStm() string + InsertTxStm() string + InsertAccessListElementStm() string + InsertRctStm() string + InsertLogStm() string + InsertStateStm() string + InsertAccountStm() string + InsertStorageStm() string + InsertIPLDStm() string + InsertIPLDsStm() string +} + +// Tx interface to accommodate different concrete SQL transaction types +type Tx interface { + QueryRow(ctx context.Context, sql string, args ...interface{}) ScannableRow + Exec(ctx context.Context, sql string, args ...interface{}) (Result, error) + Commit(ctx context.Context) error + Rollback(ctx context.Context) error +} + +// ScannableRow interface to accommodate different concrete row types +type ScannableRow interface { + Scan(dest ...interface{}) error + StructScan(dest interface{}) error +} + +// Result interface to accommodate different concrete result types +type Result interface { + RowsAffected() (int64, error) +} + +// Stats interface to accommodate different concrete sql stats types +type Stats interface { + MaxOpen() int64 + Open() int64 + InUse() int64 + Idle() int64 + WaitCount() int64 + WaitDuration() time.Duration + MaxIdleClosed() int64 + MaxLifetimeClosed() int64 +} diff --git a/statediff/indexer/metrics.go b/statediff/indexer/database/sql/metrics.go similarity index 79% rename from statediff/indexer/metrics.go rename to statediff/indexer/database/sql/metrics.go index 2d37816f6..b0946a722 100644 --- a/statediff/indexer/metrics.go +++ b/statediff/indexer/database/sql/metrics.go @@ -1,7 +1,22 @@ -package indexer +// VulcanizeDB +// Copyright © 2021 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 . + +package sql import ( - "database/sql" "strings" "github.com/ethereum/go-ethereum/metrics" @@ -79,7 +94,7 @@ func RegisterIndexerMetrics(reg metrics.Registry) indexerMetricsHandles { } type dbMetricsHandles struct { - // Maximum number of open connections to the database + // Maximum number of open connections to the sql maxOpen metrics.Gauge // The number of established connections both in use and idle open metrics.Gauge @@ -120,13 +135,13 @@ func RegisterDBMetrics(reg metrics.Registry) dbMetricsHandles { return ctx } -func (met *dbMetricsHandles) Update(stats sql.DBStats) { - met.maxOpen.Update(int64(stats.MaxOpenConnections)) - met.open.Update(int64(stats.OpenConnections)) - met.inUse.Update(int64(stats.InUse)) - met.idle.Update(int64(stats.Idle)) - met.waitedFor.Inc(stats.WaitCount) - met.blockedMilliseconds.Inc(stats.WaitDuration.Milliseconds()) - met.closedMaxIdle.Inc(stats.MaxIdleClosed) - met.closedMaxLifetime.Inc(stats.MaxLifetimeClosed) +func (met *dbMetricsHandles) Update(stats Stats) { + met.maxOpen.Update(stats.MaxOpen()) + met.open.Update(stats.Open()) + met.inUse.Update(stats.InUse()) + met.idle.Update(stats.Idle()) + met.waitedFor.Inc(stats.WaitCount()) + met.blockedMilliseconds.Inc(stats.WaitDuration().Milliseconds()) + met.closedMaxIdle.Inc(stats.MaxIdleClosed()) + met.closedMaxLifetime.Inc(stats.MaxLifetimeClosed()) } diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go new file mode 100644 index 000000000..07e3dfe21 --- /dev/null +++ b/statediff/indexer/database/sql/postgres/config.go @@ -0,0 +1,81 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package postgres + +import ( + "fmt" + "time" + + "github.com/ethereum/go-ethereum/statediff/indexer/shared" +) + +type DriverType string + +const ( + PGX DriverType = "PGX" + SQLX DriverType = "SQLX" +) + +// DefaultConfig are default parameters for connecting to a Postgres sql +var DefaultConfig = Config{ + Hostname: "localhost", + Port: 5432, + DatabaseName: "vulcanize_test", + Username: "postgres", + Password: "", +} + +// Config holds params for a Postgres db +type Config struct { + // conn string params + Hostname string + Port int + DatabaseName string + Username string + Password string + + // conn settings + MaxConns int + MaxIdle int + MinConns int + MaxConnIdleTime time.Duration + MaxConnLifetime time.Duration + ConnTimeout time.Duration + + // node info params + ID string + ClientName string + + // driver type + Driver DriverType +} + +func (c Config) Type() shared.DBType { + return shared.POSTGRES +} + +func (c Config) DbConnectionString() string { + if len(c.Username) > 0 && len(c.Password) > 0 { + return fmt.Sprintf("postgresql://%s:%s@%s:%d/%s?sslmode=disable", + c.Username, c.Password, c.Hostname, c.Port, c.DatabaseName) + } + if len(c.Username) > 0 && len(c.Password) == 0 { + return fmt.Sprintf("postgresql://%s@%s:%d/%s?sslmode=disable", + c.Username, c.Hostname, c.Port, c.DatabaseName) + } + return fmt.Sprintf("postgresql://%s:%d/%s?sslmode=disable", c.Hostname, c.Port, c.DatabaseName) +} diff --git a/statediff/indexer/database/sql/postgres/database.go b/statediff/indexer/database/sql/postgres/database.go new file mode 100644 index 000000000..3fe7f652e --- /dev/null +++ b/statediff/indexer/database/sql/postgres/database.go @@ -0,0 +1,112 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package postgres + +import "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + +var _ sql.Database = &DB{} + +const ( + createNodeStm = `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES ($1, $2, $3, $4, $5) + ON CONFLICT (genesis_block, network_id, node_id, chain_id) + DO UPDATE + SET genesis_block = $1, + network_id = $2, + node_id = $3, + client_name = $4, + chain_id = $5 + RETURNING id` +) + +// NewPostgresDB returns a postgres.DB using the provided driver +func NewPostgresDB(driver sql.Driver) *DB { + return &DB{driver} +} + +// DB implements sql.Databse using a configured driver and Postgres statement syntax +type DB struct { + sql.Driver +} + +// InsertHeaderStm satisfies the sql.Statements interface +func (db *DB) InsertHeaderStm() string { + return `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) + VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) + ON CONFLICT (block_number, block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) + RETURNING id` +} + +// InsertUncleStm satisfies the sql.Statements interface +func (db *DB) InsertUncleStm() string { + return `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES ($1, $2, $3, $4, $5, $6) + ON CONFLICT (header_id, block_hash) DO UPDATE SET (parent_hash, cid, reward, mh_key) = ($3, $4, $5, $6)` +} + +// InsertTxStm satisfies the sql.Statements interface +func (db *DB) InsertTxStm() string { + return `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) + ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = ($3, $4, $5, $6, $7, $8, $9) + RETURNING id` +} + +// InsertAccessListElementStm satisfies the sql.Statements interface +func (db *DB) InsertAccessListElementStm() string { + return `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) + ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)` +} + +// InsertRctStm satisfies the sql.Statements interface +func (db *DB) InsertRctStm() string { + return `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) + ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8) + RETURNING id` +} + +// InsertLogStm satisfies the sql.Statements interface +func (db *DB) InsertLogStm() string { + return `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)` +} + +// InsertStateStm satisfies the sql.Statements interface +func (db *DB) InsertStateStm() string { + return `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) + ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) + RETURNING id` +} + +// InsertAccountStm satisfies the sql.Statements interface +func (db *DB) InsertAccountStm() string { + return `INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5) + ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = ($2, $3, $4, $5)` +} + +// InsertStorageStm satisfies the sql.Statements interface +func (db *DB) InsertStorageStm() string { + return `INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) + ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)` +} + +// InsertIPLDStm satisfies the sql.Statements interface +func (db *DB) InsertIPLDStm() string { + return `INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING` +} + +// InsertIPLDsStm satisfies the sql.Statements interface +func (db *DB) InsertIPLDsStm() string { + return `INSERT INTO public.blocks (key, data) VALUES (unnest($1::TEXT[]), unnest($2::BYTEA[])) ON CONFLICT (key) DO NOTHING` +} diff --git a/statediff/indexer/postgres/errors.go b/statediff/indexer/database/sql/postgres/errors.go similarity index 100% rename from statediff/indexer/postgres/errors.go rename to statediff/indexer/database/sql/postgres/errors.go diff --git a/statediff/indexer/database/sql/postgres/pgx.go b/statediff/indexer/database/sql/postgres/pgx.go new file mode 100644 index 000000000..d94c35083 --- /dev/null +++ b/statediff/indexer/database/sql/postgres/pgx.go @@ -0,0 +1,255 @@ +// 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 . + +package postgres + +import ( + "context" + "time" + + "github.com/georgysavva/scany/pgxscan" + "github.com/jackc/pgconn" + "github.com/jackc/pgx/v4" + "github.com/jackc/pgx/v4/pgxpool" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/node" +) + +// PGXDriver driver, implements sql.Driver +type PGXDriver struct { + ctx context.Context + pool *pgxpool.Pool + nodeInfo node.Info + nodeID int64 +} + +// NewPGXDriver returns a new pgx driver +// it initializes the connection pool and creates the node info table +func NewPGXDriver(ctx context.Context, config Config, node node.Info) (*PGXDriver, error) { + pgConf, err := MakeConfig(config) + if err != nil { + return nil, err + } + dbPool, err := pgxpool.ConnectConfig(ctx, pgConf) + if err != nil { + return nil, ErrDBConnectionFailed(err) + } + pg := &PGXDriver{ctx: ctx, pool: dbPool, nodeInfo: node} + nodeErr := pg.createNode() + if nodeErr != nil { + return &PGXDriver{}, ErrUnableToSetNode(nodeErr) + } + return pg, nil +} + +// MakeConfig creates a pgxpool.Config from the provided Config +func MakeConfig(config Config) (*pgxpool.Config, error) { + conf, err := pgxpool.ParseConfig("") + if err != nil { + return nil, err + } + + //conf.ConnConfig.BuildStatementCache = nil + conf.ConnConfig.Config.Host = config.Hostname + conf.ConnConfig.Config.Port = uint16(config.Port) + conf.ConnConfig.Config.Database = config.DatabaseName + conf.ConnConfig.Config.User = config.Username + conf.ConnConfig.Config.Password = config.Password + + if config.ConnTimeout != 0 { + conf.ConnConfig.Config.ConnectTimeout = config.ConnTimeout + } + if config.MaxConns != 0 { + conf.MaxConns = int32(config.MaxConns) + } + if config.MinConns != 0 { + conf.MinConns = int32(config.MinConns) + } + if config.MaxConnLifetime != 0 { + conf.MaxConnLifetime = config.MaxConnLifetime + } + if config.MaxConnIdleTime != 0 { + conf.MaxConnIdleTime = config.MaxConnIdleTime + } + return conf, nil +} + +func (pgx *PGXDriver) createNode() error { + var nodeID int64 + err := pgx.pool.QueryRow( + pgx.ctx, + createNodeStm, + pgx.nodeInfo.GenesisBlock, pgx.nodeInfo.NetworkID, + pgx.nodeInfo.ID, pgx.nodeInfo.ClientName, + pgx.nodeInfo.ChainID).Scan(&nodeID) + if err != nil { + return ErrUnableToSetNode(err) + } + pgx.nodeID = nodeID + return nil +} + +// QueryRow satisfies sql.Database +func (pgx *PGXDriver) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { + row := pgx.pool.QueryRow(ctx, sql, args...) + return rowWrapper{row: row} +} + +// Exec satisfies sql.Database +func (pgx *PGXDriver) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) { + res, err := pgx.pool.Exec(ctx, sql, args...) + return resultWrapper{ct: res}, err +} + +// Select satisfies sql.Database +func (pgx *PGXDriver) Select(ctx context.Context, dest interface{}, query string, args ...interface{}) error { + return pgxscan.Select(ctx, pgx.pool, dest, query, args...) +} + +// Get satisfies sql.Database +func (pgx *PGXDriver) Get(ctx context.Context, dest interface{}, query string, args ...interface{}) error { + return pgxscan.Get(ctx, pgx.pool, dest, query, args...) +} + +// Begin satisfies sql.Database +func (pgx *PGXDriver) Begin(ctx context.Context) (sql.Tx, error) { + tx, err := pgx.pool.Begin(ctx) + if err != nil { + return nil, err + } + return pgxTxWrapper{tx: tx}, nil +} + +func (pgx *PGXDriver) Stats() sql.Stats { + stats := pgx.pool.Stat() + return pgxStatsWrapper{stats: stats} +} + +// NodeInfo satisfies sql.Database +func (pgx *PGXDriver) NodeInfo() node.Info { + return pgx.nodeInfo +} + +// NodeID satisfies sql.Database +func (pgx *PGXDriver) NodeID() int64 { + return pgx.nodeID +} + +// Close satisfies sql.Database/io.Closer +func (pgx *PGXDriver) Close() error { + pgx.pool.Close() + return nil +} + +// Context satisfies sql.Database +func (pgx *PGXDriver) Context() context.Context { + return pgx.ctx +} + +type rowWrapper struct { + row pgx.Row +} + +// Scan satisfies sql.ScannableRow +func (r rowWrapper) Scan(dest ...interface{}) error { + return r.row.Scan(dest) +} + +// StructScan satisfies sql.ScannableRow +func (r rowWrapper) StructScan(dest interface{}) error { + return pgxscan.ScanRow(dest, r.row.(pgx.Rows)) +} + +type resultWrapper struct { + ct pgconn.CommandTag +} + +// RowsAffected satisfies sql.Result +func (r resultWrapper) RowsAffected() (int64, error) { + return r.ct.RowsAffected(), nil +} + +type pgxStatsWrapper struct { + stats *pgxpool.Stat +} + +// MaxOpen satisfies sql.Stats +func (s pgxStatsWrapper) MaxOpen() int64 { + return int64(s.stats.MaxConns()) +} + +// Open satisfies sql.Stats +func (s pgxStatsWrapper) Open() int64 { + return int64(s.stats.TotalConns()) +} + +// InUse satisfies sql.Stats +func (s pgxStatsWrapper) InUse() int64 { + return int64(s.stats.AcquiredConns()) +} + +// Idle satisfies sql.Stats +func (s pgxStatsWrapper) Idle() int64 { + return int64(s.stats.IdleConns()) +} + +// WaitCount satisfies sql.Stats +func (s pgxStatsWrapper) WaitCount() int64 { + return s.stats.EmptyAcquireCount() +} + +// WaitDuration satisfies sql.Stats +func (s pgxStatsWrapper) WaitDuration() time.Duration { + return s.stats.AcquireDuration() +} + +// MaxIdleClosed satisfies sql.Stats +func (s pgxStatsWrapper) MaxIdleClosed() int64 { + // this stat isn't supported by pgxpool, but we don't want to panic + return 0 +} + +// MaxLifetimeClosed satisfies sql.Stats +func (s pgxStatsWrapper) MaxLifetimeClosed() int64 { + return s.stats.CanceledAcquireCount() +} + +type pgxTxWrapper struct { + tx pgx.Tx +} + +// QueryRow satisfies sql.Tx +func (t pgxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { + row := t.tx.QueryRow(ctx, sql, args...) + return rowWrapper{row: row} +} + +// Exec satisfies sql.Tx +func (t pgxTxWrapper) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) { + res, err := t.tx.Exec(ctx, sql, args...) + return resultWrapper{ct: res}, err +} + +// Commit satisfies sql.Tx +func (t pgxTxWrapper) Commit(ctx context.Context) error { + return t.tx.Commit(ctx) +} + +// Rollback satisfies sql.Tx +func (t pgxTxWrapper) Rollback(ctx context.Context) error { + return t.tx.Rollback(ctx) +} diff --git a/statediff/indexer/database/sql/postgres/pgx_test.go b/statediff/indexer/database/sql/postgres/pgx_test.go new file mode 100644 index 000000000..aadb12835 --- /dev/null +++ b/statediff/indexer/database/sql/postgres/pgx_test.go @@ -0,0 +1,122 @@ +// 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 . + +package postgres_test + +import ( + "context" + "fmt" + "math/big" + "strings" + "testing" + + "github.com/jackc/pgx/pgtype" + "github.com/jackc/pgx/v4/pgxpool" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/node" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +var ( + pgConfig, _ = postgres.MakeConfig(postgres.DefaultConfig) + ctx = context.Background() +) + +func expectContainsSubstring(t *testing.T, full string, sub string) { + if !strings.Contains(full, sub) { + t.Fatalf("Expected \"%v\" to contain substring \"%v\"\n", full, sub) + } +} + +func TestPostgresPGX(t *testing.T) { + t.Run("connects to the sql", func(t *testing.T) { + dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig) + if err != nil { + t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err) + } + defer dbPool.Close() + if dbPool == nil { + t.Fatal("DB pool is nil") + } + }) + + t.Run("serializes big.Int to db", func(t *testing.T) { + // postgres driver doesn't support go big.Int type + // various casts in golang uint64, int64, overflow for + // transaction value (in wei) even though + // postgres numeric can handle an arbitrary + // sized int, so use string representation of big.Int + // and cast on insert + + dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig) + if err != nil { + t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err) + } + defer dbPool.Close() + + bi := new(big.Int) + bi.SetString("34940183920000000000", 10) + test_helpers.ExpectEqual(t, bi.String(), "34940183920000000000") + + defer dbPool.Exec(ctx, `DROP TABLE IF EXISTS example`) + _, err = dbPool.Exec(ctx, "CREATE TABLE example ( id INTEGER, data NUMERIC )") + if err != nil { + t.Fatal(err) + } + + sqlStatement := ` + INSERT INTO example (id, data) + VALUES (1, cast($1 AS NUMERIC))` + _, err = dbPool.Exec(ctx, sqlStatement, bi.String()) + if err != nil { + t.Fatal(err) + } + + var data pgtype.Numeric + err = dbPool.QueryRow(ctx, `SELECT data FROM example WHERE id = 1`).Scan(&data) + if err != nil { + t.Fatal(err) + } + + test_helpers.ExpectEqual(t, bi.String(), data) + actual := new(big.Int) + actual.Set(data.Int) + test_helpers.ExpectEqual(t, actual, bi) + }) + + t.Run("throws error when can't connect to the database", func(t *testing.T) { + goodInfo := node.Info{GenesisBlock: "GENESIS", NetworkID: "1", ID: "x123", ClientName: "geth"} + _, err := postgres.NewPGXDriver(ctx, postgres.Config{}, goodInfo) + if err == nil { + t.Fatal("Expected an error") + } + + expectContainsSubstring(t, err.Error(), postgres.DbConnectionFailedMsg) + }) + + t.Run("throws error when can't create node", func(t *testing.T) { + badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) + badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} + + _, err := postgres.NewPGXDriver(ctx, postgres.DefaultConfig, badInfo) + if err == nil { + t.Fatal("Expected an error") + } + + expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg) + }) +} diff --git a/statediff/indexer/postgres/postgres_suite_test.go b/statediff/indexer/database/sql/postgres/postgres_suite_test.go similarity index 100% rename from statediff/indexer/postgres/postgres_suite_test.go rename to statediff/indexer/database/sql/postgres/postgres_suite_test.go diff --git a/statediff/indexer/database/sql/postgres/sqlx.go b/statediff/indexer/database/sql/postgres/sqlx.go new file mode 100644 index 000000000..2abf82d89 --- /dev/null +++ b/statediff/indexer/database/sql/postgres/sqlx.go @@ -0,0 +1,197 @@ +// 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 . + +package postgres + +import ( + "context" + coresql "database/sql" + "time" + + "github.com/jmoiron/sqlx" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/node" +) + +// SQLXDriver driver, implements sql.Driver +type SQLXDriver struct { + ctx context.Context + db *sqlx.DB + nodeInfo node.Info + nodeID int64 +} + +// NewSQLXDriver returns a new sqlx driver for Postgres +// it initializes the connection pool and creates the node info table +func NewSQLXDriver(ctx context.Context, config Config, node node.Info) (*SQLXDriver, error) { + db, err := sqlx.ConnectContext(ctx, "postgres", config.DbConnectionString()) + if err != nil { + return &SQLXDriver{}, ErrDBConnectionFailed(err) + } + if config.MaxConns > 0 { + db.SetMaxOpenConns(config.MaxConns) + } + if config.MaxIdle > 0 { + db.SetMaxIdleConns(config.MaxIdle) + } + if config.MaxConnLifetime > 0 { + lifetime := config.MaxConnLifetime + db.SetConnMaxLifetime(lifetime) + } + driver := &SQLXDriver{ctx: ctx, db: db, nodeInfo: node} + if err := driver.createNode(); err != nil { + return &SQLXDriver{}, ErrUnableToSetNode(err) + } + return driver, nil +} + +func (driver *SQLXDriver) createNode() error { + var nodeID int64 + err := driver.db.QueryRowx( + createNodeStm, + driver.nodeInfo.GenesisBlock, driver.nodeInfo.NetworkID, + driver.nodeInfo.ID, driver.nodeInfo.ClientName, + driver.nodeInfo.ChainID).Scan(&nodeID) + if err != nil { + return ErrUnableToSetNode(err) + } + driver.nodeID = nodeID + return nil +} + +// QueryRow satisfies sql.Database +func (driver *SQLXDriver) QueryRow(_ context.Context, sql string, args ...interface{}) sql.ScannableRow { + return driver.db.QueryRowx(sql, args...) +} + +// Exec satisfies sql.Database +func (driver *SQLXDriver) Exec(_ context.Context, sql string, args ...interface{}) (sql.Result, error) { + return driver.db.Exec(sql, args...) +} + +// Select satisfies sql.Database +func (driver *SQLXDriver) Select(_ context.Context, dest interface{}, query string, args ...interface{}) error { + return driver.db.Select(dest, query, args...) +} + +// Get satisfies sql.Database +func (driver *SQLXDriver) Get(_ context.Context, dest interface{}, query string, args ...interface{}) error { + return driver.db.Get(dest, query, args...) +} + +// Begin satisfies sql.Database +func (driver *SQLXDriver) Begin(_ context.Context) (sql.Tx, error) { + tx, err := driver.db.Beginx() + if err != nil { + return nil, err + } + return sqlxTxWrapper{tx: tx}, nil +} + +func (driver *SQLXDriver) Stats() sql.Stats { + stats := driver.db.Stats() + return sqlxStatsWrapper{stats: stats} +} + +// NodeInfo satisfies sql.Database +func (driver *SQLXDriver) NodeInfo() node.Info { + return driver.nodeInfo +} + +// NodeID satisfies sql.Database +func (driver *SQLXDriver) NodeID() int64 { + return driver.nodeID +} + +// Close satisfies sql.Database/io.Closer +func (driver *SQLXDriver) Close() error { + return driver.db.Close() +} + +// Context satisfies sql.Database +func (driver *SQLXDriver) Context() context.Context { + return driver.ctx +} + +type sqlxStatsWrapper struct { + stats coresql.DBStats +} + +// MaxOpen satisfies sql.Stats +func (s sqlxStatsWrapper) MaxOpen() int64 { + return int64(s.stats.MaxOpenConnections) +} + +// Open satisfies sql.Stats +func (s sqlxStatsWrapper) Open() int64 { + return int64(s.stats.OpenConnections) +} + +// InUse satisfies sql.Stats +func (s sqlxStatsWrapper) InUse() int64 { + return int64(s.stats.InUse) +} + +// Idle satisfies sql.Stats +func (s sqlxStatsWrapper) Idle() int64 { + return int64(s.stats.Idle) +} + +// WaitCount satisfies sql.Stats +func (s sqlxStatsWrapper) WaitCount() int64 { + return s.stats.WaitCount +} + +// WaitDuration satisfies sql.Stats +func (s sqlxStatsWrapper) WaitDuration() time.Duration { + return s.stats.WaitDuration +} + +// MaxIdleClosed satisfies sql.Stats +func (s sqlxStatsWrapper) MaxIdleClosed() int64 { + return s.stats.MaxIdleClosed +} + +// MaxLifetimeClosed satisfies sql.Stats +func (s sqlxStatsWrapper) MaxLifetimeClosed() int64 { + return s.stats.MaxLifetimeClosed +} + +type sqlxTxWrapper struct { + tx *sqlx.Tx +} + +// QueryRow satisfies sql.Tx +func (t sqlxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { + row := t.tx.QueryRow(sql, args...) + return rowWrapper{row: row} +} + +// Exec satisfies sql.Tx +func (t sqlxTxWrapper) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) { + return t.tx.Exec(sql, args...) +} + +// Commit satisfies sql.Tx +func (t sqlxTxWrapper) Commit(ctx context.Context) error { + return t.tx.Commit() +} + +// Rollback satisfies sql.Tx +func (t sqlxTxWrapper) Rollback(ctx context.Context) error { + return t.tx.Rollback() +} diff --git a/statediff/indexer/postgres/postgres_test.go b/statediff/indexer/database/sql/postgres/sqlx_test.go similarity index 66% rename from statediff/indexer/postgres/postgres_test.go rename to statediff/indexer/database/sql/postgres/sqlx_test.go index f3bbdffd0..37164e0f7 100644 --- a/statediff/indexer/postgres/postgres_test.go +++ b/statediff/indexer/database/sql/postgres/sqlx_test.go @@ -25,33 +25,19 @@ import ( "github.com/jmoiron/sqlx" _ "github.com/lib/pq" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/node" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" - "github.com/ethereum/go-ethereum/statediff/indexer/shared" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) -var DBParams = postgres.ConnectionParams{ - Name: "vulcanize_public", - Password: "password", - Port: 5432, - Hostname: "localhost", - User: "vdbm", -} - -func expectContainsSubstring(t *testing.T, full string, sub string) { - if !strings.Contains(full, sub) { - t.Fatalf("Expected \"%v\" to contain substring \"%v\"\n", full, sub) - } -} - -func TestPostgresDB(t *testing.T) { +func TestPostgresSQLX(t *testing.T) { var sqlxdb *sqlx.DB t.Run("connects to the database", func(t *testing.T) { var err error - pgConfig := postgres.DbConnectionString(DBParams) + connStr := postgres.DefaultConfig.DbConnectionString() - sqlxdb, err = sqlx.Connect("postgres", pgConfig) + sqlxdb, err = sqlx.Connect("postgres", connStr) if err != nil { t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig, err) @@ -69,8 +55,8 @@ func TestPostgresDB(t *testing.T) { // sized int, so use string representation of big.Int // and cast on insert - pgConnectString := postgres.DbConnectionString(DBParams) - db, err := sqlx.Connect("postgres", pgConnectString) + connStr := postgres.DefaultConfig.DbConnectionString() + db, err := sqlx.Connect("postgres", connStr) if err != nil { t.Fatal(err) } @@ -80,7 +66,7 @@ func TestPostgresDB(t *testing.T) { bi := new(big.Int) bi.SetString("34940183920000000000", 10) - shared.ExpectEqual(t, bi.String(), "34940183920000000000") + test_helpers.ExpectEqual(t, bi.String(), "34940183920000000000") defer db.Exec(`DROP TABLE IF EXISTS example`) _, err = db.Exec("CREATE TABLE example ( id INTEGER, data NUMERIC )") @@ -102,19 +88,15 @@ func TestPostgresDB(t *testing.T) { t.Fatal(err) } - shared.ExpectEqual(t, bi.String(), data) + test_helpers.ExpectEqual(t, bi.String(), data) actual := new(big.Int) actual.SetString(data, 10) - shared.ExpectEqual(t, actual, bi) + test_helpers.ExpectEqual(t, actual, bi) }) t.Run("throws error when can't connect to the database", func(t *testing.T) { - invalidDatabase := postgres.ConnectionParams{} - node := node.Info{GenesisBlock: "GENESIS", NetworkID: "1", ID: "x123", ClientName: "geth"} - - _, err := postgres.NewDB(postgres.DbConnectionString(invalidDatabase), - postgres.ConnectionConfig{}, node) - + goodInfo := node.Info{GenesisBlock: "GENESIS", NetworkID: "1", ID: "x123", ClientName: "geth"} + _, err := postgres.NewSQLXDriver(ctx, postgres.Config{}, goodInfo) if err == nil { t.Fatal("Expected an error") } @@ -124,13 +106,13 @@ func TestPostgresDB(t *testing.T) { t.Run("throws error when can't create node", func(t *testing.T) { badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) - node := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} - - _, err := postgres.NewDB(postgres.DbConnectionString(DBParams), postgres.ConnectionConfig{}, node) + badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} + _, err := postgres.NewSQLXDriver(ctx, postgres.DefaultConfig, badInfo) if err == nil { t.Fatal("Expected an error") } + expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg) }) } diff --git a/statediff/indexer/database/sql/postgres/test_helpers.go b/statediff/indexer/database/sql/postgres/test_helpers.go new file mode 100644 index 000000000..491701c4b --- /dev/null +++ b/statediff/indexer/database/sql/postgres/test_helpers.go @@ -0,0 +1,42 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package postgres + +import ( + "context" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/node" +) + +// SetupSQLXDB is used to setup a sqlx db for tests +func SetupSQLXDB() (sql.Database, error) { + driver, err := NewSQLXDriver(context.Background(), DefaultConfig, node.Info{}) + if err != nil { + return nil, err + } + return NewPostgresDB(driver), nil +} + +// SetupPGXDB is used to setup a pgx db for tests +func SetupPGXDB() (sql.Database, error) { + driver, err := NewPGXDriver(context.Background(), DefaultConfig, node.Info{}) + if err != nil { + return nil, err + } + return NewPostgresDB(driver), nil +} diff --git a/statediff/indexer/test_helpers.go b/statediff/indexer/database/sql/test_helpers.go similarity index 69% rename from statediff/indexer/test_helpers.go rename to statediff/indexer/database/sql/test_helpers.go index 024bb58f0..cebddb9d1 100644 --- a/statediff/indexer/test_helpers.go +++ b/statediff/indexer/database/sql/test_helpers.go @@ -14,46 +14,46 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package indexer +package sql import ( + "context" "testing" - - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" ) // TearDownDB is used to tear down the watcher dbs after tests -func TearDownDB(t *testing.T, db *postgres.DB) { - tx, err := db.Beginx() +func TearDownDB(t *testing.T, db Database) { + ctx := context.Background() + tx, err := db.Begin(ctx) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM eth.header_cids`) + _, err = tx.Exec(ctx, `DELETE FROM eth.header_cids`) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM eth.transaction_cids`) + _, err = tx.Exec(ctx, `DELETE FROM eth.transaction_cids`) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM eth.receipt_cids`) + _, err = tx.Exec(ctx, `DELETE FROM eth.receipt_cids`) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM eth.state_cids`) + _, err = tx.Exec(ctx, `DELETE FROM eth.state_cids`) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM eth.storage_cids`) + _, err = tx.Exec(ctx, `DELETE FROM eth.storage_cids`) if err != nil { t.Fatal(err) } - _, err = tx.Exec(`DELETE FROM blocks`) + _, err = tx.Exec(ctx, `DELETE FROM blocks`) if err != nil { t.Fatal(err) } - err = tx.Commit() + err = tx.Commit(ctx) if err != nil { t.Fatal(err) } diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go new file mode 100644 index 000000000..ea276dfbf --- /dev/null +++ b/statediff/indexer/database/sql/writer.go @@ -0,0 +1,141 @@ +// 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 . + +package sql + +import ( + "fmt" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/statediff/indexer/models" +) + +var ( + nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") +) + +// Writer handles processing and writing of indexed IPLD objects to Postgres +type Writer struct { + db Database +} + +// NewWriter creates a new pointer to a Writer +func NewWriter(db Database) *Writer { + return &Writer{ + db: db, + } +} + +func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) (int64, error) { + var headerID int64 + err := tx.QueryRow(in.db.Context(), in.db.InsertHeaderStm(), + header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, in.db.NodeID(), header.Reward, header.StateRoot, header.TxRoot, + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee).Scan(&headerID) + if err != nil { + return 0, fmt.Errorf("error upserting header_cids entry: %v", err) + } + indexerMetrics.blocks.Inc(1) + return headerID, nil +} + +func (in *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel, headerID int64) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertUncleStm(), + uncle.BlockHash, headerID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey) + if err != nil { + return fmt.Errorf("error upserting uncle_cids entry: %v", err) + } + return nil +} + +func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel, headerID int64) (int64, error) { + var txID int64 + err := tx.QueryRow(in.db.Context(), in.db.InsertTxStm(), + headerID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type).Scan(&txID) + if err != nil { + return 0, fmt.Errorf("error upserting transaction_cids entry: %v", err) + } + indexerMetrics.transactions.Inc(1) + return txID, nil +} + +func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel, txID int64) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertAccessListElementStm(), + txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) + if err != nil { + return fmt.Errorf("error upserting access_list_element entry: %v", err) + } + indexerMetrics.accessListEntries.Inc(1) + return nil +} + +func (in *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel, txID int64) (int64, error) { + var receiptID int64 + err := tx.QueryRow(in.db.Context(), in.db.InsertRctStm(), + txID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID) + if err != nil { + return 0, fmt.Errorf("error upserting receipt_cids entry: %w", err) + } + indexerMetrics.receipts.Inc(1) + return receiptID, nil +} + +func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel, receiptID int64) error { + for _, log := range logs { + _, err := tx.Exec(in.db.Context(), in.db.InsertLogStm(), + 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) + } + indexerMetrics.logs.Inc(1) + } + return nil +} + +func (in *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel, headerID int64) (int64, error) { + var stateID int64 + var stateKey string + if stateNode.StateKey != nullHash.String() { + stateKey = stateNode.StateKey + } + err := tx.QueryRow(in.db.Context(), in.db.InsertStateStm(), + headerID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey).Scan(&stateID) + if err != nil { + return 0, fmt.Errorf("error upserting state_cids entry: %v", err) + } + return stateID, nil +} + +func (in *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel, stateID int64) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertAccountStm(), + stateID, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) + if err != nil { + return fmt.Errorf("error upserting state_accounts entry: %v", err) + } + return nil +} + +func (in *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel, stateID int64) error { + var storageKey string + if storageCID.StorageKey != nullHash.String() { + storageKey = storageCID.StorageKey + } + _, err := tx.Exec(in.db.Context(), in.db.InsertStorageStm(), + stateID, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) + if err != nil { + return fmt.Errorf("error upserting storage_cids entry: %v", err) + } + return nil +} diff --git a/statediff/indexer/helpers.go b/statediff/indexer/helpers.go deleted file mode 100644 index 4e4f30c19..000000000 --- a/statediff/indexer/helpers.go +++ /dev/null @@ -1,45 +0,0 @@ -// 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 . - -package indexer - -import ( - "fmt" - - "github.com/ethereum/go-ethereum/params" - "github.com/ethereum/go-ethereum/statediff/types" -) - -// ResolveFromNodeType wrapper around NodeType.Int() so that we maintain backwards compatibility -func ResolveFromNodeType(nodeType types.NodeType) int { - return nodeType.Int() -} - -// ChainConfig returns the appropriate ethereum chain config for the provided chain id -func ChainConfig(chainID uint64) (*params.ChainConfig, error) { - switch chainID { - case 1: - return params.MainnetChainConfig, nil - case 3: - return params.RopstenChainConfig, nil - case 4: - return params.RinkebyChainConfig, nil - case 5: - return params.GoerliChainConfig, nil - default: - return nil, fmt.Errorf("chain config for chainid %d not available", chainID) - } -} diff --git a/statediff/indexer/interfaces/interfaces.go b/statediff/indexer/interfaces/interfaces.go new file mode 100644 index 000000000..d32c117eb --- /dev/null +++ b/statediff/indexer/interfaces/interfaces.go @@ -0,0 +1,46 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package interfaces + +import ( + "io" + "math/big" + "time" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" + sdtypes "github.com/ethereum/go-ethereum/statediff/types" +) + +// StateDiffIndexer interface required to index statediff data +type StateDiffIndexer interface { + PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (Batch, error) + PushStateNode(tx Batch, stateNode sdtypes.StateNode) error + PushCodeAndCodeHash(tx Batch, codeAndCodeHash sdtypes.CodeAndCodeHash) error + ReportDBMetrics(delay time.Duration, quit <-chan bool) + io.Closer +} + +// Batch required for indexing data atomically +type Batch interface { + Submit(err error) error +} + +// Config used to configure different underlying implementations +type Config interface { + Type() shared.DBType +} diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12252078 b/statediff/indexer/ipld/eip2930_test_data/eth-block-12252078 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12252078 rename to statediff/indexer/ipld/eip2930_test_data/eth-block-12252078 diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12365585 b/statediff/indexer/ipld/eip2930_test_data/eth-block-12365585 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12365585 rename to statediff/indexer/ipld/eip2930_test_data/eth-block-12365585 diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12365586 b/statediff/indexer/ipld/eip2930_test_data/eth-block-12365586 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-block-12365586 rename to statediff/indexer/ipld/eip2930_test_data/eth-block-12365586 diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12252078 b/statediff/indexer/ipld/eip2930_test_data/eth-receipts-12252078 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12252078 rename to statediff/indexer/ipld/eip2930_test_data/eth-receipts-12252078 diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12365585 b/statediff/indexer/ipld/eip2930_test_data/eth-receipts-12365585 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12365585 rename to statediff/indexer/ipld/eip2930_test_data/eth-receipts-12365585 diff --git a/statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12365586 b/statediff/indexer/ipld/eip2930_test_data/eth-receipts-12365586 similarity index 100% rename from statediff/indexer/ipfs/ipld/eip2930_test_data/eth-receipts-12365586 rename to statediff/indexer/ipld/eip2930_test_data/eth-receipts-12365586 diff --git a/statediff/indexer/ipfs/ipld/eth_account.go b/statediff/indexer/ipld/eth_account.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_account.go rename to statediff/indexer/ipld/eth_account.go diff --git a/statediff/indexer/ipfs/ipld/eth_account_test.go b/statediff/indexer/ipld/eth_account_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_account_test.go rename to statediff/indexer/ipld/eth_account_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_header.go b/statediff/indexer/ipld/eth_header.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_header.go rename to statediff/indexer/ipld/eth_header.go diff --git a/statediff/indexer/ipfs/ipld/eth_header_test.go b/statediff/indexer/ipld/eth_header_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_header_test.go rename to statediff/indexer/ipld/eth_header_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_log.go b/statediff/indexer/ipld/eth_log.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_log.go rename to statediff/indexer/ipld/eth_log.go diff --git a/statediff/indexer/ipfs/ipld/eth_log_trie.go b/statediff/indexer/ipld/eth_log_trie.go similarity index 98% rename from statediff/indexer/ipfs/ipld/eth_log_trie.go rename to statediff/indexer/ipld/eth_log_trie.go index 2e36f0a68..49b7a7fc6 100644 --- a/statediff/indexer/ipfs/ipld/eth_log_trie.go +++ b/statediff/indexer/ipld/eth_log_trie.go @@ -89,7 +89,7 @@ func newLogTrie() *logTrie { } // getNodes invokes the localTrie, which computes the root hash of the -// log trie and returns its database keys, to return a slice +// log trie and returns its sql keys, to return a slice // of EthLogTrie nodes. func (rt *logTrie) getNodes() ([]*EthLogTrie, error) { keys, err := rt.getKeys() diff --git a/statediff/indexer/ipfs/ipld/eth_parser.go b/statediff/indexer/ipld/eth_parser.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_parser.go rename to statediff/indexer/ipld/eth_parser.go diff --git a/statediff/indexer/ipfs/ipld/eth_parser_test.go b/statediff/indexer/ipld/eth_parser_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_parser_test.go rename to statediff/indexer/ipld/eth_parser_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_receipt.go b/statediff/indexer/ipld/eth_receipt.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_receipt.go rename to statediff/indexer/ipld/eth_receipt.go diff --git a/statediff/indexer/ipfs/ipld/eth_receipt_trie.go b/statediff/indexer/ipld/eth_receipt_trie.go similarity index 98% rename from statediff/indexer/ipfs/ipld/eth_receipt_trie.go rename to statediff/indexer/ipld/eth_receipt_trie.go index fc1480703..e187e7d9d 100644 --- a/statediff/indexer/ipfs/ipld/eth_receipt_trie.go +++ b/statediff/indexer/ipld/eth_receipt_trie.go @@ -121,7 +121,7 @@ func NewRctTrie() *rctTrie { } // GetNodes invokes the localTrie, which computes the root hash of the -// transaction trie and returns its database keys, to return a slice +// transaction trie and returns its sql keys, to return a slice // of EthRctTrie nodes. func (rt *rctTrie) GetNodes() ([]*EthRctTrie, error) { keys, err := rt.getKeys() diff --git a/statediff/indexer/ipfs/ipld/eth_state.go b/statediff/indexer/ipld/eth_state.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_state.go rename to statediff/indexer/ipld/eth_state.go diff --git a/statediff/indexer/ipfs/ipld/eth_state_test.go b/statediff/indexer/ipld/eth_state_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_state_test.go rename to statediff/indexer/ipld/eth_state_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_storage.go b/statediff/indexer/ipld/eth_storage.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_storage.go rename to statediff/indexer/ipld/eth_storage.go diff --git a/statediff/indexer/ipfs/ipld/eth_storage_test.go b/statediff/indexer/ipld/eth_storage_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_storage_test.go rename to statediff/indexer/ipld/eth_storage_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_tx.go b/statediff/indexer/ipld/eth_tx.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_tx.go rename to statediff/indexer/ipld/eth_tx.go diff --git a/statediff/indexer/ipfs/ipld/eth_tx_test.go b/statediff/indexer/ipld/eth_tx_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_tx_test.go rename to statediff/indexer/ipld/eth_tx_test.go diff --git a/statediff/indexer/ipfs/ipld/eth_tx_trie.go b/statediff/indexer/ipld/eth_tx_trie.go similarity index 98% rename from statediff/indexer/ipfs/ipld/eth_tx_trie.go rename to statediff/indexer/ipld/eth_tx_trie.go index 7e79ff164..943cf15ae 100644 --- a/statediff/indexer/ipfs/ipld/eth_tx_trie.go +++ b/statediff/indexer/ipld/eth_tx_trie.go @@ -121,7 +121,7 @@ func newTxTrie() *txTrie { } // getNodes invokes the localTrie, which computes the root hash of the -// transaction trie and returns its database keys, to return a slice +// transaction trie and returns its sql keys, to return a slice // of EthTxTrie nodes. func (tt *txTrie) getNodes() ([]*EthTxTrie, error) { keys, err := tt.getKeys() diff --git a/statediff/indexer/ipfs/ipld/eth_tx_trie_test.go b/statediff/indexer/ipld/eth_tx_trie_test.go similarity index 100% rename from statediff/indexer/ipfs/ipld/eth_tx_trie_test.go rename to statediff/indexer/ipld/eth_tx_trie_test.go diff --git a/statediff/indexer/ipfs/ipld/shared.go b/statediff/indexer/ipld/shared.go similarity index 96% rename from statediff/indexer/ipfs/ipld/shared.go rename to statediff/indexer/ipld/shared.go index 993e00b42..e5c22a3c6 100644 --- a/statediff/indexer/ipfs/ipld/shared.go +++ b/statediff/indexer/ipld/shared.go @@ -27,7 +27,7 @@ import ( "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" + sdtrie "github.com/ethereum/go-ethereum/statediff/trie_helpers" sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/ethereum/go-ethereum/trie" ) @@ -143,7 +143,7 @@ func (lt *localTrie) commit() error { return nil } -// getKeys returns the stored keys of the memory database +// getKeys returns the stored keys of the memory sql // of the localTrie for further processing. func (lt *localTrie) getKeys() ([][]byte, error) { if err := lt.commit(); err != nil { @@ -167,7 +167,7 @@ type nodeKey struct { TrieKey []byte } -// getLeafKeys returns the stored leaf keys from the memory database +// getLeafKeys returns the stored leaf keys from the memory sql // of the localTrie for further processing. func (lt *localTrie) getLeafKeys() ([]*nodeKey, error) { if err := lt.commit(); err != nil { diff --git a/statediff/indexer/ipfs/ipld/test_data/error-tx-eth-block-body-json-999999 b/statediff/indexer/ipld/test_data/error-tx-eth-block-body-json-999999 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/error-tx-eth-block-body-json-999999 rename to statediff/indexer/ipld/test_data/error-tx-eth-block-body-json-999999 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-0 b/statediff/indexer/ipld/test_data/eth-block-body-json-0 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-0 rename to statediff/indexer/ipld/test_data/eth-block-body-json-0 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-4139497 b/statediff/indexer/ipld/test_data/eth-block-body-json-4139497 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-4139497 rename to statediff/indexer/ipld/test_data/eth-block-body-json-4139497 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-997522 b/statediff/indexer/ipld/test_data/eth-block-body-json-997522 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-997522 rename to statediff/indexer/ipld/test_data/eth-block-body-json-997522 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-999998 b/statediff/indexer/ipld/test_data/eth-block-body-json-999998 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-999998 rename to statediff/indexer/ipld/test_data/eth-block-body-json-999998 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-999999 b/statediff/indexer/ipld/test_data/eth-block-body-json-999999 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-json-999999 rename to statediff/indexer/ipld/test_data/eth-block-body-json-999999 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-rlp-997522 b/statediff/indexer/ipld/test_data/eth-block-body-rlp-997522 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-rlp-997522 rename to statediff/indexer/ipld/test_data/eth-block-body-rlp-997522 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-body-rlp-999999 b/statediff/indexer/ipld/test_data/eth-block-body-rlp-999999 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-body-rlp-999999 rename to statediff/indexer/ipld/test_data/eth-block-body-rlp-999999 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999996 b/statediff/indexer/ipld/test_data/eth-block-header-rlp-999996 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999996 rename to statediff/indexer/ipld/test_data/eth-block-header-rlp-999996 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999997 b/statediff/indexer/ipld/test_data/eth-block-header-rlp-999997 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999997 rename to statediff/indexer/ipld/test_data/eth-block-header-rlp-999997 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999999 b/statediff/indexer/ipld/test_data/eth-block-header-rlp-999999 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-block-header-rlp-999999 rename to statediff/indexer/ipld/test_data/eth-block-header-rlp-999999 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-0e8b34 b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-0e8b34 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-0e8b34 rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-0e8b34 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-56864f b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-56864f similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-56864f rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-56864f diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-6fc2d7 b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-6fc2d7 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-6fc2d7 rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-6fc2d7 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-727994 b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-727994 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-727994 rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-727994 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-c9070d b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-c9070d similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-c9070d rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-c9070d diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-d5be90 b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-d5be90 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-d5be90 rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-d5be90 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-d7f897 b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-d7f897 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-d7f897 rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-d7f897 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-eb2f5f b/statediff/indexer/ipld/test_data/eth-state-trie-rlp-eb2f5f similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-state-trie-rlp-eb2f5f rename to statediff/indexer/ipld/test_data/eth-state-trie-rlp-eb2f5f diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-000dd0 b/statediff/indexer/ipld/test_data/eth-storage-trie-rlp-000dd0 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-000dd0 rename to statediff/indexer/ipld/test_data/eth-storage-trie-rlp-000dd0 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-113049 b/statediff/indexer/ipld/test_data/eth-storage-trie-rlp-113049 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-113049 rename to statediff/indexer/ipld/test_data/eth-storage-trie-rlp-113049 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-9d1860 b/statediff/indexer/ipld/test_data/eth-storage-trie-rlp-9d1860 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-9d1860 rename to statediff/indexer/ipld/test_data/eth-storage-trie-rlp-9d1860 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-ffbcad b/statediff/indexer/ipld/test_data/eth-storage-trie-rlp-ffbcad similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-ffbcad rename to statediff/indexer/ipld/test_data/eth-storage-trie-rlp-ffbcad diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-ffc25c b/statediff/indexer/ipld/test_data/eth-storage-trie-rlp-ffc25c similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-storage-trie-rlp-ffc25c rename to statediff/indexer/ipld/test_data/eth-storage-trie-rlp-ffc25c diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-uncle-json-997522-0 b/statediff/indexer/ipld/test_data/eth-uncle-json-997522-0 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-uncle-json-997522-0 rename to statediff/indexer/ipld/test_data/eth-uncle-json-997522-0 diff --git a/statediff/indexer/ipfs/ipld/test_data/eth-uncle-json-997522-1 b/statediff/indexer/ipld/test_data/eth-uncle-json-997522-1 similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/eth-uncle-json-997522-1 rename to statediff/indexer/ipld/test_data/eth-uncle-json-997522-1 diff --git a/statediff/indexer/ipfs/ipld/test_data/tx_data b/statediff/indexer/ipld/test_data/tx_data similarity index 100% rename from statediff/indexer/ipfs/ipld/test_data/tx_data rename to statediff/indexer/ipld/test_data/tx_data diff --git a/statediff/indexer/ipfs/ipld/trie_node.go b/statediff/indexer/ipld/trie_node.go similarity index 100% rename from statediff/indexer/ipfs/ipld/trie_node.go rename to statediff/indexer/ipld/trie_node.go diff --git a/statediff/indexer/mocks/test_data.go b/statediff/indexer/mocks/test_data.go index 2d544b6ea..f437dc8e4 100644 --- a/statediff/indexer/mocks/test_data.go +++ b/statediff/indexer/mocks/test_data.go @@ -32,7 +32,7 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/rlp" - "github.com/ethereum/go-ethereum/statediff/testhelpers" + "github.com/ethereum/go-ethereum/statediff/test_helpers" sdtypes "github.com/ethereum/go-ethereum/statediff/types" ) @@ -111,7 +111,7 @@ var ( nonce1 = uint64(1) ContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0" ContractCodeHash = common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea") - ContractLeafKey = testhelpers.AddressToLeafKey(ContractAddress) + ContractLeafKey = test_helpers.AddressToLeafKey(ContractAddress) ContractAccount, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: nonce1, Balance: big.NewInt(0), @@ -127,8 +127,8 @@ var ( nonce0 = uint64(0) AccountRoot = "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421" AccountCodeHash = common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470") - AccountLeafKey = testhelpers.Account2LeafKey - RemovedLeafKey = testhelpers.Account1LeafKey + AccountLeafKey = test_helpers.Account2LeafKey + RemovedLeafKey = test_helpers.Account1LeafKey Account, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: nonce0, Balance: big.NewInt(1000), diff --git a/statediff/indexer/models/batch.go b/statediff/indexer/models/batch.go index f780a9b0d..48b2944e0 100644 --- a/statediff/indexer/models/batch.go +++ b/statediff/indexer/models/batch.go @@ -1,3 +1,19 @@ +// VulcanizeDB +// Copyright © 2021 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 . + package models import "github.com/lib/pq" diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index 5837488f8..72efe2383 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -20,8 +20,8 @@ import "github.com/lib/pq" // IPLDModel is the db model for public.blocks type IPLDModel struct { - Key string - Data []byte + Key string `db:"key"` + Data []byte `db:"data"` } // HeaderModel is the db model for eth.header_cids diff --git a/statediff/indexer/postgres/config.go b/statediff/indexer/postgres/config.go deleted file mode 100644 index c2de0a6bf..000000000 --- a/statediff/indexer/postgres/config.go +++ /dev/null @@ -1,59 +0,0 @@ -// 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 . - -package postgres - -import ( - "fmt" -) - -// Env variables -const ( - DATABASE_NAME = "DATABASE_NAME" - DATABASE_HOSTNAME = "DATABASE_HOSTNAME" - DATABASE_PORT = "DATABASE_PORT" - DATABASE_USER = "DATABASE_USER" - DATABASE_PASSWORD = "DATABASE_PASSWORD" - DATABASE_MAX_IDLE_CONNECTIONS = "DATABASE_MAX_IDLE_CONNECTIONS" - DATABASE_MAX_OPEN_CONNECTIONS = "DATABASE_MAX_OPEN_CONNECTIONS" - DATABASE_MAX_CONN_LIFETIME = "DATABASE_MAX_CONN_LIFETIME" -) - -type ConnectionParams struct { - Hostname string - Name string - User string - Password string - Port int -} - -type ConnectionConfig struct { - MaxIdle int - MaxOpen int - MaxLifetime int -} - -func DbConnectionString(params ConnectionParams) string { - if len(params.User) > 0 && len(params.Password) > 0 { - return fmt.Sprintf("postgresql://%s:%s@%s:%d/%s?sslmode=disable", - params.User, params.Password, params.Hostname, params.Port, params.Name) - } - if len(params.User) > 0 && len(params.Password) == 0 { - return fmt.Sprintf("postgresql://%s@%s:%d/%s?sslmode=disable", - params.User, params.Hostname, params.Port, params.Name) - } - return fmt.Sprintf("postgresql://%s:%d/%s?sslmode=disable", params.Hostname, params.Port, params.Name) -} diff --git a/statediff/indexer/postgres/postgres.go b/statediff/indexer/postgres/postgres.go deleted file mode 100644 index 455dac306..000000000 --- a/statediff/indexer/postgres/postgres.go +++ /dev/null @@ -1,76 +0,0 @@ -// 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 . - -package postgres - -import ( - "time" - - "github.com/jmoiron/sqlx" - _ "github.com/lib/pq" //postgres driver - - "github.com/ethereum/go-ethereum/statediff/indexer/node" -) - -type DB struct { - *sqlx.DB - Node node.Info - NodeID int64 -} - -func NewDB(connectString string, config ConnectionConfig, node node.Info) (*DB, error) { - db, connectErr := sqlx.Connect("postgres", connectString) - if connectErr != nil { - return &DB{}, ErrDBConnectionFailed(connectErr) - } - if config.MaxOpen > 0 { - db.SetMaxOpenConns(config.MaxOpen) - } - if config.MaxIdle > 0 { - db.SetMaxIdleConns(config.MaxIdle) - } - if config.MaxLifetime > 0 { - lifetime := time.Duration(config.MaxLifetime) * time.Second - db.SetConnMaxLifetime(lifetime) - } - pg := DB{DB: db, Node: node} - nodeErr := pg.CreateNode(&node) - if nodeErr != nil { - return &DB{}, ErrUnableToSetNode(nodeErr) - } - return &pg, nil -} - -func (db *DB) CreateNode(node *node.Info) error { - var nodeID int64 - err := db.QueryRow( - `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) - VALUES ($1, $2, $3, $4, $5) - ON CONFLICT (genesis_block, network_id, node_id, chain_id) - DO UPDATE - SET genesis_block = $1, - network_id = $2, - node_id = $3, - client_name = $4, - chain_id = $5 - RETURNING id`, - node.GenesisBlock, node.NetworkID, node.ID, node.ClientName, node.ChainID).Scan(&nodeID) - if err != nil { - return ErrUnableToSetNode(err) - } - db.NodeID = nodeID - return nil -} diff --git a/statediff/indexer/shared/chain_type.go b/statediff/indexer/shared/chain_type.go deleted file mode 100644 index c3dedfe38..000000000 --- a/statediff/indexer/shared/chain_type.go +++ /dev/null @@ -1,78 +0,0 @@ -// 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 . - -package shared - -import ( - "errors" - "strings" -) - -// ChainType enum for specifying blockchain -type ChainType int - -const ( - UnknownChain ChainType = iota - Ethereum - Bitcoin - Omni - EthereumClassic -) - -func (c ChainType) String() string { - switch c { - case Ethereum: - return "Ethereum" - case Bitcoin: - return "Bitcoin" - case Omni: - return "Omni" - case EthereumClassic: - return "EthereumClassic" - default: - return "" - } -} - -func (c ChainType) API() string { - switch c { - case Ethereum: - return "eth" - case Bitcoin: - return "btc" - case Omni: - return "omni" - case EthereumClassic: - return "etc" - default: - return "" - } -} - -func NewChainType(name string) (ChainType, error) { - switch strings.ToLower(name) { - case "ethereum", "eth": - return Ethereum, nil - case "bitcoin", "btc", "xbt": - return Bitcoin, nil - case "omni": - return Omni, nil - case "classic", "etc": - return EthereumClassic, nil - default: - return UnknownChain, errors.New("invalid name for chain") - } -} diff --git a/statediff/indexer/shared/constants.go b/statediff/indexer/shared/constants.go index 3dc2994c4..6d1e298ad 100644 --- a/statediff/indexer/shared/constants.go +++ b/statediff/indexer/shared/constants.go @@ -1,5 +1,5 @@ // VulcanizeDB -// Copyright © 2019 Vulcanize +// Copyright © 2021 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 @@ -17,6 +17,7 @@ package shared const ( - DefaultMaxBatchSize uint64 = 100 - DefaultMaxBatchNumber int64 = 50 + RemovedNodeStorageCID = "bagmacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya" + RemovedNodeStateCID = "baglacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya" + RemovedNodeMhKey = "/blocks/DMQMLUSGAGDPOIZ4SJ7H3MW4Y4B4BZIAWZJ4VARHHN57VWAELWC2I4A" ) diff --git a/statediff/indexer/shared/data_type.go b/statediff/indexer/shared/data_type.go deleted file mode 100644 index ccab92c1e..000000000 --- a/statediff/indexer/shared/data_type.go +++ /dev/null @@ -1,102 +0,0 @@ -// 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 . - -package shared - -import ( - "fmt" - "strings" -) - -// DataType is an enum to loosely represent type of chain data -type DataType int - -const ( - UnknownDataType DataType = iota - 1 - Full - Headers - Uncles - Transactions - Receipts - State - Storage -) - -// String() method to resolve ReSyncType enum -func (r DataType) String() string { - switch r { - case Full: - return "full" - case Headers: - return "headers" - case Uncles: - return "uncles" - case Transactions: - return "transactions" - case Receipts: - return "receipts" - case State: - return "state" - case Storage: - return "storage" - default: - return "unknown" - } -} - -// GenerateDataTypeFromString returns a DataType from a provided string -func GenerateDataTypeFromString(str string) (DataType, error) { - switch strings.ToLower(str) { - case "full", "f": - return Full, nil - case "headers", "header", "h": - return Headers, nil - case "uncles", "u": - return Uncles, nil - case "transactions", "transaction", "trxs", "txs", "trx", "tx", "t": - return Transactions, nil - case "receipts", "receipt", "rcts", "rct", "r": - return Receipts, nil - case "state": - return State, nil - case "storage": - return Storage, nil - default: - return UnknownDataType, fmt.Errorf("unrecognized resync type: %s", str) - } -} - -// SupportedDataType returns whether a DataType is supported -func SupportedDataType(d DataType) (bool, error) { - switch d { - case Full: - return true, nil - case Headers: - return true, nil - case Uncles: - return true, nil - case Transactions: - return true, nil - case Receipts: - return true, nil - case State: - return true, nil - case Storage: - return true, nil - default: - return true, nil - } -} diff --git a/statediff/indexer/ipfs/models.go b/statediff/indexer/shared/db_kind.go similarity index 83% rename from statediff/indexer/ipfs/models.go rename to statediff/indexer/shared/db_kind.go index eb0312beb..711f9d050 100644 --- a/statediff/indexer/ipfs/models.go +++ b/statediff/indexer/shared/db_kind.go @@ -1,5 +1,5 @@ // VulcanizeDB -// Copyright © 2019 Vulcanize +// Copyright © 2021 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 @@ -14,9 +14,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package ipfs +package shared -type BlockModel struct { - CID string `db:"key"` - Data []byte `db:"data"` -} +type DBType string + +const ( + POSTGRES DBType = "Postgres" + DUMP DBType = "Dump" +) diff --git a/statediff/indexer/shared/functions.go b/statediff/indexer/shared/functions.go index 7823d8f78..8b0acbb54 100644 --- a/statediff/indexer/shared/functions.go +++ b/statediff/indexer/shared/functions.go @@ -18,19 +18,12 @@ package shared import ( "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" - "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" - "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" ) -// IPLDInsertPgStr is the postgres statement string for IPLDs inserting into public.blocks -const IPLDInsertPgStr = `INSERT INTO public.blocks (key, data) VALUES ($1, $2) ON CONFLICT (key) DO NOTHING` - // HandleZeroAddrPointer will return an empty string for a nil address pointer func HandleZeroAddrPointer(to *common.Address) string { if to == nil { @@ -47,13 +40,6 @@ func HandleZeroAddr(to common.Address) string { return to.Hex() } -// Rollback sql transaction and log any error -func Rollback(tx *sqlx.Tx) { - if err := tx.Rollback(); err != nil { - log.Error(err.Error()) - } -} - // MultihashKeyFromCID converts a cid into a blockstore-prefixed multihash db key string func MultihashKeyFromCID(c cid.Cid) string { dbKey := dshelp.MultihashToDsKey(c.Hash()) @@ -69,9 +55,3 @@ func MultihashKeyFromKeccak256(hash common.Hash) (string, error) { dbKey := dshelp.MultihashToDsKey(mh) return blockstore.BlockPrefix.String() + dbKey.String(), nil } - -// PublishDirectWithDB diretly writes a previously derived mhkey => value pair to the ipld database -func PublishDirectWithDB(db *postgres.DB, key string, value []byte) error { - _, err := db.Exec(IPLDInsertPgStr, key, value) - return err -} diff --git a/statediff/indexer/reward.go b/statediff/indexer/shared/reward.go similarity index 99% rename from statediff/indexer/reward.go rename to statediff/indexer/shared/reward.go index 47e3f17b9..3d5752e25 100644 --- a/statediff/indexer/reward.go +++ b/statediff/indexer/shared/reward.go @@ -14,7 +14,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package indexer +package shared import ( "math/big" diff --git a/statediff/indexer/shared/types.go b/statediff/indexer/shared/types.go deleted file mode 100644 index 1337ba68a..000000000 --- a/statediff/indexer/shared/types.go +++ /dev/null @@ -1,44 +0,0 @@ -// 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 . - -package shared - -import ( - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/statediff/indexer/models" - "github.com/ethereum/go-ethereum/statediff/types" -) - -// TrieNode struct used to flag node as leaf or not -type TrieNode struct { - Path []byte - LeafKey common.Hash - Value []byte - Type types.NodeType -} - -// CIDPayload is a struct to hold all the CIDs and their associated meta data for indexing in Postgres -// Returned by IPLDPublisher -// Passed to CIDIndexer -type CIDPayload struct { - HeaderCID models.HeaderModel - UncleCIDs []models.UncleModel - TransactionCIDs []models.TxModel - ReceiptCIDs map[common.Hash]models.ReceiptModel - StateNodeCIDs []models.StateNodeModel - StateAccounts map[string]models.StateAccountModel - StorageNodeCIDs map[string][]models.StorageNodeModel -} diff --git a/statediff/indexer/shared/test_helpers.go b/statediff/indexer/test_helpers/test_helpers.go similarity index 59% rename from statediff/indexer/shared/test_helpers.go rename to statediff/indexer/test_helpers/test_helpers.go index d54998cd5..b519d80b5 100644 --- a/statediff/indexer/shared/test_helpers.go +++ b/statediff/indexer/test_helpers/test_helpers.go @@ -14,37 +14,20 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package shared +package test_helpers import ( "reflect" "testing" - - "github.com/ipfs/go-cid" - "github.com/multiformats/go-multihash" - - "github.com/ethereum/go-ethereum/statediff/indexer/node" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" ) +// ExpectEqual asserts the provided interfaces are deep equal func ExpectEqual(t *testing.T, got interface{}, want interface{}) { if !reflect.DeepEqual(got, want) { t.Fatalf("Expected: %v\nActual: %v", want, got) } } -// SetupDB is use to setup a db for watcher tests -func SetupDB() (*postgres.DB, error) { - uri := postgres.DbConnectionString(postgres.ConnectionParams{ - User: "vdbm", - Password: "password", - Hostname: "localhost", - Name: "vulcanize_public", - Port: 5432, - }) - return postgres.NewDB(uri, postgres.ConnectionConfig{}, node.Info{}) -} - // ListContainsString used to check if a list of strings contains a particular string func ListContainsString(sss []string, s string) bool { for _, str := range sss { @@ -54,15 +37,3 @@ func ListContainsString(sss []string, s string) bool { } return false } - -// TestCID creates a basic CID for testing purposes -func TestCID(b []byte) cid.Cid { - pref := cid.Prefix{ - Version: 1, - Codec: cid.Raw, - MhType: multihash.KECCAK_256, - MhLength: -1, - } - c, _ := pref.Sum(b) - return c -} diff --git a/statediff/indexer/writer.go b/statediff/indexer/writer.go deleted file mode 100644 index cbc058896..000000000 --- a/statediff/indexer/writer.go +++ /dev/null @@ -1,158 +0,0 @@ -// 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 . - -package indexer - -import ( - "fmt" - - "github.com/jmoiron/sqlx" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/statediff/indexer/models" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" -) - -var ( - nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") -) - -// PostgresCIDWriter handles processing and writing of indexed IPLD objects to Postgres -type PostgresCIDWriter struct { - db *postgres.DB -} - -// NewPostgresCIDWriter creates a new pointer to a PostgresCIDWriter -func NewPostgresCIDWriter(db *postgres.DB) *PostgresCIDWriter { - return &PostgresCIDWriter{ - db: db, - } -} - -func (in *PostgresCIDWriter) upsertHeaderCID(tx *sqlx.Tx, header models.HeaderModel) (int64, error) { - var headerID int64 - err := tx.QueryRowx(`INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) - VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) - ON CONFLICT (block_number, block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) - RETURNING id`, - header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, in.db.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee).Scan(&headerID) - if err != nil { - return 0, fmt.Errorf("error upserting header_cids entry: %v", err) - } - indexerMetrics.blocks.Inc(1) - return headerID, nil -} - -func (in *PostgresCIDWriter) upsertUncleCID(tx *sqlx.Tx, uncle models.UncleModel, headerID int64) error { - _, err := tx.Exec(`INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES ($1, $2, $3, $4, $5, $6) - ON CONFLICT (header_id, block_hash) DO UPDATE SET (parent_hash, cid, reward, mh_key) = ($3, $4, $5, $6)`, - uncle.BlockHash, headerID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey) - if err != nil { - return fmt.Errorf("error upserting uncle_cids entry: %v", err) - } - return nil -} - -func (in *PostgresCIDWriter) upsertTransactionCID(tx *sqlx.Tx, transaction models.TxModel, headerID int64) (int64, error) { - var txID int64 - err := tx.QueryRowx(`INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) - ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = ($3, $4, $5, $6, $7, $8, $9) - RETURNING id`, - headerID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type).Scan(&txID) - if err != nil { - return 0, fmt.Errorf("error upserting transaction_cids entry: %v", err) - } - indexerMetrics.transactions.Inc(1) - return txID, nil -} - -func (in *PostgresCIDWriter) upsertAccessListElement(tx *sqlx.Tx, accessListElement models.AccessListElementModel, txID int64) error { - _, err := tx.Exec(`INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) - ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)`, - txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) - if err != nil { - return fmt.Errorf("error upserting access_list_element entry: %v", err) - } - indexerMetrics.accessListEntries.Inc(1) - return nil -} - -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, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) - ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8) - RETURNING id`, - txID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID) - if err != nil { - 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) - } - indexerMetrics.logs.Inc(1) - } - return nil -} - -func (in *PostgresCIDWriter) upsertStateCID(tx *sqlx.Tx, stateNode models.StateNodeModel, headerID int64) (int64, error) { - var stateID int64 - var stateKey string - if stateNode.StateKey != nullHash.String() { - stateKey = stateNode.StateKey - } - err := tx.QueryRowx(`INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) - RETURNING id`, - headerID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey).Scan(&stateID) - if err != nil { - return 0, fmt.Errorf("error upserting state_cids entry: %v", err) - } - return stateID, nil -} - -func (in *PostgresCIDWriter) upsertStateAccount(tx *sqlx.Tx, stateAccount models.StateAccountModel, stateID int64) error { - _, err := tx.Exec(`INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5) - ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = ($2, $3, $4, $5)`, - stateID, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) - if err != nil { - return fmt.Errorf("error upserting state_accounts entry: %v", err) - } - return nil -} - -func (in *PostgresCIDWriter) upsertStorageCID(tx *sqlx.Tx, storageCID models.StorageNodeModel, stateID int64) error { - var storageKey string - if storageCID.StorageKey != nullHash.String() { - storageKey = storageCID.StorageKey - } - _, err := tx.Exec(`INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)`, - stateID, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) - if err != nil { - return fmt.Errorf("error upserting storage_cids entry: %v", err) - } - return nil -} diff --git a/statediff/mainnet_tests/builder_test.go b/statediff/mainnet_tests/builder_test.go index 859f00489..d838302e0 100644 --- a/statediff/mainnet_tests/builder_test.go +++ b/statediff/mainnet_tests/builder_test.go @@ -37,7 +37,7 @@ import ( "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/statediff" - "github.com/ethereum/go-ethereum/statediff/testhelpers" + "github.com/ethereum/go-ethereum/statediff/test_helpers" sdtypes "github.com/ethereum/go-ethereum/statediff/types" ) @@ -53,8 +53,8 @@ var ( block1CoinbaseAccount, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(5000000000000000000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block1CoinbaseLeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("38251692195afc818c92b485fcb8a4691af89cbe5a2ab557b83a4261be2a9a"), @@ -125,8 +125,8 @@ var ( block2CoinbaseAccount, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: big.NewInt(5000000000000000000), - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block2CoinbaseLeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("20679cbcf198c1741a6f4e4473845659a30caa8b26f8d37a0be2e2bc0d8892"), @@ -137,8 +137,8 @@ var ( block2MovedPremineAccount, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: block2MovedPremineBalance, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block2MovedPremineLeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("20f2e24db7943eab4415f99e109698863b0fecca1cf9ffc500f38cefbbe29e"), @@ -231,8 +231,8 @@ var ( block3CoinbaseAccount, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: blcok3CoinbaseBalance, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block3CoinbaseLeafNode, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3a174f00e64521a535f35e67c1aa241951c791639b2f3d060f49c5d9fa8b9e"), @@ -244,8 +244,8 @@ var ( block3MovedPremineAccount1, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: block3MovedPremineBalance1, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block3MovedPremineLeafNode1, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("3ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190"), // ce573ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190 @@ -257,8 +257,8 @@ var ( block3MovedPremineAccount2, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: 0, Balance: block3MovedPremineBalance2, - CodeHash: testhelpers.NullCodeHash.Bytes(), - Root: testhelpers.EmptyContractRoot, + CodeHash: test_helpers.NullCodeHash.Bytes(), + Root: test_helpers.EmptyContractRoot, }) block3MovedPremineLeafNode2, _ = rlp.EncodeToBytes([]interface{}{ common.Hex2Bytes("33bc1e69eedf90f402e11f6862da14ed8e50156635a04d6393bbae154012"), // ce5783bc1e69eedf90f402e11f6862da14ed8e50156635a04d6393bbae154012 @@ -480,7 +480,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { var tests = []struct { name string startingArguments statediff.Args - expected *statediff.StateObject + expected *sdtypes.StateObject }{ // note that block0 (genesis) has over 1000 nodes due to the pre-allocation for the crowd-sale // it is not feasible to write a unit test of that size at this time @@ -493,7 +493,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { BlockNumber: block1.Number(), BlockHash: block1.Hash(), }, - &statediff.StateObject{ + &sdtypes.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), Nodes: []sdtypes.StateNode{ @@ -536,7 +536,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { BlockNumber: block2.Number(), BlockHash: block2.Hash(), }, - &statediff.StateObject{ + &sdtypes.StateObject{ BlockNumber: block2.Number(), BlockHash: block2.Hash(), Nodes: []sdtypes.StateNode{ @@ -594,7 +594,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { BlockNumber: block3.Number(), BlockHash: block3.Hash(), }, - &statediff.StateObject{ + &sdtypes.StateObject{ BlockNumber: block3.Number(), BlockHash: block3.Hash(), Nodes: []sdtypes.StateNode{ diff --git a/statediff/metrics.go b/statediff/metrics.go index 7e7d6e328..afc80e40e 100644 --- a/statediff/metrics.go +++ b/statediff/metrics.go @@ -1,3 +1,19 @@ +// VulcanizeDB +// Copyright © 2021 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 . + package statediff import ( diff --git a/statediff/payload.go b/statediff/payload.go new file mode 100644 index 000000000..233141278 --- /dev/null +++ b/statediff/payload.go @@ -0,0 +1,57 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package statediff + +import ( + "encoding/json" + "math/big" +) + +// Payload packages the data to send to statediff subscriptions +type Payload struct { + BlockRlp []byte `json:"blockRlp"` + TotalDifficulty *big.Int `json:"totalDifficulty"` + ReceiptsRlp []byte `json:"receiptsRlp"` + StateObjectRlp []byte `json:"stateObjectRlp" gencodec:"required"` + + encoded []byte + err error +} + +func (sd *Payload) ensureEncoded() { + if sd.encoded == nil && sd.err == nil { + sd.encoded, sd.err = json.Marshal(sd) + } +} + +// Length to implement Encoder interface for Payload +func (sd *Payload) Length() int { + sd.ensureEncoded() + return len(sd.encoded) +} + +// Encode to implement Encoder interface for Payload +func (sd *Payload) Encode() ([]byte, error) { + sd.ensureEncoded() + return sd.encoded, sd.err +} + +// Subscription struct holds our subscription channels +type Subscription struct { + PayloadChan chan<- Payload + QuitChan chan<- bool +} diff --git a/statediff/service.go b/statediff/service.go index de6e84a65..ae2e34c6c 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -41,9 +41,9 @@ import ( "github.com/ethereum/go-ethereum/trie" ind "github.com/ethereum/go-ethereum/statediff/indexer" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" - "github.com/ethereum/go-ethereum/statediff/indexer/postgres" - . "github.com/ethereum/go-ethereum/statediff/types" + types2 "github.com/ethereum/go-ethereum/statediff/types" ) const chainEventChanSize = 20000 @@ -72,41 +72,32 @@ type blockChain interface { // IService is the state-diffing service interface type IService interface { - // Start() and Stop() + // Lifecycle Start() and Stop() methods node.Lifecycle - // Method to getting API(s) for this service + // APIs method for getting API(s) for this service APIs() []rpc.API - // Main event loop for processing state diffs + // Loop is the main event loop for processing state diffs Loop(chainEventCh chan core.ChainEvent) - // Method to subscribe to receive state diff processing output + // Subscribe method to subscribe to receive state diff processing output` Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool, params Params) - // Method to unsubscribe from state diff processing + // Unsubscribe method to unsubscribe from state diff processing Unsubscribe(id rpc.ID) error - // Method to get state diff object at specific block + // StateDiffAt method to get state diff object at specific block StateDiffAt(blockNumber uint64, params Params) (*Payload, error) - // Method to get state diff object at specific block + // StateDiffFor method to get state diff object at specific block StateDiffFor(blockHash common.Hash, params Params) (*Payload, error) - // Method to get state trie object at specific block + // StateTrieAt method to get state trie object at specific block StateTrieAt(blockNumber uint64, params Params) (*Payload, error) - // Method to stream out all code and codehash pairs - StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- CodeAndCodeHash, quitChan chan<- bool) - // Method to write state diff object directly to DB + // StreamCodeAndCodeHash method to stream out all code and codehash pairs + StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- types2.CodeAndCodeHash, quitChan chan<- bool) + // WriteStateDiffAt method to write state diff object directly to DB WriteStateDiffAt(blockNumber uint64, params Params) error - // Method to write state diff object directly to DB + // WriteStateDiffFor method to write state diff object directly to DB WriteStateDiffFor(blockHash common.Hash, params Params) error - // Event loop for progressively processing and writing diffs directly to DB + // WriteLoop event loop for progressively processing and writing diffs directly to DB WriteLoop(chainEventCh chan core.ChainEvent) } -// Wraps consructor parameters -type ServiceParams struct { - DBParams *DBParams - // Whether to enable writing state diffs directly to track blochain head - EnableWriteLoop bool - // Size of the worker pool - NumWorkers uint -} - // Service is the underlying struct for the state diffing service type Service struct { // Used to sync access to the Subscriptions @@ -122,26 +113,26 @@ type Service struct { // A mapping of subscription params rlp hash to the corresponding subscription params SubscriptionTypes map[common.Hash]Params // Cache the last block so that we can avoid having to lookup the next block's parent - BlockCache blockCache + BlockCache BlockCache // Whether or not we have any subscribers; only if we do, do we processes state diffs subscribers int32 // Interface for publishing statediffs as PG-IPLD objects - indexer ind.Indexer + indexer interfaces.StateDiffIndexer // Whether to enable writing state diffs directly to track blochain head enableWriteLoop bool // Size of the worker pool numWorkers uint } -// Wrap the cached last block for safe access from different service loops -type blockCache struct { +// BlockCache caches the last block for safe access from different service loops +type BlockCache struct { sync.Mutex blocks map[common.Hash]*types.Block maxSize uint } -func NewBlockCache(max uint) blockCache { - return blockCache{ +func NewBlockCache(max uint) BlockCache { + return BlockCache{ blocks: make(map[common.Hash]*types.Block), maxSize: max, } @@ -149,29 +140,23 @@ func NewBlockCache(max uint) blockCache { // New creates a new statediff.Service // func New(stack *node.Node, ethServ *eth.Ethereum, dbParams *DBParams, enableWriteLoop bool) error { -func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params ServiceParams) error { +func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params Config) error { blockChain := ethServ.BlockChain() - var indexer ind.Indexer + var indexer interfaces.StateDiffIndexer quitCh := make(chan bool) - if params.DBParams != nil { + if params.IndexerConfig != nil { info := nodeinfo.Info{ GenesisBlock: blockChain.Genesis().Hash().Hex(), NetworkID: strconv.FormatUint(cfg.NetworkId, 10), ChainID: blockChain.Config().ChainID.Uint64(), - ID: params.DBParams.ID, - ClientName: params.DBParams.ClientName, + ID: params.ID, + ClientName: params.ClientName, } - - // TODO: pass max idle, open, lifetime? - db, err := postgres.NewDB(params.DBParams.ConnectionURL, postgres.ConnectionConfig{}, info) + var err error + indexer, err = ind.NewStateDiffIndexer(params.Context, blockChain.Config(), info, params.IndexerConfig) if err != nil { return err } - indexer, err = ind.NewStateDiffIndexer(blockChain.Config(), db) - if err != nil { - return err - } - indexer.ReportDBMetrics(10*time.Second, quitCh) } workers := params.NumWorkers @@ -214,7 +199,7 @@ func (sds *Service) APIs() []rpc.API { // Return the parent block of currentBlock, using the cached block if available; // and cache the passed block -func (lbc *blockCache) getParentBlock(currentBlock *types.Block, bc blockChain) *types.Block { +func (lbc *BlockCache) getParentBlock(currentBlock *types.Block, bc blockChain) *types.Block { lbc.Lock() parentHash := currentBlock.ParentHash() var parentBlock *types.Block @@ -590,7 +575,7 @@ func sendNonBlockingQuit(id rpc.ID, sub Subscription) { } // StreamCodeAndCodeHash subscription method for extracting all the codehash=>code mappings that exist in the trie at the provided height -func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- CodeAndCodeHash, quitChan chan<- bool) { +func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- types2.CodeAndCodeHash, quitChan chan<- bool) { current := sds.BlockChain.GetBlockByNumber(blockNumber) log.Info("sending code and codehash", "block height", blockNumber) currentTrie, err := sds.BlockChain.StateCache().OpenTrie(current.Root()) @@ -620,7 +605,7 @@ func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- Cod log.Error("error collecting contract code", "err", err) return } - outChan <- CodeAndCodeHash{ + outChan <- types2.CodeAndCodeHash{ Hash: codeHash, Code: code, } @@ -660,7 +645,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p var totalDifficulty *big.Int var receipts types.Receipts var err error - var tx *ind.BlockTx + var tx interfaces.Batch if params.IncludeTD { totalDifficulty = sds.BlockChain.GetTd(block.Hash(), block.NumberU64()) } @@ -672,14 +657,18 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p return err } // defer handling of commit/rollback for any return case - defer tx.Close(tx, err) - output := func(node StateNode) error { + defer func() { + if err := tx.Submit(err); err != nil { + log.Error("batch transaction submission failed", "err", err) + } + }() + output := func(node types2.StateNode) error { return sds.indexer.PushStateNode(tx, node) } - codeOutput := func(c CodeAndCodeHash) error { + codeOutput := func(c types2.CodeAndCodeHash) error { return sds.indexer.PushCodeAndCodeHash(tx, c) } - err = sds.Builder.WriteStateDiffObject(StateRoots{ + err = sds.Builder.WriteStateDiffObject(types2.StateRoots{ NewStateRoot: block.Root(), OldStateRoot: parentRoot, }, params, output, codeOutput) diff --git a/statediff/service_test.go b/statediff/service_test.go index ca9a483a5..a17f89217 100644 --- a/statediff/service_test.go +++ b/statediff/service_test.go @@ -24,6 +24,8 @@ import ( "sync" "testing" + types2 "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/common" @@ -32,7 +34,7 @@ import ( "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rpc" statediff "github.com/ethereum/go-ethereum/statediff" - "github.com/ethereum/go-ethereum/statediff/testhelpers/mocks" + "github.com/ethereum/go-ethereum/statediff/test_helpers/mocks" ) func TestServiceLoop(t *testing.T) { @@ -218,7 +220,7 @@ func TestGetStateDiffAt(t *testing.T) { } func testErrorInStateDiffAt(t *testing.T) { - mockStateDiff := statediff.StateObject{ + mockStateDiff := types2.StateObject{ BlockNumber: testBlock1.Number(), BlockHash: testBlock1.Hash(), } diff --git a/statediff/test_helpers/constant.go b/statediff/test_helpers/constant.go new file mode 100644 index 000000000..ba591ebb4 --- /dev/null +++ b/statediff/test_helpers/constant.go @@ -0,0 +1,33 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package test_helpers + +import ( + "math/big" + + "github.com/ethereum/go-ethereum/params" +) + +var ( + BalanceChange1000 = int64(1000) + BalanceChange10000 = int64(10000) + BalanceChange1Ether = int64(params.Ether) + Block1Account1Balance = big.NewInt(BalanceChange10000) + Block2Account2Balance = big.NewInt(21000000000000) + GasFees = int64(params.GWei) * int64(params.TxGas) + ContractGasLimit = uint64(1000000) +) diff --git a/statediff/testhelpers/helpers.go b/statediff/test_helpers/helpers.go similarity index 99% rename from statediff/testhelpers/helpers.go rename to statediff/test_helpers/helpers.go index 168d770af..8373f7537 100644 --- a/statediff/testhelpers/helpers.go +++ b/statediff/test_helpers/helpers.go @@ -14,7 +14,7 @@ // You should have received a copy of the GNU Lesser General Public License // along with the go-ethereum library. If not, see . -package testhelpers +package test_helpers import ( "math/big" diff --git a/statediff/testhelpers/mocks/blockchain.go b/statediff/test_helpers/mocks/blockchain.go similarity index 100% rename from statediff/testhelpers/mocks/blockchain.go rename to statediff/test_helpers/mocks/blockchain.go diff --git a/statediff/testhelpers/mocks/builder.go b/statediff/test_helpers/mocks/builder.go similarity index 80% rename from statediff/testhelpers/mocks/builder.go rename to statediff/test_helpers/mocks/builder.go index ff9faf3ec..e2452301a 100644 --- a/statediff/testhelpers/mocks/builder.go +++ b/statediff/test_helpers/mocks/builder.go @@ -26,15 +26,15 @@ import ( type Builder struct { Args statediff.Args Params statediff.Params - StateRoots statediff.StateRoots - stateDiff statediff.StateObject + StateRoots sdtypes.StateRoots + stateDiff sdtypes.StateObject block *types.Block - stateTrie statediff.StateObject + stateTrie sdtypes.StateObject builderError error } // BuildStateDiffObject mock method -func (builder *Builder) BuildStateDiffObject(args statediff.Args, params statediff.Params) (statediff.StateObject, error) { +func (builder *Builder) BuildStateDiffObject(args statediff.Args, params statediff.Params) (sdtypes.StateObject, error) { builder.Args = args builder.Params = params @@ -42,7 +42,7 @@ func (builder *Builder) BuildStateDiffObject(args statediff.Args, params statedi } // BuildStateDiffObject mock method -func (builder *Builder) WriteStateDiffObject(args statediff.StateRoots, params statediff.Params, output sdtypes.StateNodeSink, codeOutput sdtypes.CodeSink) error { +func (builder *Builder) WriteStateDiffObject(args sdtypes.StateRoots, params statediff.Params, output sdtypes.StateNodeSink, codeOutput sdtypes.CodeSink) error { builder.StateRoots = args builder.Params = params @@ -50,14 +50,14 @@ func (builder *Builder) WriteStateDiffObject(args statediff.StateRoots, params s } // BuildStateTrieObject mock method -func (builder *Builder) BuildStateTrieObject(block *types.Block) (statediff.StateObject, error) { +func (builder *Builder) BuildStateTrieObject(block *types.Block) (sdtypes.StateObject, error) { builder.block = block return builder.stateTrie, builder.builderError } // SetStateDiffToBuild mock method -func (builder *Builder) SetStateDiffToBuild(stateDiff statediff.StateObject) { +func (builder *Builder) SetStateDiffToBuild(stateDiff sdtypes.StateObject) { builder.stateDiff = stateDiff } diff --git a/statediff/testhelpers/mocks/service.go b/statediff/test_helpers/mocks/service.go similarity index 100% rename from statediff/testhelpers/mocks/service.go rename to statediff/test_helpers/mocks/service.go diff --git a/statediff/testhelpers/mocks/service_test.go b/statediff/test_helpers/mocks/service_test.go similarity index 93% rename from statediff/testhelpers/mocks/service_test.go rename to statediff/test_helpers/mocks/service_test.go index 8c1fd49cf..b3b77d4bf 100644 --- a/statediff/testhelpers/mocks/service_test.go +++ b/statediff/test_helpers/mocks/service_test.go @@ -30,14 +30,14 @@ import ( "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/statediff" - "github.com/ethereum/go-ethereum/statediff/testhelpers" + "github.com/ethereum/go-ethereum/statediff/test_helpers" sdtypes "github.com/ethereum/go-ethereum/statediff/types" ) var ( emptyStorage = make([]sdtypes.StorageNode, 0) block0, block1 *types.Block - minerLeafKey = testhelpers.AddressToLeafKey(common.HexToAddress("0x0")) + minerLeafKey = test_helpers.AddressToLeafKey(common.HexToAddress("0x0")) account1, _ = rlp.EncodeToBytes(types.StateAccount{ Nonce: uint64(0), Balance: big.NewInt(10000), @@ -90,9 +90,9 @@ func TestAPI(t *testing.T) { } func testSubscriptionAPI(t *testing.T) { - blocks, chain := testhelpers.MakeChain(1, testhelpers.Genesis, testhelpers.TestChainGen) + blocks, chain := test_helpers.MakeChain(1, test_helpers.Genesis, test_helpers.TestChainGen) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] expectedBlockRlp, _ := rlp.EncodeToBytes(block1) mockReceipt := &types.Receipt{ @@ -100,7 +100,7 @@ func testSubscriptionAPI(t *testing.T) { BlockHash: block1.Hash(), } expectedReceiptBytes, _ := rlp.EncodeToBytes(types.Receipts{mockReceipt}) - expectedStateDiff := statediff.StateObject{ + expectedStateDiff := sdtypes.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), Nodes: []sdtypes.StateNode{ @@ -114,14 +114,14 @@ func testSubscriptionAPI(t *testing.T) { { Path: []byte{'\x0e'}, NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountLeafNode, StorageNodes: emptyStorage, }, @@ -176,9 +176,9 @@ func testSubscriptionAPI(t *testing.T) { } func testHTTPAPI(t *testing.T) { - blocks, chain := testhelpers.MakeChain(1, testhelpers.Genesis, testhelpers.TestChainGen) + blocks, chain := test_helpers.MakeChain(1, test_helpers.Genesis, test_helpers.TestChainGen) defer chain.Stop() - block0 = testhelpers.Genesis + block0 = test_helpers.Genesis block1 = blocks[0] expectedBlockRlp, _ := rlp.EncodeToBytes(block1) mockReceipt := &types.Receipt{ @@ -186,7 +186,7 @@ func testHTTPAPI(t *testing.T) { BlockHash: block1.Hash(), } expectedReceiptBytes, _ := rlp.EncodeToBytes(types.Receipts{mockReceipt}) - expectedStateDiff := statediff.StateObject{ + expectedStateDiff := sdtypes.StateObject{ BlockNumber: block1.Number(), BlockHash: block1.Hash(), Nodes: []sdtypes.StateNode{ @@ -200,14 +200,14 @@ func testHTTPAPI(t *testing.T) { { Path: []byte{'\x0e'}, NodeType: sdtypes.Leaf, - LeafKey: testhelpers.Account1LeafKey, + LeafKey: test_helpers.Account1LeafKey, NodeValue: account1LeafNode, StorageNodes: emptyStorage, }, { Path: []byte{'\x00'}, NodeType: sdtypes.Leaf, - LeafKey: testhelpers.BankLeafKey, + LeafKey: test_helpers.BankLeafKey, NodeValue: bankAccountLeafNode, StorageNodes: emptyStorage, }, diff --git a/statediff/testhelpers/test_data.go b/statediff/test_helpers/test_data.go similarity index 99% rename from statediff/testhelpers/test_data.go rename to statediff/test_helpers/test_data.go index 73def50a4..e5b021364 100644 --- a/statediff/testhelpers/test_data.go +++ b/statediff/test_helpers/test_data.go @@ -14,7 +14,7 @@ // You should have received a copy of the GNU Lesser General Public License // along with the go-ethereum library. If not, see . -package testhelpers +package test_helpers import ( "math/big" diff --git a/statediff/testhelpers/constant.go b/statediff/testhelpers/constant.go deleted file mode 100644 index 9788549e6..000000000 --- a/statediff/testhelpers/constant.go +++ /dev/null @@ -1,17 +0,0 @@ -package testhelpers - -import ( - "math/big" - - "github.com/ethereum/go-ethereum/params" -) - -var ( - BalanceChange1000 = int64(1000) - BalanceChange10000 = int64(10000) - BalanceChange1Ether = int64(params.Ether) - Block1Account1Balance = big.NewInt(BalanceChange10000) - Block2Account2Balance = big.NewInt(21000000000000) - GasFees = int64(params.GWei) * int64(params.TxGas) - ContractGasLimit = uint64(1000000) -) diff --git a/statediff/trie/node.go b/statediff/trie/node.go deleted file mode 100644 index 6ffc2538c..000000000 --- a/statediff/trie/node.go +++ /dev/null @@ -1,54 +0,0 @@ -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 -} diff --git a/statediff/helpers.go b/statediff/trie_helpers/helpers.go similarity index 53% rename from statediff/helpers.go rename to statediff/trie_helpers/helpers.go index eb5060c51..ce3365f2c 100644 --- a/statediff/helpers.go +++ b/statediff/trie_helpers/helpers.go @@ -17,14 +17,65 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package trie_helpers import ( + "fmt" "sort" "strings" + + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/trie" ) -func sortKeys(data AccountMap) []string { +// CheckKeyType checks what type of key we have +func CheckKeyType(elements []interface{}) (types.NodeType, error) { + if len(elements) > 2 { + return types.Branch, nil + } + if len(elements) < 2 { + return types.Unknown, fmt.Errorf("node cannot be less than two elements in length") + } + switch elements[0].([]byte)[0] / 16 { + case '\x00': + return types.Extension, nil + case '\x01': + return types.Extension, nil + case '\x02': + return types.Leaf, nil + case '\x03': + return types.Leaf, nil + default: + return types.Unknown, fmt.Errorf("unknown hex prefix") + } +} + +// ResolveNode return the state diff node pointed by the iterator. +func ResolveNode(it trie.NodeIterator, trieDB *trie.Database) (types.StateNode, []interface{}, error) { + nodePath := make([]byte, len(it.Path())) + copy(nodePath, it.Path()) + node, err := trieDB.Node(it.Hash()) + if err != nil { + return types.StateNode{}, nil, err + } + var nodeElements []interface{} + if err = rlp.DecodeBytes(node, &nodeElements); err != nil { + return types.StateNode{}, nil, err + } + ty, err := CheckKeyType(nodeElements) + if err != nil { + return types.StateNode{}, nil, err + } + return types.StateNode{ + NodeType: ty, + Path: nodePath, + NodeValue: node, + }, nodeElements, nil +} + +// SortKeys sorts the keys in the account map +func SortKeys(data types.AccountMap) []string { keys := make([]string, 0, len(data)) for key := range data { keys = append(keys, key) @@ -34,10 +85,10 @@ func sortKeys(data AccountMap) []string { return keys } -// findIntersection finds the set of strings from both arrays that are equivalent +// FindIntersection finds the set of strings from both arrays that are equivalent // a and b must first be sorted // this is used to find which keys have been both "deleted" and "created" i.e. they were updated -func findIntersection(a, b []string) []string { +func FindIntersection(a, b []string) []string { lenA := len(a) lenB := len(b) iOfA, iOfB := 0, 0 diff --git a/statediff/types.go b/statediff/types.go deleted file mode 100644 index ef8256041..000000000 --- a/statediff/types.go +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2019 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 . - -// Contains a batch of utility type declarations used by the tests. As the node -// operates on unique types, a lot of them are needed to check various features. - -package statediff - -import ( - "encoding/json" - "math/big" - - "github.com/ethereum/go-ethereum/common" - ctypes "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/statediff/types" -) - -// Subscription struct holds our subscription channels -type Subscription struct { - PayloadChan chan<- Payload - QuitChan chan<- bool -} - -// DBParams holds params for Postgres db connection -type DBParams struct { - ConnectionURL string - ID string - ClientName string -} - -// Params is used to carry in parameters from subscribing/requesting clients configuration -type Params struct { - IntermediateStateNodes bool - IntermediateStorageNodes bool - IncludeBlock bool - IncludeReceipts bool - IncludeTD bool - IncludeCode bool - WatchedAddresses []common.Address - WatchedStorageSlots []common.Hash -} - -// Args bundles the arguments for the state diff builder -type Args struct { - OldStateRoot, NewStateRoot, BlockHash common.Hash - BlockNumber *big.Int -} - -type StateRoots struct { - OldStateRoot, NewStateRoot common.Hash -} - -// Payload packages the data to send to statediff subscriptions -type Payload struct { - BlockRlp []byte `json:"blockRlp"` - TotalDifficulty *big.Int `json:"totalDifficulty"` - ReceiptsRlp []byte `json:"receiptsRlp"` - StateObjectRlp []byte `json:"stateObjectRlp" gencodec:"required"` - - encoded []byte - err error -} - -func (sd *Payload) ensureEncoded() { - if sd.encoded == nil && sd.err == nil { - sd.encoded, sd.err = json.Marshal(sd) - } -} - -// Length to implement Encoder interface for Payload -func (sd *Payload) Length() int { - sd.ensureEncoded() - return len(sd.encoded) -} - -// Encode to implement Encoder interface for Payload -func (sd *Payload) Encode() ([]byte, error) { - sd.ensureEncoded() - return sd.encoded, sd.err -} - -// StateObject is the final output structure from the builder -type StateObject struct { - BlockNumber *big.Int `json:"blockNumber" gencodec:"required"` - BlockHash common.Hash `json:"blockHash" gencodec:"required"` - Nodes []types.StateNode `json:"nodes" gencodec:"required"` - CodeAndCodeHashes []types.CodeAndCodeHash `json:"codeMapping"` -} - -// AccountMap is a mapping of hex encoded path => account wrapper -type AccountMap map[string]accountWrapper - -// accountWrapper is used to temporary associate the unpacked node with its raw values -type accountWrapper struct { - Account *ctypes.StateAccount - NodeType types.NodeType - Path []byte - NodeValue []byte - LeafKey []byte -} diff --git a/statediff/types/types.go b/statediff/types/types.go index 56babfb5b..36008a784 100644 --- a/statediff/types/types.go +++ b/statediff/types/types.go @@ -14,12 +14,39 @@ // You should have received a copy of the GNU Lesser General Public License // along with the go-ethereum library. If not, see . -// Contains a batch of utility type declarations used by the tests. As the node -// operates on unique types, a lot of them are needed to check various features. - package types -import "github.com/ethereum/go-ethereum/common" +import ( + "math/big" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" +) + +// StateRoots holds the state roots required for generating a state diff +type StateRoots struct { + OldStateRoot, NewStateRoot common.Hash +} + +// StateObject is the final output structure from the builder +type StateObject struct { + BlockNumber *big.Int `json:"blockNumber" gencodec:"required"` + BlockHash common.Hash `json:"blockHash" gencodec:"required"` + Nodes []StateNode `json:"nodes" gencodec:"required"` + CodeAndCodeHashes []CodeAndCodeHash `json:"codeMapping"` +} + +// AccountMap is a mapping of hex encoded path => account wrapper +type AccountMap map[string]AccountWrapper + +// AccountWrapper is used to temporary associate the unpacked node with its raw values +type AccountWrapper struct { + Account *types.StateAccount + NodeType NodeType + Path []byte + NodeValue []byte + LeafKey []byte +} // NodeType for explicitly setting type of node type NodeType string -- 2.45.2 From 2339b0a5afe6add87188463eb21b73ffc6b5b042 Mon Sep 17 00:00:00 2001 From: i-norden Date: Thu, 11 Nov 2021 21:51:14 -0600 Subject: [PATCH 03/33] fixes and cli integration for new options --- cmd/geth/config.go | 113 ++-- cmd/geth/main.go | 3 + cmd/geth/usage.go | 3 + cmd/utils/flags.go | 21 +- statediff/README.md | 3 + statediff/indexer/database/dump/batch_tx.go | 15 +- statediff/indexer/database/dump/config.go | 48 ++ statediff/indexer/database/dump/indexer.go | 33 +- statediff/indexer/database/sql/batch_tx.go | 22 +- statediff/indexer/database/sql/indexer.go | 14 +- .../database/sql/pgx_indexer_legacy_test.go | 88 +++ .../indexer/database/sql/pgx_indexer_test.go | 609 ++++++++++++++++++ .../indexer/database/sql/postgres/config.go | 21 +- .../indexer/database/sql/postgres/pgx.go | 20 +- .../indexer/database/sql/postgres/pgx_test.go | 6 +- .../indexer/database/sql/postgres/sqlx.go | 3 +- .../database/sql/postgres/sqlx_test.go | 5 +- ...cy_test.go => sqlx_indexer_legacy_test.go} | 23 +- .../{indexer_test.go => sqlx_indexer_test.go} | 43 +- .../indexer/database/sql/test_helpers.go | 12 + statediff/indexer/shared/db_kind.go | 19 + statediff/service.go | 22 +- 22 files changed, 1008 insertions(+), 138 deletions(-) create mode 100644 statediff/indexer/database/sql/pgx_indexer_legacy_test.go create mode 100644 statediff/indexer/database/sql/pgx_indexer_test.go rename statediff/indexer/database/sql/{indexer_legacy_test.go => sqlx_indexer_legacy_test.go} (82%) rename statediff/indexer/database/sql/{indexer_test.go => sqlx_indexer_test.go} (95%) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index cfc43d2ab..9a8b169be 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -18,6 +18,7 @@ package main import ( "bufio" + "context" "errors" "fmt" "math/big" @@ -26,10 +27,7 @@ import ( "time" "unicode" - "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" - - "github.com/ethereum/go-ethereum/eth/downloader" - "github.com/ethereum/go-ethereum/statediff" + "github.com/naoina/toml" "gopkg.in/urfave/cli.v1" "github.com/ethereum/go-ethereum/accounts/external" @@ -38,13 +36,18 @@ import ( "github.com/ethereum/go-ethereum/accounts/usbwallet" "github.com/ethereum/go-ethereum/cmd/utils" "github.com/ethereum/go-ethereum/eth/catalyst" + "github.com/ethereum/go-ethereum/eth/downloader" "github.com/ethereum/go-ethereum/eth/ethconfig" "github.com/ethereum/go-ethereum/internal/ethapi" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/params" - "github.com/naoina/toml" + "github.com/ethereum/go-ethereum/statediff" + dumpdb "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" ) var ( @@ -185,48 +188,82 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) { } if ctx.GlobalBool(utils.StateDiffFlag.Name) { - var dbConfig *sql.Config + var indexerConfig interfaces.Config + var clientName, nodeID string if ctx.GlobalIsSet(utils.StateDiffWritingFlag.Name) { - dbConfig = new(sql.Config) - dbConfig.Hostname = ctx.GlobalString(utils.StateDiffDBHostFlag.Name) - dbConfig.Port = ctx.GlobalInt(utils.StateDiffDBPortFlag.Name) - dbConfig.DatabaseName = ctx.GlobalString(utils.StateDiffDBNameFlag.Name) - dbConfig.Username = ctx.GlobalString(utils.StateDiffDBUserFlag.Name) - dbConfig.Password = ctx.GlobalString(utils.StateDiffDBPasswordFlag.Name) - + clientName = ctx.GlobalString(utils.StateDiffDBClientNameFlag.Name) if ctx.GlobalIsSet(utils.StateDiffDBNodeIDFlag.Name) { - dbConfig.ID = ctx.GlobalString(utils.StateDiffDBNodeIDFlag.Name) + nodeID = ctx.GlobalString(utils.StateDiffDBNodeIDFlag.Name) } else { utils.Fatalf("Must specify node ID for statediff DB output") } - if ctx.GlobalIsSet(utils.StateDiffDBClientNameFlag.Name) { - dbConfig.ClientName = ctx.GlobalString(utils.StateDiffDBClientNameFlag.Name) - } else { - utils.Fatalf("Must specify client name for statediff DB output") + dbTypeStr := ctx.GlobalString(utils.StateDiffDBTypeFlag.Name) + dbType, err := shared.ResolveDBType(dbTypeStr) + if err != nil { + utils.Fatalf("%v", err) } - - if ctx.GlobalIsSet(utils.StateDiffDBMinConns.Name) { - dbConfig.MinConns = ctx.GlobalInt(utils.StateDiffDBMinConns.Name) - } - if ctx.GlobalIsSet(utils.StateDiffDBMaxConns.Name) { - dbConfig.MaxConns = ctx.GlobalInt(utils.StateDiffDBMaxConns.Name) - } - if ctx.GlobalIsSet(utils.StateDiffDBMaxIdleConns.Name) { - dbConfig.MaxIdle = ctx.GlobalInt(utils.StateDiffDBMaxIdleConns.Name) - } - if ctx.GlobalIsSet(utils.StateDiffDBMaxConnLifetime.Name) { - dbConfig.MaxConnLifetime = ctx.GlobalDuration(utils.StateDiffDBMaxConnLifetime.Name) * time.Second - } - if ctx.GlobalIsSet(utils.StateDiffDBMaxConnIdleTime.Name) { - dbConfig.MaxConnIdleTime = ctx.GlobalDuration(utils.StateDiffDBMaxConnIdleTime.Name) * time.Second - } - if ctx.GlobalIsSet(utils.StateDiffDBConnTimeout.Name) { - dbConfig.ConnTimeout = ctx.GlobalDuration(utils.StateDiffDBConnTimeout.Name) * time.Second + switch dbType { + case shared.POSTGRES: + driverTypeStr := ctx.GlobalString(utils.StateDiffDBDriverTypeFlag.Name) + driverType, err := postgres.ResolveDriverType(driverTypeStr) + if err != nil { + utils.Fatalf("%v", err) + } + pgConfig := postgres.Config{ + Hostname: ctx.GlobalString(utils.StateDiffDBHostFlag.Name), + Port: ctx.GlobalInt(utils.StateDiffDBPortFlag.Name), + DatabaseName: ctx.GlobalString(utils.StateDiffDBNameFlag.Name), + Username: ctx.GlobalString(utils.StateDiffDBUserFlag.Name), + Password: ctx.GlobalString(utils.StateDiffDBPasswordFlag.Name), + ID: nodeID, + ClientName: clientName, + Driver: driverType, + } + if ctx.GlobalIsSet(utils.StateDiffDBMinConns.Name) { + pgConfig.MinConns = ctx.GlobalInt(utils.StateDiffDBMinConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConns.Name) { + pgConfig.MaxConns = ctx.GlobalInt(utils.StateDiffDBMaxConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxIdleConns.Name) { + pgConfig.MaxIdle = ctx.GlobalInt(utils.StateDiffDBMaxIdleConns.Name) + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConnLifetime.Name) { + pgConfig.MaxConnLifetime = ctx.GlobalDuration(utils.StateDiffDBMaxConnLifetime.Name) * time.Second + } + if ctx.GlobalIsSet(utils.StateDiffDBMaxConnIdleTime.Name) { + pgConfig.MaxConnIdleTime = ctx.GlobalDuration(utils.StateDiffDBMaxConnIdleTime.Name) * time.Second + } + if ctx.GlobalIsSet(utils.StateDiffDBConnTimeout.Name) { + pgConfig.ConnTimeout = ctx.GlobalDuration(utils.StateDiffDBConnTimeout.Name) * time.Second + } + indexerConfig = pgConfig + case shared.DUMP: + dumpTypeStr := ctx.GlobalString(utils.StateDiffDBDumpDst.Name) + dumpType, err := dumpdb.ResolveDumpType(dumpTypeStr) + if err != nil { + utils.Fatalf("%v", err) + } + switch dumpType { + case dumpdb.STDERR: + indexerConfig = dumpdb.Config{Dump: os.Stdout} + case dumpdb.STDOUT: + indexerConfig = dumpdb.Config{Dump: os.Stderr} + case dumpdb.DISCARD: + indexerConfig = dumpdb.Config{Dump: dumpdb.NewDiscardWriterCloser()} + default: + utils.Fatalf("unrecognized dump destination: %s", dumpType) + } + default: + utils.Fatalf("unrecognized database type: %s", dbType) } } - p := statediff.ServiceParams{ - DBParams: dbConfig, + p := statediff.Config{ + IndexerConfig: indexerConfig, + ID: nodeID, + ClientName: clientName, + Context: context.Background(), EnableWriteLoop: ctx.GlobalBool(utils.StateDiffWritingFlag.Name), NumWorkers: ctx.GlobalUint(utils.StateDiffWorkersFlag.Name), } diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 9c8dbdcfd..c92810d11 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -149,6 +149,9 @@ var ( utils.GpoIgnoreGasPriceFlag, utils.MinerNotifyFullFlag, utils.StateDiffFlag, + utils.StateDiffDBTypeFlag, + utils.StateDiffDBDriverTypeFlag, + utils.StateDiffDBDumpDst, utils.StateDiffDBNameFlag, utils.StateDiffDBPasswordFlag, utils.StateDiffDBUserFlag, diff --git a/cmd/geth/usage.go b/cmd/geth/usage.go index 1ad1b8557..68e2a3f4c 100644 --- a/cmd/geth/usage.go +++ b/cmd/geth/usage.go @@ -225,6 +225,9 @@ var AppHelpFlagGroups = []flags.FlagGroup{ Name: "STATE DIFF", Flags: []cli.Flag{ utils.StateDiffFlag, + utils.StateDiffDBTypeFlag, + utils.StateDiffDBDriverTypeFlag, + utils.StateDiffDBDumpDst, utils.StateDiffDBNameFlag, utils.StateDiffDBPasswordFlag, utils.StateDiffDBUserFlag, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 8dfb92a7e..08f9088f5 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -786,6 +786,21 @@ var ( Name: "statediff", Usage: "Enables the processing of state diffs between each block", } + StateDiffDBTypeFlag = cli.StringFlag{ + Name: "statediff.db.type", + Usage: "Statediff database type", + Value: "postgres", + } + StateDiffDBDriverTypeFlag = cli.StringFlag{ + Name: "statediff.db.driver", + Usage: "Statediff database driver type", + Value: "pgx", + } + StateDiffDBDumpDst = cli.StringFlag{ + Name: "statediff.dump.dst", + Usage: "Statediff database dump destination (default is stdout)", + Value: "stdout", + } StateDiffDBHostFlag = cli.StringFlag{ Name: "statediff.db.host", Usage: "Statediff database hostname/ip", @@ -840,6 +855,7 @@ var ( StateDiffDBClientNameFlag = cli.StringFlag{ Name: "statediff.db.clientname", Usage: "Client name to use when writing state diffs to database", + Value: "go-ethereum", } StateDiffWritingFlag = cli.BoolFlag{ Name: "statediff.writing", @@ -847,7 +863,8 @@ var ( } StateDiffWorkersFlag = cli.UintFlag{ Name: "statediff.workers", - Usage: "Number of concurrent workers to use during statediff processing (0 = 1)", + Usage: "Number of concurrent workers to use during statediff processing (default 1)", + Value: 1, } ) @@ -1804,7 +1821,7 @@ func RegisterGraphQLService(stack *node.Node, backend ethapi.Backend, cfg node.C } // RegisterStateDiffService configures and registers a service to stream state diff data over RPC -func RegisterStateDiffService(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params statediff.ServiceParams) { +func RegisterStateDiffService(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params statediff.Config) { if err := statediff.New(stack, ethServ, cfg, params); err != nil { Fatalf("Failed to register the Statediff service: %v", err) } diff --git a/statediff/README.md b/statediff/README.md index dd2eaed7f..bd5d1d43c 100644 --- a/statediff/README.md +++ b/statediff/README.md @@ -79,6 +79,9 @@ This service introduces a CLI flag namespace `statediff` `--statediff` flag is used to turn on the service `--statediff.writing` is used to tell the service to write state diff objects it produces from synced ChainEvents directly to a configured Postgres database `--statediff.workers` is used to set the number of concurrent workers to process state diff objects and write them into the database +`--statediff.db.type` is the type of database we write out to (current options: postgres and dump) +`--statediff.dump.dst` is the destination to write to when operating in database dump mode (stdout, stderr, discard) +`--statediff.db.driver` is the specific driver to use for the database (current options for postgres: pgx and sqlx) `--statediff.db.host` is the hostname/ip to dial to connect to the database `--statediff.db.port` is the port to dial to connect to the database `--statediff.db.name` is the name of the database to connect to diff --git a/statediff/indexer/database/dump/batch_tx.go b/statediff/indexer/database/dump/batch_tx.go index a0021baf7..f1754b907 100644 --- a/statediff/indexer/database/dump/batch_tx.go +++ b/statediff/indexer/database/dump/batch_tx.go @@ -30,21 +30,22 @@ import ( // BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration type BatchTx struct { - dump io.Writer - quit chan struct{} - iplds chan models.IPLDModel - ipldCache models.IPLDBatch + BlockNumber uint64 + dump io.Writer + quit chan struct{} + iplds chan models.IPLDModel + ipldCache models.IPLDBatch - close func(blockTx *BatchTx, err error) error + submit func(blockTx *BatchTx, err error) error } // Submit satisfies indexer.AtomicTx func (tx *BatchTx) Submit(err error) error { - return tx.close(tx, err) + return tx.submit(tx, err) } func (tx *BatchTx) flush() error { - if _, err := fmt.Fprintf(tx.dump, "%+v", tx.ipldCache); err != nil { + if _, err := fmt.Fprintf(tx.dump, "%+v\r\n", tx.ipldCache); err != nil { return err } tx.ipldCache = models.IPLDBatch{} diff --git a/statediff/indexer/database/dump/config.go b/statediff/indexer/database/dump/config.go index fb2e6a58c..6fb1f0a9e 100644 --- a/statediff/indexer/database/dump/config.go +++ b/statediff/indexer/database/dump/config.go @@ -17,15 +17,63 @@ package dump import ( + "fmt" "io" + "strings" "github.com/ethereum/go-ethereum/statediff/indexer/shared" ) +// DumpType to explicitly type the dump destination +type DumpType string + +const ( + STDOUT = "Stdout" + STDERR = "Stderr" + DISCARD = "Discard" + UNKNOWN = "Unknown" +) + +// ResolveDumpType resolves the dump type for the provided string +func ResolveDumpType(str string) (DumpType, error) { + switch strings.ToLower(str) { + case "stdout", "out", "std out": + return STDOUT, nil + case "stderr", "err", "std err": + return STDERR, nil + case "discard", "void", "devnull", "dev null": + return DISCARD, nil + default: + return UNKNOWN, fmt.Errorf("unrecognized dump type: %s", str) + } +} + +// Config for data dump type Config struct { Dump io.WriteCloser } +// Type satisfies interfaces.Config func (c Config) Type() shared.DBType { return shared.DUMP } + +// NewDiscardWriterCloser returns a discardWrapper wrapping io.Discard +func NewDiscardWriterCloser() io.WriteCloser { + return discardWrapper{blackhole: io.Discard} +} + +// discardWrapper wraps io.Discard with io.Closer +type discardWrapper struct { + blackhole io.Writer +} + +// Write satisfies io.Writer +func (dw discardWrapper) Write(b []byte) (int, error) { + return dw.blackhole.Write(b) +} + +// Close satisfies io.Closer +func (dw discardWrapper) Close() error { + return nil +} diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index f815305b1..ccbe28c66 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -102,11 +102,12 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip t = time.Now() blockTx := &BatchTx{ - dump: sdi.dump, - iplds: make(chan models.IPLDModel), - quit: make(chan struct{}), - ipldCache: models.IPLDBatch{}, - close: func(self *BatchTx, err error) error { + BlockNumber: height, + dump: sdi.dump, + iplds: make(chan models.IPLDModel), + quit: make(chan struct{}), + ipldCache: models.IPLDBatch{}, + submit: func(self *BatchTx, err error) error { close(self.quit) close(self.iplds) tDiff := time.Since(t) @@ -205,7 +206,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he Timestamp: header.Time, BaseFee: baseFee, } - _, err := fmt.Fprintf(sdi.dump, "%+v", mod) + _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", mod) return 0, err } @@ -228,7 +229,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNum BlockHash: uncleNode.Hash().String(), Reward: uncleReward.String(), } - if _, err := fmt.Fprintf(sdi.dump, "%+v", uncle); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", uncle); err != nil { return err } } @@ -319,7 +320,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs if txType != types.LegacyTxType { txModel.Type = &txType } - if _, err := fmt.Fprintf(sdi.dump, "%+v", txModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", txModel); err != nil { return err } @@ -334,7 +335,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs Address: accessListElement.Address.Hex(), StorageKeys: storageKeys, } - if _, err := fmt.Fprintf(sdi.dump, "%+v", accessListElementModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", accessListElementModel); err != nil { return err } } @@ -357,11 +358,11 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs rctModel.PostState = common.Bytes2Hex(receipt.PostState) } - if _, err := fmt.Fprintf(sdi.dump, "%+v", rctModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", rctModel); err != nil { return err } - if _, err := fmt.Fprintf(sdi.dump, "%+v", logDataSet); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", logDataSet); err != nil { return err } } @@ -392,7 +393,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: shared.RemovedNodeMhKey, NodeType: stateNode.NodeType.Int(), } - _, err := fmt.Fprintf(sdi.dump, "%+v", stateModel) + _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", stateModel) return err } stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) @@ -407,7 +408,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt NodeType: stateNode.NodeType.Int(), } // index the state node, collect the stateID to reference by FK - if _, err := fmt.Fprintf(sdi.dump, "%+v", stateModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", stateModel); err != nil { return err } // if we have a leaf, decode and index the account data @@ -429,7 +430,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt CodeHash: account.CodeHash, StorageRoot: account.Root.String(), } - if _, err := fmt.Fprintf(sdi.dump, "%+v", accountModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", accountModel); err != nil { return err } } @@ -445,7 +446,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: shared.RemovedNodeMhKey, NodeType: storageNode.NodeType.Int(), } - if _, err := fmt.Fprintf(sdi.dump, "%+v", storageModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", storageModel); err != nil { return err } continue @@ -461,7 +462,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: storageMhKey, NodeType: storageNode.NodeType.Int(), } - if _, err := fmt.Fprintf(sdi.dump, "%+v", storageModel); err != nil { + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", storageModel); err != nil { return err } } diff --git a/statediff/indexer/database/sql/batch_tx.go b/statediff/indexer/database/sql/batch_tx.go index 2041af1ed..ff847eec6 100644 --- a/statediff/indexer/database/sql/batch_tx.go +++ b/statediff/indexer/database/sql/batch_tx.go @@ -19,33 +19,33 @@ package sql import ( "context" - "github.com/ethereum/go-ethereum/statediff/indexer/ipld" - blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" node "github.com/ipfs/go-ipld-format" "github.com/lib/pq" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/models" ) // BatchTx wraps a sql tx with the state necessary for building the tx concurrently during trie difference iteration type BatchTx struct { - ctx context.Context - dbtx Tx - headerID int64 - stm string - quit chan struct{} - iplds chan models.IPLDModel - ipldCache models.IPLDBatch + BlockNumber uint64 + ctx context.Context + dbtx Tx + headerID int64 + stm string + quit chan struct{} + iplds chan models.IPLDModel + ipldCache models.IPLDBatch - close func(blockTx *BatchTx, err error) error + submit func(blockTx *BatchTx, err error) error } // Submit satisfies indexer.AtomicTx func (tx *BatchTx) Submit(err error) error { - return tx.close(tx, err) + return tx.submit(tx, err) } func (tx *BatchTx) flush() error { diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index 6c35cccac..b9cfd0733 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -141,13 +141,15 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } }() blockTx := &BatchTx{ - stm: sdi.dbWriter.db.InsertIPLDsStm(), - iplds: make(chan models.IPLDModel), - quit: make(chan struct{}), - ipldCache: models.IPLDBatch{}, - dbtx: tx, + ctx: sdi.ctx, + BlockNumber: height, + stm: sdi.dbWriter.db.InsertIPLDsStm(), + iplds: make(chan models.IPLDModel), + quit: make(chan struct{}), + ipldCache: models.IPLDBatch{}, + dbtx: tx, // handle transaction commit or rollback for any return case - close: func(self *BatchTx, err error) error { + submit: func(self *BatchTx, err error) error { close(self.quit) close(self.iplds) if p := recover(); p != nil { diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go new file mode 100644 index 000000000..f97f3e257 --- /dev/null +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -0,0 +1,88 @@ +// 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 . + +package sql_test + +import ( + "context" + "testing" + + "github.com/multiformats/go-multihash" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +func setupLegacyPGX(t *testing.T) { + mockLegacyBlock = legacyData.MockBlock + legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256) + + db, err = postgres.SetupPGXDB() + require.NoError(t, err) + + ind, err = sql.NewStateDiffIndexer(context.Background(), legacyData.Config, db) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + mockLegacyBlock, + legacyData.MockReceipts, + legacyData.MockBlock.Difficulty()) + require.NoError(t, err) + + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + }() + for _, node := range legacyData.StateDiffs { + err = ind.PushStateNode(tx, node) + require.NoError(t, err) + } + + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, legacyData.BlockNumber.Uint64()) +} + +func TestPGXIndexerLegacy(t *testing.T) { + t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { + setupLegacyPGX(t) + defer tearDown(t) + pgStr := `SELECT cid, td, reward, id, base_fee + FROM eth.header_cids + WHERE block_number = $1` + // check header was properly indexed + type res struct { + CID string + TD string + Reward string + ID int + BaseFee *int64 `db:"base_fee"` + } + header := new(res) + + err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) + require.NoError(t, err) + + test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) + test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") + require.Nil(t, legacyData.MockHeader.BaseFee) + require.Nil(t, header.BaseFee) + }) +} diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go new file mode 100644 index 000000000..730257b21 --- /dev/null +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -0,0 +1,609 @@ +// 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 . + +package sql_test + +import ( + "bytes" + "context" + "fmt" + "os" + "testing" + + "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" + + "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/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +func init() { + if os.Getenv("MODE") != "statediff" { + fmt.Println("Skipping statediff test") + os.Exit(0) + } + + mockBlock = mocks.MockBlock + txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts + + buf := new(bytes.Buffer) + txs.EncodeIndex(0, buf) + tx1 = make([]byte, buf.Len()) + copy(tx1, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(1, buf) + tx2 = make([]byte, buf.Len()) + copy(tx2, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(2, buf) + tx3 = make([]byte, buf.Len()) + copy(tx3, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(3, buf) + tx4 = make([]byte, buf.Len()) + copy(tx4, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(4, buf) + tx5 = make([]byte, buf.Len()) + copy(tx5, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(0, buf) + rct1 = make([]byte, buf.Len()) + copy(rct1, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(1, buf) + rct2 = make([]byte, buf.Len()) + copy(rct2, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(2, buf) + rct3 = make([]byte, buf.Len()) + copy(rct3, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(3, buf) + rct4 = make([]byte, buf.Len()) + copy(rct4, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(4, buf) + rct5 = make([]byte, buf.Len()) + copy(rct5, buf.Bytes()) + buf.Reset() + + headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256) + trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256) + trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256) + trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) + trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) + trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) + rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) + rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) + rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) + rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) + rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) + state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) + state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) + storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) +} + +func setupPGX(t *testing.T) { + db, err = postgres.SetupPGXDB() + if err != nil { + t.Fatal(err) + } + ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + mockBlock, + mocks.MockReceipts, + mocks.MockBlock.Difficulty()) + if err != nil { + t.Fatal(err) + } + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + }() + for _, node := range mocks.StateDiffs { + err = ind.PushStateNode(tx, node) + if err != nil { + t.Fatal(err) + } + } + + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) +} + +func TestPGXIndexer(t *testing.T) { + t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + pgStr := `SELECT cid, td, reward, id, base_fee + FROM eth.header_cids + WHERE block_number = $1` + // check header was properly indexed + type res struct { + CID string + TD string + Reward string + ID int + BaseFee *int64 `db:"base_fee"` + } + header := new(res) + err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).StructScan(header) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, header.CID, headerCID.String()) + test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + dc, err := cid.Decode(header.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, mocks.MockHeaderRlp) + }) + + t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + // check that txs were properly indexed + trxs := make([]string, 0) + pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.id) + WHERE header_cids.block_number = $1` + err = db.Select(context.Background(), &trxs, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(trxs), 5) + expectTrue(t, test_helpers.ListContainsString(trxs, trx1CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx2CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) + // and published + for _, c := range trxs { + dc, err := cid.Decode(c) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + switch c { + case trx1CID.String(): + test_helpers.ExpectEqual(t, data, tx1) + var txType *uint8 + pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + err = db.Get(context.Background(), &txType, pgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != nil { + t.Fatalf("expected nil tx_type, got %d", *txType) + } + case trx2CID.String(): + test_helpers.ExpectEqual(t, data, tx2) + var txType *uint8 + pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + err = db.Get(context.Background(), &txType, pgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != nil { + t.Fatalf("expected nil tx_type, got %d", *txType) + } + case trx3CID.String(): + test_helpers.ExpectEqual(t, data, tx3) + var txType *uint8 + pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + err = db.Get(context.Background(), &txType, pgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != nil { + t.Fatalf("expected nil tx_type, got %d", *txType) + } + case trx4CID.String(): + test_helpers.ExpectEqual(t, data, tx4) + var txType *uint8 + pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + err = db.Get(context.Background(), &txType, pgStr, c) + if err != nil { + t.Fatal(err) + } + if *txType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", *txType) + } + accessListElementModels := make([]models.AccessListElementModel, 0) + pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.id) WHERE cid = $1 ORDER BY access_list_element.index ASC` + err = db.Select(context.Background(), &accessListElementModels, pgStr, c) + if err != nil { + t.Fatal(err) + } + if len(accessListElementModels) != 2 { + t.Fatalf("expected two access list entries, got %d", len(accessListElementModels)) + } + model1 := models.AccessListElementModel{ + Index: accessListElementModels[0].Index, + Address: accessListElementModels[0].Address, + } + model2 := models.AccessListElementModel{ + Index: accessListElementModels[1].Index, + Address: accessListElementModels[1].Address, + StorageKeys: accessListElementModels[1].StorageKeys, + } + test_helpers.ExpectEqual(t, model1, mocks.AccessListEntry1Model) + test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) + case trx5CID.String(): + test_helpers.ExpectEqual(t, data, tx5) + var txType *uint8 + pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + err = db.Get(context.Background(), &txType, pgStr, c) + if err != nil { + t.Fatal(err) + } + if *txType != types.DynamicFeeTxType { + t.Fatalf("expected DynamicFeeTxType (2), got %d", *txType) + } + } + } + }) + + t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + + rcts := make([]string, 0) + pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids + WHERE receipt_cids.tx_id = transaction_cids.id + AND transaction_cids.header_id = header_cids.id + AND header_cids.block_number = $1 + ORDER BY transaction_cids.index` + err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + + type logIPLD struct { + Index int `db:"index"` + Address string `db:"address"` + Data []byte `db:"data"` + Topic0 string `db:"topic0"` + Topic1 string `db:"topic1"` + } + for i := range rcts { + 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.leaf_cid = $1 ORDER BY eth.log_cids.index ASC` + err = db.Select(context.Background(), &results, pgStr, rcts[i]) + require.NoError(t, err) + + // expecting MockLog1 and MockLog2 for mockReceipt4 + expectedLogs := mocks.MockReceipts[i].Logs + test_helpers.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. + test_helpers.ExpectEqual(t, logRaw, nodeElements[1].([]byte)) + } + } + }) + + t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + + // check receipts were properly indexed + rcts := make([]string, 0) + pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids + WHERE receipt_cids.tx_id = transaction_cids.id + AND transaction_cids.header_id = header_cids.id + AND header_cids.block_number = $1 order by transaction_cids.id` + err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(rcts), 5) + + for idx, rctLeafCID := range rcts { + result := make([]models.IPLDModel, 0) + pgStr = `SELECT data + FROM eth.receipt_cids + INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) + WHERE receipt_cids.leaf_cid = $1` + err = db.Select(context.Background(), &result, pgStr, rctLeafCID) + if err != nil { + t.Fatal(err) + } + + // Decode the log leaf node. + var nodeElements []interface{} + err = rlp.DecodeBytes(result[0].Data, &nodeElements) + require.NoError(t, err) + + expectedRct, err := mocks.MockReceipts[idx].MarshalBinary() + require.NoError(t, err) + + test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) + } + + // and published + for _, c := range rcts { + dc, err := cid.Decode(c) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + + switch c { + case rct1CID.String(): + test_helpers.ExpectEqual(t, data, rct1) + var postStatus uint64 + pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` + err = db.Get(context.Background(), &postStatus, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) + case rct2CID.String(): + test_helpers.ExpectEqual(t, data, rct2) + var postState string + pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` + err = db.Get(context.Background(), &postState, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) + case rct3CID.String(): + test_helpers.ExpectEqual(t, data, rct3) + var postState string + pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` + err = db.Get(context.Background(), &postState, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) + case rct4CID.String(): + test_helpers.ExpectEqual(t, data, rct4) + var postState string + pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` + err = db.Get(context.Background(), &postState, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) + case rct5CID.String(): + test_helpers.ExpectEqual(t, data, rct5) + var postState string + pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` + err = db.Get(context.Background(), &postState, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) + } + } + }) + + t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + // check that state nodes were properly indexed and published + stateNodes := make([]models.StateNodeModel, 0) + pgStr := `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + WHERE header_cids.block_number = $1 AND node_type != 3` + err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(stateNodes), 2) + for _, stateNode := range stateNodes { + var data []byte + dc, err := cid.Decode(stateNode.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + pgStr = `SELECT * from eth.state_accounts WHERE state_id = $1` + var account models.StateAccountModel + err = db.Get(context.Background(), &account, pgStr, stateNode.ID) + if err != nil { + t.Fatal(err) + } + if stateNode.CID == state1CID.String() { + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.ContractLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) + test_helpers.ExpectEqual(t, data, mocks.ContractLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ + ID: account.ID, + StateID: stateNode.ID, + Balance: "0", + CodeHash: mocks.ContractCodeHash.Bytes(), + StorageRoot: mocks.ContractRoot, + Nonce: 1, + }) + } + if stateNode.CID == state2CID.String() { + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.AccountLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) + test_helpers.ExpectEqual(t, data, mocks.AccountLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ + ID: account.ID, + StateID: stateNode.ID, + Balance: "1000", + CodeHash: mocks.AccountCodeHash.Bytes(), + StorageRoot: mocks.AccountRoot, + Nonce: 0, + }) + } + } + + // check that Removed state nodes were properly indexed and published + stateNodes = make([]models.StateNodeModel, 0) + pgStr = `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + WHERE header_cids.block_number = $1 AND node_type = 3` + err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(stateNodes), 1) + stateNode := stateNodes[0] + var data []byte + dc, err := cid.Decode(stateNode.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, stateNode.CID, shared.RemovedNodeStateCID) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x02'}) + test_helpers.ExpectEqual(t, data, []byte{}) + }) + + t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { + setupPGX(t) + defer tearDown(t) + // check that storage nodes were properly indexed + storageNodes := make([]models.StorageNodeWithStateKeyModel, 0) + pgStr := `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path + FROM eth.storage_cids, eth.state_cids, eth.header_cids + WHERE storage_cids.state_id = state_cids.id + AND state_cids.header_id = header_cids.id + AND header_cids.block_number = $1 + AND storage_cids.node_type != 3` + err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + CID: storageCID.String(), + NodeType: 2, + StorageKey: common.BytesToHash(mocks.StorageLeafKey).Hex(), + StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), + Path: []byte{}, + }) + var data []byte + dc, err := cid.Decode(storageNodes[0].CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, mocks.StorageLeafNode) + + // check that Removed storage nodes were properly indexed + storageNodes = make([]models.StorageNodeWithStateKeyModel, 0) + pgStr = `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path + FROM eth.storage_cids, eth.state_cids, eth.header_cids + WHERE storage_cids.state_id = state_cids.id + AND state_cids.header_id = header_cids.id + AND header_cids.block_number = $1 + AND storage_cids.node_type = 3` + err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + CID: shared.RemovedNodeStorageCID, + NodeType: 3, + StorageKey: common.BytesToHash(mocks.RemovedLeafKey).Hex(), + StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), + Path: []byte{'\x03'}, + }) + dc, err = cid.Decode(storageNodes[0].CID) + if err != nil { + t.Fatal(err) + } + mhKey = dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey = blockstore.BlockPrefix.String() + mhKey.String() + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) + err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, []byte{}) + }) +} diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index 07e3dfe21..aff7ac773 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -18,18 +18,33 @@ package postgres import ( "fmt" + "strings" "time" "github.com/ethereum/go-ethereum/statediff/indexer/shared" ) +// DriverType to explicity type the kind of sql driver we are using type DriverType string const ( - PGX DriverType = "PGX" - SQLX DriverType = "SQLX" + PGX DriverType = "PGX" + SQLX DriverType = "SQLX" + Unknown DriverType = "Unknown" ) +// ResolveDriverType resolves a DriverType from a provided string +func ResolveDriverType(str string) (DriverType, error) { + switch strings.ToLower(str) { + case "pgx", "pgxpool": + return PGX, nil + case "sqlx": + return SQLX, nil + default: + return Unknown, fmt.Errorf("unrecognized driver type string: %s", str) + } +} + // DefaultConfig are default parameters for connecting to a Postgres sql var DefaultConfig = Config{ Hostname: "localhost", @@ -64,10 +79,12 @@ type Config struct { Driver DriverType } +// Type satisfies interfaces.Config func (c Config) Type() shared.DBType { return shared.POSTGRES } +// DbConnectionString constructs and returns the connection string from the config func (c Config) DbConnectionString() string { if len(c.Username) > 0 && len(c.Password) > 0 { return fmt.Sprintf("postgresql://%s:%s@%s:%d/%s?sslmode=disable", diff --git a/statediff/indexer/database/sql/postgres/pgx.go b/statediff/indexer/database/sql/postgres/pgx.go index d94c35083..838c78911 100644 --- a/statediff/indexer/database/sql/postgres/pgx.go +++ b/statediff/indexer/database/sql/postgres/pgx.go @@ -105,8 +105,8 @@ func (pgx *PGXDriver) createNode() error { // QueryRow satisfies sql.Database func (pgx *PGXDriver) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { - row := pgx.pool.QueryRow(ctx, sql, args...) - return rowWrapper{row: row} + rows, _ := pgx.pool.Query(ctx, sql, args...) + return rowsWrapper{rows: rows} } // Exec satisfies sql.Database @@ -160,18 +160,18 @@ func (pgx *PGXDriver) Context() context.Context { return pgx.ctx } -type rowWrapper struct { - row pgx.Row +type rowsWrapper struct { + rows pgx.Rows } // Scan satisfies sql.ScannableRow -func (r rowWrapper) Scan(dest ...interface{}) error { - return r.row.Scan(dest) +func (r rowsWrapper) Scan(dest ...interface{}) error { + return (pgx.Row)(r.rows).Scan(dest...) } // StructScan satisfies sql.ScannableRow -func (r rowWrapper) StructScan(dest interface{}) error { - return pgxscan.ScanRow(dest, r.row.(pgx.Rows)) +func (r rowsWrapper) StructScan(dest interface{}) error { + return pgxscan.ScanRow(dest, r.rows) } type resultWrapper struct { @@ -234,8 +234,8 @@ type pgxTxWrapper struct { // QueryRow satisfies sql.Tx func (t pgxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { - row := t.tx.QueryRow(ctx, sql, args...) - return rowWrapper{row: row} + rows, _ := t.tx.Query(ctx, sql, args...) + return rowsWrapper{rows: rows} } // Exec satisfies sql.Tx diff --git a/statediff/indexer/database/sql/postgres/pgx_test.go b/statediff/indexer/database/sql/postgres/pgx_test.go index aadb12835..50b6817eb 100644 --- a/statediff/indexer/database/sql/postgres/pgx_test.go +++ b/statediff/indexer/database/sql/postgres/pgx_test.go @@ -86,15 +86,15 @@ func TestPostgresPGX(t *testing.T) { t.Fatal(err) } - var data pgtype.Numeric + var data pgtype.Text err = dbPool.QueryRow(ctx, `SELECT data FROM example WHERE id = 1`).Scan(&data) if err != nil { t.Fatal(err) } - test_helpers.ExpectEqual(t, bi.String(), data) + test_helpers.ExpectEqual(t, data, bi.String()) actual := new(big.Int) - actual.Set(data.Int) + actual.SetString(data.String, 10) test_helpers.ExpectEqual(t, actual, bi) }) diff --git a/statediff/indexer/database/sql/postgres/sqlx.go b/statediff/indexer/database/sql/postgres/sqlx.go index 2abf82d89..684fc7bf0 100644 --- a/statediff/indexer/database/sql/postgres/sqlx.go +++ b/statediff/indexer/database/sql/postgres/sqlx.go @@ -177,8 +177,7 @@ type sqlxTxWrapper struct { // QueryRow satisfies sql.Tx func (t sqlxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { - row := t.tx.QueryRow(sql, args...) - return rowWrapper{row: row} + return t.tx.QueryRowx(sql, args...) } // Exec satisfies sql.Tx diff --git a/statediff/indexer/database/sql/postgres/sqlx_test.go b/statediff/indexer/database/sql/postgres/sqlx_test.go index 37164e0f7..95975a868 100644 --- a/statediff/indexer/database/sql/postgres/sqlx_test.go +++ b/statediff/indexer/database/sql/postgres/sqlx_test.go @@ -38,9 +38,8 @@ func TestPostgresSQLX(t *testing.T) { connStr := postgres.DefaultConfig.DbConnectionString() sqlxdb, err = sqlx.Connect("postgres", connStr) - if err != nil { - t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig, err) + t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err) } if sqlxdb == nil { t.Fatal("DB is nil") @@ -88,7 +87,7 @@ func TestPostgresSQLX(t *testing.T) { t.Fatal(err) } - test_helpers.ExpectEqual(t, bi.String(), data) + test_helpers.ExpectEqual(t, data, bi.String()) actual := new(big.Int) actual.SetString(data, 10) test_helpers.ExpectEqual(t, actual, bi) diff --git a/statediff/indexer/database/sql/indexer_legacy_test.go b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go similarity index 82% rename from statediff/indexer/database/sql/indexer_legacy_test.go rename to statediff/indexer/database/sql/sqlx_indexer_legacy_test.go index f2fdb0521..840a1ccae 100644 --- a/statediff/indexer/database/sql/indexer_legacy_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go @@ -20,6 +20,9 @@ import ( "context" "testing" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -37,34 +40,38 @@ var ( legacyHeaderCID cid.Cid ) -func setupLegacy(t *testing.T) { +func setupLegacySQLX(t *testing.T) { mockLegacyBlock = legacyData.MockBlock legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256) - db, err = test_helpers.SetupDB() + db, err = postgres.SetupSQLXDB() require.NoError(t, err) - ind, err = sql.NewSQLIndexer(context.Background(), legacyData.Config, db) + ind, err = sql.NewStateDiffIndexer(context.Background(), legacyData.Config, db) require.NoError(t, err) - var tx *sql.BlockTx + var tx interfaces.Batch tx, err = ind.PushBlock( mockLegacyBlock, legacyData.MockReceipts, legacyData.MockBlock.Difficulty()) require.NoError(t, err) - defer tx.Close(tx, err) + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + }() for _, node := range legacyData.StateDiffs { err = ind.PushStateNode(tx, node) require.NoError(t, err) } - test_helpers.ExpectEqual(t, tx.BlockNumber, legacyData.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, legacyData.BlockNumber.Uint64()) } -func TestPublishAndIndexerLegacy(t *testing.T) { +func TestSQLXIndexerLegacy(t *testing.T) { t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { - setupLegacy(t) + setupLegacySQLX(t) defer tearDown(t) pgStr := `SELECT cid, td, reward, id, base_fee FROM eth.header_cids diff --git a/statediff/indexer/database/sql/indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go similarity index 95% rename from statediff/indexer/database/sql/indexer_test.go rename to statediff/indexer/database/sql/sqlx_indexer_test.go index 91d55f094..815d36915 100644 --- a/statediff/indexer/database/sql/indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -32,19 +32,20 @@ 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/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) var ( db sql.Database err error - ind *interfaces.StateDiffIndexer + ind interfaces.StateDiffIndexer ipfsPgGet = `SELECT data FROM public.blocks WHERE key = $1` tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte @@ -136,14 +137,14 @@ func init() { storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) } -func setup(t *testing.T) { - db, err = test_helpers.SetupDB() +func setupSQLX(t *testing.T) { + db, err = postgres.SetupSQLXDB() if err != nil { t.Fatal(err) } - ind, err = indexer.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db) + ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db) require.NoError(t, err) - var tx *sql.BlockTx + var tx interfaces.Batch tx, err = ind.PushBlock( mockBlock, mocks.MockReceipts, @@ -151,7 +152,11 @@ func setup(t *testing.T) { if err != nil { t.Fatal(err) } - defer tx.Close(tx, err) + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + }() for _, node := range mocks.StateDiffs { err = ind.PushStateNode(tx, node) if err != nil { @@ -159,7 +164,7 @@ func setup(t *testing.T) { } } - test_helpers.ExpectEqual(t, tx.BlockNumber, mocks.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) } func tearDown(t *testing.T) { @@ -169,9 +174,9 @@ func tearDown(t *testing.T) { } } -func TestPublishAndIndexer(t *testing.T) { +func TestSQLXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) pgStr := `SELECT cid, td, reward, id, base_fee FROM eth.header_cids @@ -208,7 +213,7 @@ func TestPublishAndIndexer(t *testing.T) { }) t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) // check that txs were properly indexed trxs := make([]string, 0) @@ -318,7 +323,7 @@ func TestPublishAndIndexer(t *testing.T) { }) t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) rcts := make([]string, 0) @@ -368,7 +373,7 @@ func TestPublishAndIndexer(t *testing.T) { }) t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) // check receipts were properly indexed @@ -470,7 +475,7 @@ func TestPublishAndIndexer(t *testing.T) { }) t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) // check that state nodes were properly indexed and published stateNodes := make([]models.StateNodeModel, 0) @@ -548,18 +553,18 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey := dshelp.MultihashToDsKey(dc.Hash()) prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() - test_helpers.ExpectEqual(t, prefixedKey, sql.RemovedNodeMhKey) + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) } - test_helpers.ExpectEqual(t, stateNode.CID, sql.RemovedNodeStateCID) + test_helpers.ExpectEqual(t, stateNode.CID, shared.RemovedNodeStateCID) test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x02'}) test_helpers.ExpectEqual(t, data, []byte{}) }) t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { - setup(t) + setupSQLX(t) defer tearDown(t) // check that storage nodes were properly indexed storageNodes := make([]models.StorageNodeWithStateKeyModel, 0) @@ -608,7 +613,7 @@ func TestPublishAndIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, len(storageNodes), 1) test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ - CID: sql.RemovedNodeStorageCID, + CID: shared.RemovedNodeStorageCID, NodeType: 3, StorageKey: common.BytesToHash(mocks.RemovedLeafKey).Hex(), StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), @@ -620,7 +625,7 @@ func TestPublishAndIndexer(t *testing.T) { } mhKey = dshelp.MultihashToDsKey(dc.Hash()) prefixedKey = blockstore.BlockPrefix.String() + mhKey.String() - test_helpers.ExpectEqual(t, prefixedKey, sql.RemovedNodeMhKey) + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) err = db.Get(context.Background(), &data, ipfsPgGet, prefixedKey) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/test_helpers.go b/statediff/indexer/database/sql/test_helpers.go index cebddb9d1..7de7beec0 100644 --- a/statediff/indexer/database/sql/test_helpers.go +++ b/statediff/indexer/database/sql/test_helpers.go @@ -49,6 +49,18 @@ func TearDownDB(t *testing.T, db Database) { if err != nil { t.Fatal(err) } + _, err = tx.Exec(ctx, `DELETE FROM eth.state_accounts`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(ctx, `DELETE FROM eth.access_list_element`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(ctx, `DELETE FROM eth.log_cids`) + if err != nil { + t.Fatal(err) + } _, err = tx.Exec(ctx, `DELETE FROM blocks`) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/shared/db_kind.go b/statediff/indexer/shared/db_kind.go index 711f9d050..78aae9f7f 100644 --- a/statediff/indexer/shared/db_kind.go +++ b/statediff/indexer/shared/db_kind.go @@ -16,9 +16,28 @@ package shared +import ( + "fmt" + "strings" +) + +// DBType to explicitly type the kind of DB type DBType string const ( POSTGRES DBType = "Postgres" DUMP DBType = "Dump" + UNKOWN DBType = "Unknown" ) + +// ResolveDBType resolves a DBType from a provided string +func ResolveDBType(str string) (DBType, error) { + switch strings.ToLower(str) { + case "postgres", "pg": + return POSTGRES, nil + case "dump", "d": + return DUMP, nil + default: + return UNKOWN, fmt.Errorf("unrecognized db type string: %s", str) + } +} diff --git a/statediff/service.go b/statediff/service.go index ae2e34c6c..8557f87d0 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -218,7 +218,6 @@ func (lbc *BlockCache) getParentBlock(currentBlock *types.Block, bc blockChain) type workerParams struct { chainEventCh <-chan core.ChainEvent - errCh <-chan error wg *sync.WaitGroup id uint } @@ -239,14 +238,21 @@ func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) { statediffMetrics.lastEventHeight.Update(int64(chainEvent.Block.Number().Uint64())) statediffMetrics.writeLoopChannelLen.Update(int64(len(chainEventCh))) chainEventFwd <- chainEvent + case err := <-errCh: + log.Error("Error from chain event subscription", "error", err) + close(sds.QuitChan) case <-sds.QuitChan: + log.Info("Quitting the statediffing writing loop") + if err := sds.indexer.Close(); err != nil { + log.Error("Error closing indexer", "err", err) + } return } } }() wg.Add(int(sds.numWorkers)) for worker := uint(0); worker < sds.numWorkers; worker++ { - params := workerParams{chainEventCh: chainEventFwd, errCh: errCh, wg: &wg, id: worker} + params := workerParams{chainEventCh: chainEventFwd, wg: &wg, id: worker} go sds.writeLoopWorker(params) } wg.Wait() @@ -291,13 +297,8 @@ func (sds *Service) writeLoopWorker(params workerParams) { } // TODO: how to handle with concurrent workers statediffMetrics.lastStatediffHeight.Update(int64(currentBlock.Number().Uint64())) - case err := <-params.errCh: - log.Warn("Error from chain event subscription", "error", err, "worker", params.id) - sds.close() - return case <-sds.QuitChan: log.Info("Quitting the statediff writing process", "worker", params.id) - sds.close() return } } @@ -335,11 +336,10 @@ func (sds *Service) Loop(chainEventCh chan core.ChainEvent) { sds.streamStateDiff(currentBlock, parentBlock.Root()) case err := <-errCh: - log.Warn("Error from chain event subscription", "error", err) - sds.close() - return + log.Error("Error from chain event subscription", "error", err) + close(sds.QuitChan) case <-sds.QuitChan: - log.Info("Quitting the statediffing process") + log.Info("Quitting the statediffing listening loop") sds.close() return } -- 2.45.2 From 4fb92b535815a21035cb52c03c3dff3aaef1e758 Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 12 Nov 2021 08:49:06 -0600 Subject: [PATCH 04/33] update example command in readme --- statediff/README.md | 2 +- statediff/service.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/statediff/README.md b/statediff/README.md index bd5d1d43c..97666d50a 100644 --- a/statediff/README.md +++ b/statediff/README.md @@ -100,7 +100,7 @@ The service can only operate in full sync mode (`--syncmode=full`), but only the e.g. ` -./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db=postgres://localhost:5432/vulcanize_testing?sslmode=disable --statediff.dbnodeid={nodeId} --statediff.dbclientname={dbClientName} +./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db.type=postgres --statediff.db.driver=sqlx --statediff.db.host=localhost --statediff.db.port=5432 --statediff.db.name=vulcanize_test --statediff.db.user=postgres --statediff.db.nodeid=nodeid --statediff.db.clientname=clientname ` ### RPC endpoints diff --git a/statediff/service.go b/statediff/service.go index 8557f87d0..31a56b809 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -306,6 +306,7 @@ func (sds *Service) writeLoopWorker(params workerParams) { // Loop is the main processing method func (sds *Service) Loop(chainEventCh chan core.ChainEvent) { + log.Info("Starting statediff listening loop") chainEventSub := sds.BlockChain.SubscribeChainEvent(chainEventCh) defer chainEventSub.Unsubscribe() errCh := chainEventSub.Err() -- 2.45.2 From 5f4968b3cd91047330ce8f3178ccff82f9a383dc Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 12 Nov 2021 09:03:56 -0600 Subject: [PATCH 05/33] ashwin's fix for failing pgx unit test --- statediff/indexer/database/sql/postgres/pgx_test.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/statediff/indexer/database/sql/postgres/pgx_test.go b/statediff/indexer/database/sql/postgres/pgx_test.go index 50b6817eb..ea66737f5 100644 --- a/statediff/indexer/database/sql/postgres/pgx_test.go +++ b/statediff/indexer/database/sql/postgres/pgx_test.go @@ -23,7 +23,6 @@ import ( "strings" "testing" - "github.com/jackc/pgx/pgtype" "github.com/jackc/pgx/v4/pgxpool" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" @@ -86,15 +85,15 @@ func TestPostgresPGX(t *testing.T) { t.Fatal(err) } - var data pgtype.Text - err = dbPool.QueryRow(ctx, `SELECT data FROM example WHERE id = 1`).Scan(&data) + var data string + err = dbPool.QueryRow(ctx, `SELECT cast(data AS TEXT) FROM example WHERE id = 1`).Scan(&data) if err != nil { t.Fatal(err) } test_helpers.ExpectEqual(t, data, bi.String()) actual := new(big.Int) - actual.SetString(data.String, 10) + actual.SetString(data, 10) test_helpers.ExpectEqual(t, actual, bi) }) -- 2.45.2 From 9a67034f2940748e34faee0b73db6008b645a6ef Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 12 Nov 2021 14:43:26 -0600 Subject: [PATCH 06/33] linting fixes --- statediff/indexer/database/dump/indexer.go | 5 +- statediff/indexer/database/sql/indexer.go | 5 +- statediff/indexer/database/sql/interfaces.go | 1 - .../database/sql/pgx_indexer_legacy_test.go | 3 +- .../indexer/database/sql/pgx_indexer_test.go | 32 +++++------ .../indexer/database/sql/postgres/config.go | 2 +- .../indexer/database/sql/postgres/pgx.go | 20 +------ .../database/sql/sqlx_indexer_legacy_test.go | 8 +-- .../indexer/database/sql/sqlx_indexer_test.go | 57 ++++++++----------- statediff/indexer/models/models.go | 2 +- statediff/indexer/shared/db_kind.go | 4 +- 11 files changed, 52 insertions(+), 87 deletions(-) diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index ccbe28c66..357a78ece 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -315,10 +315,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs Data: trx.Data(), CID: txNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(txNode.Cid()), - } - txType := trx.Type() - if txType != types.LegacyTxType { - txModel.Type = &txType + Type: trx.Type(), } if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", txModel); err != nil { return err diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index b9cfd0733..fad68bf96 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -366,10 +366,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs Data: trx.Data(), CID: txNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(txNode.Cid()), - } - txType := trx.Type() - if txType != types.LegacyTxType { - txModel.Type = &txType + Type: trx.Type(), } txID, err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel, args.headerID) if err != nil { diff --git a/statediff/indexer/database/sql/interfaces.go b/statediff/indexer/database/sql/interfaces.go index 755c4e156..3ed1a11e7 100644 --- a/statediff/indexer/database/sql/interfaces.go +++ b/statediff/indexer/database/sql/interfaces.go @@ -67,7 +67,6 @@ type Tx interface { // ScannableRow interface to accommodate different concrete row types type ScannableRow interface { Scan(dest ...interface{}) error - StructScan(dest interface{}) error } // Result interface to accommodate different concrete result types diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go index f97f3e257..d08336e63 100644 --- a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -20,6 +20,7 @@ import ( "context" "testing" + "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -76,7 +77,7 @@ func TestPGXIndexerLegacy(t *testing.T) { } header := new(res) - err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) require.NoError(t, err) test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index 730257b21..f63efe712 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -26,6 +26,7 @@ import ( "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" + "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -164,7 +165,7 @@ func TestPGXIndexer(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) if err != nil { t.Fatal(err) } @@ -216,12 +217,12 @@ func TestPGXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } + txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -231,8 +232,7 @@ func TestPGXIndexer(t *testing.T) { case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -242,8 +242,7 @@ func TestPGXIndexer(t *testing.T) { case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -253,8 +252,7 @@ func TestPGXIndexer(t *testing.T) { case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -284,8 +282,7 @@ func TestPGXIndexer(t *testing.T) { case trx5CID.String(): test_helpers.ExpectEqual(t, data, tx5) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -398,6 +395,7 @@ func TestPGXIndexer(t *testing.T) { t.Fatal(err) } + postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` switch c { case rct1CID.String(): test_helpers.ExpectEqual(t, data, rct1) @@ -411,8 +409,7 @@ func TestPGXIndexer(t *testing.T) { case rct2CID.String(): test_helpers.ExpectEqual(t, data, rct2) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -420,8 +417,7 @@ func TestPGXIndexer(t *testing.T) { case rct3CID.String(): test_helpers.ExpectEqual(t, data, rct3) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -429,8 +425,7 @@ func TestPGXIndexer(t *testing.T) { case rct4CID.String(): test_helpers.ExpectEqual(t, data, rct4) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -438,8 +433,7 @@ func TestPGXIndexer(t *testing.T) { case rct5CID.String(): test_helpers.ExpectEqual(t, data, rct5) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index aff7ac773..a7c7cc9b4 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -24,7 +24,7 @@ import ( "github.com/ethereum/go-ethereum/statediff/indexer/shared" ) -// DriverType to explicity type the kind of sql driver we are using +// DriverType to explicitly type the kind of sql driver we are using type DriverType string const ( diff --git a/statediff/indexer/database/sql/postgres/pgx.go b/statediff/indexer/database/sql/postgres/pgx.go index 838c78911..9f6701400 100644 --- a/statediff/indexer/database/sql/postgres/pgx.go +++ b/statediff/indexer/database/sql/postgres/pgx.go @@ -105,8 +105,7 @@ func (pgx *PGXDriver) createNode() error { // QueryRow satisfies sql.Database func (pgx *PGXDriver) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { - rows, _ := pgx.pool.Query(ctx, sql, args...) - return rowsWrapper{rows: rows} + return pgx.pool.QueryRow(ctx, sql, args...) } // Exec satisfies sql.Database @@ -160,20 +159,6 @@ func (pgx *PGXDriver) Context() context.Context { return pgx.ctx } -type rowsWrapper struct { - rows pgx.Rows -} - -// Scan satisfies sql.ScannableRow -func (r rowsWrapper) Scan(dest ...interface{}) error { - return (pgx.Row)(r.rows).Scan(dest...) -} - -// StructScan satisfies sql.ScannableRow -func (r rowsWrapper) StructScan(dest interface{}) error { - return pgxscan.ScanRow(dest, r.rows) -} - type resultWrapper struct { ct pgconn.CommandTag } @@ -234,8 +219,7 @@ type pgxTxWrapper struct { // QueryRow satisfies sql.Tx func (t pgxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow { - rows, _ := t.tx.Query(ctx, sql, args...) - return rowsWrapper{rows: rows} + return t.tx.QueryRow(ctx, sql, args...) } // Exec satisfies sql.Tx diff --git a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go index 840a1ccae..2ce5f494f 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go @@ -20,15 +20,15 @@ import ( "context" "testing" - "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" - "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" - "github.com/ipfs/go-cid" + "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" @@ -85,7 +85,7 @@ func TestSQLXIndexerLegacy(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) require.NoError(t, err) test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 815d36915..0fa4e8c1a 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -26,6 +26,7 @@ import ( "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" + "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -190,7 +191,7 @@ func TestSQLXIndexer(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) if err != nil { t.Fatal(err) } @@ -242,50 +243,47 @@ func TestSQLXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } + txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) - var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + var txType uint8 + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) - var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + var txType uint8 + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) - var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + var txType uint8 + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) - var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + var txType uint8 + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if *txType != types.AccessListTxType { - t.Fatalf("expected AccessListTxType (1), got %d", *txType) + if txType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txType) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.id) WHERE cid = $1 ORDER BY access_list_element.index ASC` @@ -310,8 +308,7 @@ func TestSQLXIndexer(t *testing.T) { case trx5CID.String(): test_helpers.ExpectEqual(t, data, tx5) var txType *uint8 - pgStr = `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` - err = db.Get(context.Background(), &txType, pgStr, c) + err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } @@ -423,7 +420,7 @@ func TestSQLXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - + postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` switch c { case rct1CID.String(): test_helpers.ExpectEqual(t, data, rct1) @@ -437,8 +434,7 @@ func TestSQLXIndexer(t *testing.T) { case rct2CID.String(): test_helpers.ExpectEqual(t, data, rct2) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -446,8 +442,7 @@ func TestSQLXIndexer(t *testing.T) { case rct3CID.String(): test_helpers.ExpectEqual(t, data, rct3) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -455,8 +450,7 @@ func TestSQLXIndexer(t *testing.T) { case rct4CID.String(): test_helpers.ExpectEqual(t, data, rct4) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } @@ -464,8 +458,7 @@ func TestSQLXIndexer(t *testing.T) { case rct5CID.String(): test_helpers.ExpectEqual(t, data, rct5) var postState string - pgStr = `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` - err = db.Get(context.Background(), &postState, pgStr, c) + err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { t.Fatal(err) } diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index 72efe2383..5e849193e 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -67,7 +67,7 @@ type TxModel struct { Dst string `db:"dst"` Src string `db:"src"` Data []byte `db:"tx_data"` - Type *uint8 `db:"tx_type"` + Type uint8 `db:"tx_type"` } // AccessListElementModel is the db model for eth.access_list_entry diff --git a/statediff/indexer/shared/db_kind.go b/statediff/indexer/shared/db_kind.go index 78aae9f7f..6b88164e1 100644 --- a/statediff/indexer/shared/db_kind.go +++ b/statediff/indexer/shared/db_kind.go @@ -27,7 +27,7 @@ type DBType string const ( POSTGRES DBType = "Postgres" DUMP DBType = "Dump" - UNKOWN DBType = "Unknown" + UNKNOWN DBType = "Unknown" ) // ResolveDBType resolves a DBType from a provided string @@ -38,6 +38,6 @@ func ResolveDBType(str string) (DBType, error) { case "dump", "d": return DUMP, nil default: - return UNKOWN, fmt.Errorf("unrecognized db type string: %s", str) + return UNKNOWN, fmt.Errorf("unrecognized db type string: %s", str) } } -- 2.45.2 From b36b3f83cb794dd487596180c0a0aede392563eb Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 17 Nov 2021 10:25:11 -0600 Subject: [PATCH 07/33] update to use new schema; fix pgx driver --- statediff/builder.go | 6 +- statediff/indexer/database/dump/indexer.go | 117 +++++++------- statediff/indexer/database/sql/batch_tx.go | 1 - statediff/indexer/database/sql/indexer.go | 147 +++++++++--------- .../indexer/database/sql/postgres/config.go | 3 + .../indexer/database/sql/postgres/database.go | 28 ++-- .../indexer/database/sql/postgres/pgx.go | 5 - .../indexer/database/sql/postgres/sqlx.go | 5 - statediff/indexer/database/sql/writer.go | 101 ++++++++---- statediff/indexer/interfaces/interfaces.go | 2 +- statediff/indexer/models/batch.go | 55 ++++--- statediff/indexer/models/models.go | 31 ++-- statediff/service.go | 2 +- 13 files changed, 263 insertions(+), 240 deletions(-) diff --git a/statediff/builder.go b/statediff/builder.go index eacfeca15..8dc3cece8 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -23,16 +23,14 @@ import ( "bytes" "fmt" - "github.com/ethereum/go-ethereum/statediff/trie_helpers" - - types2 "github.com/ethereum/go-ethereum/statediff/types" - "github.com/ethereum/go-ethereum/common" "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/log" "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/trie_helpers" + types2 "github.com/ethereum/go-ethereum/statediff/types" "github.com/ethereum/go-ethereum/trie" ) diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index 357a78ece..b75fb1af9 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -136,7 +136,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip t = time.Now() // Publish and index header, collect headerID - var headerID int64 + var headerID string headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) if err != nil { return nil, err @@ -181,7 +181,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // processHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { +func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (string, error) { tx.cacheIPLD(headerNode) var baseFee *int64 @@ -190,12 +190,13 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he *baseFee = header.BaseFee.Int64() } + headerID := header.Hash().String() mod := models.HeaderModel{ CID: headerNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), ParentHash: header.ParentHash.String(), BlockNumber: header.Number.String(), - BlockHash: header.Hash().String(), + BlockHash: headerID, TotalDifficulty: td.String(), Reward: reward.String(), Bloom: header.Bloom.Bytes(), @@ -207,11 +208,11 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he BaseFee: baseFee, } _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", mod) - return 0, err + return headerID, err } // processUncles publishes and indexes uncle IPLDs in Postgres -func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { +func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { // publish and index uncles for _, uncleNode := range uncleNodes { tx.cacheIPLD(uncleNode) @@ -223,6 +224,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNum uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) } uncle := models.UncleModel{ + HeaderID: headerID, CID: uncleNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(uncleNode.Cid()), ParentHash: uncleNode.ParentHash.String(), @@ -238,7 +240,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNum // processArgs bundles arguments to processReceiptsAndTxs type processArgs struct { - headerID int64 + headerID string blockNumber *big.Int receipts types.Receipts txs types.Transactions @@ -263,59 +265,24 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs tx.cacheIPLD(txNode) // Indexing - // extract topic and contract data from the receipt for indexing - mappedContracts := make(map[string]bool) // use map to avoid duplicate addresses - 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{ - 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], - } - } - // these are the contracts seen in the logs - logContracts := make([]string, 0, len(mappedContracts)) - for addr := range mappedContracts { - logContracts = append(logContracts, addr) - } - // this is the contract address if this receipt is for a contract creation tx - contract := shared.HandleZeroAddr(receipt.ContractAddress) - var contractHash string - if contract != "" { - contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() - } - // index tx first so that the receipt can reference it by FK + // index tx trx := args.txs[i] + trxID := trx.Hash().String() // derive sender for the tx that corresponds with this receipt from, err := types.Sender(signer, trx) if err != nil { return fmt.Errorf("error deriving tx sender: %v", err) } txModel := models.TxModel{ - Dst: shared.HandleZeroAddrPointer(trx.To()), - Src: shared.HandleZeroAddr(from), - TxHash: trx.Hash().String(), - Index: int64(i), - Data: trx.Data(), - CID: txNode.Cid().String(), - MhKey: shared.MultihashKeyFromCID(txNode.Cid()), - Type: trx.Type(), + HeaderID: args.headerID, + Dst: shared.HandleZeroAddrPointer(trx.To()), + Src: shared.HandleZeroAddr(from), + TxHash: trxID, + Index: int64(i), + Data: trx.Data(), + CID: txNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(txNode.Cid()), + Type: trx.Type(), } if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", txModel); err != nil { return err @@ -328,6 +295,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs storageKeys[k] = storageKey.Hex() } accessListElementModel := models.AccessListElementModel{ + TxID: trxID, Index: int64(j), Address: accessListElement.Address.Hex(), StorageKeys: storageKeys, @@ -337,12 +305,20 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs } } + // this is the contract address if this receipt is for a contract creation tx + contract := shared.HandleZeroAddr(receipt.ContractAddress) + var contractHash string + if contract != "" { + contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() + } + // index the receipt if !args.rctLeafNodeCIDs[i].Defined() { return fmt.Errorf("invalid receipt leaf node cid") } rctModel := &models.ReceiptModel{ + TxID: trxID, Contract: contract, ContractHash: contractHash, LeafCID: args.rctLeafNodeCIDs[i].String(), @@ -359,6 +335,31 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs return err } + 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") + } + + logDataSet[idx] = &models.LogsModel{ + ReceiptID: trxID, + 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], + } + } + if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", logDataSet); err != nil { return err } @@ -374,7 +375,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs } // PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql -func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode) error { +func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode, headerID string) error { tx, ok := batch.(*BatchTx) if !ok { return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) @@ -384,6 +385,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt // short circuit if it is a Removed node // this assumes the db has been initialized and a public.blocks entry for the Removed node is present stateModel := models.StateNodeModel{ + HeaderID: headerID, Path: stateNode.Path, StateKey: common.BytesToHash(stateNode.LeafKey).String(), CID: shared.RemovedNodeStateCID, @@ -398,6 +400,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) } stateModel := models.StateNodeModel{ + HeaderID: headerID, Path: stateNode.Path, StateKey: common.BytesToHash(stateNode.LeafKey).String(), CID: stateCIDStr, @@ -422,6 +425,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt return fmt.Errorf("error decoding state account rlp: %s", err.Error()) } accountModel := models.StateAccountModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Balance: account.Balance.String(), Nonce: account.Nonce, CodeHash: account.CodeHash, @@ -437,6 +442,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt // short circuit if it is a Removed node // this assumes the db has been initialized and a public.blocks entry for the Removed node is present storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Path: storageNode.Path, StorageKey: common.BytesToHash(storageNode.LeafKey).String(), CID: shared.RemovedNodeStorageCID, @@ -453,6 +460,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) } storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Path: storageNode.Path, StorageKey: common.BytesToHash(storageNode.LeafKey).String(), CID: storageCIDStr, @@ -482,7 +491,7 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAnd return nil } -// Close satisfied io.Closer +// Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { return sdi.dump.Close() } diff --git a/statediff/indexer/database/sql/batch_tx.go b/statediff/indexer/database/sql/batch_tx.go index ff847eec6..fb1b289a1 100644 --- a/statediff/indexer/database/sql/batch_tx.go +++ b/statediff/indexer/database/sql/batch_tx.go @@ -34,7 +34,6 @@ type BatchTx struct { BlockNumber uint64 ctx context.Context dbtx Tx - headerID int64 stm string quit chan struct{} iplds chan models.IPLDModel diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index fad68bf96..1e89f92ff 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -187,7 +187,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip t = time.Now() // Publish and index header, collect headerID - var headerID int64 + var headerID string headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) if err != nil { return nil, err @@ -227,13 +227,12 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String()) t = time.Now() - blockTx.headerID = headerID return blockTx, err } // processHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (int64, error) { +func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (string, error) { tx.cacheIPLD(headerNode) var baseFee *int64 @@ -241,14 +240,14 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he baseFee = new(int64) *baseFee = header.BaseFee.Int64() } - + headerID := header.Hash().String() // index header - return sdi.dbWriter.upsertHeaderCID(tx.dbtx, models.HeaderModel{ + return headerID, sdi.dbWriter.upsertHeaderCID(tx.dbtx, models.HeaderModel{ CID: headerNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), ParentHash: header.ParentHash.String(), BlockNumber: header.Number.String(), - BlockHash: header.Hash().String(), + BlockHash: headerID, TotalDifficulty: td.String(), Reward: reward.String(), Bloom: header.Bloom.Bytes(), @@ -262,7 +261,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he } // processUncles publishes and indexes uncle IPLDs in Postgres -func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { +func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber uint64, uncleNodes []*ipld2.EthHeader) error { // publish and index uncles for _, uncleNode := range uncleNodes { tx.cacheIPLD(uncleNode) @@ -274,13 +273,14 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNum uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) } uncle := models.UncleModel{ + HeaderID: headerID, CID: uncleNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(uncleNode.Cid()), ParentHash: uncleNode.ParentHash.String(), BlockHash: uncleNode.Hash().String(), Reward: uncleReward.String(), } - if err := sdi.dbWriter.upsertUncleCID(tx.dbtx, uncle, headerID); err != nil { + if err := sdi.dbWriter.upsertUncleCID(tx.dbtx, uncle); err != nil { return err } } @@ -289,7 +289,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID int64, blockNum // processArgs bundles arguments to processReceiptsAndTxs type processArgs struct { - headerID int64 + headerID string blockNumber *big.Int receipts types.Receipts txs types.Transactions @@ -313,63 +313,26 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs txNode := args.txNodes[i] tx.cacheIPLD(txNode) - // Indexing - // extract topic and contract data from the receipt for indexing - mappedContracts := make(map[string]bool) // use map to avoid duplicate addresses - 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{ - 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], - } - } - // these are the contracts seen in the logs - logContracts := make([]string, 0, len(mappedContracts)) - for addr := range mappedContracts { - logContracts = append(logContracts, addr) - } - // this is the contract address if this receipt is for a contract creation tx - contract := shared.HandleZeroAddr(receipt.ContractAddress) - var contractHash string - if contract != "" { - contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() - } - // index tx first so that the receipt can reference it by FK + // index tx trx := args.txs[i] + txID := trx.Hash().String() // derive sender for the tx that corresponds with this receipt from, err := types.Sender(signer, trx) if err != nil { return fmt.Errorf("error deriving tx sender: %v", err) } txModel := models.TxModel{ - Dst: shared.HandleZeroAddrPointer(trx.To()), - Src: shared.HandleZeroAddr(from), - TxHash: trx.Hash().String(), - Index: int64(i), - Data: trx.Data(), - CID: txNode.Cid().String(), - MhKey: shared.MultihashKeyFromCID(txNode.Cid()), - Type: trx.Type(), + HeaderID: args.headerID, + Dst: shared.HandleZeroAddrPointer(trx.To()), + Src: shared.HandleZeroAddr(from), + TxHash: txID, + Index: int64(i), + Data: trx.Data(), + CID: txNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(txNode.Cid()), + Type: trx.Type(), } - txID, err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel, args.headerID) - if err != nil { + if err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel); err != nil { return err } @@ -380,21 +343,30 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs storageKeys[k] = storageKey.Hex() } accessListElementModel := models.AccessListElementModel{ + TxID: txID, Index: int64(j), Address: accessListElement.Address.Hex(), StorageKeys: storageKeys, } - if err := sdi.dbWriter.upsertAccessListElement(tx.dbtx, accessListElementModel, txID); err != nil { + if err := sdi.dbWriter.upsertAccessListElement(tx.dbtx, accessListElementModel); err != nil { return err } } - // index the receipt + // this is the contract address if this receipt is for a contract creation tx + contract := shared.HandleZeroAddr(receipt.ContractAddress) + var contractHash string + if contract != "" { + contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() + } + + // index receipt if !args.rctLeafNodeCIDs[i].Defined() { return fmt.Errorf("invalid receipt leaf node cid") } rctModel := &models.ReceiptModel{ + TxID: txID, Contract: contract, ContractHash: contractHash, LeafCID: args.rctLeafNodeCIDs[i].String(), @@ -407,12 +379,37 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs rctModel.PostState = common.Bytes2Hex(receipt.PostState) } - receiptID, err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel, txID) - if err != nil { + if err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel); err != nil { return err } - if err = sdi.dbWriter.upsertLogCID(tx.dbtx, logDataSet, receiptID); err != nil { + // index logs + 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") + } + + logDataSet[idx] = &models.LogsModel{ + ReceiptID: txID, + 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], + } + } + + if err := sdi.dbWriter.upsertLogCID(tx.dbtx, logDataSet); err != nil { return err } } @@ -427,7 +424,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs } // PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql -func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode) error { +func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode, headerID string) error { tx, ok := batch.(*BatchTx) if !ok { return fmt.Errorf("sql batch is expected to be of type %T, got %T", &BatchTx{}, batch) @@ -437,29 +434,29 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt // short circuit if it is a Removed node // this assumes the db has been initialized and a public.blocks entry for the Removed node is present stateModel := models.StateNodeModel{ + HeaderID: headerID, Path: stateNode.Path, StateKey: common.BytesToHash(stateNode.LeafKey).String(), CID: shared.RemovedNodeStateCID, MhKey: shared.RemovedNodeMhKey, NodeType: stateNode.NodeType.Int(), } - _, err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel, tx.headerID) - return err + return sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel) } stateCIDStr, stateMhKey, err := tx.cacheRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) if err != nil { return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) } stateModel := models.StateNodeModel{ + HeaderID: headerID, Path: stateNode.Path, StateKey: common.BytesToHash(stateNode.LeafKey).String(), CID: stateCIDStr, MhKey: stateMhKey, NodeType: stateNode.NodeType.Int(), } - // index the state node, collect the stateID to reference by FK - stateID, err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel, tx.headerID) - if err != nil { + // index the state node + if err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel); err != nil { return err } // if we have a leaf, decode and index the account data @@ -476,12 +473,14 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt return fmt.Errorf("error decoding state account rlp: %s", err.Error()) } accountModel := models.StateAccountModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Balance: account.Balance.String(), Nonce: account.Nonce, CodeHash: account.CodeHash, StorageRoot: account.Root.String(), } - if err := sdi.dbWriter.upsertStateAccount(tx.dbtx, accountModel, stateID); err != nil { + if err := sdi.dbWriter.upsertStateAccount(tx.dbtx, accountModel); err != nil { return err } } @@ -491,13 +490,15 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt // short circuit if it is a Removed node // this assumes the db has been initialized and a public.blocks entry for the Removed node is present storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Path: storageNode.Path, StorageKey: common.BytesToHash(storageNode.LeafKey).String(), CID: shared.RemovedNodeStorageCID, MhKey: shared.RemovedNodeMhKey, NodeType: storageNode.NodeType.Int(), } - if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel, stateID); err != nil { + if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel); err != nil { return err } continue @@ -507,13 +508,15 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) } storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, Path: storageNode.Path, StorageKey: common.BytesToHash(storageNode.LeafKey).String(), CID: storageCIDStr, MhKey: storageMhKey, NodeType: storageNode.NodeType.Int(), } - if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel, stateID); err != nil { + if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel); err != nil { return err } } diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index a7c7cc9b4..825e50163 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -30,6 +30,7 @@ type DriverType string const ( PGX DriverType = "PGX" SQLX DriverType = "SQLX" + FILE DriverType = "File" Unknown DriverType = "Unknown" ) @@ -40,6 +41,8 @@ func ResolveDriverType(str string) (DriverType, error) { return PGX, nil case "sqlx": return SQLX, nil + case "file": + return FILE, nil default: return Unknown, fmt.Errorf("unrecognized driver type string: %s", str) } diff --git a/statediff/indexer/database/sql/postgres/database.go b/statediff/indexer/database/sql/postgres/database.go index 3fe7f652e..ef091760d 100644 --- a/statediff/indexer/database/sql/postgres/database.go +++ b/statediff/indexer/database/sql/postgres/database.go @@ -46,59 +46,55 @@ type DB struct { func (db *DB) InsertHeaderStm() string { return `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) - ON CONFLICT (block_number, block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) - RETURNING id` + ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16)` } // InsertUncleStm satisfies the sql.Statements interface func (db *DB) InsertUncleStm() string { return `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES ($1, $2, $3, $4, $5, $6) - ON CONFLICT (header_id, block_hash) DO UPDATE SET (parent_hash, cid, reward, mh_key) = ($3, $4, $5, $6)` + ON CONFLICT (block_hash) DO NOTHING` } // InsertTxStm satisfies the sql.Statements interface func (db *DB) InsertTxStm() string { return `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) - ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = ($3, $4, $5, $6, $7, $8, $9) - RETURNING id` + ON CONFLICT (tx_hash) DO NOTHING` } // InsertAccessListElementStm satisfies the sql.Statements interface func (db *DB) InsertAccessListElementStm() string { return `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) - ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)` + ON CONFLICT (tx_id, index) DO NOTHING` } // InsertRctStm satisfies the sql.Statements interface func (db *DB) InsertRctStm() string { return `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) - ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8) - RETURNING id` + ON CONFLICT (tx_id) DO NOTHING` } // InsertLogStm satisfies the sql.Statements interface func (db *DB) InsertLogStm() string { - return `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)` + return `INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10) + ON CONFLICT (rct_id, index) DO NOTHING` } // InsertStateStm satisfies the sql.Statements interface func (db *DB) InsertStateStm() string { return `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) - RETURNING id` + ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)` } // InsertAccountStm satisfies the sql.Statements interface func (db *DB) InsertAccountStm() string { - return `INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5) - ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = ($2, $3, $4, $5)` + return `INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5, $6) + ON CONFLICT (header_id, state_path) DO NOTHING` } // InsertStorageStm satisfies the sql.Statements interface func (db *DB) InsertStorageStm() string { - return `INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)` + return `INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) + ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($3, $4, $6, $7, $8)` } // InsertIPLDStm satisfies the sql.Statements interface diff --git a/statediff/indexer/database/sql/postgres/pgx.go b/statediff/indexer/database/sql/postgres/pgx.go index 9f6701400..fa9b84dd0 100644 --- a/statediff/indexer/database/sql/postgres/pgx.go +++ b/statediff/indexer/database/sql/postgres/pgx.go @@ -138,11 +138,6 @@ func (pgx *PGXDriver) Stats() sql.Stats { return pgxStatsWrapper{stats: stats} } -// NodeInfo satisfies sql.Database -func (pgx *PGXDriver) NodeInfo() node.Info { - return pgx.nodeInfo -} - // NodeID satisfies sql.Database func (pgx *PGXDriver) NodeID() int64 { return pgx.nodeID diff --git a/statediff/indexer/database/sql/postgres/sqlx.go b/statediff/indexer/database/sql/postgres/sqlx.go index 684fc7bf0..0bbd0d9e9 100644 --- a/statediff/indexer/database/sql/postgres/sqlx.go +++ b/statediff/indexer/database/sql/postgres/sqlx.go @@ -107,11 +107,6 @@ func (driver *SQLXDriver) Stats() sql.Stats { return sqlxStatsWrapper{stats: stats} } -// NodeInfo satisfies sql.Database -func (driver *SQLXDriver) NodeInfo() node.Info { - return driver.nodeInfo -} - // NodeID satisfies sql.Database func (driver *SQLXDriver) NodeID() int64 { return driver.nodeID diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go index ea276dfbf..96d13d956 100644 --- a/statediff/indexer/database/sql/writer.go +++ b/statediff/indexer/database/sql/writer.go @@ -39,41 +39,56 @@ func NewWriter(db Database) *Writer { } } -func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) (int64, error) { - var headerID int64 - err := tx.QueryRow(in.db.Context(), in.db.InsertHeaderStm(), +/* +INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) +VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) +ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) +*/ +func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertHeaderStm(), header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, in.db.NodeID(), header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee).Scan(&headerID) + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee) if err != nil { - return 0, fmt.Errorf("error upserting header_cids entry: %v", err) + return fmt.Errorf("error upserting header_cids entry: %v", err) } indexerMetrics.blocks.Inc(1) - return headerID, nil + return nil } -func (in *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel, headerID int64) error { +/* +INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES ($1, $2, $3, $4, $5, $6) +ON CONFLICT (block_hash) DO NOTHING +*/ +func (in *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error { _, err := tx.Exec(in.db.Context(), in.db.InsertUncleStm(), - uncle.BlockHash, headerID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey) + uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey) if err != nil { return fmt.Errorf("error upserting uncle_cids entry: %v", err) } return nil } -func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel, headerID int64) (int64, error) { - var txID int64 - err := tx.QueryRow(in.db.Context(), in.db.InsertTxStm(), - headerID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type).Scan(&txID) +/* +INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) +ON CONFLICT (tx_hash) DO NOTHING +*/ +func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertTxStm(), + transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type) if err != nil { - return 0, fmt.Errorf("error upserting transaction_cids entry: %v", err) + return fmt.Errorf("error upserting transaction_cids entry: %v", err) } indexerMetrics.transactions.Inc(1) - return txID, nil + return nil } -func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel, txID int64) error { +/* +INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) +ON CONFLICT (tx_id, index) DO NOTHING +*/ +func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel) error { _, err := tx.Exec(in.db.Context(), in.db.InsertAccessListElementStm(), - txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) + accessListElement.TxID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) if err != nil { return fmt.Errorf("error upserting access_list_element entry: %v", err) } @@ -81,21 +96,28 @@ func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.Access return nil } -func (in *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel, txID int64) (int64, error) { - var receiptID int64 - err := tx.QueryRow(in.db.Context(), in.db.InsertRctStm(), - txID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID) +/* +INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) +ON CONFLICT (tx_id) DO NOTHING +*/ +func (in *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel) error { + _, err := tx.Exec(in.db.Context(), in.db.InsertRctStm(), + rct.TxID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot) if err != nil { - return 0, fmt.Errorf("error upserting receipt_cids entry: %w", err) + return fmt.Errorf("error upserting receipt_cids entry: %w", err) } indexerMetrics.receipts.Inc(1) - return receiptID, nil + return nil } -func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel, receiptID int64) error { +/* +INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10) +ON CONFLICT (rct_id, index) DO NOTHING +*/ +func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error { for _, log := range logs { _, err := tx.Exec(in.db.Context(), in.db.InsertLogStm(), - log.LeafCID, log.LeafMhKey, receiptID, log.Address, log.Index, log.Topic0, log.Topic1, log.Topic2, log.Topic3, log.Data) + log.LeafCID, log.LeafMhKey, log.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) } @@ -104,36 +126,47 @@ func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel, receiptID int64) return nil } -func (in *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel, headerID int64) (int64, error) { - var stateID int64 +/* +INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) +ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) +*/ +func (in *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel) error { var stateKey string if stateNode.StateKey != nullHash.String() { stateKey = stateNode.StateKey } - err := tx.QueryRow(in.db.Context(), in.db.InsertStateStm(), - headerID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey).Scan(&stateID) + _, err := tx.Exec(in.db.Context(), in.db.InsertStateStm(), + stateNode.HeaderID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey) if err != nil { - return 0, fmt.Errorf("error upserting state_cids entry: %v", err) + return fmt.Errorf("error upserting state_cids entry: %v", err) } - return stateID, nil + return nil } -func (in *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel, stateID int64) error { +/* +INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5, $6) +ON CONFLICT (header_id, state_path) DO NOTHING +*/ +func (in *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel) error { _, err := tx.Exec(in.db.Context(), in.db.InsertAccountStm(), - stateID, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) + stateAccount.HeaderID, stateAccount.StatePath, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) if err != nil { return fmt.Errorf("error upserting state_accounts entry: %v", err) } return nil } -func (in *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel, stateID int64) error { +/* +INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) +ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($3, $4, $6, $7, $8) +*/ +func (in *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel) error { var storageKey string if storageCID.StorageKey != nullHash.String() { storageKey = storageCID.StorageKey } _, err := tx.Exec(in.db.Context(), in.db.InsertStorageStm(), - stateID, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) + storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) if err != nil { return fmt.Errorf("error upserting storage_cids entry: %v", err) } diff --git a/statediff/indexer/interfaces/interfaces.go b/statediff/indexer/interfaces/interfaces.go index d32c117eb..8f951230d 100644 --- a/statediff/indexer/interfaces/interfaces.go +++ b/statediff/indexer/interfaces/interfaces.go @@ -29,7 +29,7 @@ import ( // StateDiffIndexer interface required to index statediff data type StateDiffIndexer interface { PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (Batch, error) - PushStateNode(tx Batch, stateNode sdtypes.StateNode) error + PushStateNode(tx Batch, stateNode sdtypes.StateNode, headerID string) error PushCodeAndCodeHash(tx Batch, codeAndCodeHash sdtypes.CodeAndCodeHash) error ReportDBMetrics(delay time.Duration, quit <-chan bool) io.Closer diff --git a/statediff/indexer/models/batch.go b/statediff/indexer/models/batch.go index 48b2944e0..16096f292 100644 --- a/statediff/indexer/models/batch.go +++ b/statediff/indexer/models/batch.go @@ -26,7 +26,7 @@ type IPLDBatch struct { // UncleBatch holds the arguments for a batch insert of uncle data type UncleBatch struct { - HeaderID []int64 + HeaderID []string BlockHashes []string ParentHashes []string CIDs []string @@ -36,7 +36,7 @@ type UncleBatch struct { // TxBatch holds the arguments for a batch insert of tx data type TxBatch struct { - HeaderID int64 + HeaderID string Indexes []int64 TxHashes []string CIDs []string @@ -44,20 +44,20 @@ type TxBatch struct { Dsts []string Srcs []string Datas [][]byte - Types []*uint8 + Types []uint8 } // AccessListBatch holds the arguments for a batch insert of access list data type AccessListBatch struct { Indexes []int64 - TxIDs []int64 + TxIDs []string Addresses []string StorageKeysSets []pq.StringArray } // ReceiptBatch holds the arguments for a batch insert of receipt data type ReceiptBatch struct { - TxIDs []int64 + TxIDs []string LeafCIDs []string LeafMhKeys []string PostStatuses []uint64 @@ -71,7 +71,7 @@ type ReceiptBatch struct { type LogBatch struct { LeafCIDs []string LeafMhKeys []string - ReceiptIDs []int64 + ReceiptIDs []string Addresses []string Indexes []int64 Datas [][]byte @@ -83,34 +83,33 @@ type LogBatch struct { // StateBatch holds the arguments for a batch insert of state data type StateBatch struct { - ID int64 - HeaderID int64 - Path []byte - StateKey string - NodeType int - CID string - MhKey string - Diff bool + HeaderID string + Paths [][]byte + StateKeys []string + NodeTypes []int + CIDs []string + MhKeys []string + Diff bool } // AccountBatch holds the arguments for a batch insert of account data type AccountBatch struct { - ID int64 - StateID int64 - Balance string - Nonce uint64 - CodeHash []byte - StorageRoot string + HeaderID string + StatePaths [][]byte + Balances []string + Nonces []uint64 + CodeHashes [][]byte + StorageRoots []string } // StorageBatch holds the arguments for a batch insert of storage data type StorageBatch struct { - ID int64 - StateID int64 - Path []byte - StorageKey string - NodeType int - CID string - MhKey string - Diff bool + HeaderID string + StatePaths [][]string + Paths [][]byte + StorageKeys []string + NodeTypes []int + CIDs []string + MhKeys []string + Diff bool } diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index 5e849193e..7d2fc43b6 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -26,7 +26,6 @@ type IPLDModel struct { // HeaderModel is the db model for eth.header_cids type HeaderModel struct { - ID int64 `db:"id"` BlockNumber string `db:"block_number"` BlockHash string `db:"block_hash"` ParentHash string `db:"parent_hash"` @@ -47,8 +46,7 @@ type HeaderModel struct { // UncleModel is the db model for eth.uncle_cids type UncleModel struct { - ID int64 `db:"id"` - HeaderID int64 `db:"header_id"` + HeaderID string `db:"header_id"` BlockHash string `db:"block_hash"` ParentHash string `db:"parent_hash"` CID string `db:"cid"` @@ -58,8 +56,7 @@ type UncleModel struct { // TxModel is the db model for eth.transaction_cids type TxModel struct { - ID int64 `db:"id"` - HeaderID int64 `db:"header_id"` + HeaderID string `db:"header_id"` Index int64 `db:"index"` TxHash string `db:"tx_hash"` CID string `db:"cid"` @@ -72,17 +69,15 @@ type TxModel struct { // AccessListElementModel is the db model for eth.access_list_entry type AccessListElementModel struct { - ID int64 `db:"id"` Index int64 `db:"index"` - TxID int64 `db:"tx_id"` + TxID string `db:"tx_id"` Address string `db:"address"` StorageKeys pq.StringArray `db:"storage_keys"` } // ReceiptModel is the db model for eth.receipt_cids type ReceiptModel struct { - ID int64 `db:"id"` - TxID int64 `db:"tx_id"` + TxID string `db:"tx_id"` LeafCID string `db:"leaf_cid"` LeafMhKey string `db:"leaf_mh_key"` PostStatus uint64 `db:"post_status"` @@ -94,8 +89,7 @@ type ReceiptModel struct { // StateNodeModel is the db model for eth.state_cids type StateNodeModel struct { - ID int64 `db:"id"` - HeaderID int64 `db:"header_id"` + HeaderID string `db:"header_id"` Path []byte `db:"state_path"` StateKey string `db:"state_leaf_key"` NodeType int `db:"node_type"` @@ -106,8 +100,8 @@ type StateNodeModel struct { // StorageNodeModel is the db model for eth.storage_cids type StorageNodeModel struct { - ID int64 `db:"id"` - StateID int64 `db:"state_id"` + HeaderID string `db:"header_id""` + StatePath []byte `db:"state_path"` Path []byte `db:"storage_path"` StorageKey string `db:"storage_leaf_key"` NodeType int `db:"node_type"` @@ -118,8 +112,8 @@ type StorageNodeModel struct { // StorageNodeWithStateKeyModel is a db model for eth.storage_cids + eth.state_cids.state_key type StorageNodeWithStateKeyModel struct { - ID int64 `db:"id"` - StateID int64 `db:"state_id"` + HeaderID string `db:"header_id""` + StatePath []byte `db:"state_path"` Path []byte `db:"storage_path"` StateKey string `db:"state_leaf_key"` StorageKey string `db:"storage_leaf_key"` @@ -131,8 +125,8 @@ type StorageNodeWithStateKeyModel struct { // StateAccountModel is a db model for an eth state account (decoded value of state leaf node) type StateAccountModel struct { - ID int64 `db:"id"` - StateID int64 `db:"state_id"` + HeaderID string `db:"header_id"` + StatePath []byte `db:"state_path"` Balance string `db:"balance"` Nonce uint64 `db:"nonce"` CodeHash []byte `db:"code_hash"` @@ -141,10 +135,9 @@ type StateAccountModel struct { // LogsModel is the db model for eth.logs type LogsModel struct { - ID int64 `db:"id"` + ReceiptID string `db:"rct_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"` diff --git a/statediff/service.go b/statediff/service.go index 31a56b809..3fc8ac60c 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -664,7 +664,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p } }() output := func(node types2.StateNode) error { - return sds.indexer.PushStateNode(tx, node) + return sds.indexer.PushStateNode(tx, node, block.Hash().String()) } codeOutput := func(c types2.CodeAndCodeHash) error { return sds.indexer.PushCodeAndCodeHash(tx, c) -- 2.45.2 From 81a8ffd257a8e258ddf01f7782017ab0f9d2ea76 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 17 Nov 2021 12:22:31 -0600 Subject: [PATCH 08/33] indexer that writes sql stmts out to a file --- statediff/indexer/constructor.go | 10 +- statediff/indexer/database/file/batch_tx.go | 29 ++ statediff/indexer/database/file/config.go | 32 ++ statediff/indexer/database/file/helpers.go | 60 +++ statediff/indexer/database/file/indexer.go | 474 ++++++++++++++++++++ statediff/indexer/database/file/metrics.go | 94 ++++ statediff/indexer/database/file/writer.go | 240 ++++++++++ statediff/indexer/database/sql/indexer.go | 2 +- statediff/indexer/shared/db_kind.go | 3 + 9 files changed, 942 insertions(+), 2 deletions(-) create mode 100644 statediff/indexer/database/file/batch_tx.go create mode 100644 statediff/indexer/database/file/config.go create mode 100644 statediff/indexer/database/file/helpers.go create mode 100644 statediff/indexer/database/file/indexer.go create mode 100644 statediff/indexer/database/file/metrics.go create mode 100644 statediff/indexer/database/file/writer.go diff --git a/statediff/indexer/constructor.go b/statediff/indexer/constructor.go index 7a44638d0..a4cfa271d 100644 --- a/statediff/indexer/constructor.go +++ b/statediff/indexer/constructor.go @@ -20,6 +20,8 @@ import ( "context" "fmt" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" + "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" @@ -32,10 +34,16 @@ import ( // NewStateDiffIndexer creates and returns an implementation of the StateDiffIndexer interface func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, nodeInfo node.Info, config interfaces.Config) (interfaces.StateDiffIndexer, error) { switch config.Type() { + case shared.FILE: + fc, ok := config.(file.Config) + if !ok { + return nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{}) + } + return file.NewStateDiffIndexer(ctx, chainConfig, fc) case shared.POSTGRES: pgc, ok := config.(postgres.Config) if !ok { - return nil, fmt.Errorf("ostgres config is not the correct type: got %T, expected %T", config, postgres.Config{}) + return nil, fmt.Errorf("postgres config is not the correct type: got %T, expected %T", config, postgres.Config{}) } var err error var driver sql.Driver diff --git a/statediff/indexer/database/file/batch_tx.go b/statediff/indexer/database/file/batch_tx.go new file mode 100644 index 000000000..39e5d3713 --- /dev/null +++ b/statediff/indexer/database/file/batch_tx.go @@ -0,0 +1,29 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package file + +// BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration +type BatchTx struct { + BlockNumber uint64 + + submit func(blockTx *BatchTx, err error) error +} + +// Submit satisfies indexer.AtomicTx +func (tx *BatchTx) Submit(err error) error { + return tx.submit(tx, err) +} diff --git a/statediff/indexer/database/file/config.go b/statediff/indexer/database/file/config.go new file mode 100644 index 000000000..312ad7009 --- /dev/null +++ b/statediff/indexer/database/file/config.go @@ -0,0 +1,32 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package file + +import ( + "github.com/ethereum/go-ethereum/statediff/indexer/shared" +) + +// Config holds params for writing sql statements out to a file +type Config struct { + NodeID int64 // this is the nodeID used as FK in public.blocks + FilePath string +} + +// Type satisfies interfaces.Config +func (c Config) Type() shared.DBType { + return shared.FILE +} diff --git a/statediff/indexer/database/file/helpers.go b/statediff/indexer/database/file/helpers.go new file mode 100644 index 000000000..dc635110c --- /dev/null +++ b/statediff/indexer/database/file/helpers.go @@ -0,0 +1,60 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package file + +import "bytes" + +// formatPostgresStringArray parses an array of strings into the proper Postgres string representation of that array +func formatPostgresStringArray(a []string) string { + if a == nil { + return "" + } + + if n := len(a); n > 0 { + // There will be at least two curly brackets, 2*N bytes of quotes, + // and N-1 bytes of delimiters. + b := make([]byte, 1, 1+3*n) + b[0] = '{' + + b = appendArrayQuotedBytes(b, []byte(a[0])) + for i := 1; i < n; i++ { + b = append(b, ',') + b = appendArrayQuotedBytes(b, []byte(a[i])) + } + + return string(append(b, '}')) + } + + return "{}" +} + +func appendArrayQuotedBytes(b, v []byte) []byte { + b = append(b, '"') + for { + i := bytes.IndexAny(v, `"\`) + if i < 0 { + b = append(b, v...) + break + } + if i > 0 { + b = append(b, v[:i]...) + } + b = append(b, '\\', v[i]) + v = v[i+1:] + } + return append(b, '"') +} diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go new file mode 100644 index 000000000..57c5c2e1d --- /dev/null +++ b/statediff/indexer/database/file/indexer.go @@ -0,0 +1,474 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package file + +import ( + "context" + "errors" + "fmt" + "math/big" + "os" + "sync" + "time" + + "github.com/ipfs/go-cid" + node "github.com/ipfs/go-ipld-format" + "github.com/multiformats/go-multihash" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" + sdtypes "github.com/ethereum/go-ethereum/statediff/types" +) + +const defaultFilePath = "./statediff.sql" + +var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} + +var ( + indexerMetrics = RegisterIndexerMetrics(metrics.DefaultRegistry) +) + +// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of a void +type StateDiffIndexer struct { + writer *SQLWriter + chainConfig *params.ChainConfig + nodeID int64 + wg *sync.WaitGroup +} + +// NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer +func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, config Config) (*StateDiffIndexer, error) { + filePath := config.FilePath + if filePath == "" { + filePath = defaultFilePath + } + if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) { + return nil, fmt.Errorf("cannot create file, file (%s) already exists", filePath) + } + file, err := os.Create(filePath) + if err != nil { + return nil, fmt.Errorf("unable to create file (%s), err: %v", filePath, err) + } + w := NewSQLWriter(file) + wg := new(sync.WaitGroup) + w.Loop() + return &StateDiffIndexer{ + writer: w, + chainConfig: chainConfig, + nodeID: config.NodeID, + wg: wg, + }, nil +} + +// ReportDBMetrics has nothing to report for dump +func (sdi *StateDiffIndexer) ReportDBMetrics(time.Duration, <-chan bool) {} + +// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts) +// Returns an initiated DB transaction which must be Closed via defer to commit or rollback +func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) { + start, t := time.Now(), time.Now() + blockHash := block.Hash() + blockHashStr := blockHash.String() + height := block.NumberU64() + traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHashStr) + transactions := block.Transactions() + // Derive any missing fields + if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, transactions); err != nil { + return nil, err + } + + // Generate the block iplds + headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, rctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := ipld2.FromBlockAndReceipts(block, receipts) + if err != nil { + return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) + } + + if len(txNodes) != len(rctNodes) || len(rctNodes) != len(rctLeafNodeCIDs) { + return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d), and receipt trie leaf nodes (%d) to be equal", len(txNodes), len(rctNodes), len(rctLeafNodeCIDs)) + } + if len(txTrieNodes) != len(rctTrieNodes) { + return nil, fmt.Errorf("expected number of tx trie (%d) and rct trie (%d) nodes to be equal", len(txTrieNodes), len(rctTrieNodes)) + } + + // Calculate reward + var reward *big.Int + // in PoA networks block reward is 0 + if sdi.chainConfig.Clique != nil { + reward = big.NewInt(0) + } else { + reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) + } + t = time.Now() + + blockTx := &BatchTx{ + BlockNumber: height, + submit: func(self *BatchTx, err error) error { + tDiff := time.Since(t) + indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) + t = time.Now() + if err := sdi.writer.flush(); err != nil { + traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) + log.Debug(traceMsg) + return err + } + tDiff = time.Since(t) + indexerMetrics.tPostgresCommit.Update(tDiff) + traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String()) + traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) + log.Debug(traceMsg) + return err + }, + } + tDiff := time.Since(t) + indexerMetrics.tFreePostgres.Update(tDiff) + traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String()) + t = time.Now() + + // write header, collect headerID + headerID := sdi.processHeader(block.Header(), headerNode, reward, totalDifficulty) + tDiff = time.Since(t) + indexerMetrics.tHeaderProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) + t = time.Now() + + // write uncles + sdi.processUncles(headerID, height, uncleNodes) + tDiff = time.Since(t) + indexerMetrics.tUncleProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String()) + t = time.Now() + + // write receipts and txs + err = sdi.processReceiptsAndTxs(processArgs{ + headerID: headerID, + blockNumber: block.Number(), + receipts: receipts, + txs: transactions, + rctNodes: rctNodes, + rctTrieNodes: rctTrieNodes, + txNodes: txNodes, + txTrieNodes: txTrieNodes, + logTrieNodes: logTrieNodes, + logLeafNodeCIDs: logLeafNodeCIDs, + rctLeafNodeCIDs: rctLeafNodeCIDs, + }) + if err != nil { + return nil, err + } + tDiff = time.Since(t) + indexerMetrics.tTxAndRecProcessing.Update(tDiff) + traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String()) + t = time.Now() + + return blockTx, err +} + +// processHeader write a header IPLD insert SQL stmt to a file +// it returns the headerID +func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node.Node, reward, td *big.Int) string { + sdi.writer.upsertIPLDNode(headerNode) + + var baseFee *int64 + if header.BaseFee != nil { + baseFee = new(int64) + *baseFee = header.BaseFee.Int64() + } + headerID := header.Hash().String() + sdi.writer.upsertHeaderCID(models.HeaderModel{ + NodeID: sdi.nodeID, + CID: headerNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), + ParentHash: header.ParentHash.String(), + BlockNumber: header.Number.String(), + BlockHash: headerID, + TotalDifficulty: td.String(), + Reward: reward.String(), + Bloom: header.Bloom.Bytes(), + StateRoot: header.Root.String(), + RctRoot: header.ReceiptHash.String(), + TxRoot: header.TxHash.String(), + UncleRoot: header.UncleHash.String(), + Timestamp: header.Time, + BaseFee: baseFee, + }) + return headerID +} + +// processUncles writes uncle IPLD insert SQL stmts to a file +func (sdi *StateDiffIndexer) processUncles(headerID string, blockNumber uint64, uncleNodes []*ipld2.EthHeader) { + // publish and index uncles + for _, uncleNode := range uncleNodes { + sdi.writer.upsertIPLDNode(uncleNode) + var uncleReward *big.Int + // in PoA networks uncle reward is 0 + if sdi.chainConfig.Clique != nil { + uncleReward = big.NewInt(0) + } else { + uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) + } + sdi.writer.upsertUncleCID(models.UncleModel{ + HeaderID: headerID, + CID: uncleNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(uncleNode.Cid()), + ParentHash: uncleNode.ParentHash.String(), + BlockHash: uncleNode.Hash().String(), + Reward: uncleReward.String(), + }) + } +} + +// processArgs bundles arguments to processReceiptsAndTxs +type processArgs struct { + headerID string + blockNumber *big.Int + receipts types.Receipts + txs types.Transactions + rctNodes []*ipld2.EthReceipt + rctTrieNodes []*ipld2.EthRctTrie + txNodes []*ipld2.EthTx + txTrieNodes []*ipld2.EthTxTrie + logTrieNodes [][]*ipld2.EthLogTrie + logLeafNodeCIDs [][]cid.Cid + rctLeafNodeCIDs []cid.Cid +} + +// processReceiptsAndTxs writes receipt and tx IPLD insert SQL stmts to a file +func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { + // Process receipts and txs + signer := types.MakeSigner(sdi.chainConfig, args.blockNumber) + for i, receipt := range args.receipts { + for _, logTrieNode := range args.logTrieNodes[i] { + sdi.writer.upsertIPLDNode(logTrieNode) + } + txNode := args.txNodes[i] + sdi.writer.upsertIPLDNode(txNode) + + // index tx + trx := args.txs[i] + txID := trx.Hash().String() + // derive sender for the tx that corresponds with this receipt + from, err := types.Sender(signer, trx) + if err != nil { + return fmt.Errorf("error deriving tx sender: %v", err) + } + txModel := models.TxModel{ + HeaderID: args.headerID, + Dst: shared.HandleZeroAddrPointer(trx.To()), + Src: shared.HandleZeroAddr(from), + TxHash: txID, + Index: int64(i), + Data: trx.Data(), + CID: txNode.Cid().String(), + MhKey: shared.MultihashKeyFromCID(txNode.Cid()), + Type: trx.Type(), + } + sdi.writer.upsertTransactionCID(txModel) + + // index access list if this is one + for j, accessListElement := range trx.AccessList() { + storageKeys := make([]string, len(accessListElement.StorageKeys)) + for k, storageKey := range accessListElement.StorageKeys { + storageKeys[k] = storageKey.Hex() + } + accessListElementModel := models.AccessListElementModel{ + TxID: txID, + Index: int64(j), + Address: accessListElement.Address.Hex(), + StorageKeys: storageKeys, + } + sdi.writer.upsertAccessListElement(accessListElementModel) + } + + // this is the contract address if this receipt is for a contract creation tx + contract := shared.HandleZeroAddr(receipt.ContractAddress) + var contractHash string + if contract != "" { + contractHash = crypto.Keccak256Hash(common.HexToAddress(contract).Bytes()).String() + } + + // index receipt + if !args.rctLeafNodeCIDs[i].Defined() { + return fmt.Errorf("invalid receipt leaf node cid") + } + + rctModel := &models.ReceiptModel{ + TxID: txID, + Contract: contract, + ContractHash: contractHash, + LeafCID: args.rctLeafNodeCIDs[i].String(), + LeafMhKey: shared.MultihashKeyFromCID(args.rctLeafNodeCIDs[i]), + LogRoot: args.rctNodes[i].LogRoot.String(), + } + if len(receipt.PostState) == 0 { + rctModel.PostStatus = receipt.Status + } else { + rctModel.PostState = common.Bytes2Hex(receipt.PostState) + } + sdi.writer.upsertReceiptCID(rctModel) + + // index logs + 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") + } + + logDataSet[idx] = &models.LogsModel{ + ReceiptID: txID, + 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], + } + } + sdi.writer.upsertLogCID(logDataSet) + } + + // publish trie nodes, these aren't indexed directly + for i, n := range args.txTrieNodes { + sdi.writer.upsertIPLDNode(n) + sdi.writer.upsertIPLDNode(args.rctTrieNodes[i]) + } + + return nil +} + +// PushStateNode writes a state diff node object (including any child storage nodes) IPLD insert SQL stmt to a file +func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateNode, headerID string) error { + // publish the state node + if stateNode.NodeType == sdtypes.Removed { + // short circuit if it is a Removed node + // this assumes the db has been initialized and a public.blocks entry for the Removed node is present + stateModel := models.StateNodeModel{ + HeaderID: headerID, + Path: stateNode.Path, + StateKey: common.BytesToHash(stateNode.LeafKey).String(), + CID: shared.RemovedNodeStateCID, + MhKey: shared.RemovedNodeMhKey, + NodeType: stateNode.NodeType.Int(), + } + sdi.writer.upsertStateCID(stateModel) + return nil + } + stateCIDStr, stateMhKey, err := sdi.writer.upsertIPLDRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) + if err != nil { + return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) + } + stateModel := models.StateNodeModel{ + HeaderID: headerID, + Path: stateNode.Path, + StateKey: common.BytesToHash(stateNode.LeafKey).String(), + CID: stateCIDStr, + MhKey: stateMhKey, + NodeType: stateNode.NodeType.Int(), + } + // index the state node + sdi.writer.upsertStateCID(stateModel) + // if we have a leaf, decode and index the account data + if stateNode.NodeType == sdtypes.Leaf { + var i []interface{} + if err := rlp.DecodeBytes(stateNode.NodeValue, &i); err != nil { + return fmt.Errorf("error decoding state leaf node rlp: %s", err.Error()) + } + if len(i) != 2 { + return fmt.Errorf("eth IPLDPublisher expected state leaf node rlp to decode into two elements") + } + var account types.StateAccount + if err := rlp.DecodeBytes(i[1].([]byte), &account); err != nil { + return fmt.Errorf("error decoding state account rlp: %s", err.Error()) + } + accountModel := models.StateAccountModel{ + HeaderID: headerID, + StatePath: stateNode.Path, + Balance: account.Balance.String(), + Nonce: account.Nonce, + CodeHash: account.CodeHash, + StorageRoot: account.Root.String(), + } + sdi.writer.upsertStateAccount(accountModel) + } + // if there are any storage nodes associated with this node, publish and index them + for _, storageNode := range stateNode.StorageNodes { + if storageNode.NodeType == sdtypes.Removed { + // short circuit if it is a Removed node + // this assumes the db has been initialized and a public.blocks entry for the Removed node is present + storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, + Path: storageNode.Path, + StorageKey: common.BytesToHash(storageNode.LeafKey).String(), + CID: shared.RemovedNodeStorageCID, + MhKey: shared.RemovedNodeMhKey, + NodeType: storageNode.NodeType.Int(), + } + sdi.writer.upsertStorageCID(storageModel) + continue + } + storageCIDStr, storageMhKey, err := sdi.writer.upsertIPLDRaw(ipld2.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) + if err != nil { + return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) + } + storageModel := models.StorageNodeModel{ + HeaderID: headerID, + StatePath: stateNode.Path, + Path: storageNode.Path, + StorageKey: common.BytesToHash(storageNode.LeafKey).String(), + CID: storageCIDStr, + MhKey: storageMhKey, + NodeType: storageNode.NodeType.Int(), + } + sdi.writer.upsertStorageCID(storageModel) + } + + return nil +} + +// PushCodeAndCodeHash writes code and codehash pairs insert SQL stmts to a file +func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAndCodeHash sdtypes.CodeAndCodeHash) error { + // codec doesn't matter since db key is multihash-based + mhKey, err := shared.MultihashKeyFromKeccak256(codeAndCodeHash.Hash) + if err != nil { + return fmt.Errorf("error deriving multihash key from codehash: %v", err) + } + sdi.writer.upsertIPLDDirect(mhKey, codeAndCodeHash.Code) + return nil +} + +// Close satisfies io.Closer +func (sdi *StateDiffIndexer) Close() error { + return sdi.writer.Close() +} diff --git a/statediff/indexer/database/file/metrics.go b/statediff/indexer/database/file/metrics.go new file mode 100644 index 000000000..ca6e88f2b --- /dev/null +++ b/statediff/indexer/database/file/metrics.go @@ -0,0 +1,94 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package file + +import ( + "strings" + + "github.com/ethereum/go-ethereum/metrics" +) + +const ( + namespace = "statediff" +) + +// Build a fully qualified metric name +func metricName(subsystem, name string) string { + if name == "" { + return "" + } + parts := []string{namespace, name} + if subsystem != "" { + parts = []string{namespace, subsystem, name} + } + // Prometheus uses _ but geth metrics uses / and replaces + return strings.Join(parts, "/") +} + +type indexerMetricsHandles struct { + // The total number of processed blocks + blocks metrics.Counter + // The total number of processed transactions + transactions metrics.Counter + // The total number of processed receipts + receipts metrics.Counter + // The total number of processed logs + logs metrics.Counter + // The total number of access list entries processed + accessListEntries metrics.Counter + // Time spent waiting for free postgres tx + tFreePostgres metrics.Timer + // Postgres transaction commit duration + tPostgresCommit metrics.Timer + // Header processing time + tHeaderProcessing metrics.Timer + // Uncle processing time + tUncleProcessing metrics.Timer + // Tx and receipt processing time + tTxAndRecProcessing metrics.Timer + // State, storage, and code combined processing time + tStateStoreCodeProcessing metrics.Timer +} + +func RegisterIndexerMetrics(reg metrics.Registry) indexerMetricsHandles { + ctx := indexerMetricsHandles{ + blocks: metrics.NewCounter(), + transactions: metrics.NewCounter(), + receipts: metrics.NewCounter(), + logs: metrics.NewCounter(), + accessListEntries: metrics.NewCounter(), + tFreePostgres: metrics.NewTimer(), + tPostgresCommit: metrics.NewTimer(), + tHeaderProcessing: metrics.NewTimer(), + tUncleProcessing: metrics.NewTimer(), + tTxAndRecProcessing: metrics.NewTimer(), + tStateStoreCodeProcessing: metrics.NewTimer(), + } + subsys := "indexer" + reg.Register(metricName(subsys, "blocks"), ctx.blocks) + reg.Register(metricName(subsys, "transactions"), ctx.transactions) + reg.Register(metricName(subsys, "receipts"), ctx.receipts) + reg.Register(metricName(subsys, "logs"), ctx.logs) + reg.Register(metricName(subsys, "access_list_entries"), ctx.accessListEntries) + reg.Register(metricName(subsys, "t_free_postgres"), ctx.tFreePostgres) + reg.Register(metricName(subsys, "t_postgres_commit"), ctx.tPostgresCommit) + reg.Register(metricName(subsys, "t_header_processing"), ctx.tHeaderProcessing) + reg.Register(metricName(subsys, "t_uncle_processing"), ctx.tUncleProcessing) + reg.Register(metricName(subsys, "t_tx_receipt_processing"), ctx.tTxAndRecProcessing) + reg.Register(metricName(subsys, "t_state_store_code_processing"), ctx.tStateStoreCodeProcessing) + return ctx +} diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go new file mode 100644 index 000000000..2a836c8b3 --- /dev/null +++ b/statediff/indexer/database/file/writer.go @@ -0,0 +1,240 @@ +// 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 . + +package file + +import ( + "fmt" + "os" + + blockstore "github.com/ipfs/go-ipfs-blockstore" + dshelp "github.com/ipfs/go-ipfs-ds-help" + node "github.com/ipfs/go-ipld-format" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/models" +) + +var ( + nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") + collatedStmtSize = 65336 // min(linuxPipeSize, macOSPipeSize) +) + +// SQLWriter writes sql statements to a file +type SQLWriter struct { + file *os.File + stmts chan []byte + collatedStmt []byte + collationIndex int + + quitChan chan struct{} + doneChan chan struct{} +} + +// NewSQLWriter creates a new pointer to a Writer +func NewSQLWriter(file *os.File) *SQLWriter { + return &SQLWriter{ + file: file, + stmts: make(chan []byte), + collatedStmt: make([]byte, collatedStmtSize), + quitChan: make(chan struct{}), + doneChan: make(chan struct{}), + } +} + +// Loop enables concurrent writes to the underlying os.File +// since os.File does not buffer, it utilizes an internal buffer that is the size of a unix pipe +// by using copy() and tracking the index/size of the buffer, we require only the initial memory allocation +func (sqw *SQLWriter) Loop() { + sqw.collationIndex = 0 + go func() { + defer func() { + close(sqw.doneChan) + }() + var l int + for { + select { + case stmt := <-sqw.stmts: + l = len(stmt) + if l+sqw.collationIndex+1 > collatedStmtSize { + if err := sqw.flush(); err != nil { + log.Error("error writing cached sql stmts to file", "err", err) + } + } + copy(sqw.collatedStmt[sqw.collationIndex:sqw.collationIndex+l-1], stmt) + sqw.collationIndex += l + case <-sqw.quitChan: + if err := sqw.flush(); err != nil { + log.Error("error writing cached sql stmts to file", "err", err) + } + return + } + } + }() +} + +// Close satisfies io.Closer +func (sqw *SQLWriter) Close() error { + close(sqw.quitChan) + <-sqw.doneChan + return nil +} + +func (sqw *SQLWriter) flush() error { + if _, err := sqw.file.Write(sqw.collatedStmt[0 : sqw.collationIndex-1]); err != nil { + return err + } + sqw.collationIndex = 0 + return nil +} + +const ( + ipldInsert = `INSERT INTO public.blocks (key, data) VALUES (%s, %x) ON CONFLICT (key) DO NOTHING;\n` + + headerInsert = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) +VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, %d, %d) +ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, %d);\n` + + headerInsertWithoutBaseFee = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) +VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, %d, NULL) +ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, NULL);\n` + + uncleInsert = `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES (%s, %s, %s, %s, %s, %s) +ON CONFLICT (block_hash) DO NOTHING;\n` + + txInsert = `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %d) +ON CONFLICT (tx_hash) DO NOTHING;\n` + + alInsert = `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES (%s, %d, %s, %s) +ON CONFLICT (tx_id, index) DO NOTHING;\n` + + rctInsert = `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES (%s, %s, %s, %s, %s, %s, %d, %s) +ON CONFLICT (tx_id) DO NOTHING;\n` + + logInsert = `INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES (%s, %s, %s, %s, %d, %s, %s, %s, %s, %s) +ON CONFLICT (rct_id, index) DO NOTHING;\n` + + stateInsert = `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES (%s, %s, %s, %s, %d, %t, %s) +ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = (%s, %s, %d, %t, %s);\n` + + accountInsert = `INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) VALUES (%s, %s, %s, %d, %s, %s) +ON CONFLICT (header_id, state_path) DO NOTHING;\n` + + storageInsert = `INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES (%s, %s, %s, %s, %s, %d, %t, %s) +ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = (%s, %s, %d, %t, %s);\n` +) + +func (sqw *SQLWriter) upsertIPLD(ipld models.IPLDModel) { + sqw.stmts <- []byte(fmt.Sprintf(ipldInsert, ipld.Key, ipld.Data)) +} + +func (sqw *SQLWriter) upsertIPLDDirect(key string, value []byte) { + sqw.upsertIPLD(models.IPLDModel{ + Key: key, + Data: value, + }) +} + +func (sqw *SQLWriter) upsertIPLDNode(i node.Node) { + sqw.upsertIPLD(models.IPLDModel{ + Key: blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(i.Cid().Hash()).String(), + Data: i.RawData(), + }) +} + +func (sqw *SQLWriter) upsertIPLDRaw(codec, mh uint64, raw []byte) (string, string, error) { + c, err := ipld.RawdataToCid(codec, raw, mh) + if err != nil { + return "", "", err + } + prefixedKey := blockstore.BlockPrefix.String() + dshelp.MultihashToDsKey(c.Hash()).String() + sqw.upsertIPLD(models.IPLDModel{ + Key: prefixedKey, + Data: raw, + }) + return c.String(), prefixedKey, err +} + +func (sqw *SQLWriter) upsertHeaderCID(header models.HeaderModel) { + var stmt string + if header.BaseFee == nil { + stmt = fmt.Sprintf(headerInsertWithoutBaseFee, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, + header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, + header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, + header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey) + } else { + stmt = fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, + header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee, + header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, + header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, header.BaseFee) + } + sqw.stmts <- []byte(stmt) + indexerMetrics.blocks.Inc(1) +} + +func (sqw *SQLWriter) upsertUncleCID(uncle models.UncleModel) { + sqw.stmts <- []byte(fmt.Sprintf(uncleInsert, uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey)) +} + +func (sqw *SQLWriter) upsertTransactionCID(transaction models.TxModel) { + sqw.stmts <- []byte(fmt.Sprintf(txInsert, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type)) + indexerMetrics.transactions.Inc(1) +} + +func (sqw *SQLWriter) upsertAccessListElement(accessListElement models.AccessListElementModel) { + sqw.stmts <- []byte(fmt.Sprintf(alInsert, accessListElement.TxID, accessListElement.Index, accessListElement.Address, formatPostgresStringArray(accessListElement.StorageKeys))) + indexerMetrics.accessListEntries.Inc(1) +} + +func (sqw *SQLWriter) upsertReceiptCID(rct *models.ReceiptModel) { + sqw.stmts <- []byte(fmt.Sprintf(rctInsert, rct.TxID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot)) + indexerMetrics.receipts.Inc(1) +} + +func (sqw *SQLWriter) upsertLogCID(logs []*models.LogsModel) { + for _, l := range logs { + sqw.stmts <- []byte(fmt.Sprintf(logInsert, l.LeafCID, l.LeafMhKey, l.ReceiptID, l.Address, l.Index, l.Topic0, l.Topic1, l.Topic2, l.Topic3, l.Data)) + indexerMetrics.logs.Inc(1) + } +} + +func (sqw *SQLWriter) upsertStateCID(stateNode models.StateNodeModel) { + var stateKey string + if stateNode.StateKey != nullHash.String() { + stateKey = stateNode.StateKey + } + sqw.stmts <- []byte(fmt.Sprintf(stateInsert, stateNode.HeaderID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, + true, stateNode.MhKey, stateKey, stateNode.CID, stateNode.NodeType, true, stateNode.MhKey)) +} + +func (sqw *SQLWriter) upsertStateAccount(stateAccount models.StateAccountModel) { + sqw.stmts <- []byte(fmt.Sprintf(accountInsert, stateAccount.HeaderID, stateAccount.StatePath, stateAccount.Balance, + stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot)) +} + +func (sqw *SQLWriter) upsertStorageCID(storageCID models.StorageNodeModel) { + var storageKey string + if storageCID.StorageKey != nullHash.String() { + storageKey = storageCID.StorageKey + } + sqw.stmts <- []byte(fmt.Sprintf(storageInsert, storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, + storageCID.Path, storageCID.NodeType, true, storageCID.MhKey, storageKey, storageCID.CID, storageCID.NodeType, + true, storageCID.MhKey)) +} diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index 1e89f92ff..b557ec903 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -539,7 +539,7 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAnd return nil } -// Close satisfied io.Closer +// Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { return sdi.dbWriter.db.Close() } diff --git a/statediff/indexer/shared/db_kind.go b/statediff/indexer/shared/db_kind.go index 6b88164e1..7e7997f95 100644 --- a/statediff/indexer/shared/db_kind.go +++ b/statediff/indexer/shared/db_kind.go @@ -27,6 +27,7 @@ type DBType string const ( POSTGRES DBType = "Postgres" DUMP DBType = "Dump" + FILE DBType = "File" UNKNOWN DBType = "Unknown" ) @@ -37,6 +38,8 @@ func ResolveDBType(str string) (DBType, error) { return POSTGRES, nil case "dump", "d": return DUMP, nil + case "file", "f", "fs": + return FILE, nil default: return UNKNOWN, fmt.Errorf("unrecognized db type string: %s", str) } -- 2.45.2 From aa984711bd95f404debbc739385d01f58c33def0 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 17 Nov 2021 12:22:40 -0600 Subject: [PATCH 09/33] cli integration --- cmd/geth/config.go | 9 +++++++++ cmd/geth/main.go | 2 ++ cmd/geth/usage.go | 2 ++ cmd/utils/flags.go | 10 +++++++++- statediff/indexer/constructor.go | 3 +-- statediff/indexer/database/file/config.go | 2 +- statediff/indexer/database/file/writer.go | 4 +--- .../indexer/database/sql/pgx_indexer_legacy_test.go | 2 +- statediff/indexer/database/sql/sqlx_indexer_test.go | 2 +- statediff/indexer/models/models.go | 4 ++-- 10 files changed, 29 insertions(+), 11 deletions(-) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index 9a8b169be..bb93509e0 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -45,6 +45,7 @@ import ( "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/statediff" dumpdb "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/shared" @@ -204,6 +205,14 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) { utils.Fatalf("%v", err) } switch dbType { + case shared.FILE: + if !ctx.GlobalIsSet(utils.StateDiffFileNodeRowIDFlag.Name) { + utils.Fatalf("In statediff file writing mode a node row ID must be provided") + } + indexerConfig = file.Config{ + NodeID: int64(ctx.GlobalInt(utils.StateDiffFileNodeRowIDFlag.Name)), + FilePath: ctx.GlobalString(utils.StateDiffFilePath.Name), + } case shared.POSTGRES: driverTypeStr := ctx.GlobalString(utils.StateDiffDBDriverTypeFlag.Name) driverType, err := postgres.ResolveDriverType(driverTypeStr) diff --git a/cmd/geth/main.go b/cmd/geth/main.go index c92810d11..e027eddf9 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -167,6 +167,8 @@ var ( utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, utils.StateDiffWorkersFlag, + utils.StateDiffFileNodeRowIDFlag, + utils.StateDiffFilePath, configFileFlag, utils.CatalystFlag, } diff --git a/cmd/geth/usage.go b/cmd/geth/usage.go index 68e2a3f4c..7f768c11a 100644 --- a/cmd/geth/usage.go +++ b/cmd/geth/usage.go @@ -243,6 +243,8 @@ var AppHelpFlagGroups = []flags.FlagGroup{ utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, utils.StateDiffWorkersFlag, + utils.StateDiffFileNodeRowIDFlag, + utils.StateDiffFilePath, }, }, { diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 08f9088f5..26f904aaa 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -788,7 +788,7 @@ var ( } StateDiffDBTypeFlag = cli.StringFlag{ Name: "statediff.db.type", - Usage: "Statediff database type", + Usage: "Statediff database type (current options: postgres, file, dump)", Value: "postgres", } StateDiffDBDriverTypeFlag = cli.StringFlag{ @@ -852,6 +852,14 @@ var ( Name: "statediff.db.nodeid", Usage: "Node ID to use when writing state diffs to database", } + StateDiffFileNodeRowIDFlag = cli.IntFlag{ + Name: "statediff.file.nodeid", + Usage: "Node row ID to use as FK when writing state diffs to database", + } + StateDiffFilePath = cli.StringFlag{ + Name: "statediff.file.path", + Usage: "Full path (including filename) to write statediff data out to when operating in file mode", + } StateDiffDBClientNameFlag = cli.StringFlag{ Name: "statediff.db.clientname", Usage: "Client name to use when writing state diffs to database", diff --git a/statediff/indexer/constructor.go b/statediff/indexer/constructor.go index a4cfa271d..60e6dadb7 100644 --- a/statediff/indexer/constructor.go +++ b/statediff/indexer/constructor.go @@ -20,10 +20,9 @@ import ( "context" "fmt" - "github.com/ethereum/go-ethereum/statediff/indexer/database/file" - "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" diff --git a/statediff/indexer/database/file/config.go b/statediff/indexer/database/file/config.go index 312ad7009..fd9e488c9 100644 --- a/statediff/indexer/database/file/config.go +++ b/statediff/indexer/database/file/config.go @@ -22,7 +22,7 @@ import ( // Config holds params for writing sql statements out to a file type Config struct { - NodeID int64 // this is the nodeID used as FK in public.blocks + NodeID int64 // this is the nodeID used as FK in eth.header_cids FilePath string } diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 2a836c8b3..d95fba328 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -63,9 +63,7 @@ func NewSQLWriter(file *os.File) *SQLWriter { func (sqw *SQLWriter) Loop() { sqw.collationIndex = 0 go func() { - defer func() { - close(sqw.doneChan) - }() + defer close(sqw.doneChan) var l int for { select { diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go index d08336e63..0373ff5ee 100644 --- a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -53,7 +53,7 @@ func setupLegacyPGX(t *testing.T) { } }() for _, node := range legacyData.StateDiffs { - err = ind.PushStateNode(tx, node) + err = ind.PushStateNode(tx, node, legacyData.MockBlock.Hash().String()) require.NoError(t, err) } diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 0fa4e8c1a..1861d65d4 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -159,7 +159,7 @@ func setupSQLX(t *testing.T) { } }() for _, node := range mocks.StateDiffs { - err = ind.PushStateNode(tx, node) + err = ind.PushStateNode(tx, node, mockBlock.Hash().String()) if err != nil { t.Fatal(err) } diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index 7d2fc43b6..60d83d96e 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -100,7 +100,7 @@ type StateNodeModel struct { // StorageNodeModel is the db model for eth.storage_cids type StorageNodeModel struct { - HeaderID string `db:"header_id""` + HeaderID string `db:"header_id"` StatePath []byte `db:"state_path"` Path []byte `db:"storage_path"` StorageKey string `db:"storage_leaf_key"` @@ -112,7 +112,7 @@ type StorageNodeModel struct { // StorageNodeWithStateKeyModel is a db model for eth.storage_cids + eth.state_cids.state_key type StorageNodeWithStateKeyModel struct { - HeaderID string `db:"header_id""` + HeaderID string `db:"header_id"` StatePath []byte `db:"state_path"` Path []byte `db:"storage_path"` StateKey string `db:"state_leaf_key"` -- 2.45.2 From 4ee8214562fbe4f925d7e997bae63b4029049878 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 17 Nov 2021 19:37:41 -0600 Subject: [PATCH 10/33] fix unit tests --- .../database/sql/pgx_indexer_legacy_test.go | 16 ++--- .../indexer/database/sql/pgx_indexer_test.go | 60 +++++++++---------- .../database/sql/sqlx_indexer_legacy_test.go | 14 ++--- .../indexer/database/sql/sqlx_indexer_test.go | 58 +++++++++--------- statediff/indexer/database/sql/writer.go | 2 +- statediff/service.go | 9 +++ 6 files changed, 84 insertions(+), 75 deletions(-) diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go index 0373ff5ee..21b74b3b2 100644 --- a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -20,7 +20,6 @@ import ( "context" "testing" - "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -64,20 +63,21 @@ func TestPGXIndexerLegacy(t *testing.T) { t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { setupLegacyPGX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, id, base_fee + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed type res struct { - CID string - TD string - Reward string - ID int - BaseFee *int64 `db:"base_fee"` + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).Scan( + &header.CID, &header.TD, &header.Reward, &header.BlockHash, &header.BaseFee) require.NoError(t, err) test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index f63efe712..26cc660ec 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -140,7 +140,7 @@ func setupPGX(t *testing.T) { } }() for _, node := range mocks.StateDiffs { - err = ind.PushStateNode(tx, node) + err = ind.PushStateNode(tx, node, mockBlock.Hash().String()) if err != nil { t.Fatal(err) } @@ -153,16 +153,16 @@ func TestPGXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupPGX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, id, base_fee + pgStr := `SELECT cid, td, reward, block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed type res struct { - CID string - TD string - Reward string - ID int - BaseFee *int64 `db:"base_fee"` + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) @@ -192,7 +192,7 @@ func TestPGXIndexer(t *testing.T) { defer tearDown(t) // check that txs were properly indexed trxs := make([]string, 0) - pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.id) + pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1` err = db.Select(context.Background(), &trxs, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -260,7 +260,7 @@ func TestPGXIndexer(t *testing.T) { t.Fatalf("expected AccessListTxType (1), got %d", *txType) } accessListElementModels := make([]models.AccessListElementModel, 0) - pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.id) WHERE cid = $1 ORDER BY access_list_element.index ASC` + pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_element.index ASC` err = db.Select(context.Background(), &accessListElementModels, pgStr, c) if err != nil { t.Fatal(err) @@ -299,8 +299,8 @@ func TestPGXIndexer(t *testing.T) { rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids - WHERE receipt_cids.tx_id = transaction_cids.id - AND transaction_cids.header_id = header_cids.id + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 ORDER BY transaction_cids.index` err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) @@ -317,8 +317,8 @@ func TestPGXIndexer(t *testing.T) { } for i := range rcts { 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) + 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.rct_id = receipt_cids.tx_id) INNER JOIN public.blocks ON (log_cids.leaf_mh_key = blocks.key) WHERE receipt_cids.leaf_cid = $1 ORDER BY eth.log_cids.index ASC` err = db.Select(context.Background(), &results, pgStr, rcts[i]) @@ -350,9 +350,9 @@ func TestPGXIndexer(t *testing.T) { // check receipts were properly indexed rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids - WHERE receipt_cids.tx_id = transaction_cids.id - AND transaction_cids.header_id = header_cids.id - AND header_cids.block_number = $1 order by transaction_cids.id` + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 order by transaction_cids.index` err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) @@ -447,8 +447,8 @@ func TestPGXIndexer(t *testing.T) { defer tearDown(t) // check that state nodes were properly indexed and published stateNodes := make([]models.StateNodeModel, 0) - pgStr := `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id - FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + pgStr := `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1 AND node_type != 3` err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -467,9 +467,9 @@ func TestPGXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - pgStr = `SELECT * from eth.state_accounts WHERE state_id = $1` + pgStr = `SELECT * from eth.state_accounts WHERE header_id = $1 AND state_path = $2` var account models.StateAccountModel - err = db.Get(context.Background(), &account, pgStr, stateNode.ID) + err = db.Get(context.Background(), &account, pgStr, stateNode.HeaderID, stateNode.Path) if err != nil { t.Fatal(err) } @@ -479,8 +479,8 @@ func TestPGXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) test_helpers.ExpectEqual(t, data, mocks.ContractLeafNode) test_helpers.ExpectEqual(t, account, models.StateAccountModel{ - ID: account.ID, - StateID: stateNode.ID, + HeaderID: account.HeaderID, + StatePath: stateNode.Path, Balance: "0", CodeHash: mocks.ContractCodeHash.Bytes(), StorageRoot: mocks.ContractRoot, @@ -493,8 +493,8 @@ func TestPGXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) test_helpers.ExpectEqual(t, data, mocks.AccountLeafNode) test_helpers.ExpectEqual(t, account, models.StateAccountModel{ - ID: account.ID, - StateID: stateNode.ID, + HeaderID: account.HeaderID, + StatePath: stateNode.Path, Balance: "1000", CodeHash: mocks.AccountCodeHash.Bytes(), StorageRoot: mocks.AccountRoot, @@ -505,8 +505,8 @@ func TestPGXIndexer(t *testing.T) { // check that Removed state nodes were properly indexed and published stateNodes = make([]models.StateNodeModel, 0) - pgStr = `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id - FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + pgStr = `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1 AND node_type = 3` err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -538,8 +538,8 @@ func TestPGXIndexer(t *testing.T) { storageNodes := make([]models.StorageNodeWithStateKeyModel, 0) pgStr := `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path FROM eth.storage_cids, eth.state_cids, eth.header_cids - WHERE storage_cids.state_id = state_cids.id - AND state_cids.header_id = header_cids.id + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 AND storage_cids.node_type != 3` err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) @@ -571,8 +571,8 @@ func TestPGXIndexer(t *testing.T) { storageNodes = make([]models.StorageNodeWithStateKeyModel, 0) pgStr = `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path FROM eth.storage_cids, eth.state_cids, eth.header_cids - WHERE storage_cids.state_id = state_cids.id - AND state_cids.header_id = header_cids.id + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 AND storage_cids.node_type = 3` err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) diff --git a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go index 2ce5f494f..4349850ed 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go @@ -62,7 +62,7 @@ func setupLegacySQLX(t *testing.T) { } }() for _, node := range legacyData.StateDiffs { - err = ind.PushStateNode(tx, node) + err = ind.PushStateNode(tx, node, mockLegacyBlock.Hash().String()) require.NoError(t, err) } @@ -73,16 +73,16 @@ func TestSQLXIndexerLegacy(t *testing.T) { t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { setupLegacySQLX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, id, base_fee + pgStr := `SELECT cid, td, reward, block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed type res struct { - CID string - TD string - Reward string - ID int - BaseFee *int64 `db:"base_fee"` + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 1861d65d4..09ee62fa3 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -179,16 +179,16 @@ func TestSQLXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupSQLX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, id, base_fee + pgStr := `SELECT cid, td, reward, block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed type res struct { - CID string - TD string - Reward string - ID int - BaseFee *int64 `db:"base_fee"` + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) @@ -218,7 +218,7 @@ func TestSQLXIndexer(t *testing.T) { defer tearDown(t) // check that txs were properly indexed trxs := make([]string, 0) - pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.id) + pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1` err = db.Select(context.Background(), &trxs, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -286,7 +286,7 @@ func TestSQLXIndexer(t *testing.T) { t.Fatalf("expected AccessListTxType (1), got %d", txType) } accessListElementModels := make([]models.AccessListElementModel, 0) - pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.id) WHERE cid = $1 ORDER BY access_list_element.index ASC` + pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_element.index ASC` err = db.Select(context.Background(), &accessListElementModels, pgStr, c) if err != nil { t.Fatal(err) @@ -325,8 +325,8 @@ func TestSQLXIndexer(t *testing.T) { rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids - WHERE receipt_cids.tx_id = transaction_cids.id - AND transaction_cids.header_id = header_cids.id + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 ORDER BY transaction_cids.index` err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) @@ -343,8 +343,8 @@ func TestSQLXIndexer(t *testing.T) { } for i := range rcts { 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) + 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.rct_id = receipt_cids.tx_id) INNER JOIN public.blocks ON (log_cids.leaf_mh_key = blocks.key) WHERE receipt_cids.leaf_cid = $1 ORDER BY eth.log_cids.index ASC` err = db.Select(context.Background(), &results, pgStr, rcts[i]) @@ -376,9 +376,9 @@ func TestSQLXIndexer(t *testing.T) { // check receipts were properly indexed rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids - WHERE receipt_cids.tx_id = transaction_cids.id - AND transaction_cids.header_id = header_cids.id - AND header_cids.block_number = $1 order by transaction_cids.id` + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 order by transaction_cids.index` err = db.Select(context.Background(), &rcts, pgStr, mocks.BlockNumber.Uint64()) if err != nil { t.Fatal(err) @@ -472,8 +472,8 @@ func TestSQLXIndexer(t *testing.T) { defer tearDown(t) // check that state nodes were properly indexed and published stateNodes := make([]models.StateNodeModel, 0) - pgStr := `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id - FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + pgStr := `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1 AND node_type != 3` err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -492,9 +492,9 @@ func TestSQLXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - pgStr = `SELECT * from eth.state_accounts WHERE state_id = $1` + pgStr = `SELECT * from eth.state_accounts WHERE header_id = $1 AND state_path = $2` var account models.StateAccountModel - err = db.Get(context.Background(), &account, pgStr, stateNode.ID) + err = db.Get(context.Background(), &account, pgStr, stateNode.HeaderID, stateNode.Path) if err != nil { t.Fatal(err) } @@ -504,8 +504,8 @@ func TestSQLXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) test_helpers.ExpectEqual(t, data, mocks.ContractLeafNode) test_helpers.ExpectEqual(t, account, models.StateAccountModel{ - ID: account.ID, - StateID: stateNode.ID, + HeaderID: account.HeaderID, + StatePath: stateNode.Path, Balance: "0", CodeHash: mocks.ContractCodeHash.Bytes(), StorageRoot: mocks.ContractRoot, @@ -518,8 +518,8 @@ func TestSQLXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) test_helpers.ExpectEqual(t, data, mocks.AccountLeafNode) test_helpers.ExpectEqual(t, account, models.StateAccountModel{ - ID: account.ID, - StateID: stateNode.ID, + HeaderID: account.HeaderID, + StatePath: stateNode.Path, Balance: "1000", CodeHash: mocks.AccountCodeHash.Bytes(), StorageRoot: mocks.AccountRoot, @@ -530,8 +530,8 @@ func TestSQLXIndexer(t *testing.T) { // check that Removed state nodes were properly indexed and published stateNodes = make([]models.StateNodeModel, 0) - pgStr = `SELECT state_cids.id, state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id - FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.id) + pgStr = `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1 AND node_type = 3` err = db.Select(context.Background(), &stateNodes, pgStr, mocks.BlockNumber.Uint64()) if err != nil { @@ -563,8 +563,8 @@ func TestSQLXIndexer(t *testing.T) { storageNodes := make([]models.StorageNodeWithStateKeyModel, 0) pgStr := `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path FROM eth.storage_cids, eth.state_cids, eth.header_cids - WHERE storage_cids.state_id = state_cids.id - AND state_cids.header_id = header_cids.id + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 AND storage_cids.node_type != 3` err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) @@ -596,8 +596,8 @@ func TestSQLXIndexer(t *testing.T) { storageNodes = make([]models.StorageNodeWithStateKeyModel, 0) pgStr = `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path FROM eth.storage_cids, eth.state_cids, eth.header_cids - WHERE storage_cids.state_id = state_cids.id - AND state_cids.header_id = header_cids.id + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash AND header_cids.block_number = $1 AND storage_cids.node_type = 3` err = db.Select(context.Background(), &storageNodes, pgStr, mocks.BlockNumber.Uint64()) diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go index 96d13d956..94b38c7e1 100644 --- a/statediff/indexer/database/sql/writer.go +++ b/statediff/indexer/database/sql/writer.go @@ -42,7 +42,7 @@ func NewWriter(db Database) *Writer { /* INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) -ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) +ON CONFLICT (block_hash) DO UPDATE SET (block_number, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($1, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) */ func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { _, err := tx.Exec(in.db.Context(), in.db.InsertHeaderStm(), diff --git a/statediff/service.go b/statediff/service.go index 3fc8ac60c..04aaac458 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -239,8 +239,14 @@ func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) { statediffMetrics.writeLoopChannelLen.Update(int64(len(chainEventCh))) chainEventFwd <- chainEvent case err := <-errCh: + println("here") log.Error("Error from chain event subscription", "error", err) close(sds.QuitChan) + log.Info("Quitting the statediffing writing loop") + if err := sds.indexer.Close(); err != nil { + log.Error("Error closing indexer", "err", err) + } + return case <-sds.QuitChan: log.Info("Quitting the statediffing writing loop") if err := sds.indexer.Close(); err != nil { @@ -339,6 +345,9 @@ func (sds *Service) Loop(chainEventCh chan core.ChainEvent) { case err := <-errCh: log.Error("Error from chain event subscription", "error", err) close(sds.QuitChan) + log.Info("Quitting the statediffing listening loop") + sds.close() + return case <-sds.QuitChan: log.Info("Quitting the statediffing listening loop") sds.close() -- 2.45.2 From bb788f71c676eacac0726ab71d5f0cdf09fae0c8 Mon Sep 17 00:00:00 2001 From: i-norden Date: Thu, 18 Nov 2021 08:14:57 -0600 Subject: [PATCH 11/33] use node_id as PK/FK --- cmd/geth/config.go | 4 -- cmd/geth/main.go | 1 - cmd/geth/usage.go | 1 - cmd/utils/flags.go | 4 -- statediff/README.md | 3 +- statediff/indexer/constructor.go | 1 + statediff/indexer/database/file/config.go | 3 +- statediff/indexer/database/file/indexer.go | 4 +- statediff/indexer/database/file/writer.go | 18 +++++-- statediff/indexer/database/sql/interfaces.go | 2 +- .../indexer/database/sql/pgx_indexer_test.go | 36 +++++++------- .../indexer/database/sql/postgres/config.go | 7 +-- .../indexer/database/sql/postgres/database.go | 11 +---- .../indexer/database/sql/postgres/pgx.go | 11 ++--- .../indexer/database/sql/postgres/sqlx.go | 11 ++--- statediff/indexer/models/models.go | 2 +- statediff/test_helpers/mocks/service_test.go | 47 +++++++++++-------- 17 files changed, 84 insertions(+), 82 deletions(-) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index bb93509e0..d77e261f9 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -206,11 +206,7 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) { } switch dbType { case shared.FILE: - if !ctx.GlobalIsSet(utils.StateDiffFileNodeRowIDFlag.Name) { - utils.Fatalf("In statediff file writing mode a node row ID must be provided") - } indexerConfig = file.Config{ - NodeID: int64(ctx.GlobalInt(utils.StateDiffFileNodeRowIDFlag.Name)), FilePath: ctx.GlobalString(utils.StateDiffFilePath.Name), } case shared.POSTGRES: diff --git a/cmd/geth/main.go b/cmd/geth/main.go index e027eddf9..990b40a60 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -167,7 +167,6 @@ var ( utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, utils.StateDiffWorkersFlag, - utils.StateDiffFileNodeRowIDFlag, utils.StateDiffFilePath, configFileFlag, utils.CatalystFlag, diff --git a/cmd/geth/usage.go b/cmd/geth/usage.go index 7f768c11a..885cc2c16 100644 --- a/cmd/geth/usage.go +++ b/cmd/geth/usage.go @@ -243,7 +243,6 @@ var AppHelpFlagGroups = []flags.FlagGroup{ utils.StateDiffDBClientNameFlag, utils.StateDiffWritingFlag, utils.StateDiffWorkersFlag, - utils.StateDiffFileNodeRowIDFlag, utils.StateDiffFilePath, }, }, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 26f904aaa..ccc9ac89e 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -852,10 +852,6 @@ var ( Name: "statediff.db.nodeid", Usage: "Node ID to use when writing state diffs to database", } - StateDiffFileNodeRowIDFlag = cli.IntFlag{ - Name: "statediff.file.nodeid", - Usage: "Node row ID to use as FK when writing state diffs to database", - } StateDiffFilePath = cli.StringFlag{ Name: "statediff.file.path", Usage: "Full path (including filename) to write statediff data out to when operating in file mode", diff --git a/statediff/README.md b/statediff/README.md index 97666d50a..7170363ae 100644 --- a/statediff/README.md +++ b/statediff/README.md @@ -79,7 +79,7 @@ This service introduces a CLI flag namespace `statediff` `--statediff` flag is used to turn on the service `--statediff.writing` is used to tell the service to write state diff objects it produces from synced ChainEvents directly to a configured Postgres database `--statediff.workers` is used to set the number of concurrent workers to process state diff objects and write them into the database -`--statediff.db.type` is the type of database we write out to (current options: postgres and dump) +`--statediff.db.type` is the type of database we write out to (current options: postgres, dump, file) `--statediff.dump.dst` is the destination to write to when operating in database dump mode (stdout, stderr, discard) `--statediff.db.driver` is the specific driver to use for the database (current options for postgres: pgx and sqlx) `--statediff.db.host` is the hostname/ip to dial to connect to the database @@ -95,6 +95,7 @@ This service introduces a CLI flag namespace `statediff` `--statediff.db.maxconnlifetime` is the maximum lifetime for a connection (in seconds) `--statediff.db.nodeid` is the node id to use in the Postgres database `--statediff.db.clientname` is the client name to use in the Postgres database +`--statediff.file.path` full path (including filename) to write statediff data out to when operating in file mode The service can only operate in full sync mode (`--syncmode=full`), but only the historical RPC endpoints require an archive node (`--gcmode=archive`) diff --git a/statediff/indexer/constructor.go b/statediff/indexer/constructor.go index 60e6dadb7..bfb746080 100644 --- a/statediff/indexer/constructor.go +++ b/statediff/indexer/constructor.go @@ -38,6 +38,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, n if !ok { return nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{}) } + fc.NodeInfo = nodeInfo return file.NewStateDiffIndexer(ctx, chainConfig, fc) case shared.POSTGRES: pgc, ok := config.(postgres.Config) diff --git a/statediff/indexer/database/file/config.go b/statediff/indexer/database/file/config.go index fd9e488c9..2553174a3 100644 --- a/statediff/indexer/database/file/config.go +++ b/statediff/indexer/database/file/config.go @@ -17,13 +17,14 @@ package file import ( + "github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/ethereum/go-ethereum/statediff/indexer/shared" ) // Config holds params for writing sql statements out to a file type Config struct { - NodeID int64 // this is the nodeID used as FK in eth.header_cids FilePath string + NodeInfo node.Info } // Type satisfies interfaces.Config diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index 57c5c2e1d..1cc19480a 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -55,7 +55,7 @@ var ( type StateDiffIndexer struct { writer *SQLWriter chainConfig *params.ChainConfig - nodeID int64 + nodeID string wg *sync.WaitGroup } @@ -78,7 +78,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, c return &StateDiffIndexer{ writer: w, chainConfig: chainConfig, - nodeID: config.NodeID, + nodeID: config.NodeInfo.ID, wg: wg, }, nil } diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index d95fba328..5ee169229 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -28,6 +28,7 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/models" + nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" ) var ( @@ -102,15 +103,18 @@ func (sqw *SQLWriter) flush() error { } const ( + nodeInsert = `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES (%s, %s, %s, %s, %d) + ON CONFLICT (node_id) DO NOTHING;\n` + ipldInsert = `INSERT INTO public.blocks (key, data) VALUES (%s, %x) ON CONFLICT (key) DO NOTHING;\n` headerInsert = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) -VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, %d, %d) -ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, %d);\n` +VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, %d, %d) +ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, %d);\n` headerInsertWithoutBaseFee = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) -VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, %d, NULL) -ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %d, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, NULL);\n` +VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, %d, NULL) +ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, NULL);\n` uncleInsert = `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES (%s, %s, %s, %s, %s, %s) ON CONFLICT (block_hash) DO NOTHING;\n` @@ -137,6 +141,12 @@ ON CONFLICT (header_id, state_path) DO NOTHING;\n` ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = (%s, %s, %d, %t, %s);\n` ) +// ON CONFLICT (node_id) DO UPDATE SET genesis_block = %s, network_id = %s, client_name = %s, chain_id = %s;\n` + +func (sqw *SQLWriter) upsertNode(node nodeinfo.Info) { + sqw.stmts <- []byte(fmt.Sprintf(nodeInsert, node.GenesisBlock, node.NetworkID, node.ID, node.ClientName, node.ChainID)) +} + func (sqw *SQLWriter) upsertIPLD(ipld models.IPLDModel) { sqw.stmts <- []byte(fmt.Sprintf(ipldInsert, ipld.Key, ipld.Data)) } diff --git a/statediff/indexer/database/sql/interfaces.go b/statediff/indexer/database/sql/interfaces.go index 3ed1a11e7..445b35d9b 100644 --- a/statediff/indexer/database/sql/interfaces.go +++ b/statediff/indexer/database/sql/interfaces.go @@ -36,7 +36,7 @@ type Driver interface { Get(ctx context.Context, dest interface{}, query string, args ...interface{}) error Begin(ctx context.Context) (Tx, error) Stats() Stats - NodeID() int64 + NodeID() string Context() context.Context io.Closer } diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index 26cc660ec..a86927341 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -26,7 +26,6 @@ import ( "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" - "github.com/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" @@ -153,7 +152,7 @@ func TestPGXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupPGX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, block_hash, base_fee + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -165,7 +164,12 @@ func TestPGXIndexer(t *testing.T) { BaseFee *int64 `db:"base_fee"` } header := new(res) - err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) + err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).Scan( + &header.CID, + &header.TD, + &header.Reward, + &header.BlockHash, + &header.BaseFee) if err != nil { t.Fatal(err) } @@ -221,43 +225,43 @@ func TestPGXIndexer(t *testing.T) { switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) - var txType *uint8 + var txType uint8 err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected tx_type 0, got %d", txType) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) - var txType *uint8 + var txType uint8 err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected tx_type 0, got %d", txType) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) - var txType *uint8 + var txType uint8 err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if txType != nil { - t.Fatalf("expected nil tx_type, got %d", *txType) + if txType != 0 { + t.Fatalf("expected tx_type 0, got %d", txType) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) - var txType *uint8 + var txType uint8 err = db.Get(context.Background(), &txType, txTypePgStr, c) if err != nil { t.Fatal(err) } - if *txType != types.AccessListTxType { - t.Fatalf("expected AccessListTxType (1), got %d", *txType) + if txType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txType) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_element.index ASC` @@ -467,7 +471,7 @@ func TestPGXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - pgStr = `SELECT * from eth.state_accounts WHERE header_id = $1 AND state_path = $2` + pgStr = `SELECT header_id, state_path, cast(balance AS TEXT), nonce, code_hash, storage_root from eth.state_accounts WHERE header_id = $1 AND state_path = $2` var account models.StateAccountModel err = db.Get(context.Background(), &account, pgStr, stateNode.HeaderID, stateNode.Path) if err != nil { diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index 825e50163..5794bd0af 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -30,7 +30,6 @@ type DriverType string const ( PGX DriverType = "PGX" SQLX DriverType = "SQLX" - FILE DriverType = "File" Unknown DriverType = "Unknown" ) @@ -41,8 +40,6 @@ func ResolveDriverType(str string) (DriverType, error) { return PGX, nil case "sqlx": return SQLX, nil - case "file": - return FILE, nil default: return Unknown, fmt.Errorf("unrecognized driver type string: %s", str) } @@ -52,9 +49,9 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_test", + DatabaseName: "vulcanize_testing", Username: "postgres", - Password: "", + Password: "password", } // Config holds params for a Postgres db diff --git a/statediff/indexer/database/sql/postgres/database.go b/statediff/indexer/database/sql/postgres/database.go index ef091760d..213638017 100644 --- a/statediff/indexer/database/sql/postgres/database.go +++ b/statediff/indexer/database/sql/postgres/database.go @@ -22,14 +22,7 @@ var _ sql.Database = &DB{} const ( createNodeStm = `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES ($1, $2, $3, $4, $5) - ON CONFLICT (genesis_block, network_id, node_id, chain_id) - DO UPDATE - SET genesis_block = $1, - network_id = $2, - node_id = $3, - client_name = $4, - chain_id = $5 - RETURNING id` + ON CONFLICT (node_id) DO NOTHING` ) // NewPostgresDB returns a postgres.DB using the provided driver @@ -37,7 +30,7 @@ func NewPostgresDB(driver sql.Driver) *DB { return &DB{driver} } -// DB implements sql.Databse using a configured driver and Postgres statement syntax +// DB implements sql.Database using a configured driver and Postgres statement syntax type DB struct { sql.Driver } diff --git a/statediff/indexer/database/sql/postgres/pgx.go b/statediff/indexer/database/sql/postgres/pgx.go index fa9b84dd0..936a3765d 100644 --- a/statediff/indexer/database/sql/postgres/pgx.go +++ b/statediff/indexer/database/sql/postgres/pgx.go @@ -34,7 +34,7 @@ type PGXDriver struct { ctx context.Context pool *pgxpool.Pool nodeInfo node.Info - nodeID int64 + nodeID string } // NewPGXDriver returns a new pgx driver @@ -89,17 +89,16 @@ func MakeConfig(config Config) (*pgxpool.Config, error) { } func (pgx *PGXDriver) createNode() error { - var nodeID int64 - err := pgx.pool.QueryRow( + _, err := pgx.pool.Exec( pgx.ctx, createNodeStm, pgx.nodeInfo.GenesisBlock, pgx.nodeInfo.NetworkID, pgx.nodeInfo.ID, pgx.nodeInfo.ClientName, - pgx.nodeInfo.ChainID).Scan(&nodeID) + pgx.nodeInfo.ChainID) if err != nil { return ErrUnableToSetNode(err) } - pgx.nodeID = nodeID + pgx.nodeID = pgx.nodeInfo.ID return nil } @@ -139,7 +138,7 @@ func (pgx *PGXDriver) Stats() sql.Stats { } // NodeID satisfies sql.Database -func (pgx *PGXDriver) NodeID() int64 { +func (pgx *PGXDriver) NodeID() string { return pgx.nodeID } diff --git a/statediff/indexer/database/sql/postgres/sqlx.go b/statediff/indexer/database/sql/postgres/sqlx.go index 0bbd0d9e9..406b44a19 100644 --- a/statediff/indexer/database/sql/postgres/sqlx.go +++ b/statediff/indexer/database/sql/postgres/sqlx.go @@ -32,7 +32,7 @@ type SQLXDriver struct { ctx context.Context db *sqlx.DB nodeInfo node.Info - nodeID int64 + nodeID string } // NewSQLXDriver returns a new sqlx driver for Postgres @@ -60,16 +60,15 @@ func NewSQLXDriver(ctx context.Context, config Config, node node.Info) (*SQLXDri } func (driver *SQLXDriver) createNode() error { - var nodeID int64 - err := driver.db.QueryRowx( + _, err := driver.db.Exec( createNodeStm, driver.nodeInfo.GenesisBlock, driver.nodeInfo.NetworkID, driver.nodeInfo.ID, driver.nodeInfo.ClientName, - driver.nodeInfo.ChainID).Scan(&nodeID) + driver.nodeInfo.ChainID) if err != nil { return ErrUnableToSetNode(err) } - driver.nodeID = nodeID + driver.nodeID = driver.nodeInfo.ID return nil } @@ -108,7 +107,7 @@ func (driver *SQLXDriver) Stats() sql.Stats { } // NodeID satisfies sql.Database -func (driver *SQLXDriver) NodeID() int64 { +func (driver *SQLXDriver) NodeID() string { return driver.nodeID } diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index 60d83d96e..d37aa5449 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -32,7 +32,7 @@ type HeaderModel struct { CID string `db:"cid"` MhKey string `db:"mh_key"` TotalDifficulty string `db:"td"` - NodeID int64 `db:"node_id"` + NodeID string `db:"node_id"` Reward string `db:"reward"` StateRoot string `db:"state_root"` UncleRoot string `db:"uncle_root"` diff --git a/statediff/test_helpers/mocks/service_test.go b/statediff/test_helpers/mocks/service_test.go index b3b77d4bf..dde784316 100644 --- a/statediff/test_helpers/mocks/service_test.go +++ b/statediff/test_helpers/mocks/service_test.go @@ -24,6 +24,7 @@ import ( "sort" "sync" "testing" + "time" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -150,29 +151,35 @@ func testSubscriptionAPI(t *testing.T) { id := rpc.NewID() payloadChan := make(chan statediff.Payload) quitChan := make(chan bool) + wg := new(sync.WaitGroup) + go func() { + wg.Add(1) + defer wg.Done() + sort.Slice(expectedStateDiffBytes, func(i, j int) bool { return expectedStateDiffBytes[i] < expectedStateDiffBytes[j] }) + select { + case payload := <-payloadChan: + if !bytes.Equal(payload.BlockRlp, expectedBlockRlp) { + t.Errorf("payload does not have expected block\r\nactual block rlp: %v\r\nexpected block rlp: %v", payload.BlockRlp, expectedBlockRlp) + } + sort.Slice(payload.StateObjectRlp, func(i, j int) bool { return payload.StateObjectRlp[i] < payload.StateObjectRlp[j] }) + if !bytes.Equal(payload.StateObjectRlp, expectedStateDiffBytes) { + t.Errorf("payload does not have expected state diff\r\nactual state diff rlp: %v\r\nexpected state diff rlp: %v", payload.StateObjectRlp, expectedStateDiffBytes) + } + if !bytes.Equal(expectedReceiptBytes, payload.ReceiptsRlp) { + t.Errorf("payload does not have expected receipts\r\nactual receipt rlp: %v\r\nexpected receipt rlp: %v", payload.ReceiptsRlp, expectedReceiptBytes) + } + if !bytes.Equal(payload.TotalDifficulty.Bytes(), mockTotalDifficulty.Bytes()) { + t.Errorf("payload does not have expected total difficulty\r\nactual td: %d\r\nexpected td: %d", payload.TotalDifficulty.Int64(), mockTotalDifficulty.Int64()) + } + case <-quitChan: + t.Errorf("channel quit before delivering payload") + } + }() + time.Sleep(1) mockService.Subscribe(id, payloadChan, quitChan, params) blockChan <- block1 parentBlockChain <- block0 - - sort.Slice(expectedStateDiffBytes, func(i, j int) bool { return expectedStateDiffBytes[i] < expectedStateDiffBytes[j] }) - select { - case payload := <-payloadChan: - if !bytes.Equal(payload.BlockRlp, expectedBlockRlp) { - t.Errorf("payload does not have expected block\r\nactual block rlp: %v\r\nexpected block rlp: %v", payload.BlockRlp, expectedBlockRlp) - } - sort.Slice(payload.StateObjectRlp, func(i, j int) bool { return payload.StateObjectRlp[i] < payload.StateObjectRlp[j] }) - if !bytes.Equal(payload.StateObjectRlp, expectedStateDiffBytes) { - t.Errorf("payload does not have expected state diff\r\nactual state diff rlp: %v\r\nexpected state diff rlp: %v", payload.StateObjectRlp, expectedStateDiffBytes) - } - if !bytes.Equal(expectedReceiptBytes, payload.ReceiptsRlp) { - t.Errorf("payload does not have expected receipts\r\nactual receipt rlp: %v\r\nexpected receipt rlp: %v", payload.ReceiptsRlp, expectedReceiptBytes) - } - if !bytes.Equal(payload.TotalDifficulty.Bytes(), mockTotalDifficulty.Bytes()) { - t.Errorf("payload does not have expected total difficulty\r\nactual td: %d\r\nexpected td: %d", payload.TotalDifficulty.Int64(), mockTotalDifficulty.Int64()) - } - case <-quitChan: - t.Errorf("channel quit before delivering payload") - } + wg.Wait() } func testHTTPAPI(t *testing.T) { -- 2.45.2 From 960033eb930ef91b6de8c015ea0011f3b482ffce Mon Sep 17 00:00:00 2001 From: i-norden Date: Thu, 18 Nov 2021 14:56:26 -0600 Subject: [PATCH 12/33] misc fixes/adjustments --- statediff/builder.go | 2 +- statediff/indexer/constructor.go | 4 + statediff/indexer/database/file/indexer.go | 9 +- statediff/indexer/database/file/writer.go | 107 ++++++++++-------- .../indexer/database/sql/batch_writer.go | 4 +- .../indexer/database/sql/pgx_indexer_test.go | 2 +- .../indexer/database/sql/postgres/config.go | 4 +- .../indexer/database/sql/postgres/database.go | 2 +- .../indexer/database/sql/sqlx_indexer_test.go | 2 +- .../indexer/database/sql/test_helpers.go | 2 +- statediff/indexer/database/sql/writer.go | 2 +- statediff/service.go | 1 - 12 files changed, 75 insertions(+), 66 deletions(-) diff --git a/statediff/builder.go b/statediff/builder.go index 8dc3cece8..7811c3e82 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -167,7 +167,7 @@ func (sdb *builder) BuildStateDiffObject(args Args, params Params) (types2.State }, nil } -// Writes a statediff object to output callback +// WriteStateDiffObject writes a statediff object to output callback func (sdb *builder) WriteStateDiffObject(args types2.StateRoots, params Params, output types2.StateNodeSink, codeOutput types2.CodeSink) error { if !params.IntermediateStateNodes || len(params.WatchedAddresses) > 0 { // if we are watching only specific accounts then we are only diffing leaf nodes diff --git a/statediff/indexer/constructor.go b/statediff/indexer/constructor.go index bfb746080..9a66dba89 100644 --- a/statediff/indexer/constructor.go +++ b/statediff/indexer/constructor.go @@ -20,6 +20,7 @@ import ( "context" "fmt" + "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/statediff/indexer/database/dump" "github.com/ethereum/go-ethereum/statediff/indexer/database/file" @@ -34,6 +35,7 @@ import ( func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, nodeInfo node.Info, config interfaces.Config) (interfaces.StateDiffIndexer, error) { switch config.Type() { case shared.FILE: + log.Info("Starting statediff service in SQL file writing mode") fc, ok := config.(file.Config) if !ok { return nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{}) @@ -41,6 +43,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, n fc.NodeInfo = nodeInfo return file.NewStateDiffIndexer(ctx, chainConfig, fc) case shared.POSTGRES: + log.Info("Starting statediff service in Postgres writing mode") pgc, ok := config.(postgres.Config) if !ok { return nil, fmt.Errorf("postgres config is not the correct type: got %T, expected %T", config, postgres.Config{}) @@ -63,6 +66,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, n } return sql.NewStateDiffIndexer(ctx, chainConfig, postgres.NewPostgresDB(driver)) case shared.DUMP: + log.Info("Starting statediff service in data dump mode") dumpc, ok := config.(dump.Config) if !ok { return nil, fmt.Errorf("dump config is not the correct type: got %T, expected %T", config, dump.Config{}) diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index 1cc19480a..454d1e3d1 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -72,9 +72,12 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, c if err != nil { return nil, fmt.Errorf("unable to create file (%s), err: %v", filePath, err) } + log.Info("Writing statediff SQL statements to file", "file", filePath) w := NewSQLWriter(file) wg := new(sync.WaitGroup) w.Loop() + w.upsertNode(config.NodeInfo) + w.upsertIPLDDirect(shared.RemovedNodeMhKey, []byte{}) return &StateDiffIndexer{ writer: w, chainConfig: chainConfig, @@ -130,11 +133,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) t = time.Now() - if err := sdi.writer.flush(); err != nil { - traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String()) - log.Debug(traceMsg) - return err - } + sdi.writer.Flush() tDiff = time.Since(t) indexerMetrics.tPostgresCommit.Update(tDiff) traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String()) diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 5ee169229..fdfa87b08 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -25,7 +25,6 @@ import ( node "github.com/ipfs/go-ipld-format" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/models" nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" @@ -33,7 +32,8 @@ import ( var ( nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") - collatedStmtSize = 65336 // min(linuxPipeSize, macOSPipeSize) + pipeSize = 65336 // min(linuxPipeSize, macOSPipeSize) + collatedStmtSize = pipeSize * 16 ) // SQLWriter writes sql statements to a file @@ -43,18 +43,22 @@ type SQLWriter struct { collatedStmt []byte collationIndex int - quitChan chan struct{} - doneChan chan struct{} + flushChan chan struct{} + flushFinished chan struct{} + quitChan chan struct{} + doneChan chan struct{} } // NewSQLWriter creates a new pointer to a Writer func NewSQLWriter(file *os.File) *SQLWriter { return &SQLWriter{ - file: file, - stmts: make(chan []byte), - collatedStmt: make([]byte, collatedStmtSize), - quitChan: make(chan struct{}), - doneChan: make(chan struct{}), + file: file, + stmts: make(chan []byte), + collatedStmt: make([]byte, collatedStmtSize), + flushChan: make(chan struct{}), + flushFinished: make(chan struct{}), + quitChan: make(chan struct{}), + doneChan: make(chan struct{}), } } @@ -72,16 +76,21 @@ func (sqw *SQLWriter) Loop() { l = len(stmt) if l+sqw.collationIndex+1 > collatedStmtSize { if err := sqw.flush(); err != nil { - log.Error("error writing cached sql stmts to file", "err", err) + panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err)) } } - copy(sqw.collatedStmt[sqw.collationIndex:sqw.collationIndex+l-1], stmt) + copy(sqw.collatedStmt[sqw.collationIndex:sqw.collationIndex+l], stmt) sqw.collationIndex += l case <-sqw.quitChan: if err := sqw.flush(); err != nil { - log.Error("error writing cached sql stmts to file", "err", err) + panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err)) } return + case <-sqw.flushChan: + if err := sqw.flush(); err != nil { + panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err)) + } + sqw.flushFinished <- struct{}{} } } }() @@ -94,8 +103,14 @@ func (sqw *SQLWriter) Close() error { return nil } +// Flush sends a flush signal to the looping process +func (sqw *SQLWriter) Flush() { + sqw.flushChan <- struct{}{} + <-sqw.flushFinished +} + func (sqw *SQLWriter) flush() error { - if _, err := sqw.file.Write(sqw.collatedStmt[0 : sqw.collationIndex-1]); err != nil { + if _, err := sqw.file.Write(sqw.collatedStmt[0:sqw.collationIndex]); err != nil { return err } sqw.collationIndex = 0 @@ -103,46 +118,43 @@ func (sqw *SQLWriter) flush() error { } const ( - nodeInsert = `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES (%s, %s, %s, %s, %d) - ON CONFLICT (node_id) DO NOTHING;\n` + nodeInsert = "INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES " + + "('%s', '%s', '%s', '%s', %d);\n" - ipldInsert = `INSERT INTO public.blocks (key, data) VALUES (%s, %x) ON CONFLICT (key) DO NOTHING;\n` + ipldInsert = "INSERT INTO public.blocks (key, data) VALUES ('%s', '%x');\n" - headerInsert = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) -VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, %d, %d) -ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, %d);\n` + headerInsert = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, " + + "state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + + "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%x', %d, '%s', %d, %d);\n" - headerInsertWithoutBaseFee = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) -VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, %d, NULL) -ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %d, %s, eth.header_cids.times_validated + 1, NULL);\n` + headerInsertWithoutBaseFee = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, " + + "reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + + "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%x', %d, '%s', %d, NULL);\n" - uncleInsert = `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES (%s, %s, %s, %s, %s, %s) -ON CONFLICT (block_hash) DO NOTHING;\n` + uncleInsert = "INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES " + + "('%s', '%s', '%s', '%s', '%s', '%s');\n" - txInsert = `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES (%s, %s, %s, %s, %s, %d, %s, %s, %d) -ON CONFLICT (tx_hash) DO NOTHING;\n` + txInsert = "INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) " + + "VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '%x', %d);\n" - alInsert = `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES (%s, %d, %s, %s) -ON CONFLICT (tx_id, index) DO NOTHING;\n` + alInsert = "INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ('%s', %d, '%s', '%s');\n" - rctInsert = `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES (%s, %s, %s, %s, %s, %s, %d, %s) -ON CONFLICT (tx_id) DO NOTHING;\n` + rctInsert = "INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, " + + "post_status, log_root) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', %d, '%s');\n" - logInsert = `INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES (%s, %s, %s, %s, %d, %s, %s, %s, %s, %s) -ON CONFLICT (rct_id, index) DO NOTHING;\n` + logInsert = "INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, " + + "topic3, log_data) VALUES ('%s', '%s', '%s', '%s', %d, '%s', '%s', '%s', '%s', '%x');\n" - stateInsert = `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES (%s, %s, %s, %s, %d, %t, %s) -ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = (%s, %s, %d, %t, %s);\n` + stateInsert = "INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) " + + "VALUES ('%s', '%s', '%s', '%x', %d, %t, '%s');\n" - accountInsert = `INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) VALUES (%s, %s, %s, %d, %s, %s) -ON CONFLICT (header_id, state_path) DO NOTHING;\n` + accountInsert = "INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) " + + "VALUES ('%s', '%x', '%s', %d, '%x', '%s');\n" - storageInsert = `INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES (%s, %s, %s, %s, %s, %d, %t, %s) -ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = (%s, %s, %d, %t, %s);\n` + storageInsert = "INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, " + + "node_type, diff, mh_key) VALUES ('%s', '%x', '%s', '%s', '%x', %d, %t, '%s');\n" ) -// ON CONFLICT (node_id) DO UPDATE SET genesis_block = %s, network_id = %s, client_name = %s, chain_id = %s;\n` - func (sqw *SQLWriter) upsertNode(node nodeinfo.Info) { sqw.stmts <- []byte(fmt.Sprintf(nodeInsert, node.GenesisBlock, node.NetworkID, node.ID, node.ClientName, node.ChainID)) } @@ -183,15 +195,11 @@ func (sqw *SQLWriter) upsertHeaderCID(header models.HeaderModel) { if header.BaseFee == nil { stmt = fmt.Sprintf(headerInsertWithoutBaseFee, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, - header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, - header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey) + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1) } else { stmt = fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee, - header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, - header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, header.BaseFee) + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee) } sqw.stmts <- []byte(stmt) indexerMetrics.blocks.Inc(1) @@ -228,8 +236,8 @@ func (sqw *SQLWriter) upsertStateCID(stateNode models.StateNodeModel) { if stateNode.StateKey != nullHash.String() { stateKey = stateNode.StateKey } - sqw.stmts <- []byte(fmt.Sprintf(stateInsert, stateNode.HeaderID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, - true, stateNode.MhKey, stateKey, stateNode.CID, stateNode.NodeType, true, stateNode.MhKey)) + sqw.stmts <- []byte(fmt.Sprintf(stateInsert, stateNode.HeaderID, stateKey, stateNode.CID, stateNode.Path, + stateNode.NodeType, true, stateNode.MhKey)) } func (sqw *SQLWriter) upsertStateAccount(stateAccount models.StateAccountModel) { @@ -243,6 +251,5 @@ func (sqw *SQLWriter) upsertStorageCID(storageCID models.StorageNodeModel) { storageKey = storageCID.StorageKey } sqw.stmts <- []byte(fmt.Sprintf(storageInsert, storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, - storageCID.Path, storageCID.NodeType, true, storageCID.MhKey, storageKey, storageCID.CID, storageCID.NodeType, - true, storageCID.MhKey)) + storageCID.Path, storageCID.NodeType, true, storageCID.MhKey)) } diff --git a/statediff/indexer/database/sql/batch_writer.go b/statediff/indexer/database/sql/batch_writer.go index 05c882259..f186d8052 100644 --- a/statediff/indexer/database/sql/batch_writer.go +++ b/statediff/indexer/database/sql/batch_writer.go @@ -43,7 +43,7 @@ const ( txCIDsPgStr string = `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9)) ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = (excluded.cid, excluded.dst, excluded.src, excluded.index, excluded.mh_key, excluded.tx_data, excluded.tx_type) RETURNING id` - accessListPgStr string = `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES (unnest($1), unnest($2), unnest($3), unnest($4)) + accessListPgStr string = `INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES (unnest($1), unnest($2), unnest($3), unnest($4)) ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = (excluded.address, excluded.storage_keys)` rctCIDsPgStr string = `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8)) ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = (excluded.leaf_cid, excluded.contract, excluded.contract_hash, excluded.leaf_mh_key, excluded.post_state, excluded.post_status, excluded.log_root) @@ -138,7 +138,7 @@ func (pbw *PostgresBatchWriter) upsertTransactionCID(tx *sqlx.Tx, transaction mo } func (pbw *PostgresBatchWriter) upsertAccessListElement(tx *sqlx.Tx, accessListElement models.AccessListElementModel, txID int64) error { - _, err := tx.Exec(`INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) + _, err := tx.Exec(`INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)`, txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) if err != nil { diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index a86927341..710ad23d9 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -264,7 +264,7 @@ func TestPGXIndexer(t *testing.T) { t.Fatalf("expected AccessListTxType (1), got %d", txType) } accessListElementModels := make([]models.AccessListElementModel, 0) - pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_element.index ASC` + pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` err = db.Select(context.Background(), &accessListElementModels, pgStr, c) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index 5794bd0af..a7c7cc9b4 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -49,9 +49,9 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_testing", + DatabaseName: "vulcanize_test", Username: "postgres", - Password: "password", + Password: "", } // Config holds params for a Postgres db diff --git a/statediff/indexer/database/sql/postgres/database.go b/statediff/indexer/database/sql/postgres/database.go index 213638017..99fae1c02 100644 --- a/statediff/indexer/database/sql/postgres/database.go +++ b/statediff/indexer/database/sql/postgres/database.go @@ -56,7 +56,7 @@ func (db *DB) InsertTxStm() string { // InsertAccessListElementStm satisfies the sql.Statements interface func (db *DB) InsertAccessListElementStm() string { - return `INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) + return `INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) ON CONFLICT (tx_id, index) DO NOTHING` } diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 09ee62fa3..e0b5f2967 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -286,7 +286,7 @@ func TestSQLXIndexer(t *testing.T) { t.Fatalf("expected AccessListTxType (1), got %d", txType) } accessListElementModels := make([]models.AccessListElementModel, 0) - pgStr = `SELECT access_list_element.* FROM eth.access_list_element INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_element.index ASC` + pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` err = db.Select(context.Background(), &accessListElementModels, pgStr, c) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/test_helpers.go b/statediff/indexer/database/sql/test_helpers.go index 7de7beec0..46f9b766b 100644 --- a/statediff/indexer/database/sql/test_helpers.go +++ b/statediff/indexer/database/sql/test_helpers.go @@ -53,7 +53,7 @@ func TearDownDB(t *testing.T, db Database) { if err != nil { t.Fatal(err) } - _, err = tx.Exec(ctx, `DELETE FROM eth.access_list_element`) + _, err = tx.Exec(ctx, `DELETE FROM eth.access_list_elements`) if err != nil { t.Fatal(err) } diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go index 94b38c7e1..3089b6d50 100644 --- a/statediff/indexer/database/sql/writer.go +++ b/statediff/indexer/database/sql/writer.go @@ -83,7 +83,7 @@ func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error } /* -INSERT INTO eth.access_list_element (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) +INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) ON CONFLICT (tx_id, index) DO NOTHING */ func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel) error { diff --git a/statediff/service.go b/statediff/service.go index 04aaac458..5334b4b31 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -239,7 +239,6 @@ func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) { statediffMetrics.writeLoopChannelLen.Update(int64(len(chainEventCh))) chainEventFwd <- chainEvent case err := <-errCh: - println("here") log.Error("Error from chain event subscription", "error", err) close(sds.QuitChan) log.Info("Quitting the statediffing writing loop") -- 2.45.2 From 7fed13e725a0653680885869d16bd90c20341ea4 Mon Sep 17 00:00:00 2001 From: i-norden Date: Thu, 18 Nov 2021 18:10:50 -0600 Subject: [PATCH 13/33] update README --- statediff/README.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/statediff/README.md b/statediff/README.md index 7170363ae..92c8ef387 100644 --- a/statediff/README.md +++ b/statediff/README.md @@ -104,6 +104,13 @@ e.g. ./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db.type=postgres --statediff.db.driver=sqlx --statediff.db.host=localhost --statediff.db.port=5432 --statediff.db.name=vulcanize_test --statediff.db.user=postgres --statediff.db.nodeid=nodeid --statediff.db.clientname=clientname ` +When operating in `--statediff.db.type=file` mode, the service will write SQL statements out to the file designated by +`--statediff.file.path`. Please note that it writes out SQL statements with all `ON CONFLICT` constraint checks dropped. +This is done so that we can scale out the production of the SQL statements horizontally, merge the separate SQL files produced, +de-duplicate using unix tools (`sort statediff.sql | uniq` or `sort -u statediff.sql`), bulk load using psql +(`psql db_name --set ON_ERROR_STOP=on -f statediff.sql`), and then add our primary and foreign key constraints and indexes +back afterwards. + ### RPC endpoints The state diffing service exposes both a WS subscription endpoint, and a number of HTTP unary endpoints. -- 2.45.2 From b96c26fc9726850cab0d3bee66aaaf4aa224254d Mon Sep 17 00:00:00 2001 From: i-norden Date: Sat, 20 Nov 2021 12:24:43 -0600 Subject: [PATCH 14/33] cleanup; more unit tests --- statediff/indexer/database/file/config.go | 12 + statediff/indexer/database/file/indexer.go | 48 +- .../database/file/indexer_legacy_test.go | 131 ++++ .../indexer/database/file/indexer_test.go | 630 ++++++++++++++++++ .../indexer/database/file/test_helpers.go | 28 + statediff/indexer/database/file/writer.go | 12 +- .../indexer/database/sql/batch_writer.go | 216 ------ statediff/indexer/database/sql/indexer.go | 2 +- .../indexer/database/sql/pgx_indexer_test.go | 4 +- .../indexer/database/sql/postgres/config.go | 4 +- .../indexer/database/sql/postgres/pgx_test.go | 3 +- .../database/sql/postgres/sqlx_test.go | 16 +- .../indexer/database/sql/sqlx_indexer_test.go | 4 +- statediff/indexer/database/sql/writer.go | 43 +- 14 files changed, 873 insertions(+), 280 deletions(-) create mode 100644 statediff/indexer/database/file/indexer_legacy_test.go create mode 100644 statediff/indexer/database/file/indexer_test.go create mode 100644 statediff/indexer/database/file/test_helpers.go delete mode 100644 statediff/indexer/database/sql/batch_writer.go diff --git a/statediff/indexer/database/file/config.go b/statediff/indexer/database/file/config.go index 2553174a3..c2c6804c0 100644 --- a/statediff/indexer/database/file/config.go +++ b/statediff/indexer/database/file/config.go @@ -31,3 +31,15 @@ type Config struct { func (c Config) Type() shared.DBType { return shared.FILE } + +// TestConfig config for unit tests +var TestConfig = Config{ + FilePath: "./statediffing_test_file.sql", + NodeInfo: node.Info{ + GenesisBlock: "0xd4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3", + NetworkID: "1", + ChainID: 1, + ID: "mockNodeID", + ClientName: "go-ethereum", + }, +} diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index 454d1e3d1..e945f155d 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -53,7 +53,7 @@ var ( // StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of a void type StateDiffIndexer struct { - writer *SQLWriter + fileWriter *SQLWriter chainConfig *params.ChainConfig nodeID string wg *sync.WaitGroup @@ -79,7 +79,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, c w.upsertNode(config.NodeInfo) w.upsertIPLDDirect(shared.RemovedNodeMhKey, []byte{}) return &StateDiffIndexer{ - writer: w, + fileWriter: w, chainConfig: chainConfig, nodeID: config.NodeInfo.ID, wg: wg, @@ -133,7 +133,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip indexerMetrics.tStateStoreCodeProcessing.Update(tDiff) traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) t = time.Now() - sdi.writer.Flush() + sdi.fileWriter.Flush() tDiff = time.Since(t) indexerMetrics.tPostgresCommit.Update(tDiff) traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String()) @@ -189,7 +189,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // processHeader write a header IPLD insert SQL stmt to a file // it returns the headerID func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node.Node, reward, td *big.Int) string { - sdi.writer.upsertIPLDNode(headerNode) + sdi.fileWriter.upsertIPLDNode(headerNode) var baseFee *int64 if header.BaseFee != nil { @@ -197,7 +197,7 @@ func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node *baseFee = header.BaseFee.Int64() } headerID := header.Hash().String() - sdi.writer.upsertHeaderCID(models.HeaderModel{ + sdi.fileWriter.upsertHeaderCID(models.HeaderModel{ NodeID: sdi.nodeID, CID: headerNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(headerNode.Cid()), @@ -221,7 +221,7 @@ func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node func (sdi *StateDiffIndexer) processUncles(headerID string, blockNumber uint64, uncleNodes []*ipld2.EthHeader) { // publish and index uncles for _, uncleNode := range uncleNodes { - sdi.writer.upsertIPLDNode(uncleNode) + sdi.fileWriter.upsertIPLDNode(uncleNode) var uncleReward *big.Int // in PoA networks uncle reward is 0 if sdi.chainConfig.Clique != nil { @@ -229,7 +229,7 @@ func (sdi *StateDiffIndexer) processUncles(headerID string, blockNumber uint64, } else { uncleReward = shared.CalcUncleMinerReward(blockNumber, uncleNode.Number.Uint64()) } - sdi.writer.upsertUncleCID(models.UncleModel{ + sdi.fileWriter.upsertUncleCID(models.UncleModel{ HeaderID: headerID, CID: uncleNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(uncleNode.Cid()), @@ -261,10 +261,10 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { signer := types.MakeSigner(sdi.chainConfig, args.blockNumber) for i, receipt := range args.receipts { for _, logTrieNode := range args.logTrieNodes[i] { - sdi.writer.upsertIPLDNode(logTrieNode) + sdi.fileWriter.upsertIPLDNode(logTrieNode) } txNode := args.txNodes[i] - sdi.writer.upsertIPLDNode(txNode) + sdi.fileWriter.upsertIPLDNode(txNode) // index tx trx := args.txs[i] @@ -285,7 +285,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { MhKey: shared.MultihashKeyFromCID(txNode.Cid()), Type: trx.Type(), } - sdi.writer.upsertTransactionCID(txModel) + sdi.fileWriter.upsertTransactionCID(txModel) // index access list if this is one for j, accessListElement := range trx.AccessList() { @@ -299,7 +299,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { Address: accessListElement.Address.Hex(), StorageKeys: storageKeys, } - sdi.writer.upsertAccessListElement(accessListElementModel) + sdi.fileWriter.upsertAccessListElement(accessListElementModel) } // this is the contract address if this receipt is for a contract creation tx @@ -327,7 +327,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { } else { rctModel.PostState = common.Bytes2Hex(receipt.PostState) } - sdi.writer.upsertReceiptCID(rctModel) + sdi.fileWriter.upsertReceiptCID(rctModel) // index logs logDataSet := make([]*models.LogsModel, len(receipt.Logs)) @@ -354,13 +354,13 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { Topic3: topicSet[3], } } - sdi.writer.upsertLogCID(logDataSet) + sdi.fileWriter.upsertLogCID(logDataSet) } // publish trie nodes, these aren't indexed directly for i, n := range args.txTrieNodes { - sdi.writer.upsertIPLDNode(n) - sdi.writer.upsertIPLDNode(args.rctTrieNodes[i]) + sdi.fileWriter.upsertIPLDNode(n) + sdi.fileWriter.upsertIPLDNode(args.rctTrieNodes[i]) } return nil @@ -380,10 +380,10 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: shared.RemovedNodeMhKey, NodeType: stateNode.NodeType.Int(), } - sdi.writer.upsertStateCID(stateModel) + sdi.fileWriter.upsertStateCID(stateModel) return nil } - stateCIDStr, stateMhKey, err := sdi.writer.upsertIPLDRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) + stateCIDStr, stateMhKey, err := sdi.fileWriter.upsertIPLDRaw(ipld2.MEthStateTrie, multihash.KECCAK_256, stateNode.NodeValue) if err != nil { return fmt.Errorf("error generating and cacheing state node IPLD: %v", err) } @@ -396,7 +396,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt NodeType: stateNode.NodeType.Int(), } // index the state node - sdi.writer.upsertStateCID(stateModel) + sdi.fileWriter.upsertStateCID(stateModel) // if we have a leaf, decode and index the account data if stateNode.NodeType == sdtypes.Leaf { var i []interface{} @@ -418,7 +418,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt CodeHash: account.CodeHash, StorageRoot: account.Root.String(), } - sdi.writer.upsertStateAccount(accountModel) + sdi.fileWriter.upsertStateAccount(accountModel) } // if there are any storage nodes associated with this node, publish and index them for _, storageNode := range stateNode.StorageNodes { @@ -434,10 +434,10 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: shared.RemovedNodeMhKey, NodeType: storageNode.NodeType.Int(), } - sdi.writer.upsertStorageCID(storageModel) + sdi.fileWriter.upsertStorageCID(storageModel) continue } - storageCIDStr, storageMhKey, err := sdi.writer.upsertIPLDRaw(ipld2.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) + storageCIDStr, storageMhKey, err := sdi.fileWriter.upsertIPLDRaw(ipld2.MEthStorageTrie, multihash.KECCAK_256, storageNode.NodeValue) if err != nil { return fmt.Errorf("error generating and cacheing storage node IPLD: %v", err) } @@ -450,7 +450,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt MhKey: storageMhKey, NodeType: storageNode.NodeType.Int(), } - sdi.writer.upsertStorageCID(storageModel) + sdi.fileWriter.upsertStorageCID(storageModel) } return nil @@ -463,11 +463,11 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAnd if err != nil { return fmt.Errorf("error deriving multihash key from codehash: %v", err) } - sdi.writer.upsertIPLDDirect(mhKey, codeAndCodeHash.Code) + sdi.fileWriter.upsertIPLDDirect(mhKey, codeAndCodeHash.Code) return nil } // Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { - return sdi.writer.Close() + return sdi.fileWriter.Close() } diff --git a/statediff/indexer/database/file/indexer_legacy_test.go b/statediff/indexer/database/file/indexer_legacy_test.go new file mode 100644 index 000000000..d02927165 --- /dev/null +++ b/statediff/indexer/database/file/indexer_legacy_test.go @@ -0,0 +1,131 @@ +// 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 . + +package file_test + +import ( + "context" + "os" + "testing" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ipfs/go-cid" + "github.com/jmoiron/sqlx" + + "github.com/multiformats/go-multihash" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +var ( + legacyData = mocks.NewLegacyData() + mockLegacyBlock *types.Block + legacyHeaderCID cid.Cid +) + +func setupLegacy(t *testing.T) { + mockLegacyBlock = legacyData.MockBlock + legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256) + + ind, err := file.NewStateDiffIndexer(context.Background(), legacyData.Config, file.TestConfig) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + mockLegacyBlock, + legacyData.MockReceipts, + legacyData.MockBlock.Difficulty()) + require.NoError(t, err) + + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + if err := ind.Close(); err != nil { + t.Fatal(err) + } + }() + for _, node := range legacyData.StateDiffs { + err = ind.PushStateNode(tx, node, legacyData.MockBlock.Hash().String()) + require.NoError(t, err) + } + + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, legacyData.BlockNumber.Uint64()) + + connStr := postgres.DefaultConfig.DbConnectionString() + + sqlxdb, err = sqlx.Connect("postgres", connStr) + if err != nil { + t.Fatalf("failed to connect to db with connection string: %s err: %v", connStr, err) + } +} + +func dumpData(t *testing.T) { + sqlFileBytes, err := os.ReadFile(file.TestConfig.FilePath) + require.NoError(t, err) + + _, err = sqlxdb.Exec(string(sqlFileBytes)) + require.NoError(t, err) +} + +func tearDown(t *testing.T) { + err := os.Remove(file.TestConfig.FilePath) + require.NoError(t, err) + err = sqlxdb.Close() + require.NoError(t, err) +} + +func expectTrue(t *testing.T, value bool) { + if !value { + t.Fatalf("Assertion failed") + } +} + +func TestFIleIndexerLegacy(t *testing.T) { + t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { + setupLegacy(t) + dumpData(t) + defer tearDown(t) + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee + FROM eth.header_cids + WHERE block_number = $1` + // check header was properly indexed + type res struct { + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` + } + header := new(res) + + err = sqlxdb.QueryRow(pgStr, legacyData.BlockNumber.Uint64()).Scan( + &header.CID, &header.TD, &header.Reward, &header.BlockHash, &header.BaseFee) + require.NoError(t, err) + + test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) + test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") + require.Nil(t, legacyData.MockHeader.BaseFee) + require.Nil(t, header.BaseFee) + }) +} diff --git a/statediff/indexer/database/file/indexer_test.go b/statediff/indexer/database/file/indexer_test.go new file mode 100644 index 000000000..06efce309 --- /dev/null +++ b/statediff/indexer/database/file/indexer_test.go @@ -0,0 +1,630 @@ +// 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 . + +package file_test + +import ( + "bytes" + "context" + "fmt" + "os" + "testing" + + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" + "github.com/jmoiron/sqlx" + + "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" + + "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/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "github.com/ethereum/go-ethereum/statediff/indexer/shared" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +var ( + sqlxdb *sqlx.DB + err error + ind interfaces.StateDiffIndexer + ipfsPgGet = `SELECT data FROM public.blocks + WHERE key = $1` + tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte + mockBlock *types.Block + headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid + rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid + state1CID, state2CID, storageCID cid.Cid +) + +func init() { + if os.Getenv("MODE") != "statediff" { + fmt.Println("Skipping statediff test") + os.Exit(0) + } + + mockBlock = mocks.MockBlock + txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts + + buf := new(bytes.Buffer) + txs.EncodeIndex(0, buf) + tx1 = make([]byte, buf.Len()) + copy(tx1, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(1, buf) + tx2 = make([]byte, buf.Len()) + copy(tx2, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(2, buf) + tx3 = make([]byte, buf.Len()) + copy(tx3, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(3, buf) + tx4 = make([]byte, buf.Len()) + copy(tx4, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(4, buf) + tx5 = make([]byte, buf.Len()) + copy(tx5, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(0, buf) + rct1 = make([]byte, buf.Len()) + copy(rct1, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(1, buf) + rct2 = make([]byte, buf.Len()) + copy(rct2, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(2, buf) + rct3 = make([]byte, buf.Len()) + copy(rct3, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(3, buf) + rct4 = make([]byte, buf.Len()) + copy(rct4, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(4, buf) + rct5 = make([]byte, buf.Len()) + copy(rct5, buf.Bytes()) + buf.Reset() + + headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256) + trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256) + trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256) + trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) + trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) + trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) + rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) + rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) + rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) + rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) + rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) + state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) + state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) + storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) +} + +func setup(t *testing.T) { + ind, err := file.NewStateDiffIndexer(context.Background(), legacyData.Config, file.TestConfig) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + mockBlock, + mocks.MockReceipts, + mocks.MockBlock.Difficulty()) + if err != nil { + t.Fatal(err) + } + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + if err := ind.Close(); err != nil { + t.Fatal(err) + } + }() + for _, node := range mocks.StateDiffs { + err = ind.PushStateNode(tx, node, mockBlock.Hash().String()) + require.NoError(t, err) + } + + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) + + connStr := postgres.DefaultConfig.DbConnectionString() + + sqlxdb, err = sqlx.Connect("postgres", connStr) + if err != nil { + t.Fatalf("failed to connect to db with connection string: %s err: %v", connStr, err) + } +} + +func TestSQLXIndexer(t *testing.T) { + t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + pgStr := `SELECT cid, td, reward, block_hash, base_fee + FROM eth.header_cids + WHERE block_number = $1` + // check header was properly indexed + type res struct { + CID string + TD string + Reward string + BlockHash string `db:"block_hash"` + BaseFee *int64 `db:"base_fee"` + } + header := new(res) + err = sqlxdb.QueryRowx(pgStr, mocks.BlockNumber.Uint64()).StructScan(header) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, header.CID, headerCID.String()) + test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) + test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + dc, err := cid.Decode(header.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, mocks.MockHeaderRlp) + }) + + t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + + // check that txs were properly indexed + trxs := make([]string, 0) + pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) + WHERE header_cids.block_number = $1` + err = sqlxdb.Select(&trxs, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(trxs), 5) + expectTrue(t, test_helpers.ListContainsString(trxs, trx1CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx2CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) + expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) + // and published + for _, c := range trxs { + dc, err := cid.Decode(c) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + switch c { + case trx1CID.String(): + test_helpers.ExpectEqual(t, data, tx1) + var txType uint8 + err = sqlxdb.Get(&txType, txTypePgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) + } + case trx2CID.String(): + test_helpers.ExpectEqual(t, data, tx2) + var txType uint8 + err = sqlxdb.Get(&txType, txTypePgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) + } + case trx3CID.String(): + test_helpers.ExpectEqual(t, data, tx3) + var txType uint8 + err = sqlxdb.Get(&txType, txTypePgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txType) + } + case trx4CID.String(): + test_helpers.ExpectEqual(t, data, tx4) + var txType uint8 + err = sqlxdb.Get(&txType, txTypePgStr, c) + if err != nil { + t.Fatal(err) + } + if txType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txType) + } + accessListElementModels := make([]models.AccessListElementModel, 0) + pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` + err = sqlxdb.Select(&accessListElementModels, pgStr, c) + if err != nil { + t.Fatal(err) + } + if len(accessListElementModels) != 2 { + t.Fatalf("expected two access list entries, got %d", len(accessListElementModels)) + } + model1 := models.AccessListElementModel{ + Index: accessListElementModels[0].Index, + Address: accessListElementModels[0].Address, + } + model2 := models.AccessListElementModel{ + Index: accessListElementModels[1].Index, + Address: accessListElementModels[1].Address, + StorageKeys: accessListElementModels[1].StorageKeys, + } + test_helpers.ExpectEqual(t, model1, mocks.AccessListEntry1Model) + test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) + case trx5CID.String(): + test_helpers.ExpectEqual(t, data, tx5) + var txType *uint8 + err = sqlxdb.Get(&txType, txTypePgStr, c) + if err != nil { + t.Fatal(err) + } + if *txType != types.DynamicFeeTxType { + t.Fatalf("expected DynamicFeeTxType (2), got %d", *txType) + } + } + } + }) + + t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + + rcts := make([]string, 0) + pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 + ORDER BY transaction_cids.index` + err = sqlxdb.Select(&rcts, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + + type logIPLD struct { + Index int `db:"index"` + Address string `db:"address"` + Data []byte `db:"data"` + Topic0 string `db:"topic0"` + Topic1 string `db:"topic1"` + } + for i := range rcts { + 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.rct_id = receipt_cids.tx_id) + INNER JOIN public.blocks ON (log_cids.leaf_mh_key = blocks.key) + WHERE receipt_cids.leaf_cid = $1 ORDER BY eth.log_cids.index ASC` + err = sqlxdb.Select(&results, pgStr, rcts[i]) + require.NoError(t, err) + + // expecting MockLog1 and MockLog2 for mockReceipt4 + expectedLogs := mocks.MockReceipts[i].Logs + test_helpers.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. + test_helpers.ExpectEqual(t, logRaw, nodeElements[1].([]byte)) + } + } + }) + + t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + + // check receipts were properly indexed + rcts := make([]string, 0) + pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids + WHERE receipt_cids.tx_id = transaction_cids.tx_hash + AND transaction_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 order by transaction_cids.index` + err = sqlxdb.Select(&rcts, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(rcts), 5) + + for idx, rctLeafCID := range rcts { + result := make([]models.IPLDModel, 0) + pgStr = `SELECT data + FROM eth.receipt_cids + INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) + WHERE receipt_cids.leaf_cid = $1` + err = sqlxdb.Select(&result, pgStr, rctLeafCID) + if err != nil { + t.Fatal(err) + } + + // Decode the log leaf node. + var nodeElements []interface{} + err = rlp.DecodeBytes(result[0].Data, &nodeElements) + require.NoError(t, err) + + expectedRct, err := mocks.MockReceipts[idx].MarshalBinary() + require.NoError(t, err) + + test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) + } + + // and published + for _, c := range rcts { + dc, err := cid.Decode(c) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + var data []byte + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` + switch c { + case rct1CID.String(): + test_helpers.ExpectEqual(t, data, rct1) + var postStatus uint64 + pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` + err = sqlxdb.Get(&postStatus, pgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) + case rct2CID.String(): + test_helpers.ExpectEqual(t, data, rct2) + var postState string + err = sqlxdb.Get(&postState, postStatePgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) + case rct3CID.String(): + test_helpers.ExpectEqual(t, data, rct3) + var postState string + err = sqlxdb.Get(&postState, postStatePgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) + case rct4CID.String(): + test_helpers.ExpectEqual(t, data, rct4) + var postState string + err = sqlxdb.Get(&postState, postStatePgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) + case rct5CID.String(): + test_helpers.ExpectEqual(t, data, rct5) + var postState string + err = sqlxdb.Get(&postState, postStatePgStr, c) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) + } + } + }) + + t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + + // check that state nodes were properly indexed and published + stateNodes := make([]models.StateNodeModel, 0) + pgStr := `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) + WHERE header_cids.block_number = $1 AND node_type != 3` + err = sqlxdb.Select(&stateNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(stateNodes), 2) + for _, stateNode := range stateNodes { + var data []byte + dc, err := cid.Decode(stateNode.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + pgStr = `SELECT * from eth.state_accounts WHERE header_id = $1 AND state_path = $2` + var account models.StateAccountModel + err = sqlxdb.Get(&account, pgStr, stateNode.HeaderID, stateNode.Path) + if err != nil { + t.Fatal(err) + } + if stateNode.CID == state1CID.String() { + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.ContractLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x06'}) + test_helpers.ExpectEqual(t, data, mocks.ContractLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ + HeaderID: account.HeaderID, + StatePath: stateNode.Path, + Balance: "0", + CodeHash: mocks.ContractCodeHash.Bytes(), + StorageRoot: mocks.ContractRoot, + Nonce: 1, + }) + } + if stateNode.CID == state2CID.String() { + test_helpers.ExpectEqual(t, stateNode.NodeType, 2) + test_helpers.ExpectEqual(t, stateNode.StateKey, common.BytesToHash(mocks.AccountLeafKey).Hex()) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x0c'}) + test_helpers.ExpectEqual(t, data, mocks.AccountLeafNode) + test_helpers.ExpectEqual(t, account, models.StateAccountModel{ + HeaderID: account.HeaderID, + StatePath: stateNode.Path, + Balance: "1000", + CodeHash: mocks.AccountCodeHash.Bytes(), + StorageRoot: mocks.AccountRoot, + Nonce: 0, + }) + } + } + + // check that Removed state nodes were properly indexed and published + stateNodes = make([]models.StateNodeModel, 0) + pgStr = `SELECT state_cids.cid, state_cids.state_leaf_key, state_cids.node_type, state_cids.state_path, state_cids.header_id + FROM eth.state_cids INNER JOIN eth.header_cids ON (state_cids.header_id = header_cids.block_hash) + WHERE header_cids.block_number = $1 AND node_type = 3` + err = sqlxdb.Select(&stateNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(stateNodes), 1) + stateNode := stateNodes[0] + var data []byte + dc, err := cid.Decode(stateNode.CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, stateNode.CID, shared.RemovedNodeStateCID) + test_helpers.ExpectEqual(t, stateNode.Path, []byte{'\x02'}) + test_helpers.ExpectEqual(t, data, []byte{}) + }) + + t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { + setup(t) + dumpData(t) + defer tearDown(t) + + // check that storage nodes were properly indexed + storageNodes := make([]models.StorageNodeWithStateKeyModel, 0) + pgStr := `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path + FROM eth.storage_cids, eth.state_cids, eth.header_cids + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 + AND storage_cids.node_type != 3` + err = sqlxdb.Select(&storageNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + CID: storageCID.String(), + NodeType: 2, + StorageKey: common.BytesToHash(mocks.StorageLeafKey).Hex(), + StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), + Path: []byte{}, + }) + var data []byte + dc, err := cid.Decode(storageNodes[0].CID) + if err != nil { + t.Fatal(err) + } + mhKey := dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey := blockstore.BlockPrefix.String() + mhKey.String() + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, mocks.StorageLeafNode) + + // check that Removed storage nodes were properly indexed + storageNodes = make([]models.StorageNodeWithStateKeyModel, 0) + pgStr = `SELECT storage_cids.cid, state_cids.state_leaf_key, storage_cids.storage_leaf_key, storage_cids.node_type, storage_cids.storage_path + FROM eth.storage_cids, eth.state_cids, eth.header_cids + WHERE (storage_cids.state_path, storage_cids.header_id) = (state_cids.state_path, state_cids.header_id) + AND state_cids.header_id = header_cids.block_hash + AND header_cids.block_number = $1 + AND storage_cids.node_type = 3` + err = sqlxdb.Select(&storageNodes, pgStr, mocks.BlockNumber.Uint64()) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, len(storageNodes), 1) + test_helpers.ExpectEqual(t, storageNodes[0], models.StorageNodeWithStateKeyModel{ + CID: shared.RemovedNodeStorageCID, + NodeType: 3, + StorageKey: common.BytesToHash(mocks.RemovedLeafKey).Hex(), + StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), + Path: []byte{'\x03'}, + }) + dc, err = cid.Decode(storageNodes[0].CID) + if err != nil { + t.Fatal(err) + } + mhKey = dshelp.MultihashToDsKey(dc.Hash()) + prefixedKey = blockstore.BlockPrefix.String() + mhKey.String() + test_helpers.ExpectEqual(t, prefixedKey, shared.RemovedNodeMhKey) + err = sqlxdb.Get(&data, ipfsPgGet, prefixedKey) + if err != nil { + t.Fatal(err) + } + test_helpers.ExpectEqual(t, data, []byte{}) + }) +} diff --git a/statediff/indexer/database/file/test_helpers.go b/statediff/indexer/database/file/test_helpers.go new file mode 100644 index 000000000..156a6b0d2 --- /dev/null +++ b/statediff/indexer/database/file/test_helpers.go @@ -0,0 +1,28 @@ +package file + +import "io" + +type MemWriter struct { + bytes []byte +} + +func NewMemWriter() io.WriteCloser { + return &MemWriter{} +} + +// Write satisfies io.WriteCloser +func (mw *MemWriter) Write(b []byte) (int, error) { + mw.bytes = append(mw.bytes, b...) + return len(b), nil +} + +// Close satisfies io.WriteCloser +func (mw *MemWriter) Close() error { + mw.bytes = []byte{} + return nil +} + +// ReadAll returns all the bytes written to the memory writer +func (mw *MemWriter) ReadAll() []byte { + return mw.bytes +} diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index fdfa87b08..90d21ba73 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -18,7 +18,7 @@ package file import ( "fmt" - "os" + "io" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" @@ -38,7 +38,7 @@ var ( // SQLWriter writes sql statements to a file type SQLWriter struct { - file *os.File + wc io.WriteCloser stmts chan []byte collatedStmt []byte collationIndex int @@ -50,9 +50,9 @@ type SQLWriter struct { } // NewSQLWriter creates a new pointer to a Writer -func NewSQLWriter(file *os.File) *SQLWriter { +func NewSQLWriter(wc io.WriteCloser) *SQLWriter { return &SQLWriter{ - file: file, + wc: wc, stmts: make(chan []byte), collatedStmt: make([]byte, collatedStmtSize), flushChan: make(chan struct{}), @@ -100,7 +100,7 @@ func (sqw *SQLWriter) Loop() { func (sqw *SQLWriter) Close() error { close(sqw.quitChan) <-sqw.doneChan - return nil + return sqw.wc.Close() } // Flush sends a flush signal to the looping process @@ -110,7 +110,7 @@ func (sqw *SQLWriter) Flush() { } func (sqw *SQLWriter) flush() error { - if _, err := sqw.file.Write(sqw.collatedStmt[0:sqw.collationIndex]); err != nil { + if _, err := sqw.wc.Write(sqw.collatedStmt[0:sqw.collationIndex]); err != nil { return err } sqw.collationIndex = 0 diff --git a/statediff/indexer/database/sql/batch_writer.go b/statediff/indexer/database/sql/batch_writer.go deleted file mode 100644 index f186d8052..000000000 --- a/statediff/indexer/database/sql/batch_writer.go +++ /dev/null @@ -1,216 +0,0 @@ -// VulcanizeDB -// Copyright © 2021 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 . - -package sql - -/* -import ( - "fmt" - - "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" - - "github.com/ethereum/go-ethereum/statediff/indexer/models" - "github.com/jmoiron/sqlx" -) - -*/ -/* -// PG_MAX_PARAMS is the max number of placeholders+args a statement can support -// above this limit we need to split into a separate batch -const PG_MAX_PARAMS int = 32767 - -const ( - ipldInsertPgStr string = `INSERT INTO public.blocks (key, data) VALUES (unnest($1), unnest($2)) ON CONFLICT (key) DO NOTHING` - headerCIDsPgStr string = `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) - VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9), unnest($10), unnest($11), unnest($12), unnest($13), unnest($14), unnest($15), unnest($16)) - ON CONFLICT (block_number, block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = (excluded.parent_hash, excluded.cid, excluded.td, excluded.node_id, excluded.reward, excluded.state_root, excluded.tx_root, excluded.receipt_root, excluded.uncle_root, excluded.bloom, excluded.timestamp, excluded.mh_key, eth.header_cids.times_validated + 1, excluded.base_fee) - RETURNING id` - unclesCIDsPgStr string = `INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6)) - ON CONFLICT (header_id, block_hash) DO UPDATE SET (parent_hash, cid, reward, mh_key) = (excluded.parent_hash, excluded.cid, excluded.reward, excluded.mh_key)` - txCIDsPgStr string = `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9)) - ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = (excluded.cid, excluded.dst, excluded.src, excluded.index, excluded.mh_key, excluded.tx_data, excluded.tx_type) - RETURNING id` - accessListPgStr string = `INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES (unnest($1), unnest($2), unnest($3), unnest($4)) - ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = (excluded.address, excluded.storage_keys)` - rctCIDsPgStr string = `INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8)) - ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = (excluded.leaf_cid, excluded.contract, excluded.contract_hash, excluded.leaf_mh_key, excluded.post_state, excluded.post_status, excluded.log_root) - RETURNING id` - logCIDsPgStr string = `INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, receipt_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7), unnest($8), unnest($9), unnest($10)) - ON CONFLICT (receipt_id, index) DO UPDATE SET (leaf_cid, leaf_mh_key, address, topic0, topic1, topic2, topic3, log_data) = (excluded.leaf_cid, excluded.leaf_mh_key, excluded.address, excluded.topic0, excluded.topic1, excluded.topic2, excluded.topic3, excluded.log_data)` - stateCIDsPgStr string = `INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7)) - ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = (excluded.state_leaf_key, excluded.cid, excluded.node_type, excluded.diff, excluded.mh_key) - RETURNING id` - stateAccountsPgStr string = `INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5)) - ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = (excluded.balance, excluded.nonce, excluded.code_hash, excluded.storage_root)` - storageCIDsPgStr string = `INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES (unnest($1), unnest($2), unnest($3), unnest($4), unnest($5), unnest($6), unnest($7)) - ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = (excluded.storage_leaf_key, excluded.cid, excluded.node_type, excluded.diff, excluded.mh_key)` -) - -// PostgresBatchWriter is used to write statediff data to Postgres using batch inserts/upserts -type PostgresBatchWriter struct { - db *postgres.DB - - // prepared statements (prepared inside tx) - ipldsPreparedStm *sqlx.Stmt - unclesPrepared *sqlx.Stmt - txPreparedStm *sqlx.Stmt - accessListPreparedStm *sqlx.Stmt - rctPreparedStm *sqlx.Stmt - logPreparedStm *sqlx.Stmt - statePreparedStm *sqlx.Stmt - accountPreparedStm *sqlx.Stmt - storagePreparedStm *sqlx.Stmt - - // cached arguments - queuedHeaderArgs models.HeaderModel - queuedUnclesArgs models.UncleBatch - queuedTxArgs models.TxBatch - queuedAccessListArgs models.AccessListBatch - queuedRctArgs models.ReceiptBatch - queuedLogArgs models.LogBatch - queuedStateArgs models.StateBatch - queuedAccountArgs models.AccountBatch - queuedStorageArgs models.StorageBatch -} - -// NewPostgresBatchWriter creates a new pointer to a PostgresBatchWriter -func NewPostgresBatchWriter(db *postgres.DB) *PostgresBatchWriter { - return &PostgresBatchWriter{ - db: db, - } -} - -func (pbw *PostgresBatchWriter) queueHeader(header models.HeaderModel) { - pbw.queuedHeaderArgs = header -} - -func (pbw *PostgresBatchWriter) queueUncle(uncle models.UncleModel) { - pbw.queuedUnclesArgs.BlockHashes = append(pbw.queuedUnclesArgs.BlockHashes, uncle.BlockHash) - pbw.queuedUnclesArgs.ParentHashes = append(pbw.queuedUnclesArgs.ParentHashes, uncle.ParentHash) - pbw.queuedUnclesArgs.CIDs = append(pbw.queuedUnclesArgs.CIDs, uncle.CID) - pbw.queuedUnclesArgs.MhKeys = append(pbw.queuedUnclesArgs.MhKeys, uncle.MhKey) - pbw.queuedUnclesArgs.Rewards = append(pbw.queuedUnclesArgs.Rewards, uncle.Reward) -} - -func (pbw *PostgresBatchWriter) queueTransaction(tx models.TxModel) { - pbw.queuedTxArgs.Indexes = append(pbw.queuedTxArgs.Indexes, tx.Index) - pbw.queuedTxArgs.TxHashes = append(pbw.queuedTxArgs.TxHashes, tx.TxHash) - pbw.queuedTxArgs.CIDs = append(pbw.queuedTxArgs.CIDs, tx.CID) - pbw.queuedTxArgs.MhKeys = append(pbw.queuedTxArgs.MhKeys, tx.MhKey) - pbw.queuedTxArgs.Dsts = append(pbw.queuedTxArgs.Dsts, tx.Dst) - pbw.queuedTxArgs.Srcs = append(pbw.queuedTxArgs.Srcs, tx.Src) - pbw.queuedTxArgs.Datas = append(pbw.queuedTxArgs.Datas, tx.Data) - pbw.queuedTxArgs.Types = append(pbw.queuedTxArgs.Types, tx.Type) -} - -func (pbw *PostgresBatchWriter) queueAccessListElement(al models.AccessListElementModel) { - -} - -func (pbw *PostgresBatchWriter) queueReceipt(rct models.ReceiptModel) { - -} - -func (pbw *PostgresBatchWriter) upsertTransactionCID(tx *sqlx.Tx, transaction models.TxModel, headerID int64) (int64, error) { - var txID int64 - err := tx.QueryRowx(`INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) - ON CONFLICT (header_id, tx_hash) DO UPDATE SET (cid, dst, src, index, mh_key, tx_data, tx_type) = ($3, $4, $5, $6, $7, $8, $9) - RETURNING id`, - headerID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type).Scan(&txID) - if err != nil { - return 0, fmt.Errorf("error upserting transaction_cids entry: %v", err) - } - indexerMetrics.transactions.Inc(1) - return txID, nil -} - -func (pbw *PostgresBatchWriter) upsertAccessListElement(tx *sqlx.Tx, accessListElement models.AccessListElementModel, txID int64) error { - _, err := tx.Exec(`INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) - ON CONFLICT (tx_id, index) DO UPDATE SET (address, storage_keys) = ($3, $4)`, - txID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) - if err != nil { - return fmt.Errorf("error upserting access_list_element entry: %v", err) - } - indexerMetrics.accessListEntries.Inc(1) - return nil -} - -func (pbw *PostgresBatchWriter) upsertReceiptCID(tx *sqlx.Tx, rct *models.ReceiptModel, txID int64) (int64, error) { - var receiptID int64 - err := tx.QueryRowx(`INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) - ON CONFLICT (tx_id) DO UPDATE SET (leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) = ($2, $3, $4, $5, $6, $7, $8) - RETURNING id`, - txID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot).Scan(&receiptID) - if err != nil { - return 0, fmt.Errorf("error upserting receipt_cids entry: %w", err) - } - indexerMetrics.receipts.Inc(1) - return receiptID, nil -} - -func (pbw *PostgresBatchWriter) 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) - } - indexerMetrics.logs.Inc(1) - } - return nil -} - -func (pbw *PostgresBatchWriter) upsertStateCID(tx *sqlx.Tx, stateNode models.StateNodeModel, headerID int64) (int64, error) { - var stateID int64 - var stateKey string - if stateNode.StateKey != nullHash.String() { - stateKey = stateNode.StateKey - } - err := tx.QueryRowx(`INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) - RETURNING id`, - headerID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey).Scan(&stateID) - if err != nil { - return 0, fmt.Errorf("error upserting state_cids entry: %v", err) - } - return stateID, nil -} - -func (pbw *PostgresBatchWriter) upsertStateAccount(tx *sqlx.Tx, stateAccount models.StateAccountModel, stateID int64) error { - _, err := tx.Exec(`INSERT INTO eth.state_accounts (state_id, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5) - ON CONFLICT (state_id) DO UPDATE SET (balance, nonce, code_hash, storage_root) = ($2, $3, $4, $5)`, - stateID, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) - if err != nil { - return fmt.Errorf("error upserting state_accounts entry: %v", err) - } - return nil -} - -func (pbw *PostgresBatchWriter) upsertStorageCID(tx *sqlx.Tx, storageCID models.StorageNodeModel, stateID int64) error { - var storageKey string - if storageCID.StorageKey != nullHash.String() { - storageKey = storageCID.StorageKey - } - _, err := tx.Exec(`INSERT INTO eth.storage_cids (state_id, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) - ON CONFLICT (state_id, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7)`, - stateID, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) - if err != nil { - return fmt.Errorf("error upserting storage_cids entry: %v", err) - } - return nil -} -*/ diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index b557ec903..8eb6ed9de 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -541,5 +541,5 @@ func (sdi *StateDiffIndexer) PushCodeAndCodeHash(batch interfaces.Batch, codeAnd // Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { - return sdi.dbWriter.db.Close() + return sdi.dbWriter.Close() } diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index 710ad23d9..1b49151a4 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -140,9 +140,7 @@ func setupPGX(t *testing.T) { }() for _, node := range mocks.StateDiffs { err = ind.PushStateNode(tx, node, mockBlock.Hash().String()) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) } test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index a7c7cc9b4..5794bd0af 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -49,9 +49,9 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_test", + DatabaseName: "vulcanize_testing", Username: "postgres", - Password: "", + Password: "password", } // Config holds params for a Postgres db diff --git a/statediff/indexer/database/sql/postgres/pgx_test.go b/statediff/indexer/database/sql/postgres/pgx_test.go index ea66737f5..c4e818029 100644 --- a/statediff/indexer/database/sql/postgres/pgx_test.go +++ b/statediff/indexer/database/sql/postgres/pgx_test.go @@ -111,7 +111,8 @@ func TestPostgresPGX(t *testing.T) { badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} - _, err := postgres.NewPGXDriver(ctx, postgres.DefaultConfig, badInfo) + d, err := postgres.NewPGXDriver(ctx, postgres.DefaultConfig, badInfo) + defer d.Close() if err == nil { t.Fatal("Expected an error") } diff --git a/statediff/indexer/database/sql/postgres/sqlx_test.go b/statediff/indexer/database/sql/postgres/sqlx_test.go index 95975a868..7476b99e9 100644 --- a/statediff/indexer/database/sql/postgres/sqlx_test.go +++ b/statediff/indexer/database/sql/postgres/sqlx_test.go @@ -39,11 +39,15 @@ func TestPostgresSQLX(t *testing.T) { sqlxdb, err = sqlx.Connect("postgres", connStr) if err != nil { - t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err) + t.Fatalf("failed to connect to db with connection string: %s err: %v", connStr, err) } if sqlxdb == nil { t.Fatal("DB is nil") } + err = sqlxdb.Close() + if err != nil { + t.Fatal(err) + } }) t.Run("serializes big.Int to db", func(t *testing.T) { @@ -59,9 +63,7 @@ func TestPostgresSQLX(t *testing.T) { if err != nil { t.Fatal(err) } - if err != nil { - t.Fatal(err) - } + defer db.Close() bi := new(big.Int) bi.SetString("34940183920000000000", 10) @@ -107,11 +109,15 @@ func TestPostgresSQLX(t *testing.T) { badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} - _, err := postgres.NewSQLXDriver(ctx, postgres.DefaultConfig, badInfo) + d, err := postgres.NewSQLXDriver(ctx, postgres.DefaultConfig, badInfo) if err == nil { t.Fatal("Expected an error") } expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg) + err = d.Close() + if err != nil { + t.Fatal(err) + } }) } diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index e0b5f2967..b60b44a82 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -160,9 +160,7 @@ func setupSQLX(t *testing.T) { }() for _, node := range mocks.StateDiffs { err = ind.PushStateNode(tx, node, mockBlock.Hash().String()) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) } test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go index 3089b6d50..f426263e0 100644 --- a/statediff/indexer/database/sql/writer.go +++ b/statediff/indexer/database/sql/writer.go @@ -39,14 +39,19 @@ func NewWriter(db Database) *Writer { } } +// Close satisfies io.Closer +func (w *Writer) Close() error { + return w.db.Close() +} + /* INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) ON CONFLICT (block_hash) DO UPDATE SET (block_number, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($1, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) */ -func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertHeaderStm(), - header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, in.db.NodeID(), header.Reward, header.StateRoot, header.TxRoot, +func (w *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertHeaderStm(), + header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, w.db.NodeID(), header.Reward, header.StateRoot, header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee) if err != nil { return fmt.Errorf("error upserting header_cids entry: %v", err) @@ -59,8 +64,8 @@ func (in *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES ($1, $2, $3, $4, $5, $6) ON CONFLICT (block_hash) DO NOTHING */ -func (in *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertUncleStm(), +func (w *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertUncleStm(), uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey) if err != nil { return fmt.Errorf("error upserting uncle_cids entry: %v", err) @@ -72,8 +77,8 @@ func (in *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error { INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) ON CONFLICT (tx_hash) DO NOTHING */ -func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertTxStm(), +func (w *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertTxStm(), transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type) if err != nil { return fmt.Errorf("error upserting transaction_cids entry: %v", err) @@ -86,8 +91,8 @@ func (in *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ($1, $2, $3, $4) ON CONFLICT (tx_id, index) DO NOTHING */ -func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertAccessListElementStm(), +func (w *Writer) upsertAccessListElement(tx Tx, accessListElement models.AccessListElementModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertAccessListElementStm(), accessListElement.TxID, accessListElement.Index, accessListElement.Address, accessListElement.StorageKeys) if err != nil { return fmt.Errorf("error upserting access_list_element entry: %v", err) @@ -100,8 +105,8 @@ func (in *Writer) upsertAccessListElement(tx Tx, accessListElement models.Access INSERT INTO eth.receipt_cids (tx_id, leaf_cid, contract, contract_hash, leaf_mh_key, post_state, post_status, log_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) ON CONFLICT (tx_id) DO NOTHING */ -func (in *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertRctStm(), +func (w *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertRctStm(), rct.TxID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot) if err != nil { return fmt.Errorf("error upserting receipt_cids entry: %w", err) @@ -114,9 +119,9 @@ func (in *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel) error { INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, topic3, log_data) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10) ON CONFLICT (rct_id, index) DO NOTHING */ -func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error { +func (w *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error { for _, log := range logs { - _, err := tx.Exec(in.db.Context(), in.db.InsertLogStm(), + _, err := tx.Exec(w.db.Context(), w.db.InsertLogStm(), log.LeafCID, log.LeafMhKey, log.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) @@ -130,12 +135,12 @@ func (in *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error { INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7) ON CONFLICT (header_id, state_path) DO UPDATE SET (state_leaf_key, cid, node_type, diff, mh_key) = ($2, $3, $5, $6, $7) */ -func (in *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel) error { +func (w *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel) error { var stateKey string if stateNode.StateKey != nullHash.String() { stateKey = stateNode.StateKey } - _, err := tx.Exec(in.db.Context(), in.db.InsertStateStm(), + _, err := tx.Exec(w.db.Context(), w.db.InsertStateStm(), stateNode.HeaderID, stateKey, stateNode.CID, stateNode.Path, stateNode.NodeType, true, stateNode.MhKey) if err != nil { return fmt.Errorf("error upserting state_cids entry: %v", err) @@ -147,8 +152,8 @@ func (in *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel) error { INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5, $6) ON CONFLICT (header_id, state_path) DO NOTHING */ -func (in *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel) error { - _, err := tx.Exec(in.db.Context(), in.db.InsertAccountStm(), +func (w *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel) error { + _, err := tx.Exec(w.db.Context(), w.db.InsertAccountStm(), stateAccount.HeaderID, stateAccount.StatePath, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) if err != nil { return fmt.Errorf("error upserting state_accounts entry: %v", err) @@ -160,12 +165,12 @@ func (in *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountMode INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, node_type, diff, mh_key) VALUES ($1, $2, $3, $4, $5, $6, $7, $8) ON CONFLICT (header_id, state_path, storage_path) DO UPDATE SET (storage_leaf_key, cid, node_type, diff, mh_key) = ($3, $4, $6, $7, $8) */ -func (in *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel) error { +func (w *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel) error { var storageKey string if storageCID.StorageKey != nullHash.String() { storageKey = storageCID.StorageKey } - _, err := tx.Exec(in.db.Context(), in.db.InsertStorageStm(), + _, err := tx.Exec(w.db.Context(), w.db.InsertStorageStm(), storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) if err != nil { return fmt.Errorf("error upserting storage_cids entry: %v", err) -- 2.45.2 From b9a82f6350ae1f27b7dd9aa191e1bd2c28f29f5c Mon Sep 17 00:00:00 2001 From: i-norden Date: Sat, 20 Nov 2021 13:51:54 -0600 Subject: [PATCH 15/33] basefee is big.Int, it won't always fit in int64 --- statediff/indexer/database/dump/indexer.go | 6 +- statediff/indexer/database/file/indexer.go | 6 +- .../database/file/indexer_legacy_test.go | 31 +++---- .../indexer/database/file/indexer_test.go | 32 ++++---- .../indexer/database/file/test_helpers.go | 80 ++++++++++++++----- statediff/indexer/database/file/writer.go | 18 ++--- statediff/indexer/database/sql/indexer.go | 6 +- .../database/sql/pgx_indexer_legacy_test.go | 2 +- .../indexer/database/sql/pgx_indexer_test.go | 8 +- .../indexer/database/sql/postgres/pgx_test.go | 5 +- .../database/sql/postgres/sqlx_test.go | 6 +- .../database/sql/sqlx_indexer_legacy_test.go | 2 +- .../indexer/database/sql/sqlx_indexer_test.go | 6 +- .../indexer/database/sql/test_helpers.go | 8 ++ statediff/indexer/mocks/test_data.go | 6 +- statediff/indexer/models/models.go | 32 ++++---- 16 files changed, 148 insertions(+), 106 deletions(-) diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index b75fb1af9..faad44e85 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -184,10 +184,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (string, error) { tx.cacheIPLD(headerNode) - var baseFee *int64 + var baseFee *string if header.BaseFee != nil { - baseFee = new(int64) - *baseFee = header.BaseFee.Int64() + baseFee = new(string) + *baseFee = header.BaseFee.String() } headerID := header.Hash().String() diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index e945f155d..5c2bbcb52 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -191,10 +191,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node.Node, reward, td *big.Int) string { sdi.fileWriter.upsertIPLDNode(headerNode) - var baseFee *int64 + var baseFee *string if header.BaseFee != nil { - baseFee = new(int64) - *baseFee = header.BaseFee.Int64() + baseFee = new(string) + *baseFee = header.BaseFee.String() } headerID := header.Hash().String() sdi.fileWriter.upsertHeaderCID(models.HeaderModel{ diff --git a/statediff/indexer/database/file/indexer_legacy_test.go b/statediff/indexer/database/file/indexer_legacy_test.go index d02927165..9259f4c13 100644 --- a/statediff/indexer/database/file/indexer_legacy_test.go +++ b/statediff/indexer/database/file/indexer_legacy_test.go @@ -18,22 +18,21 @@ package file_test import ( "context" + "errors" "os" "testing" - "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/statediff/indexer/database/file" - "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ipfs/go-cid" "github.com/jmoiron/sqlx" - "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" - "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) @@ -46,7 +45,10 @@ var ( func setupLegacy(t *testing.T) { mockLegacyBlock = legacyData.MockBlock legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256) - + if _, err := os.Stat(file.TestConfig.FilePath); !errors.Is(err, os.ErrNotExist) { + err := os.Remove(file.TestConfig.FilePath) + require.NoError(t, err) + } ind, err := file.NewStateDiffIndexer(context.Background(), legacyData.Config, file.TestConfig) require.NoError(t, err) var tx interfaces.Batch @@ -69,7 +71,7 @@ func setupLegacy(t *testing.T) { require.NoError(t, err) } - test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, legacyData.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.(*file.BatchTx).BlockNumber, legacyData.BlockNumber.Uint64()) connStr := postgres.DefaultConfig.DbConnectionString() @@ -88,6 +90,7 @@ func dumpData(t *testing.T) { } func tearDown(t *testing.T) { + file.TearDownDB(t, sqlxdb) err := os.Remove(file.TestConfig.FilePath) require.NoError(t, err) err = sqlxdb.Close() @@ -100,12 +103,12 @@ func expectTrue(t *testing.T, value bool) { } } -func TestFIleIndexerLegacy(t *testing.T) { - t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { +func TestFileIndexerLegacy(t *testing.T) { + t.Run("Publish and index header IPLDs", func(t *testing.T) { setupLegacy(t) dumpData(t) defer tearDown(t) - pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee + pgStr := `SELECT cid, td, reward, block_hash, base_fee FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -113,13 +116,11 @@ func TestFIleIndexerLegacy(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + BlockHash string `db:"block_hash"` + BaseFee *string `db:"base_fee"` } header := new(res) - - err = sqlxdb.QueryRow(pgStr, legacyData.BlockNumber.Uint64()).Scan( - &header.CID, &header.TD, &header.Reward, &header.BlockHash, &header.BaseFee) + err = sqlxdb.QueryRowx(pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) require.NoError(t, err) test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) diff --git a/statediff/indexer/database/file/indexer_test.go b/statediff/indexer/database/file/indexer_test.go index 06efce309..b0489fc08 100644 --- a/statediff/indexer/database/file/indexer_test.go +++ b/statediff/indexer/database/file/indexer_test.go @@ -19,29 +19,29 @@ package file_test import ( "bytes" "context" + "errors" "fmt" "os" "testing" - "github.com/ethereum/go-ethereum/statediff/indexer/database/file" - "github.com/jmoiron/sqlx" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/indexer/models" + "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/jmoiron/sqlx" "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" - "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/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/mocks" - "github.com/ethereum/go-ethereum/statediff/indexer/models" - "github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) @@ -135,7 +135,11 @@ func init() { } func setup(t *testing.T) { - ind, err := file.NewStateDiffIndexer(context.Background(), legacyData.Config, file.TestConfig) + if _, err := os.Stat(file.TestConfig.FilePath); !errors.Is(err, os.ErrNotExist) { + err := os.Remove(file.TestConfig.FilePath) + require.NoError(t, err) + } + ind, err = file.NewStateDiffIndexer(context.Background(), mocks.TestConfig, file.TestConfig) require.NoError(t, err) var tx interfaces.Batch tx, err = ind.PushBlock( @@ -158,7 +162,7 @@ func setup(t *testing.T) { require.NoError(t, err) } - test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) + test_helpers.ExpectEqual(t, tx.(*file.BatchTx).BlockNumber, mocks.BlockNumber.Uint64()) connStr := postgres.DefaultConfig.DbConnectionString() @@ -168,7 +172,7 @@ func setup(t *testing.T) { } } -func TestSQLXIndexer(t *testing.T) { +func TestFileIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setup(t) dumpData(t) @@ -181,18 +185,19 @@ func TestSQLXIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + BlockHash string `db:"block_hash"` + BaseFee *string `db:"base_fee"` } header := new(res) err = sqlxdb.QueryRowx(pgStr, mocks.BlockNumber.Uint64()).StructScan(header) if err != nil { t.Fatal(err) } + test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) @@ -206,7 +211,6 @@ func TestSQLXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, data, mocks.MockHeaderRlp) }) - t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { setup(t) dumpData(t) diff --git a/statediff/indexer/database/file/test_helpers.go b/statediff/indexer/database/file/test_helpers.go index 156a6b0d2..27d204d55 100644 --- a/statediff/indexer/database/file/test_helpers.go +++ b/statediff/indexer/database/file/test_helpers.go @@ -1,28 +1,64 @@ package file -import "io" +import ( + "testing" -type MemWriter struct { - bytes []byte -} + "github.com/jmoiron/sqlx" +) -func NewMemWriter() io.WriteCloser { - return &MemWriter{} -} +// TearDownDB is used to tear down the watcher dbs after tests +func TearDownDB(t *testing.T, db *sqlx.DB) { + tx, err := db.Begin() + if err != nil { + t.Fatal(err) + } -// Write satisfies io.WriteCloser -func (mw *MemWriter) Write(b []byte) (int, error) { - mw.bytes = append(mw.bytes, b...) - return len(b), nil -} - -// Close satisfies io.WriteCloser -func (mw *MemWriter) Close() error { - mw.bytes = []byte{} - return nil -} - -// ReadAll returns all the bytes written to the memory writer -func (mw *MemWriter) ReadAll() []byte { - return mw.bytes + _, err = tx.Exec(`DELETE FROM eth.header_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.uncle_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.transaction_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.receipt_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.state_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.storage_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.state_accounts`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.access_list_elements`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM eth.log_cids`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM blocks`) + if err != nil { + t.Fatal(err) + } + _, err = tx.Exec(`DELETE FROM nodes`) + if err != nil { + t.Fatal(err) + } + err = tx.Commit() + if err != nil { + t.Fatal(err) + } } diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 90d21ba73..6329ecae1 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -121,21 +121,21 @@ const ( nodeInsert = "INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES " + "('%s', '%s', '%s', '%s', %d);\n" - ipldInsert = "INSERT INTO public.blocks (key, data) VALUES ('%s', '%x');\n" + ipldInsert = "INSERT INTO public.blocks (key, data) VALUES ('%s', '\\x%x');\n" headerInsert = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, " + "state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + - "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%x', %d, '%s', %d, %d);\n" + "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s', %d, %s);\n" headerInsertWithoutBaseFee = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, " + "reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + - "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%x', %d, '%s', %d, NULL);\n" + "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s', %d, NULL);\n" uncleInsert = "INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES " + "('%s', '%s', '%s', '%s', '%s', '%s');\n" txInsert = "INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) " + - "VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '%x', %d);\n" + "VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '\\x%x', %d);\n" alInsert = "INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ('%s', %d, '%s', '%s');\n" @@ -143,16 +143,16 @@ const ( "post_status, log_root) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', %d, '%s');\n" logInsert = "INSERT INTO eth.log_cids (leaf_cid, leaf_mh_key, rct_id, address, index, topic0, topic1, topic2, " + - "topic3, log_data) VALUES ('%s', '%s', '%s', '%s', %d, '%s', '%s', '%s', '%s', '%x');\n" + "topic3, log_data) VALUES ('%s', '%s', '%s', '%s', %d, '%s', '%s', '%s', '%s', '\\x%x');\n" stateInsert = "INSERT INTO eth.state_cids (header_id, state_leaf_key, cid, state_path, node_type, diff, mh_key) " + - "VALUES ('%s', '%s', '%s', '%x', %d, %t, '%s');\n" + "VALUES ('%s', '%s', '%s', '\\x%x', %d, %t, '%s');\n" accountInsert = "INSERT INTO eth.state_accounts (header_id, state_path, balance, nonce, code_hash, storage_root) " + - "VALUES ('%s', '%x', '%s', %d, '%x', '%s');\n" + "VALUES ('%s', '\\x%x', '%s', %d, '\\x%x', '%s');\n" storageInsert = "INSERT INTO eth.storage_cids (header_id, state_path, storage_leaf_key, cid, storage_path, " + - "node_type, diff, mh_key) VALUES ('%s', '%x', '%s', '%s', '%x', %d, %t, '%s');\n" + "node_type, diff, mh_key) VALUES ('%s', '\\x%x', '%s', '%s', '\\x%x', %d, %t, '%s');\n" ) func (sqw *SQLWriter) upsertNode(node nodeinfo.Info) { @@ -199,7 +199,7 @@ func (sqw *SQLWriter) upsertHeaderCID(header models.HeaderModel) { } else { stmt = fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee) + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, *header.BaseFee) } sqw.stmts <- []byte(stmt) indexerMetrics.blocks.Inc(1) diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index 8eb6ed9de..d5f7c3660 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -235,10 +235,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (string, error) { tx.cacheIPLD(headerNode) - var baseFee *int64 + var baseFee *string if header.BaseFee != nil { - baseFee = new(int64) - *baseFee = header.BaseFee.Int64() + baseFee = new(string) + *baseFee = header.BaseFee.String() } headerID := header.Hash().String() // index header diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go index 21b74b3b2..7dc38a3ca 100644 --- a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -60,7 +60,7 @@ func setupLegacyPGX(t *testing.T) { } func TestPGXIndexerLegacy(t *testing.T) { - t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { + t.Run("Publish and index header IPLDs", func(t *testing.T) { setupLegacyPGX(t) defer tearDown(t) pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index 1b49151a4..a378424db 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -150,7 +150,7 @@ func TestPGXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupPGX(t) defer tearDown(t) - pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, cast(base_fee AS TEXT) FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -158,8 +158,8 @@ func TestPGXIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + BlockHash string `db:"block_hash"` + BaseFee *string `db:"base_fee"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).Scan( @@ -174,7 +174,7 @@ func TestPGXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/postgres/pgx_test.go b/statediff/indexer/database/sql/postgres/pgx_test.go index c4e818029..64616e356 100644 --- a/statediff/indexer/database/sql/postgres/pgx_test.go +++ b/statediff/indexer/database/sql/postgres/pgx_test.go @@ -47,10 +47,10 @@ func TestPostgresPGX(t *testing.T) { if err != nil { t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err) } - defer dbPool.Close() if dbPool == nil { t.Fatal("DB pool is nil") } + dbPool.Close() }) t.Run("serializes big.Int to db", func(t *testing.T) { @@ -111,8 +111,7 @@ func TestPostgresPGX(t *testing.T) { badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} - d, err := postgres.NewPGXDriver(ctx, postgres.DefaultConfig, badInfo) - defer d.Close() + _, err := postgres.NewPGXDriver(ctx, postgres.DefaultConfig, badInfo) if err == nil { t.Fatal("Expected an error") } diff --git a/statediff/indexer/database/sql/postgres/sqlx_test.go b/statediff/indexer/database/sql/postgres/sqlx_test.go index 7476b99e9..03f24e9f5 100644 --- a/statediff/indexer/database/sql/postgres/sqlx_test.go +++ b/statediff/indexer/database/sql/postgres/sqlx_test.go @@ -109,15 +109,11 @@ func TestPostgresSQLX(t *testing.T) { badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100)) badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"} - d, err := postgres.NewSQLXDriver(ctx, postgres.DefaultConfig, badInfo) + _, err := postgres.NewSQLXDriver(ctx, postgres.DefaultConfig, badInfo) if err == nil { t.Fatal("Expected an error") } expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg) - err = d.Close() - if err != nil { - t.Fatal(err) - } }) } diff --git a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go index 4349850ed..d0ed3568b 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go @@ -70,7 +70,7 @@ func setupLegacySQLX(t *testing.T) { } func TestSQLXIndexerLegacy(t *testing.T) { - t.Run("Publish and index header IPLDs in a legacy tx", func(t *testing.T) { + t.Run("Publish and index header IPLDs", func(t *testing.T) { setupLegacySQLX(t) defer tearDown(t) pgStr := `SELECT cid, td, reward, block_hash, base_fee diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index b60b44a82..68f06ecaa 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -185,8 +185,8 @@ func TestSQLXIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + BlockHash string `db:"block_hash"` + BaseFee *string `db:"base_fee"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) @@ -196,7 +196,7 @@ func TestSQLXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.Int64()) + test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/test_helpers.go b/statediff/indexer/database/sql/test_helpers.go index 46f9b766b..b1032f8ff 100644 --- a/statediff/indexer/database/sql/test_helpers.go +++ b/statediff/indexer/database/sql/test_helpers.go @@ -33,6 +33,10 @@ func TearDownDB(t *testing.T, db Database) { if err != nil { t.Fatal(err) } + _, err = tx.Exec(ctx, `DELETE FROM eth.uncle_cids`) + if err != nil { + t.Fatal(err) + } _, err = tx.Exec(ctx, `DELETE FROM eth.transaction_cids`) if err != nil { t.Fatal(err) @@ -65,6 +69,10 @@ func TearDownDB(t *testing.T, db Database) { if err != nil { t.Fatal(err) } + _, err = tx.Exec(ctx, `DELETE FROM nodes`) + if err != nil { + t.Fatal(err) + } err = tx.Commit(ctx) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/mocks/test_data.go b/statediff/indexer/mocks/test_data.go index f437dc8e4..be61edb87 100644 --- a/statediff/indexer/mocks/test_data.go +++ b/statediff/indexer/mocks/test_data.go @@ -22,18 +22,16 @@ import ( "crypto/rand" "math/big" - "github.com/ethereum/go-ethereum/statediff/indexer/models" - - "github.com/ethereum/go-ethereum/trie" - "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/ethereum/go-ethereum/statediff/test_helpers" sdtypes "github.com/ethereum/go-ethereum/statediff/types" + "github.com/ethereum/go-ethereum/trie" ) // Test variables diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index d37aa5449..e471d20ec 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -26,22 +26,22 @@ type IPLDModel struct { // HeaderModel is the db model for eth.header_cids type HeaderModel struct { - BlockNumber string `db:"block_number"` - BlockHash string `db:"block_hash"` - ParentHash string `db:"parent_hash"` - CID string `db:"cid"` - MhKey string `db:"mh_key"` - TotalDifficulty string `db:"td"` - NodeID string `db:"node_id"` - Reward string `db:"reward"` - StateRoot string `db:"state_root"` - UncleRoot string `db:"uncle_root"` - TxRoot string `db:"tx_root"` - RctRoot string `db:"receipt_root"` - Bloom []byte `db:"bloom"` - Timestamp uint64 `db:"timestamp"` - TimesValidated int64 `db:"times_validated"` - BaseFee *int64 `db:"base_fee"` + BlockNumber string `db:"block_number"` + BlockHash string `db:"block_hash"` + ParentHash string `db:"parent_hash"` + CID string `db:"cid"` + MhKey string `db:"mh_key"` + TotalDifficulty string `db:"td"` + NodeID string `db:"node_id"` + Reward string `db:"reward"` + StateRoot string `db:"state_root"` + UncleRoot string `db:"uncle_root"` + TxRoot string `db:"tx_root"` + RctRoot string `db:"receipt_root"` + Bloom []byte `db:"bloom"` + Timestamp uint64 `db:"timestamp"` + TimesValidated int64 `db:"times_validated"` + BaseFee *string `db:"base_fee"` } // UncleModel is the db model for eth.uncle_cids -- 2.45.2 From 6c285d6d1dac467d58e5232a2c62d67d4e15543b Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 26 Nov 2021 08:33:21 -0600 Subject: [PATCH 16/33] adjust for schema updates --- statediff/indexer/database/dump/indexer.go | 15 +++---- statediff/indexer/database/file/indexer.go | 9 ++++- .../database/file/indexer_legacy_test.go | 8 ++-- .../indexer/database/file/indexer_test.go | 8 ++-- statediff/indexer/database/file/writer.go | 40 ++++++++----------- statediff/indexer/database/sql/indexer.go | 9 ++++- .../database/sql/pgx_indexer_legacy_test.go | 8 ++-- .../indexer/database/sql/pgx_indexer_test.go | 10 ++--- .../indexer/database/sql/postgres/config.go | 4 +- .../indexer/database/sql/postgres/database.go | 6 +-- .../database/sql/sqlx_indexer_legacy_test.go | 6 +-- .../indexer/database/sql/sqlx_indexer_test.go | 8 ++-- statediff/indexer/database/sql/writer.go | 23 ++++++----- statediff/indexer/mocks/test_data.go | 2 + statediff/indexer/models/models.go | 33 +++++++-------- 15 files changed, 103 insertions(+), 86 deletions(-) diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index faad44e85..b2fc70d27 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -184,12 +184,6 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode node.Node, reward, td *big.Int) (string, error) { tx.cacheIPLD(headerNode) - var baseFee *string - if header.BaseFee != nil { - baseFee = new(string) - *baseFee = header.BaseFee.String() - } - headerID := header.Hash().String() mod := models.HeaderModel{ CID: headerNode.Cid().String(), @@ -205,7 +199,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he TxRoot: header.TxHash.String(), UncleRoot: header.UncleHash.String(), Timestamp: header.Time, - BaseFee: baseFee, + Coinbase: header.Coinbase.String(), } _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", mod) return headerID, err @@ -268,6 +262,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs // index tx trx := args.txs[i] trxID := trx.Hash().String() + + var val string + if trx.Value() != nil { + val = trx.Value().String() + } + // derive sender for the tx that corresponds with this receipt from, err := types.Sender(signer, trx) if err != nil { @@ -283,6 +283,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs CID: txNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(txNode.Cid()), Type: trx.Type(), + Value: val, } if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", txModel); err != nil { return err diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index 5c2bbcb52..a9f5dd0fb 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -212,7 +212,7 @@ func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode node TxRoot: header.TxHash.String(), UncleRoot: header.UncleHash.String(), Timestamp: header.Time, - BaseFee: baseFee, + Coinbase: header.Coinbase.String(), }) return headerID } @@ -269,6 +269,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { // index tx trx := args.txs[i] txID := trx.Hash().String() + + var val string + if trx.Value() != nil { + val = trx.Value().String() + } + // derive sender for the tx that corresponds with this receipt from, err := types.Sender(signer, trx) if err != nil { @@ -284,6 +290,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { CID: txNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(txNode.Cid()), Type: trx.Type(), + Value: val, } sdi.fileWriter.upsertTransactionCID(txModel) diff --git a/statediff/indexer/database/file/indexer_legacy_test.go b/statediff/indexer/database/file/indexer_legacy_test.go index 9259f4c13..56bca2683 100644 --- a/statediff/indexer/database/file/indexer_legacy_test.go +++ b/statediff/indexer/database/file/indexer_legacy_test.go @@ -108,7 +108,7 @@ func TestFileIndexerLegacy(t *testing.T) { setupLegacy(t) dumpData(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, block_hash, base_fee + pgStr := `SELECT cid, td, reward, block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -116,8 +116,8 @@ func TestFileIndexerLegacy(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *string `db:"base_fee"` + BlockHash string `db:"block_hash"` + Coinbase string `db:"coinbase"` } header := new(res) err = sqlxdb.QueryRowx(pgStr, legacyData.BlockNumber.Uint64()).StructScan(header) @@ -126,7 +126,7 @@ func TestFileIndexerLegacy(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") + test_helpers.ExpectEqual(t, header.Coinbase, legacyData.MockBlock.Coinbase().String()) require.Nil(t, legacyData.MockHeader.BaseFee) - require.Nil(t, header.BaseFee) }) } diff --git a/statediff/indexer/database/file/indexer_test.go b/statediff/indexer/database/file/indexer_test.go index b0489fc08..b0a8835ee 100644 --- a/statediff/indexer/database/file/indexer_test.go +++ b/statediff/indexer/database/file/indexer_test.go @@ -177,7 +177,7 @@ func TestFileIndexer(t *testing.T) { setup(t) dumpData(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, block_hash, base_fee + pgStr := `SELECT cid, td, reward, block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -185,8 +185,8 @@ func TestFileIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *string `db:"base_fee"` + BlockHash string `db:"block_hash"` + Coinbase string `db:"coinbase"` } header := new(res) err = sqlxdb.QueryRowx(pgStr, mocks.BlockNumber.Uint64()).StructScan(header) @@ -197,7 +197,7 @@ func TestFileIndexer(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) + test_helpers.ExpectEqual(t, header.Coinbase, mocks.MockHeader.Coinbase.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 6329ecae1..62fd5cb8d 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -124,18 +124,14 @@ const ( ipldInsert = "INSERT INTO public.blocks (key, data) VALUES ('%s', '\\x%x');\n" headerInsert = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, " + - "state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + - "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s', %d, %s);\n" - - headerInsertWithoutBaseFee = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, " + - "reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) VALUES " + - "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s', %d, NULL);\n" + "state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) VALUES " + + "('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s', %d, '%s');\n" uncleInsert = "INSERT INTO eth.uncle_cids (block_hash, header_id, parent_hash, cid, reward, mh_key) VALUES " + "('%s', '%s', '%s', '%s', '%s', '%s');\n" - txInsert = "INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) " + - "VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '\\x%x', %d);\n" + txInsert = "INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type, " + + "value) VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '\\x%x', %d, '%s');\n" alInsert = "INSERT INTO eth.access_list_elements (tx_id, index, address, storage_keys) VALUES ('%s', %d, '%s', '%s');\n" @@ -191,42 +187,40 @@ func (sqw *SQLWriter) upsertIPLDRaw(codec, mh uint64, raw []byte) (string, strin } func (sqw *SQLWriter) upsertHeaderCID(header models.HeaderModel) { - var stmt string - if header.BaseFee == nil { - stmt = fmt.Sprintf(headerInsertWithoutBaseFee, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, - header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1) - } else { - stmt = fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, - header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, *header.BaseFee) - } + stmt := fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, + header.TotalDifficulty, header.NodeID, header.Reward, header.StateRoot, header.TxRoot, + header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.Coinbase) sqw.stmts <- []byte(stmt) indexerMetrics.blocks.Inc(1) } func (sqw *SQLWriter) upsertUncleCID(uncle models.UncleModel) { - sqw.stmts <- []byte(fmt.Sprintf(uncleInsert, uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID, uncle.Reward, uncle.MhKey)) + sqw.stmts <- []byte(fmt.Sprintf(uncleInsert, uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID, + uncle.Reward, uncle.MhKey)) } func (sqw *SQLWriter) upsertTransactionCID(transaction models.TxModel) { - sqw.stmts <- []byte(fmt.Sprintf(txInsert, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type)) + sqw.stmts <- []byte(fmt.Sprintf(txInsert, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, + transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type, transaction.Value)) indexerMetrics.transactions.Inc(1) } func (sqw *SQLWriter) upsertAccessListElement(accessListElement models.AccessListElementModel) { - sqw.stmts <- []byte(fmt.Sprintf(alInsert, accessListElement.TxID, accessListElement.Index, accessListElement.Address, formatPostgresStringArray(accessListElement.StorageKeys))) + sqw.stmts <- []byte(fmt.Sprintf(alInsert, accessListElement.TxID, accessListElement.Index, accessListElement.Address, + formatPostgresStringArray(accessListElement.StorageKeys))) indexerMetrics.accessListEntries.Inc(1) } func (sqw *SQLWriter) upsertReceiptCID(rct *models.ReceiptModel) { - sqw.stmts <- []byte(fmt.Sprintf(rctInsert, rct.TxID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, rct.PostState, rct.PostStatus, rct.LogRoot)) + sqw.stmts <- []byte(fmt.Sprintf(rctInsert, rct.TxID, rct.LeafCID, rct.Contract, rct.ContractHash, rct.LeafMhKey, + rct.PostState, rct.PostStatus, rct.LogRoot)) indexerMetrics.receipts.Inc(1) } func (sqw *SQLWriter) upsertLogCID(logs []*models.LogsModel) { for _, l := range logs { - sqw.stmts <- []byte(fmt.Sprintf(logInsert, l.LeafCID, l.LeafMhKey, l.ReceiptID, l.Address, l.Index, l.Topic0, l.Topic1, l.Topic2, l.Topic3, l.Data)) + sqw.stmts <- []byte(fmt.Sprintf(logInsert, l.LeafCID, l.LeafMhKey, l.ReceiptID, l.Address, l.Index, l.Topic0, + l.Topic1, l.Topic2, l.Topic3, l.Data)) indexerMetrics.logs.Inc(1) } } diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index d5f7c3660..72e978aad 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -256,7 +256,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he TxRoot: header.TxHash.String(), UncleRoot: header.UncleHash.String(), Timestamp: header.Time, - BaseFee: baseFee, + Coinbase: header.Coinbase.String(), }) } @@ -316,6 +316,12 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs // index tx trx := args.txs[i] txID := trx.Hash().String() + + var val string + if trx.Value() != nil { + val = trx.Value().String() + } + // derive sender for the tx that corresponds with this receipt from, err := types.Sender(signer, trx) if err != nil { @@ -331,6 +337,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs CID: txNode.Cid().String(), MhKey: shared.MultihashKeyFromCID(txNode.Cid()), Type: trx.Type(), + Value: val, } if err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel); err != nil { return err diff --git a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go index 7dc38a3ca..768652b46 100644 --- a/statediff/indexer/database/sql/pgx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_legacy_test.go @@ -63,7 +63,7 @@ func TestPGXIndexerLegacy(t *testing.T) { t.Run("Publish and index header IPLDs", func(t *testing.T) { setupLegacyPGX(t) defer tearDown(t) - pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, base_fee + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -72,18 +72,18 @@ func TestPGXIndexerLegacy(t *testing.T) { TD string Reward string BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + Coinbase string `db:"coinbase"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).Scan( - &header.CID, &header.TD, &header.Reward, &header.BlockHash, &header.BaseFee) + &header.CID, &header.TD, &header.Reward, &header.BlockHash, &header.Coinbase) require.NoError(t, err) test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") + test_helpers.ExpectEqual(t, header.Coinbase, legacyData.MockHeader.Coinbase.String()) require.Nil(t, legacyData.MockHeader.BaseFee) - require.Nil(t, header.BaseFee) }) } diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index a378424db..f78882986 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -150,7 +150,7 @@ func TestPGXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupPGX(t) defer tearDown(t) - pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, cast(base_fee AS TEXT) + pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -158,8 +158,8 @@ func TestPGXIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *string `db:"base_fee"` + BlockHash string `db:"block_hash"` + Coinbase string `db:"coinbase"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).Scan( @@ -167,14 +167,14 @@ func TestPGXIndexer(t *testing.T) { &header.TD, &header.Reward, &header.BlockHash, - &header.BaseFee) + &header.Coinbase) if err != nil { t.Fatal(err) } test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) + test_helpers.ExpectEqual(t, header.Coinbase, mocks.MockHeader.Coinbase.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index 5794bd0af..a7c7cc9b4 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -49,9 +49,9 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_testing", + DatabaseName: "vulcanize_test", Username: "postgres", - Password: "password", + Password: "", } // Config holds params for a Postgres db diff --git a/statediff/indexer/database/sql/postgres/database.go b/statediff/indexer/database/sql/postgres/database.go index 99fae1c02..4cff518a0 100644 --- a/statediff/indexer/database/sql/postgres/database.go +++ b/statediff/indexer/database/sql/postgres/database.go @@ -37,9 +37,9 @@ type DB struct { // InsertHeaderStm satisfies the sql.Statements interface func (db *DB) InsertHeaderStm() string { - return `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) + return `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) - ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16)` + ON CONFLICT (block_hash) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16)` } // InsertUncleStm satisfies the sql.Statements interface @@ -50,7 +50,7 @@ func (db *DB) InsertUncleStm() string { // InsertTxStm satisfies the sql.Statements interface func (db *DB) InsertTxStm() string { - return `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) + return `INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type, value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10) ON CONFLICT (tx_hash) DO NOTHING` } diff --git a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go index d0ed3568b..08f3f080e 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_legacy_test.go @@ -73,7 +73,7 @@ func TestSQLXIndexerLegacy(t *testing.T) { t.Run("Publish and index header IPLDs", func(t *testing.T) { setupLegacySQLX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, block_hash, base_fee + pgStr := `SELECT cid, td, reward, block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -82,7 +82,7 @@ func TestSQLXIndexerLegacy(t *testing.T) { TD string Reward string BlockHash string `db:"block_hash"` - BaseFee *int64 `db:"base_fee"` + Coinbase string `db:"coinbase"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) @@ -91,7 +91,7 @@ func TestSQLXIndexerLegacy(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, legacyHeaderCID.String()) test_helpers.ExpectEqual(t, header.TD, legacyData.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "5000000000000011250") + test_helpers.ExpectEqual(t, header.Coinbase, legacyData.MockHeader.Coinbase.String()) require.Nil(t, legacyData.MockHeader.BaseFee) - require.Nil(t, header.BaseFee) }) } diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 68f06ecaa..d4854e989 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -177,7 +177,7 @@ func TestSQLXIndexer(t *testing.T) { t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) { setupSQLX(t) defer tearDown(t) - pgStr := `SELECT cid, td, reward, block_hash, base_fee + pgStr := `SELECT cid, td, reward, block_hash, coinbase FROM eth.header_cids WHERE block_number = $1` // check header was properly indexed @@ -185,8 +185,8 @@ func TestSQLXIndexer(t *testing.T) { CID string TD string Reward string - BlockHash string `db:"block_hash"` - BaseFee *string `db:"base_fee"` + BlockHash string `db:"block_hash"` + Coinbase string `db:"coinbase"` } header := new(res) err = db.QueryRow(context.Background(), pgStr, mocks.BlockNumber.Uint64()).(*sqlx.Row).StructScan(header) @@ -196,7 +196,7 @@ func TestSQLXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, header.CID, headerCID.String()) test_helpers.ExpectEqual(t, header.TD, mocks.MockBlock.Difficulty().String()) test_helpers.ExpectEqual(t, header.Reward, "2000000000000021250") - test_helpers.ExpectEqual(t, *header.BaseFee, mocks.MockHeader.BaseFee.String()) + test_helpers.ExpectEqual(t, header.Coinbase, mocks.MockHeader.Coinbase.String()) dc, err := cid.Decode(header.CID) if err != nil { t.Fatal(err) diff --git a/statediff/indexer/database/sql/writer.go b/statediff/indexer/database/sql/writer.go index f426263e0..3f1dfc0b5 100644 --- a/statediff/indexer/database/sql/writer.go +++ b/statediff/indexer/database/sql/writer.go @@ -45,14 +45,15 @@ func (w *Writer) Close() error { } /* -INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) +INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) -ON CONFLICT (block_hash) DO UPDATE SET (block_number, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, base_fee) = ($1, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) +ON CONFLICT (block_hash) DO UPDATE SET (block_number, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) = ($1, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, eth.header_cids.times_validated + 1, $16) */ func (w *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error { _, err := tx.Exec(w.db.Context(), w.db.InsertHeaderStm(), - header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, w.db.NodeID(), header.Reward, header.StateRoot, header.TxRoot, - header.RctRoot, header.UncleRoot, header.Bloom, header.Timestamp, header.MhKey, 1, header.BaseFee) + header.BlockNumber, header.BlockHash, header.ParentHash, header.CID, header.TotalDifficulty, w.db.NodeID(), + header.Reward, header.StateRoot, header.TxRoot, header.RctRoot, header.UncleRoot, header.Bloom, + header.Timestamp, header.MhKey, 1, header.Coinbase) if err != nil { return fmt.Errorf("error upserting header_cids entry: %v", err) } @@ -74,12 +75,13 @@ func (w *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error { } /* -INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9) +INSERT INTO eth.transaction_cids (header_id, tx_hash, cid, dst, src, index, mh_key, tx_data, tx_type, value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10) ON CONFLICT (tx_hash) DO NOTHING */ func (w *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error { _, err := tx.Exec(w.db.Context(), w.db.InsertTxStm(), - transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, transaction.MhKey, transaction.Data, transaction.Type) + transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst, transaction.Src, transaction.Index, + transaction.MhKey, transaction.Data, transaction.Type, transaction.Value) if err != nil { return fmt.Errorf("error upserting transaction_cids entry: %v", err) } @@ -122,7 +124,8 @@ ON CONFLICT (rct_id, index) DO NOTHING func (w *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error { for _, log := range logs { _, err := tx.Exec(w.db.Context(), w.db.InsertLogStm(), - log.LeafCID, log.LeafMhKey, log.ReceiptID, log.Address, log.Index, log.Topic0, log.Topic1, log.Topic2, log.Topic3, log.Data) + log.LeafCID, log.LeafMhKey, log.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) } @@ -154,7 +157,8 @@ ON CONFLICT (header_id, state_path) DO NOTHING */ func (w *Writer) upsertStateAccount(tx Tx, stateAccount models.StateAccountModel) error { _, err := tx.Exec(w.db.Context(), w.db.InsertAccountStm(), - stateAccount.HeaderID, stateAccount.StatePath, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, stateAccount.StorageRoot) + stateAccount.HeaderID, stateAccount.StatePath, stateAccount.Balance, stateAccount.Nonce, stateAccount.CodeHash, + stateAccount.StorageRoot) if err != nil { return fmt.Errorf("error upserting state_accounts entry: %v", err) } @@ -171,7 +175,8 @@ func (w *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel) err storageKey = storageCID.StorageKey } _, err := tx.Exec(w.db.Context(), w.db.InsertStorageStm(), - storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, true, storageCID.MhKey) + storageCID.HeaderID, storageCID.StatePath, storageKey, storageCID.CID, storageCID.Path, storageCID.NodeType, + true, storageCID.MhKey) if err != nil { return fmt.Errorf("error upserting storage_cids entry: %v", err) } diff --git a/statediff/indexer/mocks/test_data.go b/statediff/indexer/mocks/test_data.go index be61edb87..35d70d2f9 100644 --- a/statediff/indexer/mocks/test_data.go +++ b/statediff/indexer/mocks/test_data.go @@ -49,6 +49,7 @@ var ( Difficulty: big.NewInt(5000000), Extra: []byte{}, BaseFee: big.NewInt(params.InitialBaseFee), + Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476777"), } MockTransactions, MockReceipts, SenderAddr = createTransactionsAndReceipts(TestConfig, BlockNumber) MockBlock = types.NewBlock(&MockHeader, MockTransactions, nil, MockReceipts, new(trie.Trie)) @@ -216,6 +217,7 @@ func NewLegacyData() *LegacyData { ReceiptHash: common.HexToHash("0x0"), Difficulty: big.NewInt(5000000), Extra: []byte{}, + Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476888"), } mockTransactions, mockReceipts, senderAddr := createLegacyTransactionsAndReceipts(config, blockNumber) diff --git a/statediff/indexer/models/models.go b/statediff/indexer/models/models.go index e471d20ec..2caed1bcb 100644 --- a/statediff/indexer/models/models.go +++ b/statediff/indexer/models/models.go @@ -26,22 +26,22 @@ type IPLDModel struct { // HeaderModel is the db model for eth.header_cids type HeaderModel struct { - BlockNumber string `db:"block_number"` - BlockHash string `db:"block_hash"` - ParentHash string `db:"parent_hash"` - CID string `db:"cid"` - MhKey string `db:"mh_key"` - TotalDifficulty string `db:"td"` - NodeID string `db:"node_id"` - Reward string `db:"reward"` - StateRoot string `db:"state_root"` - UncleRoot string `db:"uncle_root"` - TxRoot string `db:"tx_root"` - RctRoot string `db:"receipt_root"` - Bloom []byte `db:"bloom"` - Timestamp uint64 `db:"timestamp"` - TimesValidated int64 `db:"times_validated"` - BaseFee *string `db:"base_fee"` + BlockNumber string `db:"block_number"` + BlockHash string `db:"block_hash"` + ParentHash string `db:"parent_hash"` + CID string `db:"cid"` + MhKey string `db:"mh_key"` + TotalDifficulty string `db:"td"` + NodeID string `db:"node_id"` + Reward string `db:"reward"` + StateRoot string `db:"state_root"` + UncleRoot string `db:"uncle_root"` + TxRoot string `db:"tx_root"` + RctRoot string `db:"receipt_root"` + Bloom []byte `db:"bloom"` + Timestamp uint64 `db:"timestamp"` + TimesValidated int64 `db:"times_validated"` + Coinbase string `db:"coinbase"` } // UncleModel is the db model for eth.uncle_cids @@ -65,6 +65,7 @@ type TxModel struct { Src string `db:"src"` Data []byte `db:"tx_data"` Type uint8 `db:"tx_type"` + Value string `db:"value"` } // AccessListElementModel is the db model for eth.access_list_entry -- 2.45.2 From a3e9d544a64cf6bb67e4d95d495a0d4c7dedd1ca Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 26 Nov 2021 09:12:10 -0600 Subject: [PATCH 17/33] finish unit tests --- .../indexer/database/file/indexer_test.go | 65 ++++++++++++------- .../indexer/database/sql/pgx_indexer_test.go | 62 ++++++++++++------ .../indexer/database/sql/sqlx_indexer_test.go | 62 ++++++++++++------ statediff/indexer/mocks/test_data.go | 2 +- 4 files changed, 126 insertions(+), 65 deletions(-) diff --git a/statediff/indexer/database/file/indexer_test.go b/statediff/indexer/database/file/indexer_test.go index b0a8835ee..c2be9f993 100644 --- a/statediff/indexer/database/file/indexer_test.go +++ b/statediff/indexer/database/file/indexer_test.go @@ -52,6 +52,8 @@ var ( ipfsPgGet = `SELECT data FROM public.blocks WHERE key = $1` tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte + txs types.Transactions + rcts types.Receipts mockBlock *types.Block headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid @@ -65,7 +67,7 @@ func init() { } mockBlock = mocks.MockBlock - txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts + txs, rcts = mocks.MockBlock.Transactions(), mocks.MockReceipts buf := new(bytes.Buffer) txs.EncodeIndex(0, buf) @@ -231,6 +233,10 @@ func TestFileIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) // and published + type txResult struct { + TxType uint8 `db:"tx_type"` + Value string + } for _, c := range trxs { dc, err := cid.Decode(c) if err != nil { @@ -243,47 +249,59 @@ func TestFileIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + txTypeAndValueStr := `SELECT tx_type, value FROM eth.transaction_cids WHERE cid = $1` switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) - var txType uint8 - err = sqlxdb.Get(&txType, txTypePgStr, c) + txRes := new(txResult) + err = sqlxdb.QueryRowx(txTypeAndValueStr, c).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != txs[0].Value().String() { + t.Fatalf("expected tx value %s got %s", txs[0].Value().String(), txRes.Value) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) - var txType uint8 - err = sqlxdb.Get(&txType, txTypePgStr, c) + txRes := new(txResult) + err = sqlxdb.QueryRowx(txTypeAndValueStr, c).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != txs[1].Value().String() { + t.Fatalf("expected tx value %s got %s", txs[1].Value().String(), txRes.Value) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) - var txType uint8 - err = sqlxdb.Get(&txType, txTypePgStr, c) + txRes := new(txResult) + err = sqlxdb.QueryRowx(txTypeAndValueStr, c).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != txs[2].Value().String() { + t.Fatalf("expected tx value %s got %s", txs[2].Value().String(), txRes.Value) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) - var txType uint8 - err = sqlxdb.Get(&txType, txTypePgStr, c) + txRes := new(txResult) + err = sqlxdb.QueryRowx(txTypeAndValueStr, c).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != types.AccessListTxType { - t.Fatalf("expected AccessListTxType (1), got %d", txType) + if txRes.TxType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txRes.TxType) + } + if txRes.Value != txs[3].Value().String() { + t.Fatalf("expected tx value %s got %s", txs[3].Value().String(), txRes.Value) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` @@ -307,13 +325,16 @@ func TestFileIndexer(t *testing.T) { test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) case trx5CID.String(): test_helpers.ExpectEqual(t, data, tx5) - var txType *uint8 - err = sqlxdb.Get(&txType, txTypePgStr, c) + txRes := new(txResult) + err = sqlxdb.QueryRowx(txTypeAndValueStr, c).StructScan(txRes) if err != nil { t.Fatal(err) } - if *txType != types.DynamicFeeTxType { - t.Fatalf("expected DynamicFeeTxType (2), got %d", *txType) + if txRes.TxType != types.DynamicFeeTxType { + t.Fatalf("expected DynamicFeeTxType (2), got %d", txRes.TxType) + } + if txRes.Value != txs[4].Value().String() { + t.Fatalf("expected tx value %s got %s", txs[4].Value().String(), txRes.Value) } } } diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index f78882986..bdb524032 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -207,6 +207,11 @@ func TestPGXIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) // and published + transactions := mocks.MockBlock.Transactions() + type txResult struct { + TxType uint8 `db:"tx_type"` + Value string + } for _, c := range trxs { dc, err := cid.Decode(c) if err != nil { @@ -219,47 +224,59 @@ func TestPGXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + txTypeAndValueStr := `SELECT tx_type, CAST(value as TEXT) FROM eth.transaction_cids WHERE cid = $1` switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).Scan(&txRes.TxType, &txRes.Value) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected tx_type 0, got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[0].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[0].Value().String(), txRes.Value) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).Scan(&txRes.TxType, &txRes.Value) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected tx_type 0, got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[1].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[1].Value().String(), txRes.Value) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).Scan(&txRes.TxType, &txRes.Value) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected tx_type 0, got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[2].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[2].Value().String(), txRes.Value) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).Scan(&txRes.TxType, &txRes.Value) if err != nil { t.Fatal(err) } - if txType != types.AccessListTxType { - t.Fatalf("expected AccessListTxType (1), got %d", txType) + if txRes.TxType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txRes.TxType) + } + if txRes.Value != transactions[3].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[3].Value().String(), txRes.Value) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` @@ -283,13 +300,16 @@ func TestPGXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) case trx5CID.String(): test_helpers.ExpectEqual(t, data, tx5) - var txType *uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).Scan(&txRes.TxType, &txRes.Value) if err != nil { t.Fatal(err) } - if *txType != types.DynamicFeeTxType { - t.Fatalf("expected DynamicFeeTxType (2), got %d", *txType) + if txRes.TxType != types.DynamicFeeTxType { + t.Fatalf("expected DynamicFeeTxType (2), got %d", txRes.TxType) + } + if txRes.Value != transactions[4].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[4].Value().String(), txRes.Value) } } } diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index d4854e989..9ab1163ec 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -229,6 +229,11 @@ func TestSQLXIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) // and published + transactions := mocks.MockBlock.Transactions() + type txResult struct { + TxType uint8 `db:"tx_type"` + Value string + } for _, c := range trxs { dc, err := cid.Decode(c) if err != nil { @@ -241,47 +246,59 @@ func TestSQLXIndexer(t *testing.T) { if err != nil { t.Fatal(err) } - txTypePgStr := `SELECT tx_type FROM eth.transaction_cids WHERE cid = $1` + txTypeAndValueStr := `SELECT tx_type, value FROM eth.transaction_cids WHERE cid = $1` switch c { case trx1CID.String(): test_helpers.ExpectEqual(t, data, tx1) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).(*sqlx.Row).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[0].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[0].Value().String(), txRes.Value) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).(*sqlx.Row).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[1].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[1].Value().String(), txRes.Value) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).(*sqlx.Row).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != 0 { - t.Fatalf("expected LegacyTxType (0), got %d", txType) + if txRes.TxType != 0 { + t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) + } + if txRes.Value != transactions[2].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[2].Value().String(), txRes.Value) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) - var txType uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).(*sqlx.Row).StructScan(txRes) if err != nil { t.Fatal(err) } - if txType != types.AccessListTxType { - t.Fatalf("expected AccessListTxType (1), got %d", txType) + if txRes.TxType != types.AccessListTxType { + t.Fatalf("expected AccessListTxType (1), got %d", txRes.TxType) + } + if txRes.Value != transactions[3].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[3].Value().String(), txRes.Value) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` @@ -305,13 +322,16 @@ func TestSQLXIndexer(t *testing.T) { test_helpers.ExpectEqual(t, model2, mocks.AccessListEntry2Model) case trx5CID.String(): test_helpers.ExpectEqual(t, data, tx5) - var txType *uint8 - err = db.Get(context.Background(), &txType, txTypePgStr, c) + txRes := new(txResult) + err = db.QueryRow(context.Background(), txTypeAndValueStr, c).(*sqlx.Row).StructScan(txRes) if err != nil { t.Fatal(err) } - if *txType != types.DynamicFeeTxType { - t.Fatalf("expected DynamicFeeTxType (2), got %d", *txType) + if txRes.TxType != types.DynamicFeeTxType { + t.Fatalf("expected DynamicFeeTxType (2), got %d", txRes.TxType) + } + if txRes.Value != transactions[4].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[4].Value().String(), txRes.Value) } } } diff --git a/statediff/indexer/mocks/test_data.go b/statediff/indexer/mocks/test_data.go index 35d70d2f9..dccf72e60 100644 --- a/statediff/indexer/mocks/test_data.go +++ b/statediff/indexer/mocks/test_data.go @@ -308,7 +308,7 @@ func createTransactionsAndReceipts(config *params.ChainConfig, blockNumber *big. GasPrice: big.NewInt(100), Gas: 50, To: &AnotherAddress, - Value: big.NewInt(1000), + Value: big.NewInt(999), Data: []byte{}, AccessList: types.AccessList{ AccessListEntry1, -- 2.45.2 From 6b457b0c134f0c6caa0df1717de20cdcc9bfd0a8 Mon Sep 17 00:00:00 2001 From: i-norden Date: Sun, 28 Nov 2021 23:51:34 -0600 Subject: [PATCH 18/33] test harnest for arbitrary mainnet blocks and receipts --- .../file/mainnet_tests/indexer_test.go | 123 ++++++++++ .../file/mainnet_tests/test_helpers.go | 224 ++++++++++++++++++ .../database/sql/indexer_shared_test.go | 112 +++++++++ .../indexer/database/sql/pgx_indexer_test.go | 81 ------- .../indexer/database/sql/sqlx_indexer_test.go | 100 -------- 5 files changed, 459 insertions(+), 181 deletions(-) create mode 100644 statediff/indexer/database/file/mainnet_tests/indexer_test.go create mode 100644 statediff/indexer/database/file/mainnet_tests/test_helpers.go create mode 100644 statediff/indexer/database/sql/indexer_shared_test.go diff --git a/statediff/indexer/database/file/mainnet_tests/indexer_test.go b/statediff/indexer/database/file/mainnet_tests/indexer_test.go new file mode 100644 index 000000000..7b7ba8ff6 --- /dev/null +++ b/statediff/indexer/database/file/mainnet_tests/indexer_test.go @@ -0,0 +1,123 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package mainnet_tests + +import ( + "context" + "errors" + "fmt" + "os" + "testing" + + "github.com/ipfs/go-cid" + "github.com/jmoiron/sqlx" + "github.com/multiformats/go-multihash" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/statediff/indexer/database/file" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +var ( + testBlock *types.Block + testReceipts types.Receipts + testHeaderCID cid.Cid + sqlxdb *sqlx.DB + err error + chainConf = params.MainnetChainConfig +) + +func init() { + if os.Getenv("MODE") != "statediff" { + fmt.Println("Skipping statediff test") + os.Exit(0) + } +} + +func setup(t *testing.T) { + testBlock, testReceipts, err = TestBlocksAndReceiptsFromEnv() + require.NoError(t, err) + headerRLP, err := rlp.EncodeToBytes(testBlock.Header()) + require.NoError(t, err) + + testHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, headerRLP, multihash.KECCAK_256) + if _, err := os.Stat(file.TestConfig.FilePath); !errors.Is(err, os.ErrNotExist) { + err := os.Remove(file.TestConfig.FilePath) + require.NoError(t, err) + } + ind, err := file.NewStateDiffIndexer(context.Background(), chainConf, file.TestConfig) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + testBlock, + testReceipts, + testBlock.Difficulty()) + require.NoError(t, err) + + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + if err := ind.Close(); err != nil { + t.Fatal(err) + } + }() + for _, node := range mocks.StateDiffs { + err = ind.PushStateNode(tx, node, testBlock.Hash().String()) + require.NoError(t, err) + } + + test_helpers.ExpectEqual(t, tx.(*file.BatchTx).BlockNumber, testBlock.Number().Uint64()) + + connStr := postgres.DefaultConfig.DbConnectionString() + + sqlxdb, err = sqlx.Connect("postgres", connStr) + if err != nil { + t.Fatalf("failed to connect to db with connection string: %s err: %v", connStr, err) + } +} + +func dumpData(t *testing.T) { + sqlFileBytes, err := os.ReadFile(file.TestConfig.FilePath) + require.NoError(t, err) + + _, err = sqlxdb.Exec(string(sqlFileBytes)) + require.NoError(t, err) +} + +func tearDown(t *testing.T) { + file.TearDownDB(t, sqlxdb) + err := os.Remove(file.TestConfig.FilePath) + require.NoError(t, err) + err = sqlxdb.Close() + require.NoError(t, err) +} + +func TestPushBlockAndState(t *testing.T) { + t.Run("Test PushBlock and PushStateNode", func(t *testing.T) { + setup(t) + dumpData(t) + tearDown(t) + }) +} diff --git a/statediff/indexer/database/file/mainnet_tests/test_helpers.go b/statediff/indexer/database/file/mainnet_tests/test_helpers.go new file mode 100644 index 000000000..ed372a4c2 --- /dev/null +++ b/statediff/indexer/database/file/mainnet_tests/test_helpers.go @@ -0,0 +1,224 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package mainnet_tests + +import ( + "bufio" + "context" + "errors" + "fmt" + "math/big" + "os" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/ethereum/go-ethereum/rlp" +) + +const ( + defaultBlockFilePath = "./block" + defaultReceiptsFilePath = "./receipts" +) + +const ( + TEST_RAW_URL = "TEST_RAW_URL" + TEST_BLOCK_NUMBER = "TEST_BLOCK_NUMBER" +) + +// TestConfig holds configuration params for mainnet tests +type TestConfig struct { + RawURL string + BlockNumber *big.Int +} + +// DefaultTestConfig is the default TestConfig +var DefaultTestConfig = TestConfig{ + RawURL: "http://127.0.0.1:8545", + BlockNumber: big.NewInt(12914665), +} + +// TestBlocksAndReceiptsFromEnv retrieves the block and receipts using env variables to override default config +func TestBlocksAndReceiptsFromEnv() (*types.Block, types.Receipts, error) { + conf := DefaultTestConfig + rawURL := os.Getenv(TEST_RAW_URL) + if rawURL == "" { + fmt.Println("Warning: no raw url configured for statediffing mainnet tests") + } else { + conf.RawURL = rawURL + } + blockNumberStr := os.Getenv(TEST_BLOCK_NUMBER) + blockNumber, ok := new(big.Int).SetString(blockNumberStr, 10) + if !ok { + fmt.Println("Warning: no blockNumber configured for statediffing mainnet tests") + } else { + conf.BlockNumber = blockNumber + } + return TestBlocksAndReceipts(conf) +} + +// TestBlocksAndReceipts retrieves the block and receipts for the provided test config +// It first tries to load files from the local system before setting up and using an ethclient.Client to pull the data +func TestBlocksAndReceipts(conf TestConfig) (*types.Block, types.Receipts, error) { + var cli *ethclient.Client + var err error + var block *types.Block + var receipts types.Receipts + blockFilePath := fmt.Sprintf("%s.%s.rlp", defaultBlockFilePath, conf.BlockNumber.String()) + if _, err = os.Stat(blockFilePath); !errors.Is(err, os.ErrNotExist) { + block, err = LoadBlockRLP(blockFilePath) + if err != nil { + cli, err = ethclient.Dial(conf.RawURL) + if err != nil { + return nil, nil, err + } + block, err = FetchBlock(cli, conf.BlockNumber) + if err != nil { + return nil, nil, err + } + } + } else { + cli, err = ethclient.Dial(conf.RawURL) + if err != nil { + return nil, nil, err + } + block, err = FetchBlock(cli, conf.BlockNumber) + if err != nil { + return nil, nil, err + } + } + receiptsFilePath := fmt.Sprintf("%s.%s.enc", defaultReceiptsFilePath, conf.BlockNumber.String()) + if _, err = os.Stat(receiptsFilePath); !errors.Is(err, os.ErrNotExist) { + receipts, err = LoadReceiptsEncoding(receiptsFilePath, len(block.Transactions())) + if err != nil { + if cli == nil { + cli, err = ethclient.Dial(conf.RawURL) + if err != nil { + return nil, nil, err + } + } + receipts, err = FetchReceipts(cli, block) + if err != nil { + return nil, nil, err + } + } + } else { + if cli == nil { + cli, err = ethclient.Dial(conf.RawURL) + if err != nil { + return nil, nil, err + } + } + receipts, err = FetchReceipts(cli, block) + if err != nil { + return nil, nil, err + } + } + return block, receipts, nil +} + +// FetchBlock fetches the block at the provided height using the ethclient.Client +func FetchBlock(cli *ethclient.Client, blockNumber *big.Int) (*types.Block, error) { + return cli.BlockByNumber(context.Background(), blockNumber) +} + +// FetchReceipts fetches the receipts for the provided block using the ethclient.Client +func FetchReceipts(cli *ethclient.Client, block *types.Block) (types.Receipts, error) { + receipts := make(types.Receipts, len(block.Transactions())) + for i, tx := range block.Transactions() { + rct, err := cli.TransactionReceipt(context.Background(), tx.Hash()) + if err != nil { + return nil, err + } + receipts[i] = rct + } + return receipts, nil +} + +// WriteBlockRLP writes out the RLP encoding of the block to the provided filePath +func WriteBlockRLP(filePath string, block *types.Block) error { + if filePath == "" { + filePath = fmt.Sprintf("%s_%s.rlp", defaultBlockFilePath, block.Number().String()) + } + if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) { + return fmt.Errorf("cannot create file, file (%s) already exists", filePath) + } + file, err := os.Create(filePath) + if err != nil { + return fmt.Errorf("unable to create file (%s), err: %v", filePath, err) + } + if err := block.EncodeRLP(file); err != nil { + return err + } + return file.Close() +} + +// LoadBlockRLP loads block from the rlp at filePath +func LoadBlockRLP(filePath string) (*types.Block, error) { + blockBytes, err := os.ReadFile(filePath) + if err != nil { + return nil, err + } + block := new(types.Block) + return block, rlp.DecodeBytes(blockBytes, block) +} + +// LoadReceiptsEncoding loads receipts from the encoding at filePath +func LoadReceiptsEncoding(filePath string, cap int) (types.Receipts, error) { + file, err := os.Open(filePath) + if err != nil { + return nil, err + } + defer file.Close() + scanner := bufio.NewScanner(file) + receipts := make(types.Receipts, 0, cap) + for scanner.Scan() { + rctBinary := scanner.Bytes() + rct := new(types.Receipt) + if err := rct.UnmarshalBinary(rctBinary); err != nil { + return nil, err + } + receipts = append(receipts, rct) + } + return receipts, nil +} + +// WriteReceiptsEncoding writes out the consensus encoding of the receipts to the provided io.WriteCloser +func WriteReceiptsEncoding(filePath string, blockNumber *big.Int, receipts types.Receipts) error { + if filePath == "" { + filePath = fmt.Sprintf("%s_%s.enc", defaultReceiptsFilePath, blockNumber.String()) + } + if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) { + return fmt.Errorf("cannot create file, file (%s) already exists", filePath) + } + file, err := os.Create(filePath) + if err != nil { + return fmt.Errorf("unable to create file (%s), err: %v", filePath, err) + } + for _, rct := range receipts { + rctEncoding, err := rct.MarshalBinary() + if err != nil { + return err + } + if _, err := file.Write(rctEncoding); err != nil { + return err + } + if _, err := file.Write([]byte("\n")); err != nil { + return err + } + } + return file.Close() +} diff --git a/statediff/indexer/database/sql/indexer_shared_test.go b/statediff/indexer/database/sql/indexer_shared_test.go new file mode 100644 index 000000000..46ad3a3f4 --- /dev/null +++ b/statediff/indexer/database/sql/indexer_shared_test.go @@ -0,0 +1,112 @@ +package sql_test + +import ( + "bytes" + "fmt" + "os" + "testing" + + "github.com/ipfs/go-cid" + "github.com/multiformats/go-multihash" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/ipld" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" +) + +var ( + db sql.Database + err error + ind interfaces.StateDiffIndexer + ipfsPgGet = `SELECT data FROM public.blocks + WHERE key = $1` + tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte + mockBlock *types.Block + headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid + rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid + state1CID, state2CID, storageCID cid.Cid +) + +func init() { + if os.Getenv("MODE") != "statediff" { + fmt.Println("Skipping statediff test") + os.Exit(0) + } + + mockBlock = mocks.MockBlock + txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts + + buf := new(bytes.Buffer) + txs.EncodeIndex(0, buf) + tx1 = make([]byte, buf.Len()) + copy(tx1, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(1, buf) + tx2 = make([]byte, buf.Len()) + copy(tx2, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(2, buf) + tx3 = make([]byte, buf.Len()) + copy(tx3, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(3, buf) + tx4 = make([]byte, buf.Len()) + copy(tx4, buf.Bytes()) + buf.Reset() + + txs.EncodeIndex(4, buf) + tx5 = make([]byte, buf.Len()) + copy(tx5, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(0, buf) + rct1 = make([]byte, buf.Len()) + copy(rct1, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(1, buf) + rct2 = make([]byte, buf.Len()) + copy(rct2, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(2, buf) + rct3 = make([]byte, buf.Len()) + copy(rct3, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(3, buf) + rct4 = make([]byte, buf.Len()) + copy(rct4, buf.Bytes()) + buf.Reset() + + rcts.EncodeIndex(4, buf) + rct5 = make([]byte, buf.Len()) + copy(rct5, buf.Bytes()) + buf.Reset() + + headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256) + trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256) + trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256) + trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) + trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) + trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) + rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) + rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) + rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) + rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) + rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) + state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) + state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) + storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) +} + +func expectTrue(t *testing.T, value bool) { + if !value { + t.Fatalf("Assertion failed") + } +} diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index bdb524032..ce6fb13c0 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -17,16 +17,12 @@ package sql_test import ( - "bytes" "context" - "fmt" - "os" "testing" "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" "github.com/ethereum/go-ethereum/common" @@ -35,89 +31,12 @@ import ( "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" - "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) -func init() { - if os.Getenv("MODE") != "statediff" { - fmt.Println("Skipping statediff test") - os.Exit(0) - } - - mockBlock = mocks.MockBlock - txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts - - buf := new(bytes.Buffer) - txs.EncodeIndex(0, buf) - tx1 = make([]byte, buf.Len()) - copy(tx1, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(1, buf) - tx2 = make([]byte, buf.Len()) - copy(tx2, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(2, buf) - tx3 = make([]byte, buf.Len()) - copy(tx3, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(3, buf) - tx4 = make([]byte, buf.Len()) - copy(tx4, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(4, buf) - tx5 = make([]byte, buf.Len()) - copy(tx5, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(0, buf) - rct1 = make([]byte, buf.Len()) - copy(rct1, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(1, buf) - rct2 = make([]byte, buf.Len()) - copy(rct2, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(2, buf) - rct3 = make([]byte, buf.Len()) - copy(rct3, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(3, buf) - rct4 = make([]byte, buf.Len()) - copy(rct4, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(4, buf) - rct5 = make([]byte, buf.Len()) - copy(rct5, buf.Bytes()) - buf.Reset() - - headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256) - trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256) - trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256) - trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) - trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) - trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) - rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) - rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) - rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) - rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) - rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) - state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) - state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) - storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) -} - func setupPGX(t *testing.T) { db, err = postgres.SetupPGXDB() if err != nil { diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 9ab1163ec..3403fa3dc 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -17,17 +17,13 @@ package sql_test import ( - "bytes" "context" - "fmt" - "os" "testing" "github.com/ipfs/go-cid" blockstore "github.com/ipfs/go-ipfs-blockstore" dshelp "github.com/ipfs/go-ipfs-ds-help" "github.com/jmoiron/sqlx" - "github.com/multiformats/go-multihash" "github.com/stretchr/testify/require" "github.com/ethereum/go-ethereum/common" @@ -36,108 +32,12 @@ import ( "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" - "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" ) -var ( - db sql.Database - err error - ind interfaces.StateDiffIndexer - ipfsPgGet = `SELECT data FROM public.blocks - WHERE key = $1` - tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte - mockBlock *types.Block - headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid - rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid - state1CID, state2CID, storageCID cid.Cid -) - -func expectTrue(t *testing.T, value bool) { - if !value { - t.Fatalf("Assertion failed") - } -} - -func init() { - if os.Getenv("MODE") != "statediff" { - fmt.Println("Skipping statediff test") - os.Exit(0) - } - - mockBlock = mocks.MockBlock - txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts - - buf := new(bytes.Buffer) - txs.EncodeIndex(0, buf) - tx1 = make([]byte, buf.Len()) - copy(tx1, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(1, buf) - tx2 = make([]byte, buf.Len()) - copy(tx2, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(2, buf) - tx3 = make([]byte, buf.Len()) - copy(tx3, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(3, buf) - tx4 = make([]byte, buf.Len()) - copy(tx4, buf.Bytes()) - buf.Reset() - - txs.EncodeIndex(4, buf) - tx5 = make([]byte, buf.Len()) - copy(tx5, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(0, buf) - rct1 = make([]byte, buf.Len()) - copy(rct1, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(1, buf) - rct2 = make([]byte, buf.Len()) - copy(rct2, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(2, buf) - rct3 = make([]byte, buf.Len()) - copy(rct3, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(3, buf) - rct4 = make([]byte, buf.Len()) - copy(rct4, buf.Bytes()) - buf.Reset() - - rcts.EncodeIndex(4, buf) - rct5 = make([]byte, buf.Len()) - copy(rct5, buf.Bytes()) - buf.Reset() - - headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256) - trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256) - trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256) - trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) - trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) - trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) - rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) - rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) - rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) - rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) - rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) - state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) - state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) - storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) -} - func setupSQLX(t *testing.T) { db, err = postgres.SetupSQLXDB() if err != nil { -- 2.45.2 From 904163dcb38dca489ba68dc9895a28712700b570 Mon Sep 17 00:00:00 2001 From: i-norden Date: Tue, 30 Nov 2021 10:29:07 -0600 Subject: [PATCH 19/33] cache problematic block locally for quicker testing/easier CI testing --- .../file/mainnet_tests/block_12914664.rlp | Bin 0 -> 37150 bytes .../file/mainnet_tests/receipts_12914664.rlp | Bin 0 -> 89259 bytes .../mainnet_tests/statediffing_test_file.sql | 0 .../file/mainnet_tests/test_helpers.go | 77 ++++++++++-------- 4 files changed, 44 insertions(+), 33 deletions(-) create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12914664.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12914664.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/statediffing_test_file.sql diff --git a/statediff/indexer/database/file/mainnet_tests/block_12914664.rlp b/statediff/indexer/database/file/mainnet_tests/block_12914664.rlp new file mode 100644 index 0000000000000000000000000000000000000000..b8aaeaa61e65dd9928b92f2dbecb84837a5e14dc GIT binary patch literal 37150 zcmeFZWn7g@)HlqgySqU`8Uc}R>6BFI5Ge^M>28qNbX_2zw4@;2NJ$EaNOvirAPCR) z96e|2+3Gp>bANc>-}~+_d(D5%Tx-qDnwd3g=DJSeC{Ez906OOTy;CxWGB~-kwV}K*Ww%G{GBNvPm(kq{;%k57)&s(@axsuZCPa%#J9_ zQ3J@So6y)NEyG!fbZW6%S~H*exW73q>_O((w^y#UrB4PpX4-{&OHJRDZMVz4m8_lE zZf(4D@n+D_$M*QyOcrWDJqS+bfQa6Ng&+m@9c#JYC3;XL4;3htRFYc;L1U~8wLV`9 z#jMQ_JtA-(oif6CE0I&Utd-%`wbEq)<5x*HOrCH*HAKy;rwg{ykMM0`jA~!U$F+rH z*$8a5;mp9!26@MM%vTUe%gSHA43{F020HFF46%2U5DM;(pjU9+csa@&pot|OL%%w#46ji23&yP%Ij#oSM#dxGyWDcwmp`eJ{C7NKwd%tlv1BT zqKrl=FQX3N+eO|)#lkpZQB}hd%qUEMNmSHHqqYtzlFuyQws&9_Ce#nVnTARDE{@FH zskJ^=0ly>YMi5`!oq<^Vz)*6oKwc#_%D^t1^}ywY%D@H9fglsm&E%+iCim7dz<3E`|i+h*wSvN zTa=Gsz^&?X@Fb8~Hv!lxpi97A@gZDDzUM(!ltEBz`P8N#zB*{L8Ln{VZdm29eIV)< zF_c73acUfUgzk)aq4B_*No5^jB~8Min0=fYSK#3Mn1DCElKRaZZk66%Nf)0^N7<7o zdpCv5)1C$fQ==4-Cv*btKEO;^(|2#ys~*LP3|Wn7m$vTE9p<*?vzez0_t$)jbzt1DM

ocF?=ZS_~IWO|2Ke?$!?g(2;5$?c<4;WvNER(L#b<+j*Dr~Kc_;LeVi}Ztvr9u5{(l23d@wt?CBR1h-D#f8|YU8CY?m?`3O?Ex;J`>IRjTUyIu!;{k$#l-+Qwy-NJW#@yNt>Xx191RA7$ z(4@?v1pdqRS+-T=kl}0K$s_Sp_B%YkpzKTc{<}H;NeQR*!7%~M)LeN{uU@+^i?9mX z3Zvlb`=KA1U@p^0*9|;OK)XH+c$(v57Lia&T8DO#5P8?Squ=t<`gk#C$lpSkWaNme z1W^}ti{!tnTY_v&w!#yUeN@y4M`-l`u4yAP^`A=qjWFEKm}Uf4xLW3ptge5=uCmS9 zdrg~l34V`Wm0++{;!QAm;AbHTz$LCH8Ohd7glzG7?E`x~mAGyo+eigJ3L!@`gz(&r(WcY z=Vp-tcIsRE6$`!W!>K6JCz?m^$^l0trrr%+mgcQpoarzl-Z`R$=mXEl-mJ=#%Xb(t z+!p?r@+_d##eA6~^4jn!SyIU-p%1rjeCG3N8wf)qG5Pf5cIy{=e@a-?rid^F$7QV+ z0m4Y!NW1;O0$(EeA|YYE;B{pz8ldlUJ2nMZQWos{Hur-{$w|8!ZBzbQfSG{(mVuCP z%;6scED=iL!z6128yAt7UH}78r>(le+C{x4G>XMEx)dUi380*>el6#JG;3OixT>Ba zQrv`460XGHZ<+gkgGB%3t*5{xv0`dA-8$Xo&^eMyNIe%gIVBheljf(1ATord(^xXeiv zXdbUFUU16HL-2MB5FLLwh2^_*zx`hDUPYxu^)+V0>f=eYu=DT=+oghCb;5K(7C`1%|VsgeBB<{x>L!Z;#y+2uchd$=f(ajSguRr?H14^P| zK!r@+#_KBGVZxhFm!FaQ)Jsi;f!c8CZfC%KsiVQC2aLImhy02ZxurV2?HB}b&O2MQ zNn^GfJ1AGfiiQCzwB59B+BpwQWbZSsj*`Z`mJcOL&&}C3&JGxxv4D1nM9gg`D2aYs z{3eHODv3K8#s~YJUf$=5Eiz#UF`JI4R)0uj4Ke9+1gnir2b`~7OL}6KyN1%1@5?${ z%guZ@^Eeg^u*Z^`7wNmd9LN*>OkV_ghCR;7n-s&~(O)v7=Brx;`lBDop(JWr$R&7O zSJ0?%X)s#7&T>%jf?F#LzP`2HYUvM&===zuI|3@OV{uF-5_wIsO<3YZUtK+uV(rNV zr^?T!0T1#oUplzMp!-te?7fAS)CwO3+|5}A8>P!)_VRxAi98UAXi}C?60=#EZ}$s4 zh!M#n^mfMGkIo?xYYjtAx9&erLHyFHb;N<)e9hN}&zt&vU-vFXo{a{uX)p)sD&@5} zFk!&a04_(wbuKN_EWX(G_V}owM(63WsBX4v5!#P9LOx4>l7UFX4oHTQ=p>F+Tw#{- zYI?oo&MncsI!>kgRnY0d|$*_>n7&uyz8bg4NvSt@ur zU}RGQ*YGPVIv;+djjJtyn{Y_+6yye*BN?C8eHLgMk;_nyJiZrzB^!(mf3XV}2r`NV z_kn5hXyLYfRV?Cn9I{oFOK_ONLa|z!ljR_m$^^J@^{40));~s1K~P8Q=G|T>Krw!l zf2ESYPx!S2$Os#dp`s%Z)_ho;X@?kYmR5lDxh0Vi+fl(;U(S(Eh`|R7GO6Nu(FdcN zbl!yvxPc00tP!c7EWDjTEVY4LBX^p<)me2p3{_d^nn94>m!Yx$K27|14NQOS)v9pH zT2cy<6s5)Ocfs?;1tFg}12~HT)m0=H{Sa-@lC1muBsEE7nLYyV`Y|GkujbEFSc#o* z9v8v?DJ>9TI2K%%XKl9CbWN9#reZ&?MOYVJDX0`#8rox*c;#cgI z#$Wz(Dg6UkwPq5qzN-!g#{}#?=VDcqyyd5)Q~vQr6yB+2Ar*JL{rc`hS?75~n%38V z&>r`Mq2^1FyO?6y;7pMem5fteCIxAkYx(>${(bNw#0MbD!w5#%j2VaYhKv<>nTjD? zmx_ljoWZUX(|yf^+N!CDq+G_vNF*5ZpORHm2d= z>Xv;EtZQH^1v0eko(%5sl+K6E{|oi|i-HhOL;J}~X8Tatt)9@ZC+`!!HO^M9mG zK!)@`Vn3weq0_CbMS_)=&ZE(>_0^a+r%^>*Y}=vv-209%*6rB5;R>$VPFqu+PQ0tH zVgyXYTc0RYr}0N7Chi%m^WIq-c7HKj(9-i#W2VS~H&_?IltuM?p^aQTv%WswGu|c5 z9`9IKP-R0W@jKH~RnH9PmwfWBHp<>(yDpTV7N;lAj%tu%o_oqsz!`}cd z_g=ZQvc26O!m*uLcs$f)$LVaBjBA5%>cpmn=642~BijW0=f-C1*QxV>dD+6vk3QVCPGQ&2;tz5^MO0QJW|1qQ+^yaS7% zzXUQJt0;X&h*{NrsVnA^K3;kaOG?h_;Xia87hsTIy`WJp{_G+bv=NG`Nr|o2@RDG6XB#0VfzOa zza|L?SIYwwiuB;%7y%-_kU(QziH-Q=PJll_V?lqYU&=PI77>N}|NMe6|7#4`DH zqi{@lESk9HFUp3hItxH9V$)ZfAup)Fm9jz^K%4e++!P@zl9}b%@e{+R~U7A z^BxF$SyIfZb;=WUn_zcDG-OT1jh@y6DZESGG?6u7 z8Ne8ns+~0&Y*_R0kw+{^wO25|kz(A;g_3O+zu3Bced@R*0D1UNaeu7hu&fWpx^eFr zT#QJ6`1?Xx?=t~GDm?a^%p0(K{VTe>(8>t3mtfG6d3(Q4%P;gnJuFcyk8|q3buU8H z;JNZ_qeSl>AYfv+kD>7tO@{nZi+D>@MrxE>vm$F{R$_*j!rDUE(QzRt82Ri{C0Rf> z{?8`3A0vCG_JqY@9getib$;?!TK0%-Oo#b@rK;OiFl33NjjS)_d4qDl>2})vjJD6Y zB4<+*=h(8ZX%5>=u##J!0)nk@D-!*N%8cvTVWcdpp9-G>^MYo|&#`BVDca@*3Xg65 zzNlLAhsmhk=A`Ntw~6bwoxGV~o+z#Clgt$c!kZ;uS1V!Zu#=YPetH0cK{Sv19qX^xh`&Aw_A<+%9--HOPQ`L$J#jGcd|Fbx&7eS zir`OafuIlo?7VyTk+^L(o)~@AGiTCgM#010^nO9Aj?)}WKs9SGn@k!gQDbe$c_lg8 zYBkbm;Vy@~#HzPK{1dkmGJxBz?vd~ZXK%~frU7AwMCi38%2$Ox?)uL3vWzJub`l=j zQvF@A#7)%hFwo5m@O{SQu8eRKe$y*%VaUkuqrwkbzf^2^Z^!F}S3T*byR=FQZw#Ex ze6}bv&)sP})k$emYWQRTtNODH%KLU9DIukLz#~S_E^kTwGY(5aT#HR~x)yZ_WLiS- zb%d(hzckP_m4c%@68Amg1+Q0*pVu@swe$CW(d+Q;tj{3B$s+NMev&=LK8_dyOJ{yB znGV8SQf&5TncWsMJPhi772u9iVVh;Q5z&ScGNoMLZoI~@qTsIZ+7Y?S?NZ;l1%eX< zW%yv^*04p$tQ0y24x^H|9fP`GDySBF>D z!b0Z3Jrg@CGg$e4f_>)@{RjPj^0>d?e^EaH9D{rQBlCY1fk}X9VZwia0jZNu$38;y z@pg8$J_zq&=Z+3LHXT27)Dr7Hh33PcK`=0AsFO3Xwig=sU+rUo7)&VzzYgfmu%ripU)3%s8F6QFUjtGm)7J2RgsNRZ`L@OglI3X$p1(KqSze@JT8o;2N0nl11J{ zF$H0cdTp73ezB=a?@=UU4B0i08l+bCGC--Rr%TiF{EerC`D%DhtKuiWIfqJQ)!xU1 zs7~pHGfz+8BOz;6&@L%t^4h5cle1DLL@jL#){Z+8^d#Oc${g3)hfH1!sbZ8;(AxZi zMzZ99#sqq9-AB?-M%XbElbJ&tsJJXD_+1=+;9)M~6h?-()XBX>2O?XGSA9i{(B=P0 zGLs%oJbD!G6{J?E7v{j>*dg86;{g1P^U%B%M zP^0t;%*@2yE9@Aavy4peW^e0V)O-*Jh$yX6f{r)ql%i9@1NGG~8riF&j3dcqHLe&{ z3vU|g4oXB*WKmQf+5!d7U)}9zEkDT~_GN0%f-g=rPeAWYrV+exH=Y6%x$r5IsV}k< z|J;|<>Ex4_zso5vT9dQBKX)^U^lSNb=m~Tc{FLH=DKYL%n;vaa%f6_6gLieeB zwrWaVrNQojy_MU{z_E6E!2hH?oL#!6Pw|h&{R3bG>`kpEpKe!|A9!;H zFI?Q_QxeUPcn~EcdrOyZnP>C93t%8}S{}|;4SHs2=D4`zj_){}>0Rrbgc`ZVDtk0( z!*T-Of90Dse&qR?HJbkmYb1x(Q7rh8WyHz+%*Q!p&%R2yfjKF76lnhMHt*1RQS6#0 zlLCC_oR_7KGkg8)hV6uCr<;-IO(03h}y>!bJbcRBO)y!L*cbW7n+4jP?DED?WK5 z&WD3{*QSn^N5qo~BjB3^25>4VK*xSEp~~eR8#$YjMx8=CgE*mF;`e6mP$KsNLHk&k z9W+3B$B1z_cieTEu8UV#cPQ))X|M#m?Fn8CjaeKmZ&QLEcjn96K7@OAY7^0tAc73i zu_!>@6Ruh$Zu6bt44J}o7iUzF0!y|I>FX@lG2L~>_vtgjzd*`Dq*7K>(^E*LgzA{T zNG8DJu-RpJW2=S7w47>^?RM#zUJk)DE2U~?g; zkJ({}C1EfhZXr=nrSUqs25p6$ISoCSVSb7vnhc*w*hoX%^#)^F$rdTvk7&K`V z;w`}e%AHh#2BB$?Wo%+?S%(%hhS|b`+|f8RvY7P9;nF_-BA#{pBWV7Q^p{Ry1b~c8 zm;IgHa`|?(($ZPeTTfEm>8cxE;A~qEsyo?>w*v}AX-B~)S>oW-7&w}f&sy-G+0R-vpB4#!4$p;fo z7otN+V&xKK&ozJG9_uyNko20sd)Y<}eewJPwT(w1TkqxzJR2;YCTUMJ0xk`}W`{PI2|^zviCcCxV~J=*Q!K@0z~gBdp;zL#w7)r;T! zm;mbKU9nrLaR%C-Tv-+}IXqE6^()7e9d|P}8(Pj>wHO7|d>1Z}PrERNBW>82c!reB zPnVVNKXv5@9A~4^!`b#bE@b;t*MQge!S-!pC5slZb(jOi6!{aRvAW)Yl`vH8Fqf*% zyTKVskUHq-(KIDjjCUUP8%-~m0HS)y51wLtPS%X zDK z3aqY$g99bhQcoG=^YbCV=EK3ESE0d=Vk7)=SVGDI(B(tCcSZZs){K&eh!$GAX794B zmGD6Gf22)71}#5gKcv5#YnuzBkao+sgnK)544!FIKNIL3D2C1tYZyWqBnltyBv98i4imX(Z5_vw=zyUCO{cAs4hdNQSXr_Vi6 z?a`ZRw(uvQ$iL>=x$2+o(HMTQNBcubH^jzyXlVYXIu3;=1V~`P)V^6*;q>4)G&z$^VrT zLcnTmXuhlO#?FiRxs-Wlt)K^JH5)f4`>>e+y2Q(JJhiTOKC8T)ZvJ?5>-C9|bEZ~R zl;Du$yMvJ&Ye1Guf?3%=Aqwb7=_6Nf;#p@3iPWgIUnv2JwT0MH4jSo*8&QwRBh z;ohC^sK1%ht$ts_vwpO;A2;8t1hIV?U-ei-Xl-E75->1ms1-T^gZ{3yVDNtj{Vss= zo$bST*w5s9VJLo#6i9?o;tvS*ZsYhW`3Ouu@UJp|oU<|ld4ozmd^>e-&2l9dhYrk> zRT~1*%h`S0gf1?dbfsEl02u1#t`?dRmXXK0D-uivjRpnQcsurh5Um=WeKtLw^>NtO z4R=6w1NgGgyUHlj*7z4rzSQ#>@O`E}&yM`^-mCTZQmQ2gizgp;hUVPUdTUgrl)t5pgcT|R*@AqlQmVX4GG<`I> zWgl!M1J)#MzRH6fD?8xEA|82aiLa?kZX-rBxs^SU4AhkJN?H+-=Yzqw9EfC z+K+4cSWfUt%10c>ZHxQ>DbeRXx4t{>gx32DEy8Q;#|Z4L(^DI=7SAOaj|Mn&%Hf!m zxMdEWKkXe`C%`dxq3#6SVuoXmaIu`^uADb0LN0M~H>}P-qLZ)l&nxmZk8<|zLe7VNQiT#;gNf`GyK zwX+a8*I{-RBu67;^fJxB_j{c#YT#;$wEM-Z)ePTI3y7DFSD2A;&Bh(WXW`TT55q(Xg1 z5SDayP%wUuRCt9KAqCMLQtV!dP`6#`h!Mnw?AKJKkXQhUW?`W%6ov}@o<3ztkeag0 zJ_A>2zx-Ko|AeZE2alT@b?2bZl2^FIwN)2s6%(NC#nlE5?%Gzx3CDYfVwM;_i@aKB zA4`4e=Y{&IUs8ZR3KDR|3}%9tG$fG^{KHSwF*S-1Rw$Ov6ZC19kS^7g+2(sZ--a$(dL$p0X&%+B}jT&k3`)N$xRIT=7US>cZ0$)j@8|7xRmAx zMPob@aya+x6U(`L!d&uHg+?hENkHE|=}2U4t-VUrBFT@NwA76fRNBhiws~g4^$NA6 z@7j|rF-jTLc%D0?CsyKds`o!${p6$K-35wIOBzv$Dsah6t0MBa zssm$AWddS)Sjl$*?8`nZx(e%Tm-?_>3CMyKaGeA6uRJgUC*B2^O5JQ8&C`)dMN-4l zs^n$MJnUOu-Z34dCH7_Bc|v4;0rDjU;GWw_|19%2@(arVO-@O_5&q8c{JY#2>RVTQkN&Tw5b^1- ze;G7l1a9Umz3_FTMM9P@VEkx!epfZ)xjTH$nG#JTz31CG2O)qzZsKEkVaL5VaDSD! zuDA(3)(6XHblxl7BtlM;A@f7WaVX!6?q}`^DB|XZ6CMorWR{{@O#<6_xA3~%^|W-d znnslU9Y&y9uap=#d-yTU_-=VsY+pgAD?+1@GO)ox8yJMH`A(Q)kS zahfn(+EOi)LsC!)(kEkvE5j$^$AyT0suVCi4~Dfi!q^bQim3d4L!2PgV@t}&+GLb2 zO;DVrJ-NbB*C@PlA+^4C7yaaE2@yaM$V!uh9EJQ@Sb_Eq?}3VM6v-QxO`FQASwv?a zjo;$^t!hX+_*B=ORv%|KAidOZp_<$D0@)E^0LBfEL(|1{kr;96b7@P7DsqMw+nCG5 zKQWuzA{sPAuC`w;GTPK0ZQihonJK0Y!oM8S*HMNvAUBOj1@%xLPIrkDR{G6;?@sB$^YvG{*$(X$d`d`GC@& z==ulz=bH@HnwT+9Ua<%eSYyoya6K0LT*|_Lg0CgT_~_i7yqZiBxEnN@Lmu#0nbD7A zK5yKUg^lfzyq95RJ+^Bh&1?!Va2%)qZQbFo2S4An+7Z%OrPVc5Nd8=Z+oKTgk=tLF z81s(ruHE^SN?&+PfF7l~GhArz){{8E%6wd~E5I2D>5*~p836=B zc#xA!OX30f=aj5d*AFH^G-qbTpF=*}RH`$#M?VInG22bcZl|W_6ZIY^AFV&PSlAX6 zUr|M>cyf$egF#Gm9FF|8GeBuKz%n5f)Um4bMw2V1op|(N3X<*x{eF35#M+_Is#3ul zF{SG|Ma1eX$!I*)`&3vjr44y~!a#>#R%g*V@0L!H^V<$TxA{BM%$CicUJSCCY;78$ z1Gjx$&hrr|87{C8@4t-?P~dsBeUqz4Op3O3tW@k7WZ3Z@bi zX#jNBri@jM5qb(Lu`#~I`N7OEIl4m7+3B(axi?)o{&Tp+Oo!OXx!RPD!OvctWK~G~kr36?E-I#Y1THkXydwC9o8OB|k z1e{_iOXc*4+QGIy`X%P(W$q?=_a(4qXo{Rpg>O{}_Z$~;|Jg_ZwJqTAQAHc&ehT{{ zCdP9ew%N2xOb9n&%;_=_>K0`UF;e2FNvmBGjI9z!X$R2#35#X|k`c8pf5N3m((t3K zi9?GV!Pen?935bbM+B9biqUu*(RmY_8pbbg5{Q&!(=E>9lXAAf5CZ;{#+n*Y0j1f3WDjwv%k z9>)$e27?A*U{DZb6UYqa55auE7#-W$L8&1$K-AKl-RP$zkR(y};-drQYi6ezUwzxL z$Y;eqYbGy_Rrf-SI-STHK3d-2${SqTt4^QvPXW5Cq!p-FB$_mF2iMTG*YqFDv`ZYOR>By;7GYjq) zMgpL|u96bub7o(!1QOe-%jON&?ei==_}^$BDSqyDIG_M+o!z`bmR6lPZGx9udRcJx z<(WAmeB=WF*LwW(8C`SM4Q5&VP&d#$!X2tnSs|2{p3!O9yJZhWP)s#_Qz8X~DM2cB zdtF9vZ(n72?}!9sRimgfUQv%a$tb!MQ2v&P(jM29x4XkjulIDPN&u0wsD!Zq2tq?- z{=ko?jq)rz`t~-=O`t3PqMx4JDbJ8XU&8zwlL@f@oxv+=(bVgeLa0W<=aA!zHQ@05 zK>D5h!`W`Kir0fll;^IVXX>sc8l4%w7d&xaHi3a(hXeeO)z8+wU&Q?3$Qh|XlpcML`JR_O){-_3-M$qQ(W2j`ta5PjfS&`9ISCV%__0 zJ9MI?ucOf$Hhoz-tyV)R0j#=f|J}+BYq&B2qSvD9ur%5UR|dt2Ri0h#FZbv7F3`p* zzXd$JhM=KP0FcbMrfh8cni0yv_m~BVj`9fHkLxhfDv+0_h^MBwBXw*afc0(mLEuvd zYs|b@GM>k-dih9T%;|j}-N+4XkuXrJXVCa#?k|_=!>=i``^B&OUh4|Z=+Vo|IKn0e z4L@r|pk8Es?lxM#4lp0KsV8>5FBA77Uo=mP46$?}Cw?#A#F=Z1Z`5EqQQD@6jlrE5`KHCC$JYGjX<0SQ8C_~qe( zRyWlr;&}wYzWg07Hk^n!8QQ`l7Mt{zZy^^~zODL**sedDng7kj6-H?Dh*>Z@n>F7k z)3KdtIJ-Pm)fx=v3(beM1%c-OlScFUC-xlze3$x10&;Ohy6#=nhVg>Mhr9ifLW)wi zPe^V15YRucACRq-vNEF5qXiJK-$!S}B`CBLS*A?dD+;g8-BeU2KPT19=6tu-MG4xz zWS5qPUVNNf+2#(!i<7r&nH^@^sq%Jg3OTCSW>uTc!Ohn!$4fSK2pu<( zH&|M&yh_;|f#*sIA|V%GL8(&kzUwvO5%sw493G5gni0s0|9pb?UhDc~Ave74V``Rb z1D1H}eXoU+DKJsXDQUN(wlVzBm7ULr=O{to^J7op@R5O6$OLGmF)JmMim1&>QR`+6 zkBvhxm(qQW*39gE%nIfj?ar=K1}CS?NK$G!yKA3zX%q>|vUF`yf{dO@XqOW9ez=Mq zy?RQdy^>ShgKm$NE}0&#8|> zG=m?KC2>ut3{$2 zFC$pqSp-FA6s*RNI+Z!}dmR?6Kd=J^KE6=BlkBRU$Tf|g^O2$trFulAd)dfTFxkjy z`5GmNxY8>-QLA}_V5lI{W8I&I-lXHLb5kK*Z;`X>`MHm-%fj=?$=DgeCLFjq+A+UW ze7%X+snT?#@mq0qzl%NO;y7##YZ;x6SD)~A)tpza7A{W-+2?NjE%NmX4sNPZ; zoZ5s?>K2X7sP401%!1W=jm~k&jKicQHhWxbFCc5YA8nWh>Xsm~<@w>#zIUPCiw{K{ z$hvx)QCCxfENJm9=^RhR_Zcb53#Xp9cZ0^jJD#DKZ{Nb%dh%&kGvPjurUa22l)F)1 z)ais?QS9oRZdYvPVwMTP<&?UzSUDXL=0&f;=-nC)w~{!uHOr4G$P*%~%7W(Aq$wsa z<`y#hGP;KXbU5<4d(21EkaOAOYT?QM&=rXb!#a5Vml=BE>wbr;o%*$ybcy^HX z%ch0P>~+-0sId}l$;4)y+HO`6a+ILAV_{v`FBLs@Zmm@`MIDkh9WBukUFu=5Y2e)2 z;=G;@!Z=AQu5JOjsoCgVJ*;uBb1$6<$j``TcxyH)lw7s}V>~bdRK4%l{8{RsUKmeM zqd4i9qMW;}4omkN%xaI@Hu7xx0%+GuhBy=+-R>c`_Rd-B^mQq>#-M}~AH^t78HDFP zraD1@+>!Zq+>Q8r5@!6-aQd?-v?!?mk=Ys;jd|6w({_@7A{(t1yj@qHd=awm>`~Fk41KF z*PC0ybL*4Mfv7vf|l-pSv%AM znM-#3UbadECv(o*tg&8i@C&*g^LYt!2JT%(S~)4`uA_ZE^zePSm)O+|6uSE09j!cbM@8`MeCY&|0rUoVyR$=i)%oTz)z`;y-wvD+eikA@kHR_O zP=@xFf@E zJMDb}Im7(dfD*u+>Iiq^T%0Y8Op}76n7yX{hCvP28{V92J2C58;N!k~-xgod=dc&w zI`iNyp{vkc!V%hg*6H&C8}ogu076Ki#zESbR}=sK{fF}l5K3YdThCCnG$SyNEx@<% z+%kEj<5>Lug1U=j4aB>P+M)&;onN=KIEn)XLT-s%h*+19qN@9!N$qFlh3z97)^2oZ zDj}Rv5x&-k_{HB&ju5eL!LCd_{J-&wMgOh~99*Mc->Z2eX&3cAif?l29p2t)EqLROcw+ zEXo$f%XZ&PMz(8uL7*>e|2#C*@|-=)yoYyc{3!)!J>ccLEgHp29aOIstD_FQq0b^z znoa$*R#Hbhhb24N*x2+ubcDF>lomM~Qx7_xyu$CkgXw{pK01M`!BnoFO9|pa_YN_# z(1q;jgTyal+o(S+#)CX4_Ewg8{T>Bk4k6IrpfxjMuB6d zBuPgpc@b}5(B&Q#jfc9O=ab8s1sz=`)e zqjBE7kw`X)oK%~^s2pvxjpHJ$|Gp!YbuhTtm(r0UM;g{?&+>0-O}_NqO;;!*`h(=K z;eqU))7%BSO+~pnt)8Q1G}+T{dzQLuCpP3YW(Y~FTnNFj0w$|{GgZh(%4f2kr^5Q5 zn`jLXYi8pJb*Pu|7FrLvk|Afd--5sJ-P$mHmN3wB&tTz^w(S4^%=T+1e1B$JDUB(P zW07-E&@!4UT_N`1KH;b1AA1Wr{i{HZri}d6?Oz;!eCrD6=lw6!|KiM6h~Hh~QZT@m zuf3XPNQ4;L%P&oIY4ZTHZL^slE=x8MFtaXhew?zEE=ZPm?`6LDGnD`}gNIrs$@kk< z12HY{P@N#MeY>WFU_l0!w8WBye8P0YFlEgfJ{s0^$ED^YlpYy+X zd3qDS;{ToplK5R7CBqo6!M#=?8^QWJ3TnFjGHMi%&m@XVXYfDMgh!#~|4sStLW(f* ze#eK^k1)zWLw?W+SahA0dmiDef&9K}|E1BE8O%h>n|?1QFXTC+yu+{(jed24fcS)@ z_q#_<2&oc|C}TEMkpu|Mot;&s_(Yj}m*Jm@@9OtYHks>MPQaig9|rd{*AR#h z%jBQm%a<66Sa?MX`dsrk!W#9KTF%&X3d>x@rPuolJBY@XuH84U_h7Pz7h!DCv5iJu z%?3;G&ca;8UY+FP$5b2qU*T0FfJe;L?*h5e&n97#uzm+ z{!qJbDz&(PSb0TDayechv$|KJJhk5gee-wn838u#;XIxi$$6fO@#c0ulFKg~;pbQo zaQmKWZ#{pE^d=k-VbsxXb-r(Yl&JDZsI2`Ajr+eoS zcx;e)*CL;OKX7W;}P1 zvl6l9@LEZJaQpUMscZ$5KIdL-6^)J%fL-S0>fp8Kui8hbGi&^+m`k!d-b;qU2aXEo z@48i2U>q0X{aM$5;yJKAW74%Zlgh4c{!5=+!!dp}Luh8fzzEC?`J6q^5)4>b-*vWD(<5y8Xxkat)#gr!vD_pnEtU-! zQHc8_mFwPUq9!?yXfmWpW%F~0AsEMb!lztG`ovG_csu}<{LOIXEWxQS&SXzGBPg-9 zDM;joabv~8lorB}qKOt9V+2Nbei58}WUXEw4L8rG{FafWv+Tp0zcVCiSf)i#d zPE{fLOfp1MaK9I96t)wTQz(8M_w`;|;7_9@3a?We%qS9I@+z5)TXaKrcFS zIOv}@IZ}`DuqHGw8$Soa(%9UNkK?|@fPShHV!EGb#XT4*`5)iA{ORfh8Dh|VYS^r+i$nR6Y6s&U>SS$S z6RGyYd1(HB4`$$zLYId@128Zs2olUN=C_#2Eb2OSj(ukr(fG(L#%;Qd=@RA1G-W)m zj0ID&UaW3FN-?J&pPf>`BGn7&)|HG6rk3JhL;L&Arq6rfO-Q-f*4z)#s_xVxeV zFD@H>(4Nd$k8G_TKp8t>%E`8x#NFXVw;Q7w<02uPuHZ!*?MOk*);8*-f+O0QbvC2~ zxmzH}@7yX0@~tCS_YpTBO5`T2X+vH1RjfdMelz1WhNuAh%swA)4*3+x0x#g0oQpXN zQl?@NzA?kDJwEM7NCDdD%)Hml%pLxQ0yUFrEb+<(sVUZ@Zdv=3IC7LxFIE#olP)wr zIQEnr@;i3-DJ9EjkkoPC1Vm$>C*k&@uh`pDfGDCH$KLGBRnS*H$V;C3EVIH=q4T`- z;x$(;sZ0{&>zY0psM60^@<*lBv!WLgDq?u%mfoOEAX zf_oH+@_A@O+>81}-=I64>V)BF+HkG^agp9vV+Qdof-~N`&qdEz)Ngt#a~TlhZUw|U zaYLK{8FDcPD8u+4k&v4lhK>AiXR9&fwe*cO>P(Vt@6u}4Mn1cgnoNSfc41BdHiWrOt(2>ALre3dF? zPc}ahixi#&`sH0qLOJ@KhD?Avdr2q8#%UK;hKnH5R~_5-QJ{2IVeeg^q98A&+jEO`?`=NPtFNQ z(?952!wtgz20<7KxzTLYdpw)E z5R-hR!YRM9du)&;zNLKnY9czm1ST@7{> z#0@OTb-E>O%{9Jt;_tjZNAqm*!NsYi%NTywQFsQMz8^sUNElZ`myp!&I($U4!DEpaOnf&Y%wa2Wd?)=gxvMXzKltAQe1hMZbPcbb=K@#OC!A{_ zGw$ZWqhcElV!{$9T|05q$U1JQd7k9Lyaa2L@Lhn;*)1x4nF&fRJ$vA*QXq8cR;kbDwaNg2{ zYC=&)Dmrg|@G;W+7%3}BT;`tRgaCN*ukDB8LkfP9AQ6&le7wCuY?_qhP1=+Zq9rHx#vcxhC!eF^#(eD46`eD`V|4$IO{3O6Zm4vZzH8WA7;Q5@`jzv{~KOz z`SD{7zw;n|H;n$-;@<~K|H=O~0(r~5{nt4#?S95bV&QX5Ue5c@gZf$C{~<1;Afe>- zv$C+{-^A4*<+wt^W3C0SGmXv{@nml%>u2S^;EVv`5JsiTv5Y^rWelqG9`TraWi*1* zO=8A780Dr7#d3!L59C#u!Yf;-9IMNst{vAF1z#Gqpy=Kr5%I^iYd72GSacje%yv3se;ft5<08K-8|pL$T@<>aqqQaB+Ev z-+eaj?>_%G6i8-23Nht3Ica~x2cY?(wJsZUX~LmcM}!JYkx^0fk}a6f{2%FGObwV# za3}sZLO0rek-N zMdu)dT_PSmmu91|{me@UO{!tc2a=5l8~nsd4DD}9a#adq7rJdh2*=r0@`&c)`{Cf| zG4C&-{C4yvU?5UVMYUF`7Si*kiD}FXYkr)oJUTGGZWRBl6?@~rd1LGEUPywD%^zIOoZG<#eKuTxgtH2?y({4%uYL|N2Bol=b|0_063U% zs^o)yFkHj=AaH*3Su0GuPj5}fx;%HVNiEq@6F@W64GDVziYL=mS$OST*vPn*OEM4< zX(At2XQ^kc@a{PB@hr!nq}rhiuc=O|&IXCIAqde|)6ugoK5Gb_adA5F2NfzS>WW`d zrtAxXZc|2z96yL&P3&!b(%X2E$ST+{Aq$y22gFZ@K!xkVe_T&!q*a`{Xe@r*BoQ_f` z+mk7IKOcnXP1oecpe>FktBCL8jVLLvl%lkii`+^tB-59iE8di6ATo&M>uMZ2<3f^} z{R#5%E@AZB>5=5+C|RMbvi`4nAjEsg{vREv)m&9qmsID_(VmLu0Z?+!1PWQhR?(Fi$CW>z6%#5A&;^|(tq zP@XO6U`>Z?fH>4@Ch&{V1hQ1#jZY~nkf1#WgM9A%sMX2+Pi-caA^|-O-kNbmhvz0! z1OrNDy1YGTN1To5hLa8>>2^w@G^ zRD<_6aPWl6`X~&w32Nk_9cSi$RBrJ3wv{SV`Nf!jk+%eeXJTNvBq=oc(+iVpVte0* zs-d1?&H2B_>gTs3ARrMh@q{Uz_+8*cwGon|1<_O>gg1A&)vKaKbZ+5c&=eES`cN8m zvF!lV8gGl0-6iUui|9+&7n%+E&Ib|ltR*cC99#wzgh%K9g4!LuumekdedAXVW-4=+ z44KC#W?cvo?2z^GGhikZ0tP0py^n^d_>neo(R9XQ{XlIHd0b)sH62JT75?;E+#m?B z>kL+}p@>2Sapm%01-ZG_&u+-4c!R(Gsv-e5e~^x4BmRf) zEUYGxahlk7yd%3{l%hdQ749k~ipC7VpRNI~;%{ju48 zDd_)Q?U6xGon*_V1d#z{URr05DtEGbOkvC-4`wa~Pu4ev^gw{ui8@qrFpV&Lb+B(Z zSodi8cc$)S0I1LLEG&?XX#~#5p>6yN~RxH$#+*)P~KQDjLIE z?+h$3T>3?AIe=4W{nK};p!la2P zxvp)2O|g;i6!pIh|5i>v@A+)^i}y?L5vHsYBSzY3zlT>lO8!=yfLYqR^IB0W}^wBlPH~M*F%!QnI42YRIkOvtAF#g&mR%4Bci|`Z6%@IYJh}OF1ZD`_j1ll}zO| z4s-t?XcWyEPkE7lF$UQ07D~Lxub(cIMJmgOhBMrKjS`Qq>T-`ch-{{SfhRYW99y9G z`^<~ki;l7G_#py>Q->w@AD1(gB!K<4AXX5M~N>k2mSmQAbUdX z>Ix{5*`uuXtq*7@iO16^bLq%?fHb1{Ent#8Ya+wE?8Eq=sTlp0ny`<>H%PSxBAU}s z0UujV#x5*(tI63mF^-;V?F~luo0zf)0yq4cyY-2cpeN$Un0U-VzN3`#w)Li(aPs-6 zhK#i>!`U%|TXTU?78xB^pf-~e#$TTQe*^?%h=FRWt!)5&zb4Gy-gGPu9g=(XnFQtY zu!LdRK79Yr@QI9j1t_b@eZSDS?=-`-s5$a6qA6x?;$UN{B5waf1C#7gClnyMF&}jk zM33%XPz;^kvEc9+CU~s_M%$ayNTXd#Y4n4Ye?u&HKi#&IYHZ^h%^Qp=$3{8b&lNql zS@c~#&(8-V4h-6(G*F#HWF8w(=iIEPZ^wPK`L+;$8>#PF_6W-pn&)2_(yhwBXUGEP zHVYdZjE#7&+IqwqOrH-ukUM0>n>oH>K^NXl9KTKgH8(wboKxLz{qlW#B|KwR3D%W2 zk#q&qif`$9mS&7r=8{8($D-U73)TUhv7x0nfkWy2S%#RWg%zk)1M)iZASvG|z;TG1 zX#w>grBqHb4xN;JOJCXGwI*PYswtaw$e%_N7qK~B3?}fjyo>MyR(=Nzj6v%s=IR=6 zkD6hM7VKG9P|CVCTYxk-W_hPtBk5CH0I8Il`%=d5X$#8b-?nVHnbIktThEufDkHCq z5i&R7T0=v9TTfSp*+m(v9_sP1*=hJTH^dI3MIg{fIQxl`Pj&z+;oQ$hd3p_^oFANB zS3c3~b>8;g=StyZQEGVmzUFGt2CsRTpi`hw@h4g}7SD~lhX6zHEP681oh3RUt*O?- zjzKM0{OuiCJ(yG@RP%mXj|`fB<7|`v+vX+Wc#UC=X42H%yY8sv5z}CXVR3T20x`X# zk|W!tbtWW$G0SF_Pw-CNEdxV^GO6ef0g}_+^o{={X$Q6&K+`82OuYIK(kmn8(fC=4^oRms5t zC&`r=_fKc_U-M#-7I>6(xBRyW4`JI6z#l=C zr4E#Y<%l#a?O)FKN1gl9hEFvS!X4HF4v(Dt0Ob0DShg@n~@p>h2SzFTsBV_m;ZU_c%siTJR za$?y?61nK|o0OqeYXuWKn40U%9{3-6v2erV$!ZIjo}7_+(?|?|!+BI?R_hvD27LeD zmu)fr#ncObuTvN}rwy(NrsjGh#Smn-6WqT2LDc7+O6=E9<8F<o ztE9>_oiRz5pgq89rg9=ICzY+a>S_+w>YL~ z1M48(2(}A@8ND96?Vtn#p05;vi#J|4zMO{2%CDw9{2H5zArUaqcjTwEI`Ka7O`G-+ zsNsQy7a&8b2|~=Q)7v>BRa#fMnuV?eB7{Ed3=Mo`zBH5Omyn;LLGAo7p<{0g+jTo) zNUgk`#>5Bz&S7@P*Z?u)Uf=zMysZ@wVoCS5hQb3p`+2h1mX|V#m)we9HxwnL_=UU& zlcM%>7TC9#S0Dy?yh>d(iQEnCc){$7(ni?00=mp5R>xLqBe6Sc2K*E##nl^LSE55c%IENrAMyHsIyg$TQaOXZIz%!J|sidm=f%vdKi1|ff zv&vVdW|qAs)R(skUgu;fiyR<1`yrma`H>pf&Xos)5v2v0?=3%;|34b&^7lhH|5Yyh zH-cC%{9Z88Lzl-hE+);hqktpS|84M316{xXX}P@Gs?kf9kyvBR;2?)@c(lxtFNMDB zzJ~>*gY&EM0A-mi^{IolxBAMIW$TX_b@0A*6G^e~jMm=9D=P_DPlx9F*8cHj_bcy@ zW$w(ewTWoEEUCOUWCTqjz(&JOIL845??897lHB%T1il?kZfw#c_PFe|ixwZ+%77BG zbfP{@M;H{{(TnDCJQ5b$S5+787fVz$(yfh3BzZP5JcZ{1Xbtw0hVW3T7TL%hiqsz0 zglXWc^Fyen=n@q0zW3z7oM{VGPPq_N(d1aAr2Bt>MdWzJWGzJ2n<+Bw^Rq+pTn4ZsD zC8J^@T!YCX#v#ddp2@6lWoS@AS@wOb)73adj%M^okTHRyEykvO!1Hh`P@2HDPoOA#AcS)7gKbSzUVIe=(`zR!Z>)v*Xta_ z`u8kma+r6CCz>nKG9E@=yDx-d1jvKQ9dRCr%JR6!)8xg>0UlsgDoF8C`~g-EB`t8Z3z;M@P6+l|tBEJTaTJFX25ffe`z!srm zk8!WyRVLRnJ1}jwG^~)SAL8>;Reet8)h$AX(Vi){67z5~h|233(&BoV1${AGYn&r$`+D)A8#iSc*4E(x&dfhh+x4KE@8WJQhbbi1r6Jq`xxbQ@9 z-Z1$7!l5oIn+9J))h?`|#ln{e3^2nDeE%;ndH{q9xRQ?ES0&)29B+6d@Tx3-A!_Mtk4XB?4*n6?X7+$NC-G|cfhc_&E1Z4qjIN(+yX ztjl7l!4K9SO;+q9AV1QPjSuwWRCW*Z@n1e1{(1#KtmZbQnsIq(W$A9xZfullc8!6r zbPq#WQ6O!sDBv+3fNDfz8omR(PY&oB1=gQ!Ee=XUYE+{=GPY}rBjPq3p_ZoxofCzF z2G}QiT=@$p@sRE|jg0&c>~Q>ypts?U1;jN^VKiP^ha_G_V5~Pvg3Bar|5!Wyj+&jr z_^cL^T07brIYJeHI-z^8>^rOHbfD}X*^h3}L2@}^&9MB-Bi)ZPp{s{l&?wmFl2r>yno)e2d1gpF?ch+B@)cZ$_f3p05Ml#%~RF@THw?YFcu zZlcIbCXec36PT?KaAOvs>0AMJOfa9R%03eQ5VU^3=)Q=HHySSoW(C@6>t5?ypwLIj>ygpZN}e z0c6a?%T3TK96wI|W@m;FVuI1Jz>i3Ltbljqlre*7Gr&uqT}$}hft|ngQJ%Awj`@*8 zF`clS=(oF5(=Slj1E^q587OV1M~Vp2kNwWo+H)z}?VY;)D4>y6afm%-FFmeEkY$lZAGJ<$&CP z4~>BGL;lgmA$)5a0TrOVq__98_$IS>%ry;xng(d_z0-|dAB#l;Zni#Ew=C(dY4n_X z@T`%~b^qwMzGK+JVlA0ruZ9(LUpakR`Zyz$#eGb z-i$EGp)b1kN>+PK z*LR7gHZ~fY>T80LuRmTgbcWSuhWUw$dc2F~r6_v-dVH=k($n+C8suYc-G|DwN`w0C zs2xhcUR$|hO1+lujco8J`xNz)POQWiJ7}?XsKGt`z`akKdEqK p?AHROnA@!4t`Sr25?ojeUi=j*1+sB!^NCJO88UAiG9ugS{|95bC2&>b2_eyPfu{RVr4cldN^h zcoFa$PU%(Ducc8r>LYb9BnL;MnyXYA9Xt)A$08w8mZmHTy?Jcxf|(_fJXSKDEyHRr zI_4dcCpx7If9GA3Ex8_?S<>tH_MWK~W^P)%( zGPZO51re22Uhd?0`CS6zH;`bs>(sr-var}*KCRb%fBv$;@pcZ+J`oJC6+G3Lvq_~@ zXHti#Epxz;`X5hGf-PF*`)do8Xp4E(BwEbX`>Zmg;n&n%SCvn>;x6tFSV>dGMwm23p-25LN*Ty%h(e++A z#+}U9>~yw4i?!`%S6r7yFyPDpwpd!c{c1FWHQU>mJW@K1{TCDkuQlr$XtFIZ?_%`~ zfqAXu6nZ>30`po0%_^Wu=Pu2uRG0__{t;nRSx{7ld2NfAw??cVQa&mow$S{#<-)$@ zTfbv04<$(HQJ;FAwY(0NS?xNq0r%m1TJD%7qj`x)%d3RPs8D<{&))yvjFf)cxET>u z4?)(!t*2HD51GV3fpe%-99dm1$s>iy`{78FVI#G0;m2hS2R^89WJ21t0=Ktht*`Ah zl$Uh`z-G(dIg-juTmEt5A(cz7;i|(=mxd7xI5ULv2Sg;G_+s8W*yNGYZR|h4EW{U! zJX1}!1r}fQ&0uZNuk?8#hV)5@k^K)AK$18(R1&TTBrd2Hp&3KabqfS*hI#G1>qWe0 zd5!+w`lSzqXKUvGwdR$cyU3&|Eu68_pkvc>AezqXKU@#*PbTZp#E$dl^aMevAwbgJi7p;1GZL(? zi$u}ilMyI#RguxaE+N`o*}A%S$MVxgUh4QN`reEKHTS-*v1OvEPf)uzUNFv?d$s7L zXUDU*PnF8Maeu%(UX+cp=LL(b9Mq^#u~@6a8(L4QUOK4$wTlD;#{9T>t&&cWPY@-wYj z$&?YHQAIaa+UfCqrj31Dze3Lvtg8;`b3Da|PfSZaInUdCPwNj?!l$S9xoQ49Aam6^ zR*jFd#oM+$>lneqrld5YK;{;uc3*6|sM5XS?KbZ|-TIyHOkN`D=4Qd3o1Z3t2Qo|O zvQ0b^VN`bIS4jsfs->`9nOmp0yl2~=8fkCyBg&O}H{k4(zQsnX)>i0~&^!0n>u>h& zGS!ET%HBTtT+YpRVyuhzTA#PLL%%Jd+(9=3%BedRuD``)a+&8FQ-+7q8cyVM*T@sT z@cw}Oxl88zJTTC8>(qxayQ8Nk*;^Jf+mQD4z~z0c?qHy@x&5)JTjot|wQK7zDqW?r zsW5a>;2nqf3Mr{n(ZDWo(-JzY?h{tOTjNVNC#Dwa@Ywl6g>GZ_U4MAF)C6_wcy}Ie z+UTIs?6Qx;n=l2IT#jBxF)W>~U{RRQyt*en%o_+Bu(}S30Vp`lbtXE(q&we54r3Vz z0a#?U zfoM&o%5i$zi&(V9pZSyNk5hFT`!6U8k2~yr|30u(fhD!3yE??d=N5eir!(4WaYC7M zIFq$-a~9?@PN2h#fL0T8i^F-P0>Po;Si~a{Q30Z`Nse%^oX%qXk>pH3I zm%9}w>@5Fe_^ahzqi24+9!E6&@0Ft++$2QMv`3ESXn!qzMXh#}U4sTX=G;5Q4Nk4S#U>_bjE7SASXE5U}pDz-pTIvfUg5HS$o-;X%Csb zFm*XqwXa=nw&aQz&f`k$$lSjERqOu4hw!o>0GQxt&MuF)jsIt3E3gpFAV>Sx%F#?j zbWiE3qPZ5ei(0f`o*4R*@*z6zLiQ zVbD=T4K(DTBTNPk5>2X96GWO;eH}zJ_W9txuO|0CmON?E-k6Clp03mKo-x(YUMB{P z)b4$Dc=map!3Vb4%)M%JX)=#%Fmd#{#AkgyyRLIw)SziV^8E2J;mZgHg;~AldCl=Q zN0&UB0F*1~I49q>B-PMj z&T0A4^kXSm_rJ6xEasfvM*p>ixIda@O{{1O_J_w@=jWJMPsHNEh53OdmC$F1ndIOM zu%XaM1Y#XdvKg|K^nQ++^oB|NhrXLWL{2i>o{vkNcx=GXvB^GDd0bY%y{cJN&rN;1 zq0Z>kij~})inlKGlwiP_5&S3-crDl;b#8sfWUtjZgi5xJj}%D}N-T*4t04}*^O00k zB25gq#l%Vg$_?Jc2rdJK3>&H9e)|HK*FNMG9F{$YN7v92`HSta;Blc}gIqREIWfSo z?mo`9^wHPZ0y|$SPcY!j5FS=86#OaxXXBhVC7C=@x{dwkmx0$lRfo7-TlTMQ!5A_I z;ctptO)c@#v1bqaKlzS<6LI=O!3x0+$_BVj(T?sZss$n=7*?|{0<{FZ}fEN%!}=zEK9Z`q2Yt?YE^gg z)M{GM(R+1)Pl*Hr&Md(M1FZ!E8=F#nmC8y1sgN4^eb$0&|Mgf?oBxkO3eh_2&5I`3 zfAB!YO4$&^00MM9Vs0R2NWl}y!QmtfR;HDRB(%gBX<&v$>$)Rnw*P$m zr02ed&zm>PaemyWvVlQ7E-|^`+K;PUgI{M$9Cc&Bx8$Zxo3A0?{lu98tO+ifeXw79#lG&u zvd7v4T1PfZZe{B4tC=TCZg*xUX6v%wzQ@RqL6`ISM^)kl*}`@3-FANGK2=i{2})>J z)+2CHsJeS;fSOnSG%bK=QbMLrHb;2-tCvB{3d z>T^I7pT!xK?d0T?rJ3i%U9Wp>wHTJ$Gd9g{V?hJT0UK6scx8S_nOER(7p>u+e%xha zU87E+j@H#bG;mc0ih$JblD4P^_IdLfJno{21|DV7<1VA!rZE_L+@-h_p>swSB*Png zIz0Ab)8bj*ti5G>>EM*5Qf8@)284l9+AtGF3oH6++&eVD1r8I^5RdrUb5HcT4e4;h&CNsNuZV&u3OI(-nux zP8RK5rdI!$i78zhL@t_WQzKVG;sw`tUGwJO8q#G%WbfiJ5f!E{=W*FIyBLaGcs(jC zk1CZaBB5c^qeB{c`r=l3C7i&b+Z(GQ(~0S8jZZ>l<-y3yD-P9@rWh)iOA@8J4A!L5 z2AGI);t(WUWu)}$3}=*@1}!oeqaMh~7~$zl?Cg~V;%y^F`)K_;Y@~LIQ+6Vu!N(*DjdA1SQUr42|_6s=xCU0 zDK5b(JP2ZR%6dgv4~b$eIT}I$6*vGer~n})aWbe?S2lc{%%lYa%^!GKjt*ee_ek6# zm#P(iogd{SWLKNGs>d|#{ac^!H5eJQQx$v9M`J}W&=%4J1Kt5id#!n#I>dc)W_iUS z*q>wMKw~hFLH`iPj}wPTk^s?#QGu0EhH~UbQos_J$U3mC8jyhGku-?LlZNn0!AW7; z8kL#>>l*M!O+qiUX0hO=e!Xz6otIyQP6{;RQ_o*}sycJdf3B$kYx~_OhkISJJbTjN z`}wb)722FSH)I-*OU{)v*gSjTQq^t0wzFJW!>`JXo4JY-3^;SlwMWGd)*W!puyQ7y zdorpV^^9Dafm9-?WZ^GI9avv}CccNb~FsvUyIM2bM!L(WWkkjCPyG@YBm7RZN;si4kpbo6-7)aH3%i z(nUfB#bsLOc!RP<-Lw|Y>X7`C_K|l;bU%@+qF|RvE=qvLMviRK%|{{$p+YKwoD*Rf zorEN^2pNbzoESfoU**AE&dBcdv`-`GHyG-@hLIofEO>QcBK63j~g%#98!&OFDfR;1bd zR#712)0F!cA>VRO#iu(bFWU147bnB&Gv$}(8^5*ri(w7HpwUEwcN-coOFEp{;y+}z z^Mgd|>1%qo$eh;8CU?;5f|dLln2pcv$IHzDfHr7`A2&&Zg13Od@>J8BbybHHxW@}o zQHd&mm&5oGjDie8qDTdq^uxDUw`Y%k88`IFolvtvJExmEzl_Q@ zsy(h;$gx)I-7j}>)T(ob(kcS&PBO!_^}6`SJ@)8O#Py&2+QP9#&t!kz^?SUkkyD35 zF+GTmC(ZL2Ve{gt=fjt;@@BuEb(F=VtX&BPn)>?)4w`xff_mDw`dgCMPM@$CuOq?v z50+kY+pf4Z2!_9MD<{Oi~AaG-o76P6G6DhQx5D3L~O5*%m6XIyk-9s)tZ2faRz# zQbEV6ZGO;g&OW=?O}W-w&pxN+=#ZVJj#QOJlS}sQ-}ABC@ucaY$DL*m?PWneQa;hu zXZPwRB_=zEogXmbaD3fOPT!{(A1Sz1QTR4`orF;|=R^>CAgk8D*Fdsx>k=6mF&8pr zPOuP>-ta>kiS0}7&O#JQup<0pIdG9j%CN;n7KH&oumJE3fWTimst8*ZM>fo`c(t`; zl`|(=wAVmFz3cbGK#m3K^da)(71vL;5Iy3?~r^5!x;P(w}~Y$i5aLNCe3)G%F)(H2K?xIpMMeyt!#7l zTxtg0^o<&ZsvKc3tYEugMyGRaotuGmE3M(5Ubm{RUHm+$h)=nX^XxEbIGB0rzRexn zqpLOnTNO0Xz@toh=4`avGzLRk6^cvYDrY1t8$M&6Inz4F8`4*jt+kw|4k>o9guDq0 zDsaStoDrK)t56VvB_VPUkxgWRnYFMeXD}25)v#S9F$xG8R9?Igqog_b8a~t$smwAW8$HunOa}9l-`8vI>3q}6AG^G?UWsMfFO;`@ z?rUmfYtm_4T18G7kx(k#Aq+RCeSH}eDo z&J5vU=7;WQQ1CD+yLYCXUY7T*s#d+>rFM5Im1)an`C40ng?z7vkuvyflwqGwUOWP8;VZvJn z?OpFrCwN>bTIWn-FzWQUxW$vK12*m{o)!2sReKW;VHwr_=ad^BN^AJ%eY)7`OW*oF z;F$Y6a9b#_*`TysH>!JDhB0rM7B_S!r%L%|=up+b2~(P{tdsL-<|{P^dZ(Vv zR_#TJHTjzLx~z^$IQzO)Xl7pO0X4Td`;lNsIaAbmq?7m9#Ay~GQHz*MZ_*y!CKzas z&Bzyq&xZG2-2O_;vDY=aS}Z)ZM*AUg*+;5u1?i%IBqVhoZ6=o49)vq0OR~~Sh9V^g z+5i2&CD0Y>NXSD%1Qr(OOh3lqoFt59M25Y^$CD&EXIFr&tYhtWGc}%nre`8dd4)->$(J@ayA@*O#!MRY#&XUhP%+!=$ZE z<2O$CH}&T}O@Bi6Pbr7a^HwN}9+yf^z$(2+7gBBvV0geWOQoy?dPd$W=1^P2K&E@1 zR}n4LRLZ&#@F7tk*1FUgU1nJyk-*6ezwtb8r(;#y-#Opz&8c-Rnq@_b1|Clc-pb30 zP=N}0e%U$9?%N32MwR=C^Sl-Epf^7%{WA&1BSz^Vr-0-+Ol&A|790kH4#%C5a55kv zh@68Z=%ns4+Fyp?{Q|bd3K$M#g$0m)rD-RIG6vQl!WvvF;YoP&UN3I9dw#qes;S=8C7cUL?kq9dUQzBnt{(D zRT(n{Ehwaqw&_>mAPUSTF~Lctg0n?9Si`}hlpqb!B?SCc!6|hCnw&=v!AUhpl2l8; za(Ighj6wJWW-z=6Qc!YmfG@$o&mm>R*%jIe6CPJ*wRf(&O{=fDS*ui&x=#)}hfa(R zSV%Bn%<8!KnzyU?f(dN!QIAO{`ivf3!DOp49T{`%s`H`x8T94)xH6 zZJ&V$w`t!C7dqr|av!cR2}`ymu|g^Tp~?dXzT8kbBEiOwMpvnR_)t${w9VlCy$tc2 zL24AAZ=8gbb76{0MQM&oW%3Tl>CSq@W-c>N5hf=C5u7sZq$v*3=_WntaGQZ6t5DAQ z0S1nhxc<;KgL+!)rvsgS{M4}P_wcHi3JOLFWvWcBVPFWCU*zjvEK z>OR_LsJ834;D~sv7If1aVO(VlW-u5Bb@507?%+7H&Z}S~3EkTy6ej`3bEQvEtm6B6 zHPJQ$ZN&|i9VVSMS$;9P>ybxKCycM@-8#>be*JhaGxE19HRS%#>9T5RFQ_3yuQMb< z=rs$Gia8LAvGFD7psqA@`XK_fj@ncpRHR;X)S$*GYDI`dBlWdqFw@rI)%Yb_+9&QV z)j7>_0C?%j#OI8k-n_r^fmyold7k8zZTkdtuG=hMJ{}kQq4R?!RcBSt^Sx#dc2~~) zXJ6Po%0@6?%mGy!=gm5_5Bw@E_)=K);oVFgDSe|V%?lqBk<72)4MKp`FgFXUK{5vc zlsGZh&2&dfm#&m4;xcUOZds}KzDKT)y0-l0xVKl&1g`Jx5_O(eM5|QacJx{p;@b3j z*}N&v$HV7G_w?eo5DYjogysJzS|b9zW35a7A1g;`EeEJWYI(hfUdt$*=M)h9+8|v< zn6>b0J?Q6!a15BiE11L6J`zTZC9IyG0E85m9mQWXg4eOIE!hEKeG&u7)Wpe^FtR9@ z0iF;3l;2LMAhaw@2(3kTu(=!Z>{XZ3`)`^%4o!|*mib)|Qv*z!C;3}mcz+^m z&P6BvJDlI5jnCrSiifhLR5W{urbR16?2LZ9tHHz1LDff>Tlkrmh-99noZrySYgYQA zZ36?!nRK5rsvPzAGyOglYm+v+i&M4@Uc)o!XNgh6%nTpZxc@eve@=|@onPsu!%K%z zV>=kIoJv)pL9>SOE>}*0Jq=pJ|6)(WYP!S2w+_u3(jWL}W)$?aZ`h;SY5V z`n1AqF{*IR57$nVGCKggg4Xa)dxcu-Q=`zB#hS85@+#s8$R+dRW>>zg<_ZbC2g_OE_ z_alOz#qL1zGF7RJ@#-iiP}JfL4dc@R(I;KOqOmfVBb(;m>om4;;-jp4cBeSy&Egs8 zo7C3SIZ|uVjy!W3-R}^W)cK^7`;>ob1g|~E<1*be9SSaKIdIyX77uGR-rnlVhW$Rx z2?m8ZU*f}gtCCvX^@#w=(HaKIEn9ZPF3ZR)b7y*0+8tQz$<&eBd4)q7-D#T~xv1>B zkz=+KO&gVMl!2xk+D@r^x(Mz^G9Ny?oQ#zKIB1H)$PgpaIWb2WQIH9GzQ8~dht?`! zEv2k8Ae?TJrAr$5qWU^@7==xyaQ+hbhqhBRlN-LS~<{T z=HR=%dJHZ`F#KLel)8`hAnHwYlpIm-+e~;L(t#PACPY#)V&e24g^@GL%w$PM13)jb z(s6iz$E!n_6L*WIUa1$@;9RbtjRgyQZ&B<{xJN;=*5KY{((k1z$u0JetsBu{T8FWF zlI{jC$r5&)JSkRdnt!*g2WRU?8or;Jk%n+gw{X}FD#cvO5xR;a|%g- z$dS+;VhxA5SbY5=)Kjkb=eN)}Fd~grcqov*#g#X8Bz7`nh=a^XRBTnm$+$oC`7KTN z&V}-?_I&xvj(v1`$jf~M!1=)!t!r)Ff2ON%&dKNIwYl!}AkR>Offh0d2D~*%d-VA& zziC^cA2A98hd^;A>3WnH=LtU&>Izq3X#`-v$f{rx#}UEg#SzJHWN|D@+(=*nk@XwV zb0{5_Ij~j81n`WC5%zB&j+%sbTDA4twrG52Nh2F44<&Xw@it=X_LWO{T;}{r?xQ2e4j$lgw#d%Fao>jzt{E6gFks9%3%IGbJ#j4J z0oKw6lrt?J8B@-L{hPFtMcU7|TUunsW1t+(|4jWAV$?8H~e-U zB*Pk_OL)Tv9ZdQx;!h1adOyMMcBK9tQqt~!CXYhvI{y|`Otyt?=*LWEE&bdCI3i$U zJP_pMGKj=_5nCp*5F*JOL{XAXl&5e=D6&0-&9G-n;}Vj>T09RrYo}?lzB$vP=`78P z1-v8}<|aQ|D&cI)qVv8@7Y=|Um5%XoIWjEXu7xtOu9L@yVdR@JhxiCv8a(eyv7;ab zyD}2Y0RRbP<@>$~oq@H5^c&Cn&Pwc(dZ+8ttu6Z8I+$8Gepl2))mUEAgPvy^^~)B1 zJqi%AjVebweL%FJ--Aq}kfXKOb^RAPnu?)1n&^lI9Zg?TBN%HTW=L)fq#E!wVsd~+ zSx#>GH#(a7(&NeBEb_)ZupJQF?V00o_p61P^RgmTz`D&ZbF}VL2-!xJ`w2(0Zlkqq zqYi0(So~jg8kXoJ7%V9P*ASiw!?n7Q;Hb(6!6YO=0anqGL`INdErF6=z=EHuBw56b z0yWl>2DuRw1m1Y!K&GE0N5d$byyIZ5`wIh;2*qeE{J=Wdz}an+7V$&gOx!(h>fq%r zt@{Mew|8%`F{yt69@9xhJuHMX=)yVQmf|vK5kKvE*a<$(?f!Umo^Q8Xy`3{F@^Spn zE2lYFd-(Ci>cfUDXVSmef-I8n4_wV9?NRyR3^3YIxdLpN=qUyND{opOq+f#{gYGn`J5i#xAhIi1yWO z!&2j`MOZJnb0a!6>9munzYfDskNY&ORm6#*RklPts`9X+<-#51d0e*9!Nr+db(qui z#nfHfGsx>{BSR=>RC`=G$?KtG(JE%RW6?}!@I-XlIzDK#^UwzVxXL&zsgL6%xcJ+4 zH+(jmjp(TUx>>@Jxq53e=Vigot6MHHMIj_#lZr81lns04Y5yA4rLae#~`4ln4!O7l1cRbg<90t?+kM6Egwc=Fh4x9?=n ze8e}$;l%Pxyrt+WeBc7{yLpRN{VMa?#occ}1p5HT-10%4KF% z@pGHJdqGu~A@jp-6?i}}D9mPIE+6q8i+$tcmaSj8d`8RD zeSM=>RNHiTel2&a=a|g0APSYQ4T#G zY`5mSu9KtoaCe)dVlf#Ih90GOvjT~1oS37GD98leb+d|rBn~W~VGfes3=`1$S3T+0 zYA2BkYlG3R_~(!P+sts#dNu$Lg?`!B1++Zg@1qwA-V1-XtW=Tx9g@tLbrG+UCz0=f z()L<^-PZ-A?rSyRh;oNJ2aPx)r;<9OH{@a`r@&nDEPW>A4h-Z$7YHcqa+%v2 z<}?$Lp1Np*<0#kphuG0UO-2q&PWup_<;VrR1qy8FI~a2J(e6eUw;LI_yWt4`Yp9eM zl}h~I6dXcfz63l~awo|d8OeX7>OY#3p2~u1Qqx#e7aNm=2}MrC!uJXSfTz-ax|_N# zFzSehxan=aoOIpUxz!5apqZ5(eQswH!pn+K0o&AHw!6{8n~-f(Ihu)x7W6w{n~HYU zyDSPIL!#m+`gKH%Jdj`{S($5`0EzV(8E*t2$DYPaa&R;jfFs=40FOa*hV86hEifgl zav}ASrxSaezOdun#dnRLkpC~SPXBgk?|fSLnmWI*77a)q7_(X z{ekzR)%$8@@z`Q>2*gU-;>y%mX>>Qhr1zsG<$l}y(X?SRyc@9j;NBSL^@c_jIJan9 z8b|&^y-I8!&THA4gL~2FroA5x{wHCN)qTZa%+lMVV5QkdA)!eJ0)RMO#pewoBP-8H zES^XcTbZo7&59fhiwGP+R?9&cYm(qlv1Cyy5uFeYhTl9kcEQXNNggYi&X!@d7aj8s z$rEj=C)enaxz0?h$phZ!s+zmmotcf?3qJ0}OEO`8y>cU$%MI)f?s6Sp`Ce-3uEk?! zbRrmVW&oS7eY`{krWvhaup6+c<=gi!F5E1>@v!U7n6z5m9_1<((x=P9ok1N}XFZrS zdLKWoP~pc@k41FNG{>!8n`?PBN0O6QUJqU2m~VeoyCMs#jZAkp013wxqG=C~4sano z`?!)fb-dR93=EDAB&kFCWZ&@r&K3-942U^EV@MANCM;eb&QccVPq_uYR9d=LVXl2c zn=83i_nj5C-ZFT|h_J1R{J7YqnPYpnd6e_beRIvWJOxtQzA;~w>BgpiUeu~)zi2*3 zE>Dg(H<|IU2og zw-2XN;BpikK-uhw@Qt3*dye?}ai3|{>5ns~+l=tdc3rGoro}+q#xkP_i2qSe#b5+l z8yJ}iVzE=GZptq(w2%!du&~1IxCTRqje9uqv~$%PYp1Q~WNPTFa;DnuP8DJccjtYM zE$=_4#qvo&Md)Nv)k&B!Bwt-kzxUk zk?>MZ9;O8MFHWKmxpbR?NMIuaR_2I0SV5{T<%k3eaCpUU&2b2+uOVcSk~C6O`86Py za+B5TtX6Y5Wq)X3%U%QTmOXO#;X1RUAB$$nGd$(Yh(x~O3-e_E7^kMAHl3fgY?I|b z4e!rfH7MZBvqjLjthRe2Y6ZD6EC7B$6vYJ-s6&4(T}V9w>~k| zQ{_F&dHLy8iY27CnoSUjodCw|b-?s7Vh=Q4(uQ}ec$IWzal6dO~&K}si z&7)6q7PSp&*=S(Z2U~g+7-I+S;0B&#oAf;Pr?e-!r>m{Ui8?BSUg7+*XZLG~ullYI zn$WgRTv82YiuaIH&%!-7A3GT{b;2f&^a`5gbj9IiC_25=p_?|HK&|QK?!|)-gMx1f z4?Q@B8i?`P#vw_@>{}E_qA4L>X6~Dw4@`_ha6%hir&A7fqjIDat}6_YY8mMm8F8{ z$rGe-3R$m&kvP14;bSO(%78x>{7F{?rgza=q-ZraL+gJUH?`i8Glh3%IcsTMtX6~P zCHs6zdAo-p9gL&IMwU(rVcA`x({9uU@LtThH<{ZU?mzDMMEk$=9>cRLxvxpO1$dT=dQhY zDwSYP=>s0QRAFaR>u5_%%boc}F4wx39OqPpcJI&XaOG zls2EiGPYb?_`bZw7}vz`uWg$T7&c{Cu;+-$lRnNFJAeC!{w4i9c-%E;^T{X!c^AZ+ z`@t#2;&o^++6x*{U-+yljp2U>7loFlqt0dE_a_4i1s3r!@X9Io zaIrSTrk}{ADd0s*QxVHabr}i3a*mY>e`slHUCr-ivjx9$X`0GQU4(hXur9Uz*~b(7 zj_M`Qu&Upk`DsB&c#5JE`hKkkU;OE~Wx{eGn*e&g|ek4x!S_)EF# zxzpw)k9dFIwN%|_-WD55!HH*fpUB`>#7^QjL&pRJorxXX2C{eeWUUF!30MA$khK9({}mHf6|wvnZYcYO)xg;t2{1 zAj|(l#2jbF>VYPe@QdMzR+Oy*j6(t$00?f3Fq9q%k2c?Go2&bsJ3lC2biu&buM1Dl zTJva$sr&O0Z-ej6ywz?0v(XJow#(-DeEhtDrFbM(y!5`~d+KP+?EJB6AFiB3h|BQc z81nx&V9YLkI|en2>+~9a%8=GDQ10FE$16TQWMVfsmFv+rBJxp#_52LmDPG0S?{M+! z=Cr5hPs^6oAX<>I7=Zj}LMr9EAp(=mKnX}}Jg}xPkXj4U4f+vQ8G)QGLM$V1g7O(1 z2DF)@VRf)dF+T(q@iIZ_i{MA)XEd3!qv}S-!VULKuXpYJ(^u0!oHf;tHi>pjd3|Q#ecZK_TzY=My4~!(5Hbu6q2GN3y zRjFj<(7CsXJ3EpqFf?0B$^0yW6pPCQ1z;3VfLL9GX=REqiW8XDCh|cJ)}h8W^tf>B zhiu`2jol-=p31#o;ONOi?4w3a03&76OMPSAhFp0wzL9UE@{7im^E$FEeEH;4JQORX zq=m0%U3jx#Srbpc)7y$YF8}R7`v61#v_O7yMr&SK9Wr^n3XT*bwWQny5C)B#LW2-A zB(xkhwHZ)^335X;1ODVRL52_j=`4mKEe9zARccEp0klriuq1*un6Paij{zC{gNbkh z7NNjJYgRIYj+UA9MG7mYrZ3gKJ&J+}OM3;8e-*Hjuc9G7vhVt5qhr_5hOhIxu4j)KEbi~?f;ML`;dSi>Z8 zy(>fpCOvkHW{w9o4iWT26vhvHaH-)Z-X?|l!5+Sw4s5@=Y2Orf{4M)Q(T&bDyT#+8 zoSdApH1nLe>vgZK7Q=FT#-{mgENDE;!^g(cooi4>3hx?+i$XB8oH8m3B>pc+q5@Gs zwc&*bx1emzKrE+{kD|aZFD9&g35Df)R3nd#Ik_&!O5h*bn^O((ZtA^|`-P8nhfa=@Q}n{l~@&vl^`hmyFFU8F?IE5_x0s$ zci$_6nwWjk9I@NlrgZ5Sz7cpvCX}b?vE3DxL1o ziVO{@%0yz7886`FvNR>-IZxJ;OPmZ*q5wvd+7nT-9E=5loGt>E(c*W?fSy_G_ATEY zRBiC>dhas#xR+_zCsQYp%k`OAa|7c^vgq5x91L#^ zc(+er6Ejeteq#zUDQ46g80qjjWD}AQl^I+eP#<6!MCSx9;mUwa7!*YWP8VQD@xKs( zk|-Z<9ALHPJ2U8a%H+r)PxFOWnVk8f%RdFD&zRv}<0v^)CQT}+zFVic)qrvijSo6H zEh@Aw=~=moR(`u`d<@LA#Z*7C9>Ms3be&r&sprTVHl8)QlnEX%f`>AsWK?^J5#)A< z>qb6X9}C<(V|Hk!Jm!zzxf1j$($6fsc#R@K=c>0LH(-ovk140p>{$>$dD#12hKJRp zrKC~A7}XwE4#GJ%?F8>Z=MJS+%<$V7&2;43H8l0e`-)QfRnDz>2e|`U|z;Rfw%DLU1ZP#1ahwI)`|Oi-xE`>{2mO(?kWi z0K{rF3&2D_TkEM^*PT&AJFlE~q;svqZ`$6hwa(NKIm_yCmTaAuIy`-Hscgfwn`^AU zoU1f1ivqw${~8a+WI4}{xMcTWuI>0<+p~1pPcUH2^KOlOIjur}_b!H&GiehnLzR>D zG_0ISA<`7X$g&MtPId5$cWcd!Ca&NB0%tmngZ(8B6S!sC++;B@DiQVl_m8cMt{uBju&|=uYFG`apf!+1JXYx-sf1Bx5T7_<33(WF4JM}G(D@A?OLkXb- z3O(YFIIt%9tTKy(u|G`{$53RDCUFfT( zYX7J~M(=uwGaSbx0H=Z=#eZfV=);;lH_m9hN`jl4-P(Dr5s z23(A!z1FOUI)q=oMc0qO2y=j6xJBX>$@wqOLPq2fa(P%~KqL$N94N`}=|C8WVL>h= z$+Lh8NCycI>dN9IoKCJ`bP#gU4p4~BNKyn27GzpYTd-AHrCQ>Aw-a_bT2@%TIs8EA z<&)EzS%9%LY0A&f+%5IO+Ko#Gx1W(*wC}Qo?t2cBV<`wFuRK}QEu}Sw8@UFwGwH@> zRJ-%*nEsENA1oc+%f?lc|3b~q-nK!;wbQEILTNRD@fjEVV8uxu-I&v_g=5Sn;3XW^xIFQi z;fG@=uv#sgt6!D3rS42a+_-CUdi9~(D>F%fL`I^*c!C#lK(gTN5E2sub55CdaI*bS zQlzs2KN+#bHW-jAoC1<#g&fQ@^+>~8eu-(RC+B&a?`i$vO8E5DJ~z#u2V|~V$Exvh zws_mNXB{JWsTG*hA}_&!Gh1zrv1$JL%`bd*M|$$Tl^-?CCqA=trVaiO=LHqs_=RMHRs*@j>Ca&ip$V6jcX%iV6(a8BLX@YdpCQ~KE#segd{kw`NU>8b07 z)L*`$!?EdmLR*b%?)b_}LLO0u>H4jjStYciDr!ry#912@#} zYT3OIBLlC%Ujzq%j)O1|D+91VPm-qsDj@`v(f<`-p)g4efaJg*EnzH3JyoLZ~>j?X3HBK&!YNamV6Iqsi5u#-ibss@xZY2{>8 zxrD?EuJ5|$&A&CI%ZSL{#bY8WOkZx`*vzi*F4KFpoLgYoTVT12_(+yf!x+_`w4O@U zWng49IQB+s_+K1*gHl{QmKgeQ``CLObJX;ivUvCsaO{mH8hDgR$KI%8f_If>7b6QY zygC6ZE=Qns3ed1RD7_-u>{vG?2e}|aF+z7T>dt$jEruw80;`JvIA^*_Q zIa5%4w{5jRnEzGZMvqbRQWs&~|AJ?9rK$c~n#K1zvFG^A=sMSu3K0yycj+v3ADsu< zw0F4Rh;Rs%#3-O11ZZfyRfq}qDCiT4UvvZ|JRu2l6pRK{{R8KL6mOT5`Tu$z$b7ri z=8I)s`ht5tZP9A#YN=TFZJ)P0s_yk&JS*WW^KIt-LA7If$p$c|9qS<&aApCvt?>W! zJkVFal%}Q5XG!PlRj>oL74*9VG3x1UYi|syI1O}2gW%kKLGWI|&GD$*ZEOnbE zzf4vDj+Nj)bg!{4INsZQA^QvOHRMa>=8tm+_UxEg=F0I}A;kmA+#1NfB{!LV@4ZIq zK61qEk>`KR5v4H$x$wCy#Q_uuG65IkU@j$20*D-Nk%@stPR9{JnM_1AK`tpBu2PL?|B%_v4-&1X zuj$<)b6PK(+(EAkR`P3LHa@o>kGDq~G($&al&OTbfWy+Dq5nL;%X%U8_Ka8f?}x#o zP4EvIcbyGaY#~HsIbO?vWDF^?qOu&mVF^TE24SV}FBI^rQW@4nB?u^S8)rr^r-MOd zL;Onfcr8?g!7&OfbPo}K<3DIzifCn1Hmo!%G_kWOh{LU+l_bR+srzLtnPE# z!_=h*t@}JKo0&PL^0lfqJ!9q%J>h!fQupjt`;-4-5M$1j^U;k5ul><8MOwqADrZzS zO$ebqpSXb#LmVvXb`7y)*n_QUDsNA|+?X|U^4xN|_OoZ@m zg8%oAn>HtnI(as2@x$R7jmcl92|sna!@>?le5WN2&+StxsJVBNS8X22NatdRA$$L_ z#V9QYT zia~x6&p?=k7bS751plc`>xeb6Wpb>_{_7r)n)6Z@VeXJC^8S@4{EP#)&rL5Jr@1rt zT+N&W18uv5y!B+{cR{Sh?rY8C)FDxwXTn<#1NB77oP(Lvzp$dSN|NAaw@w}L#1=rKSA!wlaDpl+Nl*^94e$);5BQUkthj`L zC$?fi%$U_D!GBBO+Qy&mbetJ&-)hp1je+@1jTvFr`b7>ny7zsL=i2M3L-#M3TxfI> zuLx49);vAyx_e)~NB7qpe0{-cT4b}#f$a$fnYl>7(|%nodkq8#!^)YoS7=l@+V%_m z#8yO>)YjQkMuvI~4+qN89N1LL88r-5IoVn|fTmkOMfYj_3W6;$TEjoNC1&>2_FSJ; zUiIUSM9kZ%BT*_=E_8L1`;e=JM}revG||8=CY{(a+HD$xffHMpOG)ZxbS@j-;J=JD z>A5hibG#v)CE4f}d;CEKa#1bTk^>GXb9yboFrowq1riyB3c?lCMlnkyNtuk300vSx zCAP^iRSJ}lP9#5= zjiWjuq|lO}>Q6c%cvVbrL^u=vkRwWO{QSDLo+XbPZuW^y6@wY=?aUn(ELSgC7BeDW zo%aO8?{!3}`{=XyXI8{QN5tX}uDL-Nj-H%^S5U>G;unncReVPd(O?*rlip|X8KOyn zPniwr{O46sAZlHJf3!24tj%xyEdKM;1>LjfFZ{NXkK>ymHyb}_RjSwuURHz(SZjXW zJE~;}*+!M49mykF(C>h?@D;WgZj=!52shC8Jb}!J?ZA??N-Bv+A)+acCZfm5WC$p} zBckD{OPXh>v3on#g0+g{u%ZI$=no0XP2XH!SIEG^3shJ*bs5-TMEmNtVX5)eBCMC( zxe=Y3blS<(rE&P_ai6BOia0T}%9e;nRUTHfT)3k=55-7n$eW3~=S>~F+@*D&;Q99M zEjA|gFThJg-Mo-{%G2Ax4>RZ)zeyh*QkOeiC3^Md#u@aR!KiGbmcxxBDF;t{(kg;s z%h)f#(T2_7w_(!a?^tE?Xx|ann1Chq38Yvk{x0kEn?bfG+f`qcLZgGZxwV<|vf$>` zEte4AF3>9kJEJY@bLITd6xW-2^wxFfT}QyifIrFv1K1x;Mun%-yg3qLtObRxK$v!D z%ervvx}biu&g7}>|01qaN|=+arTrq{Dkl9hU$u~Hqvv!>K6n9#;ckZ?O{Ik35Y;ew(5`z!4A^$lTpn9Yz&9< z8i1kY@kn5>(oGfp%DSUR|nhdg> zDTa}?Fr*wltV7)@Cf)fI`|0AS=0lb?aU5dXuH$=C7XU_;`*EK;HgQkKlU=)O!mqB% zpqp?8DhCbgbij4pv94di#vQHUpWL`Rn(Mx1zow(G;^sf+sc9XG%0LLP6?=$OzL3>?hC zMhzE1z>?vXgl2@x(Uc>)XEfBv%3;bIUUQnXD`-?XSvCYj!=QD`#S*2YyCK0N7KED% zLnFzzF)CH5;izJjfi4^ZO?zBz_`(--VA%cUAD|`(g>_=WqoR=m_jm=BJ5KBI)|hlO2=j=fyj;Q+(rY&48u z_|4m9V)wP`TI!H1qb#V7h&vkI7ZGL@9IJo`qPce?Rne)vZ$<9z80)>9YI6{vQ)xHaM!9H1TuOdA$Fo1)aWe zXg**2x{WZSyQmoM`JA4z8h`=YGa_D|Q4${@uo5hD$p{3{<}>+??M67I)go*U-nM8HLn;-rE(8A=;3`a>!V|SFYZOut#=}}-N04^!|$EXQunp$ zD0RqI-k<&m?~d4d6FMzGbyFGw6Yt~*J z5?_A?5lcWVMrcL0tx(jziVQ44al)rIbzc|Ql|xk+bS1^m4m@JkM$BXl)Cjl;AH9Oo zF@8v7m4QU77`ce?X*Es3Pq?I0?xV+9#9x0G^u9}@JtaIx9SiU`_4lP)!5*(i-re(A zWj3-+y@iv;m-HV#gNJHJDObxDhf60^aPRUwe8H-h115WXEqpm-QqgY^v}s&5~^Ci zx8kFwY4NSry?j$8_B&Ifr2Y0kY8aI|xw2QF&$x$R)ToB(!Z)*8e6P;Tksg=xW(~Tv zpuvvtYc7}l2NoUKxA}qcOP>udVWMFu79d){M@~`>y*m-VG*owj;k^GNu<#woTW-~; zws66$z(JUW7b(Gcszen966u`82Xmqz6U=1@05E(o5HVhnBgX`Q))N4ErKCqRap3|r z`9sg|GvANgwXQ2T0r$%mE}3}AOfcVuh=NRj!%uivepu${jtEHxlb&=F z9T7JChaAy-3Fo~7{M(6N=ZH2uZnt->S5z`@P&@o)fUQ%vXHI1@ZLw@aF#KLel)8^L z(e5@G2^~@LI}XKPp<;DQX)AGl|ajc%2NNR{Eos`=L@7* zkJ;i{WzdZ$`)U?h96sw=wu^&I-9#%A+pl&0rnxWp@9L^f9@M4m#rM;e@wiaxNBea4 z%=C)rMc;dU6R06WZk?4OLjXe%tjQuyQzkbnq5(#*VY-&oTl?aTiZG+G5Fnk=7mHZ? z7$~=Cz?3rHFc<}&e5EY%-v#NXpe+?p7TB0ee1=M zp9%g}I{K2`>Y+_fU+B86>9Ll1Bg(hv&C81D03f>Wyl`bN3)X`Mlr!n!U&bm2b&a3v z8wHfBST1|^8Fqb#l}gA}+>Eap66~@*%+!^%QNtM3UdMPyITViWx7(e*lY^Q%ej~NT zh^s28EabG`b1FmO1mN+v3&@s)5=TQ)aE3%+_isuq5ry{jlk1(y<)U%3`x?@r;Bw!0 zt+Stsd2H%PS@M;l=QI!8e(}KZQ4f18IbMF)=dV02*OdPVXIstZ^vW4iQ>TER6 Date: Tue, 30 Nov 2021 17:26:16 -0600 Subject: [PATCH 20/33] fix issue with log/logTrie processing --- statediff/indexer/ipld/eth_parser.go | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/statediff/indexer/ipld/eth_parser.go b/statediff/indexer/ipld/eth_parser.go index 0b4780f8a..9e6d4787d 100644 --- a/statediff/indexer/ipld/eth_parser.go +++ b/statediff/indexer/ipld/eth_parser.go @@ -238,14 +238,24 @@ func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*E return ethRctNodes, rctTrieNodes, ethLogTrieNodes, ethLogleafNodeCids, ethRctleafNodeCids, err } +const keccak256Length = 32 + func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, error) { logTr := newLogTrie() + shortLogCIDs := make(map[uint64]cid.Cid, len(logs)) for idx, log := range logs { - ethLog, err := NewLog(log) + logRaw, err := rlp.EncodeToBytes(log) if err != nil { return nil, nil, common.Hash{}, err } - if err = logTr.Add(idx, ethLog.RawData()); err != nil { + if len(logRaw) <= keccak256Length { + logNode, err := NewLog(log) + if err != nil { + return nil, nil, common.Hash{}, err + } + shortLogCIDs[uint64(idx)] = logNode.Cid() + } + if err = logTr.Add(idx, logRaw); err != nil { return nil, nil, common.Hash{}, err } } @@ -259,8 +269,7 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro if err != nil { return nil, nil, common.Hash{}, err } - - leafNodeCids := make([]cid.Cid, len(leafNodes)) + leafNodeCids := make([]cid.Cid, len(logs)) for i, ln := range leafNodes { var idx uint @@ -271,6 +280,9 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro } leafNodeCids[idx] = ln.Cid() } + for idx, lCID := range shortLogCIDs { + leafNodeCids[idx] = lCID + } return logTrieNodes, leafNodeCids, common.BytesToHash(logTr.rootHash()), err } -- 2.45.2 From 963ed838a00ee27184e37cb091f20434f90ebe32 Mon Sep 17 00:00:00 2001 From: i-norden Date: Tue, 30 Nov 2021 17:26:35 -0600 Subject: [PATCH 21/33] remove some unecessary hashing operations --- .../file/mainnet_tests/statediffing_test_file.sql | 0 statediff/indexer/ipld/eth_log_trie.go | 9 +-------- statediff/indexer/ipld/eth_receipt_trie.go | 8 +------- statediff/indexer/ipld/eth_tx_trie.go | 6 +----- 4 files changed, 3 insertions(+), 20 deletions(-) delete mode 100644 statediff/indexer/database/file/mainnet_tests/statediffing_test_file.sql diff --git a/statediff/indexer/database/file/mainnet_tests/statediffing_test_file.sql b/statediff/indexer/database/file/mainnet_tests/statediffing_test_file.sql deleted file mode 100644 index e69de29bb..000000000 diff --git a/statediff/indexer/ipld/eth_log_trie.go b/statediff/indexer/ipld/eth_log_trie.go index 49b7a7fc6..edfd35af9 100644 --- a/statediff/indexer/ipld/eth_log_trie.go +++ b/statediff/indexer/ipld/eth_log_trie.go @@ -114,14 +114,8 @@ func (rt *logTrie) getNodeFromDB(key []byte) (*EthLogTrie, error) { if err != nil { return nil, err } - - c, err := RawdataToCid(MEthLogTrie, rawdata, multihash.KECCAK_256) - if err != nil { - return nil, err - } - tn := &TrieNode{ - cid: c, + cid: keccak256ToCid(MEthLogTrie, key), rawdata: rawdata, } return &EthLogTrie{TrieNode: tn}, nil @@ -134,7 +128,6 @@ func (rt *logTrie) getLeafNodes() ([]*EthLogTrie, []*nodeKey, error) { if err != nil { return nil, nil, err } - out := make([]*EthLogTrie, 0, len(keys)) for _, k := range keys { n, err := rt.getNodeFromDB(k.dbKey) diff --git a/statediff/indexer/ipld/eth_receipt_trie.go b/statediff/indexer/ipld/eth_receipt_trie.go index e187e7d9d..4ceef6f1f 100644 --- a/statediff/indexer/ipld/eth_receipt_trie.go +++ b/statediff/indexer/ipld/eth_receipt_trie.go @@ -166,14 +166,8 @@ func (rt *rctTrie) getNodeFromDB(key []byte) (*EthRctTrie, error) { if err != nil { return nil, err } - - cid, err := RawdataToCid(MEthTxReceiptTrie, rawdata, multihash.KECCAK_256) - if err != nil { - return nil, err - } - tn := &TrieNode{ - cid: cid, + cid: keccak256ToCid(MEthStateTrie, key), rawdata: rawdata, } diff --git a/statediff/indexer/ipld/eth_tx_trie.go b/statediff/indexer/ipld/eth_tx_trie.go index 943cf15ae..bb4f66df0 100644 --- a/statediff/indexer/ipld/eth_tx_trie.go +++ b/statediff/indexer/ipld/eth_tx_trie.go @@ -135,12 +135,8 @@ func (tt *txTrie) getNodes() ([]*EthTxTrie, error) { if err != nil { return nil, err } - c, err := RawdataToCid(MEthTxTrie, rawdata, multihash.KECCAK_256) - if err != nil { - return nil, err - } tn := &TrieNode{ - cid: c, + cid: keccak256ToCid(MEthTxTrie, k), rawdata: rawdata, } out = append(out, &EthTxTrie{TrieNode: tn}) -- 2.45.2 From 850337377af0560ea90651da06ecae01b879da26 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 1 Dec 2021 15:53:20 -0600 Subject: [PATCH 22/33] handle edge case --- statediff/indexer/ipld/eth_parser.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/statediff/indexer/ipld/eth_parser.go b/statediff/indexer/ipld/eth_parser.go index 9e6d4787d..0f294ad30 100644 --- a/statediff/indexer/ipld/eth_parser.go +++ b/statediff/indexer/ipld/eth_parser.go @@ -248,6 +248,12 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro if err != nil { return nil, nil, common.Hash{}, err } + // if len(logRaw) <= keccak256Length it is possible this value's "leaf node" + // will be stored in its parent branch but only if len(partialPathOfTheNode) + len(logRaw) <= keccak256Length + // But we can't tell what the partial path will be until the trie is Commit()-ed + // So wait until we collect all the leaf nodes, and if we are missing any at the indexes we note in shortLogCIDs + // we know that these "leaf nodes" were internalized into their parent branch node and we move forward with + // using the cid.Cid we cached in shortLogCIDs if len(logRaw) <= keccak256Length { logNode, err := NewLog(log) if err != nil { @@ -280,8 +286,12 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro } leafNodeCids[idx] = ln.Cid() } + // this is where we check which logs <= keccak256Length were actually internalized into parent branch node + // and replace those that were with the cid.Cid for the raw log IPLD for idx, lCID := range shortLogCIDs { - leafNodeCids[idx] = lCID + if !leafNodeCids[idx].Defined() { + leafNodeCids[idx] = lCID + } } return logTrieNodes, leafNodeCids, common.BytesToHash(logTr.rootHash()), err -- 2.45.2 From 9e4cbe512b0454000d6c01118fa7e492676d0791 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 1 Dec 2021 22:27:33 -0600 Subject: [PATCH 23/33] add more 'bad blocks' to mainnet_tests --- .../file/mainnet_tests/block_12579670.rlp | Bin 0 -> 4454 bytes .../file/mainnet_tests/block_12600011.rlp | Bin 0 -> 5883 bytes .../file/mainnet_tests/block_12619985.rlp | Bin 0 -> 4041 bytes .../file/mainnet_tests/block_12625121.rlp | Bin 0 -> 16079 bytes .../file/mainnet_tests/block_12655432.rlp | Bin 0 -> 4044 bytes .../file/mainnet_tests/indexer_test.go | 53 ++++++++++-------- .../file/mainnet_tests/receipts_12579670.rlp | Bin 0 -> 849935 bytes .../file/mainnet_tests/receipts_12600011.rlp | Bin 0 -> 690821 bytes .../file/mainnet_tests/receipts_12619985.rlp | Bin 0 -> 571436 bytes .../file/mainnet_tests/receipts_12625121.rlp | Bin 0 -> 706015 bytes .../file/mainnet_tests/receipts_12655432.rlp | Bin 0 -> 850429 bytes .../file/mainnet_tests/test_helpers.go | 28 ++++----- 12 files changed, 44 insertions(+), 37 deletions(-) create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12579670.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12600011.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12619985.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12625121.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/block_12655432.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12579670.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12600011.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12619985.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12625121.rlp create mode 100644 statediff/indexer/database/file/mainnet_tests/receipts_12655432.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12579670.rlp b/statediff/indexer/database/file/mainnet_tests/block_12579670.rlp new file mode 100644 index 0000000000000000000000000000000000000000..6b4f3f773f278ab72755fa70b9574ae10799bb99 GIT binary patch literal 4454 zcmeHJc|26@7r!$GgNU(|ois$YknGu)D4Hg3glfD*23fO&$uvljW$fa%SSCwl3ymz5 zy+|ZN$rMpZD9ZBQ>D`8p_w)Y#|Gnok^W5+Go^$Rw_de&j&$Gzmya?w7v-$RFXnhlD zhqPkTTZ(&_VM%lDTu_7Y1g(0XJVYA71~SBt$2>Cy!7|2v@vOw7Oekx6y(9SYxTCteR1DD za8l7R1|QsqE0h<%Y5PPj&(z{Q#k&Ru0LK9U2><{A`a%Cc1i=k3(OCon03aWL`#9Yo zL4a|?;Q(MxSLh-^ACMAsPZ%r)x%CeQt`89iK%YP$aGOH_3IPBTeK3dahUBIjB?tfy z1fUS;?tnf3-~d1$$_5ENA#w*44}%=o=?Uop%=G#Qbo^5c>0S^GUEuU^$crA00NCg} zHxx;yBp@g$0X+?5`&lKOfig(y)3eegTDTsYy(1`$SU+n^q>VNZY3AbRk5j|cl3e`V zTzoy6N_OS2(i5Sgn_nw0hW#=ba?lP!S~$VxE3G zrR3%_U9z~@ViAThBr(+#g8&hZCz3Lpct3zOAem-d#JZ}X z!B$xTe47}n{epr2MrL&y* zsB%r+ieJtOm#^x@cOeU2L{|9}5|iPXO+X|DNxnL(XHN~O7QQXc`P6;iZiCqY^|EMZ zC-%+7K>z^_4m_CTvaz|JCAs?QdD~c>xUX4-T{&i)(2H$yo_+8k2t0FEEy}`BQ0%@< z=KaJ0!kuh+_tJ3vIlg9lCG_q-7s*AKXC#bDVqzIG7bL=S{WhFNLJbVDwd2-X#^+y) zS>NH4TshWqe`_#c+COrSO2gmGh6rS2U+z%x0@P7FGwcs>gQ80QQ{0`hjP~^ynqSZL z+&X>D{2Ev8samx1%d-w$pj2HxhaS{XT~q%?^DhnFnRXw#a^fxZ#*ddWPioXXN>9(p z+zd>r5(G^~WGDNTL0_>v%A;7cq#HAKlX`R6mH?ZwI=O|i383suzkQ;d&(a z<^1w;B20ro9&C(AiBo(lbxi`tl3|k8s7n-UAygYvn6vLOR$mQVxNXK$5wTz0s{)s397qRpFVqb1xJL^!Ax z!n?hGjt>Ks%H~v1&X_(de#cg0B-JcoJhwakc|>?^$5iL)zX%$P=U024_#HDic~a60 zHy@f}qV-wVl6PISB?`IBS~sFWxI*`bX2wCDY#AbM@$Od;S&d z=Us$(pnk8F2yb`#$2JZYYBOW3mH{(+v^Xg^|NhZrpc$ZR#8Wlr`*jS=Mi68qo!_6T zNYTWwv``uw4V~c|EsIU(?19*=J}gU7*9e+PhaCU)-asXoNq& z=)X}Lqd^II>v%z@l>CJ(geac7>&LRo-{& z=EPqw9LwbKKWXT87bT&n`sA(`)W-Jv{(Rkr68~!(=Y0s6xOlZI>O{U#q*4nU%kBXL zGFH0|GqrDn&xwFR{GLoQYW(e8N*@buQoA_umgl3hff$9A-TcSag zm-HE!jUi!U`Y#qW{8o|&lQ9xjpWmp>!~WgBgjcGKdf6knGqJJHwmduFb*`IS+|JtF z#&-Zqj>7sp3)M^aM}`{ic6VxgJmDp{6I1)C;1*JdtYby1O!NMNo6me(6USaZnT$ zR3tgvh`DmmY?4;`$zNCn%O8v5aE;YFRdGk>?Wn_Ngp+}lR9oVBft=8=Rp92*bEd;& z>J{x%tE%oj!d#qgT>`%S+*6N5RI^!_Nih*NxteeEj^c9C=Jv{Bm&)Y}cnx-CX)2&6 zn2>ntwgg2CmiR@5Lu~?;DAS^{U+>0Uu@+pfOZwQSDWB=f!9w|Dq-je%w>Iv3SIqk( zN)Ovyza+rhubtOJv~KFK>z?a(a`XO}Q!+ijRF|Si;0$Y9>6wUdK#>+jk%wJh4DT-5 z^+LEHP&7+a!l*yvnc=7kvS8LN*m>+3C=t_Fj5rLkI&P{O*Qen z@grAQAq3lz^K@SkX>e$uNHjjXRvPrG>P^&OALxqGE7W;BXkK>x+nC`O73b&0iY0x5 z2PU*ZzS=E-_Tlo{Sfi_Id#=qH8$3!0Jyc{eSjDZTe%9!1GPIhQ|5*C}t)~B0)BpEs zB0L#fIWHSTUTSZ8d>lXCaMs41*>}@4MVhuPGYi~24gTV~rC)u#b?SrtMT;gOY0)Pa zj*(g$Rs8a9*9Xhe!Y~UI!tYyCWK1==V)})HV8)g6XkJ7eB`2Xq3aCro;54``S>)F@ zQ`L?#VX+~1N*ww0jpqMx<1lefj5MADpwnP?54BRsc5X*pp2JLoOS6i(ZU4f{T9Q+_ z4b2&Ma5@9Dv)Rtep3|#yP^9jHaa~pIOu)I-TJq`I?gQN;I&6}13pZ^^aJ+;MKpfC4 zB=@OOEc`jjl|6Z~Th4q>-~iPgZ*&}dRQD>~NSR33V}C+qYvtw|HE|oce>n}VN`t4b zYR54p=(^)qO+NUUB#-M@@n~*2{eq;FvzO9ob*}}q-l;snnyFPBUPq!Hs||cJD%$mM zM`({jR|=Boug>DJaFdG!KlUvjsx_XCJk{_0(TC4%G@SF%fRk~aTVCQdJoH}0JgT++ z+FCaR%tMUy-pPZ5MFPIhG>)D6EK3+Ql?UBI(V)_8cVfthVNIo9L~a7gtBLA3Q<0uh xseSmN@IJHJ78EFSJ~zqsx@qKBH4z!XBdoY0&tn7o&Zwp3JkHhTQdT(9@Ed>-TR;E+ literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/block_12600011.rlp b/statediff/indexer/database/file/mainnet_tests/block_12600011.rlp new file mode 100644 index 0000000000000000000000000000000000000000..96032b0c2152373ca99884090aecef7d84d7ad66 GIT binary patch literal 5883 zcma)AXH*m07M_F%AyO6T(z}45^kyj1B!CoYDn+<}NDC@e3<)vxB9SI#XiD!LQIsx- zbOaHlDtJLa#2}zA@!j{HuEl-p&Y$erv(J3ruIFTSSV=ojCJ=It%AL_70BP%PqTYT1 zHJv;@i)abNLkGVHZmM^eYJ$i6Q6FZyF9nt0>s;&mM>$81oaG`$T|c^>qqUD5P0}N# z_1M*)b)Ni`p5}BEvMHIC)E7Aqmf}u**9ua+o3(m#8;w8VY1oqK5RJ5C4OZNEQzvxd z8@OVM+J$n4rWs>XmIrcP&Zg%qvMlA!$svyUN|o1~CxqjxhESl0eo=2aI%_D2MI^Q;7kC5g=_-@BFH)@gt7pDBLRScB7^~aNB|Cn z00<@kfB|qAz(f|{a5@MRl#dS%M2G=o7IXmltz9kh#Vt_8#V zNjks zzN5b6|GHWw#!miO!Td7Qjb^1JwIq<_jf0K9`-?7>oGEdA@8oe-leH_i%+^9^+4l14 zPp2Lw=w0MC{^9^&M8FLOU8&_C$(5@C9eaP4W-dQ#^I`cW5JKnZIE^YuQF{imubG<~ z)oEGjeU?$s38FTO_wN_NA$mc}Ge@`*G}FULo>+-dJgktd2f&)aiPPruuT)rjq>O@? zvUrt3Ok!YRJ_H)heS=tX(u6_dXQh@YoR2FLLT4=B$}zI5&CKOZP?H5lbyBY9x(Fqt{EE(+L7)Wa_4n@f5X zVKXgzx1MxYBR2?yJgphQC&H^_FjvJQ`gWrPAXW%z@$N>cL;kI1VZRqIaI+7yzHhB@ z{|VX|r?BOFtFVig%2KBdUl8NsLKWPI zh%IB$@GEHcH^)!q35*&$sKnD$zYLvt)ET3o}dO4kg;PYUy(|&jg5G!Q08#}M>Uv0vADT2JA$HN>9rN#wxCZi&+?rYqLv|@Vx zG2gN0^v7n@jra>jyO7$hx+!}Q;U!<(V00t=hzG4}qU%xKkRQ4Yw=W{V?U^R=Jin^) zpLn+a!?PruW2NW2+AyN};eF|BSutwfwUwau#yvx<{J=*17tkPxB{e4*Rzg zQ;JND?RQGn7_Jwh?Y)}{LH|vY)%Pjpi8jTH)C2l6QUU2dgk{XG^XX==v}~=KSpLMb z`yXD8&D-Jg5dl9^Vry=ClmsY;ZQ;gMANQnEm1B`bbm+B)yJL#wMl zg7Bnsx1@*J=`l_7M=&&FwxU1RLlM_vxWAyC>6t6-&@QzbR)XXy@_c^bnFZ+@HJ(b# zpANuIpor|f5Qp9J07w6ujWwE|9(mPW2`Pzye)}dK^~rjP?47~-+H;MY-zy?E8J@8o zySAXF7Fv#YUv29Uldtd+@5%~HW8gh zx(35jQBQ0A98XS8X)&uX7Vmrk^MbaDgy+z=!+8-@0~?hxiM5kt-;$%N8!~LOyZ|Y+ z0?ZYc;|BQFLGzJ5+X~Oq<1A2=26!}8BR)cUefhc1mkZkW&=VKJ*nDdphi3;=NdcPq zRu9SB1%!_A-+(xnPa|j_qfrAS4BAvm=|f@47aP>ehA~`}`C!Y7DA46LsFJ)OdKb zI7lNCG}#SAf7bGR?jrXx)18J!nZw%fV`E?1w8@cZ#Tnl*D5)4j!73+jGYlvG={8dk zRnG^L(4y%tV&A%WcS{Ek-jVx%NstruX%hke*cyzsP<6hNb-FEpTV-7i(O96!oG=?X ze0)5w{P@@{FgT`qLfZmS+j~;aJdFSJJfI=U>$f<>d7pN2x}I%EhU7u}Z?4#-d7@zl zQpq&ceH%9r{f_UW^;S_^SOS3xE#Kq?QU*m`VL~UP#uYjd8qN@g4X3*|AdT_YSB8Q9 z=`KT>t>#_OnQ9jASej$k=A=NlRCF={(y0%&vZ4d8*0U(S?ls;bdHhU5Ie@iXCyGN> zHT>lGnX@^NkC9RuuW-%$3j`Wx&VH;br4@1m3q9f;4g0a44n=Awl-i&}jNyHp76FO% z2rs#$YHm=XyZb9t$|b$~VW-YKU5-`@^QFl(YBzak)`aQ%xGX(V5jhJU^KJmPR{WnP zK}e0=N@Y&#=-5hd5I&u`=wwu#*Z)ZZgtJ-il`v>2tE%6j3)A=^?r%zuHh!sJq{QWT zar+&djf<;T<>7nK9>M3A7=n@-Mid;GvjfFN6O?n^syJ4Xl^n(61=bsSAOsMn>Dt2f%QFufv7v_a7gE zK*J0s)_0YiIP_&mXR92OC0$Z{I$65wKz4uhXx0v6y5mYD7f0m$8pik7dp9nh9#?eB z4;;ydiB#1AXkN_q1*_%h{V8Q#r!y^E8bce-oIWY7e2-L2htZ512XJ)3#N2HQAh9b_@VRUAzrD2sZzACJnu>+GS!;` zLJebkg|BZoaIlniX{by)$CtJ1DzYh}F{%cE8W|vKMU%1|DCl4B0QAZ_x|qPY3d`MIep+pC>jjW|yl%y4Frx7gf;7 zmk;^3Z-qV8>i6a!NGTvE5+a9(3SFEAa4!}jiQE-0hAaHL+0Q(98vHeB;oQz9e;qkG zJRXfIfA9Uz2vq+1{LISvsS;N7h?4!)b#ctBt|aIdEDXN0cuh62hoYucb8f6epG2`+ zLv2Q=u9-Z4&CHQ?l70`Auvm9<^Q0ef{V1N)QtZ*8xUuW}Fs-Q4T=b~+lYA)*X@xw; z*nK$x+%W?J(MDED!uHt2gJ%`vd9o|zDd)d!yu>tyA7(h0{41A^NhJN-FfHmxC#E33 z+EG!!JSpba=Lg8h19ZQt4r{x)pwWSRR_AGcEcsi?Xn~zbD@zD}9{ghG9F|_JZRf(cY5GH|yaDib5}R1;vm9}r zv>6X^5|)O5K4+)=xmdnpJ--sm5FSf%{|_T-z7k>qGOO3RN9~MEs#|mO^N>5!mHtmL z;cs5?VuitNb~~xY6+rp0n#aX#O-;e;YjIukg=AQ<~tZ8?|J{y*NmsQK%B${D$W1rd2qBPweqt zSR3VxqWM~((og#9cv13>X@VCL!E_OF4hkiOn*oAcM7qGd~yX9|=EM3;&FV3FW$hv&>jXqxl{d_9Q7*&x$ z-uR1So2cz(_zY9Pc0Wc#BQr~JW)m)YUxsjckCD1ti9}HTLHE|o9mEvyVBc>~Bj^T+ z#V#;l)LAU4=dg_4;M3#3xwaQ%ub-1svlVx%k5@2fMTjGPYmmSm8%{*5Q%)J#+!#~C zq|_$b8oGHMpG-L|PWyD_Q}$0M^BF(ksNI)9@AZN& zx%N16qcHA<_;0RV9msGj^CBL%xu-BAa*XHYY)f`&1jH#V$-Yd663EXM_tn3YyWbLR zd}=S#u<4$nZI{yX)z}nMb=jDh)piO*9kGWj2-c=#xmT$?ILoNw@GYovpx?=@t+Hxy z;9ZTuZ)n9ZL^s=^=J%#J3r(uV_|*7p>8}xx$I(V!H!CS9>irGv zv|{I=bM%xjtcE1vw;u=(ID8-iTH}e6nW*5g?BO#bt!Gr<{f2gEdfM?^ z{wu|IfqoZeH2Xbndqg55pyd|NIH#!o#BhVu_sVY5mE*qzscJGbuddi$aCN9jDvu1k z?_v?2#-nWo^7g!u6m!PTMr1Z1Iy>KPvAgx~N>U*M!|$2iWIrXo@LQ2!BAyX26R)I( zB*sg=)-e_u-PAfnIJ5cw-WPy!@bJ!Q*SJz%`iyC+iORT2*fcuz zO&a>L1xC=1GblEZzfO|VcE6wMB(o7`G~W|>*Qw#~BL7C2wg$+{^Nuu1BCh8Z^1&mJ z7}SnaCYwuX&tCXQGTwNx=&+D~43AlcZveMfSug48FWuI=_=o7V;SSvQ3De|Q#Co{R zQ1+2K%q}(PdlalgExB9LTPL(z%+P9Y>pfq@JuUZ((x7G?^-H40)GRB53`%R?pTB>c eI-p)33O1gqBDLsr`ug95S;nEg6pVHswEYXt9E9}% literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/block_12619985.rlp b/statediff/indexer/database/file/mainnet_tests/block_12619985.rlp new file mode 100644 index 0000000000000000000000000000000000000000..0e735313f540a2353b7bbe65abbfa0d23f5bb4cd GIT binary patch literal 4041 zcmb_eXH-+!8odc2(t8O-LJ^cI0!nd!h#*4`3P@8Cl`e#$C?JM}CXN^+LHb-lI?}5! z5>R?42neVk2uO!f4ANdQ&YNMqH*3xOm@n&|v%meVoV~x?b8pTjXUis(8`L`JsbRBV zajsojH#qG2t6fEE5$rrfe8Q|*!diV#nF1c{GMjtXdLgJFqSCdhYfzZNV<19|3E*9Q zqPK$@j5j3Ovs%3#7o5l+*U(n1ILX4U15Gsrcl!>1oEwb0(gb9)kkSjzZfi0L=vm#? zHs)@o?1goEgQ(Jr)vNJhiW6dyS(neS-{yu_sA?*#&WntaErlKSIzgj)?V~E@w-&$b zK&+4#9XwGyJU9LNHN2z>a3z7a?o8A|fZqZN05B|y2BT>g7ww`2oJL}QBfjVK5hejJ z(I!2B003w#0Ep9)-vb4}=p}l*p5@wxt(P62?lL`I!X`!R|skiyzPOqL53D*ML6fu52 zif%z}iUG>XDxl>x)0>rQ6KrW1TjuH}8(q%I;X`rGcwBhPks8|%Qdgn{Mpap%mooh} z*^8(cJX9bd0=_L(2;doH5ky7Nu%7PA+7}>fKQ9}LoNwmvPmhX*YCXlv+c5!HX^^m3 zktk*u^Sn7UO6>0Wpr4jU^h}?pGX{j6zCLfO=zAK}R_k&n#T=2hHY57P=sm7%Ett&c zovq#tE4E=l=R4O@eef_c?>@ySkymR=zF8^fEY;?M#hK%9jSj|9*POZWa58|E0h?qG z80A=NRq?m5a)++hCJ_c06^?N1`39DiAoSbY2f_1;LcwYFpQ}uq@*p96q$F$7M2nvC zrIW|KuJS1uS1Qn3d@{06P^&s|t57?4Cdc-*Ls7i-1V=;tw})_vc=-^ZO& z_XND)x!0;lDsNMAe5;!p_-udue{q`08kx2Mg5r!Fas^^-`?~UZ5$<@2{tcmq%w$7S z5`n*>QTPNXlFJ)NH7s{UcHVxaGN**A^5|$_39gyvb(sC&pE({vt-vwoMKFYN4*__% zFqRk=>$CV;VDV*mTCtnK)M6wys@ghQ41wutR(qCEqaP#(v)S)t(G_qsH;Qfja*;Y6&%Z5#_Dk?sb#GVHoFjavA<1_2)90KTOFtr*H=)mP5O@UCiglWH zN6*OC`vzIAH;f-@VJd-kZdx_n_; z${E4oU7^?81HF^g)rU8>m+M*xN|=XyA|)kDd_$WL4q|;-DL8@!fADo-#ic&e)y~If zHKqlRPCMf=w*$wH7zMQYq&XestJm*T?Y5iWIFln=(QTGE5#GbL%IS}DMIFc>1*iBA zXD!UWe!U!yi4g73pp;t+ZOFYDj zl>b_Yf(gqd#zJ>@H`1tmS=1d>SJzmMn-A>~E=>2Uo#B_DQj*&f^ScFu5{afM%39i((}4_yqbv zY6DW&^BpSn|tgFsA*ZO(;WS_&nwc^?OPjpTro@Plz53{eMC^)LVc*>2WlY> z0*inme39jNhz9nbzJO4EIqHeX(07XZeW|Qd@5UKBB+~!fIRB4y9Rs?M#_NcFE=@;{vR+yhSwAFw&I6_okmQ6wKQ|9 zc|3J)gySWsFNE@1?fvz@520pS!f}^78+Vd({j=0Q_erSqEf(c@(9IP8D`uoYoA4yN z?zxN0Z>O4$?vzMQe9Fv@yN4Q@P@8cDTAv0W zKPQ!ZMzYUWtmr6{>*0G`JAws{uVo_CWA!ke= z{0W+HcW(0q$9h+Cd=b);>WTeZG4-aBaQdO@bkUozJ@#)orui?`vU%Zmj!*u0RZ=v& z`4|}I#oD&e0p6-KSmxT!o~Xc3ZuLAcmbsxh?yc>8nW-p?o$U(e^gF62$eX2+){nSoKuoOS-`qDv$fS- z)&TDy;X`@^rJ%K@r&C>2Pu{;3JN6M4W=a#&7e;_~Xx|!iwQ}6uV-De%cW}XKt)jJy z<>PCRN!w`f)jA7h6F6oMMpz2D(%bczics6{#8ValemmzgLKkVUt>oP-dMw1O3TGrFkpR4{KBrGKTh zr-@PZ6MdXCRW-_8qUWIU!xCb5n^!-=!-?~sh#l5y3Z;wb-6f zIwm9Xx!ZP$$K&c@##>om`lRA3eh|9=+MXy#wpKS3Y^CWm2&OME(BgkB8|v|AFPV{}Wr z()BE4*i53`UM~}BfgAnmYqQ;Vd`rV$TGjRslMNBbL!dG4gu6L1=TgHdiclj=y^-nb z^CiIB}1KV#p7vNN!>`=Z@^8ORDVkZIHO6*c>l#M}zKB z%@Aj#Wa*B5@G$cpe+*_9qtTkD==}IpDf-bmph~z`ezH_@7hSP8%++E1_I+|PjhM%< zXBAj!y+IaUz71dy{6Po;As~T3Xcze6L8D6g@s|n&k_rOhfe`S16hZKJe!P#?4h98D zNx^HW;IDu{LGa&TG!WANG=TTx9XJUA{x-Zy4(oye5>oI*D`+4v2K+TR83aNixdx*8 z!4=`67z9ECBY}`Ea4?7p?FS7H4a5S1_W+Fw30@xr{{bPv>!sk7i?-nJTrfj`7o?=P z=-_W&6sh1XNr9x`2!uz42f_z~!9TvbAi%5O&HWGfpYBGIZjFR&T33g*=P!gVb)tkW zwJL_0c6JRt9uEa$AcR@ncejqRa^mLT7U19wx3_Wh@CIa0^;%ooo{XbU4dTVGY0qT$ zfQgu*<&pU+9s60J0m)H0HEs7$5u58z#7be#p{&P~;o$o~Gbj`s0YH_>*21y&pZKm` zCz$Abj>T~Y-1Hy{TwmMhhoRyTL=SY0=sb(jnDfmK$F7fh1Fds`5ZFNmC9MZ}ee@;p z%hv&VvSm&dq$LTX1tD)DAw2`cGh&}ap@mGTk#d(@H)hEn9H%aET%4?l-0)Z9@0X7a z;PB3E6)&-R3Fnw7g(~=JGKc{HS5cqT-zkA>)A}zkVd0+cQ<@ZU(UJZ=YmoK z;G1A(e^Zj&M+xpx?2!YptYjmCNAFCL+o&rTe_3h}hB=0!et8>?X$_QsLUhoeRuU)A zuldKQtJA+Gxnn-({QABRX(XR;vL{bF2+ogK6xUR35(^WewI;DJN}=1MK)ATyezr3) zTWDvE=NJW+%ro3BLVmRfv!e}1JW{m%!3;=R zL0QBcQygBQGa|aavBb5D91z*zG46+YU+Wl39jY^=jOge_CivONMSuT1Ma~B? zE1R+*a>B|R;KhozM#afmx9M67kWop%avq#tr_(vdvy@O^mXB|sgE@qNp}%3*n*Y?< zNJ$CKPA+-f;EO5`F5x%pK}+K7;Rw^yXHS%6j+5Bus_1OXl>tfAhumU$f-ERC<%lcO zgM78mK13Y7e>!ud>u zpmA38i?-eKD&!hM5ChG5f;fkOr?=cc6k!7*_9$c&3~ssHtSHvGxCI@Dg(H-^6M{lfbEu$(Y}Jpm!sgAESFGqdtn%L`;s<;Rs+VI2_|Px+ zL~A$XE)OM}4l4+JAYQ5HN zR-f8oLhA6;A7jY)#n#PKws&l*Gwu%$$^j~gYad>UyWK;}W>}3S*jtcTId2%?EaCtf z!L5fs8gv7$O z4VD)8n)TgjQ19|L*V6QTmSKMMr)+wu32sI{FB`aI8Gzp1AZ<5tJ@KZ2C`R z;RybnnDEiyJ3}Ws0#oBTmT7aT%uyc8r*(*jQgp**UyPjDbh$TL4;r2vqHnUzd#7A~ zSkiy2^0fBL<@!HkOuixzIiM$(o>#Noye_%ARGQ{QuwYc5WC?T5eXEqcxUHI(@7`Vc$-PdGF2pE5vF-)!E`>Fab!I!UL+6JA4}icUe2_@3Wxago<=EXhK+r_dfTT64Jv;5W>MZq7iTt z2rYwN1I`9J!1)!+MGh^!0%TKOcT!&v`Sc;QKb z8z_Ad$x)Dxl!Mzj`NJpb?U=xO0G@oh$t;oRVitF{ys{~MvBzl{2RZ7qV)o|(IR%rq zM_{(R;owk5>K}8!rbl2wg(-IkH8l@pTC?H{8{Iqkh|+TMHfK-}1W3WL_T29JL?Tp9 zhr^fGm5gluqS3S3zfhtKr;R6i*%AAZW77Djt5^-mQ+vnsb9h+Lt$4ha@B}`(vT{D zVh~93v-V^uvFnhKiR~_P(1HRRVEup|m<|vHQo20%c?C*3iLD%S=h>PDS`6l$lNrO&qzlj97=YdIzz*$`Yq*O-qIDc7Nk?GX<#z{mZ<$!ULL%I2 z%^U%ziEz%5R*XJctB@P|h>IlMqzeNrt)57W@>Yiho?zxFq5Q+ zCA5xwyCZK|A(&NDVi6&cN9;F zsP${4DJUgaWG3t1IYc4)LSrQ*x~{{ytuZK~dZovyBX=#;yz*rwjb11E&Z1F;$FW^y z3JKLc1SCWBKZz=L7cE@VRAJb(b)m{!?>9k+^34)1chWwn$M|Jre`kf{K-!o3r7R{^ z>h4ny%vVnNHs_D2W58UslKIHK6iba{831ebnA=;}^&zE-?o!OVohkmhgR=!sTeECl zhDH`o&VT)d7txy*ww=<9QP#0Vnd4`{V`p_lC=qMMFa`SyDUv&@qEjjiDP)etue`z8 zk4b50jSLFW4M1JWnh)us#sLsX+VweAPJ4rO9z8}9x}UcKVpMzqcGCQUJ2&z{;(a-+ z_zx*|pF$12o#XA_llR`k*;15Yzt;ZP_Ri7lzc^6#4Wyv3-v@`T2eDerh_Sg zG7=djkbWdY3htfst^H@i3|{{&wg zwz|_hoys_t6ped)@>&z62-aJp{tN|EzYhiI=>7dM?iZmzb)hJ!(NdUbgRi0JNl1Ru zkaB=fOnGf|vN2Fq<=cS`7%3v%*dM_SvDiUt@LUZ)l60T>hETTqmijp@0-Lh&{4W~+ zJL`e~WQy0T<`K?Pmhg_O>v^UgCB!I@t0+IuB;~zZ!7+|I4q&JxV73ns*nijD$UuGs zEAg$~KTbFGav0`&*(L2|GYE_RBkUlh%tPxH%oqCZy>N*eIi`=eu@n%LzhitB zcS(z&q=cMi9{a{E5ne!d>k#Fmk>a5!Jzy%;rqS0Rms@VuS<{C{_%JLz4>B6W5lPcF z$4fI_E(8dD^}%{x-3Pi(Y@H<3hax?6M@UZE-i?DJ6KYbYeee9IjNvYPD@cTJGbFWA zv9pGAVVB$SD2pB;5~E5`2*vxt(t~wN_sL_XeQqY}@3nS)kr~vM-Vu&YNN{{myVoC( zKLrrrxQS5;AQ35ih%Yy+YlmXIHo26=-#~r`Nb$TlV7gC0)wxh zT_y!xF8nwC-RB_b7i&fSq($wf!waLEMZAIeV>=o6)cu-xGfy9(F`lh9QGbYGx1ObZ zH~5XJsVdgs#3}0FWplB2_!te1!g4H1AmqQ9{u|0~J4g?heDvwTUpREFm{V=Ty>?<* zu}-n=nd`O4Th()qwQoZfkfFXUBS{m4JkD7+^mdlvnzTev$S|d@=OHiF`?&AJ#juh; ztQ0I1^iR`w{Th?9fkQG+a&pUpb*cB~1rMvw{&rx|dQuQz75jE!MPH+6nmffiZhOiv z(-cyRD&&M|u}`W&gr^ zjnneQi9qP{P=$2%*iFZ$LPZ96?2?5eV*zQrh&Gg)fM38YrCLXFx>Mz?ko0hEo4W8+ zFpxxQuQHSyiE4+$`WIp8spH#-s-73`h1Z?Qsvf2G?K+=tY^lAR z*~(%jsSc5r1>}{K0$fxhSzVP;q?bJdoRSCFy0u|#r%F;-Oke9Vw0{}c(aI;3^5`uG zIz*yZ6wlpAcL}UgBiK{D-@;QbdH`27z^)={cZ}SKW^8h-tHIivC^cg3)3^YBVvtd! ztPkL3m#I|r+>D#_b&Cniw^)#C2gTB@rSn178x?6t{$Stl2ZqHV{aE?Oz?=pq)!y%= znQ4*9t%L~=pxafcJHrM4*}Ag7=pXhauW%k!Z(ERm1~X`$qS=74 z(gJ6=h7vAKCGagQ8mNR+6?16LO>fL1WE`|{Vd2`TNNk725vd=U3i^yi-79b(9Kjt-&NB^=yW@Fj}WD)Qok^S1&5fXdHj}nu_ z5$Bg|89s6S!=^D%B^iMV$#v=ED+0g}XY&$`8Cn_ukf@25RZE`z)2!S{VPtmnv;7m6`HL&AKR zwvOf^lR7bW!H+S4TdCc+1P){C;SWO7WP2mLnN`0e3W({sU8DF+z>#1za7hXOJfNgR z-?kUkSNzP@yEUP_z4fKNq5H|-DFO4TXFCKSLRONI?=j66F~c4#&dj`j5Mio@U!>5{ zw6~0*O&1&w*!RB<)tB!~d@7+#vM$HUUixY0m805u(OXffD2}@RihrO4MW^~0zY8T4 z17jmx<-_QpN|fE3Rp*%M1jeUA1+FZ4;LmF}T5%r(sBah-Cmzgx6A_|tc}-Dp&RnFb zK)X0%_Jt%)2btOa(IsE-T7iy{UpqTq|CBi8hK^jrntmIBEBH|)^2Ea!59mcm@o1z? zI4RtIk9XVYLyN19@b$Fu0pnGr*(sdpb0i|-Sb(J9yP^d>XN$+129jHM7Dfv2 z-T`hbD%2_(?M!^;Xxi)K>6`e5yanhL!lc7$XsCjxiRb^qzQkQdbSn^O$L8asw^=~r z>W+=#rcYk*#QAqp-dubvHxArFJ<1RgxnpWz&9-oEnd3FZ#nPY3@w9hjGj~47nlR*& zebK@GsgonlZRmHJ=L@+Fe{Gp8qDp?xA{X{|_636Ch%f+NpkzanWfzUC8Io|870Du4ck_Mg*Jj`N4ci?@Q8{5z&J*y) zAG0fxA0N~d2Bv;ZhOO|DF%5Jw=WVK2a(5Y?3WZs*T)9j0vxDe*(LW1UA_#&nlQ^a# zR;;~x^~t7BpDe})oiM1lSe6pzLx5=!Nt%5-^0cB~95gwyCs#yMV$Eo9^7%WH0?ez9 zrEvx$1>?$Ae-hh&!mALN>MH6b8u$_h-FKEPN|>UEbb9{*Z(fd9M%-CE`*Qt%W5l)p zru{)y-BuM#+uk;rAY z-WFKe!2Nq5W1H}SfkE2RN8j7Qy96;4UQdYcD7+Nq+4G7g)zP&8=<`Gf`CbnsGRpB) ztnJ7tyhi@IW`9&FE)Y>!#KFZk^DkU(Het(?8@+mHi?~TIKk0~@f6=LM9|b{Ipqte3 zL!EypAYE&@y3sli7gm7Qi(_u{IfpB5HPD>%@n=SsgH(V2J3j~ZSGf>!^YD;Pnwu7}FsJ95%>8Fj+E?aM6idsu5yPIBw6V zCUSEu<#Q(M)s-VV{bUATtEe-V`sk2a@~eFN{(Vlg2G{>iVss=*JBBbp`^7#K{{tg; zK1w>I{f8z3R9md01;qM|fc+Wbj@OJVR%_)1;C=FXJoL#cmu%N?cVZAjh3la6vGB*h z48vvlFFDfIx+ma2HljiiqL86@Te96GE*iDsMgWU!A^8`URJ@d^)CKl-2l!_}>NKGHGq3VQEKaX6c4P_N3}*ZD0wG&Wp?Sf5xVH zrig#0{i%#_xlHZ6g%(uy#KYuc218V;-Gt!@FYI!CliE~fi$Eo0@a^+9_WAgSZq4Y% z5a4X?0M+B$?(n(1?%Hb{ySkx{;iAcpsn@X{R~An5*vtaBnddZs4LLp@mg7d28j0`z zD9QZNNDntwr)#|j4Ugq7s=aa zAN6?}P9`zCREL{(NhFx8X<2xcAMTA0WEGedix$iiq2pRuN?xc5#rBMXMNAYAwI&Cc zJkiAfJ;q?2PRZ^xrX|oCCFFKBXDJb`qWu<+fn%)6_;~_N%)|cG zWz}DG(r~%{zcFV0Wm>_U_<$sDnelQp3x+!9vx0)W$-hmfO$%_!FMt0F{@-EzLiYr1xgS`C$61gbiuw}3Px<()a{ait zSxjpCtUoraYqn=Ik@{-zpy5WC|Uaqn}#@6epIwlf0P9&wmh`^a)}4F zxb;fnx{pNw)neIse#18mug`DP+$YIvLd26Edy@0ZUekYQ)4o|g3sZS&MRB$0wu|IG| zOo#4;75Y<-Te3fO@R!q2*d4SDtc&m=%%6Zyt=fS;-r)S|PIlAyo5zeRqzC{sPL~ZoEb!cJU?AtG8zvq>>V$6pda%nP$GplHL&v{a029`s0 zR9HSz{<47RF5x3qI0ze_*S;$}XU4V0wckA@I2CSJ$#f@Y z1Bh-M1C_G%HDSQJTf7Do_1bDvRD8i;V$MpOUWoe<$@kpe_IpGuNTs~Pw*Pss4~V@kFBW0?xa zx72~gE#AjLbdiWyw_XSW7utxSCS^+9CXi{zx^7U-R5`j0XbZZqZQ`Rk(!cqBj}ZWT zng_}0dMJ_Q9C`K*0mM*lht!*$|zZ{!!J>yLAHS_lhA|?id=V1O-bK}M}{?QgHKu9)y=U5F> zkPYU)sHBb_&C#w_aHh3?r{Z>?4FGIX{bToo0QKHWNro#KxbOo|&EUm!937fRi2~=8 zrfl1Lb3v~^(E*4#sDZsCGvNbtA^mOr_7U`g9mjU<28S}U*+l5>w|i^ zT(+9~KlKd)X>LD2Ay4al^QiJh^ZL6NSO~j7>*e|@Fqi>Y+V6lQeogj4&Iy!n1sfJzh{QF(UlI~yim9zg_;;+<6>rS42tgZ?6;x` z9H-~i8qyotJV@UTo+L8?a`Pi(&~N+}jjvJM?#g0sEM$e>1Yx`9OEBw&o6Uq*|DM?& zdwul4F>n&+g{kAeq=K7RF-31f?$#T^ORYerW;9;lYC;47G%@@|ZC>>~0x_B(Z%Fvw z9+2Z-Ls%>AwG|RwL=Vm8{>Rou4!j}7C_{c1WlJkN9bBq0F+D$tpH5lSOw)<_=pe*~ zs0`rQ>k+|vaDD21c6efUb06MrEjI3T*!M!Ap`q(`6tt_qk1M#Ad%O^z0nq=MxB>y_ z%1v2mZ$G=Gzi*cfUvC@SmoTf#BH< z(}cNi{uSnanG|%n@C*LmW`3bvDj+pD1i5rMcU~g>|AD_G-rH9^mm5v1HNYE?An?+T z-W?>QV(X9-#mJ3F4SRR_9lu7Lx?(A79{JE#FPF~EN&2NXgO}?s;pc|YWDwJ|;4=n5 zG9HJJ#vol&?!E59#*G_iqZGnpxA^fDWy-hP^pp?z0loZ^2c3`EBp?aj^2vjP+qJQ0 z*=VBdn3!v7yFgpWOuyJIe%6p?kFFnbCTqplYYvPbI9P-1B6cAvLP4M)azOjTSr>ag zHrH3zMwvt7NxTFtD>~u9-lw?VCpSG0_0Ry{8!a^f^NDxG^BNL{2G!H)i?@d#6?I&b z4%Q4-Sx@{o2~bY8L@tqVd(j&vl&;reQ_NXmWQULgUq8kN?8u=9qZ9zM(7r5QlG4_l zIxwxUvABcyfTg>pPpsqI33|xJ-W=wo{iOeUo5qNWqU~@U5PCk_y#B>Sqj{6HE+bL; z{qr@{*7-=bH-WY}JU(|$04B$W{g3vtnd?ZB1|&62mNcm-3*zoKx_5W0nvdMBee*L= z!PmxVxT{8+5(X}rSUDYzw?1;i)`D;%1xv}Kw~~Y4QL(|;+~wOl>rf@0R!%DuA7cdN z#>D$0Juj4BTGL|{nY{-z4e`g){TIX|(4~T40^1V2I+cZ*$>|XZZ|`-!Bh5WO0mHYJ ze_RL&Mc6He`qE?TbJcv%O%tADyHi%vWw@^tACZZiPsnzye(6kfO8S!BU53bZ-FMk7 z@Gg_C#6g#-POsH^t|MDX=PoC+-W6~Rd0BY3=W_j@v1w;x#Gh$@D*xh#zgq>dQhs`0 zOee7RTGfx*=-9S|ndey*J<#bZ>WQpr-Xs>&Mf(mJhEb+r1hoiU$p?VMHzGbQQ#wR!B>lk+c!}P8}dn{hNW=BkIcno4zX{`O48} z+0cUQVudZk&Sr7DDEHeBAu}7J_^h{KCRQ7J@u{ycT>sd_rb|f|i%t{SRgU zvY<79lP&UC<7sRW>+NTcYZnZ~HiBm@}nuva)-l}vWALzhe{?}PuRKi{x7zAzf_@{|D=?k4o$NlLh$zo%4{r5(%8qFm z;9_Y<+*cQCD^=2L2ozLg>14XuqA|B;UKJ<7B0RLe^PiRB#g+am?~5T!jMkB4rsu-d z;D?jT`{Lk3CJXO*B5&c2yq(nU`s2_8LH64ZOZ-x|!vT+nqZtuntJlOjo>NFi^vo3G z1VaS0AhEX(Z!Kec%Ko}ezj9TA0^u&@XS?u!DvqZF~+NaqG@)4QF7%YFStEhfA85+Iu&3PIXZvLJb@-Ci9tJH-xdvVFM^*gmc$-?XMXqx!yAU1)S??GTe$xCu) z=I*va6Ckk@D$X{cG8*g=Px9`I*3G%Oxg%4NkgbHc2f=&SwtD}un?eA^Vw!FlW=lLg zz+2e<%Y;8$L_Cvu#FRtMN48%5ci$-i6~(}7R3U9@(*jkd<1Tf4nJS_|DB==6Rb_d^ ze41oL|AN=Um;)gv?{HW_Z?E-$b6M#Uu6g^ALsgH*QC=i@{EH~yK4@fsc}XbkX4Hpw zAcb-bx0V2f;XcsX1jY;ikL5(6x8kFz#CeedtDXQ+wCe<}=a_}S-iYv7c~Ctvq1 zY?N&FRmH2M`XVtD5j zVCP|QS@x?*VS<&+h8;*X6&XOJNOjyy=@Bi@H5g?7<7J_U*iuk$HEE{?-P_3cU(tj2 z^ODtKMxQ>4M6poSwkQ1#PskBO?#oE9Qn3)FnyrpU-&}4$GM)Wov9$SE*=ld%OMM?; z&Sk}V)3UGML*A#{KR$MVmR)uZT_@yUmNFce<*x92FPx!7EnzyC8v(09`LGwo72 z@Ql|E^-}pND09`d5)kfKXjz! zXY8zu>$oA&KR1>7BA^@;L~k^|sUHJx-CW**Mi3GVZd|)!PnVssYS^|=u)asUZrf9| znBMn1=`Tf%=*?cXK5=o^R8NVeS@k>`J;)I6g(U5Jq1)S!&trLA_flDf@AKQ!Qvv=5 z)Y}vy^?`Jy2Vqc!eYUS=vo>1t&U&n$tX{Qfg4BNTPl7jvm4gly+tyFnsRy40jPEOP z_C;{f%H#!}!;P3J6K{s6>i$7B;l*+zsKzls1V2O;mB+Mn^WWo;~Pg&ZI5|?Z$z5NPw zt6ZA;+X&$1Ir6OGf1(d_@7Vlv^G35~g_p(|x(XieJx9(q5uK8n1sFeNyY3l_gUUl# z88s_Hcca7_+u2O2vD$59OE4-3LVm9V{4s~G-^edC9_Lq-b5DHbGZKx9#rnGx^v^Y+ zJr!Q#wJwgf6dy-FZUIDi^;?H{E}1w?T6iW{7Hv zxmfdb@WB@x3USwzx$E@g>>`w}zC`l*h-KV_pAj(tlSNu(A88$;zJL69w^pcv*Qt4o zdv#I1C{e=XUXy6j0${;HrhRyKwJM|*k1!SE$e!x#h$Sny?w09sai^(efY{HW!6%!m z52QlKoCM&^$N^@S+BC?Pfo649jJ9llI*j45oM!a1JRygKdPRg>>X(4cb1o}nVS(3C zHbtY-cncIEO%1!%E|qP%?e}Nj;-+-{WBY;t1gP^LcIZveECZ;mZIl|csXpl3Rc|Em z(wDcQk3fqh22|&4+gFM|P&DaflPL!lP6N%kdC<24$SmeO-99YJ+x_zkP#^mfsZr@h|h9ShP0c9f!7Mi^w zYpLgeK260#XItzo;=F>O9!NYDB)cN3ElPSc)1cCG&p)gD9}kQi7{-)H>Z&N0{{#F85+Q#F#s9ZjabX{p#g`e^<a72n_rpv^;>sz}CvpKs*o* z@G7mB0RU?4iG{`=FVq15E~p5LMeqV-Bz%1W24XLgA@|*okUk6=fFsb5Q~*w1pUVV! zrZp7+hWC#;478*eR00_!9Fn^3P%tbOfaE}>uuFo6V|II4TZPpK;lke6z6yIE&=F2- zo}Ewa34@`S!d;#HPDQ$UD=R4PRZyXLxOw@X1LKxgy!U3`n{+-AP{C6h+nmSg9lBys z89H1LBi?ppoImnXsSU6#7M&22#?^E^|zV4G7eKoGwxf7AsI$s z_DYo-n-;1st&&!`-sRUgXl$=8`l@@?A{OFlM&kwyjNnx5bIt_#|wZ~J_<9u@y z5sjsZU6zvyb2e$USwR_Ontl#*TE*Jo|7k@m%S=th-&fLDwFE<*0B%b-uPcj(*!R^RJDbVnx^CLpCqg#F2{p0<7&#(A?yljga z{Iw{cTze+@v`zxJW2@WPrILz>ZwDh*9jAm`UPRoZ&Yz9(xNTB>qApoZeO>O;|6T4W zYwDxzm2Gl|?h4{-FiyL@qz^SRsQW2|lgEc8LE4KX8)r516ITaHwEM?x4wd=`3(Zvfp;vmHsy@%CD2Ux#{cO~}1-`&q64Wv;>KovqC7BPM%AYc9PUk8Ty}0mzczo>q)#VsMYZzhpD3I`tgrq%pq7 zN?+p6l<^S>C8Ts|&`@tmK2};P_5SGx^D>;iIs21~x^|ymxhG>C8#<|Ji(7!1i%{Sl z`;G!(Z~~c1lH{;bDrqBJQ}5qpmtRtUbP5*}$=kurah@LlXhRpKPnZ6*Z3uu_BnoCx znin(EeLru;Ghk$f7+_D33@r~Gq9O_Sh*<;yU z^N$=- zpH^M50Fy;g;BnanKp0Gaj`~vcvTevjiFJ;fN(EBERlWS4&q*?^Zp*6&I7TsyF;6IJ zc&XmHaEBat? zS5OKJ#u?cfEuL3G3=Z06F7KqLAp!rQi1nZVS|9E3m+(ycl@D>t3xPuU_Ad2XhPK^# z-oCrqY<%v~=VhXVSTaY4-ZZz&YwlK1&Y=5sQZkLYqqcrFL1?Fu)A1@x`1bhzT;00p zl?Fo20-PgM2uFciunj?XFiMq5*J=qqwWNhbq^hzyQ)MD_Mt&b5m%;8N6pa7ALI1{7 zSf;tU=lQOb!hUXb6sqdY*eVX*9&pArIEygH+f&1P2aUTOyOSl%+Iae%&e;pMS+lp_ z;m}kxPl}B#-+^cCk=nT6k2%vt##|)9v{PguyhPd&ZRVSfnqYHDf5*sg5WQX}0&TYusJ#5Ka@c?|W5xdwJBibL-!jhI?0^}0M?%YfwUqLQh>Fas4J4P* zfzOIrjQr0r56^c@3d}QIaw5#!;?_P`r`mgB*IeM!S)ivy$K6+L#Z2qEy90=__~ar+_zTC= z*C+I|PGu!zctF1_qV6<}pJgy2%s?98w?gCGpmjv0{y#-z2 zmZ@o@HAKa-MUo|#%9m$WPllPk4LTiBl6eSn<^J|7-GcwRd&6gkY^Gz=;rI_|s}dty zv+iUQmTZJR)II$vuqY?^TK6Pu-aTQg^)&%E*!KVD?#Xho{FxwUpwDpfq#WZGHi^lM zSml69OY;3EH@C4A^6s0Sm(*23s50^!V`B5D`$lX-{zJq0Z{0IMMD4@dxt6)MU`zj- z=4gL5JL0q?Ncxl|`&L-J-$C$4#?%JwQ~n;*xHXN8sRSqN-HTUmR{{wHAapiMp@XWKS+dE zNhoZ0GP&(8zz@nj0MDo^J<+ktM<1~DtK8Zz>pfwsJrqFY+pZPeT zJT1|tD_>#*x7868HI)x*d~O1b7CkOVp3HH;JS!2tJ2p{6_kNM1+%|Q&(V5s|*?i$S z-HG}8sY3ss%)V9_Mwqnf@x=Kwiq3mJQr}vCh)k9OMOQPZ6MI@_^k&7r`DtKv%DWbM zKe)i&a}QSXm^Q5*2f6wa)B^C7CwKT1A{DK!8uZCjc<=FYxG{NF>s=HQVe0sgoNM}?$-8I$*ThSD z{B7mA7Yy#7_~!|?(W!fuuesy06Bk|cqR+hiZ?<~$>~)@b=mT&e z%fEHNM_>L=I~?|xmw)Zx2lhEz2MixP_j|v$_1;H~j4Zog=I1_Y{LTmVzjV{PZ@A#? zU;f;}M|L{#r?a%@?=I=l<6dzWt}m zKK;2RH}1CE0VDtLp7HN|{>*=y^~KHG$#U3tqDqr-pJAwF#Mi%0%s=U=-0&Y}B; z@t|K_d&6g4xcrl!UiQe0Pd)H*+nIM9{ujrbx9?l_yLs_@9@+Nrdp7#!PH#E&itqmO zEiZldoP%%Qa(wuFt9;XD@^Z!4@F`Y@r;?lPd#1$B_W0EO%jc{;xOhCc-^9h^|MS~5 zlNZu8zjEuW7t(EZTBnh9QpNUZFv7W*B!Zd^BsSG=>|7#KR*1#U%BZ8uU+r((;nJ! z;cKpV@+A-?3og ztB)9)xNC;n)Wu`PLo>a^4ft*H3E#f#WZ{eJDljb3>5gB$NZc&FZf z$dVr{{mIFP-17QkcRzWHcmMp(H8;NI7k{(;7KbcY=Z5RQ@};fECr&VRd*;4puJ`tn z4?5@3qrU&k<7?i3?mBNe_@MEr2PeLs{DJMByXXAB-TOoT`sX));MP-5x$+xJZyJAW z6}RPcZ<}+*TIa1d@lG8veYUY_9Q;6@_=rsXjGO-BP8i|CvtZ~O=1kr_dCwZ7gZrnx zb%0O2&7Jr$IdRhI6OZbsiKm#nXV!OM`tadHRzB+n6XzY=zhLkT3!e0)EMN1c!H4R) zb?*O{N53_G=sj0{@#BAg)=s;;WRGKhV|71NNB!vN%g=fF{?EAnl54O1&G|>Z^VGZV zS$BN;RgR2&WaCAfZFuYVm)v&sM>hN)Uwqx(&t3HAuhaoU2hV%My!*bk@jjD(^_k_x ztyV{?d~q{gn`g=B;YB-N`0bC~``BYo({Br_6f@(C8?t3wc+&sb=6g3BJ@{>5rW5}= zep^^?@$=_ha?t62d$gZ#YJKXv7k}{fe>#5N4Yv(`TbSvo|I2?{n0lo%ex3J*U6x+^ z2R~o>=!}m}+__ThtdFmJVD|q~@E`Zv!qjV<{24U=gAWb;40^)L9ll=h;7mV?S3Ed) zgbN;rKRHZa)ka2VK62=Y!2x5#2aQgT8DHu2!&5Z;@}~bBPup(`H+v0j#@dZW2lUGiri`qEY-SAAr4KTzAB|I&3odH7o|+5Knx{?;zH zyy(ntJny5Mj1Rw*EkCl^o9|wF#bMi?eCCONed%EvkM8l%m<|{|_|->!`1Cy=92xvg zX;AID^Y2;BcWU@l|6TZ+hu8c5g>%mMmwlf96lILgY~u0R4vrd|JaV$g(R(*^nnPCs$d1%rR451hDX@-}qW!|UC7&fvGFU4Oja;?Hh! z+;?_9{OJ3(A6v554&w_?8vK8j_l;h6#of+wc7J*o_x1Ic5E?-tneAFIsrfhd*(Q4j4Xor_&dI^Wz_y z{L99S&%K)eD`WW7Q>^EHYA^2T5p(Q~@16U@yAQbMtWRzESKr?Eh0hxQkF4%{O12WF;DHqJw0Nc+M9C0%(Yg`f0CGrnf|F~R z-@lQOek$1qtDj1!FjS0vuqp->hKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p?X&3XXnF@zHj@n zC422KzVM`FM~&b4!2XwRdiM<%y#33cTlmOMXMg6L7hin;QP*9#$#xID@pTVvb=ke+ z{6Pn>^tRvm>9HFxTyo0#U%lf^dtS8gq7Q%K7#%S0ZKGQrckpHp+`YqsoA-V1!zb)@ Z=F2{y1E$Y5=heGk`K(Xwbj`@f{{=n~!tDS6 literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12600011.rlp b/statediff/indexer/database/file/mainnet_tests/receipts_12600011.rlp new file mode 100644 index 0000000000000000000000000000000000000000..ae6d4f0c2d082dd5fb959d2df1acc83b64ea7751 GIT binary patch literal 690821 zcmeI2378#pVQ(h^LO}2VRs;bFixm;2Dpo`-A)ugCid#X%ia-^xqPPHp)nU;m*toD% z0jYygP%s8cMO3Iku~NlWP)bFX5-che7b=oql{YiV>DK5%~NWh;S&=Po?wjGL}{+nKYU@bBAvc*eog&U)7SuX^T5kNWb$Lq=}7=LK^& zTX4-ehkxPX!~W~3voAdJaTomLkgLzzc)NQKc;3C+T>7Jt!Ta8Kx4YkT?lh z>r6Rm%3i-%*(Hh7KjHeFzxutuJoF_8=)mCNG2Z(PZ~yG# zSL({6m%dUnmOZtqd*o-38 ztC#HY^n1_$_YZA8GWgg(TrhjDXK(k-N1pigM{RZ86P|GFznnGWCLQ4L)W;q5z5oKeV@+Y{I5XEJ(E?r zkMUP(%MZ-_*vk&M_w!pk;^0M(zH;u(v!*OMc~xJjPe1MJr|kFoZ-4rcr_6cNb2pxP z__X(||E!V0XR^j8ANl6j{Bq5q-A~)_Vb7fQ(wmOi``ka$0S>RRzS$=GOdPxIGEZ{JT=og*QD$mQIu=bidf{izxaQt1_C53! zYwfuCAun3<(~sNXpV!}gr)SfKTbL5=bt?G6~`QX`K7!4=!zphb;w&T8oYjxr`~`6V^96j z{&x;uHVGdz$z{4|^r=twu(yvKe(A&ay7l9ietPY{yX`;s`^N9z@%^3Nwe5G$pT6!^ zryhCh53hdDIezK~Z+Yh*9dP2|CqHoFDUUz;+HEd)#&z?K(c=t^)%DcJKd0Av;tdlG z4m`x~8jgKV51+Kw=||>b$-iEvZM@%VbTRrFvH1VS!g|mTjHTtR=tF2|;@?P1PcHMT z*vzpjhR6OG|61llYvSTb|DL({QXYEzH?5J&4}af3Z1t``yz07hpZkQrx%i*{aN2+D zwW_~qoj&KfSMR>=h2P$1*BRH&`KueJY`@+M4jmbM4hJ86)?eMd=I_1sBU``lqx0vz z_bXFxyMB%i3?3fiPrc-t#lLAyJM87F@s(QkVg9%5#lLAy`|PW7vE&`R_LF}w?qa$3 z%L?8RV;7H}9bVxp{@;7wa&z8BTxRl#lgF-H{Hu$Ot>F3U%nHY9`|lstp8dX`y#1nW zZ-3~9*B<`ixnG_7t-rYB^pU}b{phKe%~*5$S!-YW@lAeu;AubISq2ofAVq*zdyNroQB8# z^oj9{Ciw(e_8mNP?52}_4Nr36YI_IYb;RB~9{AdyUwgwFkNfNA&Hk_N{@xxRe$B|> z18)0_P4?Sm!K{bfaM$t2z5VPBKfdFspE^epfVGB=;N&BVo{Pj9)e#rHONpJPV`_Kf9j zGJe6p#fzr=%jo}fwckaL`oipu-oN%*kN@dyyUpJJ!{2<(%b&lhzb?J_>b+$M!nv)0-Y~`CYT`(_OF1ljE^JcgEk%yyaE)>!a?56*weg+F@VWs_b#apBZ=zH9;~{d?*=Hy{6(;s#$i zcB1h~&;4#(IQkQN^ncsg@U3SKy5Q~kAQB$*(8N3Q!9O>+_p$f;#BC=2p15}LPX&J> z3{PCS%-;j=?`5u;bo%&e`(%E~Gk*NWEspretl6*nv-Q7l(mtCU_2_@wXJqgRJZG;H zpY+{z4qN;CAG&ALo6dXmCU>lT`A>9U@bDPFdfO|%KK@E=G;MhBm0I@Ps&Cu2kimPK z@IUeQ((6V$nB?d7_|?Np4=oR4k2G`q;uYO`$!&(0{5|^D(q}jRN}W0P6&Ihk?~SKz zdh^frdDLIb-|``cUj0w2`oX-{{C#dXeA5%I-s(mFu<;u{u;|a;v-1THjPQU%L-$^E z)@d`ZIP#3!xA@{V+kfET^Zuk3h!f?+YZig>oW&0Ss4%W(BrSvQ{Mmi8=m~}51u`8 z(*DP>KI-BR@4u?A)N#N6@E<>AV92 zzWv!dj2w3S=&v}Z%)I*Y1rJ|%;tmJCXXsC_`u!Jt`PpCn(p&Goa{E0V@`#rl_08js z85z8*&F6jX`gzBEd6Qkwzv}q=PXEr{i|*O#Q#vqs_^Bt}^iSu$cl1}gb0&Z6Rs4UM z8ocWX)?@!_Pj2;yIqEC#So3=eUU=ub&fofz*YCaUMk90H_78e;6VG_;@Ad!fvP&;G z^nXswulD3tkCL2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3T8NQOr4K?0VPpZ@OsX*MDQ@mml%t#}42A>>Wl9JAUCIBe&f1g1MV5xaOS0zi{zk z|Mk?_7oPdJ3x0CQ)#q)z-Mt4q@7`@L{n5zaH@o?~uU$Xym@jX#>-kq5f8Xif*?ZAF zTYX9g=3jiyvGX5##odSBz1h>=_VUZ-o^krgyXgS;Hf72=yT0-pe=u`sXyO0RlY9E- z*WLNJH%#6BZ?4$(eK-Hf$3F7s*If9y6+F42sb_D!+q{G44-GAvvd-{HH~jsEpP9l_ z{~e|Ii`Ls|%1PheYh={L@MI1K6ipw$e(|46j!vDp$p)h^V{FXWW9-kdud$0(^fhDV z_%(}fK7Gx_$EL5xZDuUKX!1X24$T@Gn!fb0Cpk2Fj~k5sXZqOPEP29PjUSsnw88K= z4E!;5^m`mVG=0%S9=q`7GY+`+Ae@s&3o`l@g2GbooJpcQFYMycCmM zJ2bSi>!T%j<9b;(5bt+3_K@u_zwIG^|H2nN@k3YM`X^V;`^K2Z~g2^XTJD_*FNE`Z`f+LU3FlxwlAMK_!yJ$O27Tpl;%pm zy+6BR+ih?8!@Ur)A^vpf>x%KWjH-F;S{^G5!H{MFWyUAX>U;B5r=4wiFvL~zy@m5!wEB)>!d+~nl-`(m;bFwF|@=iSK;D_EcYs0f%w$%>~JnDjV z-g5K>iw@U;$^87rr1>j-d%yMKZj2}Mc7iZu4*-K~acHfJ)dfpYUe&FNl&ENW^PrK;}2R{DkBZGIj zXu&Jz?fsSg-@D-0O*Xprh$ByV&H-<}XL$7As-vPW`pTf_!5jS!@%+UuesO&+h8DY6 z@FzFrV#39F3vw~x;y>@o#S#~HJnNoZOt^Ue{kfQMao5ybEOGIjXRMox2^a6)C>KNX F{~tj8_&Wdq literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12619985.rlp b/statediff/indexer/database/file/mainnet_tests/receipts_12619985.rlp new file mode 100644 index 0000000000000000000000000000000000000000..a9ba84bd275dc988f3ab79231030dfb3e8c96a6c GIT binary patch literal 571436 zcmeI04YcNkfQRoLlbK?DH|1D0QiKt^T79U@NwvP099n8OWHrS~`G|;HlYG-*YTKr` zvWkev)+CN8CGwHBjW(v5I!PlcHfBQhotb;*c^}=|2xrf6JnyM{pWpxezyJ50=Y4wT z-fo}0Wb0L&N`de@^L_`FLu+3qD* zfApox&wIixOU8z;wd2{x|LRjafBd`e8oS}dIX~XzagUuf>#v9JALb2SxAeP5{CKzF zLsRjKr#d)%Z1B>jdfMgV2b{U_Q?L5OnP<=X{MA3->#|2&cjLD2-tvZ1W^eSccfIPW zpI-3(#eV5mU;e0_-?#1Icf9b%`~2qlpF8%z2R`Mvorj-hnEzBSebwlOg9eXz5&z{N%YWz30F0{%_~)yl}IN zR_-^x>}PwP{*Y_Gw)lYa&sgy5?cRIZv5z_R*8MK{$egWjd(pFQd-$0*j}Je`b?-=|$VQ+nZ{?(_=9sE0Sn?EvhQt8ZzhpzqG zWN_XV6X%T0-gND;wR_@;MrO=kyVri5J2*VH8V6UIKQe#tbf8Ovh zmtMBd7yth2n||Q#zEvOH>CIbw=iL2wdi08UGvB>r;aP9I{UJAwuX&n@FK>SN^Y4A) zYwmT>vS;4oi`%^B!ae5gd)PO1!qCb0*m0*X@A}x+o_6%~FYY~iFTeiBHyytBrT=j7 zu7_`S)X$gQ<3US*`MF1Jw)fnPzIM@9&e=+rUhj*W@`ShkcBi9Kp_}({GZKV^2PG0xoSB-vX(T6|#H#dLb{3pHUpALQ5 zCm*ot`gvbHYV_ijM~$C9>y(`~y?v@w<6o)r>DKn#$eaQ|IyLu6Xf|Z+ha}-hcgdH@)IxzkT7dYj*g~!{&bP zA6H!R>F;bjcdx_8ho5S_Z_rffpEG0r@HK{cvrS+9%?r0Z{kY*{;#r@6!S+Y(f4q*} z?J+v{t{*Zwe?9k)t>^yn_1u5WUEe?acgS6zInBMJr#Guab)pXY%J^Us6oqoxU-`{BE_ZJ=UzJqSMe9lc@IqIa(EZF_Jn~olP``OPQ zeDrVnv@acakBuI3@zzVW-R?I#-+0i@kNnuBhur>_6W?&dSr1z9fp_gNKKyc~`f&4a zI8y~ae1)m{9=?B=*Aw^9r|00O`dP1d;ve%lHT=AP#yt2x4lf^FIQzrzS)Yrskum=< z=Z>uXfpFsRT-`hQv*N@#llRTN(|>pq51ag1ad6_qku?t)I=p84%iv;bIv%>{5c=Hp zt|JFOD~|3x_)Gs4e|PLX-~NVWFWhE_k8JV3w|(rA4ZnWhYwzyQir@I*l3fn{*3$cb zbJJhka`jQWUH^t%{^!W?;g|6G&uz5Tc7OYvOSd@Y)mOazmAjwvx*tCDojPIY&b=_4z*x=%aeMlz^pS+r%yw4^}Z=bzr)#$<# zc3Ls~R*n76nX7-#n&!6a)W@fNt5(11GmlvPpxJ}}px2#tKfM#H@1D8#cf8d&d7C(U zvXRNd!{4WBYU2L6gYS3s(2$a(v`f8+({Jz~#`_V~tyZ~4iW9=+SWHhJN}|9#k@;}g%b=G)l0 z#~pLud7Gbn@bTwedE1r?&KkeuiaT_|@X6a9dF8iHI%Z^K@PRbtbKlMX&oF%HN!D}! zVK44aBj(_XkDmFXYoGVScb~Gwr~m62TRvd?qi_AHUfkp>Uj6;At~u+>QwKku|9^@3 z!(QB0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5dC=+vv|>yS3LL1PtG~yvhDXh z@Rmo7E#G6__<}dB+;4o@&-Oh1A=iFw@d4+bvEbL+z4x?ZA9L!h`(5ynIa}ZMqG#Rq z@H1~7AO6FRoqOCd_no)-$p;^Q-j%m)x!|nvORl&>Cw%<5FaG#LPu}3ud;Q|_edm1r zf}IcD>9yPIgrTcl@x4c!y~$x?BO|LuHymB~snLzj86CR!FB6xpaq*=KADxSlwJt7s z=hJgB;bQr-4$Q@*iw_@?i%A!^9+ry<7uP=J$XraiIQCt+m~?T&2XZmt;<_D9$i<|K Rcb}Y#Nf*mb%f-mj{{y6tEmZ&j literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12625121.rlp b/statediff/indexer/database/file/mainnet_tests/receipts_12625121.rlp new file mode 100644 index 0000000000000000000000000000000000000000..4d3a8532c2bc91401df73ac14d9328a345b21684 GIT binary patch literal 706015 zcmeI433TNJVRw@Oae$V^PY@MhEZ~C3(nV2=3{+9X4a5~&5I3|gRTL1d1cKHLt1uLG zT%uA?1fi~gAPllu1Ql?BiULZl3y3Hp{q5Xo6~}Loijie1ZrdakQ!3UT9Up!Cf?Nz-yzfD| z7`XW9_PLmJG4Jrv@w~+e0ufKTlOU^s(fZsm+sFRL;$caCH{`nu? z_(8Wm=SjD2@#&wPzTZkW_Xke+V?OlT=WQ`MzShy#&(+}EkLT1KS+TXTnG;tyecs5> zgFmw7)IomRV3yOSkC?pN3fHw@g-b4*%iEmu6-DK(FMH8pLc+}m0x%Ub4?|=Hj;j5eN0~|ho zMV`9p$!GJi@%b|B+sc$vHKp8U?{#L@A!pL_I-i?KiSv8uY6&w=4j1ui>hxEvzV zHS3(g@DXEEC(nGA4d2$xC(htPE5D!p?k+oS=?5l1VXj_s*s(`#d&3qVS+M_F?|tlX zOLiLH?4s5EgxO{H6JCD5@z34-x<6Tb;v?_*y6+x#)fay;F??U1`ua28_sg|^y3MtR z9CNRCpY)a6p75d1{YnQ6A3VkD{^f}y(~s0T_c(vqBXx%#>55k|y%$dYcfmBR;(une zufna34PE2#;lroT95UO3;KTfAbImi4_4Ffk+k-E;>PZiO_m{UC-Rtb5e)GK@*W3D0 z@1K06-m%YhH~i$M|G3ZBpYp0*|8cV;e{szk_dNGkpL+0S`z%=J;)}j?##R$t+sMcU z|8MJSA9&(+3$J|tzSq5cw|Q5bzu%i4tOIU1de=kO`RR`y`u!Vk+I*91H+sX}-n`KZ zCVqbhZ_6I3kKTRx*Oy@*!#BThc-#>WcRu&A%a2-fC5MyR=XzqA`}#&krtbX0IkTDV zu^hU#=|}3=Q=WI)hoAbbqwoEl+n&7H%g)_=-4~pH+3J2yedKQsIR8s;T7S{VUvBf- z$87TG?>~9ZAKpGObVDQiF1Xt*$?Aoo5~L+Yem1;hDEA(gA~mN7sM(aa;Z3 zr=NM&9^c(z`jJ}qfQy$uQd0*lS2(!DA9_Ew;+d0|p5qGUl)us=HMq`|+Q^W8-pvFexzo5HeKCE>ZIo$_VH)G_OE~Op6xgKtJnP7-LAP|+vjXMF}Rhf z_rKiWU7tL9uN@a3y!A&vboQgipLgoFPJHad)ERTVz3|{Q9y0wC__}9)O^?)E7r1Mm ze(tZ=dEyD9<7XWE4P?yvn9pcf>8sDe+5OCPv~qOQS8({$h2~Fw>$^O+4F*k3pFcKv z_Dc6+K1WZTKl|JC#TTw{-m<-5^3;{@&+wtYUT2T@Ej@J6&OhGs>Q8L^>MM5K_oX*K zc)z|xue$Qk zgC~ZsYSU9Mz5LXJFWO}16PCRGSI7KdkK1p#|7Uf;@WBs1;_Ayj@UD@O$!{5Ez3tWf z9((wz2dvxv-wkt=i5*Cvrk^W$Cme)_~>C5>dp=B@znc2z4mjT zKJf+rb7KCuJ9p=Z`QsjxiW%;uVkWGE)z=tYi252M9gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1= zkq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm> zRg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>> zg{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC z3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6H zqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0 zIv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6z zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NY zqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0 zxDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#t ziou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZ zU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l` zNC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&` zDn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b` zLR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$6 z1{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mp zQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b0 z9gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg< z80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vH zQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0 zT!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR z#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwH zFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+| zq=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe z6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$e zA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3 zg9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Q zs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h z4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_ zjC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9 zsA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0 zE<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLE zVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p z7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX} z(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y# zijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK z5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2 z!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~D zR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L z2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=d zMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPl zR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg z7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)sn zF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__) zjEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~ z>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>= z#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHq zh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1 z;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82n zDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{Wer zgHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_! zBOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF& zsu<~DR17Xe6(b#tiou1bVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$ z3sJ>L2cu$eA*vYZU{nk)L=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P z7+i=dMmiW3g9}l`NC%^0a3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6Y zM#bPlR58-Qs2E&`Dn>dO6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0 zbTBFg7ov)h4o1b`LR2x*!KfHqh$==p7!`vHQN>6HqhfF&su<~DR17Xe6(b#tiou1b zVx)snF}M&_jC3$61{b1=kq$=1;6hX}(!r=0T!<=0Iv5p$3sJ>L2cu$eA*vYZU{nk) zL=__)jEcd9sA8mpQ8Bm>Rg82nDh3y#ijfXR#o$6zG19@P7+i=dMmiW3g9}l`NC%^0 za3QJ~>0ne0E<_b09gK>>g{WergHbWK5LJwHFe(NYqKc6YM#bPlR58-Qs2E&`Dn>dO z6@v>=#YhLEVsIg<80lbC3@$_!BOQ#2!G)+|q=Qj0xDZv0bTBFg7ov)h4o1b`LR2x* z!KfHqi2ipJvv|?YKi>1|Pi*|^D|X!Xr8hr#?7H2ynKmBJyze6p+xK&)EIQ`MU337~Hafa^=NDbE{lbxvrT;^B z?osEgv*V!$&fEHfXKnfJ?>yyW$G`C6lfE#=og10=zCYXL)cwyL8M%FQ?a}cwet69p zqullXqjc`Hi!&c_LoP<9T#SvMdF*<*n07Hbe&!E1&c%$2FFaziTnt>i{Q5&s(`o`f0T=2;Qk2~;wYwfhpIXk`Z zkiB<3@Na&3?)aALAA0gh4_$x%iQ%ig*Zh-i`mdw@Vw=6L`}vlaUNZ8Aw>|!>O?ANV z!2{L-vt7gK3kQeJ@&1lpX&AkI^u*EebH9Jn9FO%JS37w!*Sv6YP5qCzHCOc!8C+zO z0nGV+@YpRoU~G=#=1rfzV2*QFI@wpc^aGQx9$mfUuw#$f_J%D!vS9zU-uu|&mh3dX z*+r}S($^Qa@g@yh!xz3!FY+3u8E7woXz#PE$j<1_o7{Iy@+=b%kDebK-E zXyIl5{jcx-^lx;)@WIpk*1Z=_zhbiFuWnoIM{1=j`(J;;vR99m{L}ApG4sr|eBy^=QVV)9w;yk9L|#m^C*7kHMN<%{MD&vMJ2d{%fW9KIM=J9O&I zrIww-5o=DKFgA49FzO=1XTVbzpZ)DldzL%qlRr6S&wu&+>FZsz_Hp-r`tA4m@b(|t zX=3;`|Ne9P?r_u7U;LH#9evH^i(h{Cmwod!S6-+Ch7X?NdvAXI*z~j9mv;Z?Dn3#x zUd3H$FMF2z(&A6#V*0Du*!cNZJ|q`2U)e8sz+dEI;Np-+y1+?ng(gIm`DSd~c&ehlYi@mS9`MFJt|nec9HJIZ_7?|v>h;VZeA`D%8_(kpW@ z?wlo(rE-u}@-|P)u2fK7}zu8;v;>Hd79GU(c85_TB_w{oz z^Y!Ag#rMj^z{QQ5=VHpm}<7m^VznZvNGGe6MO}K7+sd-5=&+#>Ln6e_$>KE!2tV%o*n_%|Q%kGYuX;+t=OQ!WNBt~n|fQ!XZ7UHR5C@3D$5W?lxF{MGvKZ@5={ zF<_3DQ5LRr+~9=eudWPE7=Azfl8KKP8$N%CSK-x_mHsMV`E}iCzn9b8V8;8*H1g*v8O%e;EADoIkMV+mAJ;^9<}Uu(ck*`eOB?28os@|8c(?x8~^s2 z|B{QDXSr|hx2m7PceNIVKb~cuBj5g1pCg0cvt70B>i?^%VWW2qUe?7`f7`!XTHeK} z{kx@si>v4L@0OOi7#si2HM{kkI`b34HQPNo7XufEKO+|d7gzjUE@oVOcdHlWV&LN7 zSL9;g;>!o+V#dYy?)%1E3|zeY&AAx3IOm83JMLm^ zwx7(GpSWQ8xl<=C`&%`?f;uv?(hH%>FSFv2%gR6 z%{Ds!DZhKtZ4;Lq_^L}szA^8e&%EC~SNCt-j{40Pu9*L}m+tz+z1Khbq7xsp(e~Hw z_1THx+kWT+KXv6xPk8w07w`1>n+|%^$G1B4zu)y_9WZ?G8izdeT7Rr<&MZxt1*(!>Jm4|_{i{PSwMW;kK zixfc#DxmEMQZ!i0E@~0CJ)kI$T6vhR3)DxTs0~#CWt$+U?93#&e3wk3!{K<&u6NF1 z=KlWQ|NDR6-2a`K@c8B{SFRkMGko@8bG|q{^y~g{+&7Y(0&7%t{c z)G?w$W@-IWW6m#i)~b^xUvToj8GVdB?d*vQ%$_*JTYUzR53l(6ipv-O*@dSrxp?-i zpTGF;|KlSs{XcK~*jKJPjDP#l!~SBs`)|GcgfCya@QFPySaQj(S3h#h&7au% z#gDz|wU6z1-A~2`-_IqV{_BwwAKdMbPyP6g+xGjn%l~fa_W$kcI^o)lp4i|O|FZi5 zmmj`h>3#>Fzt8cz&pBBq3|?($?(Gje`j&@>rtZ|Nlm9(;N++#n47!-}l%iF})_12y zrn%^P?$qjsR()A>-LYw|F?onPwT{QvdwA+j-Fwq}FS+2Q5AJxy$XhqM^i?0ZX}?9! zyJe<#YTs|o|NgZbpZ~zcTkUY}%?G^VsJGnw;Y-H{UwZdXAO6ure>(5dgAe;p-}$dg z-ar38T(se#I$`kSN#6GPa~_|%Qyc8~#&z7O$xx^NSm&LZJbjJP>7TmxQSQ__pZJXL z)X3VewvHpex;wS-4U7Nk`Tz5dT@Sr_+xa(*9Q#}Ua(I*B zMH`%U^g8a;`kwy(+haGboW0NRq766r%}Ey{Pknb|gJa^MHTBLN8t0_RALgmbMme+w zCJ&FS8IxbfCO?f%ewz9ioA^93MXNqh?bICQk%?EHe2LM~HLuSj)4;?B}n)7+?0}6E}Zo zzuBKZ;%9eXcK#D5-8=iCH*c};K^xEi^5||y9(mi`kBzUn%H-EGe{+jJ*yD3|y#6D5 ze|NXHeCBgsJL1uUFVG2tC(pZU%Pro$_0Z77pP^}Adxn38e&K8HdGr}qzT_R(Eq>wN z2mj|!zHj?0w|)H5qkowHOn6Ol(>-TU>~xP09$Pu< z_l6g3bntf1_!sHi5iU3JqdvN7O#M#b_!Ri}1COmbyp9LgT#mCRVPxXJ$zLkdeaxEr z^)d0AX1d=Z6Hi$6Iuj30cY&3|drkZR4;_E$H(zt*@NaLpbm85ny=nIcFWdUR?>f_; zk#9cl{3FkL_;YXg^a-!{(wvvRZSNaCb;^SA!T0;!!=F2E!|Na0Z{^RwzWfj0zV&?% z?egG0I$`kSdB6RtovylQ{v#_cT|C`|myN&VoY~L$(xMH|IpMA+4>{@3-+BC@8*jV& z?qh~OyWoaxp8U?CKX~r-M-N_ZkheN}(;q&1<3+MH-Fq8+&LFROYb$3RIJ{`%2Sx@f zTGu@teE#}*&7Yu=se8Wa&aZmL#3QqO3&$oN_{Fcd?!Q5+Uu(+5#HY#2t@>E~GV6Zk z!26iI$jafpSA1;oPk(UwioGv>-L2c)@rl`+Uh|oMzG3)#uU_%Knf>3$iv7O+M+<+x z-{|a1U%AO~i(h>1?=Js`<>Ra0)zp_`UjOxd_xk7!Uzq)+D?fD0na4l$y?@&MDxENR z@+#hHtFeiUrn~a8@wr>hJMh?7KJ=xX&)fgy2S2dn!Q~rocFYE6?)TG!?)>=6KYZnD zetiAWn|<_C^B;csOZR=jYkqLlRrZ^NHa_%XTY7jW;4`@V)%-HyC)=nrms;bFHN^tWGq&yW6kmjgF@?h&UfJ@Z}Tt1me9 z<)Lq#{G(ImEnN1VExvlsQ3qbN@Tw19agt6LJbBNv@BPNdFPi$d%(SmP!+*;RUV07d zwV&CWdv?T}a{IZne|Z1vzkmLp@9^2X57}wU@hji^HNCkt?|Aa-7T^2Abypwzs}u9g z-rTbz=9%4;%crlkVtz$pD(0!5*YT%tXy}>c{OtTFJ+qrqF@yJSXsEwR_QC3}5-JQ8 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-ww zsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj z4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9 zjD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ} zphKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{ zVo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij- zSQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw z_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)F zL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZT zRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)% z2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#g zP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG9 z6^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!e zF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^ ztcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x z#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk z7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?o zpu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2 zV;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@L zs2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell z3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKd zR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HF zeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{; zV(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq% z3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo z6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x z!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>i zu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV z3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew! zt71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8 zK3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}p zhKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP? zs4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT) zia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4U zU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xl zD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX z!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w z1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI z82eyV3@Qv2V;`)FL4~1W?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc` zL&ew!t71@Ls2KZTRSYT&6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$c zR2VA8K3Ell3PZ)%2diRGVW=4UU{wq%3>9M^tcpQ}phKjKdR>h#gP%-wwsu)xlD#kuo6@v;x#n=a{Vo+hI82eyV3@Qv2V;`)FL4~1W z?1NP?s4!HFeXuG96^4qj4_3vX!cZ~x!KxTk7%Ij-SQUc`L&ew!t71@Ls2KZTRSYT& z6=NT)ia~{;V(f!eF{m(9jD4^w1{H>iu@6?opu$iw_Q9$cR2VA8K3Ell3PZ)%2diRG zVW=4UU{wq%3>9M^tcpQ}p+m9ah z7u(%`>*XhW`Pzj~?0LbGOLo2bkz;QD#MUo<>`kwIY{%<ij#E0hHo3);nX9xfAs#{M(#Z1+{e#2=>0GIfKGVoYQwKO;D#-) z-1Ekvp_Rj%j4fJr))QYGe(KR*Q@U~G@YTbMmMvZK?7En>Mx)4cM~oS{)3U-9u3moHxO*}pmYA3t#T&t7)zS1!8aFE<-LYm=of znQ682zVOzO56u4A3ENzI#mioO+?)40^ybt5b)1)(fY-h4fE(_c{gDs;-DP_&|KR?+ zerB&bw>(2944yp2b7!A6b*Jv%<*>Ex)Ox>ZU1^##r@2#lipTqg=B_y~&9&A%wE8Qj zKJEC_T<_DTyHi6$qZ3aYn|kupojPmV$KQR*h1Xwv-yT<9bK@Ofo3rGOhpwIJo!WKe zC)XYS&UZd};*rmP$K7|_cJ3LAzxW&DQ!lal(@gHvHb2;7t6Mg^`xO(vUnYL0?mzt< zYu%~!8X0`qwRm)BBGspVtkN|0^utqo?5R67?c=}v!NEJV){R-~z{GWDP2H(gFEDkd z=D&LGyS80<`*A0(82#>w-#Po(e_na+?K8boZ@=^R_d8*qk6wQ2S${L@hxa}Hn&;2E zY{~fGH!+htHGJL9hb?&H?=4+@r@pb>T6b#Jbl=$E%fW+sRv%pZPI2}$FmR{VzSfNX zLh0!m6E8dcH#Eu{de(lXE`9IT_g(v`3pYFN1sm`DwjJjF(JlY(n5E;q%mmEjPHnr- zyXL;@AOC!8)t!1^r`rbZ)X2a^M+bNYIL)0}bERo~P5hmj!L`?S>eFwI&gGSL)|&S< zH1w?fo%+ss#{UbEQ@;EF literal 0 HcmV?d00001 diff --git a/statediff/indexer/database/file/mainnet_tests/test_helpers.go b/statediff/indexer/database/file/mainnet_tests/test_helpers.go index ae2b81435..7203649d4 100644 --- a/statediff/indexer/database/file/mainnet_tests/test_helpers.go +++ b/statediff/indexer/database/file/mainnet_tests/test_helpers.go @@ -36,9 +36,17 @@ const ( const ( TEST_RAW_URL = "TEST_RAW_URL" TEST_BLOCK_NUMBER = "TEST_BLOCK_NUMBER" - TEST_LOCAL_CACHE = "TEST_LOCAL_CACHE" ) +var problemBlocks = []int64{ + 12600011, + 12619985, + 12625121, + 12655432, + 12579670, + 12914664, +} + // TestConfig holds configuration params for mainnet tests type TestConfig struct { RawURL string @@ -53,16 +61,8 @@ var DefaultTestConfig = TestConfig{ LocalCache: true, } -// TestBlocksAndReceiptsFromEnv retrieves the block and receipts using env variables to override default config -func TestBlocksAndReceiptsFromEnv() (*types.Block, types.Receipts, error) { - conf := DefaultTestConfig - rawURL := os.Getenv(TEST_RAW_URL) - if rawURL == "" { - fmt.Printf("Warning: no raw url configured for statediffing mainnet tests, will look for local file and"+ - "then try default endpoint (%s)\r\n", DefaultTestConfig.RawURL) - } else { - conf.RawURL = rawURL - } +// TestBlockAndReceiptsFromEnv retrieves the block and receipts using env variables to override default config block number +func TestBlockAndReceiptsFromEnv(conf TestConfig) (*types.Block, types.Receipts, error) { blockNumberStr := os.Getenv(TEST_BLOCK_NUMBER) blockNumber, ok := new(big.Int).SetString(blockNumberStr, 10) if !ok { @@ -71,12 +71,12 @@ func TestBlocksAndReceiptsFromEnv() (*types.Block, types.Receipts, error) { } else { conf.BlockNumber = blockNumber } - return TestBlocksAndReceipts(conf) + return TestBlockAndReceipts(conf) } -// TestBlocksAndReceipts retrieves the block and receipts for the provided test config +// TestBlockAndReceipts retrieves the block and receipts for the provided test config // It first tries to load files from the local system before setting up and using an ethclient.Client to pull the data -func TestBlocksAndReceipts(conf TestConfig) (*types.Block, types.Receipts, error) { +func TestBlockAndReceipts(conf TestConfig) (*types.Block, types.Receipts, error) { var cli *ethclient.Client var err error var block *types.Block -- 2.45.2 From 62d51fd9d2c64aff3d8a34a5c364c3aefc17b106 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 1 Dec 2021 22:27:51 -0600 Subject: [PATCH 24/33] increase file write buffer size --- statediff/indexer/database/file/writer.go | 10 +++++----- statediff/indexer/ipld/eth_header.go | 8 ++++---- statediff/indexer/ipld/eth_header_test.go | 3 ++- statediff/indexer/ipld/eth_log.go | 5 +++-- statediff/indexer/ipld/eth_log_trie.go | 5 +++-- statediff/indexer/ipld/eth_parser.go | 5 +++-- statediff/indexer/ipld/eth_parser_test.go | 3 ++- statediff/indexer/ipld/eth_receipt.go | 3 ++- statediff/indexer/ipld/eth_tx.go | 5 +++-- statediff/indexer/ipld/trie_node.go | 3 ++- 10 files changed, 29 insertions(+), 21 deletions(-) diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 62fd5cb8d..e66e71f94 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -31,9 +31,9 @@ import ( ) var ( - nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") - pipeSize = 65336 // min(linuxPipeSize, macOSPipeSize) - collatedStmtSize = pipeSize * 16 + nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") + pipeSize = 65336 // min(linuxPipeSize, macOSPipeSize) + writeBufferSize = pipeSize * 16 * 48 ) // SQLWriter writes sql statements to a file @@ -54,7 +54,7 @@ func NewSQLWriter(wc io.WriteCloser) *SQLWriter { return &SQLWriter{ wc: wc, stmts: make(chan []byte), - collatedStmt: make([]byte, collatedStmtSize), + collatedStmt: make([]byte, writeBufferSize), flushChan: make(chan struct{}), flushFinished: make(chan struct{}), quitChan: make(chan struct{}), @@ -74,7 +74,7 @@ func (sqw *SQLWriter) Loop() { select { case stmt := <-sqw.stmts: l = len(stmt) - if l+sqw.collationIndex+1 > collatedStmtSize { + if sqw.collationIndex+l > writeBufferSize { if err := sqw.flush(); err != nil { panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err)) } diff --git a/statediff/indexer/ipld/eth_header.go b/statediff/indexer/ipld/eth_header.go index 5905bdd7e..9bc307277 100644 --- a/statediff/indexer/ipld/eth_header.go +++ b/statediff/indexer/ipld/eth_header.go @@ -20,13 +20,13 @@ import ( "encoding/json" "fmt" - "github.com/ethereum/go-ethereum/common" - - "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" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rlp" ) // EthHeader (eth-block, codec 0x90), represents an ethereum block header diff --git a/statediff/indexer/ipld/eth_header_test.go b/statediff/indexer/ipld/eth_header_test.go index d1ed13d37..ebbab2129 100644 --- a/statediff/indexer/ipld/eth_header_test.go +++ b/statediff/indexer/ipld/eth_header_test.go @@ -9,10 +9,11 @@ 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) { diff --git a/statediff/indexer/ipld/eth_log.go b/statediff/indexer/ipld/eth_log.go index ebc762065..225c44117 100644 --- a/statediff/indexer/ipld/eth_log.go +++ b/statediff/indexer/ipld/eth_log.go @@ -3,11 +3,12 @@ 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" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rlp" ) // EthLog (eth-log, codec 0x9a), represents an ethereum block header diff --git a/statediff/indexer/ipld/eth_log_trie.go b/statediff/indexer/ipld/eth_log_trie.go index edfd35af9..1df21642d 100644 --- a/statediff/indexer/ipld/eth_log_trie.go +++ b/statediff/indexer/ipld/eth_log_trie.go @@ -3,10 +3,11 @@ 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" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rlp" ) // EthLogTrie (eth-tx-trie codec 0x9p) represents diff --git a/statediff/indexer/ipld/eth_parser.go b/statediff/indexer/ipld/eth_parser.go index 0f294ad30..497facba4 100644 --- a/statediff/indexer/ipld/eth_parser.go +++ b/statediff/indexer/ipld/eth_parser.go @@ -23,11 +23,12 @@ import ( "io" "io/ioutil" + "github.com/ipfs/go-cid" + "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 diff --git a/statediff/indexer/ipld/eth_parser_test.go b/statediff/indexer/ipld/eth_parser_test.go index 9cb8d4e46..bcf28efde 100644 --- a/statediff/indexer/ipld/eth_parser_test.go +++ b/statediff/indexer/ipld/eth_parser_test.go @@ -21,10 +21,11 @@ import ( "path/filepath" "testing" + "github.com/stretchr/testify/require" + "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 diff --git a/statediff/indexer/ipld/eth_receipt.go b/statediff/indexer/ipld/eth_receipt.go index 0eb6a2f81..ccd785515 100644 --- a/statediff/indexer/ipld/eth_receipt.go +++ b/statediff/indexer/ipld/eth_receipt.go @@ -21,10 +21,11 @@ import ( "fmt" "strconv" - "github.com/ethereum/go-ethereum/core/types" "github.com/ipfs/go-cid" node "github.com/ipfs/go-ipld-format" mh "github.com/multiformats/go-multihash" + + "github.com/ethereum/go-ethereum/core/types" ) type EthReceipt struct { diff --git a/statediff/indexer/ipld/eth_tx.go b/statediff/indexer/ipld/eth_tx.go index 394e235a8..99b1f9dbe 100644 --- a/statediff/indexer/ipld/eth_tx.go +++ b/statediff/indexer/ipld/eth_tx.go @@ -22,11 +22,12 @@ import ( "strconv" "strings" - "github.com/ethereum/go-ethereum/common/hexutil" - "github.com/ethereum/go-ethereum/core/types" "github.com/ipfs/go-cid" node "github.com/ipfs/go-ipld-format" mh "github.com/multiformats/go-multihash" + + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/core/types" ) // EthTx (eth-tx codec 0x93) represents an ethereum transaction diff --git a/statediff/indexer/ipld/trie_node.go b/statediff/indexer/ipld/trie_node.go index a344bab4f..816217064 100644 --- a/statediff/indexer/ipld/trie_node.go +++ b/statediff/indexer/ipld/trie_node.go @@ -20,9 +20,10 @@ import ( "encoding/json" "fmt" - "github.com/ethereum/go-ethereum/rlp" "github.com/ipfs/go-cid" node "github.com/ipfs/go-ipld-format" + + "github.com/ethereum/go-ethereum/rlp" ) const ( -- 2.45.2 From 8c999d2468ec2b8e511d9c41538ee17f48196616 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 1 Dec 2021 23:19:28 -0600 Subject: [PATCH 25/33] increase buffer further --- statediff/indexer/database/file/writer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index e66e71f94..573bedad9 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -33,7 +33,7 @@ import ( var ( nullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") pipeSize = 65336 // min(linuxPipeSize, macOSPipeSize) - writeBufferSize = pipeSize * 16 * 48 + writeBufferSize = pipeSize * 16 * 96 ) // SQLWriter writes sql statements to a file -- 2.45.2 From d2432916f81be4c19cba0db1a62920fa442294a7 Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 17 Dec 2021 11:17:00 -0600 Subject: [PATCH 26/33] fix rct trie multicodec type --- statediff/indexer/ipld/eth_receipt_trie.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statediff/indexer/ipld/eth_receipt_trie.go b/statediff/indexer/ipld/eth_receipt_trie.go index 4ceef6f1f..75d40eedb 100644 --- a/statediff/indexer/ipld/eth_receipt_trie.go +++ b/statediff/indexer/ipld/eth_receipt_trie.go @@ -167,7 +167,7 @@ func (rt *rctTrie) getNodeFromDB(key []byte) (*EthRctTrie, error) { return nil, err } tn := &TrieNode{ - cid: keccak256ToCid(MEthStateTrie, key), + cid: keccak256ToCid(MEthTxReceiptTrie, key), rawdata: rawdata, } -- 2.45.2 From 2b69e856c64979c5d4114aaecf8f77525f1dad0a Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 17 Dec 2021 13:34:22 -0600 Subject: [PATCH 27/33] extend testing --- .../file/mainnet_tests/indexer_test.go | 12 +- .../sql/mainnet_tests/indexer_test.go | 109 ++++++++++++++++++ .../indexer/database/sql/postgres/config.go | 2 +- .../block_12579670.rlp | Bin .../block_12600011.rlp | Bin .../block_12619985.rlp | Bin .../block_12625121.rlp | Bin .../block_12655432.rlp | Bin .../block_12914664.rlp | Bin .../receipts_12579670.rlp | Bin .../receipts_12600011.rlp | Bin .../receipts_12619985.rlp | Bin .../receipts_12625121.rlp | Bin .../receipts_12655432.rlp | Bin .../receipts_12914664.rlp | Bin .../mainnet_test_helpers.go} | 9 +- 16 files changed, 121 insertions(+), 11 deletions(-) create mode 100644 statediff/indexer/database/sql/mainnet_tests/indexer_test.go rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12579670.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12600011.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12619985.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12625121.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12655432.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/block_12914664.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12579670.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12600011.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12619985.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12625121.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12655432.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests => mainnet_data}/receipts_12914664.rlp (100%) rename statediff/indexer/{database/file/mainnet_tests/test_helpers.go => test_helpers/mainnet_test_helpers.go} (96%) diff --git a/statediff/indexer/database/file/mainnet_tests/indexer_test.go b/statediff/indexer/database/file/mainnet_tests/indexer_test.go index 32850ca29..b721ff05e 100644 --- a/statediff/indexer/database/file/mainnet_tests/indexer_test.go +++ b/statediff/indexer/database/file/mainnet_tests/indexer_test.go @@ -49,21 +49,21 @@ func init() { } func TestPushBlockAndState(t *testing.T) { - conf := DefaultTestConfig - rawURL := os.Getenv(TEST_RAW_URL) + conf := test_helpers.DefaultTestConfig + rawURL := os.Getenv(test_helpers.TEST_RAW_URL) if rawURL == "" { fmt.Printf("Warning: no raw url configured for statediffing mainnet tests, will look for local file and"+ - "then try default endpoint (%s)\r\n", DefaultTestConfig.RawURL) + "then try default endpoint (%s)\r\n", test_helpers.DefaultTestConfig.RawURL) } else { conf.RawURL = rawURL } - for _, blockNumber := range problemBlocks { + for _, blockNumber := range test_helpers.ProblemBlocks { conf.BlockNumber = big.NewInt(blockNumber) - tb, trs, err := TestBlockAndReceipts(conf) + tb, trs, err := test_helpers.TestBlockAndReceipts(conf) require.NoError(t, err) testPushBlockAndState(t, tb, trs) } - testBlock, testReceipts, err := TestBlockAndReceiptsFromEnv(conf) + testBlock, testReceipts, err := test_helpers.TestBlockAndReceiptsFromEnv(conf) require.NoError(t, err) testPushBlockAndState(t, testBlock, testReceipts) } diff --git a/statediff/indexer/database/sql/mainnet_tests/indexer_test.go b/statediff/indexer/database/sql/mainnet_tests/indexer_test.go new file mode 100644 index 000000000..7a4165d41 --- /dev/null +++ b/statediff/indexer/database/sql/mainnet_tests/indexer_test.go @@ -0,0 +1,109 @@ +// VulcanizeDB +// Copyright © 2021 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 . + +package mainnet_tests + +import ( + "context" + "fmt" + "math/big" + "os" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql" + "github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" + "github.com/ethereum/go-ethereum/statediff/indexer/interfaces" + "github.com/ethereum/go-ethereum/statediff/indexer/mocks" + "github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" +) + +var ( + err error + db sql.Database + chainConf = params.MainnetChainConfig +) + +func init() { + if os.Getenv("MODE") != "statediff" { + fmt.Println("Skipping statediff test") + os.Exit(0) + } +} + +func TestPushBlockAndState(t *testing.T) { + conf := test_helpers.DefaultTestConfig + rawURL := os.Getenv(test_helpers.TEST_RAW_URL) + if rawURL == "" { + fmt.Printf("Warning: no raw url configured for statediffing mainnet tests, will look for local file and"+ + "then try default endpoint (%s)\r\n", test_helpers.DefaultTestConfig.RawURL) + } else { + conf.RawURL = rawURL + } + for _, blockNumber := range test_helpers.ProblemBlocks { + conf.BlockNumber = big.NewInt(blockNumber) + tb, trs, err := test_helpers.TestBlockAndReceipts(conf) + require.NoError(t, err) + testPushBlockAndState(t, tb, trs) + } + testBlock, testReceipts, err := test_helpers.TestBlockAndReceiptsFromEnv(conf) + require.NoError(t, err) + testPushBlockAndState(t, testBlock, testReceipts) +} + +func testPushBlockAndState(t *testing.T, block *types.Block, receipts types.Receipts) { + t.Run("Test PushBlock and PushStateNode", func(t *testing.T) { + setup(t, block, receipts) + tearDown(t) + }) +} + +func setup(t *testing.T, testBlock *types.Block, testReceipts types.Receipts) { + db, err = postgres.SetupSQLXDB() + if err != nil { + t.Fatal(err) + } + ind, err := sql.NewStateDiffIndexer(context.Background(), chainConf, db) + require.NoError(t, err) + var tx interfaces.Batch + tx, err = ind.PushBlock( + testBlock, + testReceipts, + testBlock.Difficulty()) + require.NoError(t, err) + + defer func() { + if err := tx.Submit(err); err != nil { + t.Fatal(err) + } + if err := ind.Close(); err != nil { + t.Fatal(err) + } + }() + for _, node := range mocks.StateDiffs { + err = ind.PushStateNode(tx, node, testBlock.Hash().String()) + require.NoError(t, err) + } + + test_helpers.ExpectEqual(t, tx.(*sql.BatchTx).BlockNumber, testBlock.Number().Uint64()) +} + +func tearDown(t *testing.T) { + sql.TearDownDB(t, db) +} diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index a7c7cc9b4..842c80b32 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -49,7 +49,7 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_test", + DatabaseName: "vulcanize_testing", Username: "postgres", Password: "", } diff --git a/statediff/indexer/database/file/mainnet_tests/block_12579670.rlp b/statediff/indexer/mainnet_data/block_12579670.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12579670.rlp rename to statediff/indexer/mainnet_data/block_12579670.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12600011.rlp b/statediff/indexer/mainnet_data/block_12600011.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12600011.rlp rename to statediff/indexer/mainnet_data/block_12600011.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12619985.rlp b/statediff/indexer/mainnet_data/block_12619985.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12619985.rlp rename to statediff/indexer/mainnet_data/block_12619985.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12625121.rlp b/statediff/indexer/mainnet_data/block_12625121.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12625121.rlp rename to statediff/indexer/mainnet_data/block_12625121.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12655432.rlp b/statediff/indexer/mainnet_data/block_12655432.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12655432.rlp rename to statediff/indexer/mainnet_data/block_12655432.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/block_12914664.rlp b/statediff/indexer/mainnet_data/block_12914664.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/block_12914664.rlp rename to statediff/indexer/mainnet_data/block_12914664.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12579670.rlp b/statediff/indexer/mainnet_data/receipts_12579670.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12579670.rlp rename to statediff/indexer/mainnet_data/receipts_12579670.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12600011.rlp b/statediff/indexer/mainnet_data/receipts_12600011.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12600011.rlp rename to statediff/indexer/mainnet_data/receipts_12600011.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12619985.rlp b/statediff/indexer/mainnet_data/receipts_12619985.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12619985.rlp rename to statediff/indexer/mainnet_data/receipts_12619985.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12625121.rlp b/statediff/indexer/mainnet_data/receipts_12625121.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12625121.rlp rename to statediff/indexer/mainnet_data/receipts_12625121.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12655432.rlp b/statediff/indexer/mainnet_data/receipts_12655432.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12655432.rlp rename to statediff/indexer/mainnet_data/receipts_12655432.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/receipts_12914664.rlp b/statediff/indexer/mainnet_data/receipts_12914664.rlp similarity index 100% rename from statediff/indexer/database/file/mainnet_tests/receipts_12914664.rlp rename to statediff/indexer/mainnet_data/receipts_12914664.rlp diff --git a/statediff/indexer/database/file/mainnet_tests/test_helpers.go b/statediff/indexer/test_helpers/mainnet_test_helpers.go similarity index 96% rename from statediff/indexer/database/file/mainnet_tests/test_helpers.go rename to statediff/indexer/test_helpers/mainnet_test_helpers.go index 7203649d4..141bb10fd 100644 --- a/statediff/indexer/database/file/mainnet_tests/test_helpers.go +++ b/statediff/indexer/test_helpers/mainnet_test_helpers.go @@ -14,7 +14,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -package mainnet_tests +package test_helpers import ( "context" @@ -29,8 +29,8 @@ import ( ) const ( - defaultBlockFilePath = "./block" - defaultReceiptsFilePath = "./receipts" + defaultBlockFilePath = "../../../mainnet_data/block" + defaultReceiptsFilePath = "../../../mainnet_data/receipts" ) const ( @@ -38,7 +38,8 @@ const ( TEST_BLOCK_NUMBER = "TEST_BLOCK_NUMBER" ) -var problemBlocks = []int64{ +// ProblemBlocks list of known problem blocks, with funky edge cases +var ProblemBlocks = []int64{ 12600011, 12619985, 12625121, -- 2.45.2 From 18df9abbda84e15e8ff384aea01c268e07d5b8a1 Mon Sep 17 00:00:00 2001 From: i-norden Date: Fri, 17 Dec 2021 16:38:01 -0600 Subject: [PATCH 28/33] log trie fk fix --- statediff/indexer/database/dump/indexer.go | 2 +- statediff/indexer/database/file/indexer.go | 2 +- statediff/indexer/database/sql/indexer.go | 2 +- .../sql/mainnet_tests/indexer_test.go | 8 ++--- .../indexer/database/sql/postgres/config.go | 6 ++-- statediff/indexer/ipld/eth_log_trie.go | 6 ++-- statediff/indexer/ipld/eth_parser.go | 29 ++++++++++--------- 7 files changed, 30 insertions(+), 25 deletions(-) diff --git a/statediff/indexer/database/dump/indexer.go b/statediff/indexer/database/dump/indexer.go index b2fc70d27..e450f941a 100644 --- a/statediff/indexer/database/dump/indexer.go +++ b/statediff/indexer/database/dump/indexer.go @@ -242,7 +242,7 @@ type processArgs struct { rctTrieNodes []*ipld2.EthRctTrie txNodes []*ipld2.EthTx txTrieNodes []*ipld2.EthTxTrie - logTrieNodes [][]*ipld2.EthLogTrie + logTrieNodes [][]node.Node logLeafNodeCIDs [][]cid.Cid rctLeafNodeCIDs []cid.Cid } diff --git a/statediff/indexer/database/file/indexer.go b/statediff/indexer/database/file/indexer.go index a9f5dd0fb..870c1f259 100644 --- a/statediff/indexer/database/file/indexer.go +++ b/statediff/indexer/database/file/indexer.go @@ -250,7 +250,7 @@ type processArgs struct { rctTrieNodes []*ipld2.EthRctTrie txNodes []*ipld2.EthTx txTrieNodes []*ipld2.EthTxTrie - logTrieNodes [][]*ipld2.EthLogTrie + logTrieNodes [][]node.Node logLeafNodeCIDs [][]cid.Cid rctLeafNodeCIDs []cid.Cid } diff --git a/statediff/indexer/database/sql/indexer.go b/statediff/indexer/database/sql/indexer.go index 72e978aad..c8d526d6f 100644 --- a/statediff/indexer/database/sql/indexer.go +++ b/statediff/indexer/database/sql/indexer.go @@ -297,7 +297,7 @@ type processArgs struct { rctTrieNodes []*ipld2.EthRctTrie txNodes []*ipld2.EthTx txTrieNodes []*ipld2.EthTxTrie - logTrieNodes [][]*ipld2.EthLogTrie + logTrieNodes [][]node.Node logLeafNodeCIDs [][]cid.Cid rctLeafNodeCIDs []cid.Cid } diff --git a/statediff/indexer/database/sql/mainnet_tests/indexer_test.go b/statediff/indexer/database/sql/mainnet_tests/indexer_test.go index 7a4165d41..68c9bc464 100644 --- a/statediff/indexer/database/sql/mainnet_tests/indexer_test.go +++ b/statediff/indexer/database/sql/mainnet_tests/indexer_test.go @@ -37,6 +37,7 @@ import ( var ( err error db sql.Database + ind interfaces.StateDiffIndexer chainConf = params.MainnetChainConfig ) @@ -79,7 +80,7 @@ func setup(t *testing.T, testBlock *types.Block, testReceipts types.Receipts) { if err != nil { t.Fatal(err) } - ind, err := sql.NewStateDiffIndexer(context.Background(), chainConf, db) + ind, err = sql.NewStateDiffIndexer(context.Background(), chainConf, db) require.NoError(t, err) var tx interfaces.Batch tx, err = ind.PushBlock( @@ -92,9 +93,6 @@ func setup(t *testing.T, testBlock *types.Block, testReceipts types.Receipts) { if err := tx.Submit(err); err != nil { t.Fatal(err) } - if err := ind.Close(); err != nil { - t.Fatal(err) - } }() for _, node := range mocks.StateDiffs { err = ind.PushStateNode(tx, node, testBlock.Hash().String()) @@ -106,4 +104,6 @@ func setup(t *testing.T, testBlock *types.Block, testReceipts types.Receipts) { func tearDown(t *testing.T) { sql.TearDownDB(t, db) + err = ind.Close() + require.NoError(t, err) } diff --git a/statediff/indexer/database/sql/postgres/config.go b/statediff/indexer/database/sql/postgres/config.go index 842c80b32..4fe2972ed 100644 --- a/statediff/indexer/database/sql/postgres/config.go +++ b/statediff/indexer/database/sql/postgres/config.go @@ -49,9 +49,9 @@ func ResolveDriverType(str string) (DriverType, error) { var DefaultConfig = Config{ Hostname: "localhost", Port: 5432, - DatabaseName: "vulcanize_testing", - Username: "postgres", - Password: "", + DatabaseName: "vulcanize_public", + Username: "vdbm", + Password: "password", } // Config holds params for a Postgres db diff --git a/statediff/indexer/ipld/eth_log_trie.go b/statediff/indexer/ipld/eth_log_trie.go index 1df21642d..8e8af9c79 100644 --- a/statediff/indexer/ipld/eth_log_trie.go +++ b/statediff/indexer/ipld/eth_log_trie.go @@ -3,6 +3,8 @@ package ipld import ( "fmt" + node "github.com/ipfs/go-ipld-format" + "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" @@ -92,13 +94,13 @@ func newLogTrie() *logTrie { // getNodes invokes the localTrie, which computes the root hash of the // log trie and returns its sql keys, to return a slice // of EthLogTrie nodes. -func (rt *logTrie) getNodes() ([]*EthLogTrie, error) { +func (rt *logTrie) getNodes() ([]node.Node, error) { keys, err := rt.getKeys() if err != nil { return nil, err } - out := make([]*EthLogTrie, 0, len(keys)) + out := make([]node.Node, 0, len(keys)) for _, k := range keys { n, err := rt.getNodeFromDB(k) if err != nil { diff --git a/statediff/indexer/ipld/eth_parser.go b/statediff/indexer/ipld/eth_parser.go index 497facba4..4e08f2d24 100644 --- a/statediff/indexer/ipld/eth_parser.go +++ b/statediff/indexer/ipld/eth_parser.go @@ -24,6 +24,7 @@ import ( "io/ioutil" "github.com/ipfs/go-cid" + node "github.com/ipfs/go-ipld-format" "github.com/multiformats/go-multihash" "github.com/ethereum/go-ethereum/common" @@ -124,7 +125,7 @@ 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, [][]*EthLogTrie, [][]cid.Cid, []cid.Cid, error) { +func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) (*EthHeader, []*EthHeader, []*EthTx, []*EthTxTrie, []*EthReceipt, []*EthRctTrie, [][]node.Node, [][]cid.Cid, []cid.Cid, error) { // Process the header headerNode, err := NewEthHeader(block.Header()) if err != nil { @@ -149,10 +150,10 @@ func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) (*EthHe } // Process the receipts and logs - rctNodes, tctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := processReceiptsAndLogs(receipts, + rctNodes, tctTrieNodes, logTrieAndLogNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err := processReceiptsAndLogs(receipts, block.Header().ReceiptHash[:]) - return headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, tctTrieNodes, logTrieNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err + return headerNode, uncleNodes, txNodes, txTrieNodes, rctNodes, tctTrieNodes, logTrieAndLogNodes, logLeafNodeCIDs, rctLeafNodeCIDs, err } // processTransactions will take the found transactions in a parsed block body @@ -181,11 +182,11 @@ func processTransactions(txs []*types.Transaction, expectedTxRoot []byte) ([]*Et // processReceiptsAndLogs will take in receipts // to return IPLD node slices for eth-rct, eth-rct-trie, eth-log, eth-log-trie, eth-log-trie-CID, eth-rct-trie-CID -func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*EthReceipt, []*EthRctTrie, [][]*EthLogTrie, [][]cid.Cid, []cid.Cid, error) { +func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*EthReceipt, []*EthRctTrie, [][]node.Node, [][]cid.Cid, []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)) + ethLogTrieAndLogNodes := make([][]node.Node, 0, len(rcts)) receiptTrie := NewRctTrie() @@ -196,7 +197,7 @@ func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*E return nil, nil, nil, nil, nil, err } rct.LogRoot = logTrieHash - ethLogTrieNodes = append(ethLogTrieNodes, logTrieNodes) + ethLogTrieAndLogNodes = append(ethLogTrieAndLogNodes, logTrieNodes) ethLogleafNodeCids = append(ethLogleafNodeCids, leafNodeCids) ethRct, err := NewReceipt(rct) @@ -236,14 +237,14 @@ func processReceiptsAndLogs(rcts []*types.Receipt, expectedRctRoot []byte) ([]*E ethRctleafNodeCids[idx] = rln.Cid() } - return ethRctNodes, rctTrieNodes, ethLogTrieNodes, ethLogleafNodeCids, ethRctleafNodeCids, err + return ethRctNodes, rctTrieNodes, ethLogTrieAndLogNodes, ethLogleafNodeCids, ethRctleafNodeCids, err } const keccak256Length = 32 -func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, error) { +func processLogs(logs []*types.Log) ([]node.Node, []cid.Cid, common.Hash, error) { logTr := newLogTrie() - shortLogCIDs := make(map[uint64]cid.Cid, len(logs)) + shortLog := make(map[uint64]*EthLog, len(logs)) for idx, log := range logs { logRaw, err := rlp.EncodeToBytes(log) if err != nil { @@ -260,7 +261,7 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro if err != nil { return nil, nil, common.Hash{}, err } - shortLogCIDs[uint64(idx)] = logNode.Cid() + shortLog[uint64(idx)] = logNode } if err = logTr.Add(idx, logRaw); err != nil { return nil, nil, common.Hash{}, err @@ -289,9 +290,11 @@ func processLogs(logs []*types.Log) ([]*EthLogTrie, []cid.Cid, common.Hash, erro } // this is where we check which logs <= keccak256Length were actually internalized into parent branch node // and replace those that were with the cid.Cid for the raw log IPLD - for idx, lCID := range shortLogCIDs { - if !leafNodeCids[idx].Defined() { - leafNodeCids[idx] = lCID + for i, l := range shortLog { + if !leafNodeCids[i].Defined() { + leafNodeCids[i] = l.Cid() + // if the leaf node was internalized, we append an IPLD for log itself to the list of IPLDs we need to publish + logTrieNodes = append(logTrieNodes, l) } } -- 2.45.2 From f01dd5d1b72f9370d4f4102e2b0edccdae31d7d7 Mon Sep 17 00:00:00 2001 From: i-norden Date: Mon, 20 Dec 2021 12:30:33 -0600 Subject: [PATCH 29/33] bump statediff meta version; use db v0.3.0 in compose --- docker-compose.yml | 4 ++-- params/version.go | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index f1a37ddcb..d305d1dc1 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -3,7 +3,7 @@ version: '3.2' services: ipld-eth-db: restart: always - image: vulcanize/ipld-eth-db:v0.2.0 + image: vulcanize/ipld-eth-db:v0.3.1 environment: POSTGRES_USER: "vdbm" POSTGRES_DB: "vulcanize_public" @@ -14,4 +14,4 @@ services: - "127.0.0.1:5432:5432" volumes: - geth_node: \ No newline at end of file + geth_node: diff --git a/params/version.go b/params/version.go index 367fbf8a3..77dfbf312 100644 --- a/params/version.go +++ b/params/version.go @@ -21,10 +21,10 @@ import ( ) const ( - VersionMajor = 1 // Major version component of the current release - VersionMinor = 10 // Minor version component of the current release - VersionPatch = 11 // Patch version component of the current release - VersionMeta = "statediff-0.0.27" // Version metadata to append to the version string + VersionMajor = 1 // Major version component of the current release + VersionMinor = 10 // Minor version component of the current release + VersionPatch = 11 // Patch version component of the current release + VersionMeta = "statediff-0.1.0" // Version metadata to append to the version string ) // Version holds the textual version string. -- 2.45.2 From ec421c810b57d739c1a9ac6f3f8271550b2dd663 Mon Sep 17 00:00:00 2001 From: i-norden Date: Mon, 20 Dec 2021 13:19:57 -0600 Subject: [PATCH 30/33] skip file writing tests in CI, for now --- statediff/indexer/database/file/mainnet_tests/indexer_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/statediff/indexer/database/file/mainnet_tests/indexer_test.go b/statediff/indexer/database/file/mainnet_tests/indexer_test.go index b721ff05e..b297b82d1 100644 --- a/statediff/indexer/database/file/mainnet_tests/indexer_test.go +++ b/statediff/indexer/database/file/mainnet_tests/indexer_test.go @@ -46,6 +46,10 @@ func init() { fmt.Println("Skipping statediff test") os.Exit(0) } + if os.Getenv("STATEDIFF_DB") != "file" { + fmt.Println("Skipping statediff .sql file writing mode test") + os.Exit(0) + } } func TestPushBlockAndState(t *testing.T) { -- 2.45.2 From de6fb0e15538a8bf435f9e01801270afecab59f1 Mon Sep 17 00:00:00 2001 From: i-norden Date: Mon, 20 Dec 2021 13:43:30 -0600 Subject: [PATCH 31/33] prevent parallel execution of tests in different pkgs (suspect this is what causes our deadlock to show up only in CI test env) --- Makefile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 92413dda7..6a366b86c 100644 --- a/Makefile +++ b/Makefile @@ -56,7 +56,11 @@ ios: .PHONY: statedifftest statedifftest: | $(GOOSE) - MODE=statediff go test ./statediff/... -v + MODE=statediff go test -p 1 ./statediff/... -v + +.PHONY: statediff_filewriting_test +statediff_filetest: | $(GOOSE) + MODE=statediff STATEDIFF_DB=file go test -p 1 ./statediff/... -v test: all $(GORUN) build/ci.go test -- 2.45.2 From 4d6c91b6d4dc8c1125a6477e32e7175fb8344f19 Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 22 Dec 2021 13:34:39 -0600 Subject: [PATCH 32/33] adjust write buffering --- statediff/indexer/database/file/writer.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/statediff/indexer/database/file/writer.go b/statediff/indexer/database/file/writer.go index 573bedad9..48de0853d 100644 --- a/statediff/indexer/database/file/writer.go +++ b/statediff/indexer/database/file/writer.go @@ -78,6 +78,12 @@ func (sqw *SQLWriter) Loop() { if err := sqw.flush(); err != nil { panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err)) } + if l > writeBufferSize { + if _, err := sqw.wc.Write(stmt); err != nil { + panic(fmt.Sprintf("error writing large sql stmt to file: %v", err)) + } + continue + } } copy(sqw.collatedStmt[sqw.collationIndex:sqw.collationIndex+l], stmt) sqw.collationIndex += l -- 2.45.2 From 08e82e20013839e7054e8fc4073e584c66e45bfc Mon Sep 17 00:00:00 2001 From: i-norden Date: Wed, 22 Dec 2021 13:36:28 -0600 Subject: [PATCH 33/33] fix rct unit tests --- .../indexer/database/file/indexer_test.go | 92 ++++++++++++------- .../database/sql/indexer_shared_test.go | 41 ++++++++- .../indexer/database/sql/pgx_indexer_test.go | 28 +++--- .../indexer/database/sql/sqlx_indexer_test.go | 28 +++--- 4 files changed, 127 insertions(+), 62 deletions(-) diff --git a/statediff/indexer/database/file/indexer_test.go b/statediff/indexer/database/file/indexer_test.go index c2be9f993..cd6e89b20 100644 --- a/statediff/indexer/database/file/indexer_test.go +++ b/statediff/indexer/database/file/indexer_test.go @@ -52,11 +52,10 @@ var ( ipfsPgGet = `SELECT data FROM public.blocks WHERE key = $1` tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte - txs types.Transactions - rcts types.Receipts mockBlock *types.Block headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid + rctLeaf1, rctLeaf2, rctLeaf3, rctLeaf4, rctLeaf5 []byte state1CID, state2CID, storageCID cid.Cid ) @@ -67,7 +66,7 @@ func init() { } mockBlock = mocks.MockBlock - txs, rcts = mocks.MockBlock.Transactions(), mocks.MockReceipts + txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts buf := new(bytes.Buffer) txs.EncodeIndex(0, buf) @@ -126,14 +125,42 @@ func init() { trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) - rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) - rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) - rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) - rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) - rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) + + receiptTrie := ipld.NewRctTrie() + + receiptTrie.Add(0, rct1) + receiptTrie.Add(1, rct2) + receiptTrie.Add(2, rct3) + receiptTrie.Add(3, rct4) + receiptTrie.Add(4, rct5) + + rctLeafNodes, keys, _ := receiptTrie.GetLeafNodes() + + rctleafNodeCids := make([]cid.Cid, len(rctLeafNodes)) + orderedRctLeafNodes := make([][]byte, len(rctLeafNodes)) + for i, rln := range rctLeafNodes { + var idx uint + + r := bytes.NewReader(keys[i].TrieKey) + rlp.Decode(r, &idx) + rctleafNodeCids[idx] = rln.Cid() + orderedRctLeafNodes[idx] = rln.RawData() + } + + rct1CID = rctleafNodeCids[0] + rct2CID = rctleafNodeCids[1] + rct3CID = rctleafNodeCids[2] + rct4CID = rctleafNodeCids[3] + rct5CID = rctleafNodeCids[4] + + rctLeaf1 = orderedRctLeafNodes[0] + rctLeaf2 = orderedRctLeafNodes[1] + rctLeaf3 = orderedRctLeafNodes[2] + rctLeaf4 = orderedRctLeafNodes[3] + rctLeaf5 = orderedRctLeafNodes[4] } func setup(t *testing.T) { @@ -218,7 +245,7 @@ func TestFileIndexer(t *testing.T) { dumpData(t) defer tearDown(t) - // check that txs were properly indexed + // check that txs were properly indexed and published trxs := make([]string, 0) pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1` @@ -232,7 +259,8 @@ func TestFileIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) - // and published + + transactions := mocks.MockBlock.Transactions() type txResult struct { TxType uint8 `db:"tx_type"` Value string @@ -261,8 +289,8 @@ func TestFileIndexer(t *testing.T) { if txRes.TxType != 0 { t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) } - if txRes.Value != txs[0].Value().String() { - t.Fatalf("expected tx value %s got %s", txs[0].Value().String(), txRes.Value) + if txRes.Value != transactions[0].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[0].Value().String(), txRes.Value) } case trx2CID.String(): test_helpers.ExpectEqual(t, data, tx2) @@ -274,8 +302,8 @@ func TestFileIndexer(t *testing.T) { if txRes.TxType != 0 { t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) } - if txRes.Value != txs[1].Value().String() { - t.Fatalf("expected tx value %s got %s", txs[1].Value().String(), txRes.Value) + if txRes.Value != transactions[1].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[1].Value().String(), txRes.Value) } case trx3CID.String(): test_helpers.ExpectEqual(t, data, tx3) @@ -287,8 +315,8 @@ func TestFileIndexer(t *testing.T) { if txRes.TxType != 0 { t.Fatalf("expected LegacyTxType (0), got %d", txRes.TxType) } - if txRes.Value != txs[2].Value().String() { - t.Fatalf("expected tx value %s got %s", txs[2].Value().String(), txRes.Value) + if txRes.Value != transactions[2].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[2].Value().String(), txRes.Value) } case trx4CID.String(): test_helpers.ExpectEqual(t, data, tx4) @@ -300,8 +328,8 @@ func TestFileIndexer(t *testing.T) { if txRes.TxType != types.AccessListTxType { t.Fatalf("expected AccessListTxType (1), got %d", txRes.TxType) } - if txRes.Value != txs[3].Value().String() { - t.Fatalf("expected tx value %s got %s", txs[3].Value().String(), txRes.Value) + if txRes.Value != transactions[3].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[3].Value().String(), txRes.Value) } accessListElementModels := make([]models.AccessListElementModel, 0) pgStr = `SELECT access_list_elements.* FROM eth.access_list_elements INNER JOIN eth.transaction_cids ON (tx_id = transaction_cids.tx_hash) WHERE cid = $1 ORDER BY access_list_elements.index ASC` @@ -333,8 +361,8 @@ func TestFileIndexer(t *testing.T) { if txRes.TxType != types.DynamicFeeTxType { t.Fatalf("expected DynamicFeeTxType (2), got %d", txRes.TxType) } - if txRes.Value != txs[4].Value().String() { - t.Fatalf("expected tx value %s got %s", txs[4].Value().String(), txRes.Value) + if txRes.Value != transactions[4].Value().String() { + t.Fatalf("expected tx value %s got %s", transactions[4].Value().String(), txRes.Value) } } } @@ -396,7 +424,7 @@ func TestFileIndexer(t *testing.T) { dumpData(t) defer tearDown(t) - // check receipts were properly indexed + // check receipts were properly indexed and published rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids WHERE receipt_cids.tx_id = transaction_cids.tx_hash @@ -407,14 +435,19 @@ func TestFileIndexer(t *testing.T) { t.Fatal(err) } test_helpers.ExpectEqual(t, len(rcts), 5) + expectTrue(t, test_helpers.ListContainsString(rcts, rct1CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct2CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct3CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct4CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct5CID.String())) - for idx, rctLeafCID := range rcts { + for idx, c := range rcts { result := make([]models.IPLDModel, 0) pgStr = `SELECT data FROM eth.receipt_cids INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) WHERE receipt_cids.leaf_cid = $1` - err = sqlxdb.Select(&result, pgStr, rctLeafCID) + err = sqlxdb.Select(&result, pgStr, c) if err != nil { t.Fatal(err) } @@ -428,10 +461,7 @@ func TestFileIndexer(t *testing.T) { require.NoError(t, err) test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) - } - // and published - for _, c := range rcts { dc, err := cid.Decode(c) if err != nil { t.Fatal(err) @@ -446,7 +476,7 @@ func TestFileIndexer(t *testing.T) { postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` switch c { case rct1CID.String(): - test_helpers.ExpectEqual(t, data, rct1) + test_helpers.ExpectEqual(t, data, rctLeaf1) var postStatus uint64 pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` err = sqlxdb.Get(&postStatus, pgStr, c) @@ -455,7 +485,7 @@ func TestFileIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) case rct2CID.String(): - test_helpers.ExpectEqual(t, data, rct2) + test_helpers.ExpectEqual(t, data, rctLeaf2) var postState string err = sqlxdb.Get(&postState, postStatePgStr, c) if err != nil { @@ -463,7 +493,7 @@ func TestFileIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) case rct3CID.String(): - test_helpers.ExpectEqual(t, data, rct3) + test_helpers.ExpectEqual(t, data, rctLeaf3) var postState string err = sqlxdb.Get(&postState, postStatePgStr, c) if err != nil { @@ -471,7 +501,7 @@ func TestFileIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) case rct4CID.String(): - test_helpers.ExpectEqual(t, data, rct4) + test_helpers.ExpectEqual(t, data, rctLeaf4) var postState string err = sqlxdb.Get(&postState, postStatePgStr, c) if err != nil { @@ -479,7 +509,7 @@ func TestFileIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) case rct5CID.String(): - test_helpers.ExpectEqual(t, data, rct5) + test_helpers.ExpectEqual(t, data, rctLeaf5) var postState string err = sqlxdb.Get(&postState, postStatePgStr, c) if err != nil { diff --git a/statediff/indexer/database/sql/indexer_shared_test.go b/statediff/indexer/database/sql/indexer_shared_test.go index 46ad3a3f4..8bbab22ba 100644 --- a/statediff/indexer/database/sql/indexer_shared_test.go +++ b/statediff/indexer/database/sql/indexer_shared_test.go @@ -6,6 +6,8 @@ import ( "os" "testing" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" @@ -26,6 +28,7 @@ var ( mockBlock *types.Block headerCID, trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid + rctLeaf1, rctLeaf2, rctLeaf3, rctLeaf4, rctLeaf5 []byte state1CID, state2CID, storageCID cid.Cid ) @@ -95,14 +98,42 @@ func init() { trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256) trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256) trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256) - rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256) - rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256) - rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256) - rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256) - rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256) state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256) state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256) storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256) + + receiptTrie := ipld.NewRctTrie() + + receiptTrie.Add(0, rct1) + receiptTrie.Add(1, rct2) + receiptTrie.Add(2, rct3) + receiptTrie.Add(3, rct4) + receiptTrie.Add(4, rct5) + + rctLeafNodes, keys, _ := receiptTrie.GetLeafNodes() + + rctleafNodeCids := make([]cid.Cid, len(rctLeafNodes)) + orderedRctLeafNodes := make([][]byte, len(rctLeafNodes)) + for i, rln := range rctLeafNodes { + var idx uint + + r := bytes.NewReader(keys[i].TrieKey) + rlp.Decode(r, &idx) + rctleafNodeCids[idx] = rln.Cid() + orderedRctLeafNodes[idx] = rln.RawData() + } + + rct1CID = rctleafNodeCids[0] + rct2CID = rctleafNodeCids[1] + rct3CID = rctleafNodeCids[2] + rct4CID = rctleafNodeCids[3] + rct5CID = rctleafNodeCids[4] + + rctLeaf1 = orderedRctLeafNodes[0] + rctLeaf2 = orderedRctLeafNodes[1] + rctLeaf3 = orderedRctLeafNodes[2] + rctLeaf4 = orderedRctLeafNodes[3] + rctLeaf5 = orderedRctLeafNodes[4] } func expectTrue(t *testing.T, value bool) { diff --git a/statediff/indexer/database/sql/pgx_indexer_test.go b/statediff/indexer/database/sql/pgx_indexer_test.go index ce6fb13c0..426160cf9 100644 --- a/statediff/indexer/database/sql/pgx_indexer_test.go +++ b/statediff/indexer/database/sql/pgx_indexer_test.go @@ -111,7 +111,7 @@ func TestPGXIndexer(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { setupPGX(t) defer tearDown(t) - // check that txs were properly indexed + // check that txs were properly indexed and published trxs := make([]string, 0) pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1` @@ -125,7 +125,7 @@ func TestPGXIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) - // and published + transactions := mocks.MockBlock.Transactions() type txResult struct { TxType uint8 `db:"tx_type"` @@ -288,7 +288,7 @@ func TestPGXIndexer(t *testing.T) { setupPGX(t) defer tearDown(t) - // check receipts were properly indexed + // check receipts were properly indexed and published rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids WHERE receipt_cids.tx_id = transaction_cids.tx_hash @@ -299,14 +299,19 @@ func TestPGXIndexer(t *testing.T) { t.Fatal(err) } test_helpers.ExpectEqual(t, len(rcts), 5) + expectTrue(t, test_helpers.ListContainsString(rcts, rct1CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct2CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct3CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct4CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct5CID.String())) - for idx, rctLeafCID := range rcts { + for idx, c := range rcts { result := make([]models.IPLDModel, 0) pgStr = `SELECT data FROM eth.receipt_cids INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) WHERE receipt_cids.leaf_cid = $1` - err = db.Select(context.Background(), &result, pgStr, rctLeafCID) + err = db.Select(context.Background(), &result, pgStr, c) if err != nil { t.Fatal(err) } @@ -320,10 +325,7 @@ func TestPGXIndexer(t *testing.T) { require.NoError(t, err) test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) - } - // and published - for _, c := range rcts { dc, err := cid.Decode(c) if err != nil { t.Fatal(err) @@ -339,7 +341,7 @@ func TestPGXIndexer(t *testing.T) { postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` switch c { case rct1CID.String(): - test_helpers.ExpectEqual(t, data, rct1) + test_helpers.ExpectEqual(t, data, rctLeaf1) var postStatus uint64 pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` err = db.Get(context.Background(), &postStatus, pgStr, c) @@ -348,7 +350,7 @@ func TestPGXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) case rct2CID.String(): - test_helpers.ExpectEqual(t, data, rct2) + test_helpers.ExpectEqual(t, data, rctLeaf2) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -356,7 +358,7 @@ func TestPGXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) case rct3CID.String(): - test_helpers.ExpectEqual(t, data, rct3) + test_helpers.ExpectEqual(t, data, rctLeaf3) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -364,7 +366,7 @@ func TestPGXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) case rct4CID.String(): - test_helpers.ExpectEqual(t, data, rct4) + test_helpers.ExpectEqual(t, data, rctLeaf4) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -372,7 +374,7 @@ func TestPGXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) case rct5CID.String(): - test_helpers.ExpectEqual(t, data, rct5) + test_helpers.ExpectEqual(t, data, rctLeaf5) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { diff --git a/statediff/indexer/database/sql/sqlx_indexer_test.go b/statediff/indexer/database/sql/sqlx_indexer_test.go index 3403fa3dc..2c7ad4da5 100644 --- a/statediff/indexer/database/sql/sqlx_indexer_test.go +++ b/statediff/indexer/database/sql/sqlx_indexer_test.go @@ -114,7 +114,7 @@ func TestSQLXIndexer(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { setupSQLX(t) defer tearDown(t) - // check that txs were properly indexed + // check that txs were properly indexed and published trxs := make([]string, 0) pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) WHERE header_cids.block_number = $1` @@ -128,7 +128,7 @@ func TestSQLXIndexer(t *testing.T) { expectTrue(t, test_helpers.ListContainsString(trxs, trx3CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx4CID.String())) expectTrue(t, test_helpers.ListContainsString(trxs, trx5CID.String())) - // and published + transactions := mocks.MockBlock.Transactions() type txResult struct { TxType uint8 `db:"tx_type"` @@ -291,7 +291,7 @@ func TestSQLXIndexer(t *testing.T) { setupSQLX(t) defer tearDown(t) - // check receipts were properly indexed + // check receipts were properly indexed and published rcts := make([]string, 0) pgStr := `SELECT receipt_cids.leaf_cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids WHERE receipt_cids.tx_id = transaction_cids.tx_hash @@ -302,14 +302,19 @@ func TestSQLXIndexer(t *testing.T) { t.Fatal(err) } test_helpers.ExpectEqual(t, len(rcts), 5) + expectTrue(t, test_helpers.ListContainsString(rcts, rct1CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct2CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct3CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct4CID.String())) + expectTrue(t, test_helpers.ListContainsString(rcts, rct5CID.String())) - for idx, rctLeafCID := range rcts { + for idx, c := range rcts { result := make([]models.IPLDModel, 0) pgStr = `SELECT data FROM eth.receipt_cids INNER JOIN public.blocks ON (receipt_cids.leaf_mh_key = public.blocks.key) WHERE receipt_cids.leaf_cid = $1` - err = db.Select(context.Background(), &result, pgStr, rctLeafCID) + err = db.Select(context.Background(), &result, pgStr, c) if err != nil { t.Fatal(err) } @@ -323,10 +328,7 @@ func TestSQLXIndexer(t *testing.T) { require.NoError(t, err) test_helpers.ExpectEqual(t, expectedRct, nodeElements[1].([]byte)) - } - // and published - for _, c := range rcts { dc, err := cid.Decode(c) if err != nil { t.Fatal(err) @@ -341,7 +343,7 @@ func TestSQLXIndexer(t *testing.T) { postStatePgStr := `SELECT post_state FROM eth.receipt_cids WHERE leaf_cid = $1` switch c { case rct1CID.String(): - test_helpers.ExpectEqual(t, data, rct1) + test_helpers.ExpectEqual(t, data, rctLeaf1) var postStatus uint64 pgStr = `SELECT post_status FROM eth.receipt_cids WHERE leaf_cid = $1` err = db.Get(context.Background(), &postStatus, pgStr, c) @@ -350,7 +352,7 @@ func TestSQLXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postStatus, mocks.ExpectedPostStatus) case rct2CID.String(): - test_helpers.ExpectEqual(t, data, rct2) + test_helpers.ExpectEqual(t, data, rctLeaf2) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -358,7 +360,7 @@ func TestSQLXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState1) case rct3CID.String(): - test_helpers.ExpectEqual(t, data, rct3) + test_helpers.ExpectEqual(t, data, rctLeaf3) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -366,7 +368,7 @@ func TestSQLXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState2) case rct4CID.String(): - test_helpers.ExpectEqual(t, data, rct4) + test_helpers.ExpectEqual(t, data, rctLeaf4) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { @@ -374,7 +376,7 @@ func TestSQLXIndexer(t *testing.T) { } test_helpers.ExpectEqual(t, postState, mocks.ExpectedPostState3) case rct5CID.String(): - test_helpers.ExpectEqual(t, data, rct5) + test_helpers.ExpectEqual(t, data, rctLeaf5) var postState string err = db.Get(context.Background(), &postState, postStatePgStr, c) if err != nil { -- 2.45.2