tendermint/p2p/pex/pex_reactor.go

771 lines
21 KiB
Go
Raw Normal View History

2018-01-20 21:12:04 -05:00
package pex
2015-10-25 18:21:51 -07:00
import (
"fmt"
"reflect"
"sync"
2015-10-25 18:21:51 -07:00
"time"
"github.com/pkg/errors"
2018-05-24 17:56:48 +04:00
amino "github.com/tendermint/go-amino"
2018-07-01 22:36:49 -04:00
cmn "github.com/tendermint/tendermint/libs/common"
2018-01-20 21:12:04 -05:00
"github.com/tendermint/tendermint/p2p"
2018-01-21 00:33:53 -05:00
"github.com/tendermint/tendermint/p2p/conn"
2015-10-25 18:21:51 -07:00
)
2018-01-20 21:12:04 -05:00
type Peer = p2p.Peer
2015-10-25 18:21:51 -07:00
const (
2017-01-16 23:57:07 +04:00
// PexChannel is a channel for PEX messages
2017-01-12 17:56:40 +04:00
PexChannel = byte(0x00)
2017-01-16 23:57:07 +04:00
// over-estimate of max NetAddress size
// hexID (40) + IP (16) + Port (2) + Name (100) ...
// NOTE: dont use massive DNS name ..
maxAddressSize = 256
// NOTE: amplificaiton factor!
2018-04-28 15:39:09 -04:00
// small request results in up to maxMsgSize response
maxMsgSize = maxAddressSize * maxGetSelection
// ensure we have enough peers
defaultEnsurePeersPeriod = 30 * time.Second
2018-01-14 01:40:29 -05:00
// Seed/Crawler constants
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// minTimeBetweenCrawls is a minimum time between attempts to crawl a peer.
minTimeBetweenCrawls = 2 * time.Minute
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// check some peers every this
crawlPeerPeriod = 30 * time.Second
2018-03-11 13:20:51 +04:00
2018-03-11 14:13:34 +04:00
maxAttemptsToDial = 16 // ~ 35h in total (last attempt - 18h)
2018-03-24 17:49:42 +01:00
// if node connects to seed, it does not have any trusted peers.
// Especially in the beginning, node should have more trusted peers than
// untrusted.
biasToSelectNewPeers = 30 // 70 to select good peers
2015-10-25 18:21:51 -07:00
)
type errMaxAttemptsToDial struct {
}
func (e errMaxAttemptsToDial) Error() string {
return fmt.Sprintf("reached max attempts %d to dial", maxAttemptsToDial)
}
type errTooEarlyToDial struct {
backoffDuration time.Duration
lastDialed time.Time
}
func (e errTooEarlyToDial) Error() string {
return fmt.Sprintf(
"too early to dial (backoff duration: %d, last dialed: %v, time since: %v)",
e.backoffDuration, e.lastDialed, time.Since(e.lastDialed))
}
2017-01-11 15:03:29 +04:00
// PEXReactor handles PEX (peer exchange) and ensures that an
// adequate number of peers are connected to the switch.
2017-01-16 23:57:07 +04:00
//
// It uses `AddrBook` (address book) to store `NetAddress`es of the peers.
2017-01-17 22:30:03 +04:00
//
// ## Preventing abuse
//
2018-01-14 03:22:01 -05:00
// Only accept pexAddrsMsg from peers we sent a corresponding pexRequestMsg too.
// Only accept one pexRequestMsg every ~defaultEnsurePeersPeriod.
2015-10-25 18:21:51 -07:00
type PEXReactor struct {
2018-01-20 21:12:04 -05:00
p2p.BaseReactor
2015-10-25 18:21:51 -07:00
2018-01-20 21:12:04 -05:00
book AddrBook
config *PEXReactorConfig
2018-03-20 21:41:08 +01:00
ensurePeersPeriod time.Duration // TODO: should go in the config
2017-01-12 17:56:40 +04:00
2018-01-14 03:22:01 -05:00
// maps to prevent abuse
2018-01-14 13:03:57 -05:00
requestsSent *cmn.CMap // ID->struct{}: unanswered send requests
lastReceivedRequests *cmn.CMap // ID->time.Time: last time peer requested from us
seedAddrs []*p2p.NetAddress
2018-03-11 14:00:49 +04:00
attemptsToDial sync.Map // address (string) -> {number of attempts (int), last time dialed (time.Time)}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// seed/crawled mode fields
crawlPeerInfos map[p2p.ID]crawlPeerInfo
2015-10-25 18:21:51 -07:00
}
func (r *PEXReactor) minReceiveRequestInterval() time.Duration {
2018-03-20 21:41:08 +01:00
// NOTE: must be less than ensurePeersPeriod, otherwise we'll request
// peers too quickly from others and they'll think we're bad!
return r.ensurePeersPeriod / 3
2018-03-20 21:41:08 +01:00
}
// PEXReactorConfig holds reactor specific configuration data.
type PEXReactorConfig struct {
// Seed/Crawler mode
SeedMode bool
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// We want seeds to only advertise good peers. Therefore they should wait at
// least as long as we expect it to take for a peer to become good before
// disconnecting.
SeedDisconnectWaitPeriod time.Duration
// Seeds is a list of addresses reactor may use
// if it can't connect to peers in the addrbook.
Seeds []string
}
2018-03-11 14:00:49 +04:00
type _attemptsToDial struct {
number int
lastDialed time.Time
}
2017-01-16 23:57:07 +04:00
// NewPEXReactor creates new PEX reactor.
2018-01-20 21:12:04 -05:00
func NewPEXReactor(b AddrBook, config *PEXReactorConfig) *PEXReactor {
2017-01-11 15:03:29 +04:00
r := &PEXReactor{
2018-01-14 03:22:01 -05:00
book: b,
config: config,
ensurePeersPeriod: defaultEnsurePeersPeriod,
requestsSent: cmn.NewCMap(),
lastReceivedRequests: cmn.NewCMap(),
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
crawlPeerInfos: make(map[p2p.ID]crawlPeerInfo),
2015-10-25 18:21:51 -07:00
}
2018-01-20 21:12:04 -05:00
r.BaseReactor = *p2p.NewBaseReactor("PEXReactor", r)
2017-01-11 15:03:29 +04:00
return r
2015-10-25 18:21:51 -07:00
}
2017-01-16 23:57:07 +04:00
// OnStart implements BaseService
2017-01-11 15:03:29 +04:00
func (r *PEXReactor) OnStart() error {
err := r.book.Start()
if err != nil && err != cmn.ErrAlreadyStarted {
2017-09-21 12:38:48 -04:00
return err
}
2018-01-14 03:56:15 -05:00
numOnline, seedAddrs, err := r.checkSeeds()
if err != nil {
2018-01-14 03:56:15 -05:00
return err
} else if numOnline == 0 && r.book.Empty() {
return errors.New("Address book is empty and couldn't resolve any seed nodes")
2018-01-14 03:56:15 -05:00
}
r.seedAddrs = seedAddrs
2018-01-14 01:40:29 -05:00
// Check if this node should run
// in seed/crawler mode
if r.config.SeedMode {
go r.crawlPeersRoutine()
} else {
go r.ensurePeersRoutine()
}
2015-10-25 18:21:51 -07:00
return nil
}
2017-01-16 23:57:07 +04:00
// OnStop implements BaseService
2017-01-11 15:03:29 +04:00
func (r *PEXReactor) OnStop() {
r.book.Stop()
2015-10-25 18:21:51 -07:00
}
2017-01-11 15:03:29 +04:00
// GetChannels implements Reactor
2018-01-21 00:33:53 -05:00
func (r *PEXReactor) GetChannels() []*conn.ChannelDescriptor {
return []*conn.ChannelDescriptor{
{
2015-10-25 18:21:51 -07:00
ID: PexChannel,
Priority: 1,
SendQueueCapacity: 10,
},
}
}
2017-01-11 15:03:29 +04:00
// AddPeer implements Reactor by adding peer to the address book (if inbound)
// or by requesting more addresses (if outbound).
2017-09-12 20:49:22 -04:00
func (r *PEXReactor) AddPeer(p Peer) {
if p.IsOutbound() {
2018-01-14 03:22:01 -05:00
// For outbound peers, the address is already in the books -
// either via DialPeersAsync or r.Receive.
// Ask it for more peers if we need.
2017-01-11 15:03:29 +04:00
if r.book.NeedMoreAddrs() {
r.RequestAddrs(p)
2015-10-25 18:21:51 -07:00
}
2018-01-13 16:06:31 -05:00
} else {
2018-04-28 13:08:44 -04:00
// inbound peer is its own source
addr, err := p.NodeInfo().NetAddress()
if err != nil {
r.Logger.Error("Failed to get peer NetAddress", "err", err, "peer", p)
return
}
// Make it explicit that addr and src are the same for an inbound peer.
2018-04-28 13:08:44 -04:00
src := addr
// add to book. dont RequestAddrs right away because
// we don't trust inbound as much - let ensurePeersRoutine handle it.
err = r.book.AddAddress(addr, src)
r.logErrAddrBook(err)
}
}
2018-04-28 13:08:44 -04:00
// RemovePeer implements Reactor by resetting peer's requests info.
func (r *PEXReactor) RemovePeer(p Peer, reason interface{}) {
id := string(p.ID())
r.requestsSent.Delete(id)
r.lastReceivedRequests.Delete(id)
}
func (r *PEXReactor) logErrAddrBook(err error) {
if err != nil {
switch err.(type) {
case ErrAddrBookNilAddr:
r.Logger.Error("Failed to add new address", "err", err)
default:
// non-routable, self, full book, private, etc.
r.Logger.Debug("Failed to add new address", "err", err)
}
2015-10-25 18:21:51 -07:00
}
}
2017-01-11 15:03:29 +04:00
// Receive implements Reactor by handling incoming PEX messages.
2017-09-12 20:49:22 -04:00
func (r *PEXReactor) Receive(chID byte, src Peer, msgBytes []byte) {
msg, err := decodeMsg(msgBytes)
2015-10-25 18:21:51 -07:00
if err != nil {
2018-03-04 13:42:45 +04:00
r.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
r.Switch.StopPeerForError(src, err)
2015-10-25 18:21:51 -07:00
return
}
2017-12-11 14:37:57 -05:00
r.Logger.Debug("Received message", "src", src, "chId", chID, "msg", msg)
2015-10-25 18:21:51 -07:00
switch msg := msg.(type) {
case *pexRequestMessage:
// NOTE: this is a prime candidate for amplification attacks,
2018-03-20 21:41:08 +01:00
// so it's important we
// 1) restrict how frequently peers can request
// 2) limit the output size
// If we're a seed and this is an inbound peer,
// respond once and disconnect.
if r.config.SeedMode && !src.IsOutbound() {
id := string(src.ID())
v := r.lastReceivedRequests.Get(id)
if v != nil {
// FlushStop/StopPeer are already
// running in a go-routine.
return
}
r.lastReceivedRequests.Set(id, time.Now())
// Send addrs and disconnect
2018-03-24 17:49:42 +01:00
r.SendAddrs(src, r.book.GetSelectionWithBias(biasToSelectNewPeers))
go func() {
// In a go-routine so it doesn't block .Receive.
src.FlushStop()
r.Switch.StopPeerGracefully(src)
}()
} else {
// Check we're not receiving requests too frequently.
if err := r.receiveRequest(src); err != nil {
r.Switch.StopPeerForError(src, err)
return
}
r.SendAddrs(src, r.book.GetSelection())
}
2015-10-25 18:21:51 -07:00
case *pexAddrsMessage:
// If we asked for addresses, add them to the book
if err := r.ReceiveAddrs(msg.Addrs, src); err != nil {
2018-01-14 03:22:01 -05:00
r.Switch.StopPeerForError(src, err)
return
2015-10-25 18:21:51 -07:00
}
default:
2017-05-02 11:53:32 +04:00
r.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
2015-10-25 18:21:51 -07:00
}
}
2018-03-20 21:41:08 +01:00
// enforces a minimum amount of time between requests
2018-01-14 03:22:01 -05:00
func (r *PEXReactor) receiveRequest(src Peer) error {
id := string(src.ID())
v := r.lastReceivedRequests.Get(id)
if v == nil {
// initialize with empty time
lastReceived := time.Time{}
r.lastReceivedRequests.Set(id, lastReceived)
return nil
}
lastReceived := v.(time.Time)
if lastReceived.Equal(time.Time{}) {
// first time gets a free pass. then we start tracking the time
2018-01-14 13:03:57 -05:00
lastReceived = time.Now()
2018-01-14 03:22:01 -05:00
r.lastReceivedRequests.Set(id, lastReceived)
return nil
}
now := time.Now()
2018-03-20 21:41:08 +01:00
minInterval := r.minReceiveRequestInterval()
if now.Sub(lastReceived) < minInterval {
return fmt.Errorf("peer (%v) sent next PEX request too soon. lastReceived: %v, now: %v, minInterval: %v. Disconnecting",
2018-03-20 21:41:08 +01:00
src.ID(),
lastReceived,
now,
minInterval,
)
2018-01-14 03:22:01 -05:00
}
r.lastReceivedRequests.Set(id, now)
return nil
}
// RequestAddrs asks peer for more addresses if we do not already have a
// request out for this peer.
func (r *PEXReactor) RequestAddrs(p Peer) {
2018-01-14 03:22:01 -05:00
id := string(p.ID())
if r.requestsSent.Has(id) {
return
}
r.Logger.Debug("Request addrs", "from", p)
2018-01-14 03:22:01 -05:00
r.requestsSent.Set(id, struct{}{})
p.Send(PexChannel, cdc.MustMarshalBinaryBare(&pexRequestMessage{}))
2015-10-25 18:21:51 -07:00
}
// ReceiveAddrs adds the given addrs to the addrbook if theres an open
2018-01-14 03:22:01 -05:00
// request for this peer and deletes the open request.
// If there's no open request for the src peer, it returns an error.
2018-01-21 00:33:53 -05:00
func (r *PEXReactor) ReceiveAddrs(addrs []*p2p.NetAddress, src Peer) error {
2018-04-28 13:08:44 -04:00
id := string(src.ID())
2018-01-14 03:22:01 -05:00
if !r.requestsSent.Has(id) {
return errors.New("unsolicited pexAddrsMessage")
2018-01-14 03:22:01 -05:00
}
r.requestsSent.Delete(id)
srcAddr, err := src.NodeInfo().NetAddress()
if err != nil {
return err
}
srcIsSeed := false
for _, seedAddr := range r.seedAddrs {
if seedAddr.Equals(srcAddr) {
srcIsSeed = true
break
}
}
2018-01-14 03:22:01 -05:00
for _, netAddr := range addrs {
// NOTE: we check netAddr validity and routability in book#AddAddress.
err = r.book.AddAddress(netAddr, srcAddr)
if err != nil {
r.logErrAddrBook(err)
// XXX: should we be strict about incoming data and disconnect from a
// peer here too?
continue
}
// If this address came from a seed node, try to connect to it without
// waiting (#2093)
if srcIsSeed {
r.Logger.Info("Will dial address, which came from seed", "addr", netAddr, "seed", srcAddr)
go func(addr *p2p.NetAddress) {
err := r.dialPeer(addr)
if err != nil {
switch err.(type) {
case errMaxAttemptsToDial, errTooEarlyToDial:
r.Logger.Debug(err.Error(), "addr", addr)
default:
r.Logger.Error(err.Error(), "addr", addr)
}
}
}(netAddr)
}
2018-01-14 03:22:01 -05:00
}
2018-01-14 03:22:01 -05:00
return nil
}
2017-01-11 15:03:29 +04:00
// SendAddrs sends addrs to the peer.
2018-01-21 00:33:53 -05:00
func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*p2p.NetAddress) {
p.Send(PexChannel, cdc.MustMarshalBinaryBare(&pexAddrsMessage{Addrs: netAddrs}))
2015-10-25 18:21:51 -07:00
}
2017-01-12 00:17:15 +04:00
// SetEnsurePeersPeriod sets period to ensure peers connected.
func (r *PEXReactor) SetEnsurePeersPeriod(d time.Duration) {
r.ensurePeersPeriod = d
}
2015-10-25 18:21:51 -07:00
// Ensures that sufficient peers are connected. (continuous)
2017-01-11 15:03:29 +04:00
func (r *PEXReactor) ensurePeersRoutine() {
var (
seed = cmn.NewRand()
jitter = seed.Int63n(r.ensurePeersPeriod.Nanoseconds())
)
// Randomize first round of communication to avoid thundering herd.
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// If no peers are present directly start connecting so we guarantee swift
// setup with the help of configured seeds.
if r.nodeHasSomePeersOrDialingAny() {
time.Sleep(time.Duration(jitter))
}
2015-10-25 18:21:51 -07:00
2018-01-14 03:56:15 -05:00
// fire once immediately.
// ensures we dial the seeds right away if the book is empty
r.ensurePeers()
// fire periodically
ticker := time.NewTicker(r.ensurePeersPeriod)
2015-10-25 18:21:51 -07:00
for {
select {
case <-ticker.C:
2017-01-11 15:03:29 +04:00
r.ensurePeers()
case <-r.Quit():
2017-01-12 17:56:40 +04:00
ticker.Stop()
return
2015-10-25 18:21:51 -07:00
}
}
}
2017-01-12 00:17:15 +04:00
// ensurePeers ensures that sufficient peers are connected. (once)
2017-04-20 12:49:54 +04:00
//
// heuristic that we haven't perfected yet, or, perhaps is manually edited by
// the node operator. It should not be used to compute what addresses are
// already connected or not.
2017-01-11 15:03:29 +04:00
func (r *PEXReactor) ensurePeers() {
var (
out, in, dial = r.Switch.NumPeers()
numToDial = r.Switch.MaxNumOutboundPeers() - (out + dial)
)
r.Logger.Info(
"Ensure peers",
"numOutPeers", out,
"numInPeers", in,
"numDialing", dial,
"numToDial", numToDial,
)
2015-10-25 18:21:51 -07:00
if numToDial <= 0 {
return
}
2017-11-16 04:30:38 +00:00
// bias to prefer more vetted peers when we have fewer connections.
// not perfect, but somewhate ensures that we prioritize connecting to more-vetted
2017-12-10 19:00:44 -05:00
// NOTE: range here is [10, 90]. Too high ?
newBias := cmn.MinInt(out, 8)*10 + 10
2015-10-25 18:21:51 -07:00
2018-01-21 00:33:53 -05:00
toDial := make(map[p2p.ID]*p2p.NetAddress)
2017-11-16 04:30:38 +00:00
// Try maxAttempts times to pick numToDial addresses to dial
maxAttempts := numToDial * 3
2017-11-16 04:30:38 +00:00
for i := 0; i < maxAttempts && len(toDial) < numToDial; i++ {
try := r.book.PickAddress(newBias)
if try == nil {
continue
2015-10-25 18:21:51 -07:00
}
2018-01-13 16:14:28 -05:00
if _, selected := toDial[try.ID]; selected {
2017-11-20 19:30:05 +00:00
continue
}
if r.Switch.IsDialingOrExistingAddress(try) {
2015-10-25 18:21:51 -07:00
continue
}
2018-03-20 21:41:08 +01:00
// TODO: consider moving some checks from toDial into here
// so we don't even consider dialing peers that we want to wait
2018-04-28 15:01:33 -04:00
// before dialling again, or have dialed too many times already
2017-11-16 04:30:38 +00:00
r.Logger.Info("Will dial address", "addr", try)
2018-01-13 16:14:28 -05:00
toDial[try.ID] = try
2015-10-25 18:21:51 -07:00
}
// Dial picked addresses
for _, addr := range toDial {
go func(addr *p2p.NetAddress) {
err := r.dialPeer(addr)
if err != nil {
switch err.(type) {
case errMaxAttemptsToDial, errTooEarlyToDial:
r.Logger.Debug(err.Error(), "addr", addr)
default:
r.Logger.Error(err.Error(), "addr", addr)
}
}
}(addr)
2015-10-25 18:21:51 -07:00
}
2017-01-11 15:03:29 +04:00
if r.book.NeedMoreAddrs() {
// 1) Pick a random peer and ask for more.
peers := r.Switch.Peers().List()
peersCount := len(peers)
if peersCount > 0 {
peer := peers[cmn.RandInt()%peersCount] // nolint: gas
r.Logger.Info("We need more addresses. Sending pexRequest to random peer", "peer", peer)
r.RequestAddrs(peer)
2015-10-25 18:21:51 -07:00
}
// 2) Dial seeds if we are not dialing anyone.
// This is done in addition to asking a peer for addresses to work-around
// peers not participating in PEX.
if len(toDial) == 0 {
r.Logger.Info("No addresses to dial. Falling back to seeds")
r.dialSeeds()
}
2018-01-14 03:56:15 -05:00
}
}
2018-03-20 21:41:08 +01:00
func (r *PEXReactor) dialAttemptsInfo(addr *p2p.NetAddress) (attempts int, lastDialed time.Time) {
_attempts, ok := r.attemptsToDial.Load(addr.DialString())
if !ok {
return
2018-03-11 14:00:49 +04:00
}
2018-03-20 21:41:08 +01:00
atd := _attempts.(_attemptsToDial)
return atd.number, atd.lastDialed
}
func (r *PEXReactor) dialPeer(addr *p2p.NetAddress) error {
2018-03-20 21:41:08 +01:00
attempts, lastDialed := r.dialAttemptsInfo(addr)
2018-03-11 13:20:51 +04:00
if attempts > maxAttemptsToDial {
// TODO(melekes): have a blacklist in the addrbook with peers whom we've
// failed to connect to. Then we can clean up attemptsToDial, which acts as
// a blacklist currently.
// https://github.com/tendermint/tendermint/issues/3572
2018-03-11 13:20:51 +04:00
r.book.MarkBad(addr)
return errMaxAttemptsToDial{}
2018-03-11 13:20:51 +04:00
}
2018-03-08 15:55:23 +04:00
// exponential backoff if it's not our first attempt to dial given address
2018-03-11 14:00:49 +04:00
if attempts > 0 {
jitterSeconds := time.Duration(cmn.RandFloat64() * float64(time.Second)) // 1s == (1e9 ns)
2018-03-08 15:55:23 +04:00
backoffDuration := jitterSeconds + ((1 << uint(attempts)) * time.Second)
2018-03-11 14:13:34 +04:00
sinceLastDialed := time.Since(lastDialed)
2018-03-11 14:00:49 +04:00
if sinceLastDialed < backoffDuration {
return errTooEarlyToDial{backoffDuration, lastDialed}
2018-03-11 14:00:49 +04:00
}
2018-03-08 15:55:23 +04:00
}
err := r.Switch.DialPeerWithAddress(addr)
2018-03-08 15:55:23 +04:00
if err != nil {
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
if _, ok := err.(p2p.ErrCurrentlyDialingOrExistingAddress); ok {
return err
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
}
markAddrInBookBasedOnErr(addr, r.book, err)
switch err.(type) {
case p2p.ErrSwitchAuthenticationFailure:
// NOTE: addr is removed from addrbook in markAddrInBookBasedOnErr
r.attemptsToDial.Delete(addr.DialString())
default:
r.attemptsToDial.Store(addr.DialString(), _attemptsToDial{attempts + 1, time.Now()})
2018-03-08 15:55:23 +04:00
}
return errors.Wrapf(err, "dialing failed (attempts: %d)", attempts+1)
2018-03-08 15:55:23 +04:00
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// cleanup any history
r.attemptsToDial.Delete(addr.DialString())
return nil
2018-03-08 15:55:23 +04:00
}
// checkSeeds checks that addresses are well formed.
// Returns number of seeds we can connect to, along with all seeds addrs.
// return err if user provided any badly formatted seed addresses.
// Doesn't error if the seed node can't be reached.
// numOnline returns -1 if no seed nodes were in the initial configuration.
func (r *PEXReactor) checkSeeds() (numOnline int, netAddrs []*p2p.NetAddress, err error) {
2018-01-14 03:56:15 -05:00
lSeeds := len(r.config.Seeds)
2018-01-14 13:03:57 -05:00
if lSeeds == 0 {
return -1, nil, nil
2018-01-14 13:03:57 -05:00
}
netAddrs, errs := p2p.NewNetAddressStrings(r.config.Seeds)
numOnline = lSeeds - len(errs)
2018-01-14 13:03:57 -05:00
for _, err := range errs {
switch e := err.(type) {
case p2p.ErrNetAddressLookup:
r.Logger.Error("Connecting to seed failed", "err", e)
default:
return 0, nil, errors.Wrap(e, "seed node configuration has error")
2018-01-14 03:56:15 -05:00
}
}
return numOnline, netAddrs, nil
2018-01-14 03:56:15 -05:00
}
// randomly dial seeds until we connect to one or exhaust them
func (r *PEXReactor) dialSeeds() {
perm := cmn.RandPerm(len(r.seedAddrs))
2018-01-20 21:12:04 -05:00
// perm := r.Switch.rng.Perm(lSeeds)
for _, i := range perm {
// dial a random seed
seedAddr := r.seedAddrs[i]
err := r.Switch.DialPeerWithAddress(seedAddr)
p2p: introduce peerConn to simplify peer creation (#1226) * expose AuthEnc in the P2P config if AuthEnc is true, dialed peers must have a node ID in the address and it must match the persistent pubkey from the secret handshake. Refs #1157 * fixes after my own review * fix docs * fix build failure ``` p2p/pex/pex_reactor_test.go:288:88: cannot use seed.NodeInfo().NetAddress() (type *p2p.NetAddress) as type string in array or slice literal ``` * p2p: introduce peerConn to simplify peer creation * Introduce `peerConn` containing the known fields of `peer` * `peer` only created in `sw.addPeer` once handshake is complete and NodeInfo is checked * Eliminates some mutable variables and makes the code flow better * Simplifies the `newXxxPeer` funcs * Use ID instead of PubKey where possible. * SetPubKeyFilter -> SetIDFilter * nodeInfo.Validate takes ID * remove peer.PubKey() * persistent node ids * fixes from review * test: use ip_plus_id.sh more * fix invalid memory panic during fast_sync test ``` 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: panic: runtime error: invalid memory address or nil pointer dereference 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: [signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x98dd3e] 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: goroutine 3432 [running]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.newOutboundPeerConn(0xc423fd1380, 0xc420933e00, 0x1, 0x1239a60, 0 xc420128c40, 0x2, 0x42caf6, 0xc42001f300, 0xc422831d98, 0xc4227951c0, ...) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/peer.go:123 +0x31e 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).addOutboundPeerWithConfig(0xc4200ad040, 0xc423fd1380, 0 xc420933e00, 0xc423f48801, 0x28, 0x2) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:455 +0x12b 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).DialPeerWithAddress(0xc4200ad040, 0xc423fd1380, 0x1, 0x 0, 0x0) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:371 +0xdc 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).reconnectToPeer(0xc4200ad040, 0x123e000, 0xc42007bb00) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:290 +0x25f 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: created by github.com/tendermint/tendermint/p2p.(*Switch).StopPeerForError 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:256 +0x1b7 ```
2018-02-27 06:54:40 -05:00
if err == nil {
return
}
p2p: introduce peerConn to simplify peer creation (#1226) * expose AuthEnc in the P2P config if AuthEnc is true, dialed peers must have a node ID in the address and it must match the persistent pubkey from the secret handshake. Refs #1157 * fixes after my own review * fix docs * fix build failure ``` p2p/pex/pex_reactor_test.go:288:88: cannot use seed.NodeInfo().NetAddress() (type *p2p.NetAddress) as type string in array or slice literal ``` * p2p: introduce peerConn to simplify peer creation * Introduce `peerConn` containing the known fields of `peer` * `peer` only created in `sw.addPeer` once handshake is complete and NodeInfo is checked * Eliminates some mutable variables and makes the code flow better * Simplifies the `newXxxPeer` funcs * Use ID instead of PubKey where possible. * SetPubKeyFilter -> SetIDFilter * nodeInfo.Validate takes ID * remove peer.PubKey() * persistent node ids * fixes from review * test: use ip_plus_id.sh more * fix invalid memory panic during fast_sync test ``` 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: panic: runtime error: invalid memory address or nil pointer dereference 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: [signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x98dd3e] 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: goroutine 3432 [running]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.newOutboundPeerConn(0xc423fd1380, 0xc420933e00, 0x1, 0x1239a60, 0 xc420128c40, 0x2, 0x42caf6, 0xc42001f300, 0xc422831d98, 0xc4227951c0, ...) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/peer.go:123 +0x31e 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).addOutboundPeerWithConfig(0xc4200ad040, 0xc423fd1380, 0 xc420933e00, 0xc423f48801, 0x28, 0x2) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:455 +0x12b 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).DialPeerWithAddress(0xc4200ad040, 0xc423fd1380, 0x1, 0x 0, 0x0) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:371 +0xdc 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).reconnectToPeer(0xc4200ad040, 0x123e000, 0xc42007bb00) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:290 +0x25f 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: created by github.com/tendermint/tendermint/p2p.(*Switch).StopPeerForError 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:256 +0x1b7 ```
2018-02-27 06:54:40 -05:00
r.Switch.Logger.Error("Error dialing seed", "err", err, "seed", seedAddr)
}
// do not write error message if there were no seeds specified in config
if len(r.seedAddrs) > 0 {
r.Switch.Logger.Error("Couldn't connect to any seeds")
}
}
2018-03-09 16:23:52 +04:00
// AttemptsToDial returns the number of attempts to dial specific address. It
// returns 0 if never attempted or successfully connected.
func (r *PEXReactor) AttemptsToDial(addr *p2p.NetAddress) int {
2018-03-11 14:00:49 +04:00
lAttempts, attempted := r.attemptsToDial.Load(addr.DialString())
2018-03-09 16:23:52 +04:00
if attempted {
2018-03-11 14:00:49 +04:00
return lAttempts.(_attemptsToDial).number
2018-03-09 16:23:52 +04:00
}
return 0
2018-03-09 16:23:52 +04:00
}
//----------------------------------------------------------
2018-01-09 20:12:41 -05:00
// Explores the network searching for more peers. (continuous)
// Seed/Crawler Mode causes this node to quickly disconnect
// from peers, except other seed nodes.
2018-01-14 01:40:29 -05:00
func (r *PEXReactor) crawlPeersRoutine() {
// If we have any seed nodes, consult them first
if len(r.seedAddrs) > 0 {
r.dialSeeds()
} else {
// Do an initial crawl
r.crawlPeers(r.book.GetSelection())
}
2018-01-09 20:12:41 -05:00
// Fire periodically
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
ticker := time.NewTicker(crawlPeerPeriod)
2018-01-09 20:12:41 -05:00
for {
select {
case <-ticker.C:
r.attemptDisconnects()
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
r.crawlPeers(r.book.GetSelection())
r.cleanupCrawlPeerInfos()
case <-r.Quit():
2018-01-09 20:12:41 -05:00
return
}
}
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// nodeHasSomePeersOrDialingAny returns true if the node is connected to some
// peers or dialing them currently.
func (r *PEXReactor) nodeHasSomePeersOrDialingAny() bool {
out, in, dial := r.Switch.NumPeers()
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
return out+in+dial > 0
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// crawlPeerInfo handles temporary data needed for the network crawling
// performed during seed/crawler mode.
type crawlPeerInfo struct {
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
Addr *p2p.NetAddress `json:"addr"`
// The last time we crawled the peer or attempted to do so.
LastCrawled time.Time `json:"last_crawled"`
2018-01-09 20:12:41 -05:00
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// crawlPeers will crawl the network looking for new peer addresses.
func (r *PEXReactor) crawlPeers(addrs []*p2p.NetAddress) {
now := time.Now()
2018-01-09 20:12:41 -05:00
for _, addr := range addrs {
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
peerInfo, ok := r.crawlPeerInfos[addr.ID]
2018-01-09 20:12:41 -05:00
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// Do not attempt to connect with peers we recently crawled.
if ok && now.Sub(peerInfo.LastCrawled) < minTimeBetweenCrawls {
2018-01-09 20:12:41 -05:00
continue
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
// Record crawling attempt.
r.crawlPeerInfos[addr.ID] = crawlPeerInfo{
Addr: addr,
LastCrawled: now,
}
err := r.dialPeer(addr)
2018-01-09 20:12:41 -05:00
if err != nil {
switch err.(type) {
case errMaxAttemptsToDial, errTooEarlyToDial:
r.Logger.Debug(err.Error(), "addr", addr)
default:
r.Logger.Error(err.Error(), "addr", addr)
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
}
2018-01-09 20:12:41 -05:00
continue
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
peer := r.Switch.Peers().Get(addr.ID)
if peer != nil {
r.RequestAddrs(peer)
}
2018-01-09 20:12:41 -05:00
}
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
func (r *PEXReactor) cleanupCrawlPeerInfos() {
for id, info := range r.crawlPeerInfos {
// If we did not crawl a peer for 24 hours, it means the peer was removed
// from the addrbook => remove
//
// 10000 addresses / maxGetSelection = 40 cycles to get all addresses in
// the ideal case,
// 40 * crawlPeerPeriod ~ 20 minutes
if time.Since(info.LastCrawled) > 24*time.Hour {
delete(r.crawlPeerInfos, id)
}
}
}
// attemptDisconnects checks if we've been with each peer long enough to disconnect
2018-01-09 20:12:41 -05:00
func (r *PEXReactor) attemptDisconnects() {
for _, peer := range r.Switch.Peers().List() {
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
if peer.Status().Duration < r.config.SeedDisconnectWaitPeriod {
2018-01-09 20:12:41 -05:00
continue
}
if peer.IsPersistent() {
2018-01-09 20:12:41 -05:00
continue
}
r.Switch.StopPeerGracefully(peer)
2018-01-14 13:03:57 -05:00
}
2015-10-25 18:21:51 -07:00
}
p2p: seed mode refactoring (#3011) ListOfKnownAddresses is removed panic if addrbook size is less than zero CrawlPeers does not attempt to connect to existing or peers we're currently dialing various perf. fixes improved tests (though not complete) move IsDialingOrExistingAddress check into DialPeerWithAddress (Fixes #2716) * addrbook: preallocate memory when saving addrbook to file * addrbook: remove oldestFirst struct and check for ID * oldestFirst replaced with sort.Slice * ID is now mandatory, so no need to check * addrbook: remove ListOfKnownAddresses GetSelection is used instead in seed mode. * addrbook: panic if size is less than 0 * rewrite addrbook#saveToFile to not use a counter * test AttemptDisconnects func * move IsDialingOrExistingAddress check into DialPeerWithAddress * save and cleanup crawl peer data * get rid of DefaultSeedDisconnectWaitPeriod * make linter happy * fix TestPEXReactorSeedMode * fix comment * add a changelog entry * Apply suggestions from code review Co-Authored-By: melekes <anton.kalyaev@gmail.com> * rename ErrDialingOrExistingAddress to ErrCurrentlyDialingOrExistingAddress * lowercase errors * do not persist seed data pros: - no extra files - less IO cons: - if the node crashes, seed might crawl a peer too soon * fixes after Ethan's review * add a changelog entry * we should only consult Switch about peers checking addrbook size does not make sense since only PEX reactor uses it for dialing peers! https://github.com/tendermint/tendermint/pull/3011#discussion_r270948875
2019-04-03 11:22:52 +02:00
func markAddrInBookBasedOnErr(addr *p2p.NetAddress, book AddrBook, err error) {
// TODO: detect more "bad peer" scenarios
switch err.(type) {
case p2p.ErrSwitchAuthenticationFailure:
book.MarkBad(addr)
default:
book.MarkAttempt(addr)
}
}
2015-10-25 18:21:51 -07:00
//-----------------------------------------------------------------------------
// Messages
2017-01-16 23:57:07 +04:00
// PexMessage is a primary type for PEX messages. Underneath, it could contain
// either pexRequestMessage, or pexAddrsMessage messages.
2015-10-25 18:21:51 -07:00
type PexMessage interface{}
2018-03-26 06:40:02 +02:00
func RegisterPexMessage(cdc *amino.Codec) {
cdc.RegisterInterface((*PexMessage)(nil), nil)
cdc.RegisterConcrete(&pexRequestMessage{}, "tendermint/p2p/PexRequestMessage", nil)
cdc.RegisterConcrete(&pexAddrsMessage{}, "tendermint/p2p/PexAddrsMessage", nil)
}
2015-10-25 18:21:51 -07:00
func decodeMsg(bz []byte) (msg PexMessage, err error) {
2018-04-09 15:14:33 +03:00
if len(bz) > maxMsgSize {
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
2018-04-09 15:14:33 +03:00
}
err = cdc.UnmarshalBinaryBare(bz, &msg)
2015-10-25 18:21:51 -07:00
return
}
/*
A pexRequestMessage requests additional peer addresses.
*/
type pexRequestMessage struct {
}
func (m *pexRequestMessage) String() string {
return "[pexRequest]"
}
/*
A message with announced peer addresses.
*/
type pexAddrsMessage struct {
2018-01-21 00:33:53 -05:00
Addrs []*p2p.NetAddress
2015-10-25 18:21:51 -07:00
}
func (m *pexAddrsMessage) String() string {
return fmt.Sprintf("[pexAddrs %v]", m.Addrs)
}