Skip to content

Commit

Permalink
Create HandlerBlockchain Interface
Browse files Browse the repository at this point in the history
  • Loading branch information
quentinlesceller committed Mar 22, 2023
1 parent 1e10009 commit 9f202af
Show file tree
Hide file tree
Showing 22 changed files with 196 additions and 57 deletions.
4 changes: 2 additions & 2 deletions eth/api.go
Expand Up @@ -30,10 +30,10 @@ import (

"github.com/blockcypher/go-ethereum/common"
"github.com/blockcypher/go-ethereum/common/hexutil"
"github.com/blockcypher/go-ethereum/core"
"github.com/blockcypher/go-ethereum/core/rawdb"
"github.com/blockcypher/go-ethereum/core/state"
"github.com/blockcypher/go-ethereum/core/types"
"github.com/blockcypher/go-ethereum/eth/protocols/eth"
"github.com/blockcypher/go-ethereum/internal/ethapi"
"github.com/blockcypher/go-ethereum/log"
"github.com/blockcypher/go-ethereum/rlp"
Expand Down Expand Up @@ -184,7 +184,7 @@ func (api *AdminAPI) ExportChain(file string, first *uint64, last *uint64) (bool
return true, nil
}

func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool {
func hasAllBlocks(chain eth.HandlerBlockchain, bs []*types.Block) bool {
for _, b := range bs {
if !chain.HasBlock(b.Hash(), b.NumberU64()) {
return false
Expand Down
21 changes: 19 additions & 2 deletions eth/backend.go
Expand Up @@ -68,7 +68,7 @@ type Ethereum struct {

// Handlers
txPool *txpool.TxPool
blockchain *core.BlockChain
blockchain eth.HandlerBlockchain
handler *handler
ethDialCandidates enode.Iterator
snapDialCandidates enode.Iterator
Expand Down Expand Up @@ -101,6 +101,23 @@ type Ethereum struct {
shutdownTracker *shutdowncheck.ShutdownTracker // Tracks if and when the node has shutdown ungracefully
}

func NewMin(handler *handler, chainDb ethdb.Database, ethDialCandidates enode.Iterator, snapDialCandidates enode.Iterator, networkID uint64, bc eth.HandlerBlockchain, merger *consensus.Merger, params *params.ChainConfig) *Ethereum {
conf := ethconfig.Defaults
conf.SyncMode = downloader.FullSync
conf.SnapshotCache = 0
ethereum := Ethereum{
blockchain: bc,
merger: merger,
config: &conf,
handler: handler,
ethDialCandidates: ethDialCandidates,
snapDialCandidates: snapDialCandidates,
networkID: networkID,
chainDb: chainDb,
}
return &ethereum
}

// New creates a new Ethereum object (including the
// initialisation of the common Ethereum object)
func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
Expand Down Expand Up @@ -467,7 +484,7 @@ func (s *Ethereum) IsMining() bool { return s.miner.Mining() }
func (s *Ethereum) Miner() *miner.Miner { return s.miner }

func (s *Ethereum) AccountManager() *accounts.Manager { return s.accountManager }
func (s *Ethereum) BlockChain() *core.BlockChain { return s.blockchain }
func (s *Ethereum) BlockChain() eth.HandlerBlockchain { return s.blockchain }
func (s *Ethereum) TxPool() *txpool.TxPool { return s.txPool }
func (s *Ethereum) EventMux() *event.TypeMux { return s.eventMux }
func (s *Ethereum) Engine() consensus.Engine { return s.engine }
Expand Down
8 changes: 4 additions & 4 deletions eth/catalyst/api.go
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/blockcypher/go-ethereum/beacon/engine"
"github.com/blockcypher/go-ethereum/common"
"github.com/blockcypher/go-ethereum/common/hexutil"
"github.com/blockcypher/go-ethereum/core/rawdb"
"github.com/blockcypher/go-ethereum/core/types"
"github.com/blockcypher/go-ethereum/eth"
"github.com/blockcypher/go-ethereum/eth/downloader"
Expand Down Expand Up @@ -289,7 +288,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
PayloadID: id,
}
}
if rawdb.ReadCanonicalHash(api.eth.ChainDb(), block.NumberU64()) != update.HeadBlockHash {
if api.eth.BlockChain().GetHeaderByNumber(block.NumberU64()).Hash() != update.HeadBlockHash {
// Block is not canonical, set head.
if latestValid, err := api.eth.BlockChain().SetCanonical(block); err != nil {
return engine.ForkChoiceResponse{PayloadStatus: engine.PayloadStatusV1{Status: engine.INVALID, LatestValidHash: &latestValid}}, err
Expand Down Expand Up @@ -317,7 +316,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
if finalBlock == nil {
log.Warn("Final block not available in database", "hash", update.FinalizedBlockHash)
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("final block not available in database"))
} else if rawdb.ReadCanonicalHash(api.eth.ChainDb(), finalBlock.NumberU64()) != update.FinalizedBlockHash {
} else if api.eth.BlockChain().GetHeaderByNumber(finalBlock.NumberU64()).Hash() != update.FinalizedBlockHash {
log.Warn("Final block not in canonical chain", "number", block.NumberU64(), "hash", update.HeadBlockHash)
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("final block not in canonical chain"))
}
Expand All @@ -331,7 +330,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
log.Warn("Safe block not available in database")
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("safe block not available in database"))
}
if rawdb.ReadCanonicalHash(api.eth.ChainDb(), safeBlock.NumberU64()) != update.SafeBlockHash {
if api.eth.BlockChain().GetHeaderByNumber(safeBlock.NumberU64()).Hash() != update.SafeBlockHash {
log.Warn("Safe block not in canonical chain")
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("safe block not in canonical chain"))
}
Expand Down Expand Up @@ -546,6 +545,7 @@ func (api *ConsensusAPI) newPayload(params engine.ExecutableData) (engine.Payloa
// be called by the newpayload command when the block seems to be ok, but some
// prerequisite prevents it from being processed (e.g. no parent, or snap sync).
func (api *ConsensusAPI) delayPayloadImport(block *types.Block) (engine.PayloadStatusV1, error) {
return engine.PayloadStatusV1{Status: engine.SYNCING}, errors.New("delayPayloadImport is not supported for bolt")
// Sanity check that this block's parent is not on a previously invalidated
// chain. If it is, mark the block as invalid too.
if res := api.checkInvalidAncestor(block.ParentHash(), block.Hash()); res != nil {
Expand Down
18 changes: 16 additions & 2 deletions eth/handler.go
Expand Up @@ -80,7 +80,7 @@ type HandlerConfig handlerConfig
// node network handler.
type handlerConfig struct {
Database ethdb.Database // Database for direct sync insertions
Chain *core.BlockChain // Blockchain to serve data from
Chain eth.HandlerBlockchain // Blockchain to serve data from
TxPool txPool // Transaction pool to propagate from
Merger *consensus.Merger // The manager for eth1/2 transition
Network uint64 // Network identifier to adfvertise
Expand All @@ -91,6 +91,20 @@ type handlerConfig struct {
RequiredBlocks map[uint64]common.Hash // Hard coded map of required block hashes for sync challenges
}

type Handler handler

func (h *Handler) BroadcastTransactions(txs types.Transactions) {
(*handler)(h).BroadcastTransactions(txs)
}

func (h *Handler) PeerCount() int {
return h.peers.len()
}

func (h *Handler) Start(maxPeers int) {
(*handler)(h).Start(maxPeers)
}

type handler struct {
networkID uint64
forkFilter forkid.Filter // Fork ID filter, constant across the lifetime of the node
Expand All @@ -103,7 +117,7 @@ type handler struct {

database ethdb.Database
txpool txPool
chain *core.BlockChain
chain eth.HandlerBlockchain
maxPeers int

downloader *downloader.Downloader
Expand Down
5 changes: 2 additions & 3 deletions eth/handler_eth.go
Expand Up @@ -23,7 +23,6 @@ import (
"time"

"github.com/blockcypher/go-ethereum/common"
"github.com/blockcypher/go-ethereum/core"
"github.com/blockcypher/go-ethereum/core/types"
"github.com/blockcypher/go-ethereum/eth/protocols/eth"
"github.com/blockcypher/go-ethereum/p2p/enode"
Expand All @@ -33,8 +32,8 @@ import (
// packets that are sent as replies or broadcasts.
type ethHandler handler

func (h *ethHandler) Chain() *core.BlockChain { return h.chain }
func (h *ethHandler) TxPool() eth.TxPool { return h.txpool }
func (h *ethHandler) Chain() eth.HandlerBlockchain { return h.chain }
func (h *ethHandler) TxPool() eth.TxPool { return h.txpool }

// RunPeer is invoked when a peer joins on the `eth` protocol.
func (h *ethHandler) RunPeer(peer *eth.Peer, hand eth.Handler) error {
Expand Down
2 changes: 1 addition & 1 deletion eth/handler_eth_test.go
Expand Up @@ -49,7 +49,7 @@ type testEthHandler struct {
txBroadcasts event.Feed
}

func (h *testEthHandler) Chain() *core.BlockChain { panic("no backing chain") }
func (h *testEthHandler) Chain() eth.HandlerBlockchain { panic("no backing chain") }
func (h *testEthHandler) TxPool() eth.TxPool { panic("no backing tx pool") }
func (h *testEthHandler) AcceptTxs() bool { return true }
func (h *testEthHandler) RunPeer(*eth.Peer, eth.Handler) error { panic("not used in tests") }
Expand Down
4 changes: 2 additions & 2 deletions eth/handler_snap.go
Expand Up @@ -17,7 +17,7 @@
package eth

import (
"github.com/blockcypher/go-ethereum/core"
"github.com/blockcypher/go-ethereum/eth/protocols/eth"
"github.com/blockcypher/go-ethereum/eth/protocols/snap"
"github.com/blockcypher/go-ethereum/p2p/enode"
)
Expand All @@ -26,7 +26,7 @@ import (
// packets that are sent as replies or broadcasts.
type snapHandler handler

func (h *snapHandler) Chain() *core.BlockChain { return h.chain }
func (h *snapHandler) Chain() eth.HandlerBlockchain { return h.chain }

// RunPeer is invoked when a peer joins on the `snap` protocol.
func (h *snapHandler) RunPeer(peer *snap.Peer, hand snap.Handler) error {
Expand Down
4 changes: 2 additions & 2 deletions eth/protocols/eth/discovery.go
Expand Up @@ -38,7 +38,7 @@ func (e enrEntry) ENRKey() string {

// StartENRUpdater starts the `eth` ENR updater loop, which listens for chain
// head events and updates the requested node record whenever a fork is passed.
func StartENRUpdater(chain *core.BlockChain, ln *enode.LocalNode) {
func StartENRUpdater(chain HandlerBlockchain, ln *enode.LocalNode) {
var newHead = make(chan core.ChainHeadEvent, 10)
sub := chain.SubscribeChainHeadEvent(newHead)

Expand All @@ -58,7 +58,7 @@ func StartENRUpdater(chain *core.BlockChain, ln *enode.LocalNode) {
}

// currentENREntry constructs an `eth` ENR entry based on the current state of the chain.
func currentENREntry(chain *core.BlockChain) *enrEntry {
func currentENREntry(chain HandlerBlockchain) *enrEntry {
head := chain.CurrentHeader()
return &enrEntry{
ForkID: forkid.NewID(chain.Config(), chain.Genesis().Hash(), head.Number.Uint64(), head.Time),
Expand Down
105 changes: 103 additions & 2 deletions eth/protocols/eth/handler.go
Expand Up @@ -18,17 +18,26 @@ package eth

import (
"fmt"
"io"
"math/big"
"time"

"github.com/blockcypher/go-ethereum/common"
"github.com/blockcypher/go-ethereum/consensus"
"github.com/blockcypher/go-ethereum/core"
"github.com/blockcypher/go-ethereum/core/rawdb"
"github.com/blockcypher/go-ethereum/core/state"
"github.com/blockcypher/go-ethereum/core/state/snapshot"
"github.com/blockcypher/go-ethereum/core/types"
"github.com/blockcypher/go-ethereum/core/vm"
"github.com/blockcypher/go-ethereum/event"
"github.com/blockcypher/go-ethereum/metrics"
"github.com/blockcypher/go-ethereum/p2p"
"github.com/blockcypher/go-ethereum/p2p/enode"
"github.com/blockcypher/go-ethereum/p2p/enr"
"github.com/blockcypher/go-ethereum/params"
"github.com/blockcypher/go-ethereum/rlp"
"github.com/blockcypher/go-ethereum/trie"
)

const (
Expand All @@ -55,6 +64,98 @@ const (
maxReceiptsServe = 1024
)

type HandlerBlockchain interface {
SetHead(head uint64) error
CurrentBlock() *types.Header
CurrentSnapBlock() *types.Header

Genesis() *types.Block

HasBlock(hash common.Hash, number uint64) bool
HasFastBlock(hash common.Hash, number uint64) bool
GetBlock(hash common.Hash, number uint64) *types.Block
GetBlockByHash(hash common.Hash) *types.Block
StopInsert()
InsertBlockWithoutSetHead(block *types.Block) error
InsertReceiptChain(blockChain types.Blocks, receiptChain []types.Receipts, ancientLimit uint64) (int, error)

SetTxLookupLimit(limit uint64)
TxLookupLimit() uint64

InsertChain(chain types.Blocks) (int, error)

InsertHeaderChain(chain []*types.Header, checkFreq int) (int, error)
CurrentHeader() *types.Header
GetTd(hash common.Hash, number uint64) *big.Int
GetHeader(hash common.Hash, number uint64) *types.Header
GetHeaderByHash(hash common.Hash) *types.Header
GetHeadersFrom(number, count uint64) []rlp.RawValue
HasHeader(hash common.Hash, number uint64) bool
GetHeaderByNumber(number uint64) *types.Header

Config() *params.ChainConfig
Engine() consensus.Engine

// Additionnal func for particular handler
// required by snap
SnapSyncCommitHead(common.Hash) error
Snapshots() *snapshot.Tree
ContractCode(hash common.Hash) ([]byte, error)
StateCache() state.Database

// required by discovery.go
SubscribeChainHeadEvent(ch chan<- core.ChainHeadEvent) event.Subscription

// required by eth/handler.go
GetAncestor(hash common.Hash, number, ancestor uint64, maxNonCanonical *uint64) (common.Hash, uint64)
GetBodyRLP(hash common.Hash) rlp.RawValue
TrieNode(hash common.Hash) ([]byte, error)
ContractCodeWithPrefix(hash common.Hash) ([]byte, error)
GetReceiptsByHash(hash common.Hash) types.Receipts
TrieDB() *trie.Database

// required for eth/handler_test.go
GetBlockByNumber(number uint64) *types.Block
GetCanonicalHash(number uint64) common.Hash
StateAt(root common.Hash) (*state.StateDB, error)
Stop()

// required for eth/api_backend.go
CurrentSafeBlock() *types.Header
CurrentFinalBlock() *types.Header
GetBody(hash common.Hash) *types.Body
GetVMConfig() *vm.Config
SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription
SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription
SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription
SubscribeChainSideEvent(ch chan<- core.ChainSideEvent) event.Subscription

// required for eth/backend.go
Export(w io.Writer) error
ExportN(w io.Writer, first uint64, last uint64) error
ResetWithGenesisBlock(genesis *types.Block) error

// required for eth/api.go
SetFinalized(block *types.Header)
SetSafe(block *types.Header)
SetCanonical(head *types.Block) (common.Hash, error)
SetTrieFlushInterval(interval time.Duration)

// required by eth/state_accesor.go
Processor() core.Processor

// required by eth/catalyst/api.go
HasBlockAndState(hash common.Hash, number uint64) bool

// required by miner/worker.go
WriteBlockAndSetHead(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB, emitHeadEvent bool) (status core.WriteStatus, err error)
GetBlocksFromHash(hash common.Hash, n int) (blocks []*types.Block)

// required by les
SubscribeBlockProcessingEvent(ch chan<- bool) event.Subscription
GetTransactionLookup(hash common.Hash) *rawdb.LegacyTxLookupEntry
}

// Handler is a callback to invoke from an outside runner after the boilerplate
// exchanges have passed.
type Handler func(peer *Peer) error
Expand All @@ -63,7 +164,7 @@ type Handler func(peer *Peer) error
// callback methods to invoke on remote deliveries.
type Backend interface {
// Chain retrieves the blockchain object to serve data.
Chain() *core.BlockChain
Chain() HandlerBlockchain

// TxPool retrieves the transaction pool object to serve data.
TxPool() TxPool
Expand Down Expand Up @@ -135,7 +236,7 @@ type NodeInfo struct {
}

// nodeInfo retrieves some `eth` protocol metadata about the running host node.
func nodeInfo(chain *core.BlockChain, network uint64) *NodeInfo {
func nodeInfo(chain HandlerBlockchain, network uint64) *NodeInfo {
head := chain.CurrentBlock()
hash := head.Hash()

Expand Down
4 changes: 2 additions & 2 deletions eth/protocols/eth/handler_test.go
Expand Up @@ -129,8 +129,8 @@ func (b *testBackend) close() {
b.chain.Stop()
}

func (b *testBackend) Chain() *core.BlockChain { return b.chain }
func (b *testBackend) TxPool() TxPool { return b.txpool }
func (b *testBackend) Chain() HandlerBlockchain { return b.chain }
func (b *testBackend) TxPool() TxPool { return b.txpool }

func (b *testBackend) RunPeer(peer *Peer, handler Handler) error {
// Normally the backend would do peer maintenance and handshakes. All that
Expand Down

0 comments on commit 9f202af

Please sign in to comment.