Add a flag to start processing statediff if we are caught up to the head of the chain

This commit is contained in:
Abdul Rabbani 2022-03-10 13:04:25 -05:00
parent 53c1322cb4
commit e409059477
9 changed files with 510 additions and 39 deletions

View File

@ -258,8 +258,9 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) {
Context: context.Background(),
EnableWriteLoop: ctx.GlobalBool(utils.StateDiffWritingFlag.Name),
NumWorkers: ctx.GlobalUint(utils.StateDiffWorkersFlag.Name),
WaitForSync: ctx.GlobalBool(utils.StateDiffWaitForSync.Name),
}
utils.RegisterStateDiffService(stack, eth, &cfg.Eth, p)
utils.RegisterStateDiffService(stack, eth, &cfg.Eth, p, backend)
}
// Configure GraphQL if requested

View File

@ -176,6 +176,7 @@ var (
utils.StateDiffWritingFlag,
utils.StateDiffWorkersFlag,
utils.StateDiffFilePath,
utils.StateDiffWaitForSync,
configFileFlag,
}

View File

@ -246,6 +246,7 @@ var AppHelpFlagGroups = []flags.FlagGroup{
utils.StateDiffWritingFlag,
utils.StateDiffWorkersFlag,
utils.StateDiffFilePath,
utils.StateDiffWaitForSync,
},
},
{

View File

@ -881,6 +881,10 @@ var (
Usage: "Number of concurrent workers to use during statediff processing (default 1)",
Value: 1,
}
StateDiffWaitForSync = cli.BoolFlag{
Name: "statediff.waitforsync",
Usage: "Should the statediff service wait for geth to catch up to the head of the chain?",
}
)
// MakeDataDir retrieves the currently requested data directory, terminating
@ -1865,8 +1869,8 @@ func RegisterGraphQLService(stack *node.Node, backend ethapi.Backend, cfg node.C
}
// RegisterStateDiffService configures and registers a service to stream state diff data over RPC
func RegisterStateDiffService(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params statediff.Config) {
if err := statediff.New(stack, ethServ, cfg, params); err != nil {
func RegisterStateDiffService(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params statediff.Config, backend ethapi.Backend) {
if err := statediff.New(stack, ethServ, cfg, params, backend); err != nil {
Fatalf("Failed to register the Statediff service: %v", err)
}
}

View File

@ -6,11 +6,12 @@ either relaying the state objects to RPC subscribers or writing them directly to
It also exposes RPC endpoints for fetching or writing to Postgres the state diff at a specific block height
or for a specific block hash, this operates on historical block and state data and so depends on a complete state archive.
Data is emitted in this differential format in order to make it feasible to IPLD-ize and index the *entire* Ethereum state
Data is emitted in this differential format in order to make it feasible to IPLD-ize and index the _entire_ Ethereum state
(including intermediate state and storage trie nodes). If this state diff process is ran continuously from genesis,
the entire state at any block can be materialized from the cumulative differentials up to that point.
## Statediff object
A state diff `StateObject` is the collection of all the state and storage trie nodes that have been updated in a given block.
For convenience, we also associate these nodes with the block number and hash, and optionally the set of code hashes and code for any
contracts deployed in this block.
@ -52,6 +53,7 @@ type CodeAndCodeHash struct {
Code []byte `json:"code"`
}
```
These objects are packed into a `Payload` structure which can additionally associate the `StateObject`
with the block (header, uncles, and transactions), receipts, and total difficulty.
This `Payload` encapsulates all of the differential data at a given block, and allows us to index the entire Ethereum data structure
@ -71,9 +73,11 @@ type Payload struct {
```
## Usage
This state diffing service runs as an auxiliary service concurrent to the regular syncing process of the geth node.
### CLI configuration
This service introduces a CLI flag namespace `statediff`
`--statediff` flag is used to turn on the service
@ -100,9 +104,7 @@ This service introduces a CLI flag namespace `statediff`
The service can only operate in full sync mode (`--syncmode=full`), but only the historical RPC endpoints require an archive node (`--gcmode=archive`)
e.g.
`
./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db.type=postgres --statediff.db.driver=sqlx --statediff.db.host=localhost --statediff.db.port=5432 --statediff.db.name=vulcanize_test --statediff.db.user=postgres --statediff.db.nodeid=nodeid --statediff.db.clientname=clientname
`
`./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db.type=postgres --statediff.db.driver=sqlx --statediff.db.host=localhost --statediff.db.port=5432 --statediff.db.name=vulcanize_test --statediff.db.user=postgres --statediff.db.nodeid=nodeid --statediff.db.clientname=clientname`
When operating in `--statediff.db.type=file` mode, the service will write SQL statements out to the file designated by
`--statediff.file.path`. Please note that it writes out SQL statements with all `ON CONFLICT` constraint checks dropped.
@ -112,6 +114,7 @@ de-duplicate using unix tools (`sort statediff.sql | uniq` or `sort -u statediff
back afterwards.
### RPC endpoints
The state diffing service exposes both a WS subscription endpoint, and a number of HTTP unary endpoints.
Each of these endpoints requires a set of parameters provided by the caller
@ -137,6 +140,7 @@ contracts deployed in this block; whether to limit the diffing process to a list
whether to limit the diffing process to a list of specific storage slot keys.
#### Subscription endpoint
A websocket supporting RPC endpoint is exposed for subscribing to state diff `StateObjects` that come off the head of the chain while the geth node syncs.
```go
@ -182,7 +186,9 @@ for {
```
#### Unary endpoints
The service also exposes unary RPC endpoints for retrieving the state diff `StateObject` for a specific block height/hash.
```go
// StateDiffAt returns a state diff payload at the specific blockheight
StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error)
@ -195,12 +201,14 @@ To expose this endpoint the node needs to have the HTTP server turned on (`--htt
and the `statediff` namespace exposed (`--http.api=statediff`).
### Direct indexing into Postgres
If `--statediff.writing` is set, the service will convert the state diff `StateObject` data into IPLD objects, persist them directly to Postgres,
and generate secondary indexes around the IPLD data.
The schema and migrations for this Postgres database are provided in `statediff/db/`.
#### Postgres setup
We use [pressly/goose](https://github.com/pressly/goose) as our Postgres migration manager.
You can also load the Postgres schema directly into a database using
@ -209,6 +217,7 @@ You can also load the Postgres schema directly into a database using
This will only work on a version 12.4 Postgres database.
#### Schema overview
Our Postgres schemas are built around a single IPFS backing Postgres IPLD blockstore table (`public.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).
@ -227,6 +236,7 @@ table contains a `state_id` foreign key which references the `id` for the `state
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 updated (diffed).
### Optimization
On mainnet this process is extremely IO intensive and requires significant resources to allow it to keep up with the head of the chain.
The state diff processing time for a specific block is dependent on the number and complexity of the state changes that occur in a block and
the number of updated state nodes that are available in the in-memory cache vs must be retrieved from disc.
@ -236,6 +246,7 @@ This can be done by increasing the overall `--cache` allocation and/or by increa
usage with `--cache.trie`.
## Versioning, Branches, Rebasing, and Releasing
Internal tagged releases are maintained for building the latest version of statediffing geth or using it as a go mod dependency.
When a new core go-ethereum version is released, statediffing geth is rebased onto and adjusted to work with the new tag.
@ -244,18 +255,20 @@ need to be able to squash our work before performing a rebase. To this end we re
the full incremental history.
### Versioning
Versioning for of statediffing geth follows the below format:
`{Root Version}-statediff-{Statediff Version}`
Example: `v1.10.16-statediff-3.0.2`
Where "root version" is the version of the tagged release from the core go-ethereum repository that our release is rebased on top of
and "statediff version" is the version tracking the state of the statediffing service code.
E.g. the version at the time of writing this is v1.10.3-statediff-0.0.23, v0.0.23 of the statediffing code rebased on top of the v1.10.3 core tag.
- The first section, `v1.10.16`, corresponds to the release of the root branch this version is rebased onto (e.g., [](https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16)[https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16](https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16))
- The second section, `3.0.2`, corresponds to the version of our statediffing code. The major version here (3) should always correspond with the major version of the `ipld-eth-db` schema version it works with (e.g., [](https://github.com/vulcanize/ipld-eth-db/releases/tag/v3.0.6)[https://github.com/vulcanize/ipld-eth-db/releases/tag/v3.0.6](https://github.com/vulcanize/ipld-eth-db/releases/tag/v3.0.6)); it is only bumped when we bump the major version of the schema.
- The major version of the schema is only bumped when a breaking change is made to the schema.
- The minor version is bumped when a new feature is added, or a fix is performed that breaks or updates the statediffing API or CLI in some way.
- The patch version is bumped whenever minor fixes/patches/features are done that dont change/break API/CLI compatibility.
- We are very strict about the first section and the major version of the statediffing code, but some discretion is required when deciding to bump minor versus patch version of the statediffing code.
The statediff version is included in the `VersionMeta` in params/version.go
### Branches
We maintain two official kinds of branches:
Major Branch: `{Root Version}-statediff`
@ -271,7 +284,9 @@ If a developer is unsure what version their patch should affect, they should rem
they can open a PR against the targeted root branch and be directed to the appropriate feature version and branch.
### Rebasing
When a new root tagged release comes out we rebase our statediffing code on top of the new tag using the following process:
1. Checkout a new major branch for the tag from the current major branch
2. On the new major branch, squash all our commits since the last major rebase
3. On the new major branch, perform the rebase against the new tag

View File

@ -35,6 +35,8 @@ type Config struct {
EnableWriteLoop bool
// Size of the worker pool
NumWorkers uint
// Should the statediff service wait until geth has synced to the head of the blockchain?
WaitForSync bool
// Context
Context context.Context
}

View File

@ -33,6 +33,7 @@ import (
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/internal/ethapi"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/node"
@ -118,11 +119,15 @@ type Service struct {
SubscriptionTypes map[common.Hash]Params
// Cache the last block so that we can avoid having to lookup the next block's parent
BlockCache BlockCache
// The publicBackendAPI which provides useful information about the current state
BackendAPI ethapi.Backend
// Should the statediff service wait for geth to sync to head?
WaitforSync bool
// Whether or not we have any subscribers; only if we do, do we processes state diffs
subscribers int32
// Interface for publishing statediffs as PG-IPLD objects
indexer interfaces.StateDiffIndexer
// Whether to enable writing state diffs directly to track blochain head
// Whether to enable writing state diffs directly to track blockchain head.
enableWriteLoop bool
// Size of the worker pool
numWorkers uint
@ -146,7 +151,7 @@ func NewBlockCache(max uint) BlockCache {
// New creates a new statediff.Service
// func New(stack *node.Node, ethServ *eth.Ethereum, dbParams *DBParams, enableWriteLoop bool) error {
func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params Config) error {
func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params Config, backend ethapi.Backend) error {
blockChain := ethServ.BlockChain()
var indexer interfaces.StateDiffIndexer
quitCh := make(chan bool)
@ -177,6 +182,8 @@ func New(stack *node.Node, ethServ *eth.Ethereum, cfg *ethconfig.Config, params
Subscriptions: make(map[common.Hash]map[rpc.ID]Subscription),
SubscriptionTypes: make(map[common.Hash]Params),
BlockCache: NewBlockCache(workers),
BackendAPI: backend,
WaitforSync: params.WaitForSync,
indexer: indexer,
enableWriteLoop: params.EnableWriteLoop,
numWorkers: workers,
@ -528,10 +535,48 @@ func (sds *Service) Unsubscribe(id rpc.ID) error {
return nil
}
// This function will check the status of geth syncing.
// It will return false if geth has finished syncing.
// It will return a non false value if geth is not done syncing.
func (sds *Service) GetSyncStatus(pubEthAPI *ethapi.PublicEthereumAPI) (interface{}, error) {
syncStatus, err := pubEthAPI.Syncing()
if err != nil {
return nil, err
}
return syncStatus, err
}
// This function calls GetSyncStatus to check if we have caught up to head.
// It will keep looking and checking if we have caught up to head.
// It will only complete if we catch up to head, otherwise it will keep looping forever.
func (sds *Service) WaitForSync() error {
log.Info("We are going to wait for geth to sync to head!")
// Has the geth node synced to head?
Synced := false
pubEthAPI := ethapi.NewPublicEthereumAPI(sds.BackendAPI)
for !Synced {
syncStatus, err := sds.GetSyncStatus(pubEthAPI)
if err != nil {
return err
}
if syncStatus == false {
log.Info("Geth has caught up to the head of the chain")
Synced = true
}
}
return nil
}
// Start is used to begin the service
func (sds *Service) Start() error {
log.Info("Starting statediff service")
if sds.WaitforSync {
log.Info("Statediff service will wait until geth has caught up to the head of the chain.")
sds.WaitForSync()
log.Info("Continuing with startdiff start process")
}
chainEventCh := make(chan core.ChainEvent, chainEventChanSize)
go sds.Loop(chainEventCh)

View File

@ -23,18 +23,18 @@ import (
"reflect"
"sync"
"testing"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/internal/ethapi"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/rpc"
statediff "github.com/ethereum/go-ethereum/statediff"
"github.com/ethereum/go-ethereum/statediff/test_helpers/mocks"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
)
func TestServiceLoop(t *testing.T) {
@ -291,3 +291,151 @@ func testErrorInStateDiffAt(t *testing.T) {
t.Logf("Actual state diff payload does not equal expected.\nactual:%+v\nexpected: %+v", expectedStateDiffPayload, stateDiffPayload)
}
}
func TestWaitForSync(t *testing.T) {
testWaitForSync(t)
testGetSyncStatus(t)
}
// This function will create a backend and service object which includes a generic Backend
func createServiceWithMockBackend(curBlock uint64, highestBlock uint64) (*mocks.Backend, *statediff.Service) {
builder := mocks.Builder{}
blockChain := mocks.BlockChain{}
backend := mocks.Backend{
StartingBlock: 1,
CurrBlock: curBlock,
HighestBlock: highestBlock,
SyncedAccounts: 5,
SyncedAccountBytes: 5,
SyncedBytecodes: 5,
SyncedBytecodeBytes: 5,
SyncedStorage: 5,
SyncedStorageBytes: 5,
HealedTrienodes: 5,
HealedTrienodeBytes: 5,
HealedBytecodes: 5,
HealedBytecodeBytes: 5,
HealingTrienodes: 5,
HealingBytecode: 5,
}
service := &statediff.Service{
Mutex: sync.Mutex{},
Builder: &builder,
BlockChain: &blockChain,
QuitChan: make(chan bool),
Subscriptions: make(map[common.Hash]map[rpc.ID]statediff.Subscription),
SubscriptionTypes: make(map[common.Hash]statediff.Params),
BlockCache: statediff.NewBlockCache(1),
BackendAPI: &backend,
WaitforSync: true,
}
return &backend, service
}
// This function will test to make sure that the state diff waits
// until the blockchain has caught up to head!
func testWaitForSync(t *testing.T) {
t.Log("Starting Sync")
_, service := createServiceWithMockBackend(10, 10)
err := service.WaitForSync()
if err != nil {
t.Fatal("Sync Failed")
}
t.Log("Sync Complete")
}
// This test will run the WaitForSync() at the start of the execusion
// It will then incrementally increase the currentBlock to match the highestBlock
// At each interval it will run the GetSyncStatus to ensure that the return value is not false.
// It will also check to make sure that the WaitForSync() function has not completed!
func testGetSyncStatus(t *testing.T) {
t.Log("Starting Get Sync Status Test")
var highestBlock uint64 = 5
// Create a backend and a service
// the backend is lagging behind the sync.
backend, service := createServiceWithMockBackend(0, highestBlock)
checkSyncComplete := make(chan int, 1)
go func() {
// Start the sync function which will wait for the sync
// Once the sync is complete add a value to the checkSyncComplet channel
t.Log("Starting Sync")
err := service.WaitForSync()
if err != nil {
t.Error("Sync Failed")
checkSyncComplete <- 1
}
t.Log("We have finally synced!")
checkSyncComplete <- 0
}()
tables := []struct {
currentBlock uint64
highestBlock uint64
}{
{1, highestBlock},
{2, highestBlock},
{3, highestBlock},
{4, highestBlock},
{5, highestBlock},
}
time.Sleep(2 * time.Second)
for _, table := range tables {
// Iterate over each block
// Once the highest block reaches the current block the sync should complete
// Update the backend current block value
t.Log("Updating Current Block to: ", table.currentBlock)
backend.CurrBlock = table.currentBlock
pubEthAPI := ethapi.NewPublicEthereumAPI(service.BackendAPI)
syncStatus, err := service.GetSyncStatus(pubEthAPI)
if err != nil {
t.Fatal("Sync Failed")
}
time.Sleep(1 * time.Second)
// Make sure if syncStatus is false that WaitForSync has completed!
if syncStatus == false && len(checkSyncComplete) == 0 {
t.Error("Sync is complete but WaitForSync is not")
}
if syncStatus != false && len(checkSyncComplete) == 1 {
t.Error("Sync is not complete but WaitForSync is")
}
// Make sure sync hasn't completed and that the checkSyncComplete channel is empty
if syncStatus != false && len(checkSyncComplete) == 0 {
continue
}
// This code will only be run if the sync is complete and the WaitForSync function is complete
//t.Log("Backend: ", backend)
//t.Log("Sync Status: ", syncStatus)
// If syncstatus is complete, make sure that the blocks match
if syncStatus == false && table.currentBlock != table.highestBlock {
t.Errorf("syncStatus indicated sync was complete even when current block, %d, and highest block %d aren't equal",
table.currentBlock, table.highestBlock)
}
// Make sure that WaitForSync completed once the current block caught up to head!
if len(checkSyncComplete) == 1 {
checkSyncCompleteVal := <-checkSyncComplete
if checkSyncCompleteVal != 0 {
t.Errorf("syncStatus indicated sync was complete but the checkSyncComplete has a value of %d",
checkSyncCompleteVal)
} else {
t.Log("Test Passed!")
}
} else {
t.Error("checkSyncComplete is empty: ", len(checkSyncComplete))
}
}
}

View File

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