initial plugeth work

refactor packages, flags, subscriptions

also DRY up builder tests

use mockgen
This commit is contained in:
Roy Crihfield 2023-06-23 20:42:55 +08:00
parent be3344850c
commit 9f967abfb9
81 changed files with 3160 additions and 2212 deletions

13
Makefile Normal file
View File

@ -0,0 +1,13 @@
MOCKGEN ?= mockgen
MOCKS_DIR := $(CURDIR)/test_helpers/mocks
mocks: $(MOCKS_DIR)/gen_backend.go
.PHONY: mocks
$(MOCKS_DIR)/gen_backend.go:
$(MOCKGEN) --package mocks --destination $@ \
github.com/openrelayxyz/plugeth-utils/core Backend,Downloader
docker: mocks
docker build . -t "cerc/plugeth-statediff:local"
.PHONY: docker

62
adapt/adapter.go Normal file
View File

@ -0,0 +1,62 @@
package adapt
import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie"
plugeth "github.com/openrelayxyz/plugeth-utils/core"
plugeth_params "github.com/openrelayxyz/plugeth-utils/restricted/params"
)
func StateAccount(a *plugeth.StateAccount) *types.StateAccount {
return &types.StateAccount{
Nonce: a.Nonce,
Balance: a.Balance,
Root: common.Hash(a.Root),
CodeHash: a.CodeHash,
}
}
func ChainConfig(cc *plugeth_params.ChainConfig) *params.ChainConfig {
return &params.ChainConfig{
ChainID: cc.ChainID,
HomesteadBlock: cc.HomesteadBlock,
DAOForkBlock: cc.DAOForkBlock,
DAOForkSupport: cc.DAOForkSupport,
EIP150Block: cc.EIP150Block,
EIP155Block: cc.EIP155Block,
EIP158Block: cc.EIP158Block,
ByzantiumBlock: cc.ByzantiumBlock,
ConstantinopleBlock: cc.ConstantinopleBlock,
PetersburgBlock: cc.PetersburgBlock,
IstanbulBlock: cc.IstanbulBlock,
MuirGlacierBlock: cc.MuirGlacierBlock,
BerlinBlock: cc.BerlinBlock,
LondonBlock: cc.LondonBlock,
}
}
func NodeIterator(it plugeth.NodeIterator) trie.NodeIterator {
return nodeiter{it}
}
type nodeiter struct {
plugeth.NodeIterator
}
func (it nodeiter) Hash() common.Hash {
return common.Hash(it.NodeIterator.Hash())
}
func (it nodeiter) Parent() common.Hash {
return common.Hash(it.NodeIterator.Parent())
}
func (it nodeiter) AddResolver(resolver trie.NodeResolver) {
r := func(owner plugeth.Hash, path []byte, hash plugeth.Hash) []byte {
return resolver(common.Hash(owner), path, common.Hash(hash))
}
it.NodeIterator.AddResolver(r)
}

157
api.go
View File

