Separated PendingBlock behaviour to be chain agnostic (#7859)

Instead of getting the pending block with latest payload id, we just
store the latest block built and serve it in remote backend
This commit is contained in:
Giulio rebuffo 2023-07-10 19:22:03 +02:00 committed by GitHub
parent 7d4c678a90
commit 6272559fb7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 163 additions and 120 deletions

View File

@ -13,6 +13,7 @@ import (
"github.com/c2h5oh/datasize"
"github.com/ledgerwatch/erigon/core/rawdb/blockio"
"github.com/ledgerwatch/erigon/node/nodecfg"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/snapshotsync/freezeblocks"
"github.com/ledgerwatch/log/v3"
"github.com/ledgerwatch/secp256k1"
@ -1474,13 +1475,14 @@ func newSync(ctx context.Context, db kv.RwDB, miningConfig *params.MiningConfig,
<-ctx.Done()
close(miningCancel)
}()
miningSync := stagedsync.New(
stagedsync.MiningStages(ctx,
stagedsync.StageMiningCreateBlockCfg(db, miner, *chainConfig, engine, nil, nil, dirs.Tmp, blockReader),
stagedsync.StageMiningExecCfg(db, miner, events, *chainConfig, engine, &vm.Config{}, dirs.Tmp, nil, 0, nil, nil, blockReader),
stagedsync.StageHashStateCfg(db, dirs, historyV3),
stagedsync.StageTrieCfg(db, false, true, false, dirs.Tmp, blockReader, nil, historyV3, agg),
stagedsync.StageMiningFinishCfg(db, *chainConfig, engine, miner, miningCancel, blockReader),
stagedsync.StageMiningFinishCfg(db, *chainConfig, engine, miner, miningCancel, blockReader, builder.NewLatestBlockBuiltStore()),
),
stagedsync.MiningUnwindOrder,
stagedsync.MiningPruneOrder,

View File

@ -30,6 +30,7 @@ import (
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/ethdb/privateapi"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/jsonrpc/contracts"
"github.com/ledgerwatch/erigon/turbo/stages"
"github.com/ledgerwatch/log/v3"
@ -295,7 +296,7 @@ func CreateTestGrpcConn(t *testing.T, m *stages.MockSentry) (context.Context, *g
server := grpc.NewServer()
remote.RegisterETHBACKENDServer(server, privateapi.NewEthBackendServer(ctx, nil, m.DB, m.Notifications.Events,
m.BlockReader, log.New()))
m.BlockReader, log.New(), builder.NewLatestBlockBuiltStore()))
txpool.RegisterTxpoolServer(server, m.TxPoolGrpcServer)
txpool.RegisterMiningServer(server, privateapi.NewMiningServer(ctx, &IsMiningMock{}, ethashApi, m.Log))
listener := bufconn.Listen(1024 * 1024)

View File

@ -1,15 +1,10 @@
package rpcservices
import (
"bytes"
"context"
"fmt"
"github.com/ledgerwatch/erigon-lib/gointerfaces/engine"
types2 "github.com/ledgerwatch/erigon-lib/gointerfaces/types"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/rlp"
"google.golang.org/protobuf/types/known/emptypb"
)
type EngineBackend struct {
@ -41,21 +36,3 @@ func (back *EngineBackend) EngineGetPayloadBodiesByHashV1(ctx context.Context, r
func (back *EngineBackend) EngineGetPayloadBodiesByRangeV1(ctx context.Context, request *engine.EngineGetPayloadBodiesByRangeV1Request) (*engine.EngineGetPayloadBodiesV1Response, error) {
return back.server.EngineGetPayloadBodiesByRangeV1(ctx, request)
}
func (back *EngineBackend) PendingBlock(ctx context.Context) (*types.Block, error) {
blockRlp, err := back.server.PendingBlock(ctx, &emptypb.Empty{})
if err != nil {
return nil, fmt.Errorf("ETHBACKENDClient.PendingBlock() error: %w", err)
}
if blockRlp == nil {
return nil, nil
}
var block types.Block
err = rlp.Decode(bytes.NewReader(blockRlp.BlockRlp), &block)
if err != nil {
return nil, fmt.Errorf("decoding block from %x: %w", blockRlp.BlockRlp, err)
}
return &block, nil
}

View File

@ -1,6 +1,7 @@
package rpcservices
import (
"bytes"
"context"
"encoding/json"
"errors"
@ -138,6 +139,24 @@ func (back *RemoteBackend) NetPeerCount(ctx context.Context) (uint64, error) {
return res.Count, nil
}
func (back *RemoteBackend) PendingBlock(ctx context.Context) (*types.Block, error) {
blockRlp, err := back.remoteEthBackend.PendingBlock(ctx, &emptypb.Empty{})
if err != nil {
return nil, fmt.Errorf("ETHBACKENDClient.PendingBlock() error: %w", err)
}
if blockRlp == nil {
return nil, nil
}
var block types.Block
err = rlp.Decode(bytes.NewReader(blockRlp.BlockRlp), &block)
if err != nil {
return nil, fmt.Errorf("decoding block from %x: %w", blockRlp.BlockRlp, err)
}
return &block, nil
}
func (back *RemoteBackend) ProtocolVersion(ctx context.Context) (uint64, error) {
res, err := back.remoteEthBackend.ProtocolVersion(ctx, &remote.ProtocolVersionRequest{})
if err != nil {

View File

@ -38,6 +38,7 @@ import (
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/rawdb/blockio"
"github.com/ledgerwatch/erigon/ethdb/prune"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/engineapi"
"github.com/ledgerwatch/erigon/turbo/engineapi/engine_helpers"
"github.com/ledgerwatch/erigon/turbo/jsonrpc"
@ -213,6 +214,7 @@ func New(stack *node.Node, config *ethconfig.Config, logger log.Logger) (*Ethere
if err != nil {
return nil, err
}
latestBlockBuiltStore := builder.NewLatestBlockBuiltStore()
if err := chainKv.Update(context.Background(), func(tx kv.RwTx) error {
if err = stagedsync.UpdateMetrics(tx); err != nil {
@ -520,7 +522,7 @@ func New(stack *node.Node, config *ethconfig.Config, logger log.Logger) (*Ethere
stagedsync.StageMiningExecCfg(backend.chainDB, miner, backend.notifications.Events, *backend.chainConfig, backend.engine, &vm.Config{}, tmpdir, nil, 0, backend.txPool, backend.txPoolDB, blockReader),
stagedsync.StageHashStateCfg(backend.chainDB, dirs, config.HistoryV3),
stagedsync.StageTrieCfg(backend.chainDB, false, true, true, tmpdir, blockReader, nil, config.HistoryV3, backend.agg),
stagedsync.StageMiningFinishCfg(backend.chainDB, *backend.chainConfig, backend.engine, miner, backend.miningSealingQuit, backend.blockReader),
stagedsync.StageMiningFinishCfg(backend.chainDB, *backend.chainConfig, backend.engine, miner, backend.miningSealingQuit, backend.blockReader, latestBlockBuiltStore),
), stagedsync.MiningUnwindOrder, stagedsync.MiningPruneOrder,
logger)
@ -539,7 +541,7 @@ func New(stack *node.Node, config *ethconfig.Config, logger log.Logger) (*Ethere
stagedsync.StageMiningExecCfg(backend.chainDB, miningStatePos, backend.notifications.Events, *backend.chainConfig, backend.engine, &vm.Config{}, tmpdir, interrupt, param.PayloadId, backend.txPool, backend.txPoolDB, blockReader),
stagedsync.StageHashStateCfg(backend.chainDB, dirs, config.HistoryV3),
stagedsync.StageTrieCfg(backend.chainDB, false, true, true, tmpdir, blockReader, nil, config.HistoryV3, backend.agg),
stagedsync.StageMiningFinishCfg(backend.chainDB, *backend.chainConfig, backend.engine, miningStatePos, backend.miningSealingQuit, backend.blockReader),
stagedsync.StageMiningFinishCfg(backend.chainDB, *backend.chainConfig, backend.engine, miningStatePos, backend.miningSealingQuit, backend.blockReader, latestBlockBuiltStore),
), stagedsync.MiningUnwindOrder, stagedsync.MiningPruneOrder,
logger)
// We start the mining step
@ -551,7 +553,7 @@ func New(stack *node.Node, config *ethconfig.Config, logger log.Logger) (*Ethere
}
// Initialize ethbackend
ethBackendRPC := privateapi.NewEthBackendServer(ctx, backend, backend.chainDB, backend.notifications.Events, blockReader, logger)
ethBackendRPC := privateapi.NewEthBackendServer(ctx, backend, backend.chainDB, backend.notifications.Events, blockReader, logger, latestBlockBuiltStore)
// intiialize engine backend
engineSrv := engineapi.NewEngineServer(ctx, logger, chainConfig, assembleBlockPOS, backend.chainDB, blockReader, backend.sentriesClient.Hd, config.Miner.EnabledPOS)
backend.engineBackendRPC = direct.NewEngineClient(engineSrv)

View File

@ -5,6 +5,7 @@ import (
"github.com/ledgerwatch/erigon-lib/chain"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/services"
"github.com/ledgerwatch/log/v3"
@ -13,12 +14,13 @@ import (
)
type MiningFinishCfg struct {
db kv.RwDB
chainConfig chain.Config
engine consensus.Engine
sealCancel chan struct{}
miningState MiningState
blockReader services.FullBlockReader
db kv.RwDB
chainConfig chain.Config
engine consensus.Engine
sealCancel chan struct{}
miningState MiningState
blockReader services.FullBlockReader
latestBlockBuiltStore *builder.LatestBlockBuiltStore
}
func StageMiningFinishCfg(
@ -28,14 +30,16 @@ func StageMiningFinishCfg(
miningState MiningState,
sealCancel chan struct{},
blockReader services.FullBlockReader,
latestBlockBuiltStore *builder.LatestBlockBuiltStore,
) MiningFinishCfg {
return MiningFinishCfg{
db: db,
chainConfig: chainConfig,
engine: engine,
miningState: miningState,
sealCancel: sealCancel,
blockReader: blockReader,
db: db,
chainConfig: chainConfig,
engine: engine,
miningState: miningState,
sealCancel: sealCancel,
blockReader: blockReader,
latestBlockBuiltStore: latestBlockBuiltStore,
}
}
@ -59,7 +63,7 @@ func SpawnMiningFinishStage(s *StageState, tx kv.RwTx, cfg MiningFinishCfg, quit
// return nil
//}
//prev = sealHash
cfg.latestBlockBuiltStore.AddBlockBuilt(block)
if cfg.miningState.MiningResultPOSCh != nil {
cfg.miningState.MiningResultPOSCh <- blockWithReceipts
return nil

View File

@ -18,6 +18,7 @@ import (
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/services"
"github.com/ledgerwatch/erigon/turbo/shards"
)
@ -35,11 +36,12 @@ var EthBackendAPIVersion = &types2.VersionReply{Major: 3, Minor: 3, Patch: 0}
type EthBackendServer struct {
remote.UnimplementedETHBACKENDServer // must be embedded to have forward compatible implementations.
ctx context.Context
eth EthBackend
events *shards.Events
db kv.RoDB
blockReader services.FullBlockReader
ctx context.Context
eth EthBackend
events *shards.Events
db kv.RoDB
blockReader services.FullBlockReader
latestBlockBuiltStore *builder.LatestBlockBuiltStore
logsFilter *LogsFilterAggregator
logger log.Logger
@ -54,11 +56,12 @@ type EthBackend interface {
}
func NewEthBackendServer(ctx context.Context, eth EthBackend, db kv.RwDB, events *shards.Events, blockReader services.FullBlockReader,
logger log.Logger,
logger log.Logger, latestBlockBuiltStore *builder.LatestBlockBuiltStore,
) *EthBackendServer {
s := &EthBackendServer{ctx: ctx, eth: eth, events: events, db: db, blockReader: blockReader,
logsFilter: NewLogsFilterAggregator(events),
logger: logger,
logsFilter: NewLogsFilterAggregator(events),
logger: logger,
latestBlockBuiltStore: latestBlockBuiltStore,
}
ch, clean := s.events.AddLogsSubscription()
@ -92,6 +95,20 @@ func (s *EthBackendServer) Version(context.Context, *emptypb.Empty) (*types2.Ver
return EthBackendAPIVersion, nil
}
func (s *EthBackendServer) PendingBlock(_ context.Context, _ *emptypb.Empty) (*remote.PendingBlockReply, error) {
pendingBlock := s.latestBlockBuiltStore.BlockBuilt()
if pendingBlock == nil {
return nil, nil
}
blockRlp, err := rlp.EncodeToBytes(pendingBlock)
if err != nil {
return nil, err
}
return &remote.PendingBlockReply{BlockRlp: blockRlp}, nil
}
func (s *EthBackendServer) Etherbase(_ context.Context, _ *remote.EtherbaseRequest) (*remote.EtherbaseReply, error) {
out := &remote.EtherbaseReply{Address: gointerfaces.ConvertAddressToH160(libcommon.Address{})}

2
go.mod
View File

@ -3,7 +3,7 @@ module github.com/ledgerwatch/erigon
go 1.19
require (
github.com/ledgerwatch/erigon-lib v0.0.0-20230706083614-07d67970f51c
github.com/ledgerwatch/erigon-lib v0.0.0-20230708202014-b6ac7e23298f
github.com/ledgerwatch/erigon-snapshot v1.2.1-0.20230622075030-1d69651854c2
github.com/ledgerwatch/log/v3 v3.8.0
github.com/ledgerwatch/secp256k1 v1.0.0

4
go.sum
View File

@ -415,8 +415,8 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/kylelemons/godebug v0.0.0-20170224010052-a616ab194758 h1:0D5M2HQSGD3PYPwICLl+/9oulQauOuETfgFvhBDffs0=
github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c=
github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8=
github.com/ledgerwatch/erigon-lib v0.0.0-20230706083614-07d67970f51c h1:KQzvisw/sRpi630ZOw3XvRE9T+rJrWfHUKGCQyN4JVU=
github.com/ledgerwatch/erigon-lib v0.0.0-20230706083614-07d67970f51c/go.mod h1:00FZbkGJTAiS8CZyhvipZ4vkuxldc+G9Mh+BGo89Hxk=
github.com/ledgerwatch/erigon-lib v0.0.0-20230708202014-b6ac7e23298f h1:1NsUi6Oy9c2QUHXsGfHHJ/9CjR4xZ4RONGOsDFx/05w=
github.com/ledgerwatch/erigon-lib v0.0.0-20230708202014-b6ac7e23298f/go.mod h1:uNyN+0RoGYhsp5zTuPW9ENB+3fXL8gS1H/p0sKIegpA=
github.com/ledgerwatch/erigon-snapshot v1.2.1-0.20230622075030-1d69651854c2 h1:Ls2itRGHMOr2PbHRDA4g1HH8HQdwfJhRVfMPEaLQe94=
github.com/ledgerwatch/erigon-snapshot v1.2.1-0.20230622075030-1d69651854c2/go.mod h1:3AuPxZc85jkehh/HA9h8gabv5MSi3kb/ddtzBsTVJFo=
github.com/ledgerwatch/log/v3 v3.8.0 h1:gCpp7uGtIerEz1jKVPeDnbIopFPud9ZnCpBLlLBGqPU=

View File

@ -0,0 +1,29 @@
package builder
import (
"sync"
"github.com/ledgerwatch/erigon/core/types"
)
type LatestBlockBuiltStore struct {
block *types.Block
lock sync.Mutex
}
func NewLatestBlockBuiltStore() *LatestBlockBuiltStore {
return &LatestBlockBuiltStore{}
}
func (s *LatestBlockBuiltStore) AddBlockBuilt(block *types.Block) {
s.lock.Lock()
defer s.lock.Unlock()
s.block = block.Copy()
}
func (s *LatestBlockBuiltStore) BlockBuilt() *types.Block {
s.lock.Lock()
defer s.lock.Unlock()
return s.block.Copy()
}

View File

@ -0,0 +1,15 @@
package builder
import (
"testing"
"github.com/ledgerwatch/erigon/core/types"
"github.com/stretchr/testify/assert"
)
func TestLatestBlockBuilt(t *testing.T) {
s := NewLatestBlockBuiltStore()
b := types.NewBlockWithHeader(&types.Header{})
s.AddBlockBuilt(b)
assert.Equal(t, b.Header(), s.BlockBuilt().Header())
}

View File

@ -25,13 +25,11 @@ import (
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/rawdb"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/engineapi/engine_helpers"
"github.com/ledgerwatch/erigon/turbo/services"
"github.com/ledgerwatch/erigon/turbo/stages/headerdownload"
"google.golang.org/protobuf/types/known/emptypb"
)
// Configure network related parameters for the config.
@ -78,28 +76,6 @@ func NewEngineServer(ctx context.Context, logger log.Logger, config *chain.Confi
}
}
func (s *EngineServer) PendingBlock(_ context.Context, _ *emptypb.Empty) (*engine.PendingBlockReply, error) {
s.lock.Lock()
defer s.lock.Unlock()
b := s.builders[s.payloadId]
if b == nil {
return nil, nil
}
pendingBlock := b.Block()
if pendingBlock == nil {
return nil, nil
}
blockRlp, err := rlp.EncodeToBytes(pendingBlock)
if err != nil {
return nil, err
}
return &engine.PendingBlockReply{BlockRlp: blockRlp}, nil
}
func convertPayloadStatus(payloadStatus *engine_helpers.PayloadStatus) *engine.EnginePayloadStatus {
reply := engine.EnginePayloadStatus{Status: payloadStatus.Status}
if payloadStatus.Status != engine.EngineStatus_SYNCING {

View File

@ -18,7 +18,7 @@ func TestNotFoundMustReturnNil(t *testing.T) {
require := require.New(t)
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m),
m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
m.DB, nil, nil, nil, 5000000, 100_000, log.New())
ctx := context.Background()
a, err := api.GetTransactionByBlockNumberAndIndex(ctx, 10_000, 1)

View File

@ -21,7 +21,7 @@ func APIList(db kv.RoDB, borDb kv.RoDB, eth rpchelper.ApiBackend, engineBackend
logger log.Logger,
) (list []rpc.API) {
base := NewBaseApi(filters, stateCache, blockReader, agg, cfg.WithDatadir, cfg.EvmCallTimeout, engine, cfg.Dirs)
ethImpl := NewEthAPI(base, db, eth, engineBackend, txPool, mining, cfg.Gascap, cfg.ReturnDataLimit, logger)
ethImpl := NewEthAPI(base, db, eth, txPool, mining, cfg.Gascap, cfg.ReturnDataLimit, logger)
erigonImpl := NewErigonAPI(base, db, eth)
txpoolImpl := NewTxPoolAPI(base, db, txPool)
netImpl := NewNetAPIImpl(eth)
@ -146,7 +146,7 @@ func AuthAPIList(db kv.RoDB, eth rpchelper.ApiBackend, engineBackend rpchelper.E
) (list []rpc.API) {
base := NewBaseApi(filters, stateCache, blockReader, agg, cfg.WithDatadir, cfg.EvmCallTimeout, engine, cfg.Dirs)
ethImpl := NewEthAPI(base, db, eth, engineBackend, txPool, mining, cfg.Gascap, cfg.ReturnDataLimit, logger)
ethImpl := NewEthAPI(base, db, eth, txPool, mining, cfg.Gascap, cfg.ReturnDataLimit, logger)
engineImpl := NewEngineAPI(base, db, engineBackend)
list = append(list, rpc.API{

View File

@ -53,7 +53,7 @@ func TestTraceBlockByNumber(t *testing.T) {
agg := m.HistoryV3Components()
stateCache := kvcache.New(kvcache.DefaultCoherentConfig)
baseApi := NewBaseApi(nil, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs)
ethApi := NewEthAPI(baseApi, m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
ethApi := NewEthAPI(baseApi, m.DB, nil, nil, nil, 5000000, 100_000, log.New())
api := NewPrivateDebugAPI(baseApi, m.DB, 0)
for _, tt := range debugTraceTransactionTests {
var buf bytes.Buffer
@ -98,7 +98,7 @@ func TestTraceBlockByNumber(t *testing.T) {
func TestTraceBlockByHash(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ethApi := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
ethApi := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
api := NewPrivateDebugAPI(newBaseApiForTest(m), m.DB, 0)
for _, tt := range debugTraceTransactionTests {
var buf bytes.Buffer

View File

@ -29,7 +29,7 @@ func TestGetLogs(t *testing.T) {
assert := assert.New(t)
m, _, _ := rpcdaemontest.CreateTestSentry(t)
{
ethApi := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
ethApi := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
logs, err := ethApi.GetLogs(context.Background(), filters.FilterCriteria{FromBlock: big.NewInt(0), ToBlock: big.NewInt(10)})
assert.NoError(err)

View File

@ -318,7 +318,6 @@ func (api *BaseAPI) pruneMode(tx kv.Tx) (*prune.Mode, error) {
type APIImpl struct {
*BaseAPI
ethBackend rpchelper.ApiBackend
engineBackend rpchelper.EngineBackend // TODO(Giulio2002): Remove
txPool txpool.TxpoolClient
mining txpool.MiningClient
gasCache *GasPriceCache
@ -329,14 +328,13 @@ type APIImpl struct {
}
// NewEthAPI returns APIImpl instance
func NewEthAPI(base *BaseAPI, db kv.RoDB, eth rpchelper.ApiBackend, engineBackend rpchelper.EngineBackend, txPool txpool.TxpoolClient, mining txpool.MiningClient, gascap uint64, returnDataLimit int, logger log.Logger) *APIImpl {
func NewEthAPI(base *BaseAPI, db kv.RoDB, eth rpchelper.ApiBackend, txPool txpool.TxpoolClient, mining txpool.MiningClient, gascap uint64, returnDataLimit int, logger log.Logger) *APIImpl {
if gascap == 0 {
gascap = uint64(math.MaxUint64 / 2)
}
return &APIImpl{
BaseAPI: base,
engineBackend: engineBackend,
db: db,
ethBackend: eth,
txPool: txPool,

View File

@ -54,7 +54,7 @@ func TestGetTransactionReceipt(t *testing.T) {
db := m.DB
agg := m.HistoryV3Components()
stateCache := kvcache.New(kvcache.DefaultCoherentConfig)
api := NewEthAPI(NewBaseApi(nil, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), db, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(NewBaseApi(nil, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), db, nil, nil, nil, 5000000, 100_000, log.New())
// Call GetTransactionReceipt for transaction which is not in the database
if _, err := api.GetTransactionReceipt(context.Background(), common.Hash{}); err != nil {
t.Errorf("calling GetTransactionReceipt with empty hash: %v", err)
@ -63,7 +63,7 @@ func TestGetTransactionReceipt(t *testing.T) {
func TestGetTransactionReceiptUnprotected(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
// Call GetTransactionReceipt for un-protected transaction
if _, err := api.GetTransactionReceipt(context.Background(), common.HexToHash("0x3f3cb8a0e13ed2481f97f53f7095b9cbc78b6ffb779f2d3e565146371a8830ea")); err != nil {
t.Errorf("calling GetTransactionReceipt for unprotected tx: %v", err)
@ -75,7 +75,7 @@ func TestGetTransactionReceiptUnprotected(t *testing.T) {
func TestGetStorageAt_ByBlockNumber_WithRequireCanonicalDefault(t *testing.T) {
assert := assert.New(t)
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
result, err := api.GetStorageAt(context.Background(), addr, "0x0", rpc.BlockNumberOrHashWithNumber(0))
@ -89,7 +89,7 @@ func TestGetStorageAt_ByBlockNumber_WithRequireCanonicalDefault(t *testing.T) {
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault(t *testing.T) {
assert := assert.New(t)
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
result, err := api.GetStorageAt(context.Background(), addr, "0x0", rpc.BlockNumberOrHashWithHash(m.Genesis.Hash(), false))
@ -103,7 +103,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault(t *testing.T) {
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue(t *testing.T) {
assert := assert.New(t)
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
result, err := api.GetStorageAt(context.Background(), addr, "0x0", rpc.BlockNumberOrHashWithHash(m.Genesis.Hash(), true))
@ -116,7 +116,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue(t *testing.T) {
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault_BlockNotFoundError(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
offChain, err := core.GenerateChain(m.ChainConfig, m.Genesis, m.Engine, m.DB, 1, func(i int, block *core.BlockGen) {
@ -137,7 +137,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault_BlockNotFoundError
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue_BlockNotFoundError(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
offChain, err := core.GenerateChain(m.ChainConfig, m.Genesis, m.Engine, m.DB, 1, func(i int, block *core.BlockGen) {
@ -159,7 +159,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue_BlockNotFoundError(t
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault_NonCanonicalBlock(t *testing.T) {
assert := assert.New(t)
m, _, orphanedChain := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
orphanedBlock := orphanedChain[0].Blocks[0]
@ -178,7 +178,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalDefault_NonCanonicalBlock(
func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue_NonCanonicalBlock(t *testing.T) {
m, _, orphanedChain := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
addr := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
orphanedBlock := orphanedChain[0].Blocks[0]
@ -194,7 +194,7 @@ func TestGetStorageAt_ByBlockHash_WithRequireCanonicalTrue_NonCanonicalBlock(t *
func TestCall_ByBlockHash_WithRequireCanonicalDefault_NonCanonicalBlock(t *testing.T) {
m, _, orphanedChain := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
from := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
to := common.HexToAddress("0x0d3ab14bbad3d99f4203bd7a11acb94882050e7e")
@ -217,7 +217,7 @@ func TestCall_ByBlockHash_WithRequireCanonicalDefault_NonCanonicalBlock(t *testi
func TestCall_ByBlockHash_WithRequireCanonicalTrue_NonCanonicalBlock(t *testing.T) {
m, _, orphanedChain := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
from := common.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
to := common.HexToAddress("0x0d3ab14bbad3d99f4203bd7a11acb94882050e7e")

View File

@ -379,7 +379,7 @@ func (api *APIImpl) blockByNumber(ctx context.Context, number rpc.BlockNumber, t
return block, nil
}
block, err := api.engineBackend.PendingBlock(ctx)
block, err := api.ethBackend.PendingBlock(ctx)
if err != nil {
return nil, err
}

View File

@ -25,7 +25,7 @@ import (
// Gets the latest block number with the latest tag
func TestGetBlockByNumberWithLatestTag(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
b, err := api.GetBlockByNumber(context.Background(), rpc.LatestBlockNumber, false)
expected := common.HexToHash("0x5883164d4100b95e1d8e931b8b9574586a1dea7507941e6ad3c1e3a2591485fd")
if err != nil {
@ -55,7 +55,7 @@ func TestGetBlockByNumberWithLatestTag_WithHeadHashInDb(t *testing.T) {
}
tx.Commit()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
block, err := api.GetBlockByNumber(ctx, rpc.LatestBlockNumber, false)
if err != nil {
t.Errorf("error retrieving block by number: %s", err)
@ -86,7 +86,7 @@ func TestGetBlockByNumberWithPendingTag(t *testing.T) {
RplBlock: rlpBlock,
})
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
b, err := api.GetBlockByNumber(context.Background(), rpc.PendingBlockNumber, false)
if err != nil {
t.Errorf("error getting block number with pending tag: %s", err)
@ -97,7 +97,7 @@ func TestGetBlockByNumberWithPendingTag(t *testing.T) {
func TestGetBlockByNumber_WithFinalizedTag_NoFinalizedBlockInDb(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
if _, err := api.GetBlockByNumber(ctx, rpc.FinalizedBlockNumber, false); err != nil {
assert.ErrorIs(t, rpchelper.UnknownBlockError, err)
}
@ -124,7 +124,7 @@ func TestGetBlockByNumber_WithFinalizedTag_WithFinalizedBlockInDb(t *testing.T)
}
tx.Commit()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
block, err := api.GetBlockByNumber(ctx, rpc.FinalizedBlockNumber, false)
if err != nil {
t.Errorf("error retrieving block by number: %s", err)
@ -136,7 +136,7 @@ func TestGetBlockByNumber_WithFinalizedTag_WithFinalizedBlockInDb(t *testing.T)
func TestGetBlockByNumber_WithSafeTag_NoSafeBlockInDb(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
if _, err := api.GetBlockByNumber(ctx, rpc.SafeBlockNumber, false); err != nil {
assert.ErrorIs(t, rpchelper.UnknownBlockError, err)
}
@ -163,7 +163,7 @@ func TestGetBlockByNumber_WithSafeTag_WithSafeBlockInDb(t *testing.T) {
}
tx.Commit()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
block, err := api.GetBlockByNumber(ctx, rpc.SafeBlockNumber, false)
if err != nil {
t.Errorf("error retrieving block by number: %s", err)
@ -176,7 +176,7 @@ func TestGetBlockTransactionCountByHash(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
blockHash := common.HexToHash("0x6804117de2f3e6ee32953e78ced1db7b20214e0d8c745a03b8fecf7cc8ee76ef")
tx, err := m.DB.BeginRw(ctx)
@ -208,7 +208,7 @@ func TestGetBlockTransactionCountByHash(t *testing.T) {
func TestGetBlockTransactionCountByHash_ZeroTx(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
blockHash := common.HexToHash("0x5883164d4100b95e1d8e931b8b9574586a1dea7507941e6ad3c1e3a2591485fd")
tx, err := m.DB.BeginRw(ctx)
@ -240,7 +240,7 @@ func TestGetBlockTransactionCountByHash_ZeroTx(t *testing.T) {
func TestGetBlockTransactionCountByNumber(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
blockHash := common.HexToHash("0x6804117de2f3e6ee32953e78ced1db7b20214e0d8c745a03b8fecf7cc8ee76ef")
tx, err := m.DB.BeginRw(ctx)
@ -272,7 +272,7 @@ func TestGetBlockTransactionCountByNumber(t *testing.T) {
func TestGetBlockTransactionCountByNumber_ZeroTx(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
ctx := context.Background()
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
blockHash := common.HexToHash("0x5883164d4100b95e1d8e931b8b9574586a1dea7507941e6ad3c1e3a2591485fd")

View File

@ -84,7 +84,7 @@ func TestCallMany(t *testing.T) {
db := contractBackend.DB()
engine := contractBackend.Engine()
api := NewEthAPI(NewBaseApi(nil, stateCache, contractBackend.BlockReader(), contractBackend.Agg(), false, rpccfg.DefaultEvmCallTimeout, engine,
datadir.New(t.TempDir())), db, nil, nil, nil, nil, 5000000, 100_000, log.New())
datadir.New(t.TempDir())), db, nil, nil, nil, 5000000, 100_000, log.New())
callArgAddr1 := ethapi.CallArgs{From: &address, To: &tokenAddr, Nonce: &nonce,
MaxPriorityFeePerGas: (*hexutil.Big)(big.NewInt(1e9)),

View File

@ -42,7 +42,7 @@ func TestEstimateGas(t *testing.T) {
ctx, conn := rpcdaemontest.CreateTestGrpcConn(t, stages.Mock(t))
mining := txpool.NewMiningClient(conn)
ff := rpchelper.New(ctx, nil, nil, mining, func() {}, m.Log)
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
var from = libcommon.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
var to = libcommon.HexToAddress("0x0d3ab14bbad3d99f4203bd7a11acb94882050e7e")
if _, err := api.EstimateGas(context.Background(), &ethapi.CallArgs{
@ -57,7 +57,7 @@ func TestEthCallNonCanonical(t *testing.T) {
m, _, _ := rpcdaemontest.CreateTestSentry(t)
agg := m.HistoryV3Components()
stateCache := kvcache.New(kvcache.DefaultCoherentConfig)
api := NewEthAPI(NewBaseApi(nil, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(NewBaseApi(nil, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
var from = libcommon.HexToAddress("0x71562b71999873db5b286df957af199ec94617f7")
var to = libcommon.HexToAddress("0x0d3ab14bbad3d99f4203bd7a11acb94882050e7e")
if _, err := api.Call(context.Background(), ethapi.CallArgs{
@ -76,7 +76,7 @@ func TestEthCallToPrunedBlock(t *testing.T) {
m, bankAddress, contractAddress := chainWithDeployedContract(t)
doPrune(t, m.DB, pruneTo)
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
callData := hexutil.MustDecode("0x2e64cec1")
callDataBytes := hexutility.Bytes(callData)
@ -97,7 +97,7 @@ func TestGetProof(t *testing.T) {
if m.HistoryV3 {
t.Skip("not supported by Erigon3")
}
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
key := func(b byte) libcommon.Hash {
result := libcommon.Hash{}

View File

@ -30,7 +30,7 @@ func TestNewFilters(t *testing.T) {
ctx, conn := rpcdaemontest.CreateTestGrpcConn(t, stages.Mock(t))
mining := txpool.NewMiningClient(conn)
ff := rpchelper.New(ctx, nil, nil, mining, func() {}, m.Log)
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, agg, false, rpccfg.DefaultEvmCallTimeout, m.Engine, m.Dirs), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
ptf, err := api.NewPendingTransactionFilter(ctx)
assert.Nil(err)

View File

@ -27,7 +27,7 @@ func TestPendingBlock(t *testing.T) {
stateCache := kvcache.New(kvcache.DefaultCoherentConfig)
engine := ethash.NewFaker()
api := NewEthAPI(NewBaseApi(ff, stateCache, m.BlockReader, nil, false, rpccfg.DefaultEvmCallTimeout, engine,
m.Dirs), nil, nil, nil, nil, mining, 5000000, 100_000, log.New())
m.Dirs), nil, nil, nil, mining, 5000000, 100_000, log.New())
expect := uint64(12345)
b, err := rlp.EncodeToBytes(types.NewBlockWithHeader(&types.Header{Number: big.NewInt(int64(expect))}))
require.NoError(t, err)

View File

@ -15,6 +15,7 @@ import (
"github.com/ledgerwatch/erigon/eth/protocols/eth"
"github.com/ledgerwatch/erigon/ethdb/privateapi"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/rpchelper"
"github.com/ledgerwatch/erigon/turbo/stages"
"github.com/ledgerwatch/log/v3"
@ -41,7 +42,7 @@ func TestEthSubscribe(t *testing.T) {
ctx := context.Background()
logger := log.New()
backendServer := privateapi.NewEthBackendServer(ctx, nil, m.DB, m.Notifications.Events, m.BlockReader, logger)
backendServer := privateapi.NewEthBackendServer(ctx, nil, m.DB, m.Notifications.Events, m.BlockReader, logger, builder.NewLatestBlockBuiltStore())
backendClient := direct.NewEthBackendClientDirect(backendServer)
backend := rpcservices.NewRemoteBackend(backendClient, m.DB, m.BlockReader)
ff := rpchelper.New(ctx, backend, nil, nil, func() {}, m.Log)

View File

@ -40,7 +40,7 @@ func TestGasPrice(t *testing.T) {
t.Run(testCase.description, func(t *testing.T) {
m := createGasPriceTestKV(t, testCase.chainSize)
defer m.DB.Close()
eth := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, nil, 5000000, 100_000, log.New())
eth := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 100_000, log.New())
ctx := context.Background()
result, err := eth.GasPrice(ctx)

View File

@ -79,7 +79,7 @@ func TestSendRawTransaction(t *testing.T) {
ctx, conn := rpcdaemontest.CreateTestGrpcConn(t, m)
txPool := txpool.NewTxpoolClient(conn)
ff := rpchelper.New(ctx, nil, txPool, txpool.NewMiningClient(conn), func() {}, m.Log)
api := jsonrpc.NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, txPool, nil, 5000000, 100_000, logger)
api := jsonrpc.NewEthAPI(newBaseApiForTest(m), m.DB, nil, txPool, nil, 5000000, 100_000, logger)
buf := bytes.NewBuffer(nil)
err = txn.MarshalBinary(buf)

View File

@ -29,6 +29,7 @@ type ApiBackend interface {
BlockWithSenders(ctx context.Context, tx kv.Getter, hash libcommon.Hash, blockHeight uint64) (block *types.Block, senders []libcommon.Address, err error)
NodeInfo(ctx context.Context, limit uint32) ([]p2p.NodeInfo, error)
Peers(ctx context.Context) ([]*p2p.PeerInfo, error)
PendingBlock(ctx context.Context) (*types.Block, error)
}
type EngineBackend interface {
@ -37,5 +38,4 @@ type EngineBackend interface {
EngineGetPayload(ctx context.Context, payloadId uint64) (*engine.EngineGetPayloadResponse, error)
EngineGetPayloadBodiesByHashV1(ctx context.Context, request *engine.EngineGetPayloadBodiesByHashV1Request) (*engine.EngineGetPayloadBodiesV1Response, error)
EngineGetPayloadBodiesByRangeV1(ctx context.Context, request *engine.EngineGetPayloadBodiesByRangeV1Request) (*engine.EngineGetPayloadBodiesV1Response, error)
PendingBlock(ctx context.Context) (*types.Block, error)
}

View File

@ -50,6 +50,7 @@ import (
"github.com/ledgerwatch/erigon/ethdb/prune"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/turbo/builder"
"github.com/ledgerwatch/erigon/turbo/engineapi/engine_helpers"
"github.com/ledgerwatch/erigon/turbo/rpchelper"
"github.com/ledgerwatch/erigon/turbo/services"
@ -318,6 +319,7 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
panic(err)
}
}
latestBlockBuiltStore := builder.NewLatestBlockBuiltStore()
inMemoryExecution := func(batch kv.RwTx, header *types.Header, body *types.RawBody, unwindPoint uint64, headersChain []*types.Header, bodiesChain []*types.RawBody,
notifications *shards.Notifications) error {
@ -432,7 +434,7 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
stagedsync.StageMiningExecCfg(mock.DB, miner, nil, *mock.ChainConfig, mock.Engine, &vm.Config{}, dirs.Tmp, nil, 0, mock.TxPool, nil, mock.BlockReader),
stagedsync.StageHashStateCfg(mock.DB, dirs, cfg.HistoryV3),
stagedsync.StageTrieCfg(mock.DB, false, true, false, dirs.Tmp, mock.BlockReader, mock.sentriesClient.Hd, cfg.HistoryV3, mock.agg),
stagedsync.StageMiningFinishCfg(mock.DB, *mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader),
stagedsync.StageMiningFinishCfg(mock.DB, *mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader, latestBlockBuiltStore),
),
stagedsync.MiningUnwindOrder,
stagedsync.MiningPruneOrder,