Updates post review
* remove StreamCodeAndCodeHash * Makefile: local build target * clean up comments & unused
This commit is contained in:
parent
c96215f9be
commit
c4f16290c7
15
Dockerfile
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
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
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
|
||||
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
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"
|
||||
|
||||
// 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
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
@ -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),
|
||||
|
10
builder.go
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
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
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,7 +31,8 @@ import (
|
||||
)
|
||||
|
||||
var (
|
||||
pgConfig, _ = postgres.MakeConfig(postgres.TestConfig)
|
||||
pgConfig, _ = postgres.TestConfig.WithEnv()
|
||||
pgxConfig, _ = postgres.MakeConfig(pgConfig)
|
||||
ctx = context.Background()
|
||||
)
|
||||
|
||||
@ -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) {
|
||||
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
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
|
||||
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
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
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
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())
|
||||
}
|
Loading…
Reference in New Issue
Block a user