@ -19,112 +19,91 @@ package statediff
import ( import (
"context" "context"
"github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rpc" "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
// APIName is the namespace used for the state diffing service API // APIName is the namespace used for the state diffing service API
const APIName = "statediff" const APIName = "statediff"
// APIVersion is the version of the state diffing service API // APIVersion is the version of the state diffing service API
// TODO: match package version?
const APIVersion = "0.0.1" const APIVersion = "0.0.1"
// PublicStateDiffAPI provides an RPC subscription interface // PublicStateDiffAPI provides an RPC subscription interface
// that can be used to stream out state diffs as they // that can be used to stream out state diffs as they
// are produced by a full node // are produced by a full node
type PublicStateDiffAPI struct { type PublicAPI struct {
sds IService sds IService
} }
// NewPublicStateDiffAPI creates an rpc subscription interface for the underlying statediff service // NewPublicStateDiffAPI creates an rpc subscription interface for the underlying statediff service
func NewPublicStateDiffAPI(sds IService) *PublicStateDiffAPI { func NewPublicAPI(sds IService) *PublicAPI {
return &PublicStateDiffAPI{ return &PublicAPI{
sds: sds, sds: sds,
} }
} }
// Stream is the public method to setup a subscription that fires off statediff service payloads as they are created // Stream subscribes to statediff payloads as they are created.
func (api *PublicStateDiffAPI) Stream(ctx context.Context, params Params) (*rpc.Subscription, error) { func (api *PublicAPI) Stream(ctx context.Context, params Params) (<-chan Payload, error) {
// ensure that the RPC connection supports subscriptions payloadChan := make(chan Payload, chainEventChanSize)
notifier, supported := rpc.NotifierFromContext(ctx) clientChan := make(chan Payload, chainEventChanSize)
if !supported { quitChan := make(chan bool, 1)
return nil, rpc.ErrNotificationsUnsupported // subscribe to the service's payload broadcasts
} id := api.sds.Subscribe(payloadChan, quitChan, params)
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
go func() { go func() {
// subscribe to events from the statediff service defer close(clientChan)
payloadChannel := make(chan Payload, chainEventChanSize) defer close(payloadChan)
quitChan := make(chan bool, 1) defer func() {
api.sds.Subscribe(rpcSub.ID, payloadChannel, quitChan, params) if err := api.sds.Unsubscribe(id); err != nil {
// loop and await payloads and relay them to the subscriber with the notifier log.Error("Failed to unsubscribe from statediff service", "err", err)
}
}()
for { for {
select { select {
case payload := <-payloadChannel: case payload := <-payloadChan:
if err := notifier.Notify(rpcSub.ID, payload); err != nil { clientChan <- payload
log.Error("Failed to send state diff packet; error: " + err.Error()) case <-ctx.Done():
if err := api.sds.Unsubscribe(rpcSub.ID); err != nil { return
log.Error("Failed to unsubscribe from the state diff service; error: " + err.Error())
}
return
}
case err := <-rpcSub.Err():
if err != nil {
log.Error("State diff service rpcSub error: " + err.Error())
err = api.sds.Unsubscribe(rpcSub.ID)
if err != nil {
log.Error("Failed to unsubscribe from the state diff service; error: " + err.Error())
}
return
}
case <-quitChan: case <-quitChan:
// don't need to unsubscribe, service does so before sending the quit signal
return return
} }
} }
}() }()
return rpcSub, nil return clientChan, nil
} }
// StateDiffAt returns a state diff payload at the specific blockheight // StateDiffAt returns a state diff payload at the specific blockheight
func (api *PublicStateDiffAPI) StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error) { func (api *PublicAPI) StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error) {
return api.sds.StateDiffAt(blockNumber, params) return api.sds.StateDiffAt(blockNumber, params)
} }
// StateDiffFor returns a state diff payload for the specific blockhash // StateDiffFor returns a state diff payload for the specific blockhash
func (api *PublicStateDiffAPI) StateDiffFor(ctx context.Context, blockHash common.Hash, params Params) (*Payload, error) { func (api *PublicAPI) StateDiffFor(ctx context.Context, blockHash common.Hash, params Params) (*Payload, error) {
return api.sds.StateDiffFor(blockHash, params) return api.sds.StateDiffFor(blockHash, params)
} }
// StreamCodeAndCodeHash writes all of the codehash=>code pairs out to a websocket channel // StreamCodeAndCodeHash writes all of the codehash=>code pairs at a given block to a websocket channel.
func (api *PublicStateDiffAPI) StreamCodeAndCodeHash(ctx context.Context, blockNumber uint64) (*rpc.Subscription, error) { func (api *PublicAPI) StreamCodeAndCodeHash(ctx context.Context, blockNumber uint64) (<-chan types.CodeAndCodeHash, error) {
// ensure that the RPC connection supports subscriptions
notifier, supported := rpc.NotifierFromContext(ctx)
if !supported {
return nil, rpc.ErrNotificationsUnsupported
}
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
payloadChan := make(chan types.CodeAndCodeHash, chainEventChanSize) payloadChan := make(chan types.CodeAndCodeHash, chainEventChanSize)
quitChan := make(chan bool) clientChan := make(chan types.CodeAndCodeHash, chainEventChanSize)
quitChan := make(chan bool, 1)
api.sds.StreamCodeAndCodeHash(blockNumber, payloadChan, quitChan) api.sds.StreamCodeAndCodeHash(blockNumber, payloadChan, quitChan)
go func() { go func() {
defer close(clientChan)
defer close(payloadChan)
for { for {
select { select {
case payload := <-payloadChan: case payload := <-payloadChan:
if err := notifier.Notify(rpcSub.ID, payload); err != nil { clientChan <- payload
log.Error("Failed to send code and codehash packet", "err", err) case <-ctx.Done():
return
}
case err := <-rpcSub.Err():
log.Error("State diff service rpcSub error", "err", err)
return return
case <-quitChan: case <-quitChan:
return return
@ -132,11 +111,11 @@ func (api *PublicStateDiffAPI) StreamCodeAndCodeHash(ctx context.Context, blockN
} }
}() }()
return rpcSub, nil return clientChan, nil
} }
// WriteStateDiffAt writes a state diff object directly to DB at the specific blockheight // WriteStateDiffAt writes a state diff object directly to DB at the specific blockheight
func (api *PublicStateDiffAPI) WriteStateDiffAt(ctx context.Context, blockNumber uint64, params Params) JobID { func (api *PublicAPI) WriteStateDiffAt(ctx context.Context, blockNumber uint64, params Params) JobID {
var err error var err error
start, logger := countApiRequestBegin("writeStateDiffAt", blockNumber) start, logger := countApiRequestBegin("writeStateDiffAt", blockNumber)
defer countApiRequestEnd(start, logger, err) defer countApiRequestEnd(start, logger, err)
@ -145,62 +124,50 @@ func (api *PublicStateDiffAPI) WriteStateDiffAt(ctx context.Context, blockNumber
} }
// WriteStateDiffFor writes a state diff object directly to DB for the specific block hash // WriteStateDiffFor writes a state diff object directly to DB for the specific block hash
func (api *PublicStateDiffAPI) WriteStateDiffFor(ctx context.Context, blockHash common.Hash, params Params) error { func (api *PublicAPI) WriteStateDiffFor(ctx context.Context, blockHash common.Hash, params Params) error {
var err error var err error
start, logger := countApiRequestBegin("writeStateDiffFor", blockHash.Hex()) start, logger := countApiRequestBegin("writeStateDiffFor", blockHash.String())
defer countApiRequestEnd(start, logger, err) defer countApiRequestEnd(start, logger, err)
err = api.sds.WriteStateDiffFor(blockHash, params) err = api.sds.WriteStateDiffFor(blockHash, params)
return err return err
} }
// WatchAddress changes the list of watched addresses to which the direct indexing is restricted according to given operation // WatchAddress changes the list of watched addresses to which the direct indexing is restricted
func (api *PublicStateDiffAPI) WatchAddress(operation types.OperationType, args []types.WatchAddressArg) error { // for the given operation.
func (api *PublicAPI) WatchAddress(operation types.OperationType, args []types.WatchAddressArg) error {
return api.sds.WatchAddress(operation, args) return api.sds.WatchAddress(operation, args)
} }
// StreamWrites sets up a subscription that streams the status of completed calls to WriteStateDiff* // StreamWrites sets up a subscription that streams the status of completed calls to WriteStateDiff*
func (api *PublicStateDiffAPI) StreamWrites(ctx context.Context) (*rpc.Subscription, error) { func (api *PublicAPI) StreamWrites(ctx context.Context) (<-chan JobStatus, error) {
// ensure that the RPC connection supports subscriptions // subscribe to events from the statediff service
notifier, supported := rpc.NotifierFromContext(ctx) statusChan := make(chan JobStatus, chainEventChanSize)
if !supported { clientChan := make(chan JobStatus, chainEventChanSize)
return nil, rpc.ErrNotificationsUnsupported quitChan := make(chan bool, 1)
} id := api.sds.SubscribeWriteStatus(statusChan, quitChan)
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
go func() { go func() {
// subscribe to events from the statediff service
statusChan := make(chan JobStatus, chainEventChanSize)
quitChan := make(chan bool, 1)
api.sds.SubscribeWriteStatus(rpcSub.ID, statusChan, quitChan)
var err error
defer func() { defer func() {
if err = api.sds.UnsubscribeWriteStatus(rpcSub.ID); err != nil { close(statusChan)
log.Error("Failed to unsubscribe from job status stream: " + err.Error()) close(clientChan)
if err := api.sds.UnsubscribeWriteStatus(id); err != nil {
log.Error("Failed to unsubscribe from job status stream", "error", err)
} }
}() }()
// loop and await payloads and relay them to the subscriber with the notifier
for { for {
select { select {
case status := <-statusChan: case status := <-statusChan:
if err = notifier.Notify(rpcSub.ID, status); err != nil { clientChan <- status
log.Error("Failed to send job status; error: " + err.Error()) case <-ctx.Done():
return log.Error("Error from context", "error", ctx.Err())
} return
case err = <-rpcSub.Err():
if err != nil {
log.Error("statediff_StreamWrites RPC subscription error: " + err.Error())
return
}
case <-quitChan: case <-quitChan:
// don't need to unsubscribe, service does so before sending the quit signal
return return
} }
} }
}() }()
return rpcSub, nil return clientChan, nil
} }

View File

@ -21,80 +21,86 @@ package statediff
import ( import (
"bytes" "bytes"
"encoding/hex"
"fmt" "fmt"
"time" "time"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
metrics2 "github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ethereum/go-ethereum/statediff/trie_helpers"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
// "github.com/cerc-io/plugeth-statediff/adapt"
"github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
"github.com/cerc-io/plugeth-statediff/trie_helpers"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
var ( var (
emptyNode, _ = rlp.EncodeToBytes(&[]byte{}) emptyNode, _ = rlp.EncodeToBytes(&[]byte{})
emptyContractRoot = crypto.Keccak256Hash(emptyNode) emptyContractRoot = crypto.Keccak256Hash(emptyNode)
nullCodeHash = crypto.Keccak256Hash([]byte{}).Bytes() nullCodeHash = crypto.Keccak256Hash([]byte{}).Bytes()
nullNodeHash = common.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000")
zeroHashBytes = utils.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000")
// zeroHash = core.HexToHash("0000000000000000000000000000000000000000000000000000000000000000")
) )
// Builder interface exposes the method for building a state diff between two blocks // Builder interface exposes the method for building a state diff between two blocks
type Builder interface { type Builder interface {
BuildStateDiffObject(args Args, params Params) (types2.StateObject, error) BuildStateDiffObject(args Args, params Params) (sdtypes.StateObject, error)
WriteStateDiffObject(args Args, params Params, output types2.StateNodeSink, ipldOutput types2.IPLDSink) error WriteStateDiffObject(args Args, params Params, output sdtypes.StateNodeSink, ipldOutput sdtypes.IPLDSink) error
} }
type StateDiffBuilder struct { type StateDiffBuilder struct {
StateCache state.Database // StateCache state.Database
StateCache StateView
} }
type IterPair struct { type IterPair struct {
Older, Newer trie.NodeIterator Older, Newer trie.NodeIterator
} }
func StateNodeAppender(nodes *[]types2.StateLeafNode) types2.StateNodeSink { func StateNodeAppender(nodes *[]sdtypes.StateLeafNode) sdtypes.StateNodeSink {
return func(node types2.StateLeafNode) error { return func(node sdtypes.StateLeafNode) error {
*nodes = append(*nodes, node) *nodes = append(*nodes, node)
return nil return nil
} }
} }
func StorageNodeAppender(nodes *[]types2.StorageLeafNode) types2.StorageNodeSink { func StorageNodeAppender(nodes *[]sdtypes.StorageLeafNode) sdtypes.StorageNodeSink {
return func(node types2.StorageLeafNode) error { return func(node sdtypes.StorageLeafNode) error {
*nodes = append(*nodes, node) *nodes = append(*nodes, node)
return nil return nil
} }
} }
func IPLDMappingAppender(iplds *[]types2.IPLD) types2.IPLDSink { func IPLDMappingAppender(iplds *[]sdtypes.IPLD) sdtypes.IPLDSink {
return func(c types2.IPLD) error { return func(c sdtypes.IPLD) error {
*iplds = append(*iplds, c) *iplds = append(*iplds, c)
return nil return nil
} }
} }
// NewBuilder is used to create a statediff builder // NewBuilder is used to create a statediff builder
func NewBuilder(stateCache state.Database) Builder { func NewBuilder(stateCache StateView) Builder {
return &StateDiffBuilder{ return &StateDiffBuilder{
StateCache: stateCache, // state cache is safe for concurrent reads StateCache: stateCache, // state cache is safe for concurrent reads
} }
} }
// BuildStateDiffObject builds a statediff object from two blocks and the provided parameters // BuildStateDiffObject builds a statediff object from two blocks and the provided parameters
func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (types2.StateObject, error) { func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (sdtypes.StateObject, error) {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStateDiffObjectTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildStateDiffObjectTimer)
var stateNodes []types2.StateLeafNode var stateNodes []sdtypes.StateLeafNode
var iplds []types2.IPLD var iplds []sdtypes.IPLD
err := sdb.WriteStateDiffObject(args, params, StateNodeAppender(&stateNodes), IPLDMappingAppender(&iplds)) err := sdb.WriteStateDiffObject(args, params, StateNodeAppender(&stateNodes), IPLDMappingAppender(&iplds))
if err != nil { if err != nil {
return types2.StateObject{}, err return sdtypes.StateObject{}, err
} }
return types2.StateObject{ return sdtypes.StateObject{
BlockHash: args.BlockHash, BlockHash: args.BlockHash,
BlockNumber: args.BlockNumber, BlockNumber: args.BlockNumber,
Nodes: stateNodes, Nodes: stateNodes,
@ -103,9 +109,9 @@ func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (typ
} }
// WriteStateDiffObject writes a statediff object to output sinks // WriteStateDiffObject writes a statediff object to output sinks
func (sdb *StateDiffBuilder) WriteStateDiffObject(args Args, params Params, output types2.StateNodeSink, func (sdb *StateDiffBuilder) WriteStateDiffObject(args Args, params Params, output sdtypes.StateNodeSink,
ipldOutput types2.IPLDSink) error { ipldOutput sdtypes.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.WriteStateDiffObjectTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.WriteStateDiffObjectTimer)
// Load tries for old and new states // Load tries for old and new states
oldTrie, err := sdb.StateCache.OpenTrie(args.OldStateRoot) oldTrie, err := sdb.StateCache.OpenTrie(args.OldStateRoot)
if err != nil { if err != nil {
@ -131,14 +137,14 @@ func (sdb *StateDiffBuilder) WriteStateDiffObject(args Args, params Params, outp
}, },
} }
logger := log.New("hash", args.BlockHash.Hex(), "number", args.BlockNumber) logger := log.New("hash", args.BlockHash.String(), "number", args.BlockNumber)
return sdb.BuildStateDiffWithIntermediateStateNodes(iterPairs, params, output, ipldOutput, logger, nil) return sdb.BuildStateDiffWithIntermediateStateNodes(iterPairs, params, output, ipldOutput, logger, nil)
} }
func (sdb *StateDiffBuilder) BuildStateDiffWithIntermediateStateNodes(iterPairs []IterPair, params Params, func (sdb *StateDiffBuilder) BuildStateDiffWithIntermediateStateNodes(iterPairs []IterPair, params Params,
output types2.StateNodeSink, ipldOutput types2.IPLDSink, logger log.Logger, prefixPath []byte) error { output sdtypes.StateNodeSink, ipldOutput sdtypes.IPLDSink, logger log.Logger, prefixPath []byte) error {
logger.Debug("statediff BEGIN BuildStateDiffWithIntermediateStateNodes") logger.Debug("statediff BEGIN BuildStateDiffWithIntermediateStateNodes")
defer metrics2.ReportAndUpdateDuration("statediff END BuildStateDiffWithIntermediateStateNodes", time.Now(), logger, metrics2.IndexerMetrics.BuildStateDiffWithIntermediateStateNodesTimer) defer metrics.ReportAndUpdateDuration("statediff END BuildStateDiffWithIntermediateStateNodes", time.Now(), logger, metrics.IndexerMetrics.BuildStateDiffWithIntermediateStateNodesTimer)
// collect a slice of all the nodes that were touched and exist at B (B-A) // collect a slice of all the nodes that were touched and exist at B (B-A)
// a map of their leafkey to all the accounts that were touched and exist at B // a map of their leafkey to all the accounts that were touched and exist at B
// and a slice of all the paths for the nodes in both of the above sets // and a slice of all the paths for the nodes in both of the above sets
@ -191,22 +197,23 @@ func (sdb *StateDiffBuilder) BuildStateDiffWithIntermediateStateNodes(iterPairs
// a mapping of their leafkeys to all the accounts that exist in a different state at B than A // a mapping of their leafkeys to all the accounts that exist in a different state at B than A
// and a slice of the paths for all of the nodes included in both // and a slice of the paths for all of the nodes included in both
func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator, func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator,
watchedAddressesLeafPaths [][]byte, output types2.IPLDSink, logger log.Logger, prefixPath []byte) (types2.AccountMap, error) { watchedAddressesLeafPaths [][]byte, output sdtypes.IPLDSink, logger log.Logger, prefixPath []byte) (sdtypes.AccountMap, error) {
logger.Debug("statediff BEGIN createdAndUpdatedState") logger.Debug("statediff BEGIN createdAndUpdatedState")
defer metrics2.ReportAndUpdateDuration("statediff END createdAndUpdatedState", time.Now(), logger, metrics2.IndexerMetrics.CreatedAndUpdatedStateTimer) defer metrics.ReportAndUpdateDuration("statediff END createdAndUpdatedState", time.Now(), logger, metrics.IndexerMetrics.CreatedAndUpdatedStateTimer)
diffAccountsAtB := make(types2.AccountMap) diffAccountsAtB := make(sdtypes.AccountMap)
watchingAddresses := len(watchedAddressesLeafPaths) > 0
// cache the RLP of the previous node, so when we hit a leaf we have the parent (containing) node
var prevBlob []byte
it, itCount := trie.NewDifferenceIterator(a, b) it, itCount := trie.NewDifferenceIterator(a, b)
for it.Next(true) { for it.Next(true) {
// ignore node if it is not along paths of interest // ignore node if it is not along paths of interest
if watchingAddresses && !isValidPrefixPath(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) { if !isWatchedPathPrefix(watchedAddressesLeafPaths, it.Path()) {
continue continue
} }
// index values by leaf key // index values by leaf key
if it.Leaf() { if it.Leaf() {
// if it is a "value" node, we will index the value by leaf key // if it is a "value" node, we will index the value by leaf key
accountW, err := sdb.processStateValueNode(it, watchedAddressesLeafPaths, prefixPath) accountW, err := sdb.processStateValueNode(it, prevBlob)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -216,14 +223,18 @@ func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator,
// for now, just add it to diffAccountsAtB // for now, just add it to diffAccountsAtB
// we will compare to diffAccountsAtA to determine which diffAccountsAtB // we will compare to diffAccountsAtA to determine which diffAccountsAtB
// were creations and which were updates and also identify accounts that were removed going A->B // were creations and which were updates and also identify accounts that were removed going A->B
diffAccountsAtB[common.Bytes2Hex(accountW.LeafKey)] = *accountW diffAccountsAtB[hex.EncodeToString(accountW.LeafKey)] = *accountW
} else { // trie nodes will be written to blockstore only } else {
// reminder that this includes leaf nodes, since the geth iterator.Leaf() actually signifies a "value" node // trie nodes will be written to blockstore only
if bytes.Equal(it.Hash().Bytes(), nullNodeHash) { // reminder that this includes leaf nodes, since the geth iterator.Leaf() actually
// signifies a "value" node
// TODO: use Hash type
if bytes.Equal(it.Hash().Bytes(), zeroHashBytes) {
continue continue
} }
nodeVal := make([]byte, len(it.NodeBlob())) nodeVal := make([]byte, len(it.NodeBlob()))
copy(nodeVal, it.NodeBlob()) copy(nodeVal, it.NodeBlob())
// if doing a selective diff, we need to ensure this is a watched path
if len(watchedAddressesLeafPaths) > 0 { if len(watchedAddressesLeafPaths) > 0 {
var elements []interface{} var elements []interface{}
if err := rlp.DecodeBytes(nodeVal, &elements); err != nil { if err := rlp.DecodeBytes(nodeVal, &elements); err != nil {
@ -233,103 +244,76 @@ func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator,
if err != nil { if err != nil {
return nil, err return nil, err
} }
if ok { partialPath := utils.CompactToHex(elements[0].([]byte))
nodePath := append(prefixPath, it.Path()...) valueNodePath := append(it.Path(), partialPath...)
partialPath := trie.CompactToHex(elements[0].([]byte)) if ok && !isWatchedPath(watchedAddressesLeafPaths, valueNodePath) {
valueNodePath := append(nodePath, partialPath...) continue
if !isWatchedAddress(watchedAddressesLeafPaths, valueNodePath) {
continue
}
} }
} }
nodeHash := make([]byte, len(it.Hash().Bytes())) if err := output(sdtypes.IPLD{
copy(nodeHash, it.Hash().Bytes()) CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, it.Hash().Bytes()).String(),
if err := output(types2.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, nodeHash).String(),
Content: nodeVal, Content: nodeVal,
}); err != nil { }); err != nil {
return nil, err return nil, err
} }
prevBlob = nodeVal
} }
} }
logger.Debug("statediff COUNTS createdAndUpdatedStateWithIntermediateNodes", "it", itCount, "diffAccountsAtB", len(diffAccountsAtB)) logger.Debug("statediff COUNTS createdAndUpdatedStateWithIntermediateNodes", "it", itCount, "diffAccountsAtB", len(diffAccountsAtB))
metrics2.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(*itCount)) metrics.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(*itCount))
return diffAccountsAtB, it.Error() return diffAccountsAtB, it.Error()
} }
// reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something that actually exists in an MMPT // reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something
func (sdb *StateDiffBuilder) processStateValueNode(it trie.NodeIterator, watchedAddressesLeafPaths [][]byte, prefixPath []byte) (*types2.AccountWrapper, error) { // that actually exists in an MMPT
// skip if it is not a watched address func (sdb *StateDiffBuilder) processStateValueNode(it trie.NodeIterator, parentBlob []byte) (*sdtypes.AccountWrapper, error) {
// If we aren't watching any specific addresses, we are watching everything encodedPath := utils.HexToCompact(it.Path())
if len(watchedAddressesLeafPaths) > 0 && !isWatchedAddress(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) {
return nil, nil
}
// since this is a "value node", we need to move up to the "parent" node which is the actual leaf node
// it should be in the fastcache since it necessarily was recently accessed to reach the current node
parentNodeRLP, err := sdb.StateCache.TrieDB().Node(it.Parent())
if err != nil {
return nil, err
}
var nodeElements []interface{}
if err = rlp.DecodeBytes(parentNodeRLP, &nodeElements); err != nil {
return nil, err
}
parentSubPath := make([]byte, len(it.ParentPath()))
copy(parentSubPath, it.ParentPath())
parentPath := append(prefixPath, parentSubPath...)
partialPath := trie.CompactToHex(nodeElements[0].([]byte))
valueNodePath := append(parentPath, partialPath...)
encodedPath := trie.HexToCompact(valueNodePath)
leafKey := encodedPath[1:] leafKey := encodedPath[1:]
var account types.StateAccount var account types.StateAccount
accountRLP := make([]byte, len(it.LeafBlob())) if err := rlp.DecodeBytes(it.LeafBlob(), &account); err != nil {
copy(accountRLP, it.LeafBlob())
if err := rlp.DecodeBytes(accountRLP, &account); err != nil {
return nil, fmt.Errorf("error decoding account for leaf value at leaf key %x\nerror: %v", leafKey, err) return nil, fmt.Errorf("error decoding account for leaf value at leaf key %x\nerror: %v", leafKey, err)
} }
return &types2.AccountWrapper{ return &sdtypes.AccountWrapper{
LeafKey: leafKey, LeafKey: leafKey,
Account: &account, Account: &account,
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(parentNodeRLP)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(parentBlob)).String(),
}, nil }, nil
} }
// deletedOrUpdatedState returns a slice of all the pathes that are emptied at B // deletedOrUpdatedState returns a slice of all the pathes that are emptied at B
// and a mapping of their leafkeys to all the accounts that exist in a different state at A than B // and a mapping of their leafkeys to all the accounts that exist in a different state at A than B
func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffAccountsAtB types2.AccountMap, func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffAccountsAtB sdtypes.AccountMap,
watchedAddressesLeafPaths [][]byte, output types2.StateNodeSink, logger log.Logger, prefixPath []byte) (types2.AccountMap, error) { watchedAddressesLeafPaths [][]byte, output sdtypes.StateNodeSink, logger log.Logger, prefixPath []byte) (sdtypes.AccountMap, error) {
logger.Debug("statediff BEGIN deletedOrUpdatedState") logger.Debug("statediff BEGIN deletedOrUpdatedState")
defer metrics2.ReportAndUpdateDuration("statediff END deletedOrUpdatedState", time.Now(), logger, metrics2.IndexerMetrics.DeletedOrUpdatedStateTimer) defer metrics.ReportAndUpdateDuration("statediff END deletedOrUpdatedState", time.Now(), logger, metrics.IndexerMetrics.DeletedOrUpdatedStateTimer)
diffAccountAtA := make(types2.AccountMap) diffAccountAtA := make(sdtypes.AccountMap)
watchingAddresses := len(watchedAddressesLeafPaths) > 0
var prevBlob []byte
it, _ := trie.NewDifferenceIterator(b, a) it, _ := trie.NewDifferenceIterator(b, a)
for it.Next(true) { for it.Next(true) {
// ignore node if it is not along paths of interest if !isWatchedPathPrefix(watchedAddressesLeafPaths, it.Path()) {
if watchingAddresses && !isValidPrefixPath(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) {
continue continue
} }
if it.Leaf() { if it.Leaf() {
accountW, err := sdb.processStateValueNode(it, watchedAddressesLeafPaths, prefixPath) accountW, err := sdb.processStateValueNode(it, prevBlob)
if err != nil { if err != nil {
return nil, err return nil, err
} }
if accountW == nil { if accountW == nil {
continue continue
} }
leafKey := common.Bytes2Hex(accountW.LeafKey) leafKey := hex.EncodeToString(accountW.LeafKey)
diffAccountAtA[leafKey] = *accountW diffAccountAtA[leafKey] = *accountW
// if this node's leaf key did not show up in diffAccountsAtB // if this node's leaf key did not show up in diffAccountsAtB
// that means the account was deleted // that means the account was deleted
// in that case, emit an empty "removed" diff state node // in that case, emit an empty "removed" diff state node
// include empty "removed" diff storage nodes for all the storage slots // include empty "removed" diff storage nodes for all the storage slots
if _, ok := diffAccountsAtB[leafKey]; !ok { if _, ok := diffAccountsAtB[leafKey]; !ok {
diff := types2.StateLeafNode{ diff := sdtypes.StateLeafNode{
AccountWrapper: types2.AccountWrapper{ AccountWrapper: sdtypes.AccountWrapper{
Account: nil, Account: nil,
LeafKey: accountW.LeafKey, LeafKey: accountW.LeafKey,
CID: shared.RemovedNodeStateCID, CID: shared.RemovedNodeStateCID,
@ -337,7 +321,7 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffA
Removed: true, Removed: true,
} }
storageDiff := make([]types2.StorageLeafNode, 0) storageDiff := make([]sdtypes.StorageLeafNode, 0)
err := sdb.buildRemovedAccountStorageNodes(accountW.Account.Root, StorageNodeAppender(&storageDiff)) err := sdb.buildRemovedAccountStorageNodes(accountW.Account.Root, StorageNodeAppender(&storageDiff))
if err != nil { if err != nil {
return nil, fmt.Errorf("failed building storage diffs for removed state account with key %x\r\nerror: %v", leafKey, err) return nil, fmt.Errorf("failed building storage diffs for removed state account with key %x\r\nerror: %v", leafKey, err)
@ -347,6 +331,9 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffA
return nil, err return nil, err
} }
} }
} else {
prevBlob = make([]byte, len(it.NodeBlob()))
copy(prevBlob, it.NodeBlob())
} }
} }
return diffAccountAtA, it.Error() return diffAccountAtA, it.Error()
@ -356,25 +343,25 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffA
// to generate the statediff node objects for all of the accounts that existed at both A and B but in different states // to generate the statediff node objects for all of the accounts that existed at both A and B but in different states
// needs to be called before building account creations and deletions as this mutates // needs to be called before building account creations and deletions as this mutates
// those account maps to remove the accounts which were updated // those account maps to remove the accounts which were updated
func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions types2.AccountMap, updatedKeys []string, func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions sdtypes.AccountMap, updatedKeys []string,
output types2.StateNodeSink, ipldOutput types2.IPLDSink, logger log.Logger) error { output sdtypes.StateNodeSink, ipldOutput sdtypes.IPLDSink, logger log.Logger) error {
logger.Debug("statediff BEGIN buildAccountUpdates", "creations", len(creations), "deletions", len(deletions), "updatedKeys", len(updatedKeys)) logger.Debug("statediff BEGIN buildAccountUpdates", "creations", len(creations), "deletions", len(deletions), "updatedKeys", len(updatedKeys))
defer metrics2.ReportAndUpdateDuration("statediff END buildAccountUpdates ", time.Now(), logger, metrics2.IndexerMetrics.BuildAccountUpdatesTimer) defer metrics.ReportAndUpdateDuration("statediff END buildAccountUpdates ", time.Now(), logger, metrics.IndexerMetrics.BuildAccountUpdatesTimer)
var err error var err error
for _, key := range updatedKeys { for _, key := range updatedKeys {
createdAcc := creations[key] createdAcc := creations[key]
deletedAcc := deletions[key] deletedAcc := deletions[key]
storageDiff := make([]types2.StorageLeafNode, 0) storageDiff := make([]sdtypes.StorageLeafNode, 0)
if deletedAcc.Account != nil && createdAcc.Account != nil { if deletedAcc.Account != nil && createdAcc.Account != nil {
oldSR := deletedAcc.Account.Root
newSR := createdAcc.Account.Root
err = sdb.buildStorageNodesIncremental( err = sdb.buildStorageNodesIncremental(
oldSR, newSR, StorageNodeAppender(&storageDiff), ipldOutput) deletedAcc.Account.Root, createdAcc.Account.Root,
StorageNodeAppender(&storageDiff), ipldOutput,
)
if err != nil { if err != nil {
return fmt.Errorf("failed building incremental storage diffs for account with leafkey %s\r\nerror: %v", key, err) return fmt.Errorf("failed building incremental storage diffs for account with leafkey %s\r\nerror: %v", key, err)
} }
} }
if err = output(types2.StateLeafNode{ if err = output(sdtypes.StateLeafNode{
AccountWrapper: createdAcc, AccountWrapper: createdAcc,
Removed: false, Removed: false,
StorageDiff: storageDiff, StorageDiff: storageDiff,
@ -390,18 +377,18 @@ func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions types2.Acc
// buildAccountCreations returns the statediff node objects for all the accounts that exist at B but not at A // buildAccountCreations returns the statediff node objects for all the accounts that exist at B but not at A
// it also returns the code and codehash for created contract accounts // it also returns the code and codehash for created contract accounts
func (sdb *StateDiffBuilder) buildAccountCreations(accounts types2.AccountMap, output types2.StateNodeSink, func (sdb *StateDiffBuilder) buildAccountCreations(accounts sdtypes.AccountMap, output sdtypes.StateNodeSink,
ipldOutput types2.IPLDSink, logger log.Logger) error { ipldOutput sdtypes.IPLDSink, logger log.Logger) error {
logger.Debug("statediff BEGIN buildAccountCreations") logger.Debug("statediff BEGIN buildAccountCreations")
defer metrics2.ReportAndUpdateDuration("statediff END buildAccountCreations", time.Now(), logger, metrics2.IndexerMetrics.BuildAccountCreationsTimer) defer metrics.ReportAndUpdateDuration("statediff END buildAccountCreations", time.Now(), logger, metrics.IndexerMetrics.BuildAccountCreationsTimer)
for _, val := range accounts { for _, val := range accounts {
diff := types2.StateLeafNode{ diff := sdtypes.StateLeafNode{
AccountWrapper: val, AccountWrapper: val,
Removed: false, Removed: false,
} }
if !bytes.Equal(val.Account.CodeHash, nullCodeHash) { if !bytes.Equal(val.Account.CodeHash, nullCodeHash) {
// For contract creations, any storage node contained is a diff // For contract creations, any storage node contained is a diff
storageDiff := make([]types2.StorageLeafNode, 0) storageDiff := make([]sdtypes.StorageLeafNode, 0)
err := sdb.buildStorageNodesEventual(val.Account.Root, StorageNodeAppender(&storageDiff), ipldOutput) err := sdb.buildStorageNodesEventual(val.Account.Root, StorageNodeAppender(&storageDiff), ipldOutput)
if err != nil { if err != nil {
return fmt.Errorf("failed building eventual storage diffs for node with leaf key %x\r\nerror: %v", val.LeafKey, err) return fmt.Errorf("failed building eventual storage diffs for node with leaf key %x\r\nerror: %v", val.LeafKey, err)
@ -409,12 +396,12 @@ func (sdb *StateDiffBuilder) buildAccountCreations(accounts types2.AccountMap, o
diff.StorageDiff = storageDiff diff.StorageDiff = storageDiff
// emit codehash => code mappings for contract // emit codehash => code mappings for contract
codeHash := common.BytesToHash(val.Account.CodeHash) codeHash := common.BytesToHash(val.Account.CodeHash)
code, err := sdb.StateCache.ContractCode(common.Hash{}, codeHash) code, err := sdb.StateCache.ContractCode(codeHash)
if err != nil { if err != nil {
return fmt.Errorf("failed to retrieve code for codehash %s\r\n error: %v", codeHash.String(), err) return fmt.Errorf("failed to retrieve code for codehash %s\r\n error: %v", codeHash.String(), err)
} }
if err := ipldOutput(types2.IPLD{ if err := ipldOutput(sdtypes.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.RawBinary, codeHash.Bytes()).String(), CID: ipld.Keccak256ToCid(ipld.RawBinary, codeHash.Bytes()).String(),
Content: code, Content: code,
}); err != nil { }); err != nil {
return err return err
@ -430,13 +417,13 @@ func (sdb *StateDiffBuilder) buildAccountCreations(accounts types2.AccountMap, o
// buildStorageNodesEventual builds the storage diff node objects for a created account // buildStorageNodesEventual builds the storage diff node objects for a created account
// i.e. it returns all the storage nodes at this state, since there is no previous state // i.e. it returns all the storage nodes at this state, since there is no previous state
func (sdb *StateDiffBuilder) buildStorageNodesEventual(sr common.Hash, output types2.StorageNodeSink, func (sdb *StateDiffBuilder) buildStorageNodesEventual(sr common.Hash, output sdtypes.StorageNodeSink,
ipldOutput types2.IPLDSink) error { ipldOutput sdtypes.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesEventualTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildStorageNodesEventualTimer)
if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) { if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) {
return nil return nil
} }
log.Debug("Storage Root For Eventual Diff", "root", sr.Hex()) log.Debug("Storage Root For Eventual Diff", "root", sr.String())
sTrie, err := sdb.StateCache.OpenTrie(sr) sTrie, err := sdb.StateCache.OpenTrie(sr)
if err != nil { if err != nil {
log.Info("error in build storage diff eventual", "error", err) log.Info("error in build storage diff eventual", "error", err)
@ -452,12 +439,14 @@ func (sdb *StateDiffBuilder) buildStorageNodesEventual(sr common.Hash, output ty
// buildStorageNodesFromTrie returns all the storage diff node objects in the provided node interator // buildStorageNodesFromTrie returns all the storage diff node objects in the provided node interator
// including intermediate nodes can be turned on or off // including intermediate nodes can be turned on or off
func (sdb *StateDiffBuilder) buildStorageNodesFromTrie(it trie.NodeIterator, output types2.StorageNodeSink, func (sdb *StateDiffBuilder) buildStorageNodesFromTrie(it trie.NodeIterator, output sdtypes.StorageNodeSink,
ipldOutput types2.IPLDSink) error { ipldOutput sdtypes.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesFromTrieTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildStorageNodesFromTrieTimer)
var prevBlob []byte
for it.Next(true) { for it.Next(true) {
if it.Leaf() { if it.Leaf() {
storageLeafNode, err := sdb.processStorageValueNode(it) storageLeafNode, err := sdb.processStorageValueNode(it, prevBlob)
if err != nil { if err != nil {
return err return err
} }
@ -469,46 +458,47 @@ func (sdb *StateDiffBuilder) buildStorageNodesFromTrie(it trie.NodeIterator, out
copy(nodeVal, it.NodeBlob()) copy(nodeVal, it.NodeBlob())
nodeHash := make([]byte, len(it.Hash().Bytes())) nodeHash := make([]byte, len(it.Hash().Bytes()))
copy(nodeHash, it.Hash().Bytes()) copy(nodeHash, it.Hash().Bytes())
if err := ipldOutput(types2.IPLD{ if err := ipldOutput(sdtypes.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, nodeHash).String(), CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, nodeHash).String(),
Content: nodeVal, Content: nodeVal,
}); err != nil { }); err != nil {
return err return err
} }
prevBlob = nodeVal
} }
} }
return it.Error() return it.Error()
} }
// reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something that actually exists in an MMPT // reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something
func (sdb *StateDiffBuilder) processStorageValueNode(it trie.NodeIterator) (types2.StorageLeafNode, error) { // that actually exists in an MMPT
// skip if it is not a watched address func (sdb *StateDiffBuilder) processStorageValueNode(it trie.NodeIterator, parentBlob []byte) (sdtypes.StorageLeafNode, error) {
leafKey := make([]byte, len(it.LeafKey())) leafKey := make([]byte, len(it.LeafKey()))
copy(leafKey, it.LeafKey()) copy(leafKey, it.LeafKey())
value := make([]byte, len(it.LeafBlob())) value := make([]byte, len(it.LeafBlob()))
copy(value, it.LeafBlob()) copy(value, it.LeafBlob())
// since this is a "value node", we need to move up to the "parent" node which is the actual leaf node // // since this is a "value node", we need to move up to the "parent" node which is the actual leaf node
// it should be in the fastcache since it necessarily was recently accessed to reach the current node // // it should be in the fastcache since it necessarily was recently accessed to reach the current node
parentNodeRLP, err := sdb.StateCache.TrieDB().Node(it.Parent()) // parentNodeRLP, err := sdb.StateCache.TrieDB().Node(it.Parent())
if err != nil { // if err != nil {
return types2.StorageLeafNode{}, err // return sdtypes.StorageLeafNode{}, err
} // }
return types2.StorageLeafNode{ return sdtypes.StorageLeafNode{
LeafKey: leafKey, LeafKey: leafKey,
Value: value, Value: value,
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, crypto.Keccak256(parentNodeRLP)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(parentBlob)).String(),
}, nil }, nil
} }
// buildRemovedAccountStorageNodes builds the "removed" diffs for all the storage nodes for a destroyed account // buildRemovedAccountStorageNodes builds the "removed" diffs for all the storage nodes for a destroyed account
func (sdb *StateDiffBuilder) buildRemovedAccountStorageNodes(sr common.Hash, output types2.StorageNodeSink) error { func (sdb *StateDiffBuilder) buildRemovedAccountStorageNodes(sr common.Hash, output sdtypes.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildRemovedAccountStorageNodesTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildRemovedAccountStorageNodesTimer)
if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) { if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) {
return nil return nil
} }
log.Debug("Storage Root For Removed Diffs", "root", sr.Hex()) log.Debug("Storage Root For Removed Diffs", "root", sr.String())
sTrie, err := sdb.StateCache.OpenTrie(sr) sTrie, err := sdb.StateCache.OpenTrie(sr)
if err != nil { if err != nil {
log.Info("error in build removed account storage diffs", "error", err) log.Info("error in build removed account storage diffs", "error", err)
@ -523,13 +513,13 @@ func (sdb *StateDiffBuilder) buildRemovedAccountStorageNodes(sr common.Hash, out
} }
// buildRemovedStorageNodesFromTrie returns diffs for all the storage nodes in the provided node interator // buildRemovedStorageNodesFromTrie returns diffs for all the storage nodes in the provided node interator
func (sdb *StateDiffBuilder) buildRemovedStorageNodesFromTrie(it trie.NodeIterator, output types2.StorageNodeSink) error { func (sdb *StateDiffBuilder) buildRemovedStorageNodesFromTrie(it trie.NodeIterator, output sdtypes.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildRemovedStorageNodesFromTrieTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildRemovedStorageNodesFromTrieTimer)
for it.Next(true) { for it.Next(true) {
if it.Leaf() { // only leaf values are indexed, don't need to demarcate removed intermediate nodes if it.Leaf() { // only leaf values are indexed, don't need to demarcate removed intermediate nodes
leafKey := make([]byte, len(it.LeafKey())) leafKey := make([]byte, len(it.LeafKey()))
copy(leafKey, it.LeafKey()) copy(leafKey, it.LeafKey())
if err := output(types2.StorageLeafNode{ if err := output(sdtypes.StorageLeafNode{
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
LeafKey: leafKey, LeafKey: leafKey,
@ -543,18 +533,18 @@ func (sdb *StateDiffBuilder) buildRemovedStorageNodesFromTrie(it trie.NodeIterat
} }
// buildStorageNodesIncremental builds the storage diff node objects for all nodes that exist in a different state at B than A // buildStorageNodesIncremental builds the storage diff node objects for all nodes that exist in a different state at B than A
func (sdb *StateDiffBuilder) buildStorageNodesIncremental(oldSR common.Hash, newSR common.Hash, output types2.StorageNodeSink, func (sdb *StateDiffBuilder) buildStorageNodesIncremental(oldroot common.Hash, newroot common.Hash, output sdtypes.StorageNodeSink,
ipldOutput types2.IPLDSink) error { ipldOutput sdtypes.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesIncrementalTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.BuildStorageNodesIncrementalTimer)
if bytes.Equal(newSR.Bytes(), oldSR.Bytes()) { if bytes.Equal(newroot.Bytes(), oldroot.Bytes()) {
return nil return nil
} }
log.Trace("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex()) log.Trace("Storage Roots for Incremental Diff", "old", oldroot.String(), "new", newroot.String())
oldTrie, err := sdb.StateCache.OpenTrie(oldSR) oldTrie, err := sdb.StateCache.OpenTrie(oldroot)
if err != nil { if err != nil {
return err return err
} }
newTrie, err := sdb.StateCache.OpenTrie(newSR) newTrie, err := sdb.StateCache.OpenTrie(newroot)
if err != nil { if err != nil {
return err return err
} }
@ -572,42 +562,45 @@ func (sdb *StateDiffBuilder) buildStorageNodesIncremental(oldSR common.Hash, new
return nil return nil
} }
func (sdb *StateDiffBuilder) createdAndUpdatedStorage(a, b trie.NodeIterator, output types2.StorageNodeSink, func (sdb *StateDiffBuilder) createdAndUpdatedStorage(a, b trie.NodeIterator, output sdtypes.StorageNodeSink,
ipldOutput types2.IPLDSink) (map[string]bool, error) { ipldOutput sdtypes.IPLDSink) (map[string]bool, error) {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.CreatedAndUpdatedStorageTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.CreatedAndUpdatedStorageTimer)
diffSlotsAtB := make(map[string]bool) diffSlotsAtB := make(map[string]bool)
var prevBlob []byte
it, _ := trie.NewDifferenceIterator(a, b) it, _ := trie.NewDifferenceIterator(a, b)
for it.Next(true) { for it.Next(true) {
if it.Leaf() { if it.Leaf() {
storageLeafNode, err := sdb.processStorageValueNode(it) storageLeafNode, err := sdb.processStorageValueNode(it, prevBlob)
if err != nil { if err != nil {
return nil, err return nil, err
} }
if err := output(storageLeafNode); err != nil { if err := output(storageLeafNode); err != nil {
return nil, err return nil, err
} }
diffSlotsAtB[common.Bytes2Hex(storageLeafNode.LeafKey)] = true diffSlotsAtB[hex.EncodeToString(storageLeafNode.LeafKey)] = true
} else { } else {
if bytes.Equal(it.Hash().Bytes(), nullNodeHash) { if bytes.Equal(it.Hash().Bytes(), zeroHashBytes) {
continue continue
} }
nodeVal := make([]byte, len(it.NodeBlob())) nodeVal := make([]byte, len(it.NodeBlob()))
copy(nodeVal, it.NodeBlob()) copy(nodeVal, it.NodeBlob())
nodeHash := make([]byte, len(it.Hash().Bytes())) nodeHash := make([]byte, len(it.Hash().Bytes()))
copy(nodeHash, it.Hash().Bytes()) copy(nodeHash, it.Hash().Bytes())
if err := ipldOutput(types2.IPLD{ if err := ipldOutput(sdtypes.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, nodeHash).String(), CID: ipld.Keccak256ToCid(ipld.MEthStorageTrie, nodeHash).String(),
Content: nodeVal, Content: nodeVal,
}); err != nil { }); err != nil {
return nil, err return nil, err
} }
prevBlob = nodeVal
} }
} }
return diffSlotsAtB, it.Error() return diffSlotsAtB, it.Error()
} }
func (sdb *StateDiffBuilder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffSlotsAtB map[string]bool, output types2.StorageNodeSink) error { func (sdb *StateDiffBuilder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffSlotsAtB map[string]bool, output sdtypes.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.DeletedOrUpdatedStorageTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.DeletedOrUpdatedStorageTimer)
it, _ := trie.NewDifferenceIterator(b, a) it, _ := trie.NewDifferenceIterator(b, a)
for it.Next(true) { for it.Next(true) {
if it.Leaf() { if it.Leaf() {
@ -616,8 +609,8 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedStorage(a, b trie.NodeIterator, dif
// if this node's leaf key did not show up in diffSlotsAtB // if this node's leaf key did not show up in diffSlotsAtB
// that means the storage slot was vacated // that means the storage slot was vacated
// in that case, emit an empty "removed" diff storage node // in that case, emit an empty "removed" diff storage node
if _, ok := diffSlotsAtB[common.Bytes2Hex(leafKey)]; !ok { if _, ok := diffSlotsAtB[hex.EncodeToString(leafKey)]; !ok {
if err := output(types2.StorageLeafNode{ if err := output(sdtypes.StorageLeafNode{
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
LeafKey: leafKey, LeafKey: leafKey,
@ -631,26 +624,28 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedStorage(a, b trie.NodeIterator, dif
return it.Error() return it.Error()
} }
// isValidPrefixPath is used to check if a node at currentPath is a parent | ancestor to one of the addresses the builder is configured to watch // isWatchedPathPrefix checks if a node path is a prefix (ancestor) to one of the watched addresses.
func isValidPrefixPath(watchedAddressesLeafPaths [][]byte, currentPath []byte) bool { // An empty watch list means all paths are watched.
for _, watchedAddressPath := range watchedAddressesLeafPaths { func isWatchedPathPrefix(watchedLeafPaths [][]byte, path []byte) bool {
if bytes.HasPrefix(watchedAddressPath, currentPath) { if len(watchedLeafPaths) == 0 {
return true
}
for _, watched := range watchedLeafPaths {
if bytes.HasPrefix(watched, path) {
return true return true
} }
} }
return false return false
} }
// isWatchedAddress is used to check if a state account corresponds to one of the addresses the builder is configured to watch // isWatchedPath checks if a node path corresponds to one of the watched addresses
func isWatchedAddress(watchedAddressesLeafPaths [][]byte, valueNodePath []byte) bool { func isWatchedPath(watchedLeafPaths [][]byte, leafPath []byte) bool {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.IsWatchedAddressTimer) defer metrics.UpdateDuration(time.Now(), metrics.IndexerMetrics.IsWatchedAddressTimer)
for _, watchedAddressPath := range watchedAddressesLeafPaths { for _, watched := range watchedLeafPaths {
if bytes.Equal(watchedAddressPath, valueNodePath) { if bytes.Equal(watched, leafPath) {
return true return true
} }
} }
return false return false
} }

File diff suppressed because it is too large Load Diff

108
chain.go Normal file
View File

@ -0,0 +1,108 @@
package statediff
import (
"context"
"encoding/json"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/event"
plugeth "github.com/openrelayxyz/plugeth-utils/core"
// "github.com/openrelayxyz/plugeth-utils/restricted/types"
"github.com/cerc-io/plugeth-statediff/utils"
)
type blockChain interface {
// SubscribeChainEvent(ch chan<- plugeth.ChainEvent) plugeth.Subscription
SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription
CurrentBlock() *types.Header
GetBlockByHash(hash common.Hash) *types.Block
GetBlockByNumber(number uint64) *types.Block
GetReceiptsByHash(hash common.Hash) types.Receipts
GetTd(hash common.Hash, number uint64) *big.Int
// TODO LockTrie is never used
// UnlockTrie(root core.Hash)
StateCache() StateView
}
// Adapts the plugeth Backend to the blockChain interface
type backendBlockChain struct {
plugeth.Backend
ctx context.Context
// middleware?
}
type backendStateView struct {
plugeth.Backend
}
func asBlockChain(backend plugeth.Backend) blockChain {
return backendBlockChain{
Backend: backend,
ctx: context.Background(),
}
}
func (b backendBlockChain) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
bufferChan := make(chan plugeth.ChainEvent, chainEventChanSize)
sub := b.Backend.SubscribeChainEvent(bufferChan)
go func() {
for event := range bufferChan {
block := utils.MustDecode[types.Block](event.Block)
// TODO: apparently we ignore the logs
// logs := utils.MustDecode[types.Log](chainEvent.Logs)
ch <- core.ChainEvent{
Block: block,
Hash: common.Hash(event.Hash),
}
}
}()
return sub
}
func (b backendBlockChain) CurrentBlock() *types.Header {
buf := b.Backend.CurrentBlock()
return utils.MustDecode[types.Header](buf)
}
func (b backendBlockChain) GetBlockByHash(hash common.Hash) *types.Block {
buf, err := b.Backend.BlockByHash(b.ctx, plugeth.Hash(hash))
if err != nil {
panic(err)
}
return utils.MustDecode[types.Block](buf)
}
func (b backendBlockChain) GetBlockByNumber(number uint64) *types.Block {
buf, err := b.Backend.BlockByNumber(b.ctx, int64(number))
if err != nil {
panic(err)
}
return utils.MustDecode[types.Block](buf)
}
func (b backendBlockChain) GetReceiptsByHash(hash common.Hash) types.Receipts {
buf, err := b.Backend.GetReceipts(b.ctx, plugeth.Hash(hash))
if err != nil {
panic(err)
}
var receipts types.Receipts
err = json.Unmarshal(buf, &receipts)
if err != nil {
panic(err)
}
return receipts
}
func (b backendBlockChain) GetTd(hash common.Hash, number uint64) *big.Int {
return b.Backend.GetTd(b.ctx, plugeth.Hash(hash))
}
func (b backendBlockChain) StateCache() StateView {
// TODO
return nil
}

View File

@ -23,7 +23,9 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/utils"
) )
// Config contains instantiation parameters for the state diffing service // Config contains instantiation parameters for the state diffing service
@ -40,7 +42,7 @@ type Config struct {
NumWorkers uint NumWorkers uint
// Should the statediff service wait until geth has synced to the head of the blockchain? // Should the statediff service wait until geth has synced to the head of the blockchain?
WaitForSync bool WaitForSync bool
// Context // Context used during DB initialization
Context context.Context Context context.Context
} }
@ -58,7 +60,7 @@ type Params struct {
func (p *Params) ComputeWatchedAddressesLeafPaths() { func (p *Params) ComputeWatchedAddressesLeafPaths() {
p.watchedAddressesLeafPaths = make([][]byte, len(p.WatchedAddresses)) p.watchedAddressesLeafPaths = make([][]byte, len(p.WatchedAddresses))
for i, address := range p.WatchedAddresses { for i, address := range p.WatchedAddresses {
p.watchedAddressesLeafPaths[i] = keybytesToHex(crypto.Keccak256(address.Bytes())) p.watchedAddressesLeafPaths[i] = utils.KeybytesToHex(crypto.Keccak256(address[:]))
} }
} }
@ -73,15 +75,3 @@ type Args struct {
OldStateRoot, NewStateRoot, BlockHash common.Hash OldStateRoot, NewStateRoot, BlockHash common.Hash
BlockNumber *big.Int BlockNumber *big.Int
} }
// https://github.com/ethereum/go-ethereum/blob/master/trie/encoding.go#L97
func keybytesToHex(str []byte) []byte {
l := len(str)*2 + 1
var nibbles = make([]byte, l)
for i, b := range str {
nibbles[i*2] = b / 16
nibbles[i*2+1] = b % 16
}
nibbles[l-1] = 16
return nibbles
}

126
go.mod Normal file
View File

@ -0,0 +1,126 @@
module github.com/cerc-io/plugeth-statediff
go 1.19
require (
github.com/ethereum/go-ethereum v1.11.6
github.com/georgysavva/scany v0.2.9
github.com/golang/mock v1.6.0
github.com/inconshreveable/log15 v2.16.0+incompatible
github.com/ipfs/go-cid v0.2.0
github.com/jackc/pgconn v1.10.0
github.com/jackc/pgtype v1.8.1
github.com/jackc/pgx/v4 v4.13.0
github.com/jmoiron/sqlx v1.2.0
github.com/lib/pq v1.10.6
github.com/multiformats/go-multihash v0.1.0
github.com/openrelayxyz/plugeth-utils v1.1.0
github.com/pganalyze/pg_query_go/v2 v2.1.0
github.com/shopspring/decimal v1.2.0
github.com/stretchr/testify v1.8.1
github.com/thoas/go-funk v0.9.2
)
require (
github.com/DataDog/zstd v1.5.2 // indirect
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect
github.com/VictoriaMetrics/fastcache v1.6.0 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/btcsuite/btcd/btcec/v2 v2.2.0 // indirect
github.com/cespare/xxhash/v2 v2.2.0 // indirect
github.com/cockroachdb/errors v1.9.1 // indirect
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect
github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 // indirect
github.com/cockroachdb/redact v1.1.3 // indirect
github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/deckarep/golang-set/v2 v2.1.0 // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 // indirect
github.com/deepmap/oapi-codegen v1.8.2 // indirect
github.com/edsrzf/mmap-go v1.0.0 // indirect
github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 // indirect
github.com/fsnotify/fsnotify v1.6.0 // indirect
github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff // indirect
github.com/getsentry/sentry-go v0.18.0 // indirect
github.com/go-ole/go-ole v1.2.6 // indirect
github.com/go-stack/stack v1.8.1 // indirect
github.com/gofrs/flock v0.8.1 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
github.com/golang-jwt/jwt/v4 v4.3.0 // indirect
github.com/golang/protobuf v1.5.2 // indirect
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect
github.com/google/uuid v1.3.0 // indirect
github.com/gorilla/websocket v1.4.2 // indirect
github.com/graph-gophers/graphql-go v1.3.0 // indirect
github.com/hashicorp/go-bexpr v0.1.10 // indirect
github.com/holiman/bloomfilter/v2 v2.0.3 // indirect
github.com/huin/goupnp v1.0.3 // indirect
github.com/influxdata/influxdb-client-go/v2 v2.4.0 // indirect
github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c // indirect
github.com/influxdata/line-protocol v0.0.0-20210311194329-9aa0e372d097 // indirect
github.com/jackc/chunkreader/v2 v2.0.1 // indirect
github.com/jackc/pgio v1.0.0 // indirect
github.com/jackc/pgpassfile v1.0.0 // indirect
github.com/jackc/pgproto3/v2 v2.1.1 // indirect
github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b // indirect
github.com/jackc/puddle v1.1.3 // indirect
github.com/jackpal/go-nat-pmp v1.0.2 // indirect
github.com/klauspost/compress v1.15.15 // indirect
github.com/klauspost/cpuid/v2 v2.0.9 // indirect
github.com/kr/pretty v0.3.1 // indirect
github.com/kr/text v0.2.0 // indirect
github.com/mattn/go-colorable v0.1.13 // indirect
github.com/mattn/go-isatty v0.0.16 // indirect
github.com/mattn/go-runewidth v0.0.9 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect
github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 // indirect
github.com/minio/sha256-simd v1.0.0 // indirect
github.com/mitchellh/mapstructure v1.4.1 // indirect
github.com/mitchellh/pointerstructure v1.2.0 // indirect
github.com/mr-tron/base58 v1.2.0 // indirect
github.com/multiformats/go-base32 v0.0.3 // indirect
github.com/multiformats/go-base36 v0.1.0 // indirect
github.com/multiformats/go-multibase v0.0.3 // indirect
github.com/multiformats/go-varint v0.0.6 // indirect
github.com/olekukonko/tablewriter v0.0.5 // indirect
github.com/opentracing/opentracing-go v1.1.0 // indirect
github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 // indirect
github.com/pkg/errors v0.9.1 // indirect
github.com/pmezard/go-difflib v1.0.0 // indirect
github.com/prometheus/client_golang v1.14.0 // indirect
github.com/prometheus/client_model v0.3.0 // indirect
github.com/prometheus/common v0.39.0 // indirect
github.com/prometheus/procfs v0.9.0 // indirect
github.com/rogpeppe/go-internal v1.9.0 // indirect
github.com/rs/cors v1.7.0 // indirect
github.com/russross/blackfriday/v2 v2.1.0 // indirect
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
github.com/status-im/keycard-go v0.2.0 // indirect
github.com/stretchr/objx v0.5.0 // indirect
github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect
github.com/tklauser/go-sysconf v0.3.5 // indirect
github.com/tklauser/numcpus v0.2.2 // indirect
github.com/tyler-smith/go-bip39 v1.1.0 // indirect
github.com/urfave/cli/v2 v2.17.2-0.20221006022127-8f469abc00aa // indirect
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect
golang.org/x/crypto v0.1.0 // indirect
golang.org/x/exp v0.0.0-20230206171751-46f607a40771 // indirect
golang.org/x/net v0.8.0 // indirect
golang.org/x/sync v0.1.0 // indirect
golang.org/x/sys v0.6.0 // indirect
golang.org/x/term v0.6.0 // indirect
golang.org/x/text v0.8.0 // indirect
golang.org/x/time v0.0.0-20220922220347-f3bd1da661af // indirect
google.golang.org/protobuf v1.28.1 // indirect
gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect
gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect
gopkg.in/yaml.v2 v2.4.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
lukechampine.com/blake3 v1.1.6 // indirect
)
// replace github.com/ethereum/go-ethereum v1.11.6 => ../go-ethereum
replace github.com/ethereum/go-ethereum v1.11.6 => ../plugeth
replace github.com/openrelayxyz/plugeth-utils => ../plugeth-utils

762
go.sum Normal file
View File

@ -0,0 +1,762 @@
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
github.com/BurntSushi/toml v1.2.0 h1:Rt8g24XnyGTyglgET/PRUNlrUeu9F5L+7FilkXfZgs0=
github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53/go.mod h1:+3IMCy2vIlbG1XG/0ggNQv0SvxCAIpPM5b1nCz56Xno=
github.com/CloudyKit/jet/v3 v3.0.0/go.mod h1:HKQPgSJmdK8hdoAbKUUWajkHyHo4RaU5rMdUywE7VMo=
github.com/DataDog/zstd v1.5.2 h1:vUG4lAyuPCXO0TLbXvPv7EB7cNK1QV/luu55UHLrrn8=
github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw=
github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY=
github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs=
github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0=
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 h1:fLjPD/aNc3UIOA6tDi6QXUemppXK3P9BI7mr2hd6gx8=
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg=
github.com/VictoriaMetrics/fastcache v1.6.0 h1:C/3Oi3EiBCqufydp1neRZkqcwmEiuRT9c3fqvvgKm5o=
github.com/VictoriaMetrics/fastcache v1.6.0/go.mod h1:0qHz5QP0GMX4pfmMA/zt5RgfNuXJrTP0zS7DqpHGGTw=
github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY=
github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8=
github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM=
github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8=
github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
github.com/btcsuite/btcd/btcec/v2 v2.2.0 h1:fzn1qaOt32TuLjFlkzYSsBC35Q3KUjT1SwPxiMSCF5k=
github.com/btcsuite/btcd/btcec/v2 v2.2.0/go.mod h1:U7MHm051Al6XmscBQ0BoNydpOTsFAn707034b5nY8zU=
github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 h1:q0rUy8C/TYNBQS1+CGKw68tLOFYSNEs0TFnxxnS9+4U=
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/cespare/cp v0.1.0 h1:SE+dxFebS7Iik5LK0tsi1k9ZCxEaFX4AjQmoyA+1dJk=
github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44=
github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
github.com/cockroachdb/apd v1.1.0 h1:3LFP3629v+1aKXU5Q37mxmRxX/pIu1nijXydLShEq5I=
github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ=
github.com/cockroachdb/cockroach-go/v2 v2.0.3 h1:ZA346ACHIZctef6trOTwBAEvPVm1k0uLm/bb2Atc+S8=
github.com/cockroachdb/cockroach-go/v2 v2.0.3/go.mod h1:hAuDgiVgDVkfirP9JnhXEfcXEPRKBpYdGz+l7mvYSzw=
github.com/cockroachdb/datadriven v1.0.2 h1:H9MtNqVoVhvd9nCBwOyDjUEdZCREqbIdCJD93PBm/jA=
github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU=
github.com/cockroachdb/errors v1.9.1 h1:yFVvsI0VxmRShfawbt/laCIDy/mtTqqnvoNgiy5bEV8=
github.com/cockroachdb/errors v1.9.1/go.mod h1:2sxOtL2WIc096WSZqZ5h8fa17rdDq9HZOZLBCor4mBk=
github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs=
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE=
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs=
github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811 h1:ytcWPaNPhNoGMWEhDvS3zToKcDpRsLuRolQJBVGdozk=
github.com/cockroachdb/pebble v0.0.0-20230209160836-829675f94811/go.mod h1:Nb5lgvnQ2+oGlE/EyZy4+2/CxRh9KfvCXnag1vtpxVM=
github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ=
github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM=
github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk=
github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE=
github.com/cpuguy83/go-md2man/v2 v2.0.2 h1:p1EgwI/C7NhT0JmVkwCD2ZBK8j4aeHQX2pMHHBfMQ6w=
github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o=
github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
github.com/cyberdelia/templates v0.0.0-20141128023046-ca7fffd4298c/go.mod h1:GyV+0YP4qX0UQ7r2MoYZ+AvYDp12OF5yg4q8rGnyNh4=
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/deckarep/golang-set/v2 v2.1.0 h1:g47V4Or+DUdzbs8FxCCmgb6VYd+ptPAngjM6dtGktsI=
github.com/deckarep/golang-set/v2 v2.1.0/go.mod h1:VAky9rY/yGXJOLEDv3OMci+7wtDpOF4IN+y82NBOac4=
github.com/decred/dcrd/crypto/blake256 v1.0.0 h1:/8DMNYp9SGi5f0w7uCm6d6M4OU2rGFK09Y2A4Xv7EE0=
github.com/decred/dcrd/crypto/blake256 v1.0.0/go.mod h1:sQl2p6Y26YV+ZOcSTP6thNdn47hh8kt6rqSlvmrXFAc=
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 h1:YLtO71vCjJRCBcrPMtQ9nqBsqpA1m5sE92cU+pd5Mcc=
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1/go.mod h1:hyedUtir6IdtD/7lIxGeCxkaw7y45JueMRL4DIyJDKs=
github.com/deepmap/oapi-codegen v1.6.0/go.mod h1:ryDa9AgbELGeB+YEXE1dR53yAjHwFvE9iAUlWl9Al3M=
github.com/deepmap/oapi-codegen v1.8.2 h1:SegyeYGcdi0jLLrpbCMoJxnUUn8GBXHsvr4rbzjuhfU=
github.com/deepmap/oapi-codegen v1.8.2/go.mod h1:YLgSKSDv/bZQB7N4ws6luhozi3cEdRktEqrX88CvjIw=
github.com/denisenkom/go-mssqldb v0.0.0-20191124224453-732737034ffd/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU=
github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw=
github.com/docker/docker v1.6.2 h1:HlFGsy+9/xrgMmhmN+NGhCc5SHGJ7I+kHosRR1xc/aI=
github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw=
github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M=
github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM=
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0=
github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw=
github.com/ethereum/go-ethereum v1.11.6 h1:2VF8Mf7XiSUfmoNOy3D+ocfl9Qu8baQBrCNbo2CXQ8E=
github.com/ethereum/go-ethereum v1.11.6/go.mod h1:+a8pUj1tOyJ2RinsNQD4326YS+leSoKGiG/uVVb0x6Y=
github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8=
github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 h1:FtmdgXiUlNeRsoNMFlKLDt+S+6hbjVMEW6RGQ7aUf7c=
github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+uIk2RvXzuaQtkQJzzIx6lSBe1xv7hi0=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY=
github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw=
github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc=
github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff h1:tY80oXqGNY4FhTFhk+o9oFHGINQ/+vhlm8HFzi6znCI=
github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww=
github.com/georgysavva/scany v0.2.9 h1:Xt6rjYpHnMClTm/g+oZTnoSxUwiln5GqMNU+QeLNHQU=
github.com/georgysavva/scany v0.2.9/go.mod h1:yeOeC1BdIdl6hOwy8uefL2WNSlseFzbhlG/frrh65SA=
github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4=
github.com/getkin/kin-openapi v0.61.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4=
github.com/getsentry/sentry-go v0.12.0/go.mod h1:NSap0JBYWzHND8oMbyi0+XZhUalc1TBdRL1M71JZW2c=
github.com/getsentry/sentry-go v0.18.0 h1:MtBW5H9QgdcJabtZcuJG80BMOwaBpkRDZkxRkNC1sN0=
github.com/getsentry/sentry-go v0.18.0/go.mod h1:Kgon4Mby+FJ7ZWHFUAZgVaIa8sxHtnRJRLTXZr51aKQ=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s=
github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM=
github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98=
github.com/go-chi/chi/v5 v5.0.0/go.mod h1:BBug9lr0cqtdAhsu6R4AAdvufI0/XBzAQSsUqJpoZOs=
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA=
github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY=
github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8=
github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY=
github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0=
github.com/go-openapi/jsonpointer v0.19.5/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg=
github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk=
github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w=
github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA=
github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
github.com/go-stack/stack v1.8.1 h1:ntEHSVwIt7PNXNpgPmVfMrNhLtgjlmnZha2kOpuRiDw=
github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP3XYfe4=
github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo=
github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw=
github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM=
github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw=
github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU=
github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM=
github.com/gofrs/uuid v4.0.0+incompatible h1:1SD/1F5pU8p29ybwgQSwpQk+mwdRrXCYuPhW6m+TnJw=
github.com/gofrs/uuid v4.0.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM=
github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s=
github.com/gogo/googleapis v1.4.1/go.mod h1:2lpHqI5OcWCtVElxXnPt+s8oJvMpySlOyM6xDCrzib4=
github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM=
github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I=
github.com/golang-jwt/jwt/v4 v4.3.0 h1:kHL1vqdqWNfATmA0FNMdmZNMyZI1U6O31X4rlIPoBog=
github.com/golang-jwt/jwt/v4 v4.3.0/go.mod h1:/xlHOz8bRuivTWchD4jCa+NbatV+wEUSzwAxVc6locg=
github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc=
github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk=
github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw=
github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk=
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y=
github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4=
github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38=
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So=
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/graph-gophers/graphql-go v1.3.0 h1:Eb9x/q6MFpCLz7jBCiP/WTxjSDrYLR1QY41SORZyNJ0=
github.com/graph-gophers/graphql-go v1.3.0/go.mod h1:9CQHMSxwO4MprSdzoIEobiHpoLtHm77vfxsvsIN5Vuc=
github.com/hashicorp/go-bexpr v0.1.10 h1:9kuI5PFotCboP3dkDYFr/wi0gg0QVbSNz5oFRpxn4uE=
github.com/hashicorp/go-bexpr v0.1.10/go.mod h1:oxlubA2vC/gFVfX1A6JGp7ls7uCDlfJn732ehYYg+g0=
github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA=
github.com/hashicorp/golang-lru v0.5.5-0.20210104140557-80c98217689d h1:dg1dEPuWpEqDnvIw251EVy4zlP8gWbsGj4BsUKCRpYs=
github.com/hashicorp/golang-lru v0.5.5-0.20210104140557-80c98217689d/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao=
github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iURXE7ZOP9L9hSkA=
github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c h1:DZfsyhDK1hnSS5lH8l+JggqzEleHteTYfutAiVlSUM8=
github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/huin/goupnp v1.0.3 h1:N8No57ls+MnjlB+JPiCVSOyy/ot7MJTqlo7rn+NYSqQ=
github.com/huin/goupnp v1.0.3/go.mod h1:ZxNlw5WqJj6wSsRK5+YfflQGXYfccj5VgQsMNixHM7Y=
github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o=
github.com/hydrogen18/memlistener v0.0.0-20200120041712-dcc25e7acd91/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE=
github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA=
github.com/inconshreveable/log15 v2.16.0+incompatible h1:6nvMKxtGcpgm7q0KiGs+Vc+xDvUXaBqsPKHWKsinccw=
github.com/inconshreveable/log15 v2.16.0+incompatible/go.mod h1:cOaXtrgN4ScfRrD9Bre7U1thNq5RtJ8ZoP4iXVGRj6o=
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
github.com/influxdata/influxdb-client-go/v2 v2.4.0 h1:HGBfZYStlx3Kqvsv1h2pJixbCl/jhnFtxpKFAv9Tu5k=
github.com/influxdata/influxdb-client-go/v2 v2.4.0/go.mod h1:vLNHdxTJkIf2mSLvGrpj8TCcISApPoXkaxP8g9uRlW8=
github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c h1:qSHzRbhzK8RdXOsAdfDgO49TtqC1oZ+acxPrkfTxcCs=
github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo=
github.com/influxdata/line-protocol v0.0.0-20200327222509-2487e7298839/go.mod h1:xaLFMmpvUxqXtVkUJfg9QmT88cDaCJ3ZKgdZ78oO8Qo=
github.com/influxdata/line-protocol v0.0.0-20210311194329-9aa0e372d097 h1:vilfsDSy7TDxedi9gyBkMvAirat/oRcL0lFdJBf6tdM=
github.com/influxdata/line-protocol v0.0.0-20210311194329-9aa0e372d097/go.mod h1:xaLFMmpvUxqXtVkUJfg9QmT88cDaCJ3ZKgdZ78oO8Qo=
github.com/ipfs/go-cid v0.2.0 h1:01JTiihFq9en9Vz0lc0VDWvZe/uBonGpzo4THP0vcQ0=
github.com/ipfs/go-cid v0.2.0/go.mod h1:P+HXFDF4CVhaVayiEb4wkAy7zBHxBwsJyt0Y5U6MLro=
github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI=
github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0=
github.com/iris-contrib/jade v1.1.3/go.mod h1:H/geBymxJhShH5kecoiOCSssPX7QWYH7UaeZTSWddIk=
github.com/iris-contrib/pongo2 v0.0.1/go.mod h1:Ssh+00+3GAZqSQb30AvBRNxBx7rf0GqwkjqxNd0u65g=
github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw=
github.com/jackc/chunkreader v1.0.0/go.mod h1:RT6O25fNZIuasFJRyZ4R/Y2BbhasbmZXF9QQ7T3kePo=
github.com/jackc/chunkreader/v2 v2.0.0/go.mod h1:odVSm741yZoC3dpHEUXIqA9tQRhFrgOHwnPIn9lDKlk=
github.com/jackc/chunkreader/v2 v2.0.1 h1:i+RDz65UE+mmpjTfyz0MoVTnzeYxroil2G82ki7MGG8=
github.com/jackc/chunkreader/v2 v2.0.1/go.mod h1:odVSm741yZoC3dpHEUXIqA9tQRhFrgOHwnPIn9lDKlk=
github.com/jackc/pgconn v0.0.0-20190420214824-7e0022ef6ba3/go.mod h1:jkELnwuX+w9qN5YIfX0fl88Ehu4XC3keFuOJJk9pcnA=
github.com/jackc/pgconn v0.0.0-20190824142844-760dd75542eb/go.mod h1:lLjNuW/+OfW9/pnVKPazfWOgNfH2aPem8YQ7ilXGvJE=
github.com/jackc/pgconn v0.0.0-20190831204454-2fabfa3c18b7/go.mod h1:ZJKsE/KZfsUgOEh9hBm+xYTstcNHg7UPMVJqRfQxq4s=
github.com/jackc/pgconn v1.4.0/go.mod h1:Y2O3ZDF0q4mMacyWV3AstPJpeHXWGEetiFttmq5lahk=
github.com/jackc/pgconn v1.5.0/go.mod h1:QeD3lBfpTFe8WUnPZWN5KY/mB8FGMIYRdd8P8Jr0fAI=
github.com/jackc/pgconn v1.5.1-0.20200601181101-fa742c524853/go.mod h1:QeD3lBfpTFe8WUnPZWN5KY/mB8FGMIYRdd8P8Jr0fAI=
github.com/jackc/pgconn v1.6.4/go.mod h1:w2pne1C2tZgP+TvjqLpOigGzNqjBgQW9dUw/4Chex78=
github.com/jackc/pgconn v1.7.0/go.mod h1:sF/lPpNEMEOp+IYhyQGdAvrG20gWf6A1tKlr0v7JMeA=
github.com/jackc/pgconn v1.8.0/go.mod h1:1C2Pb36bGIP9QHGBYCjnyhqu7Rv3sGshaQUvmfGIB/o=
github.com/jackc/pgconn v1.9.0/go.mod h1:YctiPyvzfU11JFxoXokUOOKQXQmDMoJL9vJzHH8/2JY=
github.com/jackc/pgconn v1.9.1-0.20210724152538-d89c8390a530/go.mod h1:4z2w8XhRbP1hYxkpTuBjTS3ne3J48K83+u0zoyvg2pI=
github.com/jackc/pgconn v1.10.0 h1:4EYhlDVEMsJ30nNj0mmgwIUXoq7e9sMJrVC2ED6QlCU=
github.com/jackc/pgconn v1.10.0/go.mod h1:4z2w8XhRbP1hYxkpTuBjTS3ne3J48K83+u0zoyvg2pI=
github.com/jackc/pgio v1.0.0 h1:g12B9UwVnzGhueNavwioyEEpAmqMe1E/BN9ES+8ovkE=
github.com/jackc/pgio v1.0.0/go.mod h1:oP+2QK2wFfUWgr+gxjoBH9KGBb31Eio69xUb0w5bYf8=
github.com/jackc/pgmock v0.0.0-20190831213851-13a1b77aafa2/go.mod h1:fGZlG77KXmcq05nJLRkk0+p82V8B8Dw8KN2/V9c/OAE=
github.com/jackc/pgmock v0.0.0-20201204152224-4fe30f7445fd/go.mod h1:hrBW0Enj2AZTNpt/7Y5rr2xe/9Mn757Wtb2xeBzPv2c=
github.com/jackc/pgmock v0.0.0-20210724152146-4ad1a8207f65 h1:DadwsjnMwFjfWc9y5Wi/+Zz7xoE5ALHsRQlOctkOiHc=
github.com/jackc/pgmock v0.0.0-20210724152146-4ad1a8207f65/go.mod h1:5R2h2EEX+qri8jOWMbJCtaPWkrrNc7OHwsp2TCqp7ak=
github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM=
github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg=
github.com/jackc/pgproto3 v1.1.0/go.mod h1:eR5FA3leWg7p9aeAqi37XOTgTIbkABlvcPB3E5rlc78=
github.com/jackc/pgproto3/v2 v2.0.0-alpha1.0.20190420180111-c116219b62db/go.mod h1:bhq50y+xrl9n5mRYyCBFKkpRVTLYJVWeCc+mEAI3yXA=
github.com/jackc/pgproto3/v2 v2.0.0-alpha1.0.20190609003834-432c2951c711/go.mod h1:uH0AWtUmuShn0bcesswc4aBTWGvw0cAxIJp+6OB//Wg=
github.com/jackc/pgproto3/v2 v2.0.0-rc3/go.mod h1:ryONWYqW6dqSg1Lw6vXNMXoBJhpzvWKnT95C46ckYeM=
github.com/jackc/pgproto3/v2 v2.0.0-rc3.0.20190831210041-4c03ce451f29/go.mod h1:ryONWYqW6dqSg1Lw6vXNMXoBJhpzvWKnT95C46ckYeM=
github.com/jackc/pgproto3/v2 v2.0.1/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA=
github.com/jackc/pgproto3/v2 v2.0.2/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA=
github.com/jackc/pgproto3/v2 v2.0.5/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA=
github.com/jackc/pgproto3/v2 v2.0.6/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA=
github.com/jackc/pgproto3/v2 v2.1.1 h1:7PQ/4gLoqnl87ZxL7xjO0DR5gYuviDCZxQJsUlFW1eI=
github.com/jackc/pgproto3/v2 v2.1.1/go.mod h1:WfJCnwN3HIg9Ish/j3sgWXnAfK8A9Y0bwXYU5xKaEdA=
github.com/jackc/pgservicefile v0.0.0-20200307190119-3430c5407db8/go.mod h1:vsD4gTJCa9TptPL8sPkXrLZ+hDuNrZCnj29CQpr4X1E=
github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b h1:C8S2+VttkHFdOOCXJe+YGfa4vHYwlt4Zx+IVXQ97jYg=
github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b/go.mod h1:vsD4gTJCa9TptPL8sPkXrLZ+hDuNrZCnj29CQpr4X1E=
github.com/jackc/pgtype v0.0.0-20190421001408-4ed0de4755e0/go.mod h1:hdSHsc1V01CGwFsrv11mJRHWJ6aifDLfdV3aVjFF0zg=
github.com/jackc/pgtype v0.0.0-20190824184912-ab885b375b90/go.mod h1:KcahbBH1nCMSo2DXpzsoWOAfFkdEtEJpPbVLq8eE+mc=
github.com/jackc/pgtype v0.0.0-20190828014616-a8802b16cc59/go.mod h1:MWlu30kVJrUS8lot6TQqcg7mtthZ9T0EoIBFiJcmcyw=
github.com/jackc/pgtype v1.2.0/go.mod h1:5m2OfMh1wTK7x+Fk952IDmI4nw3nPrvtQdM0ZT4WpC0=
github.com/jackc/pgtype v1.3.0/go.mod h1:b0JqxHvPmljG+HQ5IsvQ0yqeSi4nGcDTVjFoiLDb0Ik=
github.com/jackc/pgtype v1.3.1-0.20200510190516-8cd94a14c75a/go.mod h1:vaogEUkALtxZMCH411K+tKzNpwzCKU+AnPzBKZ+I+Po=
github.com/jackc/pgtype v1.3.1-0.20200606141011-f6355165a91c/go.mod h1:cvk9Bgu/VzJ9/lxTO5R5sf80p0DiucVtN7ZxvaC4GmQ=
github.com/jackc/pgtype v1.4.2/go.mod h1:JCULISAZBFGrHaOXIIFiyfzW5VY0GRitRr8NeJsrdig=
github.com/jackc/pgtype v1.8.1-0.20210724151600-32e20a603178/go.mod h1:C516IlIV9NKqfsMCXTdChteoXmwgUceqaLfjg2e3NlM=
github.com/jackc/pgtype v1.8.1 h1:9k0IXtdJXHJbyAWQgbWr1lU+MEhPXZz6RIXxfR5oxXs=
github.com/jackc/pgtype v1.8.1/go.mod h1:LUMuVrfsFfdKGLw+AFFVv6KtHOFMwRgDDzBt76IqCA4=
github.com/jackc/pgx v3.6.2+incompatible/go.mod h1:0ZGrqGqkRlliWnWB4zKnWtjbSWbGkVEFm4TeybAXq+I=
github.com/jackc/pgx/v4 v4.0.0-20190420224344-cc3461e65d96/go.mod h1:mdxmSJJuR08CZQyj1PVQBHy9XOp5p8/SHH6a0psbY9Y=
github.com/jackc/pgx/v4 v4.0.0-20190421002000-1b8f0016e912/go.mod h1:no/Y67Jkk/9WuGR0JG/JseM9irFbnEPbuWV2EELPNuM=
github.com/jackc/pgx/v4 v4.0.0-pre1.0.20190824185557-6972a5742186/go.mod h1:X+GQnOEnf1dqHGpw7JmHqHc1NxDoalibchSk9/RWuDc=
github.com/jackc/pgx/v4 v4.5.0/go.mod h1:EpAKPLdnTorwmPUUsqrPxy5fphV18j9q3wrfRXgo+kA=
github.com/jackc/pgx/v4 v4.6.0/go.mod h1:vPh43ZzxijXUVJ+t/EmXBtFmbFVO72cuneCT9oAlxAg=
github.com/jackc/pgx/v4 v4.6.1-0.20200510190926-94ba730bb1e9/go.mod h1:t3/cdRQl6fOLDxqtlyhe9UWgfIi9R8+8v8GKV5TRA/o=
github.com/jackc/pgx/v4 v4.6.1-0.20200606145419-4e5062306904/go.mod h1:ZDaNWkt9sW1JMiNn0kdYBaLelIhw7Pg4qd+Vk6tw7Hg=
github.com/jackc/pgx/v4 v4.8.1/go.mod h1:4HOLxrl8wToZJReD04/yB20GDwf4KBYETvlHciCnwW0=
github.com/jackc/pgx/v4 v4.12.1-0.20210724153913-640aa07df17c/go.mod h1:1QD0+tgSXP7iUjYm9C1NxKhny7lq6ee99u/z+IHFcgs=
github.com/jackc/pgx/v4 v4.13.0 h1:JCjhT5vmhMAf/YwBHLvrBn4OGdIQBiFG6ym8Zmdx570=
github.com/jackc/pgx/v4 v4.13.0/go.mod h1:9P4X524sErlaxj0XSGZk7s+LD0eOyu1ZDUrrpznYDF0=
github.com/jackc/puddle v0.0.0-20190413234325-e4ced69a3a2b/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackc/puddle v0.0.0-20190608224051-11cab39313c9/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackc/puddle v1.1.0/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackc/puddle v1.1.1/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackc/puddle v1.1.2/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackc/puddle v1.1.3 h1:JnPg/5Q9xVJGfjsO5CPUOjnJps1JaRUm8I9FXVCFK94=
github.com/jackc/puddle v1.1.3/go.mod h1:m4B5Dj62Y0fbyuIc15OsIqK0+JU8nkqQjsgx7dvjSWk=
github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus=
github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc=
github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs=
github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc=
github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8=
github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA=
github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks=
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U=
github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k=
github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8=
github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE=
github.com/kataras/neffos v0.0.14/go.mod h1:8lqADm8PnbeFfL7CLXh1WHw53dG27MC3pgi2R1rmoTE=
github.com/kataras/pio v0.0.2/go.mod h1:hAoW0t9UmXi4R5Oyq5Z4irTbaTsOemSrDGUtaTl7Dro=
github.com/kataras/sitemap v0.0.5/go.mod h1:KY2eugMKiPwsJgx7+U103YZehfvNGOXURubcGyk0Bz8=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
github.com/klauspost/compress v1.15.15 h1:EF27CXIuDsYJ6mmvtBRlEuB2UVOqHG1tAXgZ7yIO+lw=
github.com/klauspost/compress v1.15.15/go.mod h1:ZcK2JAFqKOpnBlxcLsJzYfrS9X1akm9fHZNnD9+Vo/4=
github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek=
github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4=
github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk=
github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
github.com/kr/pty v1.1.8/go.mod h1:O1sed60cT9XZ5uDucP5qwvh+TE3NnUj51EiZO/lmSfw=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc=
github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4FW1e6jwpg=
github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y=
github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k=
github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.1.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.4.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/lib/pq v1.10.2/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o=
github.com/lib/pq v1.10.6 h1:jbk+ZieJ0D7EVGJYpL9QTz7/YW6UHbmdnZWYyK5cdBs=
github.com/lib/pq v1.10.6/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o=
github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
github.com/matryer/moq v0.0.0-20190312154309-6cfb0558e1bd/go.mod h1:9ELz6aaclSIGnZBoaSLZ3NAl1VTufbOrXBPvtcy6WiQ=
github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ=
github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE=
github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
github.com/mattn/go-colorable v0.1.11/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4=
github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA=
github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg=
github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ=
github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ=
github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/QdE+0=
github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI=
github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc=
github.com/mattn/go-sqlite3 v2.0.1+incompatible h1:xQ15muvnzGBHpIpdrNi1DA5x0+TcBZzsIDwmw9uTHzw=
github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc=
github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw=
github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo=
github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4=
github.com/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8=
github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc=
github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g=
github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ=
github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g=
github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM=
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
github.com/mitchellh/mapstructure v1.4.1 h1:CpVNEelQCZBooIPDn+AR3NpivK/TIKU8bDxdASFVQag=
github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo=
github.com/mitchellh/pointerstructure v1.2.0 h1:O+i9nHnXS3l/9Wu7r4NrEdwA2VFTicjUEN1uBnDo34A=
github.com/mitchellh/pointerstructure v1.2.0/go.mod h1:BRAsLI5zgXmw97Lf6s25bs8ohIXc3tViBH44KcwB2g4=
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ=
github.com/mr-tron/base58 v1.1.0/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVqeSzSU8=
github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o=
github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc=
github.com/multiformats/go-base32 v0.0.3 h1:tw5+NhuwaOjJCC5Pp82QuXbrmLzWg7uxlMFp8Nq/kkI=
github.com/multiformats/go-base32 v0.0.3/go.mod h1:pLiuGC8y0QR3Ue4Zug5UzK9LjgbkL8NSQj0zQ5Nz/AA=
github.com/multiformats/go-base36 v0.1.0 h1:JR6TyF7JjGd3m6FbLU2cOxhC0Li8z8dLNGQ89tUg4F4=
github.com/multiformats/go-base36 v0.1.0/go.mod h1:kFGE83c6s80PklsHO9sRn2NCoffoRdUUOENyW/Vv6sM=
github.com/multiformats/go-multibase v0.0.3 h1:l/B6bJDQjvQ5G52jw4QGSYeOTZoAwIO77RblWplfIqk=
github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc=
github.com/multiformats/go-multihash v0.1.0 h1:CgAgwqk3//SVEw3T+6DqI4mWMyRuDwZtOWcJT0q9+EA=
github.com/multiformats/go-multihash v0.1.0/go.mod h1:RJlXsxt6vHGaia+S8We0ErjhojtKzPP2AH4+kYM7k84=
github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY=
github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE=
github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg=
github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78=
github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec=
github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY=
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/ginkgo v1.10.3/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA=
github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY=
github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE=
github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU=
github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 h1:oYW+YCJ1pachXTQmzR3rNLYGGz4g/UgFcjb28p/viDM=
github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7/go.mod h1:CRroGNssyjTd/qIG2FyxByd2S8JEAZXBl4qUrZf8GS0=
github.com/pganalyze/pg_query_go/v2 v2.1.0 h1:donwPZ4G/X+kMs7j5eYtKjdziqyOLVp3pkUrzb9lDl8=
github.com/pganalyze/pg_query_go/v2 v2.1.0/go.mod h1:XAxmVqz1tEGqizcQ3YSdN90vCOHBWjJi8URL1er5+cA=
github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4=
github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8=
github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/prometheus/client_golang v1.14.0 h1:nJdhIvne2eSX/XRAFV9PcvFFRbrjbcTUj0VP62TMhnw=
github.com/prometheus/client_golang v1.14.0/go.mod h1:8vpkKitgIVNcqrRBWh1C4TIUQgYNtG/XQE4E/Zae36Y=
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4=
github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w=
github.com/prometheus/common v0.39.0 h1:oOyhkDq05hPZKItWVBkJ6g6AtGxi+fy7F4JvUV8uhsI=
github.com/prometheus/common v0.39.0/go.mod h1:6XBZ7lYdLCbkAVhwRsWTZn+IN5AB9F/NXd5w0BbEX0Y=
github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI=
github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
github.com/rogpeppe/go-internal v1.8.1/go.mod h1:JeRgkft04UBgHMgCIwADu4Pn6Mtm5d4nPKWu0nJ5d+o=
github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8=
github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs=
github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik=
github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU=
github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ=
github.com/rs/zerolog v1.13.0/go.mod h1:YbFCdg8HfsridGWAh22vktObvhZbQsZXe4/zB0OKkWU=
github.com/rs/zerolog v1.15.0/go.mod h1:xYTKnLHcpfU2225ny5qZjxnj9NvkumZYjJHlAThCjNc=
github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk=
github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g=
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU=
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4=
github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
github.com/shopspring/decimal v0.0.0-20200419222939-1884f454f8ea/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ=
github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc=
github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q=
github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI=
github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ=
github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE=
github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU=
github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo=
github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s=
github.com/status-im/keycard-go v0.2.0 h1:QDLFswOQu1r5jsycloeQh3bVU8n/NatHHaZobtDnDzA=
github.com/status-im/keycard-go v0.2.0/go.mod h1:wlp8ZLbsmrF6g6WjugPAx+IzoLrkdf9+mHxBEeo3Hbg=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE=
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c=
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk=
github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY=
github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc=
github.com/thoas/go-funk v0.9.2 h1:oKlNYv0AY5nyf9g+/GhMgS/UO2ces0QRdPKwkhY3VCk=
github.com/thoas/go-funk v0.9.2/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q=
github.com/tklauser/go-sysconf v0.3.5 h1:uu3Xl4nkLzQfXNsWn15rPc/HQCJKObbt1dKJeWp3vU4=
github.com/tklauser/go-sysconf v0.3.5/go.mod h1:MkWzOF4RMCshBAMXuhXJs64Rte09mITnppBXY/rYEFI=
github.com/tklauser/numcpus v0.2.2 h1:oyhllyrScuYI6g+h/zUvNXNp1wy7x8qQy3t/piefldA=
github.com/tklauser/numcpus v0.2.2/go.mod h1:x3qojaO3uyYt0i56EW/VUYs7uBvdl2fkfZFu0T9wgjM=
github.com/tyler-smith/go-bip39 v1.1.0 h1:5eUemwrMargf3BSLRRCalXT93Ns6pQJIjYQN2nyfOP8=
github.com/tyler-smith/go-bip39 v1.1.0/go.mod h1:gUYDtqQw1JS3ZJ8UWVcGTGqqr6YIN3CWg+kkNaLt55U=
github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc=
github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw=
github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0=
github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY=
github.com/urfave/cli/v2 v2.17.2-0.20221006022127-8f469abc00aa h1:5SqCsI/2Qya2bCzK15ozrqo2sZxkh0FHynJZOTVoV6Q=
github.com/urfave/cli/v2 v2.17.2-0.20221006022127-8f469abc00aa/go.mod h1:1CNUng3PtjQMtRzJO4FMXBQvkGtuYRxxiR9xMa7jMwI=
github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4=
github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w=
github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8=
github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ=
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU=
github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y=
github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q=
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 h1:bAn7/zixMGCfxrRTfdpNzjtPYqr8smhKouy9mxVdGPU=
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8=
github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI=
github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg=
github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM=
github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc=
github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q=
go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190411191339-88737f569e3a/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190911031432-227b76d455e7/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20201203163018-be400aefbc4c/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4=
golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
golang.org/x/crypto v0.1.0 h1:MDRAIl0xIo9Io2xV565hzXHw3zVseKrJKodhohM5CjU=
golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw=
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
golang.org/x/exp v0.0.0-20230206171751-46f607a40771 h1:xP7rWLUr1e1n2xkK5YB4LI0hPEy3LJC6Wk+D4pGlOJg=
golang.org/x/exp v0.0.0-20230206171751-46f607a40771/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc=
golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY=
golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc=
golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
golang.org/x/net v0.0.0-20211008194852-3b03d305991f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ=
golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc=
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o=
golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ=
golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.6.0 h1:clScbb1cHjoCkyRbWwBEUZ5H/tIFu5TAXIqaZD0Gcjw=
golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
golang.org/x/text v0.8.0 h1:57P1ETyNKtuIjB4SRd15iJxuhj8Gc416Y78H3qgMh68=
golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8=
golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20220922220347-f3bd1da661af h1:Yx9k8YCG3dvF87UAn2tu2HQLf2dt/eR1bXxpLMWeH+Y=
golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
golang.org/x/tools v0.0.0-20190425163242-31fd60d6bfdc/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
golang.org/x/tools v0.0.0-20190823170909-c4a336ef6a2f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df h1:5Pf6pFKu98ODmgnpvkJ3kFUOQGGLIzLIkbzUHp47618=
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc=
google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c=
google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
google.golang.org/genproto v0.0.0-20210624195500-8bfb893ecb84/go.mod h1:SzzZ/N+nwJDaO1kznhnlzqS8ocJICar6hYhVyhi++24=
google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw=
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM=
google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c=
google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w=
google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE=
gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y=
gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s=
gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA=
gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8=
gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k=
gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce h1:+JknDZhAj8YMt7GC73Ei8pv4MzjDUNPHgQWJdtMAaDU=
gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce/go.mod h1:5AcXVHNjg+BDxry382+8OKon8SEWiKktQR07RKPsv1c=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20191120175047-4206685974f2/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
lukechampine.com/blake3 v1.1.6 h1:H3cROdztr7RCfoaTpGZFQsrqvweFLrqS73j7L7cmR5c=
lukechampine.com/blake3 v1.1.6/go.mod h1:tkKEOtDkNtklkXtLNEOGNq5tcV90tJiA1vAA12R78LA=

View File

@ -20,15 +20,16 @@ import (
"context" "context"
"fmt" "fmt"
"github.com/ethereum/go-ethereum/log" "github.com/cerc-io/plugeth-statediff/utils/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/dump"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/dump"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/node"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
) )
// NewStateDiffIndexer creates and returns an implementation of the StateDiffIndexer interface. // NewStateDiffIndexer creates and returns an implementation of the StateDiffIndexer interface.
@ -41,7 +42,7 @@ func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, n
return nil, nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{}) return nil, nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{})
} }
fc.NodeInfo = nodeInfo fc.NodeInfo = nodeInfo
ind, err := file.NewStateDiffIndexer(ctx, chainConfig, fc) ind, err := file.NewStateDiffIndexer(chainConfig, fc)
return nil, ind, err return nil, ind, err
case shared.POSTGRES: case shared.POSTGRES:
log.Info("Starting statediff service in Postgres writing mode") log.Info("Starting statediff service in Postgres writing mode")

View File

@ -20,9 +20,9 @@ import (
"fmt" "fmt"
"io" "io"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/models"
) )
// 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

View File

@ -21,9 +21,14 @@ import (
"io" "io"
"strings" "strings"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/shared"
) )
// Config for data dump
type Config struct {
Dump io.WriteCloser
}
// DumpType to explicitly type the dump destination // DumpType to explicitly type the dump destination
type DumpType string type DumpType string
@ -31,9 +36,14 @@ const (
STDOUT = "Stdout" STDOUT = "Stdout"
STDERR = "Stderr" STDERR = "Stderr"
DISCARD = "Discard" DISCARD = "Discard"
UNKNOWN = "Unknown" INVALID = "Invalid"
) )
// Type satisfies interfaces.Config
func (c Config) Type() shared.DBType {
return shared.DUMP
}
// ResolveDumpType resolves the dump type for the provided string // ResolveDumpType resolves the dump type for the provided string
func ResolveDumpType(str string) (DumpType, error) { func ResolveDumpType(str string) (DumpType, error) {
switch strings.ToLower(str) { switch strings.ToLower(str) {
@ -44,36 +54,27 @@ func ResolveDumpType(str string) (DumpType, error) {
case "discard", "void", "devnull", "dev null": case "discard", "void", "devnull", "dev null":
return DISCARD, nil return DISCARD, nil
default: default:
return UNKNOWN, fmt.Errorf("unrecognized dump type: %s", str) return INVALID, fmt.Errorf("unrecognized dump type: %s", str)
} }
} }
// Config for data dump // Set satisfies flag.Value
type Config struct { func (d *DumpType) Set(v string) (err error) {
Dump io.WriteCloser *d, err = ResolveDumpType(v)
return
} }
// Type satisfies interfaces.Config // String satisfies flag.Value
func (c Config) Type() shared.DBType { func (d *DumpType) String() string {
return shared.DUMP return strings.ToLower(string(*d))
}
// NewDiscardWriterCloser returns a discardWrapper wrapping io.Discard
func NewDiscardWriterCloser() io.WriteCloser {
return discardWrapper{blackhole: io.Discard}
} }
// discardWrapper wraps io.Discard with io.Closer // discardWrapper wraps io.Discard with io.Closer
type discardWrapper struct { type discardWrapper struct{ io.Writer }
blackhole io.Writer
}
// Write satisfies io.Writer var Discard = discardWrapper{io.Discard}
func (dw discardWrapper) Write(b []byte) (int, error) {
return dw.blackhole.Write(b)
}
// Close satisfies io.Closer // Close satisfies io.Closer
func (dw discardWrapper) Close() error { func (discardWrapper) Close() error {
return nil return nil
} }

View File

@ -18,6 +18,7 @@ package dump
import ( import (
"bytes" "bytes"
"encoding/hex"
"fmt" "fmt"
"io" "io"
"math/big" "math/big"
@ -28,15 +29,16 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/models"
sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
@ -200,7 +202,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNu
} }
preparedHash := crypto.Keccak256Hash(uncleEncoding) preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) { if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex()) return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.String(), unclesHash.String())
} }
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256) unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil { if err != nil {
@ -294,7 +296,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs
if len(receipt.PostState) == 0 { if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status rctModel.PostStatus = receipt.Status
} else { } else {
rctModel.PostState = common.Bytes2Hex(receipt.PostState) rctModel.PostState = hex.EncodeToString(receipt.PostState)
} }
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", rctModel); err != nil { if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", rctModel); err != nil {
@ -305,7 +307,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs
for idx, l := range receipt.Logs { for idx, l := range receipt.Logs {
topicSet := make([]string, 4) topicSet := make([]string, 4)
for ti, topic := range l.Topics { for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex() topicSet[ti] = topic.String()
} }
logDataSet[idx] = &models.LogsModel{ logDataSet[idx] = &models.LogsModel{

View File

@ -20,17 +20,26 @@ import (
"fmt" "fmt"
"strings" "strings"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/shared"
) )
// Config holds params for writing out CSV or SQL files
type Config struct {
Mode FileMode
OutputDir string
FilePath string
WatchedAddressesFilePath string
NodeInfo node.Info
}
// FileMode to explicitly type the mode of file writer we are using // FileMode to explicitly type the mode of file writer we are using
type FileMode string type FileMode string
const ( const (
CSV FileMode = "CSV" CSV FileMode = "CSV"
SQL FileMode = "SQL" SQL FileMode = "SQL"
Unknown FileMode = "Unknown" Invalid FileMode = "Invalid"
) )
// ResolveFileMode resolves a FileMode from a provided string // ResolveFileMode resolves a FileMode from a provided string
@ -41,17 +50,19 @@ func ResolveFileMode(str string) (FileMode, error) {
case "sql": case "sql":
return SQL, nil return SQL, nil
default: default:
return Unknown, fmt.Errorf("unrecognized file type string: %s", str) return Invalid, fmt.Errorf("unrecognized file type string: %s", str)
} }
} }
// Config holds params for writing out CSV or SQL files // Set satisfies flag.Value
type Config struct { func (f *FileMode) Set(v string) (err error) {
Mode FileMode *f, err = ResolveFileMode(v)
OutputDir string return
FilePath string }
WatchedAddressesFilePath string
NodeInfo node.Info // Set satisfies flag.Value
func (f *FileMode) String() string {
return strings.ToLower(string(*f))
} }
// Type satisfies interfaces.Config // Type satisfies interfaces.Config

View File

@ -27,11 +27,11 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema" "github.com/cerc-io/plugeth-statediff/indexer/shared/schema"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
) )
const dbDirectory = "/file_indexer" const dbDirectory = "/file_indexer"
@ -43,7 +43,7 @@ func setupLegacyCSVIndexer(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
ind, err = file.NewStateDiffIndexer(context.Background(), test.LegacyConfig, file.CSVTestConfig) ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.CSVTestConfig)
require.NoError(t, err) require.NoError(t, err)
db, err = postgres.SetupSQLXDB() db, err = postgres.SetupSQLXDB()

View File

@ -17,7 +17,6 @@
package file_test package file_test
import ( import (
"context"
"errors" "errors"
"math/big" "math/big"
"os" "os"
@ -25,15 +24,13 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupCSVIndexer(t *testing.T) { func setupCSVIndexer(t *testing.T) {
file.CSVTestConfig.OutputDir = "./statediffing_test"
if _, err := os.Stat(file.CSVTestConfig.OutputDir); !errors.Is(err, os.ErrNotExist) { if _, err := os.Stat(file.CSVTestConfig.OutputDir); !errors.Is(err, os.ErrNotExist) {
err := os.RemoveAll(file.CSVTestConfig.OutputDir) err := os.RemoveAll(file.CSVTestConfig.OutputDir)
require.NoError(t, err) require.NoError(t, err)
@ -44,7 +41,7 @@ func setupCSVIndexer(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
ind, err = file.NewStateDiffIndexer(context.Background(), mocks.TestConfig, file.CSVTestConfig) ind, err = file.NewStateDiffIndexer(mocks.TestConfig, file.CSVTestConfig)
require.NoError(t, err) require.NoError(t, err)
db, err = postgres.SetupSQLXDB() db, err = postgres.SetupSQLXDB()
@ -69,7 +66,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexHeaderIPLDs(t, db) test.DoTestPublishAndIndexHeaderIPLDs(t, db)
}) })
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
@ -77,7 +74,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexTransactionIPLDs(t, db) test.DoTestPublishAndIndexTransactionIPLDs(t, db)
}) })
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
@ -85,7 +82,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexLogIPLDs(t, db) test.DoTestPublishAndIndexLogIPLDs(t, db)
}) })
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
@ -93,7 +90,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexReceiptIPLDs(t, db) test.DoTestPublishAndIndexReceiptIPLDs(t, db)
}) })
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
@ -101,7 +98,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexStateIPLDs(t, db) test.DoTestPublishAndIndexStateIPLDs(t, db)
}) })
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
@ -109,7 +106,7 @@ func TestCSVFileIndexer(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexStorageIPLDs(t, db) test.DoTestPublishAndIndexStorageIPLDs(t, db)
}) })
} }
@ -127,7 +124,7 @@ func TestCSVFileIndexerNonCanonical(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexTransactionsNonCanonical(t, db) test.DoTestPublishAndIndexTransactionsNonCanonical(t, db)
}) })
t.Run("Publish and index receipts", func(t *testing.T) { t.Run("Publish and index receipts", func(t *testing.T) {
@ -135,7 +132,7 @@ func TestCSVFileIndexerNonCanonical(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexReceiptsNonCanonical(t, db) test.DoTestPublishAndIndexReceiptsNonCanonical(t, db)
}) })
t.Run("Publish and index logs", func(t *testing.T) { t.Run("Publish and index logs", func(t *testing.T) {
@ -143,7 +140,7 @@ func TestCSVFileIndexerNonCanonical(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexLogsNonCanonical(t, db) test.DoTestPublishAndIndexLogsNonCanonical(t, db)
}) })
t.Run("Publish and index state nodes", func(t *testing.T) { t.Run("Publish and index state nodes", func(t *testing.T) {
@ -151,7 +148,7 @@ func TestCSVFileIndexerNonCanonical(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexStateNonCanonical(t, db) test.DoTestPublishAndIndexStateNonCanonical(t, db)
}) })
t.Run("Publish and index storage nodes", func(t *testing.T) { t.Run("Publish and index storage nodes", func(t *testing.T) {
@ -159,7 +156,7 @@ func TestCSVFileIndexerNonCanonical(t *testing.T) {
dumpCSVFileData(t) dumpCSVFileData(t)
defer tearDownCSV(t) defer tearDownCSV(t)
test.TestPublishAndIndexStorageNonCanonical(t, db) test.DoTestPublishAndIndexStorageNonCanonical(t, db)
}) })
} }

View File

@ -25,15 +25,15 @@ import (
"path/filepath" "path/filepath"
"strconv" "strconv"
"github.com/ethereum/go-ethereum/common"
"github.com/thoas/go-funk" "github.com/thoas/go-funk"
"github.com/ethereum/go-ethereum/common" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/models" nodeinfo "github.com/cerc-io/plugeth-statediff/indexer/node"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/shared/schema"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema" sdtypes "github.com/cerc-io/plugeth-statediff/types"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
) )
var ( var (
@ -92,9 +92,6 @@ func newFileWriter(path string) (ret fileWriter, err error) {
} }
func makeFileWriters(dir string, tables []*schema.Table) (fileWriters, error) { func makeFileWriters(dir string, tables []*schema.Table) (fileWriters, error) {
if err := os.MkdirAll(dir, 0755); err != nil {
return nil, err
}
writers := fileWriters{} writers := fileWriters{}
for _, tbl := range tables { for _, tbl := range tables {
w, err := newFileWriter(TableFilePath(dir, tbl.Name)) w, err := newFileWriter(TableFilePath(dir, tbl.Name))

View File

@ -18,7 +18,6 @@ package file
import ( import (
"bytes" "bytes"
"context"
"errors" "errors"
"fmt" "fmt"
"math/big" "math/big"
@ -27,21 +26,21 @@ import (
"sync/atomic" "sync/atomic"
"time" "time"
"github.com/lib/pq"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/lib/pq"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
const defaultCSVOutputDir = "./statediff_output" const defaultCSVOutputDir = "./statediff_output"
@ -63,7 +62,7 @@ type StateDiffIndexer struct {
} }
// NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer // NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer
func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, config Config) (*StateDiffIndexer, error) { func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config) (*StateDiffIndexer, error) {
var err error var err error
var writer FileWriter var writer FileWriter
@ -259,7 +258,7 @@ func (sdi *StateDiffIndexer) processUncles(headerID string, blockNumber *big.Int
} }
preparedHash := crypto.Keccak256Hash(uncleEncoding) preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) { if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex()) return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.String(), unclesHash.String())
} }
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256) unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil { if err != nil {
@ -358,7 +357,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error {
sdi.fileWriter.upsertIPLDNode(args.blockNumber.String(), args.logNodes[i][idx]) sdi.fileWriter.upsertIPLDNode(args.blockNumber.String(), args.logNodes[i][idx])
topicSet := make([]string, 4) topicSet := make([]string, 4)
for ti, topic := range l.Topics { for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex() topicSet[ti] = topic.String()
} }
logDataSet[idx] = &models.LogsModel{ logDataSet[idx] = &models.LogsModel{

View File

@ -19,12 +19,12 @@ package file
import ( import (
"math/big" "math/big"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/indexer/models"
nodeinfo "github.com/cerc-io/plugeth-statediff/indexer/node"
"github.com/cerc-io/plugeth-statediff/types"
) )
// Writer interface required by the file indexer // Writer interface required by the file indexer

View File

@ -24,16 +24,16 @@ import (
"os" "os"
"testing" "testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/test"
"github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
) )
var ( var (
@ -87,7 +87,7 @@ func setupMainnetIndexer(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
ind, err = file.NewStateDiffIndexer(context.Background(), chainConf, file.CSVTestConfig) ind, err = file.NewStateDiffIndexer(chainConf, file.CSVTestConfig)
require.NoError(t, err) require.NoError(t, err)
db, err = postgres.SetupSQLXDB() db, err = postgres.SetupSQLXDB()

View File

@ -24,12 +24,12 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
) )
var ( var (
@ -44,7 +44,7 @@ func setupLegacySQLIndexer(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
ind, err = file.NewStateDiffIndexer(context.Background(), test.LegacyConfig, file.SQLTestConfig) ind, err = file.NewStateDiffIndexer(test.LegacyConfig, file.SQLTestConfig)
require.NoError(t, err) require.NoError(t, err)
db, err = postgres.SetupSQLXDB() db, err = postgres.SetupSQLXDB()

View File

@ -17,7 +17,6 @@
package file_test package file_test
import ( import (
"context"
"errors" "errors"
"math/big" "math/big"
"os" "os"
@ -25,10 +24,10 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupIndexer(t *testing.T) { func setupIndexer(t *testing.T) {
@ -42,7 +41,7 @@ func setupIndexer(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
ind, err = file.NewStateDiffIndexer(context.Background(), mocks.TestConfig, file.SQLTestConfig) ind, err = file.NewStateDiffIndexer(mocks.TestConfig, file.SQLTestConfig)
require.NoError(t, err) require.NoError(t, err)
db, err = postgres.SetupSQLXDB() db, err = postgres.SetupSQLXDB()
@ -67,7 +66,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexHeaderIPLDs(t, db) test.DoTestPublishAndIndexHeaderIPLDs(t, db)
}) })
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
@ -75,7 +74,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexTransactionIPLDs(t, db) test.DoTestPublishAndIndexTransactionIPLDs(t, db)
}) })
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
@ -83,7 +82,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexLogIPLDs(t, db) test.DoTestPublishAndIndexLogIPLDs(t, db)
}) })
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
@ -91,7 +90,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexReceiptIPLDs(t, db) test.DoTestPublishAndIndexReceiptIPLDs(t, db)
}) })
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
@ -99,7 +98,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexStateIPLDs(t, db) test.DoTestPublishAndIndexStateIPLDs(t, db)
}) })
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
@ -107,7 +106,7 @@ func TestSQLFileIndexer(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexStorageIPLDs(t, db) test.DoTestPublishAndIndexStorageIPLDs(t, db)
}) })
} }
@ -125,7 +124,7 @@ func TestSQLFileIndexerNonCanonical(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexTransactionsNonCanonical(t, db) test.DoTestPublishAndIndexTransactionsNonCanonical(t, db)
}) })
t.Run("Publish and index receipts", func(t *testing.T) { t.Run("Publish and index receipts", func(t *testing.T) {
@ -133,7 +132,7 @@ func TestSQLFileIndexerNonCanonical(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexReceiptsNonCanonical(t, db) test.DoTestPublishAndIndexReceiptsNonCanonical(t, db)
}) })
t.Run("Publish and index logs", func(t *testing.T) { t.Run("Publish and index logs", func(t *testing.T) {
@ -141,7 +140,7 @@ func TestSQLFileIndexerNonCanonical(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexLogsNonCanonical(t, db) test.DoTestPublishAndIndexLogsNonCanonical(t, db)
}) })
t.Run("Publish and index state nodes", func(t *testing.T) { t.Run("Publish and index state nodes", func(t *testing.T) {
@ -149,7 +148,7 @@ func TestSQLFileIndexerNonCanonical(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexStateNonCanonical(t, db) test.DoTestPublishAndIndexStateNonCanonical(t, db)
}) })
t.Run("Publish and index storage nodes", func(t *testing.T) { t.Run("Publish and index storage nodes", func(t *testing.T) {
@ -157,7 +156,7 @@ func TestSQLFileIndexerNonCanonical(t *testing.T) {
dumpFileData(t) dumpFileData(t)
defer tearDown(t) defer tearDown(t)
test.TestPublishAndIndexStorageNonCanonical(t, db) test.DoTestPublishAndIndexStorageNonCanonical(t, db)
}) })
} }

View File

@ -24,15 +24,15 @@ import (
"math/big" "math/big"
"os" "os"
"github.com/ethereum/go-ethereum/common"
pg_query "github.com/pganalyze/pg_query_go/v2" pg_query "github.com/pganalyze/pg_query_go/v2"
"github.com/thoas/go-funk" "github.com/thoas/go-funk"
"github.com/ethereum/go-ethereum/common" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/models" nodeinfo "github.com/cerc-io/plugeth-statediff/indexer/node"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/types"
"github.com/ethereum/go-ethereum/statediff/types"
) )
var ( var (

View File

@ -21,9 +21,9 @@ import (
"strings" "strings"
"time" "time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/metrics"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
const ( const (

View File

@ -23,9 +23,9 @@ import (
"github.com/lib/pq" "github.com/lib/pq"
"github.com/ethereum/go-ethereum/log" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/utils/log"
) )
const startingCacheCapacity = 1024 * 24 const startingCacheCapacity = 1024 * 24

View File

@ -26,21 +26,21 @@ import (
"math/big" "math/big"
"time" "time"
"github.com/multiformats/go-multihash" core "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
metrics2 "github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld" metrics2 "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
@ -249,7 +249,7 @@ func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, he
} }
// processUncles publishes and indexes uncle IPLDs in Postgres // processUncles publishes and indexes uncle IPLDs in Postgres
func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber *big.Int, unclesHash common.Hash, uncles []*types.Header) error { func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber *big.Int, unclesHash core.Hash, uncles []*types.Header) error {
// publish and index uncles // publish and index uncles
uncleEncoding, err := rlp.EncodeToBytes(uncles) uncleEncoding, err := rlp.EncodeToBytes(uncles)
if err != nil { if err != nil {
@ -257,7 +257,7 @@ func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNu
} }
preparedHash := crypto.Keccak256Hash(uncleEncoding) preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) { if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex()) return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.String(), unclesHash.String())
} }
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256) unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil { if err != nil {
@ -350,7 +350,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs
if len(receipt.PostState) == 0 { if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status rctModel.PostStatus = receipt.Status
} else { } else {
rctModel.PostState = common.BytesToHash(receipt.PostState).String() rctModel.PostState = core.BytesToHash(receipt.PostState).String()
} }
if err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel); err != nil { if err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel); err != nil {
@ -363,7 +363,7 @@ func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs
tx.cacheIPLD(args.logNodes[i][idx]) tx.cacheIPLD(args.logNodes[i][idx])
topicSet := make([]string, 4) topicSet := make([]string, 4)
for ti, topic := range l.Topics { for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex() topicSet[ti] = topic.String()
} }
logDataSet[idx] = &models.LogsModel{ logDataSet[idx] = &models.LogsModel{
@ -401,7 +401,7 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber,
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: core.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID, CID: shared.RemovedNodeStateCID,
Removed: true, Removed: true,
} }
@ -409,12 +409,12 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
stateModel = models.StateNodeModel{ stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber,
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: core.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID, CID: stateNode.AccountWrapper.CID,
Removed: false, Removed: false,
Balance: stateNode.AccountWrapper.Account.Balance.String(), Balance: stateNode.AccountWrapper.Account.Balance.String(),
Nonce: stateNode.AccountWrapper.Account.Nonce, Nonce: stateNode.AccountWrapper.Account.Nonce,
CodeHash: common.BytesToHash(stateNode.AccountWrapper.Account.CodeHash).String(), CodeHash: core.BytesToHash(stateNode.AccountWrapper.Account.CodeHash).String(),
StorageRoot: stateNode.AccountWrapper.Account.Root.String(), StorageRoot: stateNode.AccountWrapper.Account.Root.String(),
} }
} }
@ -431,8 +431,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber,
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: core.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: core.BytesToHash(storageNode.LeafKey).String(),
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
Value: []byte{}, Value: []byte{},
@ -445,8 +445,8 @@ func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdt
storageModel := models.StorageNodeModel{ storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber, BlockNumber: tx.BlockNumber,
HeaderID: headerID, HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(), StateKey: core.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(), StorageKey: core.BytesToHash(storageNode.LeafKey).String(),
CID: storageNode.CID, CID: storageNode.CID,
Removed: false, Removed: false,
Value: storageNode.Value, Value: storageNode.Value,
@ -477,7 +477,7 @@ func (sdi *StateDiffIndexer) Close() error {
// Update the known gaps table with the gap information. // Update the known gaps table with the gap information.
// LoadWatchedAddresses reads watched addresses from the database // LoadWatchedAddresses reads watched addresses from the database
func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) { func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]core.Address, error) {
addressStrings := make([]string, 0) addressStrings := make([]string, 0)
pgStr := "SELECT address FROM eth_meta.watched_addresses" pgStr := "SELECT address FROM eth_meta.watched_addresses"
err := sdi.dbWriter.db.Select(sdi.ctx, &addressStrings, pgStr) err := sdi.dbWriter.db.Select(sdi.ctx, &addressStrings, pgStr)
@ -485,9 +485,9 @@ func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) {
return nil, fmt.Errorf("error loading watched addresses: %v", err) return nil, fmt.Errorf("error loading watched addresses: %v", err)
} }
watchedAddresses := []common.Address{} watchedAddresses := []core.Address{}
for _, addressString := range addressStrings { for _, addressString := range addressStrings {
watchedAddresses = append(watchedAddresses, common.HexToAddress(addressString)) watchedAddresses = append(watchedAddresses, core.HexToAddress(addressString))
} }
return watchedAddresses, nil return watchedAddresses, nil

