Updates post review

* remove StreamCodeAndCodeHash
* Makefile: local build target
* clean up comments & unused
This commit is contained in:
Roy Crihfield 2023-06-24 13:00:21 +08:00
parent c96215f9be
commit c4f16290c7
30 changed files with 672 additions and 260 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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 &params.ChainConfig{
ChainID: cc.ChainID,

27
api.go
View File

@ -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

View File

@ -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),

View File

@ -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
View File

@ -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
View File

@ -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=

View File

@ -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()

View File

@ -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)

View 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()

View File

@ -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)
}

View File

@ -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)

View File

@ -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")
}

View File

@ -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")
}

View File

@ -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 {

View File

@ -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)
}

View File

@ -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 {

View File

@ -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())

View File

@ -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)
// }
}

View File

@ -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

View File

@ -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 {

View File

@ -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) {

View File

@ -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

View 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
View 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)
}

View File

@ -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

View File

@ -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
}

View File

@ -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())
}