Add WriteStateSnapshot #15

Merged
roysc merged 23 commits from with-iterator-tracker into main 2023-09-28 03:35:47 +00:00
12 changed files with 184 additions and 243 deletions
Showing only changes of commit 68e2aca32e - Show all commits

View File

@ -19,26 +19,55 @@ package dump
import ( import (
"fmt" "fmt"
"io" "io"
"math/big"
"github.com/cerc-io/plugeth-statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/models" "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 // BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct { type BatchTx struct {
BlockNumber string blockNum string
dump io.Writer dump io.Writer
quit chan struct{} quit chan struct{}
iplds chan models.IPLDModel iplds chan models.IPLDModel
ipldCache models.IPLDBatch ipldCache models.IPLDBatch
submit func(blockTx *BatchTx, err error) error
} }
// Submit satisfies indexer.AtomicTx func NewBatch(number *big.Int, dest io.Writer) *BatchTx {
func (tx *BatchTx) Submit(err error) error { batch := &BatchTx{
return tx.submit(tx, err) 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 { func (tx *BatchTx) flush() error {
@ -65,7 +94,7 @@ func (tx *BatchTx) cache() {
func (tx *BatchTx) cacheDirect(key string, value []byte) { func (tx *BatchTx) cacheDirect(key string, value []byte) {
tx.iplds <- models.IPLDModel{ tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
Key: key, Key: key,
Data: value, Data: value,
} }
@ -73,7 +102,7 @@ func (tx *BatchTx) cacheDirect(key string, value []byte) {
func (tx *BatchTx) cacheIPLD(i ipld.IPLD) { func (tx *BatchTx) cacheIPLD(i ipld.IPLD) {
tx.iplds <- models.IPLDModel{ tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
Key: i.Cid().String(), Key: i.Cid().String(),
Data: i.RawData(), Data: i.RawData(),
} }

View File

@ -17,6 +17,7 @@
package dump package dump
import ( import (
"context"
"encoding/hex" "encoding/hex"
"fmt" "fmt"
"io" "io"
@ -37,7 +38,6 @@ import (
"github.com/cerc-io/plugeth-statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types" sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} 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) // 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 // 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) { 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() blockHash := block.Hash()
blockHashStr := blockHash.String() blockHashStr := blockHash.String()
height := block.NumberU64() height := block.NumberU64()
@ -91,40 +91,8 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
} else { } else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts)
} }
t = time.Now()
blockTx := &BatchTx{ blockTx := NewBatch(block.Number(), sdi.dump)
roysc marked this conversation as resolved Outdated

Cleaning this part of the code up is a big win.

Cleaning this part of the code up is a big win.
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())
t = time.Now() t = time.Now()
// Publish and index header, collect headerID // Publish and index header, collect headerID
@ -133,7 +101,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
if err != nil { if err != nil {
return nil, err return nil, err
} }
tDiff = time.Since(t) tDiff := time.Since(t)
metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String())
t = time.Now() t = time.Now()
@ -352,7 +320,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
// short circuit if it is a Removed node // 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 // this assumes the db has been initialized and a ipld.blocks entry for the Removed node is present
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID, CID: shared.RemovedNodeStateCID,
@ -360,7 +328,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
} }
} else { } else {
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID, 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 // 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 // this assumes the db has been initialized and a ipld.blocks entry for the Removed node is present
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
@ -396,7 +364,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
continue continue
} }
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
@ -440,6 +408,10 @@ func (sdi *StateDiffIndexer) DetectGaps(beginBlockNumber uint64, endBlockNumber
return nil, nil return nil, nil
} }
func (sdi *StateDiffIndexer) BeginTx(number *big.Int, _ context.Context) interfaces.Batch {
return NewBatch(number, sdi.dump)
}
// Close satisfies io.Closer // Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error { func (sdi *StateDiffIndexer) Close() error {
return sdi.dump.Close() return sdi.dump.Close()

View File

@ -16,14 +16,29 @@
package file 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 // BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct { type BatchTx struct {
BlockNumber string blockNum string
fileWriter FileWriter
submit func(blockTx *BatchTx, err error) error
} }
// Submit satisfies indexer.AtomicTx // Submit satisfies indexer.AtomicTx
func (tx *BatchTx) Submit(err error) error { func (tx *BatchTx) Submit() error {
return tx.submit(tx, err) 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)
}
} }

View File