View File

@ -5,9 +5,9 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
) )
var ( var (

View File

@ -20,7 +20,7 @@ import (
"context" "context"
"io" "io"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
) )
// Database interfaces required by the sql indexer // Database interfaces required by the sql indexer

View File

@ -4,7 +4,7 @@ import (
"context" "context"
"reflect" "reflect"
"github.com/ethereum/go-ethereum/log" "github.com/cerc-io/plugeth-statediff/utils/log"
) )
// Changing this to 1 would make sure only sequential COPYs were combined. // Changing this to 1 would make sure only sequential COPYs were combined.

View File

@ -25,13 +25,13 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/test"
"github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
) )
var ( var (

View File

@ -22,9 +22,9 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupLegacyPGXIndexer(t *testing.T) { func setupLegacyPGXIndexer(t *testing.T) {

View File

@ -23,10 +23,10 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupPGXIndexer(t *testing.T, config postgres.Config) { func setupPGXIndexer(t *testing.T, config postgres.Config) {
@ -59,7 +59,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexHeaderIPLDs(t, db) test.DoTestPublishAndIndexHeaderIPLDs(t, db)
}) })
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
@ -67,7 +67,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexTransactionIPLDs(t, db) test.DoTestPublishAndIndexTransactionIPLDs(t, db)
}) })
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
@ -75,7 +75,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexLogIPLDs(t, db) test.DoTestPublishAndIndexLogIPLDs(t, db)
}) })
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
@ -83,7 +83,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexReceiptIPLDs(t, db) test.DoTestPublishAndIndexReceiptIPLDs(t, db)
}) })
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
@ -91,7 +91,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateIPLDs(t, db) test.DoTestPublishAndIndexStateIPLDs(t, db)
}) })
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
@ -99,7 +99,7 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStorageIPLDs(t, db) test.DoTestPublishAndIndexStorageIPLDs(t, db)
}) })
t.Run("Publish and index with CopyFrom enabled.", func(t *testing.T) { t.Run("Publish and index with CopyFrom enabled.", func(t *testing.T) {
@ -110,10 +110,10 @@ func TestPGXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateIPLDs(t, db) test.DoTestPublishAndIndexStateIPLDs(t, db)
test.TestPublishAndIndexStorageIPLDs(t, db) test.DoTestPublishAndIndexStorageIPLDs(t, db)
test.TestPublishAndIndexReceiptIPLDs(t, db) test.DoTestPublishAndIndexReceiptIPLDs(t, db)
test.TestPublishAndIndexLogIPLDs(t, db) test.DoTestPublishAndIndexLogIPLDs(t, db)
}) })
} }
@ -132,7 +132,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexTransactionsNonCanonical(t, db) test.DoTestPublishAndIndexTransactionsNonCanonical(t, db)
}) })
t.Run("Publish and index receipts", func(t *testing.T) { t.Run("Publish and index receipts", func(t *testing.T) {
@ -140,7 +140,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexReceiptsNonCanonical(t, db) test.DoTestPublishAndIndexReceiptsNonCanonical(t, db)
}) })
t.Run("Publish and index logs", func(t *testing.T) { t.Run("Publish and index logs", func(t *testing.T) {
@ -148,7 +148,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexLogsNonCanonical(t, db) test.DoTestPublishAndIndexLogsNonCanonical(t, db)
}) })
t.Run("Publish and index state nodes", func(t *testing.T) { t.Run("Publish and index state nodes", func(t *testing.T) {
@ -156,7 +156,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateNonCanonical(t, db) test.DoTestPublishAndIndexStateNonCanonical(t, db)
}) })
t.Run("Publish and index storage nodes", func(t *testing.T) { t.Run("Publish and index storage nodes", func(t *testing.T) {
@ -164,7 +164,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1) defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStorageNonCanonical(t, db) test.DoTestPublishAndIndexStorageNonCanonical(t, db)
}) })
} }

