From a7b83fc63c9f07e049b050f0a46e0035a7bdb162 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 6 Aug 2023 13:28:49 +0800 Subject: [PATCH 01/22] factor out PushHeader method --- indexer/database/dump/indexer.go | 18 ++++++--- indexer/database/file/indexer.go | 16 ++++++-- indexer/database/sql/batch_tx.go | 66 +++++++++++++++++++++++++++++--- indexer/database/sql/indexer.go | 18 ++++++--- indexer/database/sql/lazy_tx.go | 4 ++ indexer/interfaces/interfaces.go | 1 + indexer/ipld/eth_parser.go | 12 ++---- test_helpers/mocks/indexer.go | 4 ++ 8 files changed, 111 insertions(+), 28 deletions(-) diff --git a/indexer/database/dump/indexer.go b/indexer/database/dump/indexer.go index 70e86b7..3c9f764 100644 --- a/indexer/database/dump/indexer.go +++ b/indexer/database/dump/indexer.go @@ -74,7 +74,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } // Generate the block iplds - headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) + txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) if err != nil { return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) } @@ -129,7 +129,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // Publish and index header, collect headerID var headerID string - headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) + headerID, err = sdi.PushHeader(blockTx, block.Header(), reward, totalDifficulty) if err != nil { return nil, err } @@ -167,9 +167,17 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip return blockTx, err } -// processHeader publishes and indexes a header IPLD in Postgres +// PushHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode ipld.IPLD, reward, td *big.Int) (string, error) { +func (sdi *StateDiffIndexer) PushHeader(batch interfaces.Batch, header *types.Header, reward, td *big.Int) (string, error) { + tx, ok := batch.(*BatchTx) + if !ok { + return "", fmt.Errorf("sql: batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } + headerNode, err := ipld.NewEthHeader(header) + if err != nil { + return "", err + } tx.cacheIPLD(headerNode) headerID := header.Hash().String() @@ -189,7 +197,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he Coinbase: header.Coinbase.String(), Canonical: true, } - _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", mod) + _, err = fmt.Fprintf(sdi.dump, "%+v\r\n", mod) return headerID, err } diff --git a/indexer/database/file/indexer.go b/indexer/database/file/indexer.go index 1177b54..65b5488 100644 --- a/indexer/database/file/indexer.go +++ b/indexer/database/file/indexer.go @@ -142,7 +142,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } // Generate the block iplds - headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) + txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) if err != nil { return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) } @@ -183,7 +183,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip t = time.Now() // write header, collect headerID - headerID := sdi.processHeader(block.Header(), headerNode, reward, totalDifficulty) + headerID, err := sdi.PushHeader(blockTx, block.Header(), reward, totalDifficulty) + if err != nil { + return nil, err + } tDiff = time.Since(t) metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) @@ -219,7 +222,12 @@ 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 ipld.IPLD, reward, td *big.Int) string { +func (sdi *StateDiffIndexer) PushHeader(_ interfaces.Batch, header *types.Header, reward, td *big.Int) (string, error) { + // Process the header + headerNode, err := ipld.NewEthHeader(header) + if err != nil { + return "", err + } sdi.fileWriter.upsertIPLDNode(header.Number.String(), headerNode) headerID := header.Hash().String() @@ -240,7 +248,7 @@ func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode ipld Coinbase: header.Coinbase.String(), Canonical: true, }) - return headerID + return headerID, nil } // processUncles publishes and indexes uncle IPLDs in Postgres diff --git a/indexer/database/sql/batch_tx.go b/indexer/database/sql/batch_tx.go index 5c35e88..dbce5b5 100644 --- a/indexer/database/sql/batch_tx.go +++ b/indexer/database/sql/batch_tx.go @@ -18,11 +18,14 @@ package sql import ( "context" + "math/big" "sync" "sync/atomic" + "time" "github.com/lib/pq" + "github.com/cerc-io/plugeth-statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/utils/log" @@ -41,14 +44,67 @@ type BatchTx struct { ipldCache models.IPLDBatch removedCacheFlag *uint32 // Tracks expected cache size and ensures cache is caught up before flush - cacheWg sync.WaitGroup - - submit func(blockTx *BatchTx, err error) error + cacheWg sync.WaitGroup + startTime time.Time } -// Submit satisfies indexer.AtomicTx +func NewBatchAt(insertStm string, ctx context.Context, number *big.Int, tx Tx, start time.Time) *BatchTx { + blockTx := &BatchTx{ + removedCacheFlag: new(uint32), + ctx: ctx, + BlockNumber: number.String(), + stm: insertStm, + iplds: make(chan models.IPLDModel), + quit: make(chan (chan<- struct{})), + ipldCache: models.IPLDBatch{ + BlockNumbers: make([]string, 0, startingCacheCapacity), + Keys: make([]string, 0, startingCacheCapacity), + Values: make([][]byte, 0, startingCacheCapacity), + }, + dbtx: tx, + startTime: time.Now(), + } + return blockTx +} + +// Submit satisfies indexer.Batch func (tx *BatchTx) Submit(err error) error { - return tx.submit(tx, err) + defer tx.close() + + metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(time.Since(tx.startTime)) + t := time.Now() + if err := tx.flush(); err != nil { + rollback(tx.ctx, tx.dbtx) + return err + } + err = tx.dbtx.Commit(tx.ctx) + metrics.IndexerMetrics.PostgresCommitTimer.Update(time.Since(t)) + return err +} + +func (tx *BatchTx) RollbackOnFailure(err error) { + defer tx.close() + + if p := recover(); p != nil { + log.Info("panic detected before tx submission, rolling back the tx", "panic", p) + rollback(tx.ctx, tx.dbtx) + panic(p) + } else if err != nil { + log.Info("error detected before tx submission, rolling back the tx", "error", err) + rollback(tx.ctx, tx.dbtx) + } +} + +func (tx *BatchTx) close() { + if tx.quit == nil { + return + } + confirm := make(chan struct{}) + tx.quit <- confirm + close(tx.quit) + <-confirm + close(tx.iplds) + tx.quit = nil } func (tx *BatchTx) flush() error { diff --git a/indexer/database/sql/indexer.go b/indexer/database/sql/indexer.go index d41f3d6..150d405 100644 --- a/indexer/database/sql/indexer.go +++ b/indexer/database/sql/indexer.go @@ -93,7 +93,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } // Generate the block iplds - headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) + txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) if err != nil { return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) } @@ -181,8 +181,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip t = time.Now() // Publish and index header, collect headerID - var headerID string - headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty) + headerID, err := sdi.PushHeader(blockTx, block.Header(), reward, totalDifficulty) if err != nil { return nil, err } @@ -230,9 +229,18 @@ func (sdi *StateDiffIndexer) DetectGaps(beginBlockNumber uint64, endBlockNumber return sdi.dbWriter.detectGaps(beginBlockNumber, endBlockNumber) } -// processHeader publishes and indexes a header IPLD in Postgres +// PushHeader publishes and indexes a header IPLD in Postgres // it returns the headerID -func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode ipld.IPLD, reward, td *big.Int) (string, error) { +func (sdi *StateDiffIndexer) PushHeader(batch interfaces.Batch, header *types.Header, reward, td *big.Int) (string, error) { + tx, ok := batch.(*BatchTx) + if !ok { + return "", fmt.Errorf("sql: batch is expected to be of type %T, got %T", &BatchTx{}, batch) + } + // Process the header + headerNode, err := ipld.NewEthHeader(header) + if err != nil { + return "", err + } tx.cacheIPLD(headerNode) headerID := header.Hash().String() diff --git a/indexer/database/sql/lazy_tx.go b/indexer/database/sql/lazy_tx.go index 7543944..d34d8ae 100644 --- a/indexer/database/sql/lazy_tx.go +++ b/indexer/database/sql/lazy_tx.go @@ -3,7 +3,9 @@ package sql import ( "context" "reflect" + "time" + "github.com/cerc-io/plugeth-statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/utils/log" ) @@ -69,10 +71,12 @@ func (tx *DelayedTx) Exec(ctx context.Context, sql string, args ...interface{}) } func (tx *DelayedTx) Commit(ctx context.Context) error { + t := time.Now() base, err := tx.db.Begin(ctx) if err != nil { return err } + metrics.IndexerMetrics.FreePostgresTimer.Update(time.Since(t)) defer func() { if p := recover(); p != nil { rollback(ctx, base) diff --git a/indexer/interfaces/interfaces.go b/indexer/interfaces/interfaces.go index ad3f42c..be7080c 100644 --- a/indexer/interfaces/interfaces.go +++ b/indexer/interfaces/interfaces.go @@ -34,6 +34,7 @@ type StateDiffIndexer interface { CurrentBlock() (*models.HeaderModel, error) HasBlock(hash common.Hash, number uint64) (bool, error) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (Batch, error) + PushHeader(batch Batch, header *types.Header, reward, td *big.Int) (string, error) PushStateNode(tx Batch, stateNode sdtypes.StateLeafNode, headerID string) error PushIPLD(tx Batch, ipld sdtypes.IPLD) error ReportDBMetrics(delay time.Duration, quit <-chan bool) diff --git a/indexer/ipld/eth_parser.go b/indexer/ipld/eth_parser.go index 9ce7155..5ec8bf9 100644 --- a/indexer/ipld/eth_parser.go +++ b/indexer/ipld/eth_parser.go @@ -22,23 +22,17 @@ import ( // 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, []*EthTx, []*EthReceipt, [][]*EthLog, error) { - // Process the header - headerNode, err := NewEthHeader(block.Header()) - if err != nil { - return nil, nil, nil, nil, err - } - +func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) ([]*EthTx, []*EthReceipt, [][]*EthLog, error) { // Process the txs txNodes, err := processTransactions(block.Transactions()) if err != nil { - return nil, nil, nil, nil, err + return nil, nil, nil, err } // Process the receipts and logs rctNodes, logNodes, err := processReceiptsAndLogs(receipts) - return headerNode, txNodes, rctNodes, logNodes, err + return txNodes, rctNodes, logNodes, err } // processTransactions will take the found transactions in a parsed block body diff --git a/test_helpers/mocks/indexer.go b/test_helpers/mocks/indexer.go index e4ca5fb..8991f9e 100644 --- a/test_helpers/mocks/indexer.go +++ b/test_helpers/mocks/indexer.go @@ -52,6 +52,10 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip return &batch{}, nil } +func (sdi *StateDiffIndexer) PushHeader(batch interfaces.Batch, header *types.Header, reward, td *big.Int) (string, error) { + return "", nil +} + func (sdi *StateDiffIndexer) PushStateNode(txi interfaces.Batch, stateNode sdtypes.StateLeafNode, headerID string) error { return nil } -- 2.45.2 From 68e2aca32ecccfedf4b7c4455e9d5769a6a53f94 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 6 Aug 2023 13:29:19 +0800 Subject: [PATCH 02/22] refactor batch --- indexer/database/dump/batch_tx.go | 55 +++++++++---- indexer/database/dump/indexer.go | 52 +++---------- indexer/database/file/batch_tx.go | 25 ++++-- indexer/database/file/indexer.go | 58 +++++--------- indexer/database/sql/batch_tx.go | 31 ++++---- indexer/database/sql/indexer.go | 124 ++++++++---------------------- indexer/interfaces/interfaces.go | 7 +- indexer/test/test.go | 33 ++------ indexer/test/test_legacy.go | 9 +-- indexer/test/test_mainnet.go | 10 +-- service.go | 9 ++- test_helpers/mocks/indexer.go | 14 +++- 12 files changed, 184 insertions(+), 243 deletions(-) diff --git a/indexer/database/dump/batch_tx.go b/indexer/database/dump/batch_tx.go index 1923622..a36d7ce 100644 --- a/indexer/database/dump/batch_tx.go +++ b/indexer/database/dump/batch_tx.go @@ -19,26 +19,55 @@ package dump import ( "fmt" "io" + "math/big" "github.com/cerc-io/plugeth-statediff/indexer/ipld" - "github.com/cerc-io/plugeth-statediff/indexer/models" + "github.com/cerc-io/plugeth-statediff/utils/log" ) // BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration type BatchTx struct { - BlockNumber string - dump io.Writer - quit chan struct{} - iplds chan models.IPLDModel - ipldCache models.IPLDBatch - - submit func(blockTx *BatchTx, err error) error + blockNum string + dump io.Writer + quit chan struct{} + iplds chan models.IPLDModel + ipldCache models.IPLDBatch } -// Submit satisfies indexer.AtomicTx -func (tx *BatchTx) Submit(err error) error { - return tx.submit(tx, err) +func NewBatch(number *big.Int, dest io.Writer) *BatchTx { + batch := &BatchTx{ + blockNum: number.String(), + dump: dest, + iplds: make(chan models.IPLDModel), + quit: make(chan struct{}), + ipldCache: models.IPLDBatch{}, + } + go batch.cache() + return batch +} + +func (self *BatchTx) Submit() error { + close(self.quit) + close(self.iplds) + + if err := self.flush(); err != nil { + return err + } + return nil +} + +func (tx *BatchTx) BlockNumber() string { + return tx.blockNum +} + +func (tx *BatchTx) RollbackOnFailure(err error) { + if p := recover(); p != nil { + log.Info("panic detected before tx submission, but rollback not supported", "panic", p) + panic(p) + } else if err != nil { + log.Info("error detected before tx submission, but rollback not supported", "error", err) + } } func (tx *BatchTx) flush() error { @@ -65,7 +94,7 @@ func (tx *BatchTx) cache() { func (tx *BatchTx) cacheDirect(key string, value []byte) { tx.iplds <- models.IPLDModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), Key: key, Data: value, } @@ -73,7 +102,7 @@ func (tx *BatchTx) cacheDirect(key string, value []byte) { func (tx *BatchTx) cacheIPLD(i ipld.IPLD) { tx.iplds <- models.IPLDModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), Key: i.Cid().String(), Data: i.RawData(), } diff --git a/indexer/database/dump/indexer.go b/indexer/database/dump/indexer.go index 3c9f764..7307989 100644 --- a/indexer/database/dump/indexer.go +++ b/indexer/database/dump/indexer.go @@ -17,6 +17,7 @@ package dump import ( + "context" "encoding/hex" "fmt" "io" @@ -37,7 +38,6 @@ import ( "github.com/cerc-io/plugeth-statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/shared" sdtypes "github.com/cerc-io/plugeth-statediff/types" - "github.com/cerc-io/plugeth-statediff/utils/log" ) var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} @@ -62,7 +62,7 @@ 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() + t := time.Now() blockHash := block.Hash() blockHashStr := blockHash.String() height := block.NumberU64() @@ -91,40 +91,8 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } else { reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) } - t = time.Now() - blockTx := &BatchTx{ - BlockNumber: block.Number().String(), - 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) - metrics.IndexerMetrics.StateStoreCodeProcessingTimer.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) - metrics.IndexerMetrics.PostgresCommitTimer.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) - metrics.IndexerMetrics.FreePostgresTimer.Update(tDiff) - - traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String()) + blockTx := NewBatch(block.Number(), sdi.dump) t = time.Now() // Publish and index header, collect headerID @@ -133,7 +101,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip if err != nil { return nil, err } - tDiff = time.Since(t) + tDiff := time.Since(t) metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) t = time.Now() @@ -352,7 +320,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 ipld.blocks entry for the Removed node is present stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: shared.RemovedNodeStateCID, @@ -360,7 +328,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt } } else { stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: stateNode.AccountWrapper.CID, @@ -383,7 +351,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 ipld.blocks entry for the Removed node is present storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -396,7 +364,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt continue } storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -440,6 +408,10 @@ func (sdi *StateDiffIndexer) DetectGaps(beginBlockNumber uint64, endBlockNumber return nil, nil } +func (sdi *StateDiffIndexer) BeginTx(number *big.Int, _ context.Context) interfaces.Batch { + return NewBatch(number, sdi.dump) +} + // Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { return sdi.dump.Close() diff --git a/indexer/database/file/batch_tx.go b/indexer/database/file/batch_tx.go index d38bd12..3096204 100644 --- a/indexer/database/file/batch_tx.go +++ b/indexer/database/file/batch_tx.go @@ -16,14 +16,29 @@ package file +import "github.com/cerc-io/plugeth-statediff/utils/log" + // BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration type BatchTx struct { - BlockNumber string - - submit func(blockTx *BatchTx, err error) error + blockNum string + fileWriter FileWriter } // Submit satisfies indexer.AtomicTx -func (tx *BatchTx) Submit(err error) error { - return tx.submit(tx, err) +func (tx *BatchTx) Submit() error { + tx.fileWriter.Flush() + return nil +} + +func (tx *BatchTx) BlockNumber() string { + return tx.blockNum +} + +func (tx *BatchTx) RollbackOnFailure(err error) { + if p := recover(); p != nil { + log.Info("panic detected before tx submission, but rollback not supported", "panic", p) + panic(p) + } else if err != nil { + log.Info("error detected before tx submission, but rollback not supported", "error", err) + } } diff --git a/indexer/database/file/indexer.go b/indexer/database/file/indexer.go index 65b5488..b213f91 100644 --- a/indexer/database/file/indexer.go +++ b/indexer/database/file/indexer.go @@ -17,6 +17,7 @@ package file import ( + "context" "errors" "fmt" "math/big" @@ -130,7 +131,7 @@ 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() + t := time.Now() blockHash := block.Hash() blockHashStr := blockHash.String() height := block.NumberU64() @@ -159,27 +160,11 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip } else { reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) } - t = time.Now() blockTx := &BatchTx{ - BlockNumber: block.Number().String(), - submit: func(self *BatchTx, err error) error { - tDiff := time.Since(t) - metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(tDiff) - traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String()) - t = time.Now() - sdi.fileWriter.Flush() - tDiff = time.Since(t) - metrics.IndexerMetrics.PostgresCommitTimer.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.Trace(traceMsg) - return err - }, + blockNum: block.Number().String(), + fileWriter: sdi.fileWriter, } - tDiff := time.Since(t) - metrics.IndexerMetrics.FreePostgresTimer.Update(tDiff) - traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String()) t = time.Now() // write header, collect headerID @@ -187,7 +172,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip if err != nil { return nil, err } - tDiff = time.Since(t) + tDiff := time.Since(t) metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) t = time.Now() @@ -382,20 +367,16 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error { } // 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.StateLeafNode, headerID string) error { - tx, ok := batch.(*BatchTx) - if !ok { - return fmt.Errorf("file: batch is expected to be of type %T, got %T", &BatchTx{}, batch) - } +func (sdi *StateDiffIndexer) PushStateNode(tx interfaces.Batch, stateNode sdtypes.StateLeafNode, headerID string) error { // publish the state node var stateModel models.StateNodeModel if stateNode.Removed { if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 { atomic.StoreUint32(&sdi.removedCacheFlag, 1) - sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStateCID, []byte{}) + sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), shared.RemovedNodeStateCID, []byte{}) } stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: shared.RemovedNodeStateCID, @@ -403,7 +384,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt } } else { stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: stateNode.AccountWrapper.CID, @@ -423,10 +404,10 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt if storageNode.Removed { if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 { atomic.StoreUint32(&sdi.removedCacheFlag, 1) - sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStorageCID, []byte{}) + sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), shared.RemovedNodeStorageCID, []byte{}) } storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -438,7 +419,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt continue } storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -453,12 +434,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt } // PushIPLD writes iplds to ipld.blocks -func (sdi *StateDiffIndexer) PushIPLD(batch interfaces.Batch, ipld sdtypes.IPLD) error { - tx, ok := batch.(*BatchTx) - if !ok { - return fmt.Errorf("file: batch is expected to be of type %T, got %T", &BatchTx{}, batch) - } - sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, ipld.CID, ipld.Content) +func (sdi *StateDiffIndexer) PushIPLD(tx interfaces.Batch, ipld sdtypes.IPLD) error { + sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), ipld.CID, ipld.Content) return nil } @@ -480,6 +457,13 @@ func (sdi *StateDiffIndexer) HasBlock(hash common.Hash, number uint64) (bool, er return false, nil } +func (sdi *StateDiffIndexer) BeginTx(number *big.Int, _ context.Context) interfaces.Batch { + return &BatchTx{ + blockNum: number.String(), + fileWriter: sdi.fileWriter, + } +} + // Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { return sdi.fileWriter.Close() diff --git a/indexer/database/sql/batch_tx.go b/indexer/database/sql/batch_tx.go index dbce5b5..d42b214 100644 --- a/indexer/database/sql/batch_tx.go +++ b/indexer/database/sql/batch_tx.go @@ -35,7 +35,7 @@ const startingCacheCapacity = 1024 * 24 // BatchTx wraps a sql tx with the state necessary for building the tx concurrently during trie difference iteration type BatchTx struct { - BlockNumber string + blockNum string ctx context.Context dbtx Tx stm string @@ -44,15 +44,14 @@ type BatchTx struct { ipldCache models.IPLDBatch removedCacheFlag *uint32 // Tracks expected cache size and ensures cache is caught up before flush - cacheWg sync.WaitGroup - startTime time.Time + cacheWg sync.WaitGroup } -func NewBatchAt(insertStm string, ctx context.Context, number *big.Int, tx Tx, start time.Time) *BatchTx { +func NewBatch(insertStm string, ctx context.Context, number *big.Int, tx Tx) *BatchTx { blockTx := &BatchTx{ removedCacheFlag: new(uint32), ctx: ctx, - BlockNumber: number.String(), + blockNum: number.String(), stm: insertStm, iplds: make(chan models.IPLDModel), quit: make(chan (chan<- struct{})), @@ -61,35 +60,39 @@ func NewBatchAt(insertStm string, ctx context.Context, number *big.Int, tx Tx, s Keys: make([]string, 0, startingCacheCapacity), Values: make([][]byte, 0, startingCacheCapacity), }, - dbtx: tx, - startTime: time.Now(), + dbtx: tx, } + go blockTx.cache() return blockTx } // Submit satisfies indexer.Batch -func (tx *BatchTx) Submit(err error) error { +func (tx *BatchTx) Submit() error { defer tx.close() - metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(time.Since(tx.startTime)) t := time.Now() if err := tx.flush(); err != nil { rollback(tx.ctx, tx.dbtx) return err } - err = tx.dbtx.Commit(tx.ctx) + err := tx.dbtx.Commit(tx.ctx) metrics.IndexerMetrics.PostgresCommitTimer.Update(time.Since(t)) return err } +func (tx *BatchTx) BlockNumber() string { + return tx.blockNum +} + func (tx *BatchTx) RollbackOnFailure(err error) { - defer tx.close() if p := recover(); p != nil { + defer tx.close() log.Info("panic detected before tx submission, rolling back the tx", "panic", p) rollback(tx.ctx, tx.dbtx) panic(p) } else if err != nil { + defer tx.close() log.Info("error detected before tx submission, rolling back the tx", "error", err) rollback(tx.ctx, tx.dbtx) } @@ -148,7 +151,7 @@ func (tx *BatchTx) cache() { func (tx *BatchTx) cacheDirect(key string, value []byte) { tx.cacheWg.Add(1) tx.iplds <- models.IPLDModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), Key: key, Data: value, } @@ -157,7 +160,7 @@ func (tx *BatchTx) cacheDirect(key string, value []byte) { func (tx *BatchTx) cacheIPLD(i ipld.IPLD) { tx.cacheWg.Add(1) tx.iplds <- models.IPLDModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), Key: i.Cid().String(), Data: i.RawData(), } @@ -168,7 +171,7 @@ func (tx *BatchTx) cacheRemoved(key string, value []byte) { atomic.StoreUint32(tx.removedCacheFlag, 1) tx.cacheWg.Add(1) tx.iplds <- models.IPLDModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), Key: key, Data: value, } diff --git a/indexer/database/sql/indexer.go b/indexer/database/sql/indexer.go index 150d405..6dab963 100644 --- a/indexer/database/sql/indexer.go +++ b/indexer/database/sql/indexer.go @@ -39,7 +39,6 @@ import ( "github.com/cerc-io/plugeth-statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/shared" sdtypes "github.com/cerc-io/plugeth-statediff/types" - "github.com/cerc-io/plugeth-statediff/utils/log" ) var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} @@ -82,11 +81,12 @@ func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bo // 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() + t := time.Now() blockHash := block.Hash() height := block.NumberU64() - traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHash) transactions := block.Transactions() + var err error + // Derive any missing fields if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil { return nil, err @@ -95,11 +95,12 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip // Generate the block iplds txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) if err != nil { - return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err) + return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %w", err) } if len(txNodes) != len(rctNodes) { - return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d)", len(txNodes), len(rctNodes)) + return nil, fmt.Errorf("expected number of transactions (%d) does not match number of receipts (%d)", + len(txNodes), len(rctNodes)) } // Calculate reward @@ -108,98 +109,35 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip if sdi.chainConfig.Clique != nil { reward = big.NewInt(0) } else { - reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) + reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), transactions, receipts) } t = time.Now() // Begin new DB tx for everything - tx := NewDelayedTx(sdi.dbWriter.db) - defer func() { - if p := recover(); p != nil { - rollback(sdi.ctx, tx) - panic(p) - } else if err != nil { - rollback(sdi.ctx, tx) - } - }() - blockTx := &BatchTx{ - removedCacheFlag: new(uint32), - ctx: sdi.ctx, - BlockNumber: block.Number().String(), - stm: sdi.dbWriter.db.InsertIPLDsStm(), - iplds: make(chan models.IPLDModel), - quit: make(chan (chan<- struct{})), - ipldCache: models.IPLDBatch{ - BlockNumbers: make([]string, 0, startingCacheCapacity), - Keys: make([]string, 0, startingCacheCapacity), - Values: make([][]byte, 0, startingCacheCapacity), - }, - dbtx: tx, - // handle transaction commit or rollback for any return case - submit: func(self *BatchTx, err error) error { - defer func() { - confirm := make(chan struct{}) - self.quit <- confirm - close(self.quit) - <-confirm - close(self.iplds) - }() - if p := recover(); p != nil { - log.Info("panic detected before tx submission, rolling back the tx", "panic", p) - rollback(sdi.ctx, tx) - panic(p) - } else if err != nil { - log.Info("error detected before tx submission, rolling back the tx", "error", err) - rollback(sdi.ctx, tx) - } else { - tDiff := time.Since(t) - metrics2.IndexerMetrics.StateStoreCodeProcessingTimer.Update(tDiff) - traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff) - t = time.Now() - if err := self.flush(); err != nil { - rollback(sdi.ctx, tx) - traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start)) - log.Debug(traceMsg) - return err - } - err = tx.Commit(sdi.ctx) - tDiff = time.Since(t) - metrics2.IndexerMetrics.PostgresCommitTimer.Update(tDiff) - traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff) - } - traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start)) - log.Debug(traceMsg) - return err - }, - } - go blockTx.cache() - - tDiff := time.Since(t) - metrics2.IndexerMetrics.FreePostgresTimer.Update(tDiff) - - traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff) - t = time.Now() + batch := NewBatch( + sdi.dbWriter.db.InsertIPLDsStm(), sdi.ctx, + block.Number(), + NewDelayedTx(sdi.dbWriter.db), + ) + // handle transaction rollback for failures in this scope + defer batch.RollbackOnFailure(err) // Publish and index header, collect headerID - headerID, err := sdi.PushHeader(blockTx, block.Header(), reward, totalDifficulty) + headerID, err := sdi.PushHeader(batch, block.Header(), reward, totalDifficulty) if err != nil { return nil, err } - tDiff = time.Since(t) - metrics2.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) - traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff) + metrics2.IndexerMetrics.HeaderProcessingTimer.Update(time.Since(t)) t = time.Now() // Publish and index uncles - err = sdi.processUncles(blockTx, headerID, block.Number(), block.UncleHash(), block.Uncles()) + err = sdi.processUncles(batch, headerID, block.Number(), block.UncleHash(), block.Uncles()) if err != nil { return nil, err } - tDiff = time.Since(t) - metrics2.IndexerMetrics.UncleProcessingTimer.Update(tDiff) - traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff) + metrics2.IndexerMetrics.UncleProcessingTimer.Update(time.Since(t)) t = time.Now() // Publish and index receipts and txs - err = sdi.processReceiptsAndTxs(blockTx, processArgs{ + err = sdi.processReceiptsAndTxs(batch, processArgs{ headerID: headerID, blockNumber: block.Number(), receipts: receipts, @@ -211,12 +149,9 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip if err != nil { return nil, err } - tDiff = time.Since(t) - metrics2.IndexerMetrics.TxAndRecProcessingTimer.Update(tDiff) - traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff) - t = time.Now() + metrics2.IndexerMetrics.TxAndRecProcessingTimer.Update(time.Since(t)) - return blockTx, err + return batch, err } // CurrentBlock returns the HeaderModel of the highest existing block in the database. @@ -414,7 +349,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt if stateNode.Removed { tx.cacheRemoved(shared.RemovedNodeStateCID, []byte{}) stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: shared.RemovedNodeStateCID, @@ -422,7 +357,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt } } else { stateModel = models.StateNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), CID: stateNode.AccountWrapper.CID, @@ -444,7 +379,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt if storageNode.Removed { tx.cacheRemoved(shared.RemovedNodeStorageCID, []byte{}) storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -458,7 +393,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt continue } storageModel := models.StorageNodeModel{ - BlockNumber: tx.BlockNumber, + BlockNumber: tx.BlockNumber(), HeaderID: headerID, StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(), @@ -489,6 +424,15 @@ func (sdi *StateDiffIndexer) HasBlock(hash common.Hash, number uint64) (bool, er return sdi.dbWriter.hasHeader(hash, number) } +func (sdi *StateDiffIndexer) BeginTx(number *big.Int, ctx context.Context) interfaces.Batch { + return NewBatch( + sdi.dbWriter.db.InsertIPLDsStm(), + ctx, + number, + NewDelayedTx(sdi.dbWriter.db), + ) +} + // Close satisfies io.Closer func (sdi *StateDiffIndexer) Close() error { return sdi.dbWriter.Close() diff --git a/indexer/interfaces/interfaces.go b/indexer/interfaces/interfaces.go index be7080c..84f76d5 100644 --- a/indexer/interfaces/interfaces.go +++ b/indexer/interfaces/interfaces.go @@ -17,6 +17,7 @@ package interfaces import ( + "context" "math/big" "time" @@ -39,6 +40,8 @@ type StateDiffIndexer interface { PushIPLD(tx Batch, ipld sdtypes.IPLD) error ReportDBMetrics(delay time.Duration, quit <-chan bool) + BeginTx(number *big.Int, ctx context.Context) Batch + // Methods used by WatchAddress API/functionality LoadWatchedAddresses() ([]common.Address, error) InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error @@ -51,7 +54,9 @@ type StateDiffIndexer interface { // Batch required for indexing data atomically type Batch interface { - Submit(err error) error + Submit() error + BlockNumber() string + RollbackOnFailure(error) } // Config used to configure different underlying implementations diff --git a/indexer/test/test.go b/indexer/test/test.go index ad43529..5f9cf7c 100644 --- a/indexer/test/test.go +++ b/indexer/test/test.go @@ -28,7 +28,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/cerc-io/plugeth-statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/mocks" @@ -48,7 +47,7 @@ func SetupTestData(t *testing.T, ind interfaces.StateDiffIndexer) { t.Fatal(err) } defer func() { - if err := tx.Submit(err); err != nil { + if err := tx.Submit(); err != nil { t.Fatal(err) } }() @@ -61,11 +60,7 @@ func SetupTestData(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) } - if batchTx, ok := tx.(*sql.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber) - } else if batchTx, ok := tx.(*file.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber) - } + require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber()) } func DoTestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) { @@ -547,13 +542,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) } - if batchTx, ok := tx1.(*sql.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber) - } else if batchTx, ok := tx1.(*file.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber) - } + require.Equal(t, mocks.BlockNumber.String(), tx1.BlockNumber()) - if err := tx1.Submit(err); err != nil { + if err := tx1.Submit(); err != nil { t.Fatal(err) } @@ -572,13 +563,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) } - if tx, ok := tx2.(*sql.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber) - } else if tx, ok := tx2.(*sql.BatchTx); ok { - require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber) - } + require.Equal(t, mocks.BlockNumber.String(), tx2.BlockNumber()) - if err := tx2.Submit(err); err != nil { + if err := tx2.Submit(); err != nil { t.Fatal(err) } @@ -597,13 +584,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) } - if batchTx, ok := tx3.(*sql.BatchTx); ok { - require.Equal(t, mocks.Block2Number.String(), batchTx.BlockNumber) - } else if batchTx, ok := tx3.(*file.BatchTx); ok { - require.Equal(t, mocks.Block2Number.String(), batchTx.BlockNumber) - } + require.Equal(t, mocks.Block2Number.String(), tx3.BlockNumber()) - if err := tx3.Submit(err); err != nil { + if err := tx3.Submit(); err != nil { t.Fatal(err) } } diff --git a/indexer/test/test_legacy.go b/indexer/test/test_legacy.go index 6b93f79..8878edc 100644 --- a/indexer/test/test_legacy.go +++ b/indexer/test/test_legacy.go @@ -20,7 +20,6 @@ import ( "context" "testing" - "github.com/cerc-io/plugeth-statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/ipld" @@ -51,7 +50,7 @@ func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) defer func() { - if err := tx.Submit(err); err != nil { + if err := tx.Submit(); err != nil { t.Fatal(err) } }() @@ -60,11 +59,7 @@ func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) { require.NoError(t, err) } - if batchTx, ok := tx.(*sql.BatchTx); ok { - require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber) - } else if batchTx, ok := tx.(*file.BatchTx); ok { - require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber) - } + require.Equal(t, legacyData.BlockNumber.String(), tx.BlockNumber()) } func TestLegacyIndexer(t *testing.T, db sql.Database) { diff --git a/indexer/test/test_mainnet.go b/indexer/test/test_mainnet.go index 01289ab..6e06458 100644 --- a/indexer/test/test_mainnet.go +++ b/indexer/test/test_mainnet.go @@ -19,8 +19,6 @@ package test import ( "testing" - "github.com/cerc-io/plugeth-statediff/indexer/database/file" - "github.com/cerc-io/plugeth-statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/mocks" "github.com/ethereum/go-ethereum/core/types" @@ -36,7 +34,7 @@ func TestBlock(t *testing.T, ind interfaces.StateDiffIndexer, testBlock *types.B require.NoError(t, err) defer func() { - if err := tx.Submit(err); err != nil { + if err := tx.Submit(); err != nil { t.Fatal(err) } }() @@ -45,9 +43,5 @@ func TestBlock(t *testing.T, ind interfaces.StateDiffIndexer, testBlock *types.B require.NoError(t, err) } - if batchTx, ok := tx.(*sql.BatchTx); ok { - require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber) - } else if batchTx, ok := tx.(*file.BatchTx); ok { - require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber) - } + require.Equal(t, testBlock.Number().String(), tx.BlockNumber()) } diff --git a/service.go b/service.go index 7cc7f80..604b2a6 100644 --- a/service.go +++ b/service.go @@ -817,6 +817,8 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p if params.IncludeReceipts { receipts = sds.BlockChain.GetReceiptsByHash(block.Hash()) } + + t := time.Now() tx, err = sds.indexer.PushBlock(block, receipts, totalDifficulty) if err != nil { return err @@ -842,9 +844,12 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p BlockHash: block.Hash(), BlockNumber: block.Number(), }, params, nodeSink, ipldSink) + if err != nil { + return err + } - // TODO this anti-pattern needs to be sorted out eventually - if err = tx.Submit(err); err != nil { + metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(time.Since(t)) + if err = tx.Submit(); err != nil { return fmt.Errorf("batch transaction submission failed: %w", err) } return nil diff --git a/test_helpers/mocks/indexer.go b/test_helpers/mocks/indexer.go index 8991f9e..403b553 100644 --- a/test_helpers/mocks/indexer.go +++ b/test_helpers/mocks/indexer.go @@ -17,6 +17,7 @@ package mocks import ( + context "context" "math/big" "time" @@ -84,10 +85,21 @@ func (sdi *StateDiffIndexer) ClearWatchedAddresses() error { return nil } +func (sdi *StateDiffIndexer) BeginTx(number *big.Int, ctx context.Context) interfaces.Batch { + return &batch{} +} + func (sdi *StateDiffIndexer) Close() error { return nil } -func (tx *batch) Submit(err error) error { +func (tx *batch) RollbackOnFailure(error) {} + +func (tx *batch) Submit() error { return nil } + +// batch.BlockNumber +func (tx *batch) BlockNumber() string { + return "0" +} -- 2.45.2 From 1781675b393e44c01aaf4923b39e5aa9912b9a60 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 27 Aug 2023 01:15:58 +0800 Subject: [PATCH 03/22] expose symm diff iterator state in companion object --- utils/iterator.go | 68 ++++++++++++++++++++++++++++------------------- 1 file changed, 40 insertions(+), 28 deletions(-) diff --git a/utils/iterator.go b/utils/iterator.go index d02cff7..06f251e 100644 --- a/utils/iterator.go +++ b/utils/iterator.go @@ -8,23 +8,8 @@ import ( ) type symmDiffIterator struct { - a, b iterState // Nodes returned are those in b - a and a - b (keys only) - yieldFromA bool // Whether next node comes from a - count int // Number of nodes scanned on either trie - eqPathIndex int // Count index of last pair of equal paths, to detect an updated key -} - -// NewSymmetricDifferenceIterator constructs a trie.NodeIterator that iterates over the symmetric difference -// of elements in a and b, i.e., the elements in a that are not in b, and vice versa. -// Returns the iterator, and a pointer to an integer recording the number of nodes seen. -func NewSymmetricDifferenceIterator(a, b trie.NodeIterator) (*symmDiffIterator, *int) { - it := &symmDiffIterator{ - a: iterState{a, true}, - b: iterState{b, true}, - // common paths are detected by a distance <=1 from this index, so put it out of reach - eqPathIndex: -2, - } - return it, &it.count + a, b iterState // Nodes returned are those in b - a and a - b (keys only) + SymmDiffAux } // pairs an iterator with a cache of its valid status @@ -33,11 +18,49 @@ type iterState struct { valid bool } +// SymmDiffAux exposes state specific to symmetric difference iteration, which is not accessible +// from the NodeIterator interface. This includes the number of nodes seen, whether the current key +// is common to both A and B, and whether the current node is sourced from A or B. +type SymmDiffAux struct { + yieldFromA bool // Whether next node comes from a + count int // Number of nodes scanned on either trie + eqPathIndex int // Count index of last pair of equal paths, to detect an updated key +} + +// NewSymmetricDifferenceIterator constructs a trie.NodeIterator that iterates over the symmetric difference +// of elements in a and b, i.e., the elements in a that are not in b, and vice versa. +// Returns the iterator, and a pointer to an auxiliary object for accessing the state not exposed by the NodeIterator interface recording the number of nodes seen. +func NewSymmetricDifferenceIterator(a, b trie.NodeIterator) (trie.NodeIterator, *SymmDiffAux) { + it := &symmDiffIterator{ + a: iterState{a, true}, + b: iterState{b, true}, + // common paths are detected by a distance <=1 between count and this index, so we start at -2 + SymmDiffAux: SymmDiffAux{eqPathIndex: -2}, + } + return it, &it.SymmDiffAux +} + func (st *iterState) Next(descend bool) bool { st.valid = st.NodeIterator.Next(descend) return st.valid } +// FromA returns true if the current node is sourced from A. +func (it *SymmDiffAux) FromA() bool { + return it.yieldFromA +} + +// CommonPath returns true if a node with the current path exists in each sub-iterator - i.e. it +// represents an updated node. +func (it *SymmDiffAux) CommonPath() bool { + return it.count-it.eqPathIndex <= 1 +} + +// Count returns the number of nodes seen. +func (it *SymmDiffAux) Count() int { + return it.count +} + func (it *symmDiffIterator) curr() *iterState { if it.yieldFromA { return &it.a @@ -45,17 +68,6 @@ func (it *symmDiffIterator) curr() *iterState { return &it.b } -// FromA returns true if the current node is sourced from A. -func (it *symmDiffIterator) FromA() bool { - return it.yieldFromA -} - -// CommonPath returns true if a node with the current path exists in each sub-iterator - i.e. it -// represents an updated node. -func (it *symmDiffIterator) CommonPath() bool { - return it.count-it.eqPathIndex <= 1 -} - func (it *symmDiffIterator) Hash() common.Hash { return it.curr().Hash() } -- 2.45.2 From 1e3daaeaee739e9a1c09691744d6721bee9e18eb Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 27 Aug 2023 01:20:29 +0800 Subject: [PATCH 04/22] WriteStateDiffTracked --- builder.go | 119 ++++++++++++++++++++++++++++++++++++++++++----------- service.go | 1 + 2 files changed, 95 insertions(+), 25 deletions(-) diff --git a/builder.go b/builder.go index fc5aed3..4a896d0 100644 --- a/builder.go +++ b/builder.go @@ -27,6 +27,7 @@ import ( "time" iterutils "github.com/cerc-io/eth-iterator-utils" + "github.com/cerc-io/eth-iterator-utils/tracker" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" @@ -58,7 +59,7 @@ type Builder interface { WriteStateDiff(Args, Params, sdtypes.StateNodeSink, sdtypes.IPLDSink) error } -type StateDiffBuilder struct { +type builder struct { // state cache is safe for concurrent reads stateCache adapt.StateView subtrieWorkers uint @@ -88,8 +89,8 @@ func syncedAppender[T any](to *[]T) func(T) error { } // NewBuilder is used to create a statediff builder -func NewBuilder(stateCache adapt.StateView) *StateDiffBuilder { - return &StateDiffBuilder{ +func NewBuilder(stateCache adapt.StateView) Builder { + return &builder{ stateCache: stateCache, subtrieWorkers: defaultSubtrieWorkers, } @@ -97,7 +98,7 @@ func NewBuilder(stateCache adapt.StateView) *StateDiffBuilder { // SetSubtrieWorkers sets the number of disjoint subtries to divide among parallel workers. // Passing 0 will reset this to the default value. -func (sdb *StateDiffBuilder) SetSubtrieWorkers(n uint) { +func (sdb *builder) SetSubtrieWorkers(n uint) { if n == 0 { n = defaultSubtrieWorkers } @@ -105,7 +106,7 @@ func (sdb *StateDiffBuilder) SetSubtrieWorkers(n uint) { } // BuildStateDiffObject builds a statediff object from two blocks and the provided parameters -func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (sdtypes.StateObject, error) { +func (sdb *builder) BuildStateDiffObject(args Args, params Params) (sdtypes.StateObject, error) { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildStateDiffObjectTimer) var stateNodes []sdtypes.StateLeafNode var iplds []sdtypes.IPLD @@ -122,7 +123,7 @@ func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (sdt } // WriteStateDiff writes a statediff object to output sinks -func (sdb *StateDiffBuilder) WriteStateDiff( +func (sdb *builder) WriteStateDiff( args Args, params Params, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, @@ -141,14 +142,82 @@ func (sdb *StateDiffBuilder) WriteStateDiff( subitersB := iterutils.SubtrieIterators(trieb.NodeIterator, uint(sdb.subtrieWorkers)) logger := log.New("hash", args.BlockHash, "number", args.BlockNumber) - // errgroup will cancel if any gr fails + // errgroup will cancel if any group fails g, ctx := errgroup.WithContext(context.Background()) for i := uint(0); i < sdb.subtrieWorkers; i++ { func(subdiv uint) { g.Go(func() error { a, b := subitersA[subdiv], subitersB[subdiv] + it, aux := utils.NewSymmetricDifferenceIterator(a, b) return sdb.processAccounts(ctx, - a, b, params.watchedAddressesLeafPaths, + it, aux, + params.watchedAddressesLeafPaths, + nodeSink, ipldSink, logger, + ) + }) + }(i) + } + return g.Wait() +} + +// WriteStateDiff writes a statediff object to output sinks +func (sdb *builder) WriteStateDiffTracked( + args Args, params Params, + nodeSink sdtypes.StateNodeSink, + ipldSink sdtypes.IPLDSink, + tracker *tracker.Tracker, +) error { + defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.WriteStateDiffTimer) + // Load tries for old and new states + triea, err := sdb.stateCache.OpenTrie(args.OldStateRoot) + if err != nil { + return fmt.Errorf("error opening old state trie: %w", err) + } + trieb, err := sdb.stateCache.OpenTrie(args.NewStateRoot) + if err != nil { + return fmt.Errorf("error opening new state trie: %w", err) + } + + var subiters []trie.NodeIterator + var auxes []*utils.SymmDiffAux + // Constructor for difference iterator at a specific (recovered) path + makeIterator := func(key []byte) trie.NodeIterator { + a := triea.NodeIterator(key) + b := trieb.NodeIterator(key) + diffit, aux := utils.NewSymmetricDifferenceIterator(a, b) + // iterators are constructed in-order, so these will align + auxes = append(auxes, aux) + return diffit + } + subiters, err = tracker.Restore(makeIterator) + if err != nil { + return fmt.Errorf("error restoring iterators: %w", err) + } + + if subiters != nil { + if len(subiters) != int(sdb.subtrieWorkers) { + return fmt.Errorf("expected to restore %d iterators, got %d", sdb.subtrieWorkers, len(subiters)) + } + } else { + subitersA := iterutils.SubtrieIterators(triea.NodeIterator, uint(sdb.subtrieWorkers)) + subitersB := iterutils.SubtrieIterators(trieb.NodeIterator, uint(sdb.subtrieWorkers)) + for i := 0; i < int(sdb.subtrieWorkers); i++ { + it, aux := utils.NewSymmetricDifferenceIterator(subitersA[i], subitersB[i]) + it = tracker.Tracked(it) + subiters = append(subiters, it) + auxes = append(auxes, aux) + } + } + logger := log.New("hash", args.BlockHash, "number", args.BlockNumber) + // errgroup will cancel if any group fails + g, ctx := errgroup.WithContext(context.Background()) + for i := uint(0); i < sdb.subtrieWorkers; i++ { + func(subdiv uint) { + g.Go(func() error { + // a, b := subitersA[subdiv], subitersB[subdiv] + return sdb.processAccounts(ctx, + subiters[subdiv], auxes[subdiv], + params.watchedAddressesLeafPaths, nodeSink, ipldSink, logger, ) }) @@ -159,9 +228,10 @@ func (sdb *StateDiffBuilder) WriteStateDiff( // processAccounts processes account creations and deletions, and returns a set of updated // existing accounts, indexed by leaf key. -func (sdb *StateDiffBuilder) processAccounts( +func (sdb *builder) processAccounts( ctx context.Context, - a, b trie.NodeIterator, watchedAddressesLeafPaths [][]byte, + it trie.NodeIterator, aux *utils.SymmDiffAux, + watchedAddressesLeafPaths [][]byte, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, logger log.Logger, ) error { @@ -172,7 +242,6 @@ func (sdb *StateDiffBuilder) processAccounts( updates := make(accountUpdateMap) // Cache the RLP of the previous node. When we hit a value node this will be the parent blob. var prevBlob []byte - it, itCount := utils.NewSymmetricDifferenceIterator(a, b) prevBlob = it.NodeBlob() for it.Next(true) { select { @@ -185,7 +254,7 @@ func (sdb *StateDiffBuilder) processAccounts( if !isWatchedPathPrefix(watchedAddressesLeafPaths, it.Path()) { continue } - if it.FromA() { // Node exists in the old trie + if aux.FromA() { // Node exists in the old trie if it.Leaf() { var account types.StateAccount if err := rlp.DecodeBytes(it.LeafBlob(), &account); err != nil { @@ -194,7 +263,7 @@ func (sdb *StateDiffBuilder) processAccounts( leafKey := make([]byte, len(it.LeafKey())) copy(leafKey, it.LeafKey()) - if it.CommonPath() { + if aux.CommonPath() { // If B also contains this leaf node, this is the old state of an updated account. if update, ok := updates[string(leafKey)]; ok { update.oldRoot = account.Root @@ -219,7 +288,7 @@ func (sdb *StateDiffBuilder) processAccounts( return err } - if it.CommonPath() { + if aux.CommonPath() { // If A also contains this leaf node, this is the new state of an updated account. if update, ok := updates[string(accountW.LeafKey)]; ok { update.new = *accountW @@ -288,11 +357,11 @@ func (sdb *StateDiffBuilder) processAccounts( } } - metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(*itCount)) + metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(aux.Count())) return it.Error() } -func (sdb *StateDiffBuilder) processAccountDeletion( +func (sdb *builder) processAccountDeletion( leafKey []byte, account types.StateAccount, nodeSink sdtypes.StateNodeSink, ) error { diff := sdtypes.StateLeafNode{ @@ -309,7 +378,7 @@ func (sdb *StateDiffBuilder) processAccountDeletion( return nodeSink(diff) } -func (sdb *StateDiffBuilder) processAccountCreation( +func (sdb *builder) processAccountCreation( accountW *sdtypes.AccountWrapper, ipldSink sdtypes.IPLDSink, nodeSink sdtypes.StateNodeSink, ) error { diff := sdtypes.StateLeafNode{ @@ -340,7 +409,7 @@ func (sdb *StateDiffBuilder) processAccountCreation( // decodes account at leaf and encodes RLP data to CID // reminder: it.Leaf() == true when the iterator is positioned at a "value node" (which is not something // that actually exists in an MMPT), therefore we pass the parent node blob as the leaf RLP. -func (sdb *StateDiffBuilder) decodeStateLeaf(it trie.NodeIterator, parentBlob []byte) (*sdtypes.AccountWrapper, error) { +func (sdb *builder) decodeStateLeaf(it trie.NodeIterator, parentBlob []byte) (*sdtypes.AccountWrapper, error) { var account types.StateAccount if err := rlp.DecodeBytes(it.LeafBlob(), &account); err != nil { return nil, fmt.Errorf("error decoding account at leaf key %x: %w", it.LeafKey(), err) @@ -357,7 +426,7 @@ func (sdb *StateDiffBuilder) decodeStateLeaf(it trie.NodeIterator, parentBlob [] // processStorageCreations processes the storage node records for a newly created account // i.e. it returns all the storage nodes at this state, since there is no previous state. -func (sdb *StateDiffBuilder) processStorageCreations( +func (sdb *builder) processStorageCreations( sr common.Hash, storageSink sdtypes.StorageNodeSink, ipldSink sdtypes.IPLDSink, ) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.ProcessStorageCreationsTimer) @@ -395,7 +464,7 @@ func (sdb *StateDiffBuilder) processStorageCreations( } // processStorageUpdates builds the storage diff node objects for all nodes that exist in a different state at B than A -func (sdb *StateDiffBuilder) processStorageUpdates( +func (sdb *builder) processStorageUpdates( oldroot common.Hash, newroot common.Hash, storageSink sdtypes.StorageNodeSink, ipldSink sdtypes.IPLDSink, @@ -416,10 +485,10 @@ func (sdb *StateDiffBuilder) processStorageUpdates( var prevBlob []byte a, b := oldTrie.NodeIterator(nil), newTrie.NodeIterator(nil) - it, _ := utils.NewSymmetricDifferenceIterator(a, b) + it, aux := utils.NewSymmetricDifferenceIterator(a, b) for it.Next(true) { - if it.FromA() { - if it.Leaf() && !it.CommonPath() { + if aux.FromA() { + if it.Leaf() && !aux.CommonPath() { // If this node's leaf key is absent from B, the storage slot was vacated. // In that case, emit an empty "removed" storage node record. if err := storageSink(sdtypes.StorageLeafNode{ @@ -457,7 +526,7 @@ func (sdb *StateDiffBuilder) processStorageUpdates( } // processRemovedAccountStorage builds the "removed" diffs for all the storage nodes for a destroyed account -func (sdb *StateDiffBuilder) processRemovedAccountStorage( +func (sdb *builder) processRemovedAccountStorage( sr common.Hash, storageSink sdtypes.StorageNodeSink, ) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.ProcessRemovedAccountStorageTimer) @@ -491,7 +560,7 @@ func (sdb *StateDiffBuilder) processRemovedAccountStorage( // decodes slot at leaf and encodes RLP data to CID // reminder: it.Leaf() == true when the iterator is positioned at a "value node" (which is not something // that actually exists in an MMPT), therefore we pass the parent node blob as the leaf RLP. -func (sdb *StateDiffBuilder) decodeStorageLeaf(it trie.NodeIterator, parentBlob []byte) sdtypes.StorageLeafNode { +func (sdb *builder) decodeStorageLeaf(it trie.NodeIterator, parentBlob []byte) sdtypes.StorageLeafNode { leafKey := make([]byte, len(it.LeafKey())) copy(leafKey, it.LeafKey()) value := make([]byte, len(it.LeafBlob())) diff --git a/service.go b/service.go index 604b2a6..6a15311 100644 --- a/service.go +++ b/service.go @@ -824,6 +824,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p return err } + // TODO: review/remove the need to sync here var nodeMtx, ipldMtx sync.Mutex nodeSink := func(node types2.StateLeafNode) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.OutputTimer) -- 2.45.2 From f528cfdf714e1256be0d7159bc6e0af3aca78612 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 27 Aug 2023 11:57:26 +0800 Subject: [PATCH 05/22] comments & formatting --- config.go | 6 +++--- indexer/constructor.go | 11 ++++++++++- main/main.go | 8 ++++++-- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/config.go b/config.go index ac16f74..960b44a 100644 --- a/config.go +++ b/config.go @@ -48,16 +48,16 @@ type Config struct { SubtrieWorkers uint // Should the statediff service wait until geth has synced to the head of the blockchain? WaitForSync bool - // Context used during DB initialization + // Context passed to all DB method calls Context context.Context } // Params contains config parameters for the state diff builder type Params struct { - IncludeBlock bool + IncludeBlock bool // TODO: not used in write-requests IncludeReceipts bool IncludeTD bool - IncludeCode bool + IncludeCode bool // TODO: not used by anything? WatchedAddresses []common.Address watchedAddressesLeafPaths [][]byte } diff --git a/indexer/constructor.go b/indexer/constructor.go index c44ab64..bb288fd 100644 --- a/indexer/constructor.go +++ b/indexer/constructor.go @@ -33,7 +33,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) (sql.Database, interfaces.StateDiffIndexer, error) { +func NewStateDiffIndexer( + ctx context.Context, + chainConfig *params.ChainConfig, + nodeInfo node.Info, + config interfaces.Config, +) ( + sql.Database, + interfaces.StateDiffIndexer, + error, +) { switch config.Type() { case shared.FILE: log.Info("Starting statediff service in SQL file writing mode") diff --git a/main/main.go b/main/main.go index 0ab262f..20f9aca 100644 --- a/main/main.go +++ b/main/main.go @@ -47,8 +47,12 @@ func InitializeNode(stack core.Node, b core.Backend) { ClientName: serviceConfig.ClientName, } var err error - _, indexer, err = ind.NewStateDiffIndexer(serviceConfig.Context, - adapt.ChainConfig(backend.ChainConfig()), info, serviceConfig.IndexerConfig) + _, indexer, err = ind.NewStateDiffIndexer( + serviceConfig.Context, + adapt.ChainConfig(backend.ChainConfig()), + info, + serviceConfig.IndexerConfig, + ) if err != nil { log.Error("failed to construct indexer", "error", err) } -- 2.45.2 From 1f477f24ea2a1ffa8f0ed425fcc2340228b3eb6b Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Sun, 27 Aug 2023 11:59:40 +0800 Subject: [PATCH 06/22] rm redundant node info --- indexer/constructor.go | 3 +-- indexer/database/file/config.go | 12 ------------ indexer/database/file/csv_indexer_legacy_test.go | 2 +- indexer/database/file/csv_indexer_test.go | 2 +- indexer/database/file/indexer.go | 7 ++++--- indexer/database/file/mainnet_tests/indexer_test.go | 2 +- indexer/database/file/sql_indexer_legacy_test.go | 2 +- indexer/database/file/sql_indexer_test.go | 2 +- indexer/database/sql/postgres/config.go | 4 ---- indexer/test/test_legacy.go | 9 +++++++++ main/flags.go | 2 -- 11 files changed, 19 insertions(+), 28 deletions(-) diff --git a/indexer/constructor.go b/indexer/constructor.go index bb288fd..13f6d65 100644 --- a/indexer/constructor.go +++ b/indexer/constructor.go @@ -50,8 +50,7 @@ func NewStateDiffIndexer( if !ok { return nil, nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{}) } - fc.NodeInfo = nodeInfo - ind, err := file.NewStateDiffIndexer(chainConfig, fc) + ind, err := file.NewStateDiffIndexer(chainConfig, fc, nodeInfo) return nil, ind, err case shared.POSTGRES: log.Info("Starting statediff service in Postgres writing mode") diff --git a/indexer/database/file/config.go b/indexer/database/file/config.go index fc8fd8c..9c01327 100644 --- a/indexer/database/file/config.go +++ b/indexer/database/file/config.go @@ -20,7 +20,6 @@ import ( "fmt" "strings" - "github.com/cerc-io/plugeth-statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/shared" ) @@ -30,7 +29,6 @@ type Config struct { OutputDir string FilePath string WatchedAddressesFilePath string - NodeInfo node.Info } // FileMode to explicitly type the mode of file writer we are using @@ -70,20 +68,11 @@ func (c Config) Type() shared.DBType { return shared.FILE } -var nodeInfo = node.Info{ - GenesisBlock: "0xd4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3", - NetworkID: "1", - ChainID: 1, - ID: "mockNodeID", - ClientName: "go-ethereum", -} - // CSVTestConfig config for unit tests var CSVTestConfig = Config{ Mode: CSV, OutputDir: "./statediffing_test", WatchedAddressesFilePath: "./statediffing_watched_addresses_test_file.csv", - NodeInfo: nodeInfo, } // SQLTestConfig config for unit tests @@ -91,5 +80,4 @@ var SQLTestConfig = Config{ Mode: SQL, FilePath: "./statediffing_test_file.sql", WatchedAddressesFilePath: "./statediffing_watched_addresses_test_file.sql", - NodeInfo: nodeInfo, } diff --git a/indexer/database/file/csv_indexer_legacy_test.go b/indexer/database/file/csv_indexer_legacy_test.go index c117f75..238423b 100644 --- a/indexer/database/file/csv_indexer_legacy_test.go +++ b/indexer/database/file/csv_indexer_legacy_test.go @@ -43,7 +43,7 @@ func setupLegacyCSVIndexer(t *testing.T) { require.NoError(t, err) } - ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.CSVTestConfig) + ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.CSVTestConfig, test.LegacyNodeInfo) require.NoError(t, err) db, err = postgres.SetupSQLXDB() diff --git a/indexer/database/file/csv_indexer_test.go b/indexer/database/file/csv_indexer_test.go index 4e6526f..06aa366 100644 --- a/indexer/database/file/csv_indexer_test.go +++ b/indexer/database/file/csv_indexer_test.go @@ -41,7 +41,7 @@ func setupCSVIndexer(t *testing.T) { require.NoError(t, err) } - ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.CSVTestConfig) + ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.CSVTestConfig, test.LegacyNodeInfo) require.NoError(t, err) db, err = postgres.SetupSQLXDB() diff --git a/indexer/database/file/indexer.go b/indexer/database/file/indexer.go index b213f91..ce56acb 100644 --- a/indexer/database/file/indexer.go +++ b/indexer/database/file/indexer.go @@ -38,6 +38,7 @@ import ( "github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/models" + "github.com/cerc-io/plugeth-statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/shared" sdtypes "github.com/cerc-io/plugeth-statediff/types" "github.com/cerc-io/plugeth-statediff/utils/log" @@ -62,7 +63,7 @@ type StateDiffIndexer struct { } // NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer -func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config) (*StateDiffIndexer, error) { +func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config, nodeInfo node.Info) (*StateDiffIndexer, error) { var err error var writer FileWriter @@ -115,12 +116,12 @@ func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config) (*State wg := new(sync.WaitGroup) writer.Loop() - writer.upsertNode(config.NodeInfo) + writer.upsertNode(nodeInfo) return &StateDiffIndexer{ fileWriter: writer, chainConfig: chainConfig, - nodeID: config.NodeInfo.ID, + nodeID: nodeInfo.ID, wg: wg, }, nil } diff --git a/indexer/database/file/mainnet_tests/indexer_test.go b/indexer/database/file/mainnet_tests/indexer_test.go index 3fe53a0..cdb8042 100644 --- a/indexer/database/file/mainnet_tests/indexer_test.go +++ b/indexer/database/file/mainnet_tests/indexer_test.go @@ -83,7 +83,7 @@ func setupMainnetIndexer(t *testing.T) { require.NoError(t, err) } - ind, err = file.NewStateDiffIndexer(chainConf, file.CSVTestConfig) + ind, err = file.NewStateDiffIndexer(chainConf, file.CSVTestConfig, test.LegacyNodeInfo) require.NoError(t, err) db, err = postgres.SetupSQLXDB() diff --git a/indexer/database/file/sql_indexer_legacy_test.go b/indexer/database/file/sql_indexer_legacy_test.go index b46348a..6d721d5 100644 --- a/indexer/database/file/sql_indexer_legacy_test.go +++ b/indexer/database/file/sql_indexer_legacy_test.go @@ -44,7 +44,7 @@ func setupLegacySQLIndexer(t *testing.T) { require.NoError(t, err) } - ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.SQLTestConfig) + ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.SQLTestConfig, test.LegacyNodeInfo) require.NoError(t, err) db, err = postgres.SetupSQLXDB() diff --git a/indexer/database/file/sql_indexer_test.go b/indexer/database/file/sql_indexer_test.go index 8663f27..3aab7b8 100644 --- a/indexer/database/file/sql_indexer_test.go +++ b/indexer/database/file/sql_indexer_test.go @@ -41,7 +41,7 @@ func setupIndexer(t *testing.T) { require.NoError(t, err) } - ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.SQLTestConfig) + ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.SQLTestConfig, test.LegacyNodeInfo) require.NoError(t, err) db, err = postgres.SetupSQLXDB() diff --git a/indexer/database/sql/postgres/config.go b/indexer/database/sql/postgres/config.go index a2c63b4..cb26e53 100644 --- a/indexer/database/sql/postgres/config.go +++ b/indexer/database/sql/postgres/config.go @@ -44,10 +44,6 @@ type Config struct { ConnTimeout time.Duration LogStatements bool - // node info params - ID string - ClientName string - // driver type Driver DriverType diff --git a/indexer/test/test_legacy.go b/indexer/test/test_legacy.go index 8878edc..efe5664 100644 --- a/indexer/test/test_legacy.go +++ b/indexer/test/test_legacy.go @@ -24,6 +24,7 @@ import ( "github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/mocks" + "github.com/cerc-io/plugeth-statediff/indexer/node" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/params" "github.com/ipfs/go-cid" @@ -36,6 +37,14 @@ var ( legacyData = mocks.NewLegacyData(LegacyConfig) mockLegacyBlock *types.Block legacyHeaderCID cid.Cid + // Mainnet node info + LegacyNodeInfo = node.Info{ + GenesisBlock: "0xd4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3", + NetworkID: "1", + ChainID: 1, + ID: "mockNodeID", + ClientName: "go-ethereum", + } ) func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) { diff --git a/main/flags.go b/main/flags.go index c1f5c1a..27cd497 100644 --- a/main/flags.go +++ b/main/flags.go @@ -173,8 +173,6 @@ func initConfig() { case shared.FILE: indexerConfig = fileConfig case shared.POSTGRES: - dbConfig.ID = config.ID - dbConfig.ClientName = config.ClientName indexerConfig = dbConfig case shared.DUMP: switch dbDumpDst { -- 2.45.2 From 7c4f8432db4621a858ba8ec761ce0876fe52ed57 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 30 Aug 2023 11:09:53 +0800 Subject: [PATCH 07/22] tweaks --- builder.go | 99 ++++++++++++++--------------- indexer/database/file/indexer.go | 2 +- indexer/shared/schema/table_test.go | 3 +- types/types.go | 2 +- 4 files changed, 52 insertions(+), 54 deletions(-) diff --git a/builder.go b/builder.go index 4a896d0..f5658ec 100644 --- a/builder.go +++ b/builder.go @@ -89,7 +89,7 @@ func syncedAppender[T any](to *[]T) func(T) error { } // NewBuilder is used to create a statediff builder -func NewBuilder(stateCache adapt.StateView) Builder { +func NewBuilder(stateCache adapt.StateView) *builder { return &builder{ stateCache: stateCache, subtrieWorkers: defaultSubtrieWorkers, @@ -195,39 +195,36 @@ func (sdb *builder) WriteStateDiffTracked( } if subiters != nil { - if len(subiters) != int(sdb.subtrieWorkers) { - return fmt.Errorf("expected to restore %d iterators, got %d", sdb.subtrieWorkers, len(subiters)) + // Completed iterators are not saved by the tracker, so restoring fewer than configured is ok, + // but having too many is not expected. + if len(subiters) > int(sdb.subtrieWorkers) { + return fmt.Errorf("restored too many iterators: expected %d, got %d", + sdb.subtrieWorkers, len(subiters)) } } else { - subitersA := iterutils.SubtrieIterators(triea.NodeIterator, uint(sdb.subtrieWorkers)) - subitersB := iterutils.SubtrieIterators(trieb.NodeIterator, uint(sdb.subtrieWorkers)) - for i := 0; i < int(sdb.subtrieWorkers); i++ { - it, aux := utils.NewSymmetricDifferenceIterator(subitersA[i], subitersB[i]) - it = tracker.Tracked(it) - subiters = append(subiters, it) - auxes = append(auxes, aux) + subiters = iterutils.SubtrieIterators(makeIterator, uint(sdb.subtrieWorkers)) + for i := range subiters { + subiters[i] = tracker.Tracked(subiters[i]) } } logger := log.New("hash", args.BlockHash, "number", args.BlockNumber) // errgroup will cancel if any group fails g, ctx := errgroup.WithContext(context.Background()) - for i := uint(0); i < sdb.subtrieWorkers; i++ { + for i := range subiters { func(subdiv uint) { g.Go(func() error { - // a, b := subitersA[subdiv], subitersB[subdiv] return sdb.processAccounts(ctx, subiters[subdiv], auxes[subdiv], params.watchedAddressesLeafPaths, nodeSink, ipldSink, logger, ) }) - }(i) + }(uint(i)) } return g.Wait() } -// processAccounts processes account creations and deletions, and returns a set of updated -// existing accounts, indexed by leaf key. +// processAccounts processes account creations, deletions, and updates func (sdb *builder) processAccounts( ctx context.Context, it trie.NodeIterator, aux *utils.SymmDiffAux, @@ -241,8 +238,7 @@ func (sdb *builder) processAccounts( updates := make(accountUpdateMap) // Cache the RLP of the previous node. When we hit a value node this will be the parent blob. - var prevBlob []byte - prevBlob = it.NodeBlob() + var prevBlob = it.NodeBlob() for it.Next(true) { select { case <-ctx.Done(): @@ -281,7 +277,7 @@ func (sdb *builder) processAccounts( } continue } - // Node exists in the new trie + // Node exists in the new trie (B) if it.Leaf() { accountW, err := sdb.decodeStateLeaf(it, prevBlob) if err != nil { @@ -301,42 +297,43 @@ func (sdb *builder) processAccounts( return err } } - } else { - // New trie nodes will be written to blockstore only. - // Reminder: this includes leaf nodes, since the geth iterator.Leaf() actually - // signifies a "value" node. - if it.Hash() == zeroHash { - continue - } - // TODO - this can be handled when value node is (craeted?) - nodeVal := make([]byte, len(it.NodeBlob())) - copy(nodeVal, it.NodeBlob()) - // if doing a selective diff, we need to ensure this is a watched path - if len(watchedAddressesLeafPaths) > 0 { - var elements []interface{} - if err := rlp.DecodeBytes(nodeVal, &elements); err != nil { - return err - } - ok, err := isLeaf(elements) - if err != nil { - return err - } - if ok { - partialPath := utils.CompactToHex(elements[0].([]byte)) - valueNodePath := append(it.Path(), partialPath...) - if !isWatchedPath(watchedAddressesLeafPaths, valueNodePath) { - continue - } - } - } - if err := ipldSink(sdtypes.IPLD{ - CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, it.Hash().Bytes()).String(), - Content: nodeVal, - }); err != nil { + continue + } + // New inner trie nodes will be written to blockstore only. + // Reminder: this includes leaf nodes, since the geth iterator.Leaf() actually + // signifies a "value" node. + + // TODO: A zero hash indicates what? + if it.Hash() == zeroHash { + continue + } + nodeVal := make([]byte, len(it.NodeBlob())) + copy(nodeVal, it.NodeBlob()) + // if doing a selective diff, we need to ensure this is a watched path + if len(watchedAddressesLeafPaths) > 0 { + var elements []interface{} + if err := rlp.DecodeBytes(nodeVal, &elements); err != nil { return err } - prevBlob = nodeVal + ok, err := isLeaf(elements) + if err != nil { + return err + } + if ok { + partialPath := utils.CompactToHex(elements[0].([]byte)) + valueNodePath := append(it.Path(), partialPath...) + if !isWatchedPath(watchedAddressesLeafPaths, valueNodePath) { + continue + } + } } + if err := ipldSink(sdtypes.IPLD{ + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, it.Hash().Bytes()).String(), + Content: nodeVal, + }); err != nil { + return err + } + prevBlob = nodeVal } for key, update := range updates { diff --git a/indexer/database/file/indexer.go b/indexer/database/file/indexer.go index ce56acb..56719b8 100644 --- a/indexer/database/file/indexer.go +++ b/indexer/database/file/indexer.go @@ -206,7 +206,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip return blockTx, err } -// processHeader write a header IPLD insert SQL stmt to a file +// PushHeader write a header IPLD insert SQL stmt to a file // it returns the headerID func (sdi *StateDiffIndexer) PushHeader(_ interfaces.Batch, header *types.Header, reward, td *big.Int) (string, error) { // Process the header diff --git a/indexer/shared/schema/table_test.go b/indexer/shared/schema/table_test.go index 692a839..aaa026b 100644 --- a/indexer/shared/schema/table_test.go +++ b/indexer/shared/schema/table_test.go @@ -43,7 +43,8 @@ var testHeaderTable = Table{ "mh_key", "times_validated", "coinbase", - )} + ), +} func TestTable(t *testing.T) { headerUpsert := `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, block_number) 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, $15, $16)` diff --git a/types/types.go b/types/types.go index 9f3f60b..9d92fd2 100644 --- a/types/types.go +++ b/types/types.go @@ -53,7 +53,7 @@ type StateLeafNode struct { StorageDiff []StorageLeafNode } -// StorageLeafNode holds the data for a single storage diff node leaf node +// StorageLeafNode holds the data for a single storage diff leaf node type StorageLeafNode struct { Removed bool Value []byte -- 2.45.2 From 026bd8163680be28cb1ec79b391a79069394126a Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 31 Aug 2023 00:24:12 +0800 Subject: [PATCH 08/22] re-export indexer types for consuming libs --- indexer/reexport.go | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 indexer/reexport.go diff --git a/indexer/reexport.go b/indexer/reexport.go new file mode 100644 index 0000000..49d46e4 --- /dev/null +++ b/indexer/reexport.go @@ -0,0 +1,7 @@ +package indexer + +import "github.com/cerc-io/plugeth-statediff/indexer/interfaces" + +type Indexer = interfaces.StateDiffIndexer +type Batch = interfaces.Batch +type Config = interfaces.Config -- 2.45.2 From e6d312f98b31977d2893d3917a6109c98e7a70cd Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 6 Sep 2023 19:32:11 +0800 Subject: [PATCH 09/22] fix symm diff iterator tests --- utils/iterator_test.go | 52 +++++++++++++++++++----------------------- 1 file changed, 24 insertions(+), 28 deletions(-) diff --git a/utils/iterator_test.go b/utils/iterator_test.go index e7210ac..6874a65 100644 --- a/utils/iterator_test.go +++ b/utils/iterator_test.go @@ -45,37 +45,33 @@ func TestSymmetricDifferenceIterator(t *testing.T) { t.Run("with no difference", func(t *testing.T) { db := trie.NewDatabase(rawdb.NewMemoryDatabase()) triea := trie.NewEmpty(db) - di, count := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) + di, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) for di.Next(true) { t.Errorf("iterator should not yield any elements") } - assert.Equal(t, 0, *count) + assert.Equal(t, 0, aux.Count()) triea.MustUpdate([]byte("foo"), []byte("bar")) - di, count = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) + di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) for di.Next(true) { t.Errorf("iterator should not yield any elements") } - assert.Equal(t, 2, *count) + // two nodes visited: the leaf (value) and its parent + assert.Equal(t, 2, aux.Count()) - // TODO will fail until fixed https://github.com/ethereum/go-ethereum/pull/27838 trieb := trie.NewEmpty(db) - di, count = utils.NewSymmetricDifferenceIterator( - triea.NodeIterator([]byte("jars")), - trieb.NodeIterator(nil)) + di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator([]byte("jars")), trieb.NodeIterator(nil)) for di.Next(true) { - t.Errorf("iterator should not yield any elements, but got key %s", di.Path()) + t.Errorf("iterator should not yield any elements") } - assert.Equal(t, 0, *count) + assert.Equal(t, 0, aux.Count()) - // // TODO will fail until merged: https://github.com/ethereum/go-ethereum/pull/27838 - // di, count = utils.NewSymmetricDifferenceIterator( - // triea.NodeIterator([]byte("food")), - // trieb.NodeIterator(nil)) + // TODO will fail until merged: https://github.com/ethereum/go-ethereum/pull/27838 + // di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator([]byte("food")), trieb.NodeIterator(nil)) // for di.Next(true) { - // t.Errorf("iterator should not yield any elements, but got key %s", di.Path()) + // t.Errorf("iterator should not yield any elements") // } - // assert.Equal(t, 0, *count) + // assert.Equal(t, 0, aux.Count()) }) t.Run("small difference", func(t *testing.T) { @@ -86,32 +82,32 @@ func TestSymmetricDifferenceIterator(t *testing.T) { trieb := trie.NewEmpty(dbb) trieb.MustUpdate([]byte("foo"), []byte("bar")) - di, count := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) + di, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) leaves := 0 for di.Next(true) { if di.Leaf() { - assert.False(t, di.CommonPath()) + assert.False(t, aux.CommonPath()) assert.Equal(t, "foo", string(di.LeafKey())) assert.Equal(t, "bar", string(di.LeafBlob())) leaves++ } } assert.Equal(t, 1, leaves) - assert.Equal(t, 2, *count) + assert.Equal(t, 2, aux.Count()) trieb.MustUpdate([]byte("quux"), []byte("bars")) - di, count = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator([]byte("quux"))) + di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator([]byte("quux"))) leaves = 0 for di.Next(true) { if di.Leaf() { - assert.False(t, di.CommonPath()) + assert.False(t, aux.CommonPath()) assert.Equal(t, "quux", string(di.LeafKey())) assert.Equal(t, "bars", string(di.LeafBlob())) leaves++ } } assert.Equal(t, 1, leaves) - assert.Equal(t, 1, *count) + assert.Equal(t, 1, aux.Count()) }) dba := trie.NewDatabase(rawdb.NewMemoryDatabase()) @@ -128,20 +124,20 @@ func TestSymmetricDifferenceIterator(t *testing.T) { onlyA := make(map[string]string) onlyB := make(map[string]string) var deletions, creations []string - it, _ := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) + it, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) for it.Next(true) { if !it.Leaf() { continue } key, value := string(it.LeafKey()), string(it.LeafBlob()) - if it.FromA() { + if aux.FromA() { onlyA[key] = value - if !it.CommonPath() { + if !aux.CommonPath() { deletions = append(deletions, key) } } else { onlyB[key] = value - if !it.CommonPath() { + if !aux.CommonPath() { creations = append(creations, key) } } @@ -209,10 +205,10 @@ func TestCompareDifferenceIterators(t *testing.T) { pathsA = append(pathsA, itAonly.Path()) } - itSym, _ := utils.NewSymmetricDifferenceIterator(treeA.NodeIterator(nil), treeB.NodeIterator(nil)) + itSym, aux := utils.NewSymmetricDifferenceIterator(treeA.NodeIterator(nil), treeB.NodeIterator(nil)) var idxA, idxB int for itSym.Next(true) { - if itSym.FromA() { + if aux.FromA() { require.Equal(t, pathsA[idxA], itSym.Path()) idxA++ } else { -- 2.45.2 From afd4c05d4a22e43654f883b6990400b73e7b9a36 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 22 Sep 2023 15:39:27 +0800 Subject: [PATCH 10/22] default log level = info --- utils/log/log.go | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/log/log.go b/utils/log/log.go index 2cdeb82..307d936 100644 --- a/utils/log/log.go +++ b/utils/log/log.go @@ -17,6 +17,7 @@ func init() { // The plugeth logger is only initialized with the geth runtime, // but tests expect to have a logger available, so default to this. DefaultLogger = TestLogger + TestLogger.SetLevel(int(log15.LvlInfo)) } func Trace(m string, a ...interface{}) { DefaultLogger.Trace(m, a...) } -- 2.45.2 From 52deae0528701438afc4552f3d14525d24e93dab Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 26 Sep 2023 02:50:10 +0800 Subject: [PATCH 11/22] update tracker --- builder.go | 6 +++--- go.mod | 4 ++-- go.sum | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/builder.go b/builder.go index f5658ec..af5a16e 100644 --- a/builder.go +++ b/builder.go @@ -165,7 +165,7 @@ func (sdb *builder) WriteStateDiffTracked( args Args, params Params, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, - tracker *tracker.Tracker, + tracker tracker.Tracker, ) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.WriteStateDiffTimer) // Load tries for old and new states @@ -194,9 +194,9 @@ func (sdb *builder) WriteStateDiffTracked( return fmt.Errorf("error restoring iterators: %w", err) } - if subiters != nil { + if len(subiters) != 0 { // Completed iterators are not saved by the tracker, so restoring fewer than configured is ok, - // but having too many is not expected. + // but having too many is a problem. if len(subiters) > int(sdb.subtrieWorkers) { return fmt.Errorf("restored too many iterators: expected %d, got %d", sdb.subtrieWorkers, len(subiters)) diff --git a/go.mod b/go.mod index ffb599b..7e2e6f1 100644 --- a/go.mod +++ b/go.mod @@ -124,8 +124,8 @@ require ( ) replace ( - github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.1 - github.com/cerc-io/eth-testing => git.vdb.to/cerc-io/eth-testing v0.2.1 + github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f + github.com/cerc-io/eth-testing => git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 github.com/ethereum/go-ethereum => git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 github.com/openrelayxyz/plugeth-utils => git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230706160122-cd41de354c46 ) diff --git a/go.sum b/go.sum index 0cd8482..74a0fe7 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.1 h1:AGen4U2GaYJVzPjEo3U+GPczSfOUEMkM1nWTM+cq5Dk= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.1/go.mod h1:uiocO9elfDe78kd3c/VZ2in26V+gyXJuN+sdTxK4Xag= -git.vdb.to/cerc-io/eth-testing v0.2.1 h1:IZAX7DVgzPkSmu1xdKZ5aOemdEYbvtgae7GUl/TUNtQ= -git.vdb.to/cerc-io/eth-testing v0.2.1/go.mod h1:qdvpc/W1xvf2MKx3rMOqvFvYaYIHG77Z1g0lwsmw0Uk= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f h1:sIuSkD6U7uYD/FGfvWOBViIuaHd+YhLM0Hln+4BQM10= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f/go.mod h1:Xv+d7Q11qGJcggcfxoj2JEvJJBKj0C66I6PyG5/lz9o= +git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 h1:fWAvsSiuDqveuxwnfc8psInfLZhMqHlQnmOpOHsd8Tk= +git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0/go.mod h1:qdvpc/W1xvf2MKx3rMOqvFvYaYIHG77Z1g0lwsmw0Uk= git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 h1:KLjxHwp9Zp7xhECccmJS00RiL+VwTuUGLU7qeIctg8g= git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1/go.mod h1:cYXZu70+6xmDgIgrTD81GPasv16piiAFJnKyAbwVPMU= git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230706160122-cd41de354c46 h1:KYcbbne/RXd7AuxbUd/3hgk1jPN+33k2CKiNsUsMCC0= -- 2.45.2 From 77e00278e49a561d041d6835aae6126acc1e54ce Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 26 Sep 2023 02:50:18 +0800 Subject: [PATCH 12/22] fix test --- indexer/ipld/eth_parser_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/ipld/eth_parser_test.go b/indexer/ipld/eth_parser_test.go index fd44058..8deb260 100644 --- a/indexer/ipld/eth_parser_test.go +++ b/indexer/ipld/eth_parser_test.go @@ -92,7 +92,7 @@ func loadBlockData(t *testing.T) []testCase { func TestFromBlockAndReceipts(t *testing.T) { testCases := loadBlockData(t) for _, tc := range testCases { - _, _, _, _, err := FromBlockAndReceipts(tc.block, tc.receipts) + _, _, _, err := FromBlockAndReceipts(tc.block, tc.receipts) if err != nil { t.Fatalf("error generating IPLDs from block and receipts, err %v, kind %s, block hash %s", err, tc.kind, tc.block.Hash()) } -- 2.45.2 From 5b51346a830292a076042f8986ebd54807f93d73 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 26 Sep 2023 18:06:45 +0800 Subject: [PATCH 13/22] add missing rollback, doc --- indexer/database/sql/batch_tx.go | 1 - indexer/interfaces/interfaces.go | 3 +++ service.go | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/indexer/database/sql/batch_tx.go b/indexer/database/sql/batch_tx.go index d42b214..3225aa7 100644 --- a/indexer/database/sql/batch_tx.go +++ b/indexer/database/sql/batch_tx.go @@ -85,7 +85,6 @@ func (tx *BatchTx) BlockNumber() string { } func (tx *BatchTx) RollbackOnFailure(err error) { - if p := recover(); p != nil { defer tx.close() log.Info("panic detected before tx submission, rolling back the tx", "panic", p) diff --git a/indexer/interfaces/interfaces.go b/indexer/interfaces/interfaces.go index 84f76d5..18dc735 100644 --- a/indexer/interfaces/interfaces.go +++ b/indexer/interfaces/interfaces.go @@ -54,8 +54,11 @@ type StateDiffIndexer interface { // Batch required for indexing data atomically type Batch interface { + // Submit commits the batch transaction Submit() error + // BlockNumber is the block number of the header this batch contains BlockNumber() string + // RollbackOnFailure rolls back the batch transaction if the error is not nil RollbackOnFailure(error) } diff --git a/service.go b/service.go index 6a15311..402b71c 100644 --- a/service.go +++ b/service.go @@ -823,6 +823,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p if err != nil { return err } + defer tx.RollbackOnFailure(err) // TODO: review/remove the need to sync here var nodeMtx, ipldMtx sync.Mutex -- 2.45.2 From 64d45bdfd8d65accd581b7e176195d3116c6cb5c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 26 Sep 2023 18:07:25 +0800 Subject: [PATCH 14/22] rework iterator state stuff --- builder.go | 35 ++++++++++++++---------------- go.mod | 2 +- go.sum | 4 ++-- utils/iterator.go | 48 +++++++++++++++++++++--------------------- utils/iterator_test.go | 38 ++++++++++++++++----------------- 5 files changed, 62 insertions(+), 65 deletions(-) diff --git a/builder.go b/builder.go index af5a16e..7839238 100644 --- a/builder.go +++ b/builder.go @@ -148,9 +148,9 @@ func (sdb *builder) WriteStateDiff( func(subdiv uint) { g.Go(func() error { a, b := subitersA[subdiv], subitersB[subdiv] - it, aux := utils.NewSymmetricDifferenceIterator(a, b) + it := utils.NewSymmetricDifferenceIterator(a, b) return sdb.processAccounts(ctx, - it, aux, + it, &it.SymmDiffState, params.watchedAddressesLeafPaths, nodeSink, ipldSink, logger, ) @@ -165,7 +165,7 @@ func (sdb *builder) WriteStateDiffTracked( args Args, params Params, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, - tracker tracker.Tracker, + tracker tracker.IteratorTracker, ) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.WriteStateDiffTimer) // Load tries for old and new states @@ -178,18 +178,14 @@ func (sdb *builder) WriteStateDiffTracked( return fmt.Errorf("error opening new state trie: %w", err) } - var subiters []trie.NodeIterator - var auxes []*utils.SymmDiffAux + var subiters, bases []trie.NodeIterator // Constructor for difference iterator at a specific (recovered) path makeIterator := func(key []byte) trie.NodeIterator { a := triea.NodeIterator(key) b := trieb.NodeIterator(key) - diffit, aux := utils.NewSymmetricDifferenceIterator(a, b) - // iterators are constructed in-order, so these will align - auxes = append(auxes, aux) - return diffit + return utils.NewSymmetricDifferenceIterator(a, b) } - subiters, err = tracker.Restore(makeIterator) + subiters, bases, err = tracker.Restore(makeIterator) if err != nil { return fmt.Errorf("error restoring iterators: %w", err) } @@ -214,7 +210,7 @@ func (sdb *builder) WriteStateDiffTracked( func(subdiv uint) { g.Go(func() error { return sdb.processAccounts(ctx, - subiters[subdiv], auxes[subdiv], + subiters[subdiv], &bases[subdiv].(*utils.SymmDiffIterator).SymmDiffState, params.watchedAddressesLeafPaths, nodeSink, ipldSink, logger, ) @@ -225,9 +221,10 @@ func (sdb *builder) WriteStateDiffTracked( } // processAccounts processes account creations, deletions, and updates +// the NodeIterator and SymmDiffIterator instances should refer to the same object, will only be used func (sdb *builder) processAccounts( ctx context.Context, - it trie.NodeIterator, aux *utils.SymmDiffAux, + it trie.NodeIterator, symdiff *utils.SymmDiffState, watchedAddressesLeafPaths [][]byte, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, logger log.Logger, @@ -250,7 +247,7 @@ func (sdb *builder) processAccounts( if !isWatchedPathPrefix(watchedAddressesLeafPaths, it.Path()) { continue } - if aux.FromA() { // Node exists in the old trie + if symdiff.FromA() { // Node exists in the old trie if it.Leaf() { var account types.StateAccount if err := rlp.DecodeBytes(it.LeafBlob(), &account); err != nil { @@ -259,7 +256,7 @@ func (sdb *builder) processAccounts( leafKey := make([]byte, len(it.LeafKey())) copy(leafKey, it.LeafKey()) - if aux.CommonPath() { + if symdiff.CommonPath() { // If B also contains this leaf node, this is the old state of an updated account. if update, ok := updates[string(leafKey)]; ok { update.oldRoot = account.Root @@ -284,7 +281,7 @@ func (sdb *builder) processAccounts( return err } - if aux.CommonPath() { + if symdiff.CommonPath() { // If A also contains this leaf node, this is the new state of an updated account. if update, ok := updates[string(accountW.LeafKey)]; ok { update.new = *accountW @@ -354,7 +351,7 @@ func (sdb *builder) processAccounts( } } - metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(aux.Count())) + metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(symdiff.Count())) return it.Error() } @@ -482,10 +479,10 @@ func (sdb *builder) processStorageUpdates( var prevBlob []byte a, b := oldTrie.NodeIterator(nil), newTrie.NodeIterator(nil) - it, aux := utils.NewSymmetricDifferenceIterator(a, b) + it := utils.NewSymmetricDifferenceIterator(a, b) for it.Next(true) { - if aux.FromA() { - if it.Leaf() && !aux.CommonPath() { + if it.FromA() { + if it.Leaf() && !it.CommonPath() { // If this node's leaf key is absent from B, the storage slot was vacated. // In that case, emit an empty "removed" storage node record. if err := storageSink(sdtypes.StorageLeafNode{ diff --git a/go.mod b/go.mod index 7e2e6f1..3e43964 100644 --- a/go.mod +++ b/go.mod @@ -124,7 +124,7 @@ require ( ) replace ( - github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f + github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643 github.com/cerc-io/eth-testing => git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 github.com/ethereum/go-ethereum => git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 github.com/openrelayxyz/plugeth-utils => git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230706160122-cd41de354c46 diff --git a/go.sum b/go.sum index 74a0fe7..c7ae382 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,6 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f h1:sIuSkD6U7uYD/FGfvWOBViIuaHd+YhLM0Hln+4BQM10= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230925184550-062eb329435f/go.mod h1:Xv+d7Q11qGJcggcfxoj2JEvJJBKj0C66I6PyG5/lz9o= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643 h1:yJFyJgGVy1RMEJqPrTYyaB7fF1wpfx0Df5Bsunb+Lyg= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643/go.mod h1:Xv+d7Q11qGJcggcfxoj2JEvJJBKj0C66I6PyG5/lz9o= git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 h1:fWAvsSiuDqveuxwnfc8psInfLZhMqHlQnmOpOHsd8Tk= git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0/go.mod h1:qdvpc/W1xvf2MKx3rMOqvFvYaYIHG77Z1g0lwsmw0Uk= git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 h1:KLjxHwp9Zp7xhECccmJS00RiL+VwTuUGLU7qeIctg8g= diff --git a/utils/iterator.go b/utils/iterator.go index 06f251e..ba98a07 100644 --- a/utils/iterator.go +++ b/utils/iterator.go @@ -7,9 +7,9 @@ import ( "github.com/ethereum/go-ethereum/trie" ) -type symmDiffIterator struct { +type SymmDiffIterator struct { a, b iterState // Nodes returned are those in b - a and a - b (keys only) - SymmDiffAux + SymmDiffState } // pairs an iterator with a cache of its valid status @@ -18,10 +18,10 @@ type iterState struct { valid bool } -// SymmDiffAux exposes state specific to symmetric difference iteration, which is not accessible +// SymmDiffState exposes state specific to symmetric difference iteration, which is not accessible // from the NodeIterator interface. This includes the number of nodes seen, whether the current key // is common to both A and B, and whether the current node is sourced from A or B. -type SymmDiffAux struct { +type SymmDiffState struct { yieldFromA bool // Whether next node comes from a count int // Number of nodes scanned on either trie eqPathIndex int // Count index of last pair of equal paths, to detect an updated key @@ -30,14 +30,14 @@ type SymmDiffAux struct { // NewSymmetricDifferenceIterator constructs a trie.NodeIterator that iterates over the symmetric difference // of elements in a and b, i.e., the elements in a that are not in b, and vice versa. // Returns the iterator, and a pointer to an auxiliary object for accessing the state not exposed by the NodeIterator interface recording the number of nodes seen. -func NewSymmetricDifferenceIterator(a, b trie.NodeIterator) (trie.NodeIterator, *SymmDiffAux) { - it := &symmDiffIterator{ +func NewSymmetricDifferenceIterator(a, b trie.NodeIterator) *SymmDiffIterator { + it := &SymmDiffIterator{ a: iterState{a, true}, b: iterState{b, true}, // common paths are detected by a distance <=1 between count and this index, so we start at -2 - SymmDiffAux: SymmDiffAux{eqPathIndex: -2}, + SymmDiffState: SymmDiffState{eqPathIndex: -2}, } - return it, &it.SymmDiffAux + return it } func (st *iterState) Next(descend bool) bool { @@ -46,65 +46,65 @@ func (st *iterState) Next(descend bool) bool { } // FromA returns true if the current node is sourced from A. -func (it *SymmDiffAux) FromA() bool { +func (it *SymmDiffState) FromA() bool { return it.yieldFromA } // CommonPath returns true if a node with the current path exists in each sub-iterator - i.e. it // represents an updated node. -func (it *SymmDiffAux) CommonPath() bool { +func (it *SymmDiffState) CommonPath() bool { return it.count-it.eqPathIndex <= 1 } // Count returns the number of nodes seen. -func (it *SymmDiffAux) Count() int { +func (it *SymmDiffState) Count() int { return it.count } -func (it *symmDiffIterator) curr() *iterState { +func (it *SymmDiffIterator) curr() *iterState { if it.yieldFromA { return &it.a } return &it.b } -func (it *symmDiffIterator) Hash() common.Hash { +func (it *SymmDiffIterator) Hash() common.Hash { return it.curr().Hash() } -func (it *symmDiffIterator) Parent() common.Hash { +func (it *SymmDiffIterator) Parent() common.Hash { return it.curr().Parent() } -func (it *symmDiffIterator) Leaf() bool { +func (it *SymmDiffIterator) Leaf() bool { return it.curr().Leaf() } -func (it *symmDiffIterator) LeafKey() []byte { +func (it *SymmDiffIterator) LeafKey() []byte { return it.curr().LeafKey() } -func (it *symmDiffIterator) LeafBlob() []byte { +func (it *SymmDiffIterator) LeafBlob() []byte { return it.curr().LeafBlob() } -func (it *symmDiffIterator) LeafProof() [][]byte { +func (it *SymmDiffIterator) LeafProof() [][]byte { return it.curr().LeafProof() } -func (it *symmDiffIterator) Path() []byte { +func (it *SymmDiffIterator) Path() []byte { return it.curr().Path() } -func (it *symmDiffIterator) NodeBlob() []byte { +func (it *SymmDiffIterator) NodeBlob() []byte { return it.curr().NodeBlob() } -func (it *symmDiffIterator) AddResolver(resolver trie.NodeResolver) { +func (it *SymmDiffIterator) AddResolver(resolver trie.NodeResolver) { panic("not implemented") } -func (it *symmDiffIterator) Next(bool) bool { +func (it *SymmDiffIterator) Next(bool) bool { // NodeIterators start in a "pre-valid" state, so the first Next advances to a valid node. if it.count == 0 { if it.a.Next(true) { @@ -122,7 +122,7 @@ func (it *symmDiffIterator) Next(bool) bool { return it.a.valid || it.b.valid } -func (it *symmDiffIterator) seek() { +func (it *SymmDiffIterator) seek() { // Invariants: // - At the end of the function, the sub-iterator with the lexically lesser path // points to the next element @@ -163,7 +163,7 @@ func (it *symmDiffIterator) seek() { } } -func (it *symmDiffIterator) Error() error { +func (it *SymmDiffIterator) Error() error { if err := it.a.Error(); err != nil { return err } diff --git a/utils/iterator_test.go b/utils/iterator_test.go index 6874a65..b440fbd 100644 --- a/utils/iterator_test.go +++ b/utils/iterator_test.go @@ -45,33 +45,33 @@ func TestSymmetricDifferenceIterator(t *testing.T) { t.Run("with no difference", func(t *testing.T) { db := trie.NewDatabase(rawdb.NewMemoryDatabase()) triea := trie.NewEmpty(db) - di, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) + di := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) for di.Next(true) { t.Errorf("iterator should not yield any elements") } - assert.Equal(t, 0, aux.Count()) + assert.Equal(t, 0, di.Count()) triea.MustUpdate([]byte("foo"), []byte("bar")) - di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) + di = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), triea.NodeIterator(nil)) for di.Next(true) { t.Errorf("iterator should not yield any elements") } // two nodes visited: the leaf (value) and its parent - assert.Equal(t, 2, aux.Count()) + assert.Equal(t, 2, di.Count()) trieb := trie.NewEmpty(db) - di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator([]byte("jars")), trieb.NodeIterator(nil)) + di = utils.NewSymmetricDifferenceIterator(triea.NodeIterator([]byte("jars")), trieb.NodeIterator(nil)) for di.Next(true) { t.Errorf("iterator should not yield any elements") } - assert.Equal(t, 0, aux.Count()) + assert.Equal(t, 0, di.Count()) // TODO will fail until merged: https://github.com/ethereum/go-ethereum/pull/27838 // di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator([]byte("food")), trieb.NodeIterator(nil)) // for di.Next(true) { // t.Errorf("iterator should not yield any elements") // } - // assert.Equal(t, 0, aux.Count()) + // assert.Equal(t, 0, di.Count()) }) t.Run("small difference", func(t *testing.T) { @@ -82,32 +82,32 @@ func TestSymmetricDifferenceIterator(t *testing.T) { trieb := trie.NewEmpty(dbb) trieb.MustUpdate([]byte("foo"), []byte("bar")) - di, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) + di := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) leaves := 0 for di.Next(true) { if di.Leaf() { - assert.False(t, aux.CommonPath()) + assert.False(t, di.CommonPath()) assert.Equal(t, "foo", string(di.LeafKey())) assert.Equal(t, "bar", string(di.LeafBlob())) leaves++ } } assert.Equal(t, 1, leaves) - assert.Equal(t, 2, aux.Count()) + assert.Equal(t, 2, di.Count()) trieb.MustUpdate([]byte("quux"), []byte("bars")) - di, aux = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator([]byte("quux"))) + di = utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator([]byte("quux"))) leaves = 0 for di.Next(true) { if di.Leaf() { - assert.False(t, aux.CommonPath()) + assert.False(t, di.CommonPath()) assert.Equal(t, "quux", string(di.LeafKey())) assert.Equal(t, "bars", string(di.LeafBlob())) leaves++ } } assert.Equal(t, 1, leaves) - assert.Equal(t, 1, aux.Count()) + assert.Equal(t, 1, di.Count()) }) dba := trie.NewDatabase(rawdb.NewMemoryDatabase()) @@ -124,20 +124,20 @@ func TestSymmetricDifferenceIterator(t *testing.T) { onlyA := make(map[string]string) onlyB := make(map[string]string) var deletions, creations []string - it, aux := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) + it := utils.NewSymmetricDifferenceIterator(triea.NodeIterator(nil), trieb.NodeIterator(nil)) for it.Next(true) { if !it.Leaf() { continue } key, value := string(it.LeafKey()), string(it.LeafBlob()) - if aux.FromA() { + if it.FromA() { onlyA[key] = value - if !aux.CommonPath() { + if !it.CommonPath() { deletions = append(deletions, key) } } else { onlyB[key] = value - if !aux.CommonPath() { + if !it.CommonPath() { creations = append(creations, key) } } @@ -205,10 +205,10 @@ func TestCompareDifferenceIterators(t *testing.T) { pathsA = append(pathsA, itAonly.Path()) } - itSym, aux := utils.NewSymmetricDifferenceIterator(treeA.NodeIterator(nil), treeB.NodeIterator(nil)) + itSym := utils.NewSymmetricDifferenceIterator(treeA.NodeIterator(nil), treeB.NodeIterator(nil)) var idxA, idxB int for itSym.Next(true) { - if aux.FromA() { + if itSym.FromA() { require.Equal(t, pathsA[idxA], itSym.Path()) idxA++ } else { -- 2.45.2 From 367349b85a3c9b1c4dc193cdf87f2160ddab7846 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 26 Sep 2023 19:44:52 +0800 Subject: [PATCH 15/22] update dep --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 3e43964..d84ab58 100644 --- a/go.mod +++ b/go.mod @@ -124,8 +124,8 @@ require ( ) replace ( - github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643 - github.com/cerc-io/eth-testing => git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 + github.com/cerc-io/eth-iterator-utils => git.vdb.to/cerc-io/eth-iterator-utils v0.1.2 + github.com/cerc-io/eth-testing => git.vdb.to/cerc-io/eth-testing v0.3.1 github.com/ethereum/go-ethereum => git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 github.com/openrelayxyz/plugeth-utils => git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230706160122-cd41de354c46 ) diff --git a/go.sum b/go.sum index c7ae382..39e1e6b 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643 h1:yJFyJgGVy1RMEJqPrTYyaB7fF1wpfx0Df5Bsunb+Lyg= -git.vdb.to/cerc-io/eth-iterator-utils v0.1.2-0.20230926100620-802551012643/go.mod h1:Xv+d7Q11qGJcggcfxoj2JEvJJBKj0C66I6PyG5/lz9o= -git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0 h1:fWAvsSiuDqveuxwnfc8psInfLZhMqHlQnmOpOHsd8Tk= -git.vdb.to/cerc-io/eth-testing v0.3.1-0.20230925181540-2ea71042e7e0/go.mod h1:qdvpc/W1xvf2MKx3rMOqvFvYaYIHG77Z1g0lwsmw0Uk= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2 h1:PdMR5B9wrQSYuYpFhN+9Kc8AEZ0pTt5eKCmu8oCtFcY= +git.vdb.to/cerc-io/eth-iterator-utils v0.1.2/go.mod h1:OvXbdWbZ5viBXC/Ui1EkhsSmGB+AUX+TjGa3UDAfjfg= +git.vdb.to/cerc-io/eth-testing v0.3.1 h1:sPnlMev6oEgTjsW7GtUkSsjKNG/+X6P9q0izSejLGpM= +git.vdb.to/cerc-io/eth-testing v0.3.1/go.mod h1:qdvpc/W1xvf2MKx3rMOqvFvYaYIHG77Z1g0lwsmw0Uk= git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1 h1:KLjxHwp9Zp7xhECccmJS00RiL+VwTuUGLU7qeIctg8g= git.vdb.to/cerc-io/plugeth v0.0.0-20230808125822-691dc334fab1/go.mod h1:cYXZu70+6xmDgIgrTD81GPasv16piiAFJnKyAbwVPMU= git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230706160122-cd41de354c46 h1:KYcbbne/RXd7AuxbUd/3hgk1jPN+33k2CKiNsUsMCC0= -- 2.45.2 From c37aab8b695481e459b7c2072c76fed29ecf1743 Mon Sep 17 00:00:00 2001 From: Thomas E Lackey Date: Tue, 26 Sep 2023 20:12:19 -0500 Subject: [PATCH 16/22] Run compliance tests. --- .gitea/workflows/test.yml | 41 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/.gitea/workflows/test.yml b/.gitea/workflows/test.yml index 3b40968..245d90e 100644 --- a/.gitea/workflows/test.yml +++ b/.gitea/workflows/test.yml @@ -89,3 +89,44 @@ jobs: pip install pytest pip install -r requirements.txt pytest -v -k test_basic_db + + compliance-test: + name: Run compliance tests + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + with: + path: ./plugeth-statediff + - uses: actions/checkout@v3 + with: + repository: cerc-io/eth-statediff-compliance + ref: telackey/ci + path: ./eth-statediff-compliance + token: ${{ secrets.CICD_REPO_TOKEN }} + - uses: actions/setup-go@v4 + with: + go-version-file: 'go.mod' + check-latest: true + - name: Set up Gitea access token + env: + TOKEN: ${{ secrets.CICD_REPO_TOKEN }} + run: | + git config --global url."https://$TOKEN:@git.vdb.to/".insteadOf https://git.vdb.to/ + - name: Update go.mod for dumpdiff-geth + working-directory: ./eth-statediff-compliance/ + run: ./scripts/update-mod.sh ../plugeth-statediff dumpdiff-geth/ + - name: Update go.mod for dumpdiff-plugeth + working-directory: ./eth-statediff-compliance/ + run: ./scripts/update-mod.sh ../plugeth-statediff dumpdiff-plugeth/ + - name: Update go.mod for dumpdiff-plugeth-parallel + working-directory: ./eth-statediff-compliance/ + run: ./scripts/update-mod.sh ../plugeth-statediff dumpdiff-plugeth-parallel/ + - name: Build tools + working-directory: ./eth-statediff-compliance/ + run: make all + - name: Compare output of geth and plugeth + working-directory: ./eth-statediff-compliance/ + run: ./scripts/compare-diffs.sh geth plugeth + - name: Compare output of geth and plugeth-parallel + working-directory: ./eth-statediff-compliance/ + run: ./scripts/compare-diffs.sh geth plugeth-parallel -- 2.45.2 From dfa0660317e21181847f5b3bfe51de024b65beb9 Mon Sep 17 00:00:00 2001 From: Thomas E Lackey Date: Tue, 26 Sep 2023 20:16:54 -0500 Subject: [PATCH 17/22] Tweak path --- .gitea/workflows/test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitea/workflows/test.yml b/.gitea/workflows/test.yml index 245d90e..eafef8c 100644 --- a/.gitea/workflows/test.yml +++ b/.gitea/workflows/test.yml @@ -105,7 +105,7 @@ jobs: token: ${{ secrets.CICD_REPO_TOKEN }} - uses: actions/setup-go@v4 with: - go-version-file: 'go.mod' + go-version-file: './eth-statediff-compliance/go.mod' check-latest: true - name: Set up Gitea access token env: -- 2.45.2 From c8e7acaacd34717bb0233ad51c6d713be0e2cb24 Mon Sep 17 00:00:00 2001 From: Thomas E Lackey Date: Tue, 26 Sep 2023 20:24:23 -0500 Subject: [PATCH 18/22] jq --- .gitea/workflows/test.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitea/workflows/test.yml b/.gitea/workflows/test.yml index eafef8c..5c068e1 100644 --- a/.gitea/workflows/test.yml +++ b/.gitea/workflows/test.yml @@ -107,6 +107,8 @@ jobs: with: go-version-file: './eth-statediff-compliance/go.mod' check-latest: true + - name: Install jq + run: apt-get update && apt-get install -yq jq - name: Set up Gitea access token env: TOKEN: ${{ secrets.CICD_REPO_TOKEN }} -- 2.45.2 From 101654f30eff32fb3d24e8ef3478f0ce3b12d1ef Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 27 Sep 2023 15:50:17 +0800 Subject: [PATCH 19/22] Rm branch spec --- .gitea/workflows/test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitea/workflows/test.yml b/.gitea/workflows/test.yml index 5c068e1..dc1c606 100644 --- a/.gitea/workflows/test.yml +++ b/.gitea/workflows/test.yml @@ -100,7 +100,6 @@ jobs: - uses: actions/checkout@v3 with: repository: cerc-io/eth-statediff-compliance - ref: telackey/ci path: ./eth-statediff-compliance token: ${{ secrets.CICD_REPO_TOKEN }} - uses: actions/setup-go@v4 @@ -114,6 +113,7 @@ jobs: TOKEN: ${{ secrets.CICD_REPO_TOKEN }} run: | git config --global url."https://$TOKEN:@git.vdb.to/".insteadOf https://git.vdb.to/ + - name: Update go.mod for dumpdiff-geth working-directory: ./eth-statediff-compliance/ run: ./scripts/update-mod.sh ../plugeth-statediff dumpdiff-geth/ -- 2.45.2 From 935c2ce4b5f7cc9c5b349efbab7ae2ae729581ce Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 27 Sep 2023 15:54:19 +0800 Subject: [PATCH 20/22] Use v0.1.0 --- .gitea/workflows/test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitea/workflows/test.yml b/.gitea/workflows/test.yml index dc1c606..dcd898a 100644 --- a/.gitea/workflows/test.yml +++ b/.gitea/workflows/test.yml @@ -100,6 +100,7 @@ jobs: - uses: actions/checkout@v3 with: repository: cerc-io/eth-statediff-compliance + ref: v0.1.0 path: ./eth-statediff-compliance token: ${{ secrets.CICD_REPO_TOKEN }} - uses: actions/setup-go@v4 -- 2.45.2 From c2ceca9230b27f16eec1320eccd8971fa8844e77 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 27 Sep 2023 17:20:14 +0800 Subject: [PATCH 21/22] refactor: only support tracked snapshots tracking incremental state diffs is not feasible like this. because updates are processed after the trie has been traversed, the iterators' state doesn't fully capture the progress of the diff. since currently snapshots are the only use case for tracking, let's just support that. --- builder.go | 35 +-- builder_snapshot_test.go | 538 ++++++++++++++++++++++++++++++++++ builder_test.go | 32 +- mainnet_tests/builder_test.go | 4 +- test_helpers/builder.go | 94 +++++- utils/iterator.go | 6 + 6 files changed, 662 insertions(+), 47 deletions(-) create mode 100644 builder_snapshot_test.go diff --git a/builder.go b/builder.go index 7839238..80cdf88 100644 --- a/builder.go +++ b/builder.go @@ -161,35 +161,23 @@ func (sdb *builder) WriteStateDiff( } // WriteStateDiff writes a statediff object to output sinks -func (sdb *builder) WriteStateDiffTracked( - args Args, params Params, +func (sdb *builder) WriteStateSnapshot( + stateRoot common.Hash, params Params, nodeSink sdtypes.StateNodeSink, ipldSink sdtypes.IPLDSink, tracker tracker.IteratorTracker, ) error { defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.WriteStateDiffTimer) // Load tries for old and new states - triea, err := sdb.stateCache.OpenTrie(args.OldStateRoot) - if err != nil { - return fmt.Errorf("error opening old state trie: %w", err) - } - trieb, err := sdb.stateCache.OpenTrie(args.NewStateRoot) + tree, err := sdb.stateCache.OpenTrie(stateRoot) if err != nil { return fmt.Errorf("error opening new state trie: %w", err) } - var subiters, bases []trie.NodeIterator - // Constructor for difference iterator at a specific (recovered) path - makeIterator := func(key []byte) trie.NodeIterator { - a := triea.NodeIterator(key) - b := trieb.NodeIterator(key) - return utils.NewSymmetricDifferenceIterator(a, b) - } - subiters, bases, err = tracker.Restore(makeIterator) + subiters, _, err := tracker.Restore(tree.NodeIterator) if err != nil { return fmt.Errorf("error restoring iterators: %w", err) } - if len(subiters) != 0 { // Completed iterators are not saved by the tracker, so restoring fewer than configured is ok, // but having too many is a problem. @@ -198,21 +186,21 @@ func (sdb *builder) WriteStateDiffTracked( sdb.subtrieWorkers, len(subiters)) } } else { - subiters = iterutils.SubtrieIterators(makeIterator, uint(sdb.subtrieWorkers)) + subiters = iterutils.SubtrieIterators(tree.NodeIterator, uint(sdb.subtrieWorkers)) for i := range subiters { subiters[i] = tracker.Tracked(subiters[i]) } } - logger := log.New("hash", args.BlockHash, "number", args.BlockNumber) // errgroup will cancel if any group fails g, ctx := errgroup.WithContext(context.Background()) for i := range subiters { func(subdiv uint) { g.Go(func() error { + symdiff := utils.AlwaysBState() return sdb.processAccounts(ctx, - subiters[subdiv], &bases[subdiv].(*utils.SymmDiffIterator).SymmDiffState, + subiters[subdiv], &symdiff, params.watchedAddressesLeafPaths, - nodeSink, ipldSink, logger, + nodeSink, ipldSink, log.DefaultLogger, ) }) }(uint(i)) @@ -299,8 +287,6 @@ func (sdb *builder) processAccounts( // New inner trie nodes will be written to blockstore only. // Reminder: this includes leaf nodes, since the geth iterator.Leaf() actually // signifies a "value" node. - - // TODO: A zero hash indicates what? if it.Hash() == zeroHash { continue } @@ -350,8 +336,6 @@ func (sdb *builder) processAccounts( return err } } - - metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(symdiff.Count())) return it.Error() } @@ -457,7 +441,8 @@ func (sdb *builder) processStorageCreations( return it.Error() } -// processStorageUpdates builds the storage diff node objects for all nodes that exist in a different state at B than A +// processStorageUpdates builds the storage diff node objects for all nodes that exist in a +// different state at B than A func (sdb *builder) processStorageUpdates( oldroot common.Hash, newroot common.Hash, storageSink sdtypes.StorageNodeSink, diff --git a/builder_snapshot_test.go b/builder_snapshot_test.go new file mode 100644 index 0000000..a071c38 --- /dev/null +++ b/builder_snapshot_test.go @@ -0,0 +1,538 @@ +package statediff_test + +import ( + "testing" + + statediff "github.com/cerc-io/plugeth-statediff" + "github.com/cerc-io/plugeth-statediff/indexer/ipld" + "github.com/cerc-io/plugeth-statediff/test_helpers" + sdtypes "github.com/cerc-io/plugeth-statediff/types" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/crypto" +) + +func TestBuilderSnapshot(t *testing.T) { + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) + defer chain.Stop() + block0 = test_helpers.Genesis + block1 = blocks[0] + block2 = blocks[1] + block3 = blocks[2] + params := statediff.Params{} + + tests := []test_helpers.SnapshotTestCase{ + { + "testEmptyDiff", + common.Hash{}, + &sdtypes.StateObject{ + Nodes: emptyDiffs, + }, + }, + { + "testBlock0", + //10000 transferred from testBankAddress to account1Addr + block0.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: bankAccountAtBlock0, + LeafKey: test_helpers.BankLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock0LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock0LeafNode)).String(), + Content: bankAccountAtBlock0LeafNode, + }, + }, + }, + }, + { + "testBlock1", + //10000 transferred from testBankAddress to account1Addr + block1.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: bankAccountAtBlock1, + LeafKey: test_helpers.BankLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock1LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: minerAccountAtBlock1, + LeafKey: minerLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock1LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock1, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock1LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1BranchRootNode)).String(), + Content: block1BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock1LeafNode)).String(), + Content: bankAccountAtBlock1LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock1LeafNode)).String(), + Content: minerAccountAtBlock1LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock1LeafNode)).String(), + Content: account1AtBlock1LeafNode, + }, + }, + }, + }, + { + "testBlock2", + //1000 transferred from testBankAddress to account1Addr + //1000 transferred from account1Addr to account2Addr + block2.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: bankAccountAtBlock2, + LeafKey: test_helpers.BankLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: minerAccountAtBlock2, + LeafKey: minerLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock2, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: contractAccountAtBlock2, + LeafKey: contractLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock2LeafNode)).String()}, + StorageDiff: []sdtypes.StorageLeafNode{ + { + Removed: false, + Value: slot0StorageValue, + LeafKey: slot0StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + }, + { + Removed: false, + Value: slot1StorageValue, + LeafKey: slot1StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + }, + }, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account2AtBlock2, + LeafKey: test_helpers.Account2LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account2AtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.RawBinary, test_helpers.CodeHash.Bytes()).String(), + Content: test_helpers.ByteCodeAfterDeployment, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2BranchRootNode)).String(), + Content: block2BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock2LeafNode)).String(), + Content: bankAccountAtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock2LeafNode)).String(), + Content: minerAccountAtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String(), + Content: account1AtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock2LeafNode)).String(), + Content: contractAccountAtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(block2StorageBranchRootNode)).String(), + Content: block2StorageBranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + Content: slot0StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + Content: slot1StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account2AtBlock2LeafNode)).String(), + Content: account2AtBlock2LeafNode, + }, + }, + }, + }, + { + "testBlock3", + //the contract's storage is changed + //and the block is mined by account 2 + block3.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: minerAccountAtBlock2, + LeafKey: minerLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock2, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: bankAccountAtBlock3, + LeafKey: test_helpers.BankLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock3LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: contractAccountAtBlock3, + LeafKey: contractLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock3LeafNode)).String()}, + StorageDiff: []sdtypes.StorageLeafNode{ + + { + Removed: false, + Value: slot0StorageValue, + LeafKey: slot0StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + }, + { + Removed: false, + Value: slot1StorageValue, + LeafKey: slot1StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + }, + + { + Removed: false, + Value: slot3StorageValue, + LeafKey: slot3StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot3StorageLeafNode)).String(), + }, + }, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account2AtBlock3, + LeafKey: test_helpers.Account2LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account2AtBlock3LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.RawBinary, test_helpers.CodeHash.Bytes()).String(), + Content: test_helpers.ByteCodeAfterDeployment, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(minerAccountAtBlock2LeafNode)).String(), + Content: minerAccountAtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String(), + Content: account1AtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + Content: slot0StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + Content: slot1StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3BranchRootNode)).String(), + Content: block3BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(bankAccountAtBlock3LeafNode)).String(), + Content: bankAccountAtBlock3LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock3LeafNode)).String(), + Content: contractAccountAtBlock3LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(block3StorageBranchRootNode)).String(), + Content: block3StorageBranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot3StorageLeafNode)).String(), + Content: slot3StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account2AtBlock3LeafNode)).String(), + Content: account2AtBlock3LeafNode, + }, + }, + }, + }, + } + + for _, test := range tests { + test_helpers.RunStateSnapshot(t, chain.StateCache(), test, params) + } +} + +func TestBuilderSnapshotWithWatchedAddressList(t *testing.T) { + blocks, chain := test_helpers.MakeChain(3, test_helpers.Genesis, test_helpers.TestChainGen) + contractLeafKey = test_helpers.AddressToLeafKey(test_helpers.ContractAddr) + defer chain.Stop() + block0 = test_helpers.Genesis + block1 = blocks[0] + block2 = blocks[1] + block3 = blocks[2] + params := statediff.Params{ + WatchedAddresses: []common.Address{test_helpers.Account1Addr, test_helpers.ContractAddr}, + } + params.ComputeWatchedAddressesLeafPaths() + + var tests = []test_helpers.SnapshotTestCase{ + { + "testBlock0", + //10000 transferred from testBankAddress to account1Addr + block0.Root(), + &sdtypes.StateObject{ + Nodes: emptyDiffs, + }, + }, + { + "testBlock1", + //10000 transferred from testBankAddress to account1Addr + block1.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock1, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock1LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1BranchRootNode)).String(), + Content: block1BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock1LeafNode)).String(), + Content: account1AtBlock1LeafNode, + }, + }, + }, + }, + { + "testBlock2", + //1000 transferred from testBankAddress to account1Addr + //1000 transferred from account1Addr to account2Addr + block2.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: contractAccountAtBlock2, + LeafKey: contractLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock2LeafNode)).String(), + }, + StorageDiff: []sdtypes.StorageLeafNode{ + { + Removed: false, + Value: slot0StorageValue, + LeafKey: slot0StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + }, + { + Removed: false, + Value: slot1StorageValue, + LeafKey: slot1StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + }, + }, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock2, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.RawBinary, test_helpers.CodeHash.Bytes()).String(), + Content: test_helpers.ByteCodeAfterDeployment, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2BranchRootNode)).String(), + Content: block2BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock2LeafNode)).String(), + Content: contractAccountAtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(block2StorageBranchRootNode)).String(), + Content: block2StorageBranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + Content: slot0StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + Content: slot1StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String(), + Content: account1AtBlock2LeafNode, + }, + }, + }, + }, + { + "testBlock3", + //the contract's storage is changed + //and the block is mined by account 2 + block3.Root(), + &sdtypes.StateObject{ + Nodes: []sdtypes.StateLeafNode{ + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: account1AtBlock2, + LeafKey: test_helpers.Account1LeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String()}, + StorageDiff: emptyStorage, + }, + { + Removed: false, + AccountWrapper: sdtypes.AccountWrapper{ + Account: contractAccountAtBlock3, + LeafKey: contractLeafKey, + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock3LeafNode)).String()}, + StorageDiff: []sdtypes.StorageLeafNode{ + { + Removed: false, + Value: slot0StorageValue, + LeafKey: slot0StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + }, + { + Removed: false, + Value: slot1StorageValue, + LeafKey: slot1StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + }, + { + Removed: false, + Value: slot3StorageValue, + LeafKey: slot3StorageKey.Bytes(), + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot3StorageLeafNode)).String(), + }, + }, + }, + }, + IPLDs: []sdtypes.IPLD{ + { + CID: ipld.Keccak256ToCid(ipld.RawBinary, test_helpers.CodeHash.Bytes()).String(), + Content: test_helpers.ByteCodeAfterDeployment, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(account1AtBlock2LeafNode)).String(), + Content: account1AtBlock2LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot0StorageLeafNode)).String(), + Content: slot0StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot1StorageLeafNode)).String(), + Content: slot1StorageLeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3BranchRootNode)).String(), + Content: block3BranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(contractAccountAtBlock3LeafNode)).String(), + Content: contractAccountAtBlock3LeafNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(block3StorageBranchRootNode)).String(), + Content: block3StorageBranchRootNode, + }, + { + CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(slot3StorageLeafNode)).String(), + Content: slot3StorageLeafNode, + }, + }, + }, + }, + } + + for _, test := range tests { + test_helpers.RunStateSnapshot(t, chain.StateCache(), test, params) + } +} diff --git a/builder_test.go b/builder_test.go index 68e7ea2..1540880 100644 --- a/builder_test.go +++ b/builder_test.go @@ -503,7 +503,7 @@ func TestBuilder(t *testing.T) { block3 = blocks[2] params := statediff.Params{} - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testEmptyDiff", statediff.Args{ @@ -795,7 +795,7 @@ func TestBuilder(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block0: bankAccountAtBlock0LeafNode, block1: block1BranchRootNode, @@ -817,7 +817,7 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { } params.ComputeWatchedAddressesLeafPaths() - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testEmptyDiff", statediff.Args{ @@ -1009,7 +1009,7 @@ func TestBuilderWithWatchedAddressList(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block0: bankAccountAtBlock0LeafNode, block1: block1BranchRootNode, @@ -1028,7 +1028,7 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { block6 = blocks[5] params := statediff.Params{} - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ // blocks 0-3 are the same as in TestBuilderWithIntermediateNodes { "testBlock4", @@ -1260,7 +1260,7 @@ func TestBuilderWithRemovedAccountAndStorage(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block4: block4BranchRootNode, block5: block5BranchRootNode, @@ -1281,7 +1281,7 @@ func TestBuilderWithRemovedNonWatchedAccount(t *testing.T) { } params.ComputeWatchedAddressesLeafPaths() - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testBlock4", statediff.Args{ @@ -1395,7 +1395,7 @@ func TestBuilderWithRemovedNonWatchedAccount(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block4: block4BranchRootNode, block5: block5BranchRootNode, @@ -1416,7 +1416,7 @@ func TestBuilderWithRemovedWatchedAccount(t *testing.T) { } params.ComputeWatchedAddressesLeafPaths() - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testBlock4", statediff.Args{ @@ -1599,7 +1599,7 @@ func TestBuilderWithRemovedWatchedAccount(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block4: block4BranchRootNode, block5: block5BranchRootNode, @@ -1700,7 +1700,7 @@ func TestBuilderWithMovedAccount(t *testing.T) { block2 = blocks[1] params := statediff.Params{} - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testBlock1", statediff.Args{ @@ -1827,7 +1827,7 @@ func TestBuilderWithMovedAccount(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block1: block01BranchRootNode, block2: bankAccountAtBlock02LeafNode, @@ -2088,7 +2088,7 @@ func TestBuilderWithInternalizedLeafNode(t *testing.T) { block3 = blocks[2] params := statediff.Params{} - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testEmptyDiff", statediff.Args{ @@ -2354,7 +2354,7 @@ func TestBuilderWithInternalizedLeafNode(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block1: block1bBranchRootNode, block2: block2bBranchRootNode, @@ -2377,7 +2377,7 @@ func TestBuilderWithInternalizedLeafNodeAndWatchedAddress(t *testing.T) { } params.ComputeWatchedAddressesLeafPaths() - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ { "testEmptyDiff", statediff.Args{ @@ -2556,7 +2556,7 @@ func TestBuilderWithInternalizedLeafNodeAndWatchedAddress(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block1: block1bBranchRootNode, block2: block2bBranchRootNode, diff --git a/mainnet_tests/builder_test.go b/mainnet_tests/builder_test.go index e428643..ded3ba4 100644 --- a/mainnet_tests/builder_test.go +++ b/mainnet_tests/builder_test.go @@ -444,7 +444,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { } params := statediff.Params{} - var tests = []test_helpers.TestCase{ + var tests = []test_helpers.DiffTestCase{ // 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 { @@ -624,7 +624,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) { }, } - test_helpers.RunBuilderTests(t, chain.StateCache(), tests, params, []uint{1, 8, 32}) + test_helpers.RunBuildStateDiff(t, chain.StateCache(), tests, params) test_helpers.CheckedRoots{ block1: block1RootBranchNode, block2: block2RootBranchNode, diff --git a/test_helpers/builder.go b/test_helpers/builder.go index 9a82851..dc7fd34 100644 --- a/test_helpers/builder.go +++ b/test_helpers/builder.go @@ -4,9 +4,13 @@ import ( "bytes" "fmt" "math/big" + "math/rand" + "path/filepath" "sort" + "sync" "testing" + "github.com/cerc-io/eth-iterator-utils/tracker" statediff "github.com/cerc-io/plugeth-statediff" "github.com/cerc-io/plugeth-statediff/adapt" sdtypes "github.com/cerc-io/plugeth-statediff/types" @@ -17,12 +21,20 @@ import ( "github.com/stretchr/testify/require" ) -type TestCase struct { +var subtrieCounts = []uint{1, 8, 32} + +type DiffTestCase struct { Name string Args statediff.Args Expected *sdtypes.StateObject } +type SnapshotTestCase struct { + Name string + StateRoot common.Hash + Expected *sdtypes.StateObject +} + type CheckedRoots map[*types.Block][]byte // Replicates the statediff object, but indexes nodes by CID @@ -33,12 +45,11 @@ type normalizedStateDiff struct { IPLDs map[string]sdtypes.IPLD } -func RunBuilderTests( +func RunBuildStateDiff( t *testing.T, sdb state.Database, - tests []TestCase, + tests []DiffTestCase, params statediff.Params, - subtrieCounts []uint, ) { builder := statediff.NewBuilder(adapt.GethStateView(sdb)) for _, test := range tests { @@ -58,6 +69,81 @@ func RunBuilderTests( } } +func RunStateSnapshot( + t *testing.T, + sdb state.Database, + test SnapshotTestCase, + params statediff.Params, +) { + builder := statediff.NewBuilder(adapt.GethStateView(sdb)) + + for _, subtries := range subtrieCounts { + // Skip the recovery test for empty diffs + doRecovery := len(test.Expected.Nodes) != 0 + + t.Run(fmt.Sprintf("%s with %d subtries", test.Name, subtries), func(t *testing.T) { + builder.SetSubtrieWorkers(subtries) + var stateNodes []sdtypes.StateLeafNode + var iplds []sdtypes.IPLD + interrupt := randomInterrupt(len(test.Expected.IPLDs)) + stateAppender := failingSyncedAppender(&stateNodes, -1) + ipldAppender := failingSyncedAppender(&iplds, interrupt) + recoveryFile := filepath.Join(t.TempDir(), "recovery.txt") + build := func() error { + tr := tracker.New(recoveryFile, subtries) + defer tr.CloseAndSave() + return builder.WriteStateSnapshot( + test.StateRoot, params, stateAppender, ipldAppender, tr, + ) + } + if doRecovery { + // First attempt fails, second succeeds + if build() == nil { + t.Fatal("expected an error") + } + } + // Ensure we don't exceed the expected number of nodes. If we do, it implies the + // failed attempt got further than intended, and we have duplicates. + // ipldAppender = failingSyncedAppender(&iplds, len(test.Expected.IPLDs)) + ipldAppender = failingSyncedAppender(&iplds, -1) + if err := build(); err != nil { + t.Fatal(err) + } + diff := sdtypes.StateObject{ + Nodes: stateNodes, + IPLDs: iplds, + } + require.Equal(t, + normalize(test.Expected), + normalize(&diff), + ) + }) + } + +} + +// an appender which fails on a configured trigger +func failingSyncedAppender[T any](to *[]T, failAt int) func(T) error { + var mtx sync.Mutex + return func(item T) error { + mtx.Lock() + defer mtx.Unlock() + if len(*to) == failAt { + return fmt.Errorf("failing at %d items", failAt) + } + *to = append(*to, item) + return nil + } +} + +// function to pick random int between N/4 and 3N/4 +func randomInterrupt(N int) int { + if N < 2 { + return 0 + } + return rand.Intn(N/2) + N/4 +} + func (roots CheckedRoots) Check(t *testing.T) { // Let's also confirm that our root state nodes form the state root hash in the headers for block, node := range roots { diff --git a/utils/iterator.go b/utils/iterator.go index ba98a07..5aac38e 100644 --- a/utils/iterator.go +++ b/utils/iterator.go @@ -184,3 +184,9 @@ func compareNodes(a, b trie.NodeIterator) int { } return 0 } + +// AlwaysBState returns a dummy SymmDiffState that indicates all elements are from B, and have no +// common paths with A. This is equivalent to a diff against an empty A. +func AlwaysBState() SymmDiffState { + return SymmDiffState{yieldFromA: false, eqPathIndex: -2} +} -- 2.45.2 From aeffc86033f5ee4b2783ef377df09ada8695454c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 28 Sep 2023 08:38:38 +0800 Subject: [PATCH 22/22] check for recovery file --- test_helpers/builder.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/test_helpers/builder.go b/test_helpers/builder.go index dc7fd34..44f46bf 100644 --- a/test_helpers/builder.go +++ b/test_helpers/builder.go @@ -101,10 +101,8 @@ func RunStateSnapshot( if build() == nil { t.Fatal("expected an error") } + require.FileExists(t, recoveryFile) } - // Ensure we don't exceed the expected number of nodes. If we do, it implies the - // failed attempt got further than intended, and we have duplicates. - // ipldAppender = failingSyncedAppender(&iplds, len(test.Expected.IPLDs)) ipldAppender = failingSyncedAppender(&iplds, -1) if err := build(); err != nil { t.Fatal(err) -- 2.45.2