@ -17,6 +17,7 @@
package file package file
import ( import (
"context"
"errors" "errors"
"fmt" "fmt"
"math/big" "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) // 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 // 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) { 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() blockHash := block.Hash()
blockHashStr := blockHash.String() blockHashStr := blockHash.String()
height := block.NumberU64() height := block.NumberU64()
@ -159,27 +160,11 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
} else { } else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts)
} }
t = time.Now()
blockTx := &BatchTx{ blockTx := &BatchTx{
BlockNumber: block.Number().String(), blockNum: block.Number().String(),
submit: func(self *BatchTx, err error) error { fileWriter: sdi.fileWriter,
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
},
} }
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() t = time.Now()
// write header, collect headerID // write header, collect headerID
@ -187,7 +172,7 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
if err != nil { if err != nil {
return nil, err return nil, err
} }
tDiff = time.Since(t) tDiff := time.Since(t)
metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff) metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String()) traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String())
t = time.Now() 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 // 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 { func (sdi *StateDiffIndexer) PushStateNode(tx 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)
}
// publish the state node // publish the state node
var stateModel models.StateNodeModel var stateModel models.StateNodeModel
if stateNode.Removed { if stateNode.Removed {
if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 { if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 {
atomic.StoreUint32(&sdi.removedCacheFlag, 1) atomic.StoreUint32(&sdi.removedCacheFlag, 1)
sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStateCID, []byte{}) sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), shared.RemovedNodeStateCID, []byte{})
} }
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID, CID: shared.RemovedNodeStateCID,
@ -403,7 +384,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
} }
} else { } else {
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID, CID: stateNode.AccountWrapper.CID,
@ -423,10 +404,10 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
if storageNode.Removed { if storageNode.Removed {
if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 { if atomic.LoadUint32(&sdi.removedCacheFlag) == 0 {
atomic.StoreUint32(&sdi.removedCacheFlag, 1) atomic.StoreUint32(&sdi.removedCacheFlag, 1)
sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStorageCID, []byte{}) sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), shared.RemovedNodeStorageCID, []byte{})
} }
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
@ -438,7 +419,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
continue continue
} }
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.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 // PushIPLD writes iplds to ipld.blocks
func (sdi *StateDiffIndexer) PushIPLD(batch interfaces.Batch, ipld sdtypes.IPLD) error { func (sdi *StateDiffIndexer) PushIPLD(tx interfaces.Batch, ipld sdtypes.IPLD) error {
tx, ok := batch.(*BatchTx) sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber(), ipld.CID, ipld.Content)
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)
return nil return nil
} }
@ -480,6 +457,13 @@ func (sdi *StateDiffIndexer) HasBlock(hash common.Hash, number uint64) (bool, er
return false, nil 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 // Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error { func (sdi *StateDiffIndexer) Close() error {
return sdi.fileWriter.Close() return sdi.fileWriter.Close()

View File

@ -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 // BatchTx wraps a sql tx with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct { type BatchTx struct {
BlockNumber string blockNum string
ctx context.Context ctx context.Context
dbtx Tx dbtx Tx
stm string stm string
@ -44,15 +44,14 @@ type BatchTx struct {
ipldCache models.IPLDBatch ipldCache models.IPLDBatch
removedCacheFlag *uint32 removedCacheFlag *uint32
// Tracks expected cache size and ensures cache is caught up before flush // Tracks expected cache size and ensures cache is caught up before flush
cacheWg sync.WaitGroup cacheWg sync.WaitGroup
startTime time.Time
} }
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{ blockTx := &BatchTx{
removedCacheFlag: new(uint32), removedCacheFlag: new(uint32),
ctx: ctx, ctx: ctx,
BlockNumber: number.String(), blockNum: number.String(),
stm: insertStm, stm: insertStm,
iplds: make(chan models.IPLDModel), iplds: make(chan models.IPLDModel),
quit: make(chan (chan<- struct{})), 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), Keys: make([]string, 0, startingCacheCapacity),
Values: make([][]byte, 0, startingCacheCapacity), Values: make([][]byte, 0, startingCacheCapacity),
}, },
dbtx: tx, dbtx: tx,
startTime: time.Now(),
} }
go blockTx.cache()
return blockTx return blockTx
} }
// Submit satisfies indexer.Batch // Submit satisfies indexer.Batch
func (tx *BatchTx) Submit(err error) error { func (tx *BatchTx) Submit() error {
defer tx.close() defer tx.close()
metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(time.Since(tx.startTime))
t := time.Now() t := time.Now()
if err := tx.flush(); err != nil { if err := tx.flush(); err != nil {
rollback(tx.ctx, tx.dbtx) rollback(tx.ctx, tx.dbtx)
return err return err
} }
err = tx.dbtx.Commit(tx.ctx) err := tx.dbtx.Commit(tx.ctx)
metrics.IndexerMetrics.PostgresCommitTimer.Update(time.Since(t)) metrics.IndexerMetrics.PostgresCommitTimer.Update(time.Since(t))
return err return err
} }
func (tx *BatchTx) BlockNumber() string {
return tx.blockNum
}
func (tx *BatchTx) RollbackOnFailure(err error) { func (tx *BatchTx) RollbackOnFailure(err error) {
defer tx.close()
if p := recover(); p != nil { if p := recover(); p != nil {
defer tx.close()
log.Info("panic detected before tx submission, rolling back the tx", "panic", p) log.Info("panic detected before tx submission, rolling back the tx", "panic", p)
rollback(tx.ctx, tx.dbtx) rollback(tx.ctx, tx.dbtx)
panic(p) panic(p)
} else if err != nil { } else if err != nil {
defer tx.close()
log.Info("error detected before tx submission, rolling back the tx", "error", err) log.Info("error detected before tx submission, rolling back the tx", "error", err)
rollback(tx.ctx, tx.dbtx) rollback(tx.ctx, tx.dbtx)
} }
@ -148,7 +151,7 @@ func (tx *BatchTx) cache() {
func (tx *BatchTx) cacheDirect(key string, value []byte) { func (tx *BatchTx) cacheDirect(key string, value []byte) {
tx.cacheWg.Add(1) tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{ tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
Key: key, Key: key,
Data: value, Data: value,
} }
@ -157,7 +160,7 @@ func (tx *BatchTx) cacheDirect(key string, value []byte) {
func (tx *BatchTx) cacheIPLD(i ipld.IPLD) { func (tx *BatchTx) cacheIPLD(i ipld.IPLD) {
tx.cacheWg.Add(1) tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{ tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
Key: i.Cid().String(), Key: i.Cid().String(),
Data: i.RawData(), Data: i.RawData(),
} }
@ -168,7 +171,7 @@ func (tx *BatchTx) cacheRemoved(key string, value []byte) {
atomic.StoreUint32(tx.removedCacheFlag, 1) atomic.StoreUint32(tx.removedCacheFlag, 1)
tx.cacheWg.Add(1) tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{ tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
Key: key, Key: key,
Data: value, Data: value,
} }

View File

@ -39,7 +39,6 @@ import (
"github.com/cerc-io/plugeth-statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types" sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} 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) // 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 // 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) { 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() blockHash := block.Hash()
height := block.NumberU64() height := block.NumberU64()
traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHash)
transactions := block.Transactions() transactions := block.Transactions()
var err error
// Derive any missing fields // Derive any missing fields
if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil { if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil {
return nil, err return nil, err
@ -95,11 +95,12 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
// Generate the block iplds // Generate the block iplds
txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts) txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts)
if err != nil { 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) { 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 // Calculate reward
@ -108,98 +109,35 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
if sdi.chainConfig.Clique != nil { if sdi.chainConfig.Clique != nil {
reward = big.NewInt(0) reward = big.NewInt(0)
} else { } else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts) reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), transactions, receipts)
} }
t = time.Now() t = time.Now()
// Begin new DB tx for everything // Begin new DB tx for everything
tx := NewDelayedTx(sdi.dbWriter.db) batch := NewBatch(
defer func() { sdi.dbWriter.db.InsertIPLDsStm(), sdi.ctx,
if p := recover(); p != nil { block.Number(),
rollback(sdi.ctx, tx) NewDelayedTx(sdi.dbWriter.db),
panic(p) )
} else if err != nil { // handle transaction rollback for failures in this scope
rollback(sdi.ctx, tx) defer batch.RollbackOnFailure(err)
}
}()
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()
// Publish and index header, collect headerID // 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 { if err != nil {
return nil, err return nil, err
} }
tDiff = time.Since(t) metrics2.IndexerMetrics.HeaderProcessingTimer.Update(time.Since(t))
metrics2.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff)
t = time.Now() t = time.Now()
// Publish and index uncles // 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 { if err != nil {
return nil, err return nil, err
} }
tDiff = time.Since(t) metrics2.IndexerMetrics.UncleProcessingTimer.Update(time.Since(t))
metrics2.IndexerMetrics.UncleProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff)
t = time.Now() t = time.Now()
// Publish and index receipts and txs // Publish and index receipts and txs
err = sdi.processReceiptsAndTxs(blockTx, processArgs{ err = sdi.processReceiptsAndTxs(batch, processArgs{
headerID: headerID, headerID: headerID,
blockNumber: block.Number(), blockNumber: block.Number(),
receipts: receipts, receipts: receipts,
@ -211,12 +149,9 @@ func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receip
if err != nil { if err != nil {
return nil, err return nil, err
} }
tDiff = time.Since(t) metrics2.IndexerMetrics.TxAndRecProcessingTimer.Update(time.Since(t))
metrics2.IndexerMetrics.TxAndRecProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff)
t = time.Now()
return blockTx, err return batch, err
} }
// CurrentBlock returns the HeaderModel of the highest existing block in the database. // 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 { if stateNode.Removed {
tx.cacheRemoved(shared.RemovedNodeStateCID, []byte{}) tx.cacheRemoved(shared.RemovedNodeStateCID, []byte{})
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID, CID: shared.RemovedNodeStateCID,
@ -422,7 +357,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
} }
} else { } else {
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID, CID: stateNode.AccountWrapper.CID,
@ -444,7 +379,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
if storageNode.Removed { if storageNode.Removed {
tx.cacheRemoved(shared.RemovedNodeStorageCID, []byte{}) tx.cacheRemoved(shared.RemovedNodeStorageCID, []byte{})
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
@ -458,7 +393,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
continue continue
} }
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber(),
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.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) 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 // Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error { func (sdi *StateDiffIndexer) Close() error {
return sdi.dbWriter.Close() return sdi.dbWriter.Close()