View File

@ -23,49 +23,9 @@ import (
"strings" "strings"
"time" "time"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/shared"
) )
// DriverType to explicitly type the kind of sql driver we are using
type DriverType string
const (
PGX DriverType = "PGX"
SQLX DriverType = "SQLX"
Unknown DriverType = "Unknown"
)
// Env variables
const (
DATABASE_NAME = "DATABASE_NAME"
DATABASE_HOSTNAME = "DATABASE_HOSTNAME"
DATABASE_PORT = "DATABASE_PORT"
DATABASE_USER = "DATABASE_USER"
DATABASE_PASSWORD = "DATABASE_PASSWORD"
)
// ResolveDriverType resolves a DriverType from a provided string
func ResolveDriverType(str string) (DriverType, error) {
switch strings.ToLower(str) {
case "pgx", "pgxpool":
return PGX, nil
case "sqlx":
return SQLX, nil
default:
return Unknown, fmt.Errorf("unrecognized driver type string: %s", str)
}
}
// TestConfig specifies default parameters for connecting to a testing DB
var TestConfig = Config{
Hostname: "localhost",
Port: 8077,
DatabaseName: "cerc_testing",
Username: "vdbm",
Password: "password",
Driver: SQLX,
}
// Config holds params for a Postgres db // Config holds params for a Postgres db
type Config struct { type Config struct {
// conn string params // conn string params
@ -98,6 +58,34 @@ type Config struct {
CopyFrom bool CopyFrom bool
} }
// DriverType to explicitly type the kind of sql driver we are using
type DriverType string
const (
PGX DriverType = "PGX"
SQLX DriverType = "SQLX"
Invalid DriverType = "Invalid"
)
// Env variables
const (
DATABASE_NAME = "DATABASE_NAME"
DATABASE_HOSTNAME = "DATABASE_HOSTNAME"
DATABASE_PORT = "DATABASE_PORT"
DATABASE_USER = "DATABASE_USER"
DATABASE_PASSWORD = "DATABASE_PASSWORD"
)
// TestConfig specifies default parameters for connecting to a testing DB
var TestConfig = Config{
Hostname: "localhost",
Port: 8077,
DatabaseName: "cerc_testing",
Username: "vdbm",
Password: "password",
Driver: SQLX,
}
// Type satisfies interfaces.Config // Type satisfies interfaces.Config
func (c Config) Type() shared.DBType { func (c Config) Type() shared.DBType {
return shared.POSTGRES return shared.POSTGRES
@ -116,6 +104,7 @@ func (c Config) DbConnectionString() string {
return fmt.Sprintf("postgresql://%s:%d/%s?sslmode=disable", c.Hostname, c.Port, c.DatabaseName) return fmt.Sprintf("postgresql://%s:%d/%s?sslmode=disable", c.Hostname, c.Port, c.DatabaseName)
} }
// WithEnv overrides the config with env variables, returning a new instance
func (c Config) WithEnv() (Config, error) { func (c Config) WithEnv() (Config, error) {
if val := os.Getenv(DATABASE_NAME); val != "" { if val := os.Getenv(DATABASE_NAME); val != "" {
c.DatabaseName = val c.DatabaseName = val
@ -138,3 +127,26 @@ func (c Config) WithEnv() (Config, error) {
} }
return c, nil return c, nil
} }
// ResolveDriverType resolves a DriverType from a provided string
func ResolveDriverType(str string) (DriverType, error) {
switch strings.ToLower(str) {
case "pgx", "pgxpool":
return PGX, nil
case "sqlx":
return SQLX, nil
default:
return Invalid, fmt.Errorf("unrecognized driver type string: %s", str)
}
}
// Set satisfies flag.Value
func (d *DriverType) Set(v string) (err error) {
*d, err = ResolveDriverType(v)
return
}
// String satisfies flag.Value
func (d *DriverType) String() string {
return strings.ToLower(string(*d))
}

View File

@ -17,8 +17,8 @@
package postgres package postgres
import ( import (
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema" "github.com/cerc-io/plugeth-statediff/indexer/shared/schema"
) )
var _ sql.Database = &DB{} var _ sql.Database = &DB{}

View File

@ -18,8 +18,9 @@ package postgres
import ( import (
"context" "context"
"github.com/ethereum/go-ethereum/log"
"github.com/jackc/pgx/v4" "github.com/jackc/pgx/v4"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
type LogAdapter struct { type LogAdapter struct {
@ -31,31 +32,26 @@ func NewLogAdapter(l log.Logger) *LogAdapter {
} }
func (l *LogAdapter) Log(ctx context.Context, level pgx.LogLevel, msg string, data map[string]interface{}) { func (l *LogAdapter) Log(ctx context.Context, level pgx.LogLevel, msg string, data map[string]interface{}) {
var logger log.Logger args := make([]interface{}, 0)
if data != nil { for key, value := range data {
var args = make([]interface{}, 0) if value != nil {
for key, value := range data { args = append(args, key, value)
if value != nil {
args = append(args, key, value)
}
} }
logger = l.l.New(args...)
} else {
logger = l.l
} }
logger := l.l
switch level { switch level {
case pgx.LogLevelTrace: case pgx.LogLevelTrace:
logger.Trace(msg) logger.Trace(msg, args...)
case pgx.LogLevelDebug: case pgx.LogLevelDebug:
logger.Debug(msg) logger.Debug(msg, args...)
case pgx.LogLevelInfo: case pgx.LogLevelInfo:
logger.Info(msg) logger.Info(msg, args...)
case pgx.LogLevelWarn: case pgx.LogLevelWarn:
logger.Warn(msg) logger.Warn(msg, args...)
case pgx.LogLevelError: case pgx.LogLevelError:
logger.Error(msg) logger.Error(msg, args...)
default: default:
logger.New("INVALID_PGX_LOG_LEVEL", level).Error(msg) logger.Error(msg, "INVALID_PGX_LOG_LEVEL", level)
} }
} }

View File

@ -20,16 +20,16 @@ import (
"context" "context"
"time" "time"
"github.com/ethereum/go-ethereum/log" "github.com/cerc-io/plugeth-statediff/utils/log"
"github.com/georgysavva/scany/pgxscan" "github.com/georgysavva/scany/pgxscan"
"github.com/jackc/pgconn" "github.com/jackc/pgconn"
"github.com/jackc/pgx/v4" "github.com/jackc/pgx/v4"
"github.com/jackc/pgx/v4/pgxpool" "github.com/jackc/pgx/v4/pgxpool"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
) )
// PGXDriver driver, implements sql.Driver // PGXDriver driver, implements sql.Driver
@ -96,7 +96,7 @@ func MakeConfig(config Config) (*pgxpool.Config, error) {
} }
if config.LogStatements { if config.LogStatements {
conf.ConnConfig.Logger = NewLogAdapter(log.New()) conf.ConnConfig.Logger = NewLogAdapter(log.DefaultLogger)
} }
return conf, nil return conf, nil
@ -106,8 +106,10 @@ func (pgx *PGXDriver) createNode() error {
_, err := pgx.pool.Exec( _, err := pgx.pool.Exec(
pgx.ctx, pgx.ctx,
createNodeStm, createNodeStm,
pgx.nodeInfo.GenesisBlock, pgx.nodeInfo.NetworkID, pgx.nodeInfo.GenesisBlock,
pgx.nodeInfo.ID, pgx.nodeInfo.ClientName, pgx.nodeInfo.NetworkID,
pgx.nodeInfo.ID,
pgx.nodeInfo.ClientName,
pgx.nodeInfo.ChainID) pgx.nodeInfo.ChainID)
if err != nil { if err != nil {
return ErrUnableToSetNode(err) return ErrUnableToSetNode(err)

View File

@ -26,8 +26,8 @@ import (
"github.com/jackc/pgx/v4/pgxpool" "github.com/jackc/pgx/v4/pgxpool"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
) )
var ( var (

View File

@ -19,8 +19,7 @@ package postgres_test
import ( import (
"fmt" "fmt"
"os" "os"
// "github.com/cerc-io/plugeth-statediff/utils/log"
"github.com/ethereum/go-ethereum/log"
) )
func init() { func init() {
@ -29,5 +28,6 @@ func init() {
os.Exit(0) os.Exit(0)
} }
log.Root().SetHandler(log.DiscardHandler()) // TODO
// log.Root().SetHandler(log.DiscardHandler())
} }

View File

@ -24,9 +24,9 @@ import (
"github.com/jmoiron/sqlx" "github.com/jmoiron/sqlx"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
) )
// SQLXDriver driver, implements sql.Driver // SQLXDriver driver, implements sql.Driver

View File

@ -26,8 +26,8 @@ import (
_ "github.com/lib/pq" _ "github.com/lib/pq"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
) )
func TestPostgresSQLX(t *testing.T) { func TestPostgresSQLX(t *testing.T) {

View File

@ -19,8 +19,8 @@ package postgres
import ( import (
"context" "context"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node" "github.com/cerc-io/plugeth-statediff/indexer/node"
) )
// SetupSQLXDB is used to setup a sqlx db for tests // SetupSQLXDB is used to setup a sqlx db for tests

View File

@ -22,9 +22,9 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupLegacySQLXIndexer(t *testing.T) { func setupLegacySQLXIndexer(t *testing.T) {

View File

@ -23,10 +23,10 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres" "github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks" "github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test" "github.com/cerc-io/plugeth-statediff/indexer/test"
) )
func setupSQLXIndexer(t *testing.T) { func setupSQLXIndexer(t *testing.T) {
@ -55,7 +55,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexHeaderIPLDs(t, db) test.DoTestPublishAndIndexHeaderIPLDs(t, db)
}) })
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
@ -63,7 +63,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexTransactionIPLDs(t, db) test.DoTestPublishAndIndexTransactionIPLDs(t, db)
}) })
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) { t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
@ -71,7 +71,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexLogIPLDs(t, db) test.DoTestPublishAndIndexLogIPLDs(t, db)
}) })
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
@ -79,7 +79,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexReceiptIPLDs(t, db) test.DoTestPublishAndIndexReceiptIPLDs(t, db)
}) })
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
@ -87,7 +87,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStateIPLDs(t, db) test.DoTestPublishAndIndexStateIPLDs(t, db)
}) })
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) { t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
@ -95,7 +95,7 @@ func TestSQLXIndexer(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStorageIPLDs(t, db) test.DoTestPublishAndIndexStorageIPLDs(t, db)
}) })
} }
@ -114,7 +114,7 @@ func TestSQLXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexTransactionsNonCanonical(t, db) test.DoTestPublishAndIndexTransactionsNonCanonical(t, db)
}) })
t.Run("Publish and index receipts", func(t *testing.T) { t.Run("Publish and index receipts", func(t *testing.T) {
@ -122,7 +122,7 @@ func TestSQLXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexReceiptsNonCanonical(t, db) test.DoTestPublishAndIndexReceiptsNonCanonical(t, db)
}) })
t.Run("Publish and index logs", func(t *testing.T) { t.Run("Publish and index logs", func(t *testing.T) {
@ -130,7 +130,7 @@ func TestSQLXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexLogsNonCanonical(t, db) test.DoTestPublishAndIndexLogsNonCanonical(t, db)
}) })
t.Run("Publish and index state nodes", func(t *testing.T) { t.Run("Publish and index state nodes", func(t *testing.T) {
@ -138,7 +138,7 @@ func TestSQLXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStateNonCanonical(t, db) test.DoTestPublishAndIndexStateNonCanonical(t, db)
}) })
t.Run("Publish and index storage nodes", func(t *testing.T) { t.Run("Publish and index storage nodes", func(t *testing.T) {
@ -146,7 +146,7 @@ func TestSQLXIndexerNonCanonical(t *testing.T) {
defer tearDown(t) defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0) defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStorageNonCanonical(t, db) test.DoTestPublishAndIndexStorageNonCanonical(t, db)
}) })
} }

