refactor batch
This commit is contained in:
parent
a7b83fc63c
commit
68e2aca32e
@ -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(),
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
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()
|
||||||
|
@ -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)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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()
|
||||||
|
@ -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,
|
||||||
}
|
}
|
||||||
|
@ -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()
|
||||||
|
@ -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
|
||||||
|
@ -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() {
|
||||||
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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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) {
|
||||||
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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"
|
||||||
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user