View File

@ -17,6 +17,7 @@
package interfaces package interfaces
import ( import (
"context"
"math/big" "math/big"
"time" "time"
@ -39,6 +40,8 @@ type StateDiffIndexer interface {
PushIPLD(tx Batch, ipld sdtypes.IPLD) error PushIPLD(tx Batch, ipld sdtypes.IPLD) error
ReportDBMetrics(delay time.Duration, quit <-chan bool) ReportDBMetrics(delay time.Duration, quit <-chan bool)
BeginTx(number *big.Int, ctx context.Context) Batch
// Methods used by WatchAddress API/functionality // Methods used by WatchAddress API/functionality
LoadWatchedAddresses() ([]common.Address, error) LoadWatchedAddresses() ([]common.Address, error)
InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error
@ -51,7 +54,9 @@ type StateDiffIndexer interface {
// Batch required for indexing data atomically // Batch required for indexing data atomically
type Batch interface { type Batch interface {
Submit(err error) error Submit() error
BlockNumber() string
RollbackOnFailure(error)
} }
// Config used to configure different underlying implementations // Config used to configure different underlying implementations

View File

@ -28,7 +28,6 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "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/database/sql"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
@ -48,7 +47,7 @@ func SetupTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
t.Fatal(err) t.Fatal(err)
} }
defer func() { defer func() {
roysc marked this conversation as resolved
Review

Do you recall why we passed err in to Submit() before? I have a vague memory buzzing in the back of my mind and I want to double check that there is no longer any reason.

Do you recall why we passed err in to Submit() before? I have a vague memory buzzing in the back of my mind and I want to double check that there is no longer any reason.
Review

Basically it was due to handling rollbacks and commit in the submit function, so it needed the error to check whether to rollback. By factoring the rollback out, we avoid that, but now the caller is responsible for it.

Which reminds me that I need to defer a rollback call from the scope of service.writeStateDiff - since the tx is returned from PushBlock.

Basically it was due to handling rollbacks and commit in the submit function, so it needed the error to check whether to rollback. By factoring the rollback out, we avoid that, but now the caller is responsible for it. Which reminds me that I need to defer a rollback call from the scope of `service.writeStateDiff` - since the tx is returned from `PushBlock`.
Review

If you've added the rollback, feel free to resolve this conversation.

If you've added the rollback, feel free to resolve this conversation.
if err := tx.Submit(err); err != nil { if err := tx.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
}() }()
@ -61,11 +60,7 @@ func SetupTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
} }
if batchTx, ok := tx.(*sql.BatchTx); ok { require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber())
require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber)
} else if batchTx, ok := tx.(*file.BatchTx); ok {
require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber)
}
} }
func DoTestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) {
@ -547,13 +542,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
} }
if batchTx, ok := tx1.(*sql.BatchTx); ok { require.Equal(t, mocks.BlockNumber.String(), tx1.BlockNumber())
require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber)
} else if batchTx, ok := tx1.(*file.BatchTx); ok {
require.Equal(t, mocks.BlockNumber.String(), batchTx.BlockNumber)
}
if err := tx1.Submit(err); err != nil { if err := tx1.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -572,13 +563,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
} }
if tx, ok := tx2.(*sql.BatchTx); ok { require.Equal(t, mocks.BlockNumber.String(), tx2.BlockNumber())
require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber)
} else if tx, ok := tx2.(*sql.BatchTx); ok {
require.Equal(t, mocks.BlockNumber.String(), tx.BlockNumber)
}
if err := tx2.Submit(err); err != nil { if err := tx2.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -597,13 +584,9 @@ func SetupTestDataNonCanonical(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
} }
if batchTx, ok := tx3.(*sql.BatchTx); ok { require.Equal(t, mocks.Block2Number.String(), tx3.BlockNumber())
require.Equal(t, mocks.Block2Number.String(), batchTx.BlockNumber)
} else if batchTx, ok := tx3.(*file.BatchTx); ok {
require.Equal(t, mocks.Block2Number.String(), batchTx.BlockNumber)
}
if err := tx3.Submit(err); err != nil { if err := tx3.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
} }