View File

@ -25,8 +25,8 @@ import (
"github.com/lib/pq" "github.com/lib/pq"
"github.com/shopspring/decimal" "github.com/shopspring/decimal"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/models"
) )
// Writer handles processing and writing of indexed IPLD objects to Postgres // Writer handles processing and writing of indexed IPLD objects to Postgres

View File

@ -21,10 +21,10 @@ import (
"math/big" "math/big"
"time" "time"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
) )
// StateDiffIndexer interface required to index statediff data // StateDiffIndexer interface required to index statediff data

View File

@ -1,11 +1,10 @@
package ipld package ipld
import ( import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
) )
// EthLog (eth-log, codec 0x9a), represents an ethereum block header // EthLog (eth-log, codec 0x9a), represents an ethereum block header

View File

@ -58,7 +58,7 @@ func processTransactions(txs []*types.Transaction) ([]*EthTx, error) {
// processReceiptsAndLogs will take in receipts // processReceiptsAndLogs will take in receipts
// to return IPLD node slices for eth-rct and eth-log // to return IPLD node slices for eth-rct and eth-log
func processReceiptsAndLogs(rcts []*types.Receipt) ([]*EthReceipt, [][]*EthLog, error) { func processReceiptsAndLogs(rcts types.Receipts) ([]*EthReceipt, [][]*EthLog, error) {
// Pre allocating memory. // Pre allocating memory.
ethRctNodes := make([]*EthReceipt, len(rcts)) ethRctNodes := make([]*EthReceipt, len(rcts))
ethLogNodes := make([][]*EthLog, len(rcts)) ethLogNodes := make([][]*EthLog, len(rcts))
@ -81,6 +81,31 @@ func processReceiptsAndLogs(rcts []*types.Receipt) ([]*EthReceipt, [][]*EthLog,
return ethRctNodes, ethLogNodes, nil return ethRctNodes, ethLogNodes, nil
} }
// // processReceiptsAndLogs will take in receipts
// // to return IPLD node slices for eth-rct and eth-log
// func processReceiptsAndLogs(rcts []*types.Receipt) ([]*EthReceipt, [][]*EthLog, error) {
// // Pre allocating memory.
// ethRctNodes := make([]*EthReceipt, len(rcts))
// ethLogNodes := make([][]*EthLog, len(rcts))
// for idx, rct := range rcts {
// logNodes, err := processLogs(rct.Logs)
// if err != nil {
// return nil, nil, err
// }
// ethRct, err := NewReceipt(rct)
// if err != nil {
// return nil, nil, err
// }
// ethRctNodes[idx] = ethRct
// ethLogNodes[idx] = logNodes
// }
// return ethRctNodes, ethLogNodes, nil
// }
func processLogs(logs []*types.Log) ([]*EthLog, error) { func processLogs(logs []*types.Log) ([]*EthLog, error) {
logNodes := make([]*EthLog, len(logs)) logNodes := make([]*EthLog, len(logs))
for idx, log := range logs { for idx, log := range logs {

View File

@ -22,18 +22,18 @@ import (
"crypto/rand" "crypto/rand"
"math/big" "math/big"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/test_helpers"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
"github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
"github.com/cerc-io/plugeth-statediff/test_helpers"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
// Test variables // Test variables
@ -150,7 +150,7 @@ var (
StoragePartialPath, StoragePartialPath,
StorageValue, StorageValue,
}) })
StorageLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, crypto.Keccak256(StorageLeafNode)).String() StorageLeafNodeCID = ipld.Keccak256ToCid(ipld.MEthStorageTrie, crypto.Keccak256(StorageLeafNode)).String()
nonce1 = uint64(1) nonce1 = uint64(1)
ContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0" ContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0"
@ -169,7 +169,7 @@ var (
ContractPartialPath, ContractPartialPath,
ContractAccount, ContractAccount,
}) })
ContractLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(ContractLeafNode)).String() ContractLeafNodeCID = ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(ContractLeafNode)).String()
Contract2LeafKey = test_helpers.AddressToLeafKey(ContractAddress2) Contract2LeafKey = test_helpers.AddressToLeafKey(ContractAddress2)
storage2Location = common.HexToHash("2") storage2Location = common.HexToHash("2")
@ -195,7 +195,7 @@ var (
AccountPartialPath, AccountPartialPath,
Account, Account,
}) })
AccountLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(AccountLeafNode)).String() AccountLeafNodeCID = ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(AccountLeafNode)).String()
StateDiffs = []sdtypes.StateLeafNode{ StateDiffs = []sdtypes.StateLeafNode{
{ {

View File

@ -28,7 +28,7 @@ const (
POSTGRES DBType = "Postgres" POSTGRES DBType = "Postgres"
DUMP DBType = "Dump" DUMP DBType = "Dump"
FILE DBType = "File" FILE DBType = "File"
UNKNOWN DBType = "Unknown" INVALID DBType = "Invalid"
) )
// ResolveDBType resolves a DBType from a provided string // ResolveDBType resolves a DBType from a provided string
@ -41,6 +41,17 @@ func ResolveDBType(str string) (DBType, error) {
case "file", "f", "fs": case "file", "f", "fs":
return FILE, nil return FILE, nil
default: default:
return UNKNOWN, fmt.Errorf("unrecognized db type string: %s", str) return INVALID, fmt.Errorf("unrecognized db type string: %s", str)
} }
} }
// Set satisfies flag.Value
func (dbt *DBType) Set(v string) (err error) {
*dbt, err = ResolveDBType(v)
return
}
// String satisfies flag.Value
func (dbt *DBType) String() string {
return strings.ToLower(string(*dbt))
}

View File

@ -25,13 +25,13 @@ func HandleZeroAddrPointer(to *common.Address) string {
if to == nil { if to == nil {
return "" return ""
} }
return to.Hex() return to.String()
} }
// HandleZeroAddr will return an empty string for a 0 value address // HandleZeroAddr will return an empty string for a 0 value address
func HandleZeroAddr(to common.Address) string { func HandleZeroAddr(to common.Address) string {
if to.Hex() == "0x0000000000000000000000000000000000000000" { if to == (common.Address{}) {
return "" return ""
} }
return to.Hex() return to.String()
} }

View File

