Skip to content

Commit

Permalink
les: code refactoring (ethereum#14416)
Browse files Browse the repository at this point in the history
This commit does various code refactorings:

- generalizes and moves the request retrieval/timeout/resend logic out of LesOdr
  (will be used by a subsequent PR)
- reworks the peer management logic so that all services can register with
  peerSet to get notified about added/dropped peers (also gets rid of the ugly
  getAllPeers callback in requestDistributor)
- moves peerSet, LesOdr, requestDistributor and retrieveManager initialization
  out of ProtocolManager because I believe they do not really belong there and the
  whole init process was ugly and ad-hoc
  • Loading branch information
zsfelfoldi authored and fjl committed Jun 21, 2017
1 parent 60e27b5 commit a5d08c8
Show file tree
Hide file tree
Showing 15 changed files with 699 additions and 441 deletions.
40 changes: 28 additions & 12 deletions les/backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package les

import (
"fmt"
"sync"
"time"

"github.com/ethereum/go-ethereum/accounts"
Expand All @@ -38,6 +39,7 @@ import (
"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/params"
rpc "github.com/ethereum/go-ethereum/rpc"
)
Expand All @@ -49,9 +51,13 @@ type LightEthereum struct {
// Channel for shutting down the service
shutdownChan chan bool
// Handlers
peers *peerSet
txPool *light.TxPool
blockchain *light.LightChain
protocolManager *ProtocolManager
serverPool *serverPool
reqDist *requestDistributor
retriever *retrieveManager
// DB interfaces
chainDb ethdb.Database // Block chain database

Expand All @@ -63,6 +69,9 @@ type LightEthereum struct {

networkId uint64
netRPCService *ethapi.PublicNetAPI

quitSync chan struct{}
wg sync.WaitGroup
}

func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
Expand All @@ -76,20 +85,26 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
}
log.Info("Initialised chain configuration", "config", chainConfig)

odr := NewLesOdr(chainDb)
relay := NewLesTxRelay()
peers := newPeerSet()
quitSync := make(chan struct{})

eth := &LightEthereum{
odr: odr,
relay: relay,
chainDb: chainDb,
chainConfig: chainConfig,
chainDb: chainDb,
eventMux: ctx.EventMux,
peers: peers,
reqDist: newRequestDistributor(peers, quitSync),
accountManager: ctx.AccountManager,
engine: eth.CreateConsensusEngine(ctx, config, chainConfig, chainDb),
shutdownChan: make(chan bool),
networkId: config.NetworkId,
}
if eth.blockchain, err = light.NewLightChain(odr, eth.chainConfig, eth.engine, eth.eventMux); err != nil {

eth.relay = NewLesTxRelay(peers, eth.reqDist)
eth.serverPool = newServerPool(chainDb, quitSync, &eth.wg)
eth.retriever = newRetrieveManager(peers, eth.reqDist, eth.serverPool)
eth.odr = NewLesOdr(chainDb, eth.retriever)
if eth.blockchain, err = light.NewLightChain(eth.odr, eth.chainConfig, eth.engine, eth.eventMux); err != nil {
return nil, err
}
// Rewind the chain in case of an incompatible config upgrade.
Expand All @@ -100,13 +115,9 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
}

eth.txPool = light.NewTxPool(eth.chainConfig, eth.eventMux, eth.blockchain, eth.relay)
lightSync := config.SyncMode == downloader.LightSync
if eth.protocolManager, err = NewProtocolManager(eth.chainConfig, lightSync, config.NetworkId, eth.eventMux, eth.engine, eth.blockchain, nil, chainDb, odr, relay); err != nil {
if eth.protocolManager, err = NewProtocolManager(eth.chainConfig, true, config.NetworkId, eth.eventMux, eth.engine, eth.peers, eth.blockchain, nil, chainDb, eth.odr, eth.relay, quitSync, &eth.wg); err != nil {
return nil, err
}
relay.ps = eth.protocolManager.peers
relay.reqDist = eth.protocolManager.reqDist

eth.ApiBackend = &LesApiBackend{eth, nil}
gpoParams := config.GPO
if gpoParams.Default == nil {
Expand All @@ -116,6 +127,10 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
return eth, nil
}

func lesTopic(genesisHash common.Hash) discv5.Topic {
return discv5.Topic("LES@" + common.Bytes2Hex(genesisHash.Bytes()[0:8]))
}

type LightDummyAPI struct{}

// Etherbase is the address that mining rewards will be send to
Expand Down Expand Up @@ -188,7 +203,8 @@ func (s *LightEthereum) Protocols() []p2p.Protocol {
func (s *LightEthereum) Start(srvr *p2p.Server) error {
log.Warn("Light client mode is an experimental feature")
s.netRPCService = ethapi.NewPublicNetAPI(srvr, s.networkId)
s.protocolManager.Start(srvr)
s.serverPool.start(srvr, lesTopic(s.blockchain.Genesis().Hash()))
s.protocolManager.Start()
return nil
}

Expand Down
58 changes: 42 additions & 16 deletions les/distributor.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,11 @@ var ErrNoPeers = errors.New("no suitable peers available")
type requestDistributor struct {
reqQueue *list.List
lastReqOrder uint64
peers map[distPeer]struct{}
peerLock sync.RWMutex
stopChn, loopChn chan struct{}
loopNextSent bool
lock sync.Mutex

getAllPeers func() map[distPeer]struct{}
}

// distPeer is an LES server peer interface for the request distributor.
Expand Down Expand Up @@ -71,15 +71,39 @@ type distReq struct {
}

// newRequestDistributor creates a new request distributor
func newRequestDistributor(getAllPeers func() map[distPeer]struct{}, stopChn chan struct{}) *requestDistributor {
r := &requestDistributor{
reqQueue: list.New(),
loopChn: make(chan struct{}, 2),
stopChn: stopChn,
getAllPeers: getAllPeers,
func newRequestDistributor(peers *peerSet, stopChn chan struct{}) *requestDistributor {
d := &requestDistributor{
reqQueue: list.New(),
loopChn: make(chan struct{}, 2),
stopChn: stopChn,
peers: make(map[distPeer]struct{}),
}
if peers != nil {
peers.notify(d)
}
go r.loop()
return r
go d.loop()
return d
}

// registerPeer implements peerSetNotify
func (d *requestDistributor) registerPeer(p *peer) {
d.peerLock.Lock()
d.peers[p] = struct{}{}
d.peerLock.Unlock()
}

// unregisterPeer implements peerSetNotify
func (d *requestDistributor) unregisterPeer(p *peer) {
d.peerLock.Lock()
delete(d.peers, p)
d.peerLock.Unlock()
}

// registerTestPeer adds a new test peer
func (d *requestDistributor) registerTestPeer(p distPeer) {
d.peerLock.Lock()
d.peers[p] = struct{}{}
d.peerLock.Unlock()
}

// distMaxWait is the maximum waiting time after which further necessary waiting
Expand Down Expand Up @@ -152,8 +176,7 @@ func (sp selectPeerItem) Weight() int64 {
// nextRequest returns the next possible request from any peer, along with the
// associated peer and necessary waiting time
func (d *requestDistributor) nextRequest() (distPeer, *distReq, time.Duration) {
peers := d.getAllPeers()

checkedPeers := make(map[distPeer]struct{})
elem := d.reqQueue.Front()
var (
bestPeer distPeer
Expand All @@ -162,11 +185,14 @@ func (d *requestDistributor) nextRequest() (distPeer, *distReq, time.Duration) {
sel *weightedRandomSelect
)

for (len(peers) > 0 || elem == d.reqQueue.Front()) && elem != nil {
d.peerLock.RLock()
defer d.peerLock.RUnlock()

for (len(d.peers) > 0 || elem == d.reqQueue.Front()) && elem != nil {
req := elem.Value.(*distReq)
canSend := false
for peer, _ := range peers {
if peer.canQueue() && req.canSend(peer) {
for peer, _ := range d.peers {
if _, ok := checkedPeers[peer]; !ok && peer.canQueue() && req.canSend(peer) {
canSend = true
cost := req.getCost(peer)
wait, bufRemain := peer.waitBefore(cost)
Expand All @@ -182,7 +208,7 @@ func (d *requestDistributor) nextRequest() (distPeer, *distReq, time.Duration) {
bestWait = wait
}
}
delete(peers, peer)
checkedPeers[peer] = struct{}{}
}
}
next := elem.Next()
Expand Down
10 changes: 2 additions & 8 deletions les/distributor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,20 +122,14 @@ func testRequestDistributor(t *testing.T, resend bool) {
stop := make(chan struct{})
defer close(stop)

dist := newRequestDistributor(nil, stop)
var peers [testDistPeerCount]*testDistPeer
for i, _ := range peers {
peers[i] = &testDistPeer{}
go peers[i].worker(t, !resend, stop)
dist.registerTestPeer(peers[i])
}

dist := newRequestDistributor(func() map[distPeer]struct{} {
m := make(map[distPeer]struct{})
for _, peer := range peers {
m[peer] = struct{}{}
}
return m
}, stop)

var wg sync.WaitGroup

for i := 1; i <= testDistReqCount; i++ {
Expand Down
11 changes: 6 additions & 5 deletions les/fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ func newLightFetcher(pm *ProtocolManager) *lightFetcher {
syncDone: make(chan *peer),
maxConfirmedTd: big.NewInt(0),
}
pm.peers.notify(f)
go f.syncLoop()
return f
}
Expand Down Expand Up @@ -209,8 +210,8 @@ func (f *lightFetcher) syncLoop() {
}
}

// addPeer adds a new peer to the fetcher's peer set
func (f *lightFetcher) addPeer(p *peer) {
// registerPeer adds a new peer to the fetcher's peer set
func (f *lightFetcher) registerPeer(p *peer) {
p.lock.Lock()
p.hasBlock = func(hash common.Hash, number uint64) bool {
return f.peerHasBlock(p, hash, number)
Expand All @@ -223,8 +224,8 @@ func (f *lightFetcher) addPeer(p *peer) {
f.peers[p] = &fetcherPeerInfo{nodeByHash: make(map[common.Hash]*fetcherTreeNode)}
}

// removePeer removes a new peer from the fetcher's peer set
func (f *lightFetcher) removePeer(p *peer) {
// unregisterPeer removes a new peer from the fetcher's peer set
func (f *lightFetcher) unregisterPeer(p *peer) {
p.lock.Lock()
p.hasBlock = nil
p.lock.Unlock()
Expand Down Expand Up @@ -416,7 +417,7 @@ func (f *lightFetcher) nextRequest() (*distReq, uint64) {
f.syncing = bestSyncing

var rq *distReq
reqID := getNextReqID()
reqID := genReqID()
if f.syncing {
rq = &distReq{
getCost: func(dp distPeer) uint64 {
Expand Down
Loading

0 comments on commit a5d08c8

Please sign in to comment.