View File

@ -20,7 +20,6 @@ import (
"context" "context"
"testing" "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/database/sql"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
@ -51,7 +50,7 @@ func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
defer func() { defer func() {
if err := tx.Submit(err); err != nil { if err := tx.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
}() }()
@ -60,11 +59,7 @@ func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
require.NoError(t, err) require.NoError(t, err)
} }
if batchTx, ok := tx.(*sql.BatchTx); ok { require.Equal(t, legacyData.BlockNumber.String(), tx.BlockNumber())
require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber)
} else if batchTx, ok := tx.(*file.BatchTx); ok {
require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber)
}
} }
func TestLegacyIndexer(t *testing.T, db sql.Database) { func TestLegacyIndexer(t *testing.T, db sql.Database) {

View File

@ -19,8 +19,6 @@ package test
import ( import (
"testing" "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/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/core/types" "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) require.NoError(t, err)
defer func() { defer func() {
if err := tx.Submit(err); err != nil { if err := tx.Submit(); err != nil {
t.Fatal(err) t.Fatal(err)
} }
}() }()
@ -45,9 +43,5 @@ func TestBlock(t *testing.T, ind interfaces.StateDiffIndexer, testBlock *types.B
require.NoError(t, err) require.NoError(t, err)
} }
if batchTx, ok := tx.(*sql.BatchTx); ok { require.Equal(t, testBlock.Number().String(), tx.BlockNumber())
require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber)
} else if batchTx, ok := tx.(*file.BatchTx); ok {
require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber)
}
} }