@ -5,7 +5,7 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
. "github.com/ethereum/go-ethereum/statediff/indexer/shared/schema" . "github.com/cerc-io/plugeth-statediff/indexer/shared/schema"
) )
var testHeaderTable = Table{ var testHeaderTable = Table{

View File

@ -21,21 +21,20 @@ import (
"sort" "sort"
"testing" "testing"
"github.com/stretchr/testify/assert"
"github.com/ipfs/go-cid"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file" "github.com/ipfs/go-cid"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/stretchr/testify/assert"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/models" "github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/shared" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
"github.com/cerc-io/plugeth-statediff/indexer/test_helpers"
) )
// SetupTestData indexes a single mock block along with it's state nodes // SetupTestData indexes a single mock block along with it's state nodes
@ -69,7 +68,7 @@ func SetupTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
} }
} }
func TestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) {
pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, coinbase pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, coinbase
FROM eth.header_cids FROM eth.header_cids
WHERE block_number = $1` WHERE block_number = $1`
@ -107,7 +106,7 @@ func TestPublishAndIndexHeaderIPLDs(t *testing.T, db sql.Database) {
require.Equal(t, mocks.MockHeaderRlp, data) require.Equal(t, mocks.MockHeaderRlp, data)
} }
func TestPublishAndIndexTransactionIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexTransactionIPLDs(t *testing.T, db sql.Database) {
// check that txs were properly indexed and published // check that txs were properly indexed and published
trxs := make([]string, 0) trxs := make([]string, 0)
pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash) pgStr := `SELECT transaction_cids.cid FROM eth.transaction_cids INNER JOIN eth.header_cids ON (transaction_cids.header_id = header_cids.block_hash)
@ -209,7 +208,7 @@ func TestPublishAndIndexTransactionIPLDs(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexLogIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexLogIPLDs(t *testing.T, db sql.Database) {
rcts := make([]string, 0) rcts := make([]string, 0)
rctsPgStr := `SELECT receipt_cids.cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids rctsPgStr := `SELECT receipt_cids.cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids
WHERE receipt_cids.tx_id = transaction_cids.tx_hash WHERE receipt_cids.tx_id = transaction_cids.tx_hash
@ -251,7 +250,7 @@ func TestPublishAndIndexLogIPLDs(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexReceiptIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexReceiptIPLDs(t *testing.T, db sql.Database) {
// check receipts were properly indexed and published // check receipts were properly indexed and published
rcts := make([]string, 0) rcts := make([]string, 0)
pgStr := `SELECT receipt_cids.cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids pgStr := `SELECT receipt_cids.cid FROM eth.receipt_cids, eth.transaction_cids, eth.header_cids
@ -341,7 +340,7 @@ func TestPublishAndIndexReceiptIPLDs(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexStateIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexStateIPLDs(t *testing.T, db sql.Database) {
// check that state nodes were properly indexed and published // check that state nodes were properly indexed and published
stateNodes := make([]models.StateNodeModel, 0) stateNodes := make([]models.StateNodeModel, 0)
pgStr := `SELECT state_cids.cid, CAST(state_cids.block_number as TEXT), state_cids.state_leaf_key, state_cids.removed, pgStr := `SELECT state_cids.cid, CAST(state_cids.block_number as TEXT), state_cids.state_leaf_key, state_cids.removed,
@ -366,7 +365,7 @@ func TestPublishAndIndexStateIPLDs(t *testing.T, db sql.Database) {
} }
if stateNode.CID == state1CID.String() { if stateNode.CID == state1CID.String() {
require.Equal(t, false, stateNode.Removed) require.Equal(t, false, stateNode.Removed)
require.Equal(t, common.BytesToHash(mocks.ContractLeafKey).Hex(), stateNode.StateKey) require.Equal(t, common.BytesToHash(mocks.ContractLeafKey).String(), stateNode.StateKey)
require.Equal(t, mocks.ContractLeafNode, data) require.Equal(t, mocks.ContractLeafNode, data)
require.Equal(t, mocks.BlockNumber.String(), stateNode.BlockNumber) require.Equal(t, mocks.BlockNumber.String(), stateNode.BlockNumber)
require.Equal(t, "0", stateNode.Balance) require.Equal(t, "0", stateNode.Balance)
@ -377,7 +376,7 @@ func TestPublishAndIndexStateIPLDs(t *testing.T, db sql.Database) {
} }
if stateNode.CID == state2CID.String() { if stateNode.CID == state2CID.String() {
require.Equal(t, false, stateNode.Removed) require.Equal(t, false, stateNode.Removed)
require.Equal(t, common.BytesToHash(mocks.AccountLeafKey).Hex(), stateNode.StateKey) require.Equal(t, common.BytesToHash(mocks.AccountLeafKey).String(), stateNode.StateKey)
require.Equal(t, mocks.AccountLeafNode, data) require.Equal(t, mocks.AccountLeafNode, data)
require.Equal(t, mocks.BlockNumber.String(), stateNode.BlockNumber) require.Equal(t, mocks.BlockNumber.String(), stateNode.BlockNumber)
require.Equal(t, mocks.Balance.String(), stateNode.Balance) require.Equal(t, mocks.Balance.String(), stateNode.Balance)
@ -412,11 +411,11 @@ func TestPublishAndIndexStateIPLDs(t *testing.T, db sql.Database) {
t.Fatal(err) t.Fatal(err)
} }
if common.BytesToHash(mocks.RemovedLeafKey).Hex() == stateNode.StateKey { if common.BytesToHash(mocks.RemovedLeafKey).String() == stateNode.StateKey {
require.Equal(t, shared.RemovedNodeStateCID, stateNode.CID) require.Equal(t, shared.RemovedNodeStateCID, stateNode.CID)
require.Equal(t, true, stateNode.Removed) require.Equal(t, true, stateNode.Removed)
require.Equal(t, []byte{}, data) require.Equal(t, []byte{}, data)
} else if common.BytesToHash(mocks.Contract2LeafKey).Hex() == stateNode.StateKey { } else if common.BytesToHash(mocks.Contract2LeafKey).String() == stateNode.StateKey {
require.Equal(t, shared.RemovedNodeStateCID, stateNode.CID) require.Equal(t, shared.RemovedNodeStateCID, stateNode.CID)
require.Equal(t, true, stateNode.Removed) require.Equal(t, true, stateNode.Removed)
require.Equal(t, []byte{}, data) require.Equal(t, []byte{}, data)
@ -439,7 +438,7 @@ type StorageNodeModel struct {
} }
*/ */
func TestPublishAndIndexStorageIPLDs(t *testing.T, db sql.Database) { func DoTestPublishAndIndexStorageIPLDs(t *testing.T, db sql.Database) {
// check that storage nodes were properly indexed // check that storage nodes were properly indexed
storageNodes := make([]models.StorageNodeModel, 0) storageNodes := make([]models.StorageNodeModel, 0)
pgStr := `SELECT cast(storage_cids.block_number AS TEXT), storage_cids.header_id, storage_cids.cid, pgStr := `SELECT cast(storage_cids.block_number AS TEXT), storage_cids.header_id, storage_cids.cid,
@ -455,11 +454,11 @@ func TestPublishAndIndexStorageIPLDs(t *testing.T, db sql.Database) {
require.Equal(t, 1, len(storageNodes)) require.Equal(t, 1, len(storageNodes))
require.Equal(t, models.StorageNodeModel{ require.Equal(t, models.StorageNodeModel{
BlockNumber: mocks.BlockNumber.String(), BlockNumber: mocks.BlockNumber.String(),
HeaderID: mockBlock.Header().Hash().Hex(), HeaderID: mockBlock.Header().Hash().String(),
CID: storageCID.String(), CID: storageCID.String(),
Removed: false, Removed: false,
StorageKey: common.BytesToHash(mocks.StorageLeafKey).Hex(), StorageKey: common.BytesToHash(mocks.StorageLeafKey).String(),
StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), StateKey: common.BytesToHash(mocks.ContractLeafKey).String(),
Value: mocks.StorageValue, Value: mocks.StorageValue,
}, storageNodes[0]) }, storageNodes[0])
var data []byte var data []byte
@ -489,29 +488,29 @@ func TestPublishAndIndexStorageIPLDs(t *testing.T, db sql.Database) {
expectedStorageNodes := []models.StorageNodeModel{ // TODO: ordering is non-deterministic expectedStorageNodes := []models.StorageNodeModel{ // TODO: ordering is non-deterministic
{ {
BlockNumber: mocks.BlockNumber.String(), BlockNumber: mocks.BlockNumber.String(),
HeaderID: mockBlock.Header().Hash().Hex(), HeaderID: mockBlock.Header().Hash().String(),
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
StorageKey: common.BytesToHash(mocks.Storage2LeafKey).Hex(), StorageKey: common.BytesToHash(mocks.Storage2LeafKey).String(),
StateKey: common.BytesToHash(mocks.Contract2LeafKey).Hex(), StateKey: common.BytesToHash(mocks.Contract2LeafKey).String(),
Value: []byte{}, Value: []byte{},
}, },
{ {
BlockNumber: mocks.BlockNumber.String(), BlockNumber: mocks.BlockNumber.String(),
HeaderID: mockBlock.Header().Hash().Hex(), HeaderID: mockBlock.Header().Hash().String(),
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
StorageKey: common.BytesToHash(mocks.Storage3LeafKey).Hex(), StorageKey: common.BytesToHash(mocks.Storage3LeafKey).String(),
StateKey: common.BytesToHash(mocks.Contract2LeafKey).Hex(), StateKey: common.BytesToHash(mocks.Contract2LeafKey).String(),
Value: []byte{}, Value: []byte{},
}, },
{ {
BlockNumber: mocks.BlockNumber.String(), BlockNumber: mocks.BlockNumber.String(),
HeaderID: mockBlock.Header().Hash().Hex(), HeaderID: mockBlock.Header().Hash().String(),
CID: shared.RemovedNodeStorageCID, CID: shared.RemovedNodeStorageCID,
Removed: true, Removed: true,
StorageKey: common.BytesToHash(mocks.RemovedLeafKey).Hex(), StorageKey: common.BytesToHash(mocks.RemovedLeafKey).String(),
StateKey: common.BytesToHash(mocks.ContractLeafKey).Hex(), StateKey: common.BytesToHash(mocks.ContractLeafKey).String(),
Value: []byte{}, Value: []byte{},
}, },
} }
@ -690,7 +689,7 @@ func TestPublishAndIndexHeaderNonCanonical(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexTransactionsNonCanonical(t *testing.T, db sql.Database) { func DoTestPublishAndIndexTransactionsNonCanonical(t *testing.T, db sql.Database) {
// check indexed transactions // check indexed transactions
pgStr := `SELECT CAST(block_number as TEXT), header_id, tx_hash, cid, dst, src, index, pgStr := `SELECT CAST(block_number as TEXT), header_id, tx_hash, cid, dst, src, index,
tx_type, CAST(value as TEXT) tx_type, CAST(value as TEXT)
@ -855,7 +854,7 @@ func TestPublishAndIndexTransactionsNonCanonical(t *testing.T, db sql.Database)
} }
} }
func TestPublishAndIndexReceiptsNonCanonical(t *testing.T, db sql.Database) { func DoTestPublishAndIndexReceiptsNonCanonical(t *testing.T, db sql.Database) {
// check indexed receipts // check indexed receipts
pgStr := `SELECT CAST(block_number as TEXT), header_id, tx_id, cid, post_status, post_state, contract pgStr := `SELECT CAST(block_number as TEXT), header_id, tx_id, cid, post_status, post_state, contract
FROM eth.receipt_cids FROM eth.receipt_cids
@ -947,7 +946,7 @@ func TestPublishAndIndexReceiptsNonCanonical(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexLogsNonCanonical(t *testing.T, db sql.Database) { func DoTestPublishAndIndexLogsNonCanonical(t *testing.T, db sql.Database) {
// check indexed logs // check indexed logs
pgStr := `SELECT address, topic0, topic1, topic2, topic3, data pgStr := `SELECT address, topic0, topic1, topic2, topic3, data
FROM eth.log_cids FROM eth.log_cids
@ -1002,7 +1001,7 @@ func TestPublishAndIndexLogsNonCanonical(t *testing.T, db sql.Database) {
for i, log := range mockRct.rct.Logs { for i, log := range mockRct.rct.Logs {
topicSet := make([]string, 4) topicSet := make([]string, 4)
for ti, topic := range log.Topics { for ti, topic := range log.Topics {
topicSet[ti] = topic.Hex() topicSet[ti] = topic.String()
} }
expectedLog := models.LogsModel{ expectedLog := models.LogsModel{
@ -1021,7 +1020,7 @@ func TestPublishAndIndexLogsNonCanonical(t *testing.T, db sql.Database) {
} }
} }
func TestPublishAndIndexStateNonCanonical(t *testing.T, db sql.Database) { func DoTestPublishAndIndexStateNonCanonical(t *testing.T, db sql.Database) {
// check indexed state nodes // check indexed state nodes
pgStr := `SELECT state_leaf_key, removed, cid, diff pgStr := `SELECT state_leaf_key, removed, cid, diff
FROM eth.state_cids FROM eth.state_cids
@ -1035,7 +1034,7 @@ func TestPublishAndIndexStateNonCanonical(t *testing.T, db sql.Database) {
expectedStateNodes := make([]models.StateNodeModel, 0) expectedStateNodes := make([]models.StateNodeModel, 0)
for i, stateDiff := range mocks.StateDiffs { for i, stateDiff := range mocks.StateDiffs {
expectedStateNodes = append(expectedStateNodes, models.StateNodeModel{ expectedStateNodes = append(expectedStateNodes, models.StateNodeModel{
StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).Hex(), StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).String(),
Removed: stateDiff.Removed, Removed: stateDiff.Removed,
CID: stateNodeCIDs[i].String(), CID: stateNodeCIDs[i].String(),
Diff: true, Diff: true,
@ -1046,7 +1045,7 @@ func TestPublishAndIndexStateNonCanonical(t *testing.T, db sql.Database) {
expectedNonCanonicalBlock2StateNodes := make([]models.StateNodeModel, 0) expectedNonCanonicalBlock2StateNodes := make([]models.StateNodeModel, 0)
for i, stateDiff := range mocks.StateDiffs[:2] { for i, stateDiff := range mocks.StateDiffs[:2] {
expectedNonCanonicalBlock2StateNodes = append(expectedNonCanonicalBlock2StateNodes, models.StateNodeModel{ expectedNonCanonicalBlock2StateNodes = append(expectedNonCanonicalBlock2StateNodes, models.StateNodeModel{
StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).Hex(), StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).String(),
Removed: stateDiff.Removed, Removed: stateDiff.Removed,
CID: stateNodeCIDs[i].String(), CID: stateNodeCIDs[i].String(),
Diff: true, Diff: true,
@ -1082,7 +1081,7 @@ func TestPublishAndIndexStateNonCanonical(t *testing.T, db sql.Database) {
assert.ElementsMatch(t, expectedNonCanonicalBlock2StateNodes, stateNodes) assert.ElementsMatch(t, expectedNonCanonicalBlock2StateNodes, stateNodes)
} }
func TestPublishAndIndexStorageNonCanonical(t *testing.T, db sql.Database) { func DoTestPublishAndIndexStorageNonCanonical(t *testing.T, db sql.Database) {
// check indexed storage nodes // check indexed storage nodes
pgStr := `SELECT storage_leaf_key, state_leaf_key, removed, cid, diff, val pgStr := `SELECT storage_leaf_key, state_leaf_key, removed, cid, diff, val
FROM eth.storage_cids FROM eth.storage_cids
@ -1098,8 +1097,8 @@ func TestPublishAndIndexStorageNonCanonical(t *testing.T, db sql.Database) {
for _, stateDiff := range mocks.StateDiffs { for _, stateDiff := range mocks.StateDiffs {
for _, storageNode := range stateDiff.StorageDiff { for _, storageNode := range stateDiff.StorageDiff {
expectedStorageNodes = append(expectedStorageNodes, models.StorageNodeModel{ expectedStorageNodes = append(expectedStorageNodes, models.StorageNodeModel{
StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).Hex(), StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).Hex(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
Removed: storageNode.Removed, Removed: storageNode.Removed,
CID: storageNodeCIDs[storageNodeIndex].String(), CID: storageNodeCIDs[storageNodeIndex].String(),
Diff: true, Diff: true,
@ -1115,8 +1114,8 @@ func TestPublishAndIndexStorageNonCanonical(t *testing.T, db sql.Database) {
for _, stateDiff := range mocks.StateDiffs[:2] { for _, stateDiff := range mocks.StateDiffs[:2] {
for _, storageNode := range stateDiff.StorageDiff { for _, storageNode := range stateDiff.StorageDiff {
expectedNonCanonicalBlock2StorageNodes = append(expectedNonCanonicalBlock2StorageNodes, models.StorageNodeModel{ expectedNonCanonicalBlock2StorageNodes = append(expectedNonCanonicalBlock2StorageNodes, models.StorageNodeModel{
StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).Hex(), StateKey: common.BytesToHash(stateDiff.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).Hex(), StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
Removed: storageNode.Removed, Removed: storageNode.Removed,
CID: storageNodeCIDs[storageNodeIndex].String(), CID: storageNodeCIDs[storageNodeIndex].String(),
Diff: true, Diff: true,

View File

@ -24,12 +24,13 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ipfs/go-cid" "github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash" "github.com/multiformats/go-multihash"
"github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/cerc-io/plugeth-statediff/indexer/models"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
) )
var ( var (

View File

@ -20,13 +20,13 @@ 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/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/ipld"
"github.com/cerc-io/plugeth-statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ipfs/go-cid" "github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash" "github.com/multiformats/go-multihash"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"

View File

@ -19,11 +19,11 @@ 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/mocks"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )

View File

@ -21,10 +21,11 @@ import (
"testing" "testing"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/cerc-io/plugeth-statediff/indexer/database/sql"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/mocks"
) )
type res struct { type res struct {

View File

@ -22,7 +22,7 @@ import (
"os" "os"
"testing" "testing"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql" "github.com/cerc-io/plugeth-statediff/indexer/database/sql"
) )
// ListContainsString used to check if a list of strings contains a particular string // ListContainsString used to check if a list of strings contains a particular string

185
main/flags.go Normal file
View File

@ -0,0 +1,185 @@
package main
import (
"context"
"flag"
"os"
"github.com/cerc-io/plugeth-statediff"
"github.com/cerc-io/plugeth-statediff/indexer/database/dump"
"github.com/cerc-io/plugeth-statediff/indexer/database/file"
"github.com/cerc-io/plugeth-statediff/indexer/database/sql/postgres"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/shared"
"github.com/cerc-io/plugeth-statediff/utils"
)
var (
Flags = *flag.NewFlagSet("statediff", flag.PanicOnError)
enableStatediff bool
config = statediff.Config{
Context: context.Background(),
}
dbType = shared.POSTGRES
dbDumpDst = dump.STDOUT
dbConfig = postgres.Config{Driver: postgres.PGX}
fileConfig = file.Config{Mode: file.CSV}
)
func init() {
Flags.BoolVar(&enableStatediff,
"statediff", false,
"Enables the processing of state diffs between each block",
)
Flags.BoolVar(&config.EnableWriteLoop,
"statediff.writing", false,
"Activates progressive writing of state diffs to database as new blocks are synced",
)
Flags.StringVar(&config.ID,
"statediff.db.nodeid", "",
"Node ID to use when writing state diffs to database",
)
Flags.StringVar(&config.ClientName,
"statediff.db.clientname", "go-ethereum",
"Client name to use when writing state diffs to database",
)
Flags.UintVar(&config.NumWorkers,
"statediff.workers", 1,
"Number of concurrent workers to use during statediff processing (default 1)",
)
Flags.BoolVar(&config.WaitForSync,
"statediff.waitforsync", false,
"Should the statediff service wait for geth to catch up to the head of the chain?",
)
Flags.Var(&dbType,
"statediff.db.type",
"Statediff database type (current options: postgres, file, dump)",
)
Flags.StringVar(&dbDumpDst,
"statediff.dump.dst", "stdout",
"Statediff database dump destination (default is stdout)",
)
Flags.Var(&dbConfig.Driver,
"statediff.db.driver",
"Statediff database driver type",
)
Flags.StringVar(&dbConfig.Hostname,
"statediff.db.host", "localhost",
"Statediff database hostname/ip",
)
Flags.IntVar(&dbConfig.Port,
"statediff.db.port", 5432,
"Statediff database port",
)
Flags.StringVar(&dbConfig.DatabaseName,
"statediff.db.name", "",
"Statediff database name",
)
Flags.StringVar(&dbConfig.Password,
"statediff.db.password", "",
"Statediff database password",
)
Flags.StringVar(&dbConfig.Username,
"statediff.db.user", "postgres",
"Statediff database username",
)
Flags.DurationVar(&dbConfig.MaxConnLifetime,
"statediff.db.maxconnlifetime", 0,
"Statediff database maximum connection lifetime (in seconds)",
)
Flags.DurationVar(&dbConfig.MaxConnIdleTime,
"statediff.db.maxconnidletime", 0,
"Statediff database maximum connection idle time (in seconds)",
)
Flags.IntVar(&dbConfig.MaxConns,
"statediff.db.maxconns", 0,
"Statediff database maximum connections",
)
Flags.IntVar(&dbConfig.MinConns,
"statediff.db.minconns", 0,
"Statediff database minimum connections",
)
Flags.IntVar(&dbConfig.MaxIdle,
"statediff.db.maxidleconns", 0,
"Statediff database maximum idle connections",
)
Flags.DurationVar(&dbConfig.ConnTimeout,
"statediff.db.conntimeout", 0,
"Statediff database connection timeout (in seconds)",
)
Flags.BoolVar(&dbConfig.Upsert,
"statediff.db.upsert", false,
"Should the statediff service overwrite data existing in the database?",
)
Flags.BoolVar(&dbConfig.CopyFrom,
"statediff.db.copyfrom", false,
"Should the statediff service use COPY FROM for multiple inserts? (Note: pgx only)",
)
Flags.BoolVar(&dbConfig.LogStatements,
"statediff.db.logstatements", false,
"Should the statediff service log all database statements? (Note: pgx only)",
)
Flags.Var(&fileConfig.Mode,
"statediff.file.mode",
"Statediff file writing mode (current options: csv, sql)",
)
Flags.StringVar(&fileConfig.OutputDir,
"statediff.file.csvdir", "",
"Full path of output directory to write statediff data out to when operating in csv file mode",
)
Flags.StringVar(&fileConfig.FilePath,
"statediff.file.path", "",
"Full path (including filename) to write statediff data out to when operating in sql file mode",
)
Flags.StringVar(&fileConfig.WatchedAddressesFilePath,
"statediff.file.wapath", "",
"Full path (including filename) to write statediff watched addresses out to when operating in file mode",
)
}
func GetConfig() statediff.Config {
initConfig()
return config
}
func initConfig() {
if !enableStatediff {
config = statediff.Config{}
return
}
if !config.EnableWriteLoop {
return
}
if config.ID == "" {
utils.Fatalf("Must specify node ID for statediff DB output")
}
var indexerConfig interfaces.Config
switch dbType {
case shared.FILE:
indexerConfig = fileConfig
case shared.POSTGRES:
dbConfig.ID = config.ID
dbConfig.ClientName = config.ClientName
indexerConfig = dbConfig
case shared.DUMP:
switch dbDumpDst {
case dump.STDERR:
indexerConfig = dump.Config{Dump: os.Stdout}
case dump.STDOUT:
indexerConfig = dump.Config{Dump: os.Stderr}
case dump.DISCARD:
indexerConfig = dump.Config{Dump: dump.Discard}
default:
utils.Fatalf("unrecognized dump destination: %s", dbDumpDst)
}
default:
utils.Fatalf("unrecognized database type: %s", dbType)
}
config.IndexerConfig = indexerConfig
}

75
main/main.go Normal file
View File

@ -0,0 +1,75 @@
package main
import (
"strconv"
geth_flags "github.com/ethereum/go-ethereum/cmd/utils"
"github.com/openrelayxyz/plugeth-utils/core"
"github.com/openrelayxyz/plugeth-utils/restricted"
"github.com/cerc-io/plugeth-statediff"
"github.com/cerc-io/plugeth-statediff/utils/log"
)
var (
pluginLoader core.PluginLoader
gethContext core.Context
service *statediff.Service
)
func Initialize(ctx core.Context, pl core.PluginLoader, logger core.Logger) {
log.DefaultLogger = logger
pluginLoader = pl
gethContext = ctx
log.Info("Initialized statediff plugin")
}
func InitializeNode(stack core.Node, b core.Backend) {
backend := b.(restricted.Backend)
networkid, err := strconv.ParseUint(gethContext.String(geth_flags.NetworkIdFlag.Name), 10, 64)
if err != nil {
log.Error("cannot parse network ID", "error", err)
return
}
serviceConfig := GetConfig()
service, err = statediff.NewIndexingService(serviceConfig, backend, networkid)
if err != nil {
log.Error("failed to construct service", "error", err)
return
}
if err = service.Start(); err != nil {
log.Error("failed to start service", "error", err)
return
}
log.Info("Initialized node and backend")
}
func GetAPIs(stack core.Node, backend core.Backend) []core.API {
return []core.API{
{
Namespace: statediff.APIName,
Version: statediff.APIVersion,
Service: statediff.NewPublicAPI(service),
Public: true,
},
}
}
// StateUpdate gives us updates about state changes made in each block.
// We extract contract code here, since it's not exposed by plugeth's state interfaces.
func StateUpdate(blockRoot core.Hash,
parentRoot core.Hash,
destructs map[core.Hash]struct{},
accounts map[core.Hash][]byte,
storage map[core.Hash]map[core.Hash][]byte,
codeUpdates map[core.Hash][]byte) {
if service == nil {
log.Error("StateUpdate called before InitializeNode", "root", blockRoot)
return
}
// TODO extract code
}

View File

@ -18,13 +18,11 @@ package statediff_test
import ( import (
"bytes" "bytes"
"encoding/json"
"fmt" "fmt"
"io" "io"
"log" "log"
"math/big" "math/big"
"os" "os"
"sort"
"testing" "testing"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
@ -36,10 +34,11 @@ import (
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld" "github.com/cerc-io/plugeth-statediff"
"github.com/ethereum/go-ethereum/statediff/test_helpers" "github.com/cerc-io/plugeth-statediff/indexer/ipld"
sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/test_helpers"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
) )
var ( var (
@ -480,13 +479,8 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
t.Error(err) t.Error(err)
} }
params := statediff.Params{} params := statediff.Params{}
builder = statediff.NewBuilder(chain.StateCache())
var tests = []struct { var tests = []test_helpers.TestCase{
name string
startingArguments statediff.Args
expected *sdtypes.StateObject
}{
// note that block0 (genesis) has over 1000 nodes due to the pre-allocation for the crowd-sale // note that block0 (genesis) has over 1000 nodes due to the pre-allocation for the crowd-sale
// it is not feasible to write a unit test of that size at this time // it is not feasible to write a unit test of that size at this time
{ {
@ -507,26 +501,26 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
AccountWrapper: sdtypes.AccountWrapper{ AccountWrapper: sdtypes.AccountWrapper{
Account: block1CoinbaseAccount, Account: block1CoinbaseAccount,
LeafKey: block1CoinbaseHash.Bytes(), LeafKey: block1CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(),
}, },
StorageDiff: emptyStorage, StorageDiff: emptyStorage,
}, },
}, },
IPLDs: []sdtypes.IPLD{ IPLDs: []sdtypes.IPLD{
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1RootBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1RootBranchNode)).String(),
Content: block1RootBranchNode, Content: block1RootBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1x04BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1x04BranchNode)).String(),
Content: block1x04BranchNode, Content: block1x04BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1x040bBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1x040bBranchNode)).String(),
Content: block1x040bBranchNode, Content: block1x040bBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(),
Content: block1CoinbaseLeafNode, Content: block1CoinbaseLeafNode,
}, },
}, },
@ -552,34 +546,34 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
AccountWrapper: sdtypes.AccountWrapper{ AccountWrapper: sdtypes.AccountWrapper{
Account: block2CoinbaseAccount, Account: block2CoinbaseAccount,
LeafKey: block2CoinbaseHash.Bytes(), LeafKey: block2CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(),
}, },
StorageDiff: emptyStorage, StorageDiff: emptyStorage,
}, },
}, },
IPLDs: []sdtypes.IPLD{ IPLDs: []sdtypes.IPLD{
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2RootBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2RootBranchNode)).String(),
Content: block2RootBranchNode, Content: block2RootBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x00BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2x00BranchNode)).String(),
Content: block2x00BranchNode, Content: block2x00BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x0008BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2x0008BranchNode)).String(),
Content: block2x0008BranchNode, Content: block2x0008BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x00080dBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2x00080dBranchNode)).String(),
Content: block2x00080dBranchNode, Content: block2x00080dBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2MovedPremineLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2MovedPremineLeafNode)).String(),
Content: block2MovedPremineLeafNode, Content: block2MovedPremineLeafNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(),
Content: block2CoinbaseLeafNode, Content: block2CoinbaseLeafNode,
}, },
}, },
@ -604,7 +598,7 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
AccountWrapper: sdtypes.AccountWrapper{ AccountWrapper: sdtypes.AccountWrapper{
Account: block3MovedPremineAccount1, Account: block3MovedPremineAccount1,
LeafKey: common.HexToHash("ce573ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190").Bytes(), LeafKey: common.HexToHash("ce573ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190").Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(),
}, },
StorageDiff: emptyStorage, StorageDiff: emptyStorage,
}, },
@ -613,50 +607,50 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
AccountWrapper: sdtypes.AccountWrapper{ AccountWrapper: sdtypes.AccountWrapper{
Account: block3CoinbaseAccount, Account: block3CoinbaseAccount,
LeafKey: block3CoinbaseHash.Bytes(), LeafKey: block3CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(),
}, },
StorageDiff: emptyStorage, StorageDiff: emptyStorage,
}, },
}, },
IPLDs: []sdtypes.IPLD{ IPLDs: []sdtypes.IPLD{
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3RootBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3RootBranchNode)).String(),
Content: block3RootBranchNode, Content: block3RootBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x06BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x06BranchNode)).String(),
Content: block3x06BranchNode, Content: block3x06BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x060eBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x060eBranchNode)).String(),
Content: block3x060eBranchNode, Content: block3x060eBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0cBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x0cBranchNode)).String(),
Content: block3x0cBranchNode, Content: block3x0cBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0eBranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x0c0eBranchNode)).String(),
Content: block3x0c0eBranchNode, Content: block3x0c0eBranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0e05BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x0c0e05BranchNode)).String(),
Content: block3x0c0e05BranchNode, Content: block3x0c0e05BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0e0507BranchNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3x0c0e0507BranchNode)).String(),
Content: block3x0c0e0507BranchNode, Content: block3x0c0e0507BranchNode,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(),
Content: block3MovedPremineLeafNode1, Content: block3MovedPremineLeafNode1,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode2)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode2)).String(),
Content: block3MovedPremineLeafNode2, Content: block3MovedPremineLeafNode2,
}, },
{ {
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(), CID: ipld.Keccak256ToCid(ipld.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(),
Content: block3CoinbaseLeafNode, Content: block3CoinbaseLeafNode,
}, },
}, },
@ -664,41 +658,12 @@ func TestBuilderOnMainnetBlocks(t *testing.T) {
}, },
} }
for _, test := range tests { test_helpers.RunBuilderTests(t,
diff, err := builder.BuildStateDiffObject(test.startingArguments, params) statediff.NewBuilder(statediff.StateDatabaseView(chain.StateCache())),
if err != nil { tests, params, test_helpers.CheckedRoots{
t.Error(err) block1: block1RootBranchNode,
} block2: block2RootBranchNode,
receivedStateDiffRlp, err := rlp.EncodeToBytes(diff) block3: block3RootBranchNode,
if err != nil { })
t.Error(err)
}
expectedStateDiffRlp, err := rlp.EncodeToBytes(&test.expected)
if err != nil {
t.Error(err)
}
sort.Slice(receivedStateDiffRlp, func(i, j int) bool { return receivedStateDiffRlp[i] < receivedStateDiffRlp[j] })
sort.Slice(expectedStateDiffRlp, func(i, j int) bool { return expectedStateDiffRlp[i] < expectedStateDiffRlp[j] })
if !bytes.Equal(receivedStateDiffRlp, expectedStateDiffRlp) {
actual, err := json.Marshal(diff)
if err != nil {
t.Error(err)
}
expected, err := json.Marshal(test.expected)
if err != nil {
t.Error(err)
}
t.Logf("Test failed: %s", test.name)
t.Errorf("actual state diff: %s\r\n\r\n\r\nexpected state diff: %s", actual, expected)
}
}
if !bytes.Equal(crypto.Keccak256(block1RootBranchNode), block1.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block1RootBranchNode), block1.Root().Bytes())
}
if !bytes.Equal(crypto.Keccak256(block2RootBranchNode), block2.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block2RootBranchNode), block2.Root().Bytes())
}
if !bytes.Equal(crypto.Keccak256(block3RootBranchNode), block3.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block3RootBranchNode), block3.Root().Bytes())
}
} }

View File

@ -20,13 +20,13 @@ import (
"fmt" "fmt"
"time" "time"
"github.com/cerc-io/plugeth-statediff/utils/log"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
) )
func countStateDiffBegin(block *types.Block) (time.Time, log.Logger) { func countStateDiffBegin(block *types.Block) (time.Time, log.Logger) {
start := time.Now() start := time.Now()
logger := log.New("hash", block.Hash().Hex(), "number", block.NumberU64()) logger := log.New("hash", block.Hash().String(), "number", block.NumberU64())
defaultStatediffMetrics.underway.Inc(1) defaultStatediffMetrics.underway.Inc(1)
logger.Debug(fmt.Sprintf("writeStateDiff BEGIN [underway=%d, succeeded=%d, failed=%d, total_time=%dms]", logger.Debug(fmt.Sprintf("writeStateDiff BEGIN [underway=%d, succeeded=%d, failed=%d, total_time=%dms]",

View File

@ -28,25 +28,23 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/internal/ethapi"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/rpc"
ind "github.com/ethereum/go-ethereum/statediff/indexer"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
plugeth "github.com/openrelayxyz/plugeth-utils/core"
"github.com/openrelayxyz/plugeth-utils/restricted"
"github.com/thoas/go-funk" "github.com/thoas/go-funk"
"github.com/cerc-io/plugeth-statediff/adapt"
ind "github.com/cerc-io/plugeth-statediff/indexer"
"github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/cerc-io/plugeth-statediff/indexer/interfaces"
"github.com/cerc-io/plugeth-statediff/indexer/node"
types2 "github.com/cerc-io/plugeth-statediff/types"
"github.com/cerc-io/plugeth-statediff/utils/log"
) )
const ( const (
@ -67,29 +65,26 @@ var writeLoopParams = ParamsWithMutex{
}, },
} }
type blockChain interface { type RpcID int
SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription
CurrentBlock() *types.Header
GetBlockByHash(hash common.Hash) *types.Block
GetBlockByNumber(number uint64) *types.Block
GetReceiptsByHash(hash common.Hash) types.Receipts
GetTd(hash common.Hash, number uint64) *big.Int
UnlockTrie(root common.Hash)
StateCache() state.Database
}
// IService is the state-diffing service interface // IService is the state-diffing service interface
type IService interface { type IService interface {
// Lifecycle Start() and Stop() methods // Lifecycle methods
node.Lifecycle Start() error
Stop() error
// APIs method for getting API(s) for this service // APIs method for getting API(s) for this service
APIs() []rpc.API // APIs() []rpc.API
// Loop is the main event loop for processing state diffs // Loop is the main event loop for processing state diffs
Loop(chainEventCh chan core.ChainEvent) Loop(chainEventCh chan core.ChainEvent)
// WriteLoop event loop for progressively processing and writing diffs directly to DB
WriteLoop(chainEventCh chan core.ChainEvent)
// Subscribe method to subscribe to receive state diff processing output // Subscribe method to subscribe to receive state diff processing output
Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool, params Params) Subscribe(sub chan<- Payload, quitChan chan<- bool, params Params) RpcID
// Unsubscribe method to unsubscribe from state diff processing // Unsubscribe method to unsubscribe from state diff processing
Unsubscribe(id rpc.ID) error Unsubscribe(id RpcID) error
// StateDiffAt method to get state diff object at specific block // StateDiffAt method to get state diff object at specific block
StateDiffAt(blockNumber uint64, params Params) (*Payload, error) StateDiffAt(blockNumber uint64, params Params) (*Payload, error)
// StateDiffFor method to get state diff object at specific block // StateDiffFor method to get state diff object at specific block
@ -98,54 +93,62 @@ type IService interface {
WriteStateDiffAt(blockNumber uint64, params Params) JobID WriteStateDiffAt(blockNumber uint64, params Params) JobID
// WriteStateDiffFor method to write state diff object directly to DB // WriteStateDiffFor method to write state diff object directly to DB
WriteStateDiffFor(blockHash common.Hash, params Params) error WriteStateDiffFor(blockHash common.Hash, params Params) error
// WriteLoop event loop for progressively processing and writing diffs directly to DB
WriteLoop(chainEventCh chan core.ChainEvent)
// WatchAddress method to change the addresses being watched in write loop params // WatchAddress method to change the addresses being watched in write loop params
WatchAddress(operation types2.OperationType, args []types2.WatchAddressArg) error WatchAddress(operation types2.OperationType, args []types2.WatchAddressArg) error
// StreamCodeAndCodeHash method to export all the codehash => code mappings at a block height // StreamCodeAndCodeHash method to export all the codehash => code mappings at a block height
StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- types2.CodeAndCodeHash, quitChan chan<- bool) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- types2.CodeAndCodeHash, quitChan chan<- bool)
// SubscribeWriteStatus method to subscribe to receive state diff processing output // SubscribeWriteStatus method to subscribe to receive state diff processing output
SubscribeWriteStatus(id rpc.ID, sub chan<- JobStatus, quitChan chan<- bool) SubscribeWriteStatus(sub chan<- JobStatus, quitChan chan<- bool) RpcID
// UnsubscribeWriteStatus method to unsubscribe from state diff processing // UnsubscribeWriteStatus method to unsubscribe from state diff processing
UnsubscribeWriteStatus(id rpc.ID) error UnsubscribeWriteStatus(id RpcID) error
} }
// Service is the underlying struct for the state diffing service // Service is the underlying struct for the state diffing service
type Service struct { type Service struct {
// Used to sync access to the Subscriptions
sync.Mutex
// Used to build the state diff objects // Used to build the state diff objects
Builder Builder Builder Builder
// Used to subscribe to chain events (blocks) // Used to subscribe to chain events (blocks)
BlockChain blockChain BlockChain blockChain
// Used to signal shutdown of the service
QuitChan chan bool
// A mapping of rpc.IDs to their subscription channels, mapped to their subscription type (hash of the Params rlp)
Subscriptions map[common.Hash]map[rpc.ID]Subscription
// A mapping of subscription params rlp hash to the corresponding subscription params
SubscriptionTypes map[common.Hash]Params
// Cache the last block so that we can avoid having to lookup the next block's parent // Cache the last block so that we can avoid having to lookup the next block's parent
BlockCache BlockCache BlockCache BlockCache
// The publicBackendAPI which provides useful information about the current state // The publicBackendAPI which provides useful information about the current state
BackendAPI ethapi.Backend BackendAPI plugeth.Backend
// Should the statediff service wait for geth to sync to head? // Used to signal shutdown of the service
WaitForSync bool QuitChan chan bool
// Whether we have any subscribers
subscribers int32
// Interface for publishing statediffs as PG-IPLD objects // Interface for publishing statediffs as PG-IPLD objects
indexer interfaces.StateDiffIndexer indexer interfaces.StateDiffIndexer
// Should the statediff service wait for geth to sync to head?
WaitForSync bool
// Whether to enable writing state diffs directly to track blockchain head. // Whether to enable writing state diffs directly to track blockchain head.
enableWriteLoop bool enableWriteLoop bool
// Size of the worker pool // Size of the worker pool
numWorkers uint numWorkers uint
// Number of retry for aborted transactions due to deadlock. // Number of retry for aborted transactions due to deadlock.
maxRetry uint maxRetry uint
// Sequential ID for RPC subscriptions
lastRpcID uint64
// PayloadSubs struct {
// A mapping of RpcIDs to their subscription channels, mapped to their subscription type (hash
// of the Params RLP)
Subscriptions map[common.Hash]map[RpcID]Subscription
// A mapping of subscription params rlp hash to the corresponding subscription params
SubscriptionTypes map[common.Hash]Params
// Number of current subscribers
subscribers int32
// Used to sync access to the Subscriptions
subscriptionsMutex sync.Mutex
// }
// Write job status subscriptions // Write job status subscriptions
jobStatusSubs map[rpc.ID]statusSubscription jobStatusSubs map[RpcID]jobStatusSubscription
// Job ID ticker jobStatusSubsMutex sync.RWMutex
// Sequential ID for write jobs
lastJobID uint64 lastJobID uint64
// In flight jobs (for WriteStateDiffAt) // Map of block number to in-flight jobs (for WriteStateDiffAt)
currentJobs map[uint64]JobID currentJobs map[uint64]JobID
currentJobsMutex sync.Mutex currentJobsMutex sync.Mutex
} }
@ -159,7 +162,7 @@ type JobStatus struct {
Err error Err error
} }
type statusSubscription struct { type jobStatusSubscription struct {
statusChan chan<- JobStatus statusChan chan<- JobStatus
quitChan chan<- bool quitChan chan<- bool
} }
@ -178,67 +181,36 @@ func NewBlockCache(max uint) BlockCache {
} }
} }
// New creates a new statediff.Service // NewIndexingService creates and registers a new Service with indexing configured
// func New(stack *node.Node, ethServ *eth.Ethereum, dbParams *DBParams, enableWriteLoop bool) error { func NewIndexingService(params Config, backend restricted.Backend, networkid uint64) (*Service, error) {
// func New(stack *node.Node, blockChain *core.BlockChain, networkID uint64, params Config, backend ethapi.Backend) error { blockChain := asBlockChain(backend)
func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params Config, backend ethapi.Backend) error {
blockChain := ethServ.BlockChain()
var indexer interfaces.StateDiffIndexer var indexer interfaces.StateDiffIndexer
var err error if params.IndexerConfig != nil {
quitCh := make(chan bool) info := node.Info{
indexerConfigAvailable := params.IndexerConfig != nil GenesisBlock: blockChain.GetBlockByNumber(0).Hash().String(),
if indexerConfigAvailable { NetworkID: strconv.FormatUint(networkid, 10),
info := nodeinfo.Info{ ChainID: backend.ChainConfig().ChainID.Uint64(),
GenesisBlock: blockChain.Genesis().Hash().Hex(),
NetworkID: strconv.FormatUint(cfg.NetworkId, 10),
ChainID: blockChain.Config().ChainID.Uint64(),
ID: params.ID, ID: params.ID,
ClientName: params.ClientName, ClientName: params.ClientName,
} }
var err error var err error
_, indexer, err = ind.NewStateDiffIndexer(params.Context, blockChain.Config(), info, params.IndexerConfig) _, indexer, err = ind.NewStateDiffIndexer(params.Context, adapt.ChainConfig(backend.ChainConfig()), info, params.IndexerConfig)
if err != nil { if err != nil {
return err return nil, err
} }
indexer.ReportDBMetrics(10*time.Second, quitCh)
}
workers := params.NumWorkers
if workers == 0 {
workers = 1
}
sds := &Service{
Mutex: sync.Mutex{},
BlockChain: blockChain,
Builder: NewBuilder(blockChain.StateCache()),
QuitChan: quitCh,
Subscriptions: make(map[common.Hash]map[rpc.ID]Subscription),
SubscriptionTypes: make(map[common.Hash]Params),
BlockCache: NewBlockCache(workers),
BackendAPI: backend,
WaitForSync: params.WaitForSync,
indexer: indexer,
enableWriteLoop: params.EnableWriteLoop,
numWorkers: workers,
maxRetry: defaultRetryLimit,
jobStatusSubs: map[rpc.ID]statusSubscription{},
currentJobs: map[uint64]JobID{},
}
stack.RegisterLifecycle(sds)
stack.RegisterAPIs(sds.APIs())
if indexerConfigAvailable {
err = loadWatchedAddresses(indexer) err = loadWatchedAddresses(indexer)
if err != nil { if err != nil {
return err return nil, err
} }
} }
return nil sds := NewService(blockChain, params, backend, indexer)
// stack.RegisterLifecycle(sds)
// stack.RegisterAPIs(sds.APIs())
return sds, nil
} }
func NewService(blockChain blockChain, cfg Config, backend ethapi.Backend, indexer interfaces.StateDiffIndexer) *Service { func NewService(blockChain blockChain, cfg Config, backend plugeth.Backend, indexer interfaces.StateDiffIndexer) *Service {
workers := cfg.NumWorkers workers := cfg.NumWorkers
if workers == 0 { if workers == 0 {
workers = 1 workers = 1
@ -246,11 +218,10 @@ func NewService(blockChain blockChain, cfg Config, backend ethapi.Backend, index
quitCh := make(chan bool) quitCh := make(chan bool)
sds := &Service{ sds := &Service{
Mutex: sync.Mutex{},
BlockChain: blockChain, BlockChain: blockChain,
Builder: NewBuilder(blockChain.StateCache()), Builder: NewBuilder(blockChain.StateCache()),
QuitChan: quitCh, QuitChan: quitCh,
Subscriptions: make(map[common.Hash]map[rpc.ID]Subscription), Subscriptions: make(map[common.Hash]map[RpcID]Subscription),
SubscriptionTypes: make(map[common.Hash]Params), SubscriptionTypes: make(map[common.Hash]Params),
BlockCache: NewBlockCache(workers), BlockCache: NewBlockCache(workers),
BackendAPI: backend, BackendAPI: backend,
@ -259,7 +230,7 @@ func NewService(blockChain blockChain, cfg Config, backend ethapi.Backend, index
enableWriteLoop: cfg.EnableWriteLoop, enableWriteLoop: cfg.EnableWriteLoop,
numWorkers: workers, numWorkers: workers,
maxRetry: defaultRetryLimit, maxRetry: defaultRetryLimit,
jobStatusSubs: map[rpc.ID]statusSubscription{}, jobStatusSubs: map[RpcID]jobStatusSubscription{},
currentJobs: map[uint64]JobID{}, currentJobs: map[uint64]JobID{},
} }
@ -269,18 +240,13 @@ func NewService(blockChain blockChain, cfg Config, backend ethapi.Backend, index
return sds return sds
} }
// Protocols exports the services p2p protocols, this service has none
func (sds *Service) Protocols() []p2p.Protocol {
return []p2p.Protocol{}
}
// APIs returns the RPC descriptors the statediff.Service offers // APIs returns the RPC descriptors the statediff.Service offers
func (sds *Service) APIs() []rpc.API { func (sds *Service) APIs() []rpc.API {
return []rpc.API{ return []rpc.API{
{ {
Namespace: APIName, Namespace: APIName,
Version: APIVersion, Version: APIVersion,
Service: NewPublicStateDiffAPI(sds), Service: NewPublicAPI(sds),
Public: true, Public: true,
}, },
} }
@ -312,9 +278,9 @@ type workerParams struct {
} }
func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) { func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) {
chainEventSub := sds.BlockChain.SubscribeChainEvent(chainEventCh) sub := sds.BlockChain.SubscribeChainEvent(chainEventCh)
defer chainEventSub.Unsubscribe() defer sub.Unsubscribe()
errCh := chainEventSub.Err()
var wg sync.WaitGroup var wg sync.WaitGroup
// Process metrics for chain events, then forward to workers // Process metrics for chain events, then forward to workers
chainEventFwd := make(chan core.ChainEvent, chainEventChanSize) chainEventFwd := make(chan core.ChainEvent, chainEventChanSize)
@ -325,14 +291,15 @@ func (sds *Service) WriteLoop(chainEventCh chan core.ChainEvent) {
select { select {
case chainEvent := <-chainEventCh: case chainEvent := <-chainEventCh:
lastHeight := defaultStatediffMetrics.lastEventHeight.Value() lastHeight := defaultStatediffMetrics.lastEventHeight.Value()
nextHeight := int64(chainEvent.Block.Number().Uint64()) block := chainEvent.Block
nextHeight := int64(block.Number().Uint64())
if nextHeight-lastHeight != 1 { if nextHeight-lastHeight != 1 {
log.Warn("Statediffing service received block out-of-order", "next height", nextHeight, "last height", lastHeight) log.Warn("Statediffing service received block out-of-order", "next height", nextHeight, "last height", lastHeight)
} }
defaultStatediffMetrics.lastEventHeight.Update(nextHeight) defaultStatediffMetrics.lastEventHeight.Update(nextHeight)
defaultStatediffMetrics.writeLoopChannelLen.Update(int64(len(chainEventCh))) defaultStatediffMetrics.writeLoopChannelLen.Update(int64(len(chainEventCh)))
chainEventFwd <- chainEvent chainEventFwd <- chainEvent
case err := <-errCh: case err := <-sub.Err():
log.Error("Error from chain event subscription", "error", err) log.Error("Error from chain event subscription", "error", err)
close(sds.QuitChan) close(sds.QuitChan)
log.Info("Quitting the statediffing writing loop") log.Info("Quitting the statediffing writing loop")
@ -375,36 +342,36 @@ func (sds *Service) writeLoopWorker(params workerParams) {
defer params.wg.Done() defer params.wg.Done()
for { for {
select { select {
//Notify chain event channel of events
case chainEvent := <-params.chainEventCh: case chainEvent := <-params.chainEventCh:
log.Debug("WriteLoop(): chain event received", "event", chainEvent) log.Debug("WriteLoop(): chain event received", "event", chainEvent)
currentBlock := chainEvent.Block
parentBlock := sds.BlockCache.getParentBlock(currentBlock, sds.BlockChain) block := chainEvent.Block
if parentBlock == nil { parent := sds.BlockCache.getParentBlock(block, sds.BlockChain)
log.Error("Parent block is nil, skipping this block", "block height", currentBlock.Number()) if parent == nil {
log.Error("Parent block is nil, skipping this block", "block height", block.Number())
continue continue
} }
// chainEvent streams block from block 1, but we also need to include data from the genesis block. // chainEvent streams block from block 1, but we also need to include data from the genesis block.
if parentBlock.Number().Uint64() == genesisBlockNumber { if parent.Number().Uint64() == genesisBlockNumber {
sds.writeGenesisStateDiff(parentBlock, params.id) sds.writeGenesisStateDiff(parent, params.id)
} }
log.Info("Writing state diff", "block height", currentBlock.Number().Uint64(), "worker", params.id) log.Info("Writing state diff", "block height", block.Number().Uint64(), "worker", params.id)
writeLoopParams.RLock() writeLoopParams.RLock()
err := sds.writeStateDiffWithRetry(currentBlock, parentBlock.Root(), writeLoopParams.Params) err := sds.writeStateDiffWithRetry(block, parent.Root(), writeLoopParams.Params)
writeLoopParams.RUnlock() writeLoopParams.RUnlock()
if err != nil { if err != nil {
log.Error("statediff.Service.WriteLoop: processing error", log.Error("statediff.Service.WriteLoop: processing error",
"block height", currentBlock.Number().Uint64(), "block height", block.Number().Uint64(),
"block hash", currentBlock.Hash().Hex(), "block hash", block.Hash().String(),
"error", err.Error(), "error", err.Error(),
"worker", params.id) "worker", params.id)
continue continue
} }
// TODO: how to handle with concurrent workers // TODO: how to handle with concurrent workers
defaultStatediffMetrics.lastStatediffHeight.Update(int64(currentBlock.Number().Uint64())) defaultStatediffMetrics.lastStatediffHeight.Update(int64(block.Number().Uint64()))
case <-sds.QuitChan: case <-sds.QuitChan:
log.Info("Quitting the statediff writing process", "worker", params.id) log.Info("Quitting the statediff writing process", "worker", params.id)
return return
@ -415,36 +382,35 @@ func (sds *Service) writeLoopWorker(params workerParams) {
// Loop is the main processing method // Loop is the main processing method
func (sds *Service) Loop(chainEventCh chan core.ChainEvent) { func (sds *Service) Loop(chainEventCh chan core.ChainEvent) {
log.Info("Starting statediff listening loop") log.Info("Starting statediff listening loop")
chainEventSub := sds.BlockChain.SubscribeChainEvent(chainEventCh)
defer chainEventSub.Unsubscribe() sub := sds.BlockChain.SubscribeChainEvent(chainEventCh)
errCh := chainEventSub.Err() defer sub.Unsubscribe()
for { for {
select { select {
//Notify chain event channel of events //Notify chain event channel of events
case chainEvent := <-chainEventCh: case event := <-chainEventCh:
defaultStatediffMetrics.serviceLoopChannelLen.Update(int64(len(chainEventCh))) defaultStatediffMetrics.serviceLoopChannelLen.Update(int64(len(chainEventCh)))
log.Debug("Loop(): chain event received", "event", chainEvent) log.Debug("Loop(): chain event received", "event", event)
// if we don't have any subscribers, do not process a statediff // if we don't have any subscribers, do not process a statediff
if atomic.LoadInt32(&sds.subscribers) == 0 { if atomic.LoadInt32(&sds.subscribers) == 0 {
log.Debug("Currently no subscribers to the statediffing service; processing is halted") log.Debug("Currently no subscribers to the statediffing service; processing is halted")
continue continue
} }
currentBlock := chainEvent.Block
parentBlock := sds.BlockCache.getParentBlock(currentBlock, sds.BlockChain)
if parentBlock == nil { block := event.Block
log.Error("Parent block is nil, skipping this block", "block height", currentBlock.Number()) parent := sds.BlockCache.getParentBlock(block, sds.BlockChain)
if parent == nil {
log.Error("Parent block is nil, skipping this block", "block height", block.Number())
continue continue
} }
// chainEvent streams block from block 1, but we also need to include data from the genesis block. // chainEvent streams block from block 1, but we also need to include data from the genesis block.
if parentBlock.Number().Uint64() == genesisBlockNumber { if parent.Number().Uint64() == genesisBlockNumber {
// For genesis block we need to return the entire state trie hence we diff it with an empty trie. // For genesis block we need to return the entire state trie hence we diff it with an empty trie.
sds.streamStateDiff(parentBlock, common.Hash{}) sds.streamStateDiff(parent, common.Hash{})
} }
sds.streamStateDiff(block, parent.Root())
sds.streamStateDiff(currentBlock, parentBlock.Root()) case err := <-sub.Err():
case err := <-errCh:
log.Error("Error from chain event subscription", "error", err) log.Error("Error from chain event subscription", "error", err)
close(sds.QuitChan) close(sds.QuitChan)
log.Info("Quitting the statediffing listening loop") log.Info("Quitting the statediffing listening loop")
@ -460,11 +426,11 @@ func (sds *Service) Loop(chainEventCh chan core.ChainEvent) {
// streamStateDiff method builds the state diff payload for each subscription according to their subscription type and sends them the result // streamStateDiff method builds the state diff payload for each subscription according to their subscription type and sends them the result
func (sds *Service) streamStateDiff(currentBlock *types.Block, parentRoot common.Hash) { func (sds *Service) streamStateDiff(currentBlock *types.Block, parentRoot common.Hash) {
sds.Lock() sds.subscriptionsMutex.Lock()
for ty, subs := range sds.Subscriptions { for ty, subs := range sds.Subscriptions {
params, ok := sds.SubscriptionTypes[ty] params, ok := sds.SubscriptionTypes[ty]
if !ok { if !ok {
log.Error("no parameter set associated with this subscription", "subscription type", ty.Hex()) log.Error("no parameter set associated with this subscription", "subscription type", ty.String())
sds.closeType(ty) sds.closeType(ty)
continue continue
} }
@ -483,7 +449,7 @@ func (sds *Service) streamStateDiff(currentBlock *types.Block, parentRoot common
} }
} }
} }
sds.Unlock() sds.subscriptionsMutex.Unlock()
} }
// StateDiffAt returns a state diff object payload at the specific blockheight // StateDiffAt returns a state diff object payload at the specific blockheight
@ -541,10 +507,10 @@ func (sds *Service) processStateDiff(currentBlock *types.Block, parentRoot commo
BlockNumber: currentBlock.Number(), BlockNumber: currentBlock.Number(),
}, params) }, params)
// allow dereferencing of parent, keep current locked as it should be the next parent // allow dereferencing of parent, keep current locked as it should be the next parent
sds.BlockChain.UnlockTrie(parentRoot) // sds.BlockChain.UnlockTrie(parentRoot)
if err != nil { // if err != nil {
return nil, err // return nil, err
} // }
stateDiffRlp, err := rlp.EncodeToBytes(&stateDiff) stateDiffRlp, err := rlp.EncodeToBytes(&stateDiff)
if err != nil { if err != nil {
return nil, err return nil, err
@ -579,7 +545,7 @@ func (sds *Service) newPayload(stateObject []byte, block *types.Block, params Pa
} }
// Subscribe is used by the API to subscribe to the service loop // Subscribe is used by the API to subscribe to the service loop
func (sds *Service) Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool, params Params) { func (sds *Service) Subscribe(sub chan<- Payload, quitChan chan<- bool, params Params) RpcID {
log.Info("Subscribing to the statediff service") log.Info("Subscribing to the statediff service")
if atomic.CompareAndSwapInt32(&sds.subscribers, 0, 1) { if atomic.CompareAndSwapInt32(&sds.subscribers, 0, 1) {
log.Info("State diffing subscription received; beginning statediff processing") log.Info("State diffing subscription received; beginning statediff processing")
@ -592,26 +558,28 @@ func (sds *Service) Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- boo
by, err := rlp.EncodeToBytes(&params) by, err := rlp.EncodeToBytes(&params)
if err != nil { if err != nil {
log.Error("State diffing params need to be rlp-serializable") log.Error("State diffing params need to be rlp-serializable")
return return 0
} }
subscriptionType := crypto.Keccak256Hash(by) subscriptionType := crypto.Keccak256Hash(by)
id := RpcID(atomic.AddUint64(&sds.lastRpcID, 1))
// Add subscriber // Add subscriber
sds.Lock() sds.subscriptionsMutex.Lock()
if sds.Subscriptions[subscriptionType] == nil { if sds.Subscriptions[subscriptionType] == nil {
sds.Subscriptions[subscriptionType] = make(map[rpc.ID]Subscription) sds.Subscriptions[subscriptionType] = make(map[RpcID]Subscription)
} }
sds.Subscriptions[subscriptionType][id] = Subscription{ sds.Subscriptions[subscriptionType][id] = Subscription{
PayloadChan: sub, PayloadChan: sub,
QuitChan: quitChan, QuitChan: quitChan,
} }
sds.SubscriptionTypes[subscriptionType] = params sds.SubscriptionTypes[subscriptionType] = params
sds.Unlock() sds.subscriptionsMutex.Unlock()
return id
} }
// Unsubscribe is used to unsubscribe from the service loop // Unsubscribe is used to unsubscribe from the service loop
func (sds *Service) Unsubscribe(id rpc.ID) error { func (sds *Service) Unsubscribe(id RpcID) error {
log.Info("Unsubscribing from the statediff service", "subscription id", id) log.Info("Unsubscribing from the statediff service", "subscription id", id)
sds.Lock() sds.subscriptionsMutex.Lock()
for ty := range sds.Subscriptions { for ty := range sds.Subscriptions {
delete(sds.Subscriptions[ty], id) delete(sds.Subscriptions[ty], id)
if len(sds.Subscriptions[ty]) == 0 { if len(sds.Subscriptions[ty]) == 0 {
@ -625,23 +593,16 @@ func (sds *Service) Unsubscribe(id rpc.ID) error {
log.Info("No more subscriptions; halting statediff processing") log.Info("No more subscriptions; halting statediff processing")
} }
} }
sds.Unlock() sds.subscriptionsMutex.Unlock()
return nil return nil
} }
// GetSyncStatus will check the status of geth syncing. // GetSyncStatus will check the status of geth syncing.
// It will return false if geth has finished syncing. // It will return false if geth has finished syncing.
// It will return a true Geth is still syncing. // It will return a true Geth is still syncing.
func (sds *Service) GetSyncStatus(pubEthAPI *ethapi.EthereumAPI) (bool, error) { func (sds *Service) GetSyncStatus() bool {
syncStatus, err := pubEthAPI.Syncing() progress := sds.BackendAPI.Downloader().Progress()
if err != nil { return progress.CurrentBlock() < progress.HighestBlock()
return true, err
}
if syncStatus != false {
return true, err
}
return false, err
} }
// WaitingForSync calls GetSyncStatus to check if we have caught up to head. // WaitingForSync calls GetSyncStatus to check if we have caught up to head.
@ -651,16 +612,12 @@ func (sds *Service) WaitingForSync() error {
log.Info("We are going to wait for geth to sync to head!") log.Info("We are going to wait for geth to sync to head!")
// Has the geth node synced to head? // Has the geth node synced to head?
Synced := false synced := false
pubEthAPI := ethapi.NewEthereumAPI(sds.BackendAPI) for !synced {
for !Synced { syncing := sds.GetSyncStatus()
syncStatus, err := sds.GetSyncStatus(pubEthAPI) if !syncing {
if err != nil {
return err
}
if !syncStatus {
log.Info("Geth has caught up to the head of the chain") log.Info("Geth has caught up to the head of the chain")
Synced = true synced = true
} else { } else {
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
} }
@ -701,7 +658,7 @@ func (sds *Service) Stop() error {
// close is used to close all listening subscriptions // close is used to close all listening subscriptions
func (sds *Service) close() { func (sds *Service) close() {
sds.Lock() sds.subscriptionsMutex.Lock()
for ty, subs := range sds.Subscriptions { for ty, subs := range sds.Subscriptions {
for id, sub := range subs { for id, sub := range subs {
select { select {
@ -715,7 +672,7 @@ func (sds *Service) close() {
delete(sds.Subscriptions, ty) delete(sds.Subscriptions, ty)
delete(sds.SubscriptionTypes, ty) delete(sds.SubscriptionTypes, ty)
} }
sds.Unlock() sds.subscriptionsMutex.Unlock()
} }
// closeType is used to close all subscriptions of given type // closeType is used to close all subscriptions of given type
@ -729,7 +686,7 @@ func (sds *Service) closeType(subType common.Hash) {
delete(sds.SubscriptionTypes, subType) delete(sds.SubscriptionTypes, subType)
} }
func sendNonBlockingQuit(id rpc.ID, sub Subscription) { func sendNonBlockingQuit(id RpcID, sub Subscription) {
select { select {
case sub.QuitChan <- true: case sub.QuitChan <- true:
log.Info("closing subscription", "id", id) log.Info("closing subscription", "id", id)
@ -747,13 +704,21 @@ func (sds *Service) WriteStateDiffAt(blockNumber uint64, params Params) JobID {
if id, has := sds.currentJobs[blockNumber]; has { if id, has := sds.currentJobs[blockNumber]; has {
return id return id
} }
id := JobID(atomic.AddUint64(&sds.lastJobID, 1)) sds.lastJobID++
id := JobID(sds.lastJobID)
sds.currentJobs[blockNumber] = id sds.currentJobs[blockNumber] = id
go func() { go func() {
err := sds.writeStateDiffAt(blockNumber, params) err := sds.writeStateDiffAt(blockNumber, params)
if err != nil {
log.Error("error from writeStateDiffAt", "error", err)
}
sds.currentJobsMutex.Lock() sds.currentJobsMutex.Lock()
delete(sds.currentJobs, blockNumber) delete(sds.currentJobs, blockNumber)
sds.currentJobsMutex.Unlock() sds.currentJobsMutex.Unlock()
sds.jobStatusSubsMutex.RLock()
defer sds.jobStatusSubsMutex.RUnlock()
for _, sub := range sds.jobStatusSubs { for _, sub := range sds.jobStatusSubs {
sub.statusChan <- JobStatus{id, err} sub.statusChan <- JobStatus{id, err}
} }
@ -852,7 +817,7 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p
} }
// allow dereferencing of parent, keep current locked as it should be the next parent // allow dereferencing of parent, keep current locked as it should be the next parent
sds.BlockChain.UnlockTrie(parentRoot) // sds.BlockChain.UnlockTrie(parentRoot)
return nil return nil
} }
@ -874,23 +839,25 @@ func (sds *Service) writeStateDiffWithRetry(block *types.Block, parentRoot commo
} }
// SubscribeWriteStatus is used by the API to subscribe to the job status updates // SubscribeWriteStatus is used by the API to subscribe to the job status updates
func (sds *Service) SubscribeWriteStatus(id rpc.ID, sub chan<- JobStatus, quitChan chan<- bool) { func (sds *Service) SubscribeWriteStatus(sub chan<- JobStatus, quitChan chan<- bool) RpcID {
id := RpcID(atomic.AddUint64(&sds.lastRpcID, 1))
log.Info("Subscribing to job status updates", "subscription id", id) log.Info("Subscribing to job status updates", "subscription id", id)
sds.Lock() sds.jobStatusSubsMutex.Lock()
sds.jobStatusSubs[id] = statusSubscription{ sds.jobStatusSubs[id] = jobStatusSubscription{
statusChan: sub, statusChan: sub,
quitChan: quitChan, quitChan: quitChan,
} }
sds.Unlock() sds.jobStatusSubsMutex.Unlock()
return id
} }
// UnsubscribeWriteStatus is used to unsubscribe from job status updates // UnsubscribeWriteStatus is used to unsubscribe from job status updates
func (sds *Service) UnsubscribeWriteStatus(id rpc.ID) error { func (sds *Service) UnsubscribeWriteStatus(id RpcID) error {
log.Info("Unsubscribing from job status updates", "subscription id", id) log.Info("Unsubscribing from job status updates", "subscription id", id)
sds.Lock() sds.jobStatusSubsMutex.Lock()
close(sds.jobStatusSubs[id].quitChan) close(sds.jobStatusSubs[id].quitChan)
delete(sds.jobStatusSubs, id) delete(sds.jobStatusSubs, id)
sds.Unlock() sds.jobStatusSubsMutex.Unlock()
return nil return nil
} }
@ -904,8 +871,7 @@ func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- typ
close(quitChan) close(quitChan)
return return
} }
it := currentTrie.NodeIterator([]byte{}) leafIt := trie.NewIterator(currentTrie.NodeIterator(nil))
leafIt := trie.NewIterator(it)
go func() { go func() {
defer close(quitChan) defer close(quitChan)
for leafIt.Next() { for leafIt.Next() {
@ -920,7 +886,7 @@ func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- typ
return return
} }
codeHash := common.BytesToHash(account.CodeHash) codeHash := common.BytesToHash(account.CodeHash)
code, err := sds.BlockChain.StateCache().ContractCode(common.Hash{}, codeHash) code, err := sds.BlockChain.StateCache().ContractCode(codeHash)
if err != nil { if err != nil {
log.Error("error collecting contract code", "err", err) log.Error("error collecting contract code", "err", err)
return return
@ -947,7 +913,7 @@ func (sds *Service) WatchAddress(operation types2.OperationType, args []types2.W
case types2.Add: case types2.Add:
// filter out args having an already watched address with a warning // filter out args having an already watched address with a warning
filteredArgs, ok := funk.Filter(args, func(arg types2.WatchAddressArg) bool { filteredArgs, ok := funk.Filter(args, func(arg types2.WatchAddressArg) bool {
if funk.Contains(writeLoopParams.WatchedAddresses, common.HexToAddress(arg.Address)) { if funk.Contains(writeLoopParams.WatchedAddresses, plugeth.HexToAddress(arg.Address)) {
log.Warn("Address already being watched", "address", arg.Address) log.Warn("Address already being watched", "address", arg.Address)
return false return false
} }
@ -1052,7 +1018,7 @@ func loadWatchedAddresses(indexer interfaces.StateDiffIndexer) error {
return nil return nil
} }
// MapWatchAddressArgsToAddresses maps []WatchAddressArg to corresponding []common.Address // MapWatchAddressArgsToAddresses maps []WatchAddressArg to corresponding []core.Address
func MapWatchAddressArgsToAddresses(args []types2.WatchAddressArg) ([]common.Address, error) { func MapWatchAddressArgsToAddresses(args []types2.WatchAddressArg) ([]common.Address, error) {
addresses, ok := funk.Map(args, func(arg types2.WatchAddressArg) common.Address { addresses, ok := funk.Map(args, func(arg types2.WatchAddressArg) common.Address {
return common.HexToAddress(arg.Address) return common.HexToAddress(arg.Address)

View File

@ -29,16 +29,18 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/internal/ethapi" // "github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/rpc"
statediff "github.com/ethereum/go-ethereum/statediff"
"github.com/ethereum/go-ethereum/statediff/test_helpers/mocks"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
// plugeth "github.com/openrelayxyz/plugeth-utils/core"
"github.com/openrelayxyz/plugeth-utils/restricted/rlp"
"github.com/cerc-io/plugeth-statediff"
"github.com/cerc-io/plugeth-statediff/test_helpers/mocks"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
) )
func TestServiceLoop(t *testing.T) { func TestServiceLoop(t *testing.T) {
@ -99,17 +101,16 @@ func testErrorInChainEventLoop(t *testing.T) {
blockChain := mocks.BlockChain{} blockChain := mocks.BlockChain{}
serviceQuit := make(chan bool) serviceQuit := make(chan bool)
service := statediff.Service{ service := statediff.Service{
Mutex: sync.Mutex{},
Builder: &builder, Builder: &builder,
BlockChain: &blockChain, BlockChain: &blockChain,
QuitChan: serviceQuit, QuitChan: serviceQuit,
Subscriptions: make(map[common.Hash]map[rpc.ID]statediff.Subscription), Subscriptions: make(map[common.Hash]map[statediff.RpcID]statediff.Subscription),
SubscriptionTypes: make(map[common.Hash]statediff.Params), SubscriptionTypes: make(map[common.Hash]statediff.Params),
BlockCache: statediff.NewBlockCache(1), BlockCache: statediff.NewBlockCache(1),
} }
payloadChan := make(chan statediff.Payload, 2) payloadChan := make(chan statediff.Payload, 2)
quitChan := make(chan bool) quitChan := make(chan bool)
service.Subscribe(rpc.NewID(), payloadChan, quitChan, defaultParams) service.Subscribe(payloadChan, quitChan, defaultParams)
testRoot2 = common.HexToHash("0xTestRoot2") testRoot2 = common.HexToHash("0xTestRoot2")
blockMapping := make(map[common.Hash]*types.Block) blockMapping := make(map[common.Hash]*types.Block)
blockMapping[parentBlock1.Hash()] = parentBlock1 blockMapping[parentBlock1.Hash()] = parentBlock1
@ -187,13 +188,13 @@ func testErrorInBlockLoop(t *testing.T) {
Builder: &builder, Builder: &builder,
BlockChain: &blockChain, BlockChain: &blockChain,
QuitChan: make(chan bool), QuitChan: make(chan bool),
Subscriptions: make(map[common.Hash]map[rpc.ID]statediff.Subscription), Subscriptions: make(map[common.Hash]map[statediff.RpcID]statediff.Subscription),
SubscriptionTypes: make(map[common.Hash]statediff.Params), SubscriptionTypes: make(map[common.Hash]statediff.Params),
BlockCache: statediff.NewBlockCache(1), BlockCache: statediff.NewBlockCache(1),
} }
payloadChan := make(chan statediff.Payload) payloadChan := make(chan statediff.Payload)
quitChan := make(chan bool) quitChan := make(chan bool)
service.Subscribe(rpc.NewID(), payloadChan, quitChan, defaultParams) service.Subscribe(payloadChan, quitChan, defaultParams)
blockMapping := make(map[common.Hash]*types.Block) blockMapping := make(map[common.Hash]*types.Block)
blockMapping[parentBlock1.Hash()] = parentBlock1 blockMapping[parentBlock1.Hash()] = parentBlock1
blockChain.SetBlocksForHashes(blockMapping) blockChain.SetBlocksForHashes(blockMapping)
@ -226,7 +227,7 @@ func testErrorInBlockLoop(t *testing.T) {
} }
func TestGetStateDiffAt(t *testing.T) { func TestGetStateDiffAt(t *testing.T) {
mockStateDiff := types2.StateObject{ mockStateDiff := sdtypes.StateObject{
BlockNumber: testBlock1.Number(), BlockNumber: testBlock1.Number(),
BlockHash: testBlock1.Hash(), BlockHash: testBlock1.Hash(),
} }
@ -260,11 +261,10 @@ func TestGetStateDiffAt(t *testing.T) {
blockChain.SetBlockForNumber(testBlock1, testBlock1.NumberU64()) blockChain.SetBlockForNumber(testBlock1, testBlock1.NumberU64())
blockChain.SetReceiptsForHash(testBlock1.Hash(), testReceipts1) blockChain.SetReceiptsForHash(testBlock1.Hash(), testReceipts1)
service := statediff.Service{ service := statediff.Service{
Mutex: sync.Mutex{},
Builder: &builder, Builder: &builder,
BlockChain: &blockChain, BlockChain: &blockChain,
QuitChan: make(chan bool), QuitChan: make(chan bool),
Subscriptions: make(map[common.Hash]map[rpc.ID]statediff.Subscription), Subscriptions: make(map[common.Hash]map[statediff.RpcID]statediff.Subscription),
SubscriptionTypes: make(map[common.Hash]statediff.Params), SubscriptionTypes: make(map[common.Hash]statediff.Params),
BlockCache: statediff.NewBlockCache(1), BlockCache: statediff.NewBlockCache(1),
} }
@ -300,40 +300,29 @@ func TestGetStateDiffAt(t *testing.T) {
} }
type writeSub struct { type writeSub struct {
sub *rpc.ClientSubscription ch <-chan statediff.JobStatus
statusChan <-chan statediff.JobStatus unsubscribe func()
} }
func makeClient(svc *statediff.Service) *rpc.Client { func subscribeWritesService(t *testing.T, api *statediff.PublicAPI) writeSub {
server := rpc.NewServer() ctx, cancel := context.WithCancel(context.Background())
api := statediff.NewPublicStateDiffAPI(svc) sub, err := api.StreamWrites(ctx)
err := server.RegisterName("statediff", api) require.NoError(t, err)
if err != nil { return writeSub{sub, cancel}
panic(err)
}
return rpc.DialInProc(server)
} }
// awaitStatus awaits status update for writeStateDiffAt job func (ws writeSub) awaitStatus(job statediff.JobID, timeout time.Duration) (bool, error) {
func subscribeWrites(client *rpc.Client) (writeSub, error) { deadline := time.After(timeout)
statusChan := make(chan statediff.JobStatus)
sub, err := client.Subscribe(context.Background(), "statediff", statusChan, "streamWrites")
return writeSub{sub, statusChan}, err
}
func (ws writeSub) await(job statediff.JobID, timeout time.Duration) (bool, error) {
for { for {
select { select {
case err := <-ws.sub.Err(): case status := <-ws.ch:
return false, err
case status := <-ws.statusChan:
if status.Err != nil { if status.Err != nil {
return false, status.Err return false, status.Err
} }
if status.ID == job { if status.ID == job {
return true, nil return true, nil
} }
case <-time.After(timeout): case <-deadline:
return false, errors.New("timeout") return false, errors.New("timeout")
} }
} }
@ -351,19 +340,17 @@ func TestWriteStateDiffAt(t *testing.T) {
service := statediff.NewService(&blockChain, statediff.Config{}, &mocks.Backend{}, &indexer) service := statediff.NewService(&blockChain, statediff.Config{}, &mocks.Backend{}, &indexer)
service.Builder = &builder service.Builder = &builder
api := statediff.NewPublicAPI(service)
// delay to avoid subscription request being sent after statediff is written, // delay to avoid subscription request being sent after statediff is written
// and timeout to prevent hanging just in case it still happens writeDelay := 200 * time.Millisecond
writeDelay := 100 * time.Millisecond // timeout to prevent hanging just in case it still happens
jobTimeout := 200 * time.Millisecond jobTimeout := 2 * time.Second
client := makeClient(service)
defer client.Close()
ws, err := subscribeWrites(client) ws := subscribeWritesService(t, api)
require.NoError(t, err)
time.Sleep(writeDelay) time.Sleep(writeDelay)
job := service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams) job := service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams)
ok, err := ws.await(job, jobTimeout) ok, err := ws.awaitStatus(job, jobTimeout)
require.NoError(t, err) require.NoError(t, err)
require.True(t, ok) require.True(t, ok)
@ -375,21 +362,17 @@ func TestWriteStateDiffAt(t *testing.T) {
// unsubscribe and verify we get nothing // unsubscribe and verify we get nothing
// TODO - StreamWrites receives EOF error after unsubscribing. Doesn't seem to impact // TODO - StreamWrites receives EOF error after unsubscribing. Doesn't seem to impact
// anything but would be good to know why. // anything but would be good to know why.
ws.sub.Unsubscribe() ws.unsubscribe()
time.Sleep(writeDelay)
job = service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams) job = service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams)
ok, _ = ws.await(job, jobTimeout) ok, _ = ws.awaitStatus(job, jobTimeout)
require.False(t, ok) require.False(t, ok)
client.Close()
client = makeClient(service)
// re-subscribe and test again // re-subscribe and test again
ws, err = subscribeWrites(client) ws = subscribeWritesService(t, api)
require.NoError(t, err)
time.Sleep(writeDelay) time.Sleep(writeDelay)
job = service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams) job = service.WriteStateDiffAt(testBlock1.NumberU64(), defaultParams)
ok, err = ws.await(job, jobTimeout) ok, err = ws.awaitStatus(job, jobTimeout)
require.NoError(t, err) require.NoError(t, err)
require.True(t, ok) require.True(t, ok)
} }
@ -400,12 +383,12 @@ func TestWaitForSync(t *testing.T) {
} }
// This function will create a backend and service object which includes a generic Backend // This function will create a backend and service object which includes a generic Backend
func createServiceWithMockBackend(curBlock uint64, highestBlock uint64) (*mocks.Backend, *statediff.Service) { func createServiceWithMockBackend(t *testing.T, curBlock uint64, highestBlock uint64) (*mocks.Backend, *statediff.Service) {
builder := mocks.Builder{} builder := mocks.Builder{}
blockChain := mocks.BlockChain{} blockChain := mocks.BlockChain{}
backend := mocks.Backend{ backend := mocks.NewBackend(t, ethereum.SyncProgress{
StartingBlock: 1, StartingBlock: 1,
CurrBlock: curBlock, CurrentBlock: curBlock,
HighestBlock: highestBlock, HighestBlock: highestBlock,
SyncedAccounts: 5, SyncedAccounts: 5,
SyncedAccountBytes: 5, SyncedAccountBytes: 5,
@ -419,27 +402,26 @@ func createServiceWithMockBackend(curBlock uint64, highestBlock uint64) (*mocks.
HealedBytecodeBytes: 5, HealedBytecodeBytes: 5,
HealingTrienodes: 5, HealingTrienodes: 5,
HealingBytecode: 5, HealingBytecode: 5,
} })
service := &statediff.Service{ service := &statediff.Service{
Mutex: sync.Mutex{},
Builder: &builder, Builder: &builder,
BlockChain: &blockChain, BlockChain: &blockChain,
QuitChan: make(chan bool), QuitChan: make(chan bool),
Subscriptions: make(map[common.Hash]map[rpc.ID]statediff.Subscription), Subscriptions: make(map[common.Hash]map[statediff.RpcID]statediff.Subscription),
SubscriptionTypes: make(map[common.Hash]statediff.Params), SubscriptionTypes: make(map[common.Hash]statediff.Params),
BlockCache: statediff.NewBlockCache(1), BlockCache: statediff.NewBlockCache(1),
BackendAPI: &backend, BackendAPI: backend,
WaitForSync: true, WaitForSync: true,
} }
return &backend, service return backend, service
} }
// This function will test to make sure that the state diff waits // This function will test to make sure that the state diff waits
// until the blockchain has caught up to head! // until the blockchain has caught up to head!
func testWaitForSync(t *testing.T) { func testWaitForSync(t *testing.T) {
t.Log("Starting Sync") t.Log("Starting Sync")
_, service := createServiceWithMockBackend(10, 10) _, service := createServiceWithMockBackend(t, 10, 10)
err := service.WaitingForSync() err := service.WaitingForSync()
if err != nil { if err != nil {
t.Fatal("Sync Failed") t.Fatal("Sync Failed")
@ -456,7 +438,7 @@ func testGetSyncStatus(t *testing.T) {
var highestBlock uint64 = 5 var highestBlock uint64 = 5
// Create a backend and a service // Create a backend and a service
// the backend is lagging behind the sync. // the backend is lagging behind the sync.
backend, service := createServiceWithMockBackend(0, highestBlock) backend, service := createServiceWithMockBackend(t, 0, highestBlock)
checkSyncComplete := make(chan int, 1) checkSyncComplete := make(chan int, 1)
@ -491,13 +473,8 @@ func testGetSyncStatus(t *testing.T) {
// Update the backend current block value // Update the backend current block value
t.Log("Updating Current Block to: ", table.currentBlock) t.Log("Updating Current Block to: ", table.currentBlock)
backend.CurrBlock = table.currentBlock backend.SetCurrentBlock(table.currentBlock)
pubEthAPI := ethapi.NewEthereumAPI(service.BackendAPI) syncStatus := service.GetSyncStatus()
syncStatus, err := service.GetSyncStatus(pubEthAPI)
if err != nil {
t.Fatal("Sync Failed")
}
time.Sleep(2 * time.Second) time.Sleep(2 * time.Second)

83
state.go Normal file
View File

@ -0,0 +1,83 @@
package statediff
import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state"
// "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/trie"
plugeth "github.com/openrelayxyz/plugeth-utils/core"
// plugeth_trie "github.com/openrelayxyz/plugeth-utils/restricted/trie"
"github.com/cerc-io/plugeth-statediff/adapt"
)
// Exposes a minimal interface for state access for diff building
type StateView interface {
OpenTrie(root common.Hash) (StateTrie, error)
ContractCode(codeHash common.Hash) ([]byte, error)
}
// StateTrie is an interface exposing only the necessary methods from state.Trie
type StateTrie interface {
GetKey([]byte) []byte
// GetAccount(common.Address) (*types.StateAccount, error)
// Hash() common.Hash
NodeIterator([]byte) trie.NodeIterator
// Prove(key []byte, fromLevel uint, proofDb KeyValueWriter) error
}
// exposes a StateView from a combination of plugeth's core Backend and cached contract code
type plugethStateView struct {
b plugeth.Backend
code map[common.Hash][]byte
}
var _ StateView = &plugethStateView{}
func (p *plugethStateView) OpenTrie(root common.Hash) (StateTrie, error) {
t, err := p.b.GetTrie(plugeth.Hash(root))
if err != nil {
return nil, err
}
return adaptTrie{t}, nil
}
func (p *plugethStateView) ContractCode(hash common.Hash) ([]byte, error) {
return p.code[hash], nil
}
// adapts a state.Database to StateView - used in tests
type stateDatabaseView struct {
db state.Database
}
var _ StateView = stateDatabaseView{}
func StateDatabaseView(db state.Database) StateView {
return stateDatabaseView{db}
}
func (a stateDatabaseView) OpenTrie(root common.Hash) (StateTrie, error) {
// return adaptTrie{a.db.OpenTrie(common.Hash(root))}
return a.db.OpenTrie(common.Hash(root))
}
func (a stateDatabaseView) ContractCode(hash common.Hash) ([]byte, error) {
return a.db.ContractCode(common.Hash{}, hash)
}
// adapts geth Trie to plugeth
type adaptTrie struct {
plugeth.Trie
}
var _ StateTrie = adaptTrie{}
// func (a adaptTrie) GetAccount(addr *types.StateAccount) (*plugeth.StateAccount, error) {
// return adapt.StateAccount(a.Trie.GetAccount(addr))
// }
func (a adaptTrie) NodeIterator(start []byte) trie.NodeIterator {
return adapt.NodeIterator(a.Trie.NodeIterator(start))
}

27
test/compose.yml Normal file
View File

@ -0,0 +1,27 @@
version: "3.2"
services:
migrations:
restart: on-failure
depends_on:
- ipld-eth-db
image: git.vdb.to/cerc-io/ipld-eth-db/ipld-eth-db:v5.0.2-alpha
environment:
DATABASE_USER: "vdbm"
DATABASE_NAME: "cerc_testing"
DATABASE_PASSWORD: "password"
DATABASE_HOSTNAME: "ipld-eth-db"
DATABASE_PORT: 5432
ipld-eth-db:
image: timescale/timescaledb:latest-pg14
restart: always
command: ["postgres", "-c", "log_statement=all"]
environment:
POSTGRES_USER: "vdbm"
POSTGRES_DB: "cerc_testing"
POSTGRES_PASSWORD: "password"
ports:
- "127.0.0.1:8077:5432"
volumes:
- ../indexer/database/file:/file_indexer

71
test_helpers/builder.go Normal file
View File

@ -0,0 +1,71 @@
package test_helpers
import (
"bytes"
"encoding/json"
"sort"
"testing"
"github.com/cerc-io/plugeth-statediff"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/rlp"
"github.com/stretchr/testify/require"
)
type TestCase struct {
Name string
Args statediff.Args
Expected *sdtypes.StateObject
}
type CheckedRoots = map[*types.Block][]byte
func RunBuilderTests(
t *testing.T,
builder statediff.Builder,
tests []TestCase,
params statediff.Params,
roots CheckedRoots,
) {
for _, test := range tests {
diff, err := builder.BuildStateDiffObject(test.Args, params)
if err != nil {
t.Error(err)
}
receivedStateDiffRlp, err := rlp.EncodeToBytes(&diff)
if err != nil {
t.Error(err)
}
expectedStateDiffRlp, err := rlp.EncodeToBytes(test.Expected)
if err != nil {
t.Error(err)
}
sort.Slice(receivedStateDiffRlp, func(i, j int) bool {
return receivedStateDiffRlp[i] < receivedStateDiffRlp[j]
})
sort.Slice(expectedStateDiffRlp, func(i, j int) bool {
return expectedStateDiffRlp[i] < expectedStateDiffRlp[j]
})
if !bytes.Equal(receivedStateDiffRlp, expectedStateDiffRlp) {
actualb, err := json.Marshal(diff)
require.NoError(t, err)
expectedb, err := json.Marshal(test.Expected)
require.NoError(t, err)
var expected, actual interface{}
err = json.Unmarshal(expectedb, &expected)
require.NoError(t, err)
err = json.Unmarshal(actualb, &actual)
require.NoError(t, err)
require.Equal(t, expected, actual, test.Name)
}
}
// Let's also confirm that our root state nodes form the state root hash in the headers
for block, node := range roots {
require.Equal(t, block.Root(), crypto.Keccak256Hash(node),
"expected root does not match actual root", block.Number())
}
}

View File

@ -27,6 +27,8 @@ import (
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/cerc-io/plugeth-statediff/utils"
) )
func GenesisBlockForTesting(db ethdb.Database, addr common.Address, balance, baseFee *big.Int, initialGasLimit uint64) *types.Block { func GenesisBlockForTesting(db ethdb.Database, addr common.Address, balance, baseFee *big.Int, initialGasLimit uint64) *types.Block {
@ -65,7 +67,7 @@ func TestSelfDestructChainGen(i int, block *core.BlockGen) {
// Block 2 is mined by TestBankAddress // Block 2 is mined by TestBankAddress
// TestBankAddress self-destructs the contract // TestBankAddress self-destructs the contract
block.SetCoinbase(TestBankAddress) block.SetCoinbase(TestBankAddress)
data := common.Hex2Bytes("43D726D6") data := utils.Hex2Bytes("43D726D6")
tx, _ := types.SignTx(types.NewTransaction(1, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.GWei), data), signer, TestBankKey) tx, _ := types.SignTx(types.NewTransaction(1, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.GWei), data), signer, TestBankKey)
block.AddTx(tx) block.AddTx(tx)
} }
@ -97,7 +99,7 @@ func TestChainGen(i int, block *core.BlockGen) {
block.SetCoinbase(Account2Addr) block.SetCoinbase(Account2Addr)
//put function: c16431b9 //put function: c16431b9
//close function: 43d726d6 //close function: 43d726d6
data := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003") data := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003")
tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(TestBankAddress), ContractAddr, big.NewInt(0), params.TxGasContractCreation, big.NewInt(params.GWei), data), signer, TestBankKey) tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(TestBankAddress), ContractAddr, big.NewInt(0), params.TxGasContractCreation, big.NewInt(params.GWei), data), signer, TestBankKey)
block.AddTx(tx) block.AddTx(tx)
case 3: case 3:
@ -105,9 +107,9 @@ func TestChainGen(i int, block *core.BlockGen) {
// Two set the two original slot positions to 0 and one sets another position to a new value // Two set the two original slot positions to 0 and one sets another position to a new value
// Block 4 is mined by Account2Addr // Block 4 is mined by Account2Addr
block.SetCoinbase(Account2Addr) block.SetCoinbase(Account2Addr)
data1 := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000") data1 := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")
data2 := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000") data2 := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000")
data3 := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000009") data3 := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000009")
nonce := block.TxNonce(TestBankAddress) nonce := block.TxNonce(TestBankAddress)
tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey) tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey)
@ -123,8 +125,8 @@ func TestChainGen(i int, block *core.BlockGen) {
// It sets the one storage value to zero and the other to new value. // It sets the one storage value to zero and the other to new value.
// Block 5 is mined by Account1Addr // Block 5 is mined by Account1Addr
block.SetCoinbase(Account1Addr) block.SetCoinbase(Account1Addr)
data1 := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000") data1 := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000")
data2 := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003") data2 := utils.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003")
nonce := block.TxNonce(TestBankAddress) nonce := block.TxNonce(TestBankAddress)
tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey) tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey)
nonce++ nonce++
@ -135,7 +137,7 @@ func TestChainGen(i int, block *core.BlockGen) {
// Block 6 has a tx from Account1Key which self-destructs the contract, it transfers no value // Block 6 has a tx from Account1Key which self-destructs the contract, it transfers no value
// Block 6 is mined by Account2Addr // Block 6 is mined by Account2Addr
block.SetCoinbase(Account2Addr) block.SetCoinbase(Account2Addr)
data := common.Hex2Bytes("43D726D6") data := utils.Hex2Bytes("43D726D6")
tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(Account1Addr), ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data), signer, Account1Key) tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(Account1Addr), ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data), signer, Account1Key)
block.AddTx(tx) block.AddTx(tx)
} }
@ -158,8 +160,8 @@ func TestChainGenWithInternalLeafNode(i int, block *core.BlockGen) {
// Block 3 has two transactions which set slots 223 and 648 with small values // Block 3 has two transactions which set slots 223 and 648 with small values
// The goal here is to induce a branch node with an internalized leaf node // The goal here is to induce a branch node with an internalized leaf node
block.SetCoinbase(TestBankAddress) block.SetCoinbase(TestBankAddress)
data1 := common.Hex2Bytes("C16431B90000000000000000000000000000000000000000000000000000000000009dab0000000000000000000000000000000000000000000000000000000000000001") data1 := utils.Hex2Bytes("C16431B90000000000000000000000000000000000000000000000000000000000009dab0000000000000000000000000000000000000000000000000000000000000001")
data2 := common.Hex2Bytes("C16431B90000000000000000000000000000000000000000000000000000000000019c5d0000000000000000000000000000000000000000000000000000000000000002") data2 := utils.Hex2Bytes("C16431B90000000000000000000000000000000000000000000000000000000000019c5d0000000000000000000000000000000000000000000000000000000000000002")
nonce := block.TxNonce(TestBankAddress) nonce := block.TxNonce(TestBankAddress)
tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey) tx1, _ := types.SignTx(types.NewTransaction(nonce, ContractAddr, big.NewInt(0), 100000, big.NewInt(params.InitialBaseFee), data1), signer, TestBankKey)

View File

@ -1,265 +1,74 @@
// Copyright 2019 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package mocks package mocks
import ( import (
"context" "testing"
"math/big"
"time"
"github.com/ethereum/go-ethereum/internal/ethapi"
"github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/accounts" "github.com/golang/mock/gomock"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus" plugeth "github.com/openrelayxyz/plugeth-utils/core"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/bloombits"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rpc"
) )
var _ ethapi.Backend = &Backend{}
// Builder is a mock state diff builder
type Backend struct { type Backend struct {
StartingBlock uint64 *MockBackend
CurrBlock uint64 downloader Downloader
HighestBlock uint64
SyncedAccounts uint64
SyncedAccountBytes uint64
SyncedBytecodes uint64
SyncedBytecodeBytes uint64
SyncedStorage uint64
SyncedStorageBytes uint64
HealedTrienodes uint64
HealedTrienodeBytes uint64
HealedBytecodes uint64
HealedBytecodeBytes uint64
HealingTrienodes uint64
HealingBytecode uint64
} }
// General Ethereum API type Downloader struct {
func (backend *Backend) SyncProgress() ethereum.SyncProgress { ethereum.SyncProgress
l := ethereum.SyncProgress{ }
StartingBlock: backend.StartingBlock,
CurrentBlock: backend.CurrBlock, var _ plugeth.Backend = &Backend{}
HighestBlock: backend.HighestBlock, var _ plugeth.Downloader = &Downloader{}
SyncedAccounts: backend.SyncedAccounts,
SyncedAccountBytes: backend.SyncedAccountBytes, func NewBackend(t *testing.T, progress ethereum.SyncProgress) *Backend {
SyncedBytecodes: backend.SyncedBytecodes, ctl := gomock.NewController(t)
SyncedBytecodeBytes: backend.SyncedBytecodeBytes, dler := Downloader{progress}
SyncedStorage: backend.SyncedStorage, ret := &Backend{
SyncedStorageBytes: backend.SyncedStorageBytes, MockBackend: NewMockBackend(ctl),
HealedTrienodes: backend.HealedTrienodes, downloader: dler,
HealedTrienodeBytes: backend.HealedTrienodeBytes,
HealedBytecodes: backend.HealedBytecodes,
HealedBytecodeBytes: backend.HealedBytecodeBytes,
HealingTrienodes: backend.HealingTrienodes,
HealingBytecode: backend.HealingBytecode,
} }
return l ret.EXPECT().Downloader().Return(&dler).AnyTimes()
return ret
} }
func (backend *Backend) SuggestGasTipCap(ctx context.Context) (*big.Int, error) { func (b *Backend) SetCurrentBlock(block uint64) {
panic("not implemented") // TODO: Implement b.downloader.SyncProgress.StartingBlock = block
} }
func (backend *Backend) FeeHistory(ctx context.Context, blockCount uint64, lastBlock rpc.BlockNumber, rewardPercentiles []float64) (*big.Int, [][]*big.Int, []*big.Int, []float64, error) { func (d Downloader) Progress() plugeth.Progress {
panic("implement me") return d
} }
func (backend *Backend) ChainDb() ethdb.Database { func (d Downloader) StartingBlock() uint64 { return d.SyncProgress.StartingBlock }
panic("not implemented") // TODO: Implement func (d Downloader) CurrentBlock() uint64 { return d.SyncProgress.CurrentBlock }
} func (d Downloader) HighestBlock() uint64 { return d.SyncProgress.HighestBlock }
func (d Downloader) PulledStates() uint64 { return d.SyncProgress.PulledStates }
func (d Downloader) KnownStates() uint64 { return d.SyncProgress.KnownStates }
func (d Downloader) SyncedAccounts() uint64 { return d.SyncProgress.SyncedAccounts }
func (d Downloader) SyncedAccountBytes() uint64 { return d.SyncProgress.SyncedAccountBytes }
func (d Downloader) SyncedBytecodes() uint64 { return d.SyncProgress.SyncedBytecodes }
func (d Downloader) SyncedBytecodeBytes() uint64 { return d.SyncProgress.SyncedBytecodeBytes }
func (d Downloader) SyncedStorage() uint64 { return d.SyncProgress.SyncedStorage }
func (d Downloader) SyncedStorageBytes() uint64 { return d.SyncProgress.SyncedStorageBytes }
func (d Downloader) HealedTrienodes() uint64 { return d.SyncProgress.HealedTrienodes }
func (d Downloader) HealedTrienodeBytes() uint64 { return d.SyncProgress.HealedTrienodeBytes }
func (d Downloader) HealedBytecodes() uint64 { return d.SyncProgress.HealedBytecodes }
func (d Downloader) HealedBytecodeBytes() uint64 { return d.SyncProgress.HealedBytecodeBytes }
func (d Downloader) HealingTrienodes() uint64 { return d.SyncProgress.HealingTrienodes }
func (d Downloader) HealingBytecode() uint64 { return d.SyncProgress.HealingBytecode }
func (backend *Backend) AccountManager() *accounts.Manager { func TestBackend(t *testing.T) {
panic("not implemented") // TODO: Implement b := NewBackend(t, ethereum.SyncProgress{StartingBlock: 42})
}
func (backend *Backend) ExtRPCEnabled() bool { block := b.Downloader().Progress().StartingBlock()
panic("not implemented") // TODO: Implement if 42 != block {
} t.Fatalf("wrong StartingBlock; expected %d, got %d", 42, block)
}
func (backend *Backend) RPCGasCap() uint64 { b.SetCurrentBlock(420)
panic("not implemented") // TODO: Implement block = b.Downloader().Progress().StartingBlock()
} if 420 != block {
t.Fatalf("wrong StartingBlock; expected %d, got %d", 420, block)
func (backend *Backend) RPCEVMTimeout() time.Duration { }
panic("not implemented") // TODO: Implement
}
func (backend *Backend) RPCTxFeeCap() float64 {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) UnprotectedAllowed() bool {
panic("not implemented") // TODO: Implement
}
// Blockchain API
func (backend *Backend) SetHead(number uint64) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) HeaderByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Header, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) HeaderByHash(ctx context.Context, hash common.Hash) (*types.Header, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) HeaderByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*types.Header, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) CurrentHeader() *types.Header {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) CurrentBlock() *types.Header {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) BlockByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Block, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) BlockByHash(ctx context.Context, hash common.Hash) (*types.Block, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) BlockByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*types.Block, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) StateAndHeaderByNumber(ctx context.Context, number rpc.BlockNumber) (*state.StateDB, *types.Header, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) StateAndHeaderByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*state.StateDB, *types.Header, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetReceipts(ctx context.Context, hash common.Hash) (types.Receipts, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetBody(ctx context.Context, hash common.Hash, number rpc.BlockNumber) (*types.Body, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetTd(ctx context.Context, hash common.Hash) *big.Int {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetEVM(ctx context.Context, msg *core.Message, state *state.StateDB, header *types.Header, vmConfig *vm.Config) (*vm.EVM, func() error, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeChainHeadEvent(ch chan<- core.ChainHeadEvent) event.Subscription {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeChainSideEvent(ch chan<- core.ChainSideEvent) event.Subscription {
panic("not implemented") // TODO: Implement
}
// Transaction pool API
func (backend *Backend) SendTx(ctx context.Context, signedTx *types.Transaction) error {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetTransaction(ctx context.Context, txHash common.Hash) (*types.Transaction, common.Hash, uint64, uint64, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetPoolTransactions() (types.Transactions, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetPoolTransaction(txHash common.Hash) *types.Transaction {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetPoolNonce(ctx context.Context, addr common.Address) (uint64, error) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) Stats() (pending int, queued int) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) TxPoolContent() (map[common.Address]types.Transactions, map[common.Address]types.Transactions) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) TxPoolContentFrom(addr common.Address) (types.Transactions, types.Transactions) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeNewTxsEvent(_ chan<- core.NewTxsEvent) event.Subscription {
panic("not implemented") // TODO: Implement
}
// Filter API
func (backend *Backend) BloomStatus() (uint64, uint64) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) GetLogs(ctx context.Context, blockHash common.Hash, number uint64) ([][]*types.Log, error) {
panic("not implemented")
}
func (backend *Backend) ServiceFilter(ctx context.Context, session *bloombits.MatcherSession) {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) ChainConfig() *params.ChainConfig {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) Engine() consensus.Engine {
panic("not implemented") // TODO: Implement
}
func (backend *Backend) PendingBlockAndReceipts() (*types.Block, types.Receipts) {
return nil, nil
} }

View File

@ -0,0 +1,17 @@
package mocks_test
import (
"testing"
"github.com/cerc-io/plugeth-statediff/test_helpers/mocks"
"github.com/ethereum/go-ethereum"
)
func TestBackend(t *testing.T) {
startingblock := uint64(42)
b := mocks.NewBackend(t, ethereum.SyncProgress{StartingBlock: startingblock})
block := b.Downloader().Progress().StartingBlock()
if startingblock != block {
t.Fatalf("wrong StartingBlock; expected %d, got %d", startingblock, block)
}
}

View File

@ -21,12 +21,12 @@ import (
"math/big" "math/big"
"time" "time"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/event"
"github.com/cerc-io/plugeth-statediff"
) )
// BlockChain is a mock blockchain for testing // BlockChain is a mock blockchain for testing
@ -150,8 +150,9 @@ func (bc *BlockChain) SetTd(hash common.Hash, blockNum uint64, td *big.Int) {
bc.TDByNum[blockNum] = td bc.TDByNum[blockNum] = td
} }
func (bc *BlockChain) UnlockTrie(root common.Hash) {} // func (bc *BlockChain) UnlockTrie(root core.Hash) {}
func (bc *BlockChain) StateCache() state.Database { // TODO
func (bc *BlockChain) StateCache() statediff.StateView {
return nil return nil
} }

View File

@ -17,9 +17,9 @@
package mocks package mocks
import ( import (
"github.com/cerc-io/plugeth-statediff"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
) )
var _ statediff.Builder = &Builder{} var _ statediff.Builder = &Builder{}

View File

@ -22,8 +22,9 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
sdtypes "github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/indexer/interfaces"
sdtypes "github.com/cerc-io/plugeth-statediff/types"
) )
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{} var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
@ -48,9 +49,7 @@ func (sdi *StateDiffIndexer) PushIPLD(txi interfaces.Batch, ipld sdtypes.IPLD) e
func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bool) {} func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bool) {}
func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) { func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) { return nil, nil }
return nil, nil
}
func (sdi *StateDiffIndexer) InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error { func (sdi *StateDiffIndexer) InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error {
return nil return nil

View File

@ -25,6 +25,8 @@ import (
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/cerc-io/plugeth-statediff/utils"
) )
// AddressToLeafKey hashes an returns an address // AddressToLeafKey hashes an returns an address
@ -47,7 +49,7 @@ var (
NullCodeHash = crypto.Keccak256Hash([]byte{}) NullCodeHash = crypto.Keccak256Hash([]byte{})
StoragePath = common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes() StoragePath = common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes()
StorageKey = common.HexToHash("0000000000000000000000000000000000000000000000000000000000000001").Bytes() StorageKey = common.HexToHash("0000000000000000000000000000000000000000000000000000000000000001").Bytes()
StorageValue = common.Hex2Bytes("0x03") StorageValue = utils.Hex2Bytes("0x03")
NullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000") NullHash = common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000")
Testdb = rawdb.NewMemoryDatabase() Testdb = rawdb.NewMemoryDatabase()
@ -64,11 +66,11 @@ var (
Account2Addr = crypto.PubkeyToAddress(Account2Key.PublicKey) //0x0D3ab14BBaD3D99F4203bd7a11aCB94882050E7e Account2Addr = crypto.PubkeyToAddress(Account2Key.PublicKey) //0x0D3ab14BBaD3D99F4203bd7a11aCB94882050E7e
Account1LeafKey = AddressToLeafKey(Account1Addr) Account1LeafKey = AddressToLeafKey(Account1Addr)
Account2LeafKey = AddressToLeafKey(Account2Addr) Account2LeafKey = AddressToLeafKey(Account2Addr)
ContractCode = common.Hex2Bytes("608060405234801561001057600080fd5b50336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506040518060200160405280600160ff16815250600190600161007492919061007a565b506100e4565b82606481019282156100ae579160200282015b828111156100ad578251829060ff1690559160200191906001019061008d565b5b5090506100bb91906100bf565b5090565b6100e191905b808211156100dd5760008160009055506001016100c5565b5090565b90565b6101ca806100f36000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101746022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b806001836064811061016a57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a72305820e3747183708fb6bff3f6f7a80fb57dcc1c19f83f9cb25457a3ed5c0424bde66864736f6c634300050a0032") ContractCode = utils.Hex2Bytes("608060405234801561001057600080fd5b50336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506040518060200160405280600160ff16815250600190600161007492919061007a565b506100e4565b82606481019282156100ae579160200282015b828111156100ad578251829060ff1690559160200191906001019061008d565b5b5090506100bb91906100bf565b5090565b6100e191905b808211156100dd5760008160009055506001016100c5565b5090565b90565b6101ca806100f36000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101746022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b806001836064811061016a57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a72305820e3747183708fb6bff3f6f7a80fb57dcc1c19f83f9cb25457a3ed5c0424bde66864736f6c634300050a0032")
ByteCodeAfterDeployment = common.Hex2Bytes("608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101746022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b806001836064811061016a57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a72305820e3747183708fb6bff3f6f7a80fb57dcc1c19f83f9cb25457a3ed5c0424bde66864736f6c634300050a0032") ByteCodeAfterDeployment = utils.Hex2Bytes("608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101746022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b806001836064811061016a57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a72305820e3747183708fb6bff3f6f7a80fb57dcc1c19f83f9cb25457a3ed5c0424bde66864736f6c634300050a0032")
CodeHash = common.HexToHash("0xaaea5efba4fd7b45d7ec03918ac5d8b31aa93b48986af0e6b591f0f087c80127") CodeHash = common.HexToHash("0xaaea5efba4fd7b45d7ec03918ac5d8b31aa93b48986af0e6b591f0f087c80127")
ContractCodeForInternalLeafNode = common.Hex2Bytes("608060405234801561001057600080fd5b50336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506040518060200160405280600160ff16815250600190600161007492919061007a565b506100e6565b8262019c5e81019282156100b0579160200282015b828111156100af578251829060ff1690559160200191906001019061008f565b5b5090506100bd91906100c1565b5090565b6100e391905b808211156100df5760008160009055506001016100c7565b5090565b90565b6101cc806100f56000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101766022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b8060018362019c5e811061016c57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a7231582007250e2c86ac8989891c4aa9c4737119491578200b9104c574143607ed71642b64736f6c63430005110032") ContractCodeForInternalLeafNode = utils.Hex2Bytes("608060405234801561001057600080fd5b50336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506040518060200160405280600160ff16815250600190600161007492919061007a565b506100e6565b8262019c5e81019282156100b0579160200282015b828111156100af578251829060ff1690559160200191906001019061008f565b5b5090506100bd91906100c1565b5090565b6100e391905b808211156100df5760008160009055506001016100c7565b5090565b90565b6101cc806100f56000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101766022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b8060018362019c5e811061016c57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a7231582007250e2c86ac8989891c4aa9c4737119491578200b9104c574143607ed71642b64736f6c63430005110032")
ByteCodeAfterDeploymentForInternalLeafNode = common.Hex2Bytes("608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101766022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b8060018362019c5e811061016c57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a7231582007250e2c86ac8989891c4aa9c4737119491578200b9104c574143607ed71642b64736f6c63430005110032") ByteCodeAfterDeploymentForInternalLeafNode = utils.Hex2Bytes("608060405234801561001057600080fd5b50600436106100365760003560e01c806343d726d61461003b578063c16431b914610045575b600080fd5b61004361007d565b005b61007b6004803603604081101561005b57600080fd5b81019080803590602001909291908035906020019092919050505061015c565b005b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610122576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260228152602001806101766022913960400191505060405180910390fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b8060018362019c5e811061016c57fe5b0181905550505056fe4f6e6c79206f776e65722063616e2063616c6c20746869732066756e6374696f6e2ea265627a7a7231582007250e2c86ac8989891c4aa9c4737119491578200b9104c574143607ed71642b64736f6c63430005110032")
CodeHashForInternalizedLeafNode = common.HexToHash("8327d45b7e6ffe26fc9728db4cd3c1c8177f7af2de0d31dfe5435e83101db04f") CodeHashForInternalizedLeafNode = common.HexToHash("8327d45b7e6ffe26fc9728db4cd3c1c8177f7af2de0d31dfe5435e83101db04f")
ContractAddr common.Address ContractAddr common.Address

View File

@ -24,9 +24,9 @@ import (
"strings" "strings"
"time" "time"
metrics2 "github.com/ethereum/go-ethereum/statediff/indexer/database/metrics" metrics2 "github.com/cerc-io/plugeth-statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/types" "github.com/cerc-io/plugeth-statediff/types"
) )
// SortKeys sorts the keys in the account map // SortKeys sorts the keys in the account map

View File

@ -21,6 +21,8 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
// common "github.com/openrelayxyz/plugeth-utils/core"
// "github.com/ethereum/go-ethereum/types"
) )
// StateRoots holds the state roots required for generating a state diff // StateRoots holds the state roots required for generating a state diff

26
utils/bytes.go Normal file
View File

@ -0,0 +1,26 @@
package utils
import "encoding/hex"
// FromHex returns the bytes represented by the hexadecimal string s.
// s may be prefixed with "0x".
func FromHex(s string) []byte {
if has0xPrefix(s) {
s = s[2:]
}
if len(s)%2 == 1 {
s = "0" + s
}
return Hex2Bytes(s)
}
// has0xPrefix validates str begins with '0x' or '0X'.
func has0xPrefix(str string) bool {
return len(str) >= 2 && str[0] == '0' && (str[1] == 'x' || str[1] == 'X')
}
// Hex2Bytes returns the bytes represented by the hexadecimal string str.
func Hex2Bytes(str string) []byte {
h, _ := hex.DecodeString(str)
return h
}

64
utils/encoding.go Normal file
View File

@ -0,0 +1,64 @@
package utils
// HexToCompact converts a hex path to the compact encoded format
func HexToCompact(hex []byte) []byte {
return hexToCompact(hex)
}
func hexToCompact(hex []byte) []byte {
terminator := byte(0)
if hasTerm(hex) {
terminator = 1
hex = hex[:len(hex)-1]
}
buf := make([]byte, len(hex)/2+1)
buf[0] = terminator << 5 // the flag byte
if len(hex)&1 == 1 {
buf[0] |= 1 << 4 // odd flag
buf[0] |= hex[0] // first nibble is contained in the first byte
hex = hex[1:]
}
decodeNibbles(hex, buf[1:])
return buf
}
// CompactToHex converts a compact encoded path to hex format
func CompactToHex(compact []byte) []byte {
return compactToHex(compact)
}
func compactToHex(compact []byte) []byte {
if len(compact) == 0 {
return compact
}
base := KeybytesToHex(compact)
// delete terminator flag
if base[0] < 2 {
base = base[:len(base)-1]
}
// apply odd flag
chop := 2 - base[0]&1
return base[chop:]
}
func KeybytesToHex(str []byte) []byte {
l := len(str)*2 + 1
var nibbles = make([]byte, l)
for i, b := range str {
nibbles[i*2] = b / 16
nibbles[i*2+1] = b % 16
}
nibbles[l-1] = 16
return nibbles
}
func decodeNibbles(nibbles []byte, bytes []byte) {
for bi, ni := 0, 0; ni < len(nibbles); bi, ni = bi+1, ni+2 {
bytes[bi] = nibbles[ni]<<4 | nibbles[ni+1]
}
}
// hasTerm returns whether a hex key has the terminator flag.
func hasTerm(s []byte) bool {
return len(s) > 0 && s[len(s)-1] == 16
}

1
utils/iterator.go Normal file
View File

@ -0,0 +1 @@
package utils

57
utils/log/log.go Normal file
View File

@ -0,0 +1,57 @@
package log
import (
"github.com/inconshreveable/log15"
"github.com/openrelayxyz/plugeth-utils/core"
)
type Logger = core.Logger
var (
DefaultLogger core.Logger
)
func init() {
// The plugeth logger is only initialized with the geth runtime,
// but tests expect to have a logger available, so default to this.
DefaultLogger = TestLogger()
}
func Trace(m string, a ...interface{}) { DefaultLogger.Trace(m, a...) }
func Debug(m string, a ...interface{}) { DefaultLogger.Debug(m, a...) }
func Info(m string, a ...interface{}) { DefaultLogger.Info(m, a...) }
func Warn(m string, a ...interface{}) { DefaultLogger.Warn(m, a...) }
func Crit(m string, a ...interface{}) { DefaultLogger.Crit(m, a...) }
func Error(m string, a ...interface{}) { DefaultLogger.Error(m, a...) }
type wrapLog15 struct{ log15.Logger }
func (l wrapLog15) New(ctx ...interface{}) Logger {
return wrapLog15{l.Logger.New(ctx...)}
}
func (l wrapLog15) Trace(m string, a ...interface{}) {
l.Logger.Debug(m, a...)
}
func TestLogger(ctx ...interface{}) Logger {
return wrapLog15{log15.New(ctx...)}
}
// New returns a Logger that includes the contextual args in all output
// (workaround for missing method in plugeth)
func New(ctx ...interface{}) Logger {
return ctxLogger{DefaultLogger, ctx}
}
type ctxLogger struct {
base Logger
ctx []interface{}
}
func (l ctxLogger) Trace(m string, a ...interface{}) { l.base.Trace(m, append(l.ctx, a...)...) }
func (l ctxLogger) Debug(m string, a ...interface{}) { l.base.Debug(m, append(l.ctx, a...)...) }
func (l ctxLogger) Info(m string, a ...interface{}) { l.base.Info(m, append(l.ctx, a...)...) }
func (l ctxLogger) Warn(m string, a ...interface{}) { l.base.Warn(m, append(l.ctx, a...)...) }
func (l ctxLogger) Crit(m string, a ...interface{}) { l.base.Crit(m, append(l.ctx, a...)...) }
func (l ctxLogger) Error(m string, a ...interface{}) { l.base.Error(m, append(l.ctx, a...)...) }

28
utils/trie.go Normal file
View File

@ -0,0 +1,28 @@
package utils
import (
"github.com/openrelayxyz/plugeth-utils/core"
plugeth_types "github.com/openrelayxyz/plugeth-utils/restricted/types"
"github.com/ethereum/go-ethereum/core/types"
)
type adaptTrieHasher struct {
types.TrieHasher
}
func AdaptTrieHasher(th types.TrieHasher) plugeth_types.TrieHasher {
return &adaptTrieHasher{th}
}
// TrieHasher is the tool used to calculate the hash of derivable list.
// This is internal, do not use.
type TrieHasher interface {
Reset()
Update([]byte, []byte) error
Hash() core.Hash
}
func (ath *adaptTrieHasher) Hash() core.Hash {
return core.Hash(ath.TrieHasher.Hash())
}

23
utils/utils.go Normal file
View File

@ -0,0 +1,23 @@
package utils
import (
"fmt"
"os"
"github.com/openrelayxyz/plugeth-utils/restricted/rlp"
)
// Fatalf formats a message to standard error and exits the program.
func Fatalf(format string, args ...interface{}) {
fmt.Fprintf(os.Stderr, "Fatal: "+format+"\n", args...)
os.Exit(1)
}
func MustDecode[T any](buf []byte) *T {
var ret T
err := rlp.DecodeBytes(buf, &ret)
if err != nil {
panic(fmt.Errorf("error decoding RLP %T: %w", ret, err))
}
return &ret
}