eth: check snap satelliteness, delegate drop to eth (#22235)

* eth: check snap satelliteness, delegate drop to eth

* eth: better handle eth/snap satellite relation, merge reg/unreg paths
pull/22270/head
Péter Szilágyi 4 years ago committed by GitHub
parent 3c728fb129
commit e3430ac7df
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 94
      eth/handler.go
  2. 4
      eth/handler_eth.go
  3. 4
      eth/handler_eth_test.go
  4. 8
      eth/handler_snap.go
  5. 5
      eth/peer.go
  6. 255
      eth/peerset.go
  7. 6
      eth/protocols/eth/handler.go
  8. 8
      eth/protocols/eth/protocol.go
  9. 6
      eth/protocols/snap/handler.go
  10. 8
      eth/protocols/snap/protocol.go
  11. 4
      eth/sync.go
  12. 2
      eth/sync_test.go
  13. 11
      p2p/peer.go

@ -218,7 +218,7 @@ func newHandler(config *handlerConfig) (*handler, error) {
h.blockFetcher = fetcher.NewBlockFetcher(false, nil, h.chain.GetBlockByHash, validator, h.BroadcastBlock, heighter, nil, inserter, h.removePeer) h.blockFetcher = fetcher.NewBlockFetcher(false, nil, h.chain.GetBlockByHash, validator, h.BroadcastBlock, heighter, nil, inserter, h.removePeer)
fetchTx := func(peer string, hashes []common.Hash) error { fetchTx := func(peer string, hashes []common.Hash) error {
p := h.peers.ethPeer(peer) p := h.peers.peer(peer)
if p == nil { if p == nil {
return errors.New("unknown peer") return errors.New("unknown peer")
} }
@ -229,8 +229,17 @@ func newHandler(config *handlerConfig) (*handler, error) {
return h, nil return h, nil
} }
// runEthPeer // runEthPeer registers an eth peer into the joint eth/snap peerset, adds it to
// various subsistems and starts handling messages.
func (h *handler) runEthPeer(peer *eth.Peer, handler eth.Handler) error { func (h *handler) runEthPeer(peer *eth.Peer, handler eth.Handler) error {
// If the peer has a `snap` extension, wait for it to connect so we can have
// a uniform initialization/teardown mechanism
snap, err := h.peers.waitSnapExtension(peer)
if err != nil {
peer.Log().Error("Snapshot extension barrier failed", "err", err)
return err
}
// TODO(karalabe): Not sure why this is needed
if !h.chainSync.handlePeerEvent(peer) { if !h.chainSync.handlePeerEvent(peer) {
return p2p.DiscQuitting return p2p.DiscQuitting
} }
@ -251,37 +260,46 @@ func (h *handler) runEthPeer(peer *eth.Peer, handler eth.Handler) error {
return err return err
} }
reject := false // reserved peer slots reject := false // reserved peer slots
if atomic.LoadUint32(&h.snapSync) == 1 && !peer.SupportsCap("snap", 1) { if atomic.LoadUint32(&h.snapSync) == 1 {
if snap == nil {
// If we are running snap-sync, we want to reserve roughly half the peer // If we are running snap-sync, we want to reserve roughly half the peer
// slots for peers supporting the snap protocol. // slots for peers supporting the snap protocol.
// The logic here is; we only allow up to 5 more non-snap peers than snap-peers. // The logic here is; we only allow up to 5 more non-snap peers than snap-peers.
if all, snp := h.peers.Len(), h.peers.SnapLen(); all-snp > snp+5 { if all, snp := h.peers.len(), h.peers.snapLen(); all-snp > snp+5 {
reject = true reject = true
} }
} }
}
// Ignore maxPeers if this is a trusted peer // Ignore maxPeers if this is a trusted peer
if !peer.Peer.Info().Network.Trusted { if !peer.Peer.Info().Network.Trusted {
if reject || h.peers.Len() >= h.maxPeers { if reject || h.peers.len() >= h.maxPeers {
return p2p.DiscTooManyPeers return p2p.DiscTooManyPeers
} }
} }
peer.Log().Debug("Ethereum peer connected", "name", peer.Name()) peer.Log().Debug("Ethereum peer connected", "name", peer.Name())
// Register the peer locally // Register the peer locally
if err := h.peers.registerEthPeer(peer); err != nil { if err := h.peers.registerPeer(peer, snap); err != nil {
peer.Log().Error("Ethereum peer registration failed", "err", err) peer.Log().Error("Ethereum peer registration failed", "err", err)
return err return err
} }
defer h.removePeer(peer.ID()) defer h.removePeer(peer.ID())
p := h.peers.ethPeer(peer.ID()) p := h.peers.peer(peer.ID())
if p == nil { if p == nil {
return errors.New("peer dropped during handling") return errors.New("peer dropped during handling")
} }
// Register the peer in the downloader. If the downloader considers it banned, we disconnect // Register the peer in the downloader. If the downloader considers it banned, we disconnect
if err := h.downloader.RegisterPeer(peer.ID(), peer.Version(), peer); err != nil { if err := h.downloader.RegisterPeer(peer.ID(), peer.Version(), peer); err != nil {
peer.Log().Error("Failed to register peer in eth syncer", "err", err)
return err return err
} }
if snap != nil {
if err := h.downloader.SnapSyncer.Register(snap); err != nil {
peer.Log().Error("Failed to register peer in snap syncer", "err", err)
return err
}
}
h.chainSync.handlePeerEvent(peer) h.chainSync.handlePeerEvent(peer)
// Propagate existing transactions. new transactions appearing // Propagate existing transactions. new transactions appearing
@ -317,25 +335,23 @@ func (h *handler) runEthPeer(peer *eth.Peer, handler eth.Handler) error {
return handler(peer) return handler(peer)
} }
// runSnapPeer // runSnapExtension registers a `snap` peer into the joint eth/snap peerset and
func (h *handler) runSnapPeer(peer *snap.Peer, handler snap.Handler) error { // starts handling inbound messages. As `snap` is only a satellite protocol to
// `eth`, all subsystem registrations and lifecycle management will be done by
// the main `eth` handler to prevent strange races.
func (h *handler) runSnapExtension(peer *snap.Peer, handler snap.Handler) error {
h.peerWG.Add(1) h.peerWG.Add(1)
defer h.peerWG.Done() defer h.peerWG.Done()
// Register the peer locally if err := h.peers.registerSnapExtension(peer); err != nil {
if err := h.peers.registerSnapPeer(peer); err != nil { peer.Log().Error("Snapshot extension registration failed", "err", err)
peer.Log().Error("Snapshot peer registration failed", "err", err)
return err
}
defer h.removePeer(peer.ID())
if err := h.downloader.SnapSyncer.Register(peer); err != nil {
return err return err
} }
// Handle incoming messages until the connection is torn down
return handler(peer) return handler(peer)
} }
// removePeer unregisters a peer from the downloader and fetchers, removes it from
// the set of tracked peers and closes the network connection to it.
func (h *handler) removePeer(id string) { func (h *handler) removePeer(id string) {
// Create a custom logger to avoid printing the entire id // Create a custom logger to avoid printing the entire id
var logger log.Logger var logger log.Logger
@ -345,33 +361,27 @@ func (h *handler) removePeer(id string) {
} else { } else {
logger = log.New("peer", id[:8]) logger = log.New("peer", id[:8])
} }
// Remove the eth peer if it exists // Abort if the peer does not exist
eth := h.peers.ethPeer(id) peer := h.peers.peer(id)
if eth != nil { if peer == nil {
logger.Debug("Removing Ethereum peer") logger.Error("Ethereum peer removal failed", "err", errPeerNotRegistered)
return
}
// Remove the `eth` peer if it exists
logger.Debug("Removing Ethereum peer", "snap", peer.snapExt != nil)
// Remove the `snap` extension if it exists
if peer.snapExt != nil {
h.downloader.SnapSyncer.Unregister(id)
}
h.downloader.UnregisterPeer(id) h.downloader.UnregisterPeer(id)
h.txFetcher.Drop(id) h.txFetcher.Drop(id)
if err := h.peers.unregisterEthPeer(id); err != nil { if err := h.peers.unregisterPeer(id); err != nil {
logger.Error("Ethereum peer removal failed", "err", err) logger.Error("Ethereum peer removal failed", "err", err)
} }
}
// Remove the snap peer if it exists
snap := h.peers.snapPeer(id)
if snap != nil {
logger.Debug("Removing Snapshot peer")
h.downloader.SnapSyncer.Unregister(id)
if err := h.peers.unregisterSnapPeer(id); err != nil {
logger.Error("Snapshot peer removel failed", "err", err)
}
}
// Hard disconnect at the networking layer // Hard disconnect at the networking layer
if eth != nil { peer.Peer.Disconnect(p2p.DiscUselessPeer)
eth.Peer.Disconnect(p2p.DiscUselessPeer)
}
if snap != nil {
snap.Peer.Disconnect(p2p.DiscUselessPeer)
}
} }
func (h *handler) Start(maxPeers int) { func (h *handler) Start(maxPeers int) {
@ -417,7 +427,7 @@ func (h *handler) Stop() {
// will only announce its availability (depending what's requested). // will only announce its availability (depending what's requested).
func (h *handler) BroadcastBlock(block *types.Block, propagate bool) { func (h *handler) BroadcastBlock(block *types.Block, propagate bool) {
hash := block.Hash() hash := block.Hash()
peers := h.peers.ethPeersWithoutBlock(hash) peers := h.peers.peersWithoutBlock(hash)
// If propagation is requested, send to a subset of the peer // If propagation is requested, send to a subset of the peer
if propagate { if propagate {
@ -456,7 +466,7 @@ func (h *handler) BroadcastTransactions(txs types.Transactions, propagate bool)
// Broadcast transactions to a batch of peers not knowing about it // Broadcast transactions to a batch of peers not knowing about it
if propagate { if propagate {
for _, tx := range txs { for _, tx := range txs {
peers := h.peers.ethPeersWithoutTransaction(tx.Hash()) peers := h.peers.peersWithoutTransaction(tx.Hash())
// Send the block to a subset of our peers // Send the block to a subset of our peers
transfer := peers[:int(math.Sqrt(float64(len(peers))))] transfer := peers[:int(math.Sqrt(float64(len(peers))))]
@ -472,7 +482,7 @@ func (h *handler) BroadcastTransactions(txs types.Transactions, propagate bool)
} }
// Otherwise only broadcast the announcement to peers // Otherwise only broadcast the announcement to peers
for _, tx := range txs { for _, tx := range txs {
peers := h.peers.ethPeersWithoutTransaction(tx.Hash()) peers := h.peers.peersWithoutTransaction(tx.Hash())
for _, peer := range peers { for _, peer := range peers {
annos[peer] = append(annos[peer], tx.Hash()) annos[peer] = append(annos[peer], tx.Hash())
} }

@ -47,7 +47,7 @@ func (h *ethHandler) RunPeer(peer *eth.Peer, hand eth.Handler) error {
// PeerInfo retrieves all known `eth` information about a peer. // PeerInfo retrieves all known `eth` information about a peer.
func (h *ethHandler) PeerInfo(id enode.ID) interface{} { func (h *ethHandler) PeerInfo(id enode.ID) interface{} {
if p := h.peers.ethPeer(id.String()); p != nil { if p := h.peers.peer(id.String()); p != nil {
return p.info() return p.info()
} }
return nil return nil
@ -107,7 +107,7 @@ func (h *ethHandler) Handle(peer *eth.Peer, packet eth.Packet) error {
// handleHeaders is invoked from a peer's message handler when it transmits a batch // handleHeaders is invoked from a peer's message handler when it transmits a batch
// of headers for the local node to process. // of headers for the local node to process.
func (h *ethHandler) handleHeaders(peer *eth.Peer, headers []*types.Header) error { func (h *ethHandler) handleHeaders(peer *eth.Peer, headers []*types.Header) error {
p := h.peers.ethPeer(peer.ID()) p := h.peers.peer(peer.ID())
if p == nil { if p == nil {
return errors.New("unregistered during callback") return errors.New("unregistered during callback")
} }

@ -574,11 +574,11 @@ func testCheckpointChallenge(t *testing.T, syncmode downloader.SyncMode, checkpo
// Verify that the remote peer is maintained or dropped // Verify that the remote peer is maintained or dropped
if drop { if drop {
if peers := handler.handler.peers.Len(); peers != 0 { if peers := handler.handler.peers.len(); peers != 0 {
t.Fatalf("peer count mismatch: have %d, want %d", peers, 0) t.Fatalf("peer count mismatch: have %d, want %d", peers, 0)
} }
} else { } else {
if peers := handler.handler.peers.Len(); peers != 1 { if peers := handler.handler.peers.len(); peers != 1 {
t.Fatalf("peer count mismatch: have %d, want %d", peers, 1) t.Fatalf("peer count mismatch: have %d, want %d", peers, 1)
} }
} }

@ -30,13 +30,15 @@ func (h *snapHandler) Chain() *core.BlockChain { return h.chain }
// RunPeer is invoked when a peer joins on the `snap` protocol. // RunPeer is invoked when a peer joins on the `snap` protocol.
func (h *snapHandler) RunPeer(peer *snap.Peer, hand snap.Handler) error { func (h *snapHandler) RunPeer(peer *snap.Peer, hand snap.Handler) error {
return (*handler)(h).runSnapPeer(peer, hand) return (*handler)(h).runSnapExtension(peer, hand)
} }
// PeerInfo retrieves all known `snap` information about a peer. // PeerInfo retrieves all known `snap` information about a peer.
func (h *snapHandler) PeerInfo(id enode.ID) interface{} { func (h *snapHandler) PeerInfo(id enode.ID) interface{} {
if p := h.peers.snapPeer(id.String()); p != nil { if p := h.peers.peer(id.String()); p != nil {
return p.info() if p.snapExt != nil {
return p.snapExt.info()
}
} }
return nil return nil
} }

@ -36,8 +36,10 @@ type ethPeerInfo struct {
// ethPeer is a wrapper around eth.Peer to maintain a few extra metadata. // ethPeer is a wrapper around eth.Peer to maintain a few extra metadata.
type ethPeer struct { type ethPeer struct {
*eth.Peer *eth.Peer
snapExt *snapPeer // Satellite `snap` connection
syncDrop *time.Timer // Connection dropper if `eth` sync progress isn't validated in time syncDrop *time.Timer // Connection dropper if `eth` sync progress isn't validated in time
snapWait chan struct{} // Notification channel for snap connections
lock sync.RWMutex // Mutex protecting the internal fields lock sync.RWMutex // Mutex protecting the internal fields
} }
@ -61,9 +63,6 @@ type snapPeerInfo struct {
// snapPeer is a wrapper around snap.Peer to maintain a few extra metadata. // snapPeer is a wrapper around snap.Peer to maintain a few extra metadata.
type snapPeer struct { type snapPeer struct {
*snap.Peer *snap.Peer
ethDrop *time.Timer // Connection dropper if `eth` doesn't connect in time
lock sync.RWMutex // Mutex protecting the internal fields
} }
// info gathers and returns some `snap` protocol metadata known about a peer. // info gathers and returns some `snap` protocol metadata known about a peer.

@ -20,12 +20,10 @@ import (
"errors" "errors"
"math/big" "math/big"
"sync" "sync"
"time"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/eth/protocols/eth" "github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/eth/protocols/snap" "github.com/ethereum/go-ethereum/eth/protocols/snap"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p"
) )
@ -42,22 +40,19 @@ var (
// a peer set, but no peer with the given id exists. // a peer set, but no peer with the given id exists.
errPeerNotRegistered = errors.New("peer not registered") errPeerNotRegistered = errors.New("peer not registered")
// ethConnectTimeout is the `snap` timeout for `eth` to connect too. // errSnapWithoutEth is returned if a peer attempts to connect only on the
ethConnectTimeout = 3 * time.Second // snap protocol without advertizing the eth main protocol.
errSnapWithoutEth = errors.New("peer connected on snap without compatible eth support")
) )
// peerSet represents the collection of active peers currently participating in // peerSet represents the collection of active peers currently participating in
// the `eth` or `snap` protocols. // the `eth` protocol, with or without the `snap` extension.
type peerSet struct { type peerSet struct {
ethPeers map[string]*ethPeer // Peers connected on the `eth` protocol peers map[string]*ethPeer // Peers connected on the `eth` protocol
snapPeers map[string]*snapPeer // Peers connected on the `snap` protocol snapPeers int // Number of `snap` compatible peers for connection prioritization
ethJoinFeed event.Feed // Events when an `eth` peer successfully joins snapWait map[string]chan *snap.Peer // Peers connected on `eth` waiting for their snap extension
ethDropFeed event.Feed // Events when an `eth` peer gets dropped snapPend map[string]*snap.Peer // Peers connected on the `snap` protocol, but not yet on `eth`
snapJoinFeed event.Feed // Events when a `snap` peer joins on both `eth` and `snap`
snapDropFeed event.Feed // Events when a `snap` peer gets dropped (only if fully joined)
scope event.SubscriptionScope // Subscription group to unsubscribe everyone at once
lock sync.RWMutex lock sync.RWMutex
closed bool closed bool
@ -66,176 +61,134 @@ type peerSet struct {
// newPeerSet creates a new peer set to track the active participants. // newPeerSet creates a new peer set to track the active participants.
func newPeerSet() *peerSet { func newPeerSet() *peerSet {
return &peerSet{ return &peerSet{
ethPeers: make(map[string]*ethPeer), peers: make(map[string]*ethPeer),
snapPeers: make(map[string]*snapPeer), snapWait: make(map[string]chan *snap.Peer),
snapPend: make(map[string]*snap.Peer),
} }
} }
// subscribeEthJoin registers a subscription for peers joining (and completing // registerSnapExtension unblocks an already connected `eth` peer waiting for its
// the handshake) on the `eth` protocol. // `snap` extension, or if no such peer exists, tracks the extension for the time
func (ps *peerSet) subscribeEthJoin(ch chan<- *eth.Peer) event.Subscription { // being until the `eth` main protocol starts looking for it.
return ps.scope.Track(ps.ethJoinFeed.Subscribe(ch)) func (ps *peerSet) registerSnapExtension(peer *snap.Peer) error {
// Reject the peer if it advertises `snap` without `eth` as `snap` is only a
// satellite protocol meaningful with the chain selection of `eth`
if !peer.SupportsCap(eth.ProtocolName, eth.ProtocolVersions) {
return errSnapWithoutEth
} }
// Ensure nobody can double connect
ps.lock.Lock()
defer ps.lock.Unlock()
// subscribeEthDrop registers a subscription for peers being dropped from the id := peer.ID()
// `eth` protocol. if _, ok := ps.peers[id]; ok {
func (ps *peerSet) subscribeEthDrop(ch chan<- *eth.Peer) event.Subscription { return errPeerAlreadyRegistered // avoid connections with the same id as existing ones
return ps.scope.Track(ps.ethDropFeed.Subscribe(ch))
} }
if _, ok := ps.snapPend[id]; ok {
// subscribeSnapJoin registers a subscription for peers joining (and completing return errPeerAlreadyRegistered // avoid connections with the same id as pending ones
// the `eth` join) on the `snap` protocol.
func (ps *peerSet) subscribeSnapJoin(ch chan<- *snap.Peer) event.Subscription {
return ps.scope.Track(ps.snapJoinFeed.Subscribe(ch))
} }
// Inject the peer into an `eth` counterpart is available, otherwise save for later
// subscribeSnapDrop registers a subscription for peers being dropped from the if wait, ok := ps.snapWait[id]; ok {
// `snap` protocol. delete(ps.snapWait, id)
func (ps *peerSet) subscribeSnapDrop(ch chan<- *snap.Peer) event.Subscription { wait <- peer
return ps.scope.Track(ps.snapDropFeed.Subscribe(ch)) return nil
}
ps.snapPend[id] = peer
return nil
} }
// registerEthPeer injects a new `eth` peer into the working set, or returns an // waitExtensions blocks until all satellite protocols are connected and tracked
// error if the peer is already known. The peer is announced on the `eth` join // by the peerset.
// feed and if it completes a pending `snap` peer, also on that feed. func (ps *peerSet) waitSnapExtension(peer *eth.Peer) (*snap.Peer, error) {
func (ps *peerSet) registerEthPeer(peer *eth.Peer) error { // If the peer does not support a compatible `snap`, don't wait
ps.lock.Lock() if !peer.SupportsCap(snap.ProtocolName, snap.ProtocolVersions) {
if ps.closed { return nil, nil
ps.lock.Unlock()
return errPeerSetClosed
} }
// Ensure nobody can double connect
ps.lock.Lock()
id := peer.ID() id := peer.ID()
if _, ok := ps.ethPeers[id]; ok { if _, ok := ps.peers[id]; ok {
ps.lock.Unlock() ps.lock.Unlock()
return errPeerAlreadyRegistered return nil, errPeerAlreadyRegistered // avoid connections with the same id as existing ones
} }
ps.ethPeers[id] = &ethPeer{Peer: peer} if _, ok := ps.snapWait[id]; ok {
snap, ok := ps.snapPeers[id]
ps.lock.Unlock() ps.lock.Unlock()
return nil, errPeerAlreadyRegistered // avoid connections with the same id as pending ones
if ok {
// Previously dangling `snap` peer, stop it's timer since `eth` connected
snap.lock.Lock()
if snap.ethDrop != nil {
snap.ethDrop.Stop()
snap.ethDrop = nil
}
snap.lock.Unlock()
}
ps.ethJoinFeed.Send(peer)
if ok {
ps.snapJoinFeed.Send(snap.Peer)
}
return nil
} }
// If `snap` already connected, retrieve the peer from the pending set
if snap, ok := ps.snapPend[id]; ok {
delete(ps.snapPend, id)
// unregisterEthPeer removes a remote peer from the active set, disabling any further
// actions to/from that particular entity. The drop is announced on the `eth` drop
// feed and also on the `snap` feed if the eth/snap duality was broken just now.
func (ps *peerSet) unregisterEthPeer(id string) error {
ps.lock.Lock()
eth, ok := ps.ethPeers[id]
if !ok {
ps.lock.Unlock() ps.lock.Unlock()
return errPeerNotRegistered return snap, nil
} }
delete(ps.ethPeers, id) // Otherwise wait for `snap` to connect concurrently
wait := make(chan *snap.Peer)
snap, ok := ps.snapPeers[id] ps.snapWait[id] = wait
ps.lock.Unlock() ps.lock.Unlock()
ps.ethDropFeed.Send(eth) return <-wait, nil
if ok {
ps.snapDropFeed.Send(snap)
}
return nil
} }
// registerSnapPeer injects a new `snap` peer into the working set, or returns // registerPeer injects a new `eth` peer into the working set, or returns an error
// an error if the peer is already known. The peer is announced on the `snap` // if the peer is already known.
// join feed if it completes an existing `eth` peer. func (ps *peerSet) registerPeer(peer *eth.Peer, ext *snap.Peer) error {
// // Start tracking the new peer
// If the peer isn't yet connected on `eth` and fails to do so within a given
// amount of time, it is dropped. This enforces that `snap` is an extension to
// `eth`, not a standalone leeching protocol.
func (ps *peerSet) registerSnapPeer(peer *snap.Peer) error {
ps.lock.Lock() ps.lock.Lock()
defer ps.lock.Unlock()
if ps.closed { if ps.closed {
ps.lock.Unlock()
return errPeerSetClosed return errPeerSetClosed
} }
id := peer.ID() id := peer.ID()
if _, ok := ps.snapPeers[id]; ok { if _, ok := ps.peers[id]; ok {
ps.lock.Unlock()
return errPeerAlreadyRegistered return errPeerAlreadyRegistered
} }
ps.snapPeers[id] = &snapPeer{Peer: peer} eth := &ethPeer{
Peer: peer,
_, ok := ps.ethPeers[id]
if !ok {
// Dangling `snap` peer, start a timer to drop if `eth` doesn't connect
ps.snapPeers[id].ethDrop = time.AfterFunc(ethConnectTimeout, func() {
peer.Log().Warn("Snapshot peer missing eth, dropping", "addr", peer.RemoteAddr(), "type", peer.Name())
peer.Disconnect(p2p.DiscUselessPeer)
})
} }
ps.lock.Unlock() if ext != nil {
eth.snapExt = &snapPeer{ext}
if ok { ps.snapPeers++
ps.snapJoinFeed.Send(peer)
} }
ps.peers[id] = eth
return nil return nil
} }
// unregisterSnapPeer removes a remote peer from the active set, disabling any // unregisterPeer removes a remote peer from the active set, disabling any further
// further actions to/from that particular entity. The drop is announced on the // actions to/from that particular entity.
// `snap` drop feed. func (ps *peerSet) unregisterPeer(id string) error {
func (ps *peerSet) unregisterSnapPeer(id string) error {
ps.lock.Lock() ps.lock.Lock()
peer, ok := ps.snapPeers[id] defer ps.lock.Unlock()
peer, ok := ps.peers[id]
if !ok { if !ok {
ps.lock.Unlock()
return errPeerNotRegistered return errPeerNotRegistered
} }
delete(ps.snapPeers, id) delete(ps.peers, id)
ps.lock.Unlock() if peer.snapExt != nil {
ps.snapPeers--
peer.lock.Lock()
if peer.ethDrop != nil {
peer.ethDrop.Stop()
peer.ethDrop = nil
} }
peer.lock.Unlock()
ps.snapDropFeed.Send(peer)
return nil return nil
} }
// ethPeer retrieves the registered `eth` peer with the given id. // peer retrieves the registered peer with the given id.
func (ps *peerSet) ethPeer(id string) *ethPeer { func (ps *peerSet) peer(id string) *ethPeer {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
return ps.ethPeers[id] return ps.peers[id]
} }
// snapPeer retrieves the registered `snap` peer with the given id. // peersWithoutBlock retrieves a list of peers that do not have a given block in
func (ps *peerSet) snapPeer(id string) *snapPeer { // their set of known hashes so it might be propagated to them.
func (ps *peerSet) peersWithoutBlock(hash common.Hash) []*ethPeer {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
return ps.snapPeers[id] list := make([]*ethPeer, 0, len(ps.peers))
} for _, p := range ps.peers {
// ethPeersWithoutBlock retrieves a list of `eth` peers that do not have a given
// block in their set of known hashes so it might be propagated to them.
func (ps *peerSet) ethPeersWithoutBlock(hash common.Hash) []*ethPeer {
ps.lock.RLock()
defer ps.lock.RUnlock()
list := make([]*ethPeer, 0, len(ps.ethPeers))
for _, p := range ps.ethPeers {
if !p.KnownBlock(hash) { if !p.KnownBlock(hash) {
list = append(list, p) list = append(list, p)
} }
@ -243,14 +196,14 @@ func (ps *peerSet) ethPeersWithoutBlock(hash common.Hash) []*ethPeer {
return list return list
} }
// ethPeersWithoutTransaction retrieves a list of `eth` peers that do not have a // peersWithoutTransaction retrieves a list of peers that do not have a given
// given transaction in their set of known hashes. // transaction in their set of known hashes.
func (ps *peerSet) ethPeersWithoutTransaction(hash common.Hash) []*ethPeer { func (ps *peerSet) peersWithoutTransaction(hash common.Hash) []*ethPeer {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
list := make([]*ethPeer, 0, len(ps.ethPeers)) list := make([]*ethPeer, 0, len(ps.peers))
for _, p := range ps.ethPeers { for _, p := range ps.peers {
if !p.KnownTransaction(hash) { if !p.KnownTransaction(hash) {
list = append(list, p) list = append(list, p)
} }
@ -258,28 +211,27 @@ func (ps *peerSet) ethPeersWithoutTransaction(hash common.Hash) []*ethPeer {
return list return list
} }
// Len returns if the current number of `eth` peers in the set. Since the `snap` // len returns if the current number of `eth` peers in the set. Since the `snap`
// peers are tied to the existence of an `eth` connection, that will always be a // peers are tied to the existence of an `eth` connection, that will always be a
// subset of `eth`. // subset of `eth`.
func (ps *peerSet) Len() int { func (ps *peerSet) len() int {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
return len(ps.ethPeers) return len(ps.peers)
} }
// SnapLen returns if the current number of `snap` peers in the set. Since the `snap` // snapLen returns if the current number of `snap` peers in the set.
// peers are tied to the existence of an `eth` connection, that will always be a func (ps *peerSet) snapLen() int {
// subset of `eth`.
func (ps *peerSet) SnapLen() int {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
return len(ps.snapPeers)
return ps.snapPeers
} }
// ethPeerWithHighestTD retrieves the known peer with the currently highest total // peerWithHighestTD retrieves the known peer with the currently highest total
// difficulty. // difficulty.
func (ps *peerSet) ethPeerWithHighestTD() *eth.Peer { func (ps *peerSet) peerWithHighestTD() *eth.Peer {
ps.lock.RLock() ps.lock.RLock()
defer ps.lock.RUnlock() defer ps.lock.RUnlock()
@ -287,7 +239,7 @@ func (ps *peerSet) ethPeerWithHighestTD() *eth.Peer {
bestPeer *eth.Peer bestPeer *eth.Peer
bestTd *big.Int bestTd *big.Int
) )
for _, p := range ps.ethPeers { for _, p := range ps.peers {
if _, td := p.Head(); bestPeer == nil || td.Cmp(bestTd) > 0 { if _, td := p.Head(); bestPeer == nil || td.Cmp(bestTd) > 0 {
bestPeer, bestTd = p.Peer, td bestPeer, bestTd = p.Peer, td
} }
@ -300,10 +252,7 @@ func (ps *peerSet) close() {
ps.lock.Lock() ps.lock.Lock()
defer ps.lock.Unlock() defer ps.lock.Unlock()
for _, p := range ps.ethPeers { for _, p := range ps.peers {
p.Disconnect(p2p.DiscQuitting)
}
for _, p := range ps.snapPeers {
p.Disconnect(p2p.DiscQuitting) p.Disconnect(p2p.DiscQuitting)
} }
ps.closed = true ps.closed = true

@ -103,12 +103,12 @@ type TxPool interface {
// MakeProtocols constructs the P2P protocol definitions for `eth`. // MakeProtocols constructs the P2P protocol definitions for `eth`.
func MakeProtocols(backend Backend, network uint64, dnsdisc enode.Iterator) []p2p.Protocol { func MakeProtocols(backend Backend, network uint64, dnsdisc enode.Iterator) []p2p.Protocol {
protocols := make([]p2p.Protocol, len(protocolVersions)) protocols := make([]p2p.Protocol, len(ProtocolVersions))
for i, version := range protocolVersions { for i, version := range ProtocolVersions {
version := version // Closure version := version // Closure
protocols[i] = p2p.Protocol{ protocols[i] = p2p.Protocol{
Name: protocolName, Name: ProtocolName,
Version: version, Version: version,
Length: protocolLengths[version], Length: protocolLengths[version],
Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error { Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error {

@ -34,13 +34,13 @@ const (
ETH65 = 65 ETH65 = 65
) )
// protocolName is the official short name of the `eth` protocol used during // ProtocolName is the official short name of the `eth` protocol used during
// devp2p capability negotiation. // devp2p capability negotiation.
const protocolName = "eth" const ProtocolName = "eth"
// protocolVersions are the supported versions of the `eth` protocol (first // ProtocolVersions are the supported versions of the `eth` protocol (first
// is primary). // is primary).
var protocolVersions = []uint{ETH65, ETH64} var ProtocolVersions = []uint{ETH65, ETH64}
// protocolLengths are the number of implemented message corresponding to // protocolLengths are the number of implemented message corresponding to
// different protocol versions. // different protocol versions.

@ -77,12 +77,12 @@ type Backend interface {
// MakeProtocols constructs the P2P protocol definitions for `snap`. // MakeProtocols constructs the P2P protocol definitions for `snap`.
func MakeProtocols(backend Backend, dnsdisc enode.Iterator) []p2p.Protocol { func MakeProtocols(backend Backend, dnsdisc enode.Iterator) []p2p.Protocol {
protocols := make([]p2p.Protocol, len(protocolVersions)) protocols := make([]p2p.Protocol, len(ProtocolVersions))
for i, version := range protocolVersions { for i, version := range ProtocolVersions {
version := version // Closure version := version // Closure
protocols[i] = p2p.Protocol{ protocols[i] = p2p.Protocol{
Name: protocolName, Name: ProtocolName,
Version: version, Version: version,
Length: protocolLengths[version], Length: protocolLengths[version],
Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error { Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error {

@ -30,13 +30,13 @@ const (
snap1 = 1 snap1 = 1
) )
// protocolName is the official short name of the `snap` protocol used during // ProtocolName is the official short name of the `snap` protocol used during
// devp2p capability negotiation. // devp2p capability negotiation.
const protocolName = "snap" const ProtocolName = "snap"
// protocolVersions are the supported versions of the `snap` protocol (first // ProtocolVersions are the supported versions of the `snap` protocol (first
// is primary). // is primary).
var protocolVersions = []uint{snap1} var ProtocolVersions = []uint{snap1}
// protocolLengths are the number of implemented message corresponding to // protocolLengths are the number of implemented message corresponding to
// different protocol versions. // different protocol versions.

@ -247,11 +247,11 @@ func (cs *chainSyncer) nextSyncOp() *chainSyncOp {
} else if minPeers > cs.handler.maxPeers { } else if minPeers > cs.handler.maxPeers {
minPeers = cs.handler.maxPeers minPeers = cs.handler.maxPeers
} }
if cs.handler.peers.Len() < minPeers { if cs.handler.peers.len() < minPeers {
return nil return nil
} }
// We have enough peers, check TD // We have enough peers, check TD
peer := cs.handler.peers.ethPeerWithHighestTD() peer := cs.handler.peers.peerWithHighestTD()
if peer == nil { if peer == nil {
return nil return nil
} }

@ -70,7 +70,7 @@ func testFastSyncDisabling(t *testing.T, protocol uint) {
time.Sleep(250 * time.Millisecond) time.Sleep(250 * time.Millisecond)
// Check that fast sync was disabled // Check that fast sync was disabled
op := peerToSyncOp(downloader.FastSync, empty.handler.peers.ethPeerWithHighestTD()) op := peerToSyncOp(downloader.FastSync, empty.handler.peers.peerWithHighestTD())
if err := empty.handler.doSync(op); err != nil { if err := empty.handler.doSync(op); err != nil {
t.Fatal("sync failed:", err) t.Fatal("sync failed:", err)
} }

@ -158,11 +158,16 @@ func (p *Peer) Caps() []Cap {
return p.rw.caps return p.rw.caps
} }
// SupportsCap returns true if the peer supports the given protocol/version // SupportsCap returns true if the peer supports any of the enumerated versions
func (p *Peer) SupportsCap(protocol string, version uint) bool { // of a specific protocol.
func (p *Peer) SupportsCap(protocol string, versions []uint) bool {
for _, cap := range p.rw.caps { for _, cap := range p.rw.caps {
if cap.Name == protocol { if cap.Name == protocol {
return version <= cap.Version for _, ver := range versions {
if cap.Version == ver {
return true
}
}
} }
} }
return false return false

Loading…
Cancel
Save