Initial statediff plugin #2
15
Dockerfile
@ -3,14 +3,15 @@ FROM golang:1.20-alpine as builder
|
||||
|
||||
RUN apk add --no-cache gcc musl-dev binutils-gold linux-headers git
|
||||
|
||||
# Get and cache deps
|
||||
COPY go.mod /plugeth-statediff/
|
||||
COPY go.sum /plugeth-statediff/
|
||||
RUN cd /plugeth-statediff && go mod download
|
||||
WORKDIR /plugeth-statediff/
|
||||
|
||||
ADD . /plugeth-statediff
|
||||
RUN cd /plugeth-statediff && \
|
||||
go build --tags linkgeth --buildmode=plugin --trimpath -o statediff.so ./main
|
||||
# Get and cache deps
|
||||
COPY go.mod .
|
||||
COPY go.sum .
|
||||
RUN go mod download
|
||||
|
||||
ADD . .
|
||||
RUN go build --tags linkgeth --buildmode=plugin --trimpath -o statediff.so ./main
|
||||
|
||||
FROM alpine:latest
|
||||
|
||||
|
13
Makefile
@ -8,6 +8,15 @@ $(MOCKS_DIR)/gen_backend.go:
|
||||
$(MOCKGEN) --package mocks --destination $@ \
|
||||
github.com/openrelayxyz/plugeth-utils/core Backend,Downloader
|
||||
|
||||
docker: mocks
|
||||
docker-image: mocks
|
||||
docker build . -t "cerc/plugeth-statediff:local"
|
||||
.PHONY: docker
|
||||
.PHONY: docker-image
|
||||
|
||||
# Local build
|
||||
BUILD_FLAGS := --trimpath
|
||||
|
||||
plugin: build/statediff.so
|
||||
.PHONY: plugin
|
||||
|
||||
build/statediff.so: ./**/*.go
|
||||
go build --tags linkgeth --buildmode=plugin -o $@ $(BUILD_FLAGS) ./main
|
||||
|
39
README.md
@ -32,8 +32,7 @@ hash.
|
||||
State leaf nodes contain information about account changes, including whether they are removed, an
|
||||
account wrapper with account details and identifiers, and an array of storage leaf nodes
|
||||
representing storage changes. The IPLD type encapsulates CID-content pairs, used for code mappings
|
||||
and trie node (both intermediate and leaf) IPLD objects. Lastly, `CodeAndCodeHash` stores
|
||||
codehash-to-code mappings.
|
||||
and trie node (both intermediate and leaf) IPLD objects.
|
||||
|
||||
```go
|
||||
// Payload packages the data to send to state diff subscriptions
|
||||
@ -83,12 +82,6 @@ type IPLD struct {
|
||||
CID string
|
||||
Content []byte
|
||||
}
|
||||
|
||||
// CodeAndCodeHash struct to hold codehash => code mappings
|
||||
type CodeAndCodeHash struct {
|
||||
Hash common.Hash
|
||||
Code []byte
|
||||
}
|
||||
```
|
||||
|
||||
## Usage
|
||||
@ -152,7 +145,7 @@ type Params struct {
|
||||
IncludeReceipts bool
|
||||
IncludeTD bool
|
||||
IncludeCode bool
|
||||
WatchedAddresses []core.Address
|
||||
WatchedAddresses []common.Address
|
||||
}
|
||||
```
|
||||
|
||||
@ -209,8 +202,6 @@ for {
|
||||
}
|
||||
```
|
||||
|
||||
Additionally, the `StreamCodeAndCodeHash` subscription method streams codehash-to-code pairs at a given block to a websocket channel.
|
||||
|
||||
#### Unary endpoints
|
||||
|
||||
The service also exposes unary RPC endpoints for retrieving the state diff `StateObject` for a specific block height/hash.
|
||||
@ -220,7 +211,7 @@ The service also exposes unary RPC endpoints for retrieving the state diff `Stat
|
||||
StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error)
|
||||
|
||||
// StateDiffFor returns a state diff payload for the specific blockhash
|
||||
StateDiffFor(ctx context.Context, blockHash core.Hash, params Params) (*Payload, error)
|
||||
StateDiffFor(ctx context.Context, blockHash common.Hash, params Params) (*Payload, error)
|
||||
```
|
||||
|
||||
To expose this endpoint the node needs to have the HTTP server turned on (`--http`),
|
||||
@ -255,27 +246,25 @@ track this process:
|
||||
Our Postgres schemas are built around a single IPFS backing Postgres IPLD blockstore table
|
||||
(`ipld.blocks`) that conforms with
|
||||
[go-ds-sql](https://github.com/ipfs/go-ds-sql/blob/master/postgres/postgres.go). All IPLD objects
|
||||
are stored in this table, where `key` is the blockstore-prefixed multihash key for the IPLD object
|
||||
and `data` contains the bytes for the IPLD block (in the case of all Ethereum IPLDs, this is the RLP
|
||||
byte encoding of the Ethereum object).
|
||||
are stored in this table, where `key` is the CID for the IPLD object and `data` contains the bytes
|
||||
for the IPLD block (in the case of all Ethereum IPLDs, this is the RLP byte encoding of the Ethereum
|
||||
object).
|
||||
|
||||
The IPLD objects in this table can be traversed using an IPLD DAG interface, but since this table
|
||||
only maps multihash to raw IPLD object it is not particularly useful for searching through the data
|
||||
by looking up Ethereum objects by their constituent fields (e.g. by block number, tx
|
||||
source/recipient, state/storage trie node path). To improve the accessibility of these objects we
|
||||
create an Ethereum [advanced data
|
||||
only maps CID to raw IPLD object it is not very suitable for looking up Ethereum objects by their
|
||||
roysc marked this conversation as resolved
Outdated
|
||||
constituent fields (e.g. by tx source/recipient, state/storage trie path). To improve the
|
||||
accessibility of these objects we create an Ethereum [advanced data
|
||||
layout](https://github.com/ipld/specs#schemas-and-advanced-data-layouts) (ADL) by generating
|
||||
secondary indexes on top of the raw IPLDs in other Postgres tables.
|
||||
|
||||
These secondary index tables fall under the `eth` schema and follow an `{objectType}_cids` naming
|
||||
convention. These tables provide a view into individual fields of the underlying Ethereum IPLD
|
||||
objects, allowing lookups on these fields, and reference the raw IPLD objects stored in
|
||||
`ipld.blocks` by foreign keys to their multihash keys. Additionally, these tables maintain the
|
||||
hash-linked nature of Ethereum objects to one another. E.g. a storage trie node entry in the
|
||||
`storage_cids` table contains a `state_id` foreign key which references the `id` for the
|
||||
`state_cids` entry that contains the state leaf node for the contract that storage node belongs to,
|
||||
and in turn that `state_cids` entry contains a `header_id` foreign key which references the `id` of
|
||||
the `header_cids` entry that contains the header for the block these state and storage nodes were
|
||||
`ipld.blocks` by CID. Additionally, these tables maintain the hash-linked nature of Ethereum
|
||||
objects to one another, e.g. a storage trie node entry in the `storage_cids` table contains a
|
||||
`state_leaf_key` field referencing the `state_cids` entry for the state trie node of its owning
|
||||
contract, and that `state_cids` entry in turn contains a `header_id` field referencing the
|
||||
`block_hash` of the `header_cids` entry for the block in which these state and storage nodes were
|
||||
updated (diffed).
|
||||
|
||||
### Optimization
|
||||
|
@ -1,23 +1,11 @@
|
||||
package adapt
|
||||
|
||||
import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
|
||||
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 ¶ms.ChainConfig{
|
||||
ChainID: cc.ChainID,
|
||||
|
27
api.go
@ -29,7 +29,6 @@ import (
|
||||
const APIName = "statediff"
|
||||
|
||||
// APIVersion is the version of the state diffing service API
|
||||
// TODO: match package version?
|
||||
const APIVersion = "0.0.1"
|
||||
roysc marked this conversation as resolved
Outdated
i-norden
commented
Not sure this makes sense since the package can be updated and version changed without affecting the API Not sure this makes sense since the package can be updated and version changed without affecting the API
|
||||
|
||||
// PublicStateDiffAPI provides an RPC subscription interface
|
||||
@ -88,32 +87,6 @@ func (api *PublicAPI) StateDiffFor(ctx context.Context, blockHash common.Hash, p
|
||||
return api.sds.StateDiffFor(blockHash, params)
|
||||
}
|
||||
|
||||
// StreamCodeAndCodeHash writes all of the codehash=>code pairs at a given block to a websocket channel.
|
||||
func (api *PublicAPI) StreamCodeAndCodeHash(ctx context.Context, blockNumber uint64) (<-chan types.CodeAndCodeHash, error) {
|
||||
payloadChan := make(chan types.CodeAndCodeHash, chainEventChanSize)
|
||||
clientChan := make(chan types.CodeAndCodeHash, chainEventChanSize)
|
||||
quitChan := make(chan bool, 1)
|
||||
api.sds.StreamCodeAndCodeHash(blockNumber, payloadChan, quitChan)
|
||||
|
||||
go func() {
|
||||
defer close(clientChan)
|
||||
defer close(payloadChan)
|
||||
|
||||
for {
|
||||
select {
|
||||
case payload := <-payloadChan:
|
||||
clientChan <- payload
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-quitChan:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return clientChan, nil
|
||||
}
|
||||
|
||||
// WriteStateDiffAt writes a state diff object directly to DB at the specific blockheight
|
||||
func (api *PublicAPI) WriteStateDiffAt(ctx context.Context, blockNumber uint64, params Params) JobID {
|
||||
var err error
|
||||
i-norden marked this conversation as resolved
Outdated
i-norden
commented
We haven't ever actually used this endpoint, should consider getting rid of it. We haven't ever actually used this endpoint, should consider getting rid of it.
roysc
commented
In that case, I vote for dropping it for now, pending a use case In that case, I vote for dropping it for now, pending a use case
i-norden
commented
That sounds good to me. That sounds good to me.
|
||||
|
@ -25,8 +25,6 @@ type BlockChain interface {
|
||||
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() adapt.StateView
|
||||
roysc marked this conversation as resolved
Outdated
i-norden
commented
We had this previously because the statediffing process would fall/lag far enough behind head, below the pruning threshold of a full node, such that the full node would sometimes prune away the state before the statediffing service had used it. This shouldn't be a problem anymore after the multitude of improvements made to performance. We had this previously because the statediffing process would fall/lag far enough behind head, below the pruning threshold of a full node, such that the full node would sometimes prune away the state before the statediffing service had used it. This shouldn't be a problem anymore after the multitude of improvements made to performance.
|
||||
}
|
||||
|
||||
@ -49,8 +47,7 @@ func (b *pluginBlockChain) SubscribeChainEvent(ch chan<- core.ChainEvent) event.
|
||||
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)
|
||||
// Note: logs are processed with receipts while building the payload
|
||||
ch <- core.ChainEvent{
|
||||
Block: block,
|
||||
Hash: common.Hash(event.Hash),
|
||||
roysc marked this conversation as resolved
Outdated
i-norden
commented
We should get the logs when we lookup the Receipts and pack them into the statediff payload. We should get the logs when we lookup the Receipts and pack them into the statediff payload.
|
||||
|
10
builder.go
@ -140,7 +140,7 @@ func (sdb *StateDiffBuilder) WriteStateDiffObject(args Args, params Params, outp
|
||||
|
||||
func (sdb *StateDiffBuilder) BuildStateDiff(iterPairs []IterPair, params Params,
|
||||
output sdtypes.StateNodeSink, ipldOutput sdtypes.IPLDSink, logger log.Logger, prefixPath []byte) error {
|
||||
logger.Debug("statediff BEGIN BuildStateDiff")
|
||||
logger.Trace("statediff BEGIN BuildStateDiff")
|
||||
defer metrics.ReportAndUpdateDuration("statediff END BuildStateDiff", time.Now(), logger, metrics.IndexerMetrics.BuildStateDiffTimer)
|
||||
// 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
|
||||
@ -195,7 +195,7 @@ func (sdb *StateDiffBuilder) BuildStateDiff(iterPairs []IterPair, params Params,
|
||||
// and a slice of the paths for all of the nodes included in both
|
||||
func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator,
|
||||
watchedAddressesLeafPaths [][]byte, output sdtypes.IPLDSink, logger log.Logger, prefixPath []byte) (sdtypes.AccountMap, error) {
|
||||
logger.Debug("statediff BEGIN createdAndUpdatedState")
|
||||
logger.Trace("statediff BEGIN createdAndUpdatedState")
|
||||
defer metrics.ReportAndUpdateDuration("statediff END createdAndUpdatedState", time.Now(), logger, metrics.IndexerMetrics.CreatedAndUpdatedStateTimer)
|
||||
diffAccountsAtB := make(sdtypes.AccountMap)
|
||||
|
||||
@ -280,7 +280,7 @@ func (sdb *StateDiffBuilder) processStateValueNode(it trie.NodeIterator, parentB
|
||||
// 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 sdtypes.AccountMap,
|
||||
watchedAddressesLeafPaths [][]byte, output sdtypes.StateNodeSink, logger log.Logger, prefixPath []byte) (sdtypes.AccountMap, error) {
|
||||
logger.Debug("statediff BEGIN deletedOrUpdatedState")
|
||||
logger.Trace("statediff BEGIN deletedOrUpdatedState")
|
||||
defer metrics.ReportAndUpdateDuration("statediff END deletedOrUpdatedState", time.Now(), logger, metrics.IndexerMetrics.DeletedOrUpdatedStateTimer)
|
||||
diffAccountAtA := make(sdtypes.AccountMap)
|
||||
|
||||
@ -339,7 +339,7 @@ func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffA
|
||||
// those account maps to remove the accounts which were updated
|
||||
func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions sdtypes.AccountMap, updatedKeys []string,
|
||||
output sdtypes.StateNodeSink, ipldOutput sdtypes.IPLDSink, logger log.Logger) error {
|
||||
logger.Debug("statediff BEGIN buildAccountUpdates",
|
||||
logger.Trace("statediff BEGIN buildAccountUpdates",
|
||||
"creations", len(creations), "deletions", len(deletions), "updated", len(updatedKeys))
|
||||
defer metrics.ReportAndUpdateDuration("statediff END buildAccountUpdates ",
|
||||
time.Now(), logger, metrics.IndexerMetrics.BuildAccountUpdatesTimer)
|
||||
@ -375,7 +375,7 @@ func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions sdtypes.Ac
|
||||
// it also returns the code and codehash for created contract accounts
|
||||
func (sdb *StateDiffBuilder) buildAccountCreations(accounts sdtypes.AccountMap, output sdtypes.StateNodeSink,
|
||||
ipldOutput sdtypes.IPLDSink, logger log.Logger) error {
|
||||
logger.Debug("statediff BEGIN buildAccountCreations")
|
||||
logger.Trace("statediff BEGIN buildAccountCreations")
|
||||
defer metrics.ReportAndUpdateDuration("statediff END buildAccountCreations",
|
||||
time.Now(), logger, metrics.IndexerMetrics.BuildAccountCreationsTimer)
|
||||
for _, val := range accounts {
|
||||
|
2
go.mod
@ -124,6 +124,6 @@ require (
|
||||
replace (
|
||||
// github.com/ethereum/go-ethereum => ../plugeth
|
||||
// github.com/openrelayxyz/plugeth-utils => ../plugeth-utils
|
||||
github.com/ethereum/go-ethereum => git.vdb.to/cerc-io/plugeth v0.0.0-20230622162124-0ed9eba1decb
|
||||
github.com/ethereum/go-ethereum => git.vdb.to/cerc-io/plugeth v0.0.0-20230629081247-feb6ccc72b3f
|
||||
github.com/openrelayxyz/plugeth-utils => git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230622072028-1d3da8ce80ee
|
||||
)
|
||||
|
4
go.sum
@ -1,6 +1,6 @@
|
||||
cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
|
||||
git.vdb.to/cerc-io/plugeth v0.0.0-20230622162124-0ed9eba1decb h1:p3R953gnt/kxmJPbRRH09l3UDGwarFADxjJx4Y9SQrY=
|
||||
git.vdb.to/cerc-io/plugeth v0.0.0-20230622162124-0ed9eba1decb/go.mod h1:odpOaIpK01aVThIoAuw9YryLBJeHYOsDn9Mxm4LhB5s=
|
||||
git.vdb.to/cerc-io/plugeth v0.0.0-20230629081247-feb6ccc72b3f h1:QGxQXUa53S1Ci+gp17zjw5TI62q4fyZ/PTvHQ72FxZw=
|
||||
git.vdb.to/cerc-io/plugeth v0.0.0-20230629081247-feb6ccc72b3f/go.mod h1:odpOaIpK01aVThIoAuw9YryLBJeHYOsDn9Mxm4LhB5s=
|
||||
git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230622072028-1d3da8ce80ee h1:DJ1bR/2k7PopUtchEoTw5QHV1DHb9p0ubcb3yKJc10I=
|
||||
git.vdb.to/cerc-io/plugeth-utils v0.0.0-20230622072028-1d3da8ce80ee/go.mod h1:p5Jc8deG2yxXI8DzmrH3kHNEwlQqcOQS0pmGulsqg+M=
|
||||
github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
|
||||
|
@ -41,7 +41,7 @@ func setupCSVIndexer(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
ind, err = file.NewStateDiffIndexer(mocks.TestConfig, file.CSVTestConfig)
|
||||
ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.CSVTestConfig)
|
||||
require.NoError(t, err)
|
||||
|
||||
db, err = postgres.SetupSQLXDB()
|
||||
|
@ -130,7 +130,7 @@ func (tx fileWriters) flush() error {
|
||||
|
||||
func NewCSVWriter(path string, watchedAddressesFilePath string) (*CSVWriter, error) {
|
||||
if err := os.MkdirAll(path, 0777); err != nil {
|
||||
return nil, fmt.Errorf("unable to make MkdirAll for path: %s err: %s", path, err)
|
||||
return nil, fmt.Errorf("unable to create directory '%s': %w", path, err)
|
||||
}
|
||||
|
||||
writers, err := makeFileWriters(path, Tables)
|
||||
@ -383,7 +383,7 @@ func loadWatchedAddressesRows(filePath string) ([][]string, error) {
|
||||
return [][]string{}, nil
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("error opening watched addresses file: %v", err)
|
||||
return nil, fmt.Errorf("error opening watched addresses file: %w", err)
|
||||
}
|
||||
|
||||
defer file.Close()
|
||||
@ -399,7 +399,7 @@ func dumpWatchedAddressesRows(watchedAddressesWriter fileWriter, filteredRows []
|
||||
|
||||
file, err := os.Create(file.Name())
|
||||
if err != nil {
|
||||
return fmt.Errorf("error creating watched addresses file: %v", err)
|
||||
return fmt.Errorf("error creating watched addresses file: %w", err)
|
||||
}
|
||||
|
||||
watchedAddressesWriter.Writer = csv.NewWriter(file)
|
||||
|
@ -41,7 +41,7 @@ func setupIndexer(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
ind, err = file.NewStateDiffIndexer(mocks.TestConfig, file.SQLTestConfig)
|
||||
ind, err = file.NewStateDiffIndexer(mocks.TestChainConfig, file.SQLTestConfig)
|
||||
require.NoError(t, err)
|
||||
|
||||
db, err = postgres.SetupSQLXDB()
|
||||
|
@ -28,7 +28,8 @@ import (
|
||||
)
|
||||
|
||||
func setupLegacyPGXIndexer(t *testing.T) {
|
||||
db, err = postgres.SetupPGXDB(postgres.TestConfig)
|
||||
config, _ := postgres.TestConfig.WithEnv()
|
||||
db, err = postgres.SetupPGXDB(config)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -29,17 +29,27 @@ import (
|
||||
"github.com/cerc-io/plugeth-statediff/indexer/test"
|
||||
)
|
||||
|
||||
var defaultPgConfig postgres.Config
|
||||
|
||||
func init() {
|
||||
var err error
|
||||
defaultPgConfig, err = postgres.TestConfig.WithEnv()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func setupPGXIndexer(t *testing.T, config postgres.Config) {
|
||||
db, err = postgres.SetupPGXDB(config)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db)
|
||||
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestChainConfig, db)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
func setupPGX(t *testing.T) {
|
||||
setupPGXWithConfig(t, postgres.TestConfig)
|
||||
setupPGXWithConfig(t, defaultPgConfig)
|
||||
}
|
||||
|
||||
func setupPGXWithConfig(t *testing.T, config postgres.Config) {
|
||||
@ -48,7 +58,7 @@ func setupPGXWithConfig(t *testing.T, config postgres.Config) {
|
||||
}
|
||||
|
||||
func setupPGXNonCanonical(t *testing.T) {
|
||||
setupPGXIndexer(t, postgres.TestConfig)
|
||||
setupPGXIndexer(t, defaultPgConfig)
|
||||
test.SetupTestDataNonCanonical(t, ind)
|
||||
}
|
||||
|
||||
@ -103,7 +113,7 @@ func TestPGXIndexer(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("Publish and index with CopyFrom enabled.", func(t *testing.T) {
|
||||
config := postgres.TestConfig
|
||||
config := defaultPgConfig
|
||||
config.CopyFrom = true
|
||||
|
||||
setupPGXWithConfig(t, config)
|
||||
@ -169,7 +179,7 @@ func TestPGXIndexerNonCanonical(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestPGXWatchAddressMethods(t *testing.T) {
|
||||
setupPGXIndexer(t, postgres.TestConfig)
|
||||
setupPGXIndexer(t, defaultPgConfig)
|
||||
defer tearDown(t)
|
||||
defer checkTxClosure(t, 1, 0, 1)
|
||||
|
||||
|
@ -31,8 +31,9 @@ import (
|
||||
)
|
||||
|
||||
var (
|
||||
pgConfig, _ = postgres.MakeConfig(postgres.TestConfig)
|
||||
ctx = context.Background()
|
||||
pgConfig, _ = postgres.TestConfig.WithEnv()
|
||||
pgxConfig, _ = postgres.MakeConfig(pgConfig)
|
||||
ctx = context.Background()
|
||||
)
|
||||
|
||||
func expectContainsSubstring(t *testing.T, full string, sub string) {
|
||||
@ -43,9 +44,9 @@ func expectContainsSubstring(t *testing.T, full string, sub string) {
|
||||
|
||||
func TestPostgresPGX(t *testing.T) {
|
||||
t.Run("connects to the sql", func(t *testing.T) {
|
||||
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig)
|
||||
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgxConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err)
|
||||
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgxConfig.ConnString(), err)
|
||||
}
|
||||
if dbPool == nil {
|
||||
t.Fatal("DB pool is nil")
|
||||
@ -61,9 +62,9 @@ func TestPostgresPGX(t *testing.T) {
|
||||
// sized int, so use string representation of big.Int
|
||||
// and cast on insert
|
||||
|
||||
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig)
|
||||
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgxConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err)
|
||||
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgxConfig.ConnString(), err)
|
||||
}
|
||||
defer dbPool.Close()
|
||||
|
||||
@ -111,7 +112,7 @@ func TestPostgresPGX(t *testing.T) {
|
||||
badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100))
|
||||
badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"}
|
||||
|
||||
_, err := postgres.NewPGXDriver(ctx, postgres.TestConfig, badInfo)
|
||||
_, err := postgres.NewPGXDriver(ctx, pgConfig, badInfo)
|
||||
if err == nil {
|
||||
t.Fatal("Expected an error")
|
||||
}
|
||||
|
@ -35,7 +35,7 @@ func TestPostgresSQLX(t *testing.T) {
|
||||
|
||||
t.Run("connects to the database", func(t *testing.T) {
|
||||
var err error
|
||||
connStr := postgres.TestConfig.DbConnectionString()
|
||||
connStr := pgConfig.DbConnectionString()
|
||||
|
||||
sqlxdb, err = sqlx.Connect("postgres", connStr)
|
||||
if err != nil {
|
||||
@ -58,7 +58,7 @@ func TestPostgresSQLX(t *testing.T) {
|
||||
// sized int, so use string representation of big.Int
|
||||
// and cast on insert
|
||||
|
||||
connStr := postgres.TestConfig.DbConnectionString()
|
||||
connStr := pgConfig.DbConnectionString()
|
||||
db, err := sqlx.Connect("postgres", connStr)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -109,7 +109,7 @@ func TestPostgresSQLX(t *testing.T) {
|
||||
badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100))
|
||||
badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"}
|
||||
|
||||
_, err := postgres.NewSQLXDriver(ctx, postgres.TestConfig, badInfo)
|
||||
_, err := postgres.NewSQLXDriver(ctx, pgConfig, badInfo)
|
||||
if err == nil {
|
||||
t.Fatal("Expected an error")
|
||||
}
|
||||
|
@ -25,7 +25,10 @@ import (
|
||||
|
||||
// SetupSQLXDB is used to setup a sqlx db for tests
|
||||
func SetupSQLXDB() (sql.Database, error) {
|
||||
conf := TestConfig
|
||||
conf, err := TestConfig.WithEnv()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
conf.MaxIdle = 0
|
||||
driver, err := NewSQLXDriver(context.Background(), conf, node.Info{})
|
||||
if err != nil {
|
||||
|
@ -34,7 +34,7 @@ func setupSQLXIndexer(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db)
|
||||
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestChainConfig, db)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
|
@ -58,7 +58,7 @@ func processTransactions(txs []*types.Transaction) ([]*EthTx, error) {
|
||||
|
||||
// processReceiptsAndLogs will take in receipts
|
||||
// to return IPLD node slices for eth-rct and eth-log
|
||||
func processReceiptsAndLogs(rcts types.Receipts) ([]*EthReceipt, [][]*EthLog, error) {
|
||||
func processReceiptsAndLogs(rcts []*types.Receipt) ([]*EthReceipt, [][]*EthLog, error) {
|
||||
// Pre allocating memory.
|
||||
ethRctNodes := make([]*EthReceipt, len(rcts))
|
||||
ethLogNodes := make([][]*EthLog, len(rcts))
|
||||
@ -81,31 +81,6 @@ func processReceiptsAndLogs(rcts types.Receipts) ([]*EthReceipt, [][]*EthLog, er
|
||||
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) {
|
||||
roysc marked this conversation as resolved
Outdated
i-norden
commented
What's this commented out stuff for? What's this commented out stuff for?
roysc
commented
for disposal, just left over from checking something for disposal, just left over from checking something
|
||||
logNodes := make([]*EthLog, len(logs))
|
||||
for idx, log := range logs {
|
||||
|
@ -39,8 +39,8 @@ import (
|
||||
// Test variables
|
||||
var (
|
||||
// block data
|
||||
TestConfig = params.MainnetChainConfig
|
||||
BlockNumber = TestConfig.LondonBlock
|
||||
TestChainConfig = params.MainnetChainConfig
|
||||
BlockNumber = TestChainConfig.LondonBlock
|
||||
|
||||
// canonical block at London height
|
||||
// includes 5 transactions: 3 Legacy + 1 EIP-2930 + 1 EIP-1559
|
||||
@ -55,7 +55,7 @@ var (
|
||||
BaseFee: big.NewInt(params.InitialBaseFee),
|
||||
Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476777"),
|
||||
}
|
||||
MockTransactions, MockReceipts, SenderAddr = createTransactionsAndReceipts(TestConfig, BlockNumber)
|
||||
MockTransactions, MockReceipts, SenderAddr = createTransactionsAndReceipts(TestChainConfig, BlockNumber)
|
||||
MockBlock = types.NewBlock(&MockHeader, MockTransactions, nil, MockReceipts, trie.NewEmpty(nil))
|
||||
MockHeaderRlp, _ = rlp.EncodeToBytes(MockBlock.Header())
|
||||
|
||||
@ -63,7 +63,7 @@ var (
|
||||
// includes 2nd and 5th transactions from the canonical block
|
||||
MockNonCanonicalHeader = MockHeader
|
||||
MockNonCanonicalBlockTransactions = types.Transactions{MockTransactions[1], MockTransactions[4]}
|
||||
MockNonCanonicalBlockReceipts = createNonCanonicalBlockReceipts(TestConfig, BlockNumber, MockNonCanonicalBlockTransactions)
|
||||
MockNonCanonicalBlockReceipts = createNonCanonicalBlockReceipts(TestChainConfig, BlockNumber, MockNonCanonicalBlockTransactions)
|
||||
MockNonCanonicalBlock = types.NewBlock(&MockNonCanonicalHeader, MockNonCanonicalBlockTransactions, nil, MockNonCanonicalBlockReceipts, trie.NewEmpty(nil))
|
||||
MockNonCanonicalHeaderRlp, _ = rlp.EncodeToBytes(MockNonCanonicalBlock.Header())
|
||||
|
||||
@ -82,7 +82,7 @@ var (
|
||||
Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476777"),
|
||||
}
|
||||
MockNonCanonicalBlock2Transactions = types.Transactions{MockTransactions[2], MockTransactions[4]}
|
||||
MockNonCanonicalBlock2Receipts = createNonCanonicalBlockReceipts(TestConfig, Block2Number, MockNonCanonicalBlock2Transactions)
|
||||
MockNonCanonicalBlock2Receipts = createNonCanonicalBlockReceipts(TestChainConfig, Block2Number, MockNonCanonicalBlock2Transactions)
|
||||
MockNonCanonicalBlock2 = types.NewBlock(&MockNonCanonicalHeader2, MockNonCanonicalBlock2Transactions, nil, MockNonCanonicalBlock2Receipts, trie.NewEmpty(nil))
|
||||
MockNonCanonicalHeader2Rlp, _ = rlp.EncodeToBytes(MockNonCanonicalBlock2.Header())
|
||||
|
||||
|
27
main/main.go
@ -25,13 +25,6 @@ var (
|
||||
func Initialize(ctx core.Context, pl core.PluginLoader, logger core.Logger) {
|
||||
log.SetDefaultLogger(logger)
|
||||
|
||||
// lvl, err := strconv.ParseInt(ctx.String("verbosity"), 10, 8)
|
||||
// if err != nil {
|
||||
// log.Error("cannot parse verbosity", "error", err)
|
||||
// }
|
||||
// log.TestLogger.SetLevel(int(lvl))
|
||||
// log.SetDefaultLogger(log.TestLogger)
|
||||
|
||||
pluginLoader = pl
|
||||
roysc marked this conversation as resolved
Outdated
i-norden
commented
Is this from geth upstream or here for future use? If not, lets remove the commented out stuff. Is this from geth upstream or here for future use? If not, lets remove the commented out stuff.
roysc
commented
no, just left over from debugging logging no, just left over from debugging logging
|
||||
gethContext = ctx
|
||||
|
||||
@ -85,23 +78,3 @@ func GetAPIs(stack core.Node, backend core.Backend) []core.API {
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// 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 blockchain == nil {
|
||||
log.Warn("StateUpdate called before InitializeNode", "root", blockRoot)
|
||||
return
|
||||
}
|
||||
|
||||
// for hash, code := range codeUpdates {
|
||||
// log.Debug("UPDATING CODE", "hash", hash)
|
||||
// codeStore.Set(hash, code)
|
||||
// }
|
||||
}
|
||||
|
@ -41,6 +41,10 @@ import (
|
||||
sdtypes "github.com/cerc-io/plugeth-statediff/types"
|
||||
)
|
||||
|
||||
func init() {
|
||||
test_helpers.SilenceLogs()
|
||||
}
|
||||
|
||||
var (
|
||||
db ethdb.Database
|
||||
genesisBlock, block0, block1, block2, block3 *types.Block
|
||||
|
48
service.go
@ -31,7 +31,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
plugeth "github.com/openrelayxyz/plugeth-utils/core"
|
||||
"github.com/thoas/go-funk"
|
||||
|
||||
@ -437,11 +436,6 @@ func (sds *Service) processStateDiff(currentBlock *types.Block, parentRoot commo
|
||||
BlockHash: currentBlock.Hash(),
|
||||
BlockNumber: currentBlock.Number(),
|
||||
}, params)
|
||||
// allow dereferencing of parent, keep current locked as it should be the next parent
|
||||
// sds.BlockChain.UnlockTrie(parentRoot)
|
||||
// if err != nil {
|
||||
// return nil, err
|
||||
// }
|
||||
stateDiffRlp, err := rlp.EncodeToBytes(&stateDiff)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -754,10 +748,6 @@ func (sds *Service) writeStateDiff(block *types.Block, parentRoot common.Hash, p
|
||||
if err = tx.Submit(err); err != nil {
|
||||
return fmt.Errorf("batch transaction submission failed: %w", err)
|
||||
}
|
||||
|
||||
// allow dereferencing of parent, keep current locked as it should be the next parent
|
||||
// TODO never locked
|
||||
// sds.BlockChain.UnlockTrie(parentRoot)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -798,44 +788,6 @@ func (sds *Service) UnsubscribeWriteStatus(id SubID) {
|
||||
sds.jobStatusSubsMutex.Unlock()
|
||||
}
|
||||
|
||||
// StreamCodeAndCodeHash subscription method for extracting all the codehash=>code mappings that exist in the trie at the provided height
|
||||
func (sds *Service) StreamCodeAndCodeHash(blockNumber uint64, outChan chan<- types2.CodeAndCodeHash, quitChan chan<- bool) {
|
||||
current := sds.BlockChain.GetBlockByNumber(blockNumber)
|
||||
log.Info("sending code and codehash", "number", blockNumber)
|
||||
currentTrie, err := sds.BlockChain.StateCache().OpenTrie(current.Root())
|
||||
if err != nil {
|
||||
log.Error("error getting trie for block", "number", current.Number(), "error", err)
|
||||
close(quitChan)
|
||||
return
|
||||
}
|
||||
leafIt := trie.NewIterator(currentTrie.NodeIterator(nil))
|
||||
go func() {
|
||||
defer close(quitChan)
|
||||
for leafIt.Next() {
|
||||
select {
|
||||
case <-sds.QuitChan:
|
||||
return
|
||||
default:
|
||||
}
|
||||
account := new(types.StateAccount)
|
||||
if err := rlp.DecodeBytes(leafIt.Value, account); err != nil {
|
||||
log.Error("error decoding state account", "error", err)
|
||||
return
|
||||
}
|
||||
codeHash := common.BytesToHash(account.CodeHash)
|
||||
code, err := sds.BlockChain.StateCache().ContractCode(codeHash)
|
||||
if err != nil {
|
||||
log.Error("error collecting contract code", "error", err)
|
||||
return
|
||||
}
|
||||
outChan <- types2.CodeAndCodeHash{
|
||||
Hash: codeHash,
|
||||
Code: code,
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// WatchAddress performs one of following operations on the watched addresses in sds.writeLoopParams and the db:
|
||||
// add | remove | set | clear
|
||||
func (sds *Service) WatchAddress(operation types2.OperationType, args []types2.WatchAddressArg) error {
|
||||
|
@ -31,20 +31,17 @@ import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
geth_log "github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
|
||||
statediff "github.com/cerc-io/plugeth-statediff"
|
||||
"github.com/cerc-io/plugeth-statediff/test_helpers"
|
||||
"github.com/cerc-io/plugeth-statediff/test_helpers/mocks"
|
||||
sdtypes "github.com/cerc-io/plugeth-statediff/types"
|
||||
// "github.com/cerc-io/plugeth-statediff/utils/log"
|
||||
)
|
||||
|
||||
func init() {
|
||||
// The geth sync logs are noisy, silence them
|
||||
geth_log.Root().SetHandler(geth_log.DiscardHandler())
|
||||
// log.TestLogger.SetLevel(2)
|
||||
test_helpers.SilenceLogs()
|
||||
}
|
||||
|
||||
func TestServiceLoop(t *testing.T) {
|
||||
|
@ -149,8 +149,6 @@ func (bc *BlockChain) SetTd(hash common.Hash, blockNum uint64, td *big.Int) {
|
||||
bc.TDByNum[blockNum] = td
|
||||
}
|
||||
|
||||
// func (bc *BlockChain) UnlockTrie(root core.Hash) {}
|
||||
|
||||
// TODO
|
||||
func (bc *BlockChain) StateCache() adapt.StateView {
|
||||
return nil
|
||||
|
567
test_helpers/mocks/gen_backend.go
Normal file
@ -0,0 +1,567 @@
|
||||
// Code generated by MockGen. DO NOT EDIT.
|
||||
// Source: github.com/openrelayxyz/plugeth-utils/core (interfaces: Backend,Downloader)
|
||||
|
||||
// Package mocks is a generated GoMock package.
|
||||
package mocks
|
||||
|
||||
import (
|
||||
context "context"
|
||||
big "math/big"
|
||||
reflect "reflect"
|
||||
|
||||
gomock "github.com/golang/mock/gomock"
|
||||
core "github.com/openrelayxyz/plugeth-utils/core"
|
||||
)
|
||||
|
||||
// MockBackend is a mock of Backend interface.
|
||||
type MockBackend struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockBackendMockRecorder
|
||||
}
|
||||
|
||||
// MockBackendMockRecorder is the mock recorder for MockBackend.
|
||||
type MockBackendMockRecorder struct {
|
||||
mock *MockBackend
|
||||
}
|
||||
|
||||
// NewMockBackend creates a new mock instance.
|
||||
func NewMockBackend(ctrl *gomock.Controller) *MockBackend {
|
||||
mock := &MockBackend{ctrl: ctrl}
|
||||
mock.recorder = &MockBackendMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use.
|
||||
func (m *MockBackend) EXPECT() *MockBackendMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// BlockByHash mocks base method.
|
||||
func (m *MockBackend) BlockByHash(arg0 context.Context, arg1 core.Hash) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "BlockByHash", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// BlockByHash indicates an expected call of BlockByHash.
|
||||
func (mr *MockBackendMockRecorder) BlockByHash(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BlockByHash", reflect.TypeOf((*MockBackend)(nil).BlockByHash), arg0, arg1)
|
||||
}
|
||||
|
||||
// BlockByNumber mocks base method.
|
||||
func (m *MockBackend) BlockByNumber(arg0 context.Context, arg1 int64) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "BlockByNumber", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// BlockByNumber indicates an expected call of BlockByNumber.
|
||||
func (mr *MockBackendMockRecorder) BlockByNumber(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BlockByNumber", reflect.TypeOf((*MockBackend)(nil).BlockByNumber), arg0, arg1)
|
||||
}
|
||||
|
||||
// BloomStatus mocks base method.
|
||||
func (m *MockBackend) BloomStatus() (uint64, uint64) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "BloomStatus")
|
||||
ret0, _ := ret[0].(uint64)
|
||||
ret1, _ := ret[1].(uint64)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// BloomStatus indicates an expected call of BloomStatus.
|
||||
func (mr *MockBackendMockRecorder) BloomStatus() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BloomStatus", reflect.TypeOf((*MockBackend)(nil).BloomStatus))
|
||||
}
|
||||
|
||||
// CurrentBlock mocks base method.
|
||||
func (m *MockBackend) CurrentBlock() []byte {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "CurrentBlock")
|
||||
ret0, _ := ret[0].([]byte)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// CurrentBlock indicates an expected call of CurrentBlock.
|
||||
func (mr *MockBackendMockRecorder) CurrentBlock() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentBlock", reflect.TypeOf((*MockBackend)(nil).CurrentBlock))
|
||||
}
|
||||
|
||||
// CurrentHeader mocks base method.
|
||||
func (m *MockBackend) CurrentHeader() []byte {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "CurrentHeader")
|
||||
ret0, _ := ret[0].([]byte)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// CurrentHeader indicates an expected call of CurrentHeader.
|
||||
func (mr *MockBackendMockRecorder) CurrentHeader() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentHeader", reflect.TypeOf((*MockBackend)(nil).CurrentHeader))
|
||||
}
|
||||
|
||||
// Downloader mocks base method.
|
||||
func (m *MockBackend) Downloader() core.Downloader {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Downloader")
|
||||
ret0, _ := ret[0].(core.Downloader)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Downloader indicates an expected call of Downloader.
|
||||
func (mr *MockBackendMockRecorder) Downloader() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Downloader", reflect.TypeOf((*MockBackend)(nil).Downloader))
|
||||
}
|
||||
|
||||
// ExtRPCEnabled mocks base method.
|
||||
func (m *MockBackend) ExtRPCEnabled() bool {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "ExtRPCEnabled")
|
||||
ret0, _ := ret[0].(bool)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// ExtRPCEnabled indicates an expected call of ExtRPCEnabled.
|
||||
func (mr *MockBackendMockRecorder) ExtRPCEnabled() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExtRPCEnabled", reflect.TypeOf((*MockBackend)(nil).ExtRPCEnabled))
|
||||
}
|
||||
|
||||
// GetAccountTrie mocks base method.
|
||||
func (m *MockBackend) GetAccountTrie(arg0 core.Hash, arg1 core.Address) (core.Trie, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetAccountTrie", arg0, arg1)
|
||||
ret0, _ := ret[0].(core.Trie)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetAccountTrie indicates an expected call of GetAccountTrie.
|
||||
func (mr *MockBackendMockRecorder) GetAccountTrie(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAccountTrie", reflect.TypeOf((*MockBackend)(nil).GetAccountTrie), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetContractCode mocks base method.
|
||||
func (m *MockBackend) GetContractCode(arg0 core.Hash) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetContractCode", arg0)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetContractCode indicates an expected call of GetContractCode.
|
||||
func (mr *MockBackendMockRecorder) GetContractCode(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetContractCode", reflect.TypeOf((*MockBackend)(nil).GetContractCode), arg0)
|
||||
}
|
||||
|
||||
// GetLogs mocks base method.
|
||||
func (m *MockBackend) GetLogs(arg0 context.Context, arg1 core.Hash) ([][]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetLogs", arg0, arg1)
|
||||
ret0, _ := ret[0].([][]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetLogs indicates an expected call of GetLogs.
|
||||
func (mr *MockBackendMockRecorder) GetLogs(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetLogs", reflect.TypeOf((*MockBackend)(nil).GetLogs), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetPoolNonce mocks base method.
|
||||
func (m *MockBackend) GetPoolNonce(arg0 context.Context, arg1 core.Address) (uint64, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetPoolNonce", arg0, arg1)
|
||||
ret0, _ := ret[0].(uint64)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetPoolNonce indicates an expected call of GetPoolNonce.
|
||||
func (mr *MockBackendMockRecorder) GetPoolNonce(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetPoolNonce", reflect.TypeOf((*MockBackend)(nil).GetPoolNonce), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetPoolTransaction mocks base method.
|
||||
func (m *MockBackend) GetPoolTransaction(arg0 core.Hash) []byte {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetPoolTransaction", arg0)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// GetPoolTransaction indicates an expected call of GetPoolTransaction.
|
||||
func (mr *MockBackendMockRecorder) GetPoolTransaction(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetPoolTransaction", reflect.TypeOf((*MockBackend)(nil).GetPoolTransaction), arg0)
|
||||
}
|
||||
|
||||
// GetPoolTransactions mocks base method.
|
||||
func (m *MockBackend) GetPoolTransactions() ([][]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetPoolTransactions")
|
||||
ret0, _ := ret[0].([][]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetPoolTransactions indicates an expected call of GetPoolTransactions.
|
||||
func (mr *MockBackendMockRecorder) GetPoolTransactions() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetPoolTransactions", reflect.TypeOf((*MockBackend)(nil).GetPoolTransactions))
|
||||
}
|
||||
|
||||
// GetReceipts mocks base method.
|
||||
func (m *MockBackend) GetReceipts(arg0 context.Context, arg1 core.Hash) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetReceipts", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetReceipts indicates an expected call of GetReceipts.
|
||||
func (mr *MockBackendMockRecorder) GetReceipts(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetReceipts", reflect.TypeOf((*MockBackend)(nil).GetReceipts), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetTd mocks base method.
|
||||
func (m *MockBackend) GetTd(arg0 context.Context, arg1 core.Hash) *big.Int {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetTd", arg0, arg1)
|
||||
ret0, _ := ret[0].(*big.Int)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// GetTd indicates an expected call of GetTd.
|
||||
func (mr *MockBackendMockRecorder) GetTd(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTd", reflect.TypeOf((*MockBackend)(nil).GetTd), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetTransaction mocks base method.
|
||||
func (m *MockBackend) GetTransaction(arg0 context.Context, arg1 core.Hash) ([]byte, core.Hash, uint64, uint64, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetTransaction", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(core.Hash)
|
||||
ret2, _ := ret[2].(uint64)
|
||||
ret3, _ := ret[3].(uint64)
|
||||
ret4, _ := ret[4].(error)
|
||||
return ret0, ret1, ret2, ret3, ret4
|
||||
}
|
||||
|
||||
// GetTransaction indicates an expected call of GetTransaction.
|
||||
func (mr *MockBackendMockRecorder) GetTransaction(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTransaction", reflect.TypeOf((*MockBackend)(nil).GetTransaction), arg0, arg1)
|
||||
}
|
||||
|
||||
// GetTrie mocks base method.
|
||||
func (m *MockBackend) GetTrie(arg0 core.Hash) (core.Trie, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetTrie", arg0)
|
||||
ret0, _ := ret[0].(core.Trie)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetTrie indicates an expected call of GetTrie.
|
||||
func (mr *MockBackendMockRecorder) GetTrie(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTrie", reflect.TypeOf((*MockBackend)(nil).GetTrie), arg0)
|
||||
}
|
||||
|
||||
// HeaderByHash mocks base method.
|
||||
func (m *MockBackend) HeaderByHash(arg0 context.Context, arg1 core.Hash) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "HeaderByHash", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// HeaderByHash indicates an expected call of HeaderByHash.
|
||||
func (mr *MockBackendMockRecorder) HeaderByHash(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HeaderByHash", reflect.TypeOf((*MockBackend)(nil).HeaderByHash), arg0, arg1)
|
||||
}
|
||||
|
||||
// HeaderByNumber mocks base method.
|
||||
func (m *MockBackend) HeaderByNumber(arg0 context.Context, arg1 int64) ([]byte, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "HeaderByNumber", arg0, arg1)
|
||||
ret0, _ := ret[0].([]byte)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// HeaderByNumber indicates an expected call of HeaderByNumber.
|
||||
func (mr *MockBackendMockRecorder) HeaderByNumber(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HeaderByNumber", reflect.TypeOf((*MockBackend)(nil).HeaderByNumber), arg0, arg1)
|
||||
}
|
||||
|
||||
// RPCGasCap mocks base method.
|
||||
func (m *MockBackend) RPCGasCap() uint64 {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "RPCGasCap")
|
||||
ret0, _ := ret[0].(uint64)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// RPCGasCap indicates an expected call of RPCGasCap.
|
||||
func (mr *MockBackendMockRecorder) RPCGasCap() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RPCGasCap", reflect.TypeOf((*MockBackend)(nil).RPCGasCap))
|
||||
}
|
||||
|
||||
// RPCTxFeeCap mocks base method.
|
||||
func (m *MockBackend) RPCTxFeeCap() float64 {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "RPCTxFeeCap")
|
||||
ret0, _ := ret[0].(float64)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// RPCTxFeeCap indicates an expected call of RPCTxFeeCap.
|
||||
func (mr *MockBackendMockRecorder) RPCTxFeeCap() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RPCTxFeeCap", reflect.TypeOf((*MockBackend)(nil).RPCTxFeeCap))
|
||||
}
|
||||
|
||||
// SendTx mocks base method.
|
||||
func (m *MockBackend) SendTx(arg0 context.Context, arg1 []byte) error {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SendTx", arg0, arg1)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SendTx indicates an expected call of SendTx.
|
||||
func (mr *MockBackendMockRecorder) SendTx(arg0, arg1 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendTx", reflect.TypeOf((*MockBackend)(nil).SendTx), arg0, arg1)
|
||||
}
|
||||
|
||||
// SetHead mocks base method.
|
||||
func (m *MockBackend) SetHead(arg0 uint64) {
|
||||
m.ctrl.T.Helper()
|
||||
m.ctrl.Call(m, "SetHead", arg0)
|
||||
}
|
||||
|
||||
// SetHead indicates an expected call of SetHead.
|
||||
func (mr *MockBackendMockRecorder) SetHead(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetHead", reflect.TypeOf((*MockBackend)(nil).SetHead), arg0)
|
||||
}
|
||||
|
||||
// Stats mocks base method.
|
||||
func (m *MockBackend) Stats() (int, int) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Stats")
|
||||
ret0, _ := ret[0].(int)
|
||||
ret1, _ := ret[1].(int)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// Stats indicates an expected call of Stats.
|
||||
func (mr *MockBackendMockRecorder) Stats() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stats", reflect.TypeOf((*MockBackend)(nil).Stats))
|
||||
}
|
||||
|
||||
// SubscribeChainEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeChainEvent(arg0 chan<- core.ChainEvent) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeChainEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeChainEvent indicates an expected call of SubscribeChainEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeChainEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeChainEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeChainEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribeChainHeadEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeChainHeadEvent(arg0 chan<- core.ChainHeadEvent) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeChainHeadEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeChainHeadEvent indicates an expected call of SubscribeChainHeadEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeChainHeadEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeChainHeadEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeChainHeadEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribeChainSideEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeChainSideEvent(arg0 chan<- core.ChainSideEvent) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeChainSideEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeChainSideEvent indicates an expected call of SubscribeChainSideEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeChainSideEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeChainSideEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeChainSideEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribeLogsEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeLogsEvent(arg0 chan<- [][]byte) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeLogsEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeLogsEvent indicates an expected call of SubscribeLogsEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeLogsEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeLogsEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeLogsEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribeNewTxsEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeNewTxsEvent(arg0 chan<- core.NewTxsEvent) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeNewTxsEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeNewTxsEvent indicates an expected call of SubscribeNewTxsEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeNewTxsEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeNewTxsEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeNewTxsEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribePendingLogsEvent mocks base method.
|
||||
func (m *MockBackend) SubscribePendingLogsEvent(arg0 chan<- [][]byte) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribePendingLogsEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribePendingLogsEvent indicates an expected call of SubscribePendingLogsEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribePendingLogsEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribePendingLogsEvent", reflect.TypeOf((*MockBackend)(nil).SubscribePendingLogsEvent), arg0)
|
||||
}
|
||||
|
||||
// SubscribeRemovedLogsEvent mocks base method.
|
||||
func (m *MockBackend) SubscribeRemovedLogsEvent(arg0 chan<- []byte) core.Subscription {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SubscribeRemovedLogsEvent", arg0)
|
||||
ret0, _ := ret[0].(core.Subscription)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// SubscribeRemovedLogsEvent indicates an expected call of SubscribeRemovedLogsEvent.
|
||||
func (mr *MockBackendMockRecorder) SubscribeRemovedLogsEvent(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeRemovedLogsEvent", reflect.TypeOf((*MockBackend)(nil).SubscribeRemovedLogsEvent), arg0)
|
||||
}
|
||||
|
||||
// SuggestGasTipCap mocks base method.
|
||||
func (m *MockBackend) SuggestGasTipCap(arg0 context.Context) (*big.Int, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "SuggestGasTipCap", arg0)
|
||||
ret0, _ := ret[0].(*big.Int)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// SuggestGasTipCap indicates an expected call of SuggestGasTipCap.
|
||||
func (mr *MockBackendMockRecorder) SuggestGasTipCap(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SuggestGasTipCap", reflect.TypeOf((*MockBackend)(nil).SuggestGasTipCap), arg0)
|
||||
}
|
||||
|
||||
// TxPoolContent mocks base method.
|
||||
func (m *MockBackend) TxPoolContent() (map[core.Address][][]byte, map[core.Address][][]byte) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "TxPoolContent")
|
||||
ret0, _ := ret[0].(map[core.Address][][]byte)
|
||||
ret1, _ := ret[1].(map[core.Address][][]byte)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// TxPoolContent indicates an expected call of TxPoolContent.
|
||||
func (mr *MockBackendMockRecorder) TxPoolContent() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TxPoolContent", reflect.TypeOf((*MockBackend)(nil).TxPoolContent))
|
||||
}
|
||||
|
||||
// UnprotectedAllowed mocks base method.
|
||||
func (m *MockBackend) UnprotectedAllowed() bool {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "UnprotectedAllowed")
|
||||
ret0, _ := ret[0].(bool)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// UnprotectedAllowed indicates an expected call of UnprotectedAllowed.
|
||||
func (mr *MockBackendMockRecorder) UnprotectedAllowed() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UnprotectedAllowed", reflect.TypeOf((*MockBackend)(nil).UnprotectedAllowed))
|
||||
}
|
||||
|
||||
// MockDownloader is a mock of Downloader interface.
|
||||
type MockDownloader struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockDownloaderMockRecorder
|
||||
}
|
||||
|
||||
// MockDownloaderMockRecorder is the mock recorder for MockDownloader.
|
||||
type MockDownloaderMockRecorder struct {
|
||||
mock *MockDownloader
|
||||
}
|
||||
|
||||
// NewMockDownloader creates a new mock instance.
|
||||
func NewMockDownloader(ctrl *gomock.Controller) *MockDownloader {
|
||||
mock := &MockDownloader{ctrl: ctrl}
|
||||
mock.recorder = &MockDownloaderMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use.
|
||||
func (m *MockDownloader) EXPECT() *MockDownloaderMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// Progress mocks base method.
|
||||
func (m *MockDownloader) Progress() core.Progress {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Progress")
|
||||
ret0, _ := ret[0].(core.Progress)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Progress indicates an expected call of Progress.
|
||||
func (mr *MockDownloaderMockRecorder) Progress() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Progress", reflect.TypeOf((*MockDownloader)(nil).Progress))
|
||||
}
|
13
test_helpers/util.go
Normal file
@ -0,0 +1,13 @@
|
||||
package test_helpers
|
||||
|
||||
import (
|
||||
geth_log "github.com/ethereum/go-ethereum/log"
|
||||
|
||||
"github.com/cerc-io/plugeth-statediff/utils/log"
|
||||
)
|
||||
|
||||
// The geth sync logs are noisy, it can be useful to silence them
|
||||
func SilenceLogs() {
|
||||
geth_log.Root().SetHandler(geth_log.DiscardHandler())
|
||||
log.TestLogger.SetLevel(2)
|
||||
}
|
@ -67,12 +67,6 @@ type IPLD struct {
|
||||
Content []byte
|
||||
}
|
||||
|
||||
// CodeAndCodeHash struct to hold codehash => code mappings
|
||||
type CodeAndCodeHash struct {
|
||||
Hash common.Hash
|
||||
Code []byte
|
||||
}
|
||||
|
||||
type StateNodeSink func(node StateLeafNode) error
|
||||
type StorageNodeSink func(node StorageLeafNode) error
|
||||
type IPLDSink func(IPLD) error
|
||||
|
@ -1,7 +1,6 @@
|
||||
package log
|
||||
|
||||
import (
|
||||
// geth_log "github.com/ethereum/go-ethereum/log"
|
||||
"github.com/inconshreveable/log15"
|
||||
"github.com/openrelayxyz/plugeth-utils/core"
|
||||
)
|
||||
@ -28,10 +27,6 @@ func Crit(m string, a ...interface{}) { DefaultLogger.Crit(m, a...) }
|
||||
func Error(m string, a ...interface{}) { DefaultLogger.Error(m, a...) }
|
||||
|
||||
func SetDefaultLogger(l core.Logger) {
|
||||
// gethlogger, ok := l.(geth_log.Logger)
|
||||
// if !ok {
|
||||
// panic("not a geth Logger")
|
||||
// }
|
||||
DefaultLogger = l
|
||||
}
|
||||
|
||||
|
@ -1,28 +0,0 @@
|
||||
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())
|
||||
}
|
nitpick: in v5
key
is a full CID, not a blockstore-prefixed multihash