View File

@ -817,6 +817,8 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p
if params.IncludeReceipts { if params.IncludeReceipts {
receipts = sds.BlockChain.GetReceiptsByHash(block.Hash()) receipts = sds.BlockChain.GetReceiptsByHash(block.Hash())
} }
t := time.Now()
tx, err = sds.indexer.PushBlock(block, receipts, totalDifficulty) tx, err = sds.indexer.PushBlock(block, receipts, totalDifficulty)
if err != nil { if err != nil {
return err return err
@ -842,9 +844,12 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p
BlockHash: block.Hash(), BlockHash: block.Hash(),
BlockNumber: block.Number(), BlockNumber: block.Number(),
}, params, nodeSink, ipldSink) }, params, nodeSink, ipldSink)
if err != nil {
return err
}
// TODO this anti-pattern needs to be sorted out eventually metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(time.Since(t))
if err = tx.Submit(err); err != nil { if err = tx.Submit(); err != nil {
return fmt.Errorf("batch transaction submission failed: %w", err) return fmt.Errorf("batch transaction submission failed: %w", err)
} }
return nil return nil

View File

@ -17,6 +17,7 @@
package mocks package mocks
import ( import (
context "context"
"math/big" "math/big"
"time" "time"
@ -84,10 +85,21 @@ func (sdi *StateDiffIndexer) ClearWatchedAddresses() error {
return nil return nil
} }
func (sdi *StateDiffIndexer) BeginTx(number *big.Int, ctx context.Context) interfaces.Batch {
return &batch{}
}
func (sdi *StateDiffIndexer) Close() error { func (sdi *StateDiffIndexer) Close() error {
return nil return nil
} }
func (tx *batch) Submit(err error) error { func (tx *batch) RollbackOnFailure(error) {}
func (tx *batch) Submit() error {
return nil return nil
} }
// batch.BlockNumber
func (tx *batch) BlockNumber() string {
return "0"
}