node: refactor package node (#21105)
This PR significantly changes the APIs for instantiating Ethereum nodes in a Go program. The new APIs are not backwards-compatible, but we feel that this is made up for by the much simpler way of registering services on node.Node. You can find more information and rationale in the design document: https://gist.github.com/renaynay/5bec2de19fde66f4d04c535fd24f0775. There is also a new feature in Node's Go API: it is now possible to register arbitrary handlers on the user-facing HTTP server. In geth, this facility is used to enable GraphQL. There is a single minor change relevant for geth users in this PR: The GraphQL API is no longer available separately from the JSON-RPC HTTP server. If you want GraphQL, you need to enable it using the ./geth --http --graphql flag combination. The --graphql.port and --graphql.addr flags are no longer available.
This commit is contained in:
@ -23,6 +23,7 @@ import (
|
||||
|
||||
"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/rawdb"
|
||||
@ -282,3 +283,11 @@ func (b *LesApiBackend) ServiceFilter(ctx context.Context, session *bloombits.Ma
|
||||
go session.Multiplex(bloomRetrievalBatch, bloomRetrievalWait, b.eth.bloomRequests)
|
||||
}
|
||||
}
|
||||
|
||||
func (b *LesApiBackend) Engine() consensus.Engine {
|
||||
return b.eth.engine
|
||||
}
|
||||
|
||||
func (b *LesApiBackend) CurrentHeader() *types.Header {
|
||||
return b.eth.blockchain.CurrentHeader()
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ func TestMain(m *testing.M) {
|
||||
log.Root().SetHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(colorable.NewColorableStderr(), log.TerminalFormat(true))))
|
||||
// register the Delivery service which will run as a devp2p
|
||||
// protocol when using the exec adapter
|
||||
adapters.RegisterServices(services)
|
||||
adapters.RegisterLifecycles(services)
|
||||
os.Exit(m.Run())
|
||||
}
|
||||
|
||||
@ -392,7 +392,7 @@ func getCapacityInfo(ctx context.Context, t *testing.T, server *rpc.Client) (min
|
||||
return
|
||||
}
|
||||
|
||||
var services = adapters.Services{
|
||||
var services = adapters.LifecycleConstructors{
|
||||
"lesclient": newLesClientService,
|
||||
"lesserver": newLesServerService,
|
||||
}
|
||||
@ -414,7 +414,7 @@ func NewNetwork() (*simulations.Network, func(), error) {
|
||||
return net, teardown, nil
|
||||
}
|
||||
|
||||
func NewAdapter(adapterType string, services adapters.Services) (adapter adapters.NodeAdapter, teardown func(), err error) {
|
||||
func NewAdapter(adapterType string, services adapters.LifecycleConstructors) (adapter adapters.NodeAdapter, teardown func(), err error) {
|
||||
teardown = func() {}
|
||||
switch adapterType {
|
||||
case "sim":
|
||||
@ -454,7 +454,7 @@ func testSim(t *testing.T, serverCount, clientCount int, serverDir, clientDir []
|
||||
|
||||
for i := range clients {
|
||||
clientconf := adapters.RandomNodeConfig()
|
||||
clientconf.Services = []string{"lesclient"}
|
||||
clientconf.Lifecycles = []string{"lesclient"}
|
||||
if len(clientDir) == clientCount {
|
||||
clientconf.DataDir = clientDir[i]
|
||||
}
|
||||
@ -467,7 +467,7 @@ func testSim(t *testing.T, serverCount, clientCount int, serverDir, clientDir []
|
||||
|
||||
for i := range servers {
|
||||
serverconf := adapters.RandomNodeConfig()
|
||||
serverconf.Services = []string{"lesserver"}
|
||||
serverconf.Lifecycles = []string{"lesserver"}
|
||||
if len(serverDir) == serverCount {
|
||||
serverconf.DataDir = serverDir[i]
|
||||
}
|
||||
@ -492,26 +492,25 @@ func testSim(t *testing.T, serverCount, clientCount int, serverDir, clientDir []
|
||||
return test(ctx, net, servers, clients)
|
||||
}
|
||||
|
||||
func newLesClientService(ctx *adapters.ServiceContext) (node.Service, error) {
|
||||
func newLesClientService(ctx *adapters.ServiceContext, stack *node.Node) (node.Lifecycle, error) {
|
||||
config := eth.DefaultConfig
|
||||
config.SyncMode = downloader.LightSync
|
||||
config.Ethash.PowMode = ethash.ModeFake
|
||||
return New(ctx.NodeContext, &config)
|
||||
return New(stack, &config)
|
||||
}
|
||||
|
||||
func newLesServerService(ctx *adapters.ServiceContext) (node.Service, error) {
|
||||
func newLesServerService(ctx *adapters.ServiceContext, stack *node.Node) (node.Lifecycle, error) {
|
||||
config := eth.DefaultConfig
|
||||
config.SyncMode = downloader.FullSync
|
||||
config.LightServ = testServerCapacity
|
||||
config.LightPeers = testMaxClients
|
||||
ethereum, err := eth.New(ctx.NodeContext, &config)
|
||||
ethereum, err := eth.New(stack, &config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
server, err := NewLesServer(ethereum, &config)
|
||||
_, err = NewLesServer(stack, ethereum, &config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ethereum.AddLesServer(server)
|
||||
return ethereum, nil
|
||||
}
|
||||
|
@ -51,16 +51,6 @@ type CheckpointOracle struct {
|
||||
|
||||
// New creates a checkpoint oracle handler with given configs and callback.
|
||||
func New(config *params.CheckpointOracleConfig, getLocal func(uint64) params.TrustedCheckpoint) *CheckpointOracle {
|
||||
if config == nil {
|
||||
log.Info("Checkpoint registrar is not enabled")
|
||||
return nil
|
||||
}
|
||||
if config.Address == (common.Address{}) || uint64(len(config.Signers)) < config.Threshold {
|
||||
log.Warn("Invalid checkpoint registrar config")
|
||||
return nil
|
||||
}
|
||||
log.Info("Configured checkpoint registrar", "address", config.Address, "signers", len(config.Signers), "threshold", config.Threshold)
|
||||
|
||||
return &CheckpointOracle{
|
||||
config: config,
|
||||
getLocal: getLocal,
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts"
|
||||
"github.com/ethereum/go-ethereum/accounts/abi/bind"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
@ -37,7 +36,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/eth/gasprice"
|
||||
"github.com/ethereum/go-ethereum/event"
|
||||
"github.com/ethereum/go-ethereum/internal/ethapi"
|
||||
"github.com/ethereum/go-ethereum/les/checkpointoracle"
|
||||
lpc "github.com/ethereum/go-ethereum/les/lespay/client"
|
||||
"github.com/ethereum/go-ethereum/light"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
@ -72,14 +70,17 @@ type LightEthereum struct {
|
||||
engine consensus.Engine
|
||||
accountManager *accounts.Manager
|
||||
netRPCService *ethapi.PublicNetAPI
|
||||
|
||||
p2pServer *p2p.Server
|
||||
}
|
||||
|
||||
func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
|
||||
chainDb, err := ctx.OpenDatabase("lightchaindata", config.DatabaseCache, config.DatabaseHandles, "eth/db/chaindata/")
|
||||
// New creates an instance of the light client.
|
||||
func New(stack *node.Node, config *eth.Config) (*LightEthereum, error) {
|
||||
chainDb, err := stack.OpenDatabase("lightchaindata", config.DatabaseCache, config.DatabaseHandles, "eth/db/chaindata/")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
lespayDb, err := ctx.OpenDatabase("lespay", 0, 0, "eth/db/lespay")
|
||||
lespayDb, err := stack.OpenDatabase("lespay", 0, 0, "eth/db/lespay")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -100,17 +101,18 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
|
||||
closeCh: make(chan struct{}),
|
||||
},
|
||||
peers: peers,
|
||||
eventMux: ctx.EventMux,
|
||||
eventMux: stack.EventMux(),
|
||||
reqDist: newRequestDistributor(peers, &mclock.System{}),
|
||||
accountManager: ctx.AccountManager,
|
||||
engine: eth.CreateConsensusEngine(ctx, chainConfig, &config.Ethash, nil, false, chainDb),
|
||||
accountManager: stack.AccountManager(),
|
||||
engine: eth.CreateConsensusEngine(stack, chainConfig, &config.Ethash, nil, false, chainDb),
|
||||
bloomRequests: make(chan chan *bloombits.Retrieval),
|
||||
bloomIndexer: eth.NewBloomIndexer(chainDb, params.BloomBitsBlocksClient, params.HelperTrieConfirmations),
|
||||
valueTracker: lpc.NewValueTracker(lespayDb, &mclock.System{}, requestList, time.Minute, 1/float64(time.Hour), 1/float64(time.Hour*100), 1/float64(time.Hour*1000)),
|
||||
p2pServer: stack.Server(),
|
||||
}
|
||||
peers.subscribe((*vtSubscription)(leth.valueTracker))
|
||||
|
||||
dnsdisc, err := leth.setupDiscovery(&ctx.Config.P2P)
|
||||
dnsdisc, err := leth.setupDiscovery(&stack.Config().P2P)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -139,11 +141,7 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
|
||||
leth.txPool = light.NewTxPool(leth.chainConfig, leth.blockchain, leth.relay)
|
||||
|
||||
// Set up checkpoint oracle.
|
||||
oracle := config.CheckpointOracle
|
||||
if oracle == nil {
|
||||
oracle = params.CheckpointOracles[genesisHash]
|
||||
}
|
||||
leth.oracle = checkpointoracle.New(oracle, leth.localCheckpoint)
|
||||
leth.oracle = leth.setupOracle(stack, genesisHash, config)
|
||||
|
||||
// Note: AddChildIndexer starts the update process for the child
|
||||
leth.bloomIndexer.AddChildIndexer(leth.bloomTrieIndexer)
|
||||
@ -160,7 +158,7 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
|
||||
rawdb.WriteChainConfig(chainDb, genesisHash, chainConfig)
|
||||
}
|
||||
|
||||
leth.ApiBackend = &LesApiBackend{ctx.ExtRPCEnabled(), leth, nil}
|
||||
leth.ApiBackend = &LesApiBackend{stack.Config().ExtRPCEnabled(), leth, nil}
|
||||
gpoParams := config.GPO
|
||||
if gpoParams.Default == nil {
|
||||
gpoParams.Default = config.Miner.GasPrice
|
||||
@ -172,6 +170,14 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
|
||||
log.Warn("Ultra light client is enabled", "trustedNodes", len(leth.handler.ulc.keys), "minTrustedFraction", leth.handler.ulc.fraction)
|
||||
leth.blockchain.DisableCheckFreq()
|
||||
}
|
||||
|
||||
leth.netRPCService = ethapi.NewPublicNetAPI(leth.p2pServer, leth.config.NetworkId)
|
||||
|
||||
// Register the backend on the node
|
||||
stack.RegisterAPIs(leth.APIs())
|
||||
stack.RegisterProtocols(leth.Protocols())
|
||||
stack.RegisterLifecycle(leth)
|
||||
|
||||
return leth, nil
|
||||
}
|
||||
|
||||
@ -265,8 +271,7 @@ func (s *LightEthereum) LesVersion() int { return int(ClientP
|
||||
func (s *LightEthereum) Downloader() *downloader.Downloader { return s.handler.downloader }
|
||||
func (s *LightEthereum) EventMux() *event.TypeMux { return s.eventMux }
|
||||
|
||||
// Protocols implements node.Service, returning all the currently configured
|
||||
// network protocols to start.
|
||||
// Protocols returns all the currently configured network protocols to start.
|
||||
func (s *LightEthereum) Protocols() []p2p.Protocol {
|
||||
return s.makeProtocols(ClientProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
|
||||
if p := s.peers.peer(id.String()); p != nil {
|
||||
@ -276,9 +281,9 @@ func (s *LightEthereum) Protocols() []p2p.Protocol {
|
||||
}, s.dialCandidates)
|
||||
}
|
||||
|
||||
// Start implements node.Service, starting all internal goroutines needed by the
|
||||
// Start implements node.Lifecycle, starting all internal goroutines needed by the
|
||||
// light ethereum protocol implementation.
|
||||
func (s *LightEthereum) Start(srvr *p2p.Server) error {
|
||||
func (s *LightEthereum) Start() error {
|
||||
log.Warn("Light client mode is an experimental feature")
|
||||
|
||||
s.serverPool.start()
|
||||
@ -287,11 +292,10 @@ func (s *LightEthereum) Start(srvr *p2p.Server) error {
|
||||
s.startBloomHandlers(params.BloomBitsBlocksClient)
|
||||
s.handler.start()
|
||||
|
||||
s.netRPCService = ethapi.NewPublicNetAPI(srvr, s.config.NetworkId)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop implements node.Service, terminating all internal goroutines used by the
|
||||
// Stop implements node.Lifecycle, terminating all internal goroutines used by the
|
||||
// Ethereum protocol.
|
||||
func (s *LightEthereum) Stop() error {
|
||||
close(s.closeCh)
|
||||
@ -314,11 +318,3 @@ func (s *LightEthereum) Stop() error {
|
||||
log.Info("Light ethereum stopped")
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetClient sets the rpc client and binds the registrar contract.
|
||||
func (s *LightEthereum) SetContractBackend(backend bind.ContractBackend) {
|
||||
if s.oracle == nil {
|
||||
return
|
||||
}
|
||||
s.oracle.Start(backend)
|
||||
}
|
||||
|
@ -26,9 +26,12 @@ import (
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/eth"
|
||||
"github.com/ethereum/go-ethereum/ethclient"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/les/checkpointoracle"
|
||||
"github.com/ethereum/go-ethereum/light"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/node"
|
||||
"github.com/ethereum/go-ethereum/p2p"
|
||||
"github.com/ethereum/go-ethereum/p2p/discv5"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
@ -145,3 +148,26 @@ func (c *lesCommons) localCheckpoint(index uint64) params.TrustedCheckpoint {
|
||||
BloomRoot: light.GetBloomTrieRoot(c.chainDb, index, sectionHead),
|
||||
}
|
||||
}
|
||||
|
||||
// setupOracle sets up the checkpoint oracle contract client.
|
||||
func (c *lesCommons) setupOracle(node *node.Node, genesis common.Hash, ethconfig *eth.Config) *checkpointoracle.CheckpointOracle {
|
||||
config := ethconfig.CheckpointOracle
|
||||
if config == nil {
|
||||
// Try loading default config.
|
||||
config = params.CheckpointOracles[genesis]
|
||||
}
|
||||
if config == nil {
|
||||
log.Info("Checkpoint registrar is not enabled")
|
||||
return nil
|
||||
}
|
||||
if config.Address == (common.Address{}) || uint64(len(config.Signers)) < config.Threshold {
|
||||
log.Warn("Invalid checkpoint registrar config")
|
||||
return nil
|
||||
}
|
||||
oracle := checkpointoracle.New(config, c.localCheckpoint)
|
||||
rpcClient, _ := node.Attach()
|
||||
client := ethclient.NewClient(rpcClient)
|
||||
oracle.Start(client)
|
||||
log.Info("Configured checkpoint registrar", "address", config.Address, "signers", len(config.Signers), "threshold", config.Threshold)
|
||||
return oracle
|
||||
}
|
||||
|
@ -20,14 +20,12 @@ import (
|
||||
"crypto/ecdsa"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts/abi/bind"
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/eth"
|
||||
"github.com/ethereum/go-ethereum/les/checkpointoracle"
|
||||
"github.com/ethereum/go-ethereum/les/flowcontrol"
|
||||
"github.com/ethereum/go-ethereum/light"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/node"
|
||||
"github.com/ethereum/go-ethereum/p2p"
|
||||
"github.com/ethereum/go-ethereum/p2p/discv5"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
@ -55,9 +53,11 @@ type LesServer struct {
|
||||
minCapacity, maxCapacity, freeCapacity uint64
|
||||
threadsIdle int // Request serving threads count when system is idle.
|
||||
threadsBusy int // Request serving threads count when system is busy(block insertion).
|
||||
|
||||
p2pSrv *p2p.Server
|
||||
}
|
||||
|
||||
func NewLesServer(e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
|
||||
func NewLesServer(node *node.Node, e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
|
||||
// Collect les protocol version information supported by local node.
|
||||
lesTopics := make([]discv5.Topic, len(AdvertiseProtocolVersions))
|
||||
for i, pv := range AdvertiseProtocolVersions {
|
||||
@ -88,17 +88,15 @@ func NewLesServer(e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
|
||||
servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100),
|
||||
threadsBusy: config.LightServ/100 + 1,
|
||||
threadsIdle: threads,
|
||||
p2pSrv: node.Server(),
|
||||
}
|
||||
srv.handler = newServerHandler(srv, e.BlockChain(), e.ChainDb(), e.TxPool(), e.Synced)
|
||||
srv.costTracker, srv.minCapacity = newCostTracker(e.ChainDb(), config)
|
||||
srv.freeCapacity = srv.minCapacity
|
||||
srv.oracle = srv.setupOracle(node, e.BlockChain().Genesis().Hash(), config)
|
||||
|
||||
// Set up checkpoint oracle.
|
||||
oracle := config.CheckpointOracle
|
||||
if oracle == nil {
|
||||
oracle = params.CheckpointOracles[e.BlockChain().Genesis().Hash()]
|
||||
}
|
||||
srv.oracle = checkpointoracle.New(oracle, srv.localCheckpoint)
|
||||
// Initialize the bloom trie indexer.
|
||||
e.BloomIndexer().AddChildIndexer(srv.bloomTrieIndexer)
|
||||
|
||||
// Initialize server capacity management fields.
|
||||
srv.defParams = flowcontrol.ServerParams{
|
||||
@ -125,6 +123,11 @@ func NewLesServer(e *eth.Ethereum, config *eth.Config) (*LesServer, error) {
|
||||
"chtroot", checkpoint.CHTRoot, "bloomroot", checkpoint.BloomRoot)
|
||||
}
|
||||
srv.chtIndexer.Start(e.BlockChain())
|
||||
|
||||
node.RegisterProtocols(srv.Protocols())
|
||||
node.RegisterAPIs(srv.APIs())
|
||||
node.RegisterLifecycle(srv)
|
||||
|
||||
return srv, nil
|
||||
}
|
||||
|
||||
@ -166,14 +169,14 @@ func (s *LesServer) Protocols() []p2p.Protocol {
|
||||
}
|
||||
|
||||
// Start starts the LES server
|
||||
func (s *LesServer) Start(srvr *p2p.Server) {
|
||||
s.privateKey = srvr.PrivateKey
|
||||
func (s *LesServer) Start() error {
|
||||
s.privateKey = s.p2pSrv.PrivateKey
|
||||
s.handler.start()
|
||||
|
||||
s.wg.Add(1)
|
||||
go s.capacityManagement()
|
||||
|
||||
if srvr.DiscV5 != nil {
|
||||
if s.p2pSrv.DiscV5 != nil {
|
||||
for _, topic := range s.lesTopics {
|
||||
topic := topic
|
||||
go func() {
|
||||
@ -181,14 +184,16 @@ func (s *LesServer) Start(srvr *p2p.Server) {
|
||||
logger.Info("Starting topic registration")
|
||||
defer logger.Info("Terminated topic registration")
|
||||
|
||||
srvr.DiscV5.RegisterTopic(topic, s.closeCh)
|
||||
s.p2pSrv.DiscV5.RegisterTopic(topic, s.closeCh)
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop stops the LES service
|
||||
func (s *LesServer) Stop() {
|
||||
func (s *LesServer) Stop() error {
|
||||
close(s.closeCh)
|
||||
|
||||
// Disconnect existing sessions.
|
||||
@ -207,18 +212,8 @@ func (s *LesServer) Stop() {
|
||||
s.chtIndexer.Close()
|
||||
s.wg.Wait()
|
||||
log.Info("Les server stopped")
|
||||
}
|
||||
|
||||
func (s *LesServer) SetBloomBitsIndexer(bloomIndexer *core.ChainIndexer) {
|
||||
bloomIndexer.AddChildIndexer(s.bloomTrieIndexer)
|
||||
}
|
||||
|
||||
// SetClient sets the rpc client and starts running checkpoint contract if it is not yet watched.
|
||||
func (s *LesServer) SetContractBackend(backend bind.ContractBackend) {
|
||||
if s.oracle == nil {
|
||||
return
|
||||
}
|
||||
s.oracle.Start(backend)
|
||||
return nil
|
||||
}
|
||||
|
||||
// capacityManagement starts an event handler loop that updates the recharge curve of
|
||||
|
Reference in New Issue
Block a user