2018-01-20 21:12:04 -05:00
|
|
|
package pex
|
2015-10-25 18:21:51 -07:00
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
|
|
|
"reflect"
|
2018-01-09 20:12:41 -05:00
|
|
|
"sort"
|
2018-03-08 13:59:30 +04:00
|
|
|
"sync"
|
2015-10-25 18:21:51 -07:00
|
|
|
"time"
|
|
|
|
|
2018-08-04 17:35:08 -07:00
|
|
|
"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
|
|
|
|
2018-04-28 14:41:36 -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
|
2018-01-20 18:28:40 -05:00
|
|
|
|
|
|
|
// ensure we have enough peers
|
2018-08-15 02:25:56 +04:00
|
|
|
defaultEnsurePeersPeriod = 30 * time.Second
|
2018-01-14 01:40:29 -05:00
|
|
|
|
|
|
|
// Seed/Crawler constants
|
2018-03-24 17:04:32 +01: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.
|
|
|
|
// see consensus/reactor.go: blocksToContributeToBecomeGoodPeer
|
|
|
|
// 10000 blocks assuming 1s blocks ~ 2.7 hours.
|
|
|
|
defaultSeedDisconnectWaitPeriod = 3 * time.Hour
|
|
|
|
|
|
|
|
defaultCrawlPeerInterval = 2 * time.Minute // don't redial for this. TODO: back-off. what for?
|
|
|
|
|
|
|
|
defaultCrawlPeersPeriod = 30 * time.Second // check some peers every this
|
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
|
|
|
)
|
|
|
|
|
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
|
2018-01-09 17:09:09 -06:00
|
|
|
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
|
2018-03-08 13:59:30 +04:00
|
|
|
|
2018-07-31 13:09:01 -07:00
|
|
|
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)}
|
2015-10-25 18:21:51 -07:00
|
|
|
}
|
|
|
|
|
2018-06-29 16:03:31 +04: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!
|
2018-06-29 16:03:31 +04:00
|
|
|
return r.ensurePeersPeriod / 3
|
2018-03-20 21:41:08 +01:00
|
|
|
}
|
|
|
|
|
2018-01-09 17:09:09 -06:00
|
|
|
// PEXReactorConfig holds reactor specific configuration data.
|
|
|
|
type PEXReactorConfig struct {
|
2018-01-20 18:28:40 -05:00
|
|
|
// Seed/Crawler mode
|
|
|
|
SeedMode bool
|
|
|
|
|
|
|
|
// Seeds is a list of addresses reactor may use
|
|
|
|
// if it can't connect to peers in the addrbook.
|
2018-01-09 17:09:09 -06:00
|
|
|
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(),
|
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 {
|
2017-11-06 13:20:39 -05:00
|
|
|
err := r.book.Start()
|
2017-11-29 10:53:30 -06:00
|
|
|
if err != nil && err != cmn.ErrAlreadyStarted {
|
2017-09-21 12:38:48 -04:00
|
|
|
return err
|
|
|
|
}
|
2018-01-14 03:56:15 -05:00
|
|
|
|
2018-08-04 17:35:08 -07:00
|
|
|
numOnline, seedAddrs, err := r.checkSeeds()
|
2018-07-31 13:09:01 -07:00
|
|
|
if err != nil {
|
2018-01-14 03:56:15 -05:00
|
|
|
return err
|
2018-08-04 17:35:08 -07:00
|
|
|
} else if numOnline == 0 && r.book.Empty() {
|
|
|
|
return errors.New("Address book is empty, and could not connect to any seed nodes")
|
2018-01-14 03:56:15 -05:00
|
|
|
}
|
|
|
|
|
2018-07-31 13:09:01 -07: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{
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
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) {
|
2017-04-20 12:21:45 -04:00
|
|
|
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() {
|
2018-01-20 19:28:43 -05:00
|
|
|
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
|
2018-01-13 16:14:28 -05:00
|
|
|
addr := p.NodeInfo().NetAddress()
|
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)
|
2018-04-28 16:00:45 -04:00
|
|
|
r.logErrAddrBook(err)
|
|
|
|
}
|
|
|
|
}
|
2018-04-28 13:08:44 -04:00
|
|
|
|
2018-04-28 16:00:45 -04:00
|
|
|
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:
|
2018-07-18 02:22:09 -07:00
|
|
|
// non-routable, self, full book, private, etc.
|
2018-04-28 16:00:45 -04:00
|
|
|
r.Logger.Debug("Failed to add new address", "err", err)
|
|
|
|
}
|
2015-10-25 18:21:51 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-20 13:04:40 +04:00
|
|
|
// RemovePeer implements Reactor.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (r *PEXReactor) RemovePeer(p Peer, reason interface{}) {
|
2018-01-14 03:22:01 -05:00
|
|
|
id := string(p.ID())
|
|
|
|
r.requestsSent.Delete(id)
|
|
|
|
r.lastReceivedRequests.Delete(id)
|
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) {
|
2018-07-09 13:01:23 +04:00
|
|
|
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:
|
2018-01-20 19:28:43 -05:00
|
|
|
// Check we're not receiving too many requests
|
2018-01-14 03:22:01 -05:00
|
|
|
if err := r.receiveRequest(src); err != nil {
|
|
|
|
r.Switch.StopPeerForError(src, err)
|
|
|
|
return
|
|
|
|
}
|
2018-01-20 19:28:43 -05:00
|
|
|
|
|
|
|
// Seeds disconnect after sending a batch of addrs
|
2018-03-20 21:41:08 +01:00
|
|
|
// NOTE: this is a prime candidate for amplification attacks
|
|
|
|
// so it's important we
|
|
|
|
// 1) restrict how frequently peers can request
|
|
|
|
// 2) limit the output size
|
2018-01-20 19:28:43 -05:00
|
|
|
if r.config.SeedMode {
|
2018-03-24 17:49:42 +01:00
|
|
|
r.SendAddrs(src, r.book.GetSelectionWithBias(biasToSelectNewPeers))
|
2018-01-20 19:28:43 -05:00
|
|
|
r.Switch.StopPeerGracefully(src)
|
|
|
|
} else {
|
|
|
|
r.SendAddrs(src, r.book.GetSelection())
|
|
|
|
}
|
|
|
|
|
2015-10-25 18:21:51 -07:00
|
|
|
case *pexAddrsMessage:
|
2018-01-20 19:28:43 -05:00
|
|
|
// 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 {
|
2018-06-06 20:45:20 -07:00
|
|
|
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
|
|
|
|
}
|
|
|
|
|
2018-01-20 19:28:43 -05:00
|
|
|
// RequestAddrs asks peer for more addresses if we do not already
|
2018-01-14 03:22:01 -05:00
|
|
|
// have a request out for this peer.
|
2018-01-20 19:28:43 -05:00
|
|
|
func (r *PEXReactor) RequestAddrs(p Peer) {
|
2018-05-24 17:56:48 +04:00
|
|
|
r.Logger.Debug("Request addrs", "from", p)
|
2018-01-14 03:22:01 -05:00
|
|
|
id := string(p.ID())
|
|
|
|
if r.requestsSent.Has(id) {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
r.requestsSent.Set(id, struct{}{})
|
2018-07-09 13:11:41 +04:00
|
|
|
p.Send(PexChannel, cdc.MustMarshalBinaryBare(&pexRequestMessage{}))
|
2015-10-25 18:21:51 -07:00
|
|
|
}
|
|
|
|
|
2018-01-20 19:28:43 -05: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) {
|
2018-03-26 06:40:02 +02:00
|
|
|
return cmn.NewError("Received unsolicited pexAddrsMessage")
|
2018-01-14 03:22:01 -05:00
|
|
|
}
|
|
|
|
r.requestsSent.Delete(id)
|
|
|
|
|
|
|
|
srcAddr := src.NodeInfo().NetAddress()
|
|
|
|
for _, netAddr := range addrs {
|
2018-04-28 15:01:33 -04:00
|
|
|
// NOTE: GetSelection methods should never return nil addrs
|
2018-04-28 13:08:44 -04:00
|
|
|
if netAddr == nil {
|
|
|
|
return cmn.NewError("received nil addr")
|
|
|
|
}
|
|
|
|
|
|
|
|
err := r.book.AddAddress(netAddr, srcAddr)
|
2018-04-28 16:00:45 -04:00
|
|
|
r.logErrAddrBook(err)
|
2018-07-31 13:09:01 -07:00
|
|
|
|
|
|
|
// If this address came from a seed node, try to connect to it without waiting.
|
|
|
|
for _, seedAddr := range r.seedAddrs {
|
|
|
|
if seedAddr.Equals(srcAddr) {
|
|
|
|
r.ensurePeers()
|
|
|
|
}
|
|
|
|
}
|
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) {
|
2018-07-09 13:11:41 +04:00
|
|
|
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() {
|
2018-03-01 18:41:30 +01:00
|
|
|
var (
|
2018-04-11 11:38:30 +02:00
|
|
|
seed = cmn.NewRand()
|
2018-03-01 18:41:30 +01:00
|
|
|
jitter = seed.Int63n(r.ensurePeersPeriod.Nanoseconds())
|
|
|
|
)
|
|
|
|
|
|
|
|
// Randomize first round of communication to avoid thundering herd.
|
|
|
|
// If no potential peers are present directly start connecting so we guarantee
|
|
|
|
// swift setup with the help of configured seeds.
|
|
|
|
if r.hasPotentialPeers() {
|
|
|
|
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
|
2017-01-12 13:26:29 +04:00
|
|
|
ticker := time.NewTicker(r.ensurePeersPeriod)
|
2015-10-25 18:21:51 -07:00
|
|
|
for {
|
|
|
|
select {
|
2017-01-12 13:26:29 +04:00
|
|
|
case <-ticker.C:
|
2017-01-11 15:03:29 +04:00
|
|
|
r.ensurePeers()
|
2018-02-12 14:31:52 +04:00
|
|
|
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() {
|
2018-03-01 18:41:30 +01:00
|
|
|
var (
|
|
|
|
out, in, dial = r.Switch.NumPeers()
|
2018-08-15 02:25:56 +04:00
|
|
|
numToDial = r.Switch.MaxNumOutboundPeers() - (out + dial)
|
2018-03-01 18:41:30 +01:00
|
|
|
)
|
|
|
|
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-01-12 22:28:40 +04:00
|
|
|
|
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 ?
|
2018-03-01 18:41:30 +01:00
|
|
|
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
|
2018-03-01 18:41:30 +01:00
|
|
|
|
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
|
|
|
|
}
|
2018-09-05 19:52:22 +04:00
|
|
|
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
|
2018-03-08 13:59:30 +04:00
|
|
|
for _, addr := range toDial {
|
2018-03-08 15:55:23 +04:00
|
|
|
go r.dialPeer(addr)
|
2015-10-25 18:21:51 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// If we need more addresses, pick a random peer and ask for more.
|
2017-01-11 15:03:29 +04:00
|
|
|
if r.book.NeedMoreAddrs() {
|
2018-01-09 18:29:29 -06:00
|
|
|
peers := r.Switch.Peers().List()
|
|
|
|
peersCount := len(peers)
|
|
|
|
if peersCount > 0 {
|
2018-04-11 11:38:30 +02:00
|
|
|
peer := peers[cmn.RandInt()%peersCount] // nolint: gas
|
2018-01-09 18:29:29 -06:00
|
|
|
r.Logger.Info("We need more addresses. Sending pexRequest to random peer", "peer", peer)
|
2018-01-20 19:28:43 -05:00
|
|
|
r.RequestAddrs(peer)
|
2015-10-25 18:21:51 -07:00
|
|
|
}
|
|
|
|
}
|
2018-01-09 17:09:09 -06:00
|
|
|
|
2018-01-14 03:56:15 -05:00
|
|
|
// If we are not connected to nor dialing anybody, fallback to dialing a seed.
|
2018-03-01 18:41:30 +01:00
|
|
|
if out+in+dial+len(toDial) == 0 {
|
2018-01-14 03:56:15 -05:00
|
|
|
r.Logger.Info("No addresses to dial nor connected peers. Falling back to seeds")
|
2018-01-20 18:28:40 -05:00
|
|
|
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) {
|
|
|
|
attempts, lastDialed := r.dialAttemptsInfo(addr)
|
2018-03-11 13:20:51 +04:00
|
|
|
|
|
|
|
if attempts > maxAttemptsToDial {
|
|
|
|
r.Logger.Error("Reached max attempts to dial", "addr", addr, "attempts", attempts)
|
|
|
|
r.book.MarkBad(addr)
|
|
|
|
return
|
|
|
|
}
|
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 {
|
2018-04-11 11:38:30 +02:00
|
|
|
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 {
|
|
|
|
r.Logger.Debug("Too early to dial", "addr", addr, "backoff_duration", backoffDuration, "last_dialed", lastDialed, "time_since", sinceLastDialed)
|
|
|
|
return
|
|
|
|
}
|
2018-03-08 15:55:23 +04:00
|
|
|
}
|
|
|
|
|
|
|
|
err := r.Switch.DialPeerWithAddress(addr, false)
|
|
|
|
if err != nil {
|
2018-03-11 13:20:51 +04:00
|
|
|
r.Logger.Error("Dialing failed", "addr", addr, "err", err, "attempts", attempts)
|
2018-03-08 15:55:23 +04:00
|
|
|
// TODO: detect more "bad peer" scenarios
|
|
|
|
if _, ok := err.(p2p.ErrSwitchAuthenticationFailure); ok {
|
|
|
|
r.book.MarkBad(addr)
|
2018-03-23 09:48:27 +01:00
|
|
|
r.attemptsToDial.Delete(addr.DialString())
|
2018-03-08 15:55:23 +04:00
|
|
|
} else {
|
|
|
|
r.book.MarkAttempt(addr)
|
2018-03-23 09:48:27 +01:00
|
|
|
// FIXME: if the addr is going to be removed from the addrbook (hard to
|
|
|
|
// tell at this point), we need to Delete it from attemptsToDial, not
|
|
|
|
// record another attempt.
|
|
|
|
// record attempt
|
|
|
|
r.attemptsToDial.Store(addr.DialString(), _attemptsToDial{attempts + 1, time.Now()})
|
2018-03-08 15:55:23 +04:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// cleanup any history
|
|
|
|
r.attemptsToDial.Delete(addr.DialString())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-04 17:35:08 -07: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 {
|
2018-08-04 17:35:08 -07:00
|
|
|
return -1, nil, nil
|
2018-01-14 13:03:57 -05:00
|
|
|
}
|
2018-07-31 13:09:01 -07:00
|
|
|
netAddrs, errs := p2p.NewNetAddressStrings(r.config.Seeds)
|
2018-08-04 17:35:08 -07:00
|
|
|
numOnline = lSeeds - len(errs)
|
2018-01-14 13:03:57 -05:00
|
|
|
for _, err := range errs {
|
2018-08-04 17:35:08 -07:00
|
|
|
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
|
|
|
}
|
|
|
|
}
|
2018-08-04 17:35:08 -07:00
|
|
|
return
|
2018-01-14 03:56:15 -05:00
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
// randomly dial seeds until we connect to one or exhaust them
|
|
|
|
func (r *PEXReactor) dialSeeds() {
|
2018-08-05 11:47:01 -07:00
|
|
|
perm := cmn.RandPerm(len(r.seedAddrs))
|
2018-01-20 21:12:04 -05:00
|
|
|
// perm := r.Switch.rng.Perm(lSeeds)
|
2018-01-20 18:28:40 -05:00
|
|
|
for _, i := range perm {
|
|
|
|
// dial a random seed
|
2018-07-31 13:09:01 -07:00
|
|
|
seedAddr := r.seedAddrs[i]
|
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
|
|
|
err := r.Switch.DialPeerWithAddress(seedAddr, false)
|
|
|
|
if err == nil {
|
2018-01-20 18:28:40 -05:00
|
|
|
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)
|
2018-01-20 18:28:40 -05:00
|
|
|
}
|
|
|
|
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
|
|
|
}
|
2018-04-02 10:21:17 +02:00
|
|
|
return 0
|
2018-03-09 16:23:52 +04:00
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05: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() {
|
2018-01-09 20:12:41 -05:00
|
|
|
// Do an initial crawl
|
|
|
|
r.crawlPeers()
|
|
|
|
|
|
|
|
// Fire periodically
|
2018-01-20 18:28:40 -05:00
|
|
|
ticker := time.NewTicker(defaultCrawlPeersPeriod)
|
2018-01-09 20:12:41 -05:00
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ticker.C:
|
|
|
|
r.attemptDisconnects()
|
|
|
|
r.crawlPeers()
|
2018-02-12 14:31:52 +04:00
|
|
|
case <-r.Quit():
|
2018-01-09 20:12:41 -05:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-03-01 18:41:30 +01:00
|
|
|
// hasPotentialPeers indicates if there is a potential peer to connect to, by
|
|
|
|
// consulting the Switch as well as the AddrBook.
|
|
|
|
func (r *PEXReactor) hasPotentialPeers() bool {
|
|
|
|
out, in, dial := r.Switch.NumPeers()
|
|
|
|
|
|
|
|
return out+in+dial > 0 && len(r.book.ListOfKnownAddresses()) > 0
|
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
// crawlPeerInfo handles temporary data needed for the
|
2018-01-09 20:12:41 -05:00
|
|
|
// network crawling performed during seed/crawler mode.
|
2018-01-20 18:28:40 -05:00
|
|
|
type crawlPeerInfo struct {
|
|
|
|
// The listening address of a potential peer we learned about
|
2018-01-21 00:33:53 -05:00
|
|
|
Addr *p2p.NetAddress
|
2018-01-09 20:12:41 -05:00
|
|
|
|
|
|
|
// The last time we attempt to reach this address
|
|
|
|
LastAttempt time.Time
|
|
|
|
|
|
|
|
// The last time we successfully reached this address
|
|
|
|
LastSuccess time.Time
|
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
// oldestFirst implements sort.Interface for []crawlPeerInfo
|
2018-01-09 20:12:41 -05:00
|
|
|
// based on the LastAttempt field.
|
2018-01-20 18:28:40 -05:00
|
|
|
type oldestFirst []crawlPeerInfo
|
2018-01-09 20:12:41 -05:00
|
|
|
|
|
|
|
func (of oldestFirst) Len() int { return len(of) }
|
|
|
|
func (of oldestFirst) Swap(i, j int) { of[i], of[j] = of[j], of[i] }
|
|
|
|
func (of oldestFirst) Less(i, j int) bool { return of[i].LastAttempt.Before(of[j].LastAttempt) }
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
// getPeersToCrawl returns addresses of potential peers that we wish to validate.
|
2018-01-09 20:12:41 -05:00
|
|
|
// NOTE: The status information is ordered as described above.
|
2018-01-20 18:28:40 -05:00
|
|
|
func (r *PEXReactor) getPeersToCrawl() []crawlPeerInfo {
|
2018-01-09 20:12:41 -05:00
|
|
|
var of oldestFirst
|
|
|
|
|
2018-01-20 19:28:43 -05:00
|
|
|
// TODO: be more selective
|
2018-01-09 20:12:41 -05:00
|
|
|
addrs := r.book.ListOfKnownAddresses()
|
|
|
|
for _, addr := range addrs {
|
2018-01-20 18:28:40 -05:00
|
|
|
if len(addr.ID()) == 0 {
|
|
|
|
continue // dont use peers without id
|
2018-01-09 20:12:41 -05:00
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
of = append(of, crawlPeerInfo{
|
2018-01-09 20:12:41 -05:00
|
|
|
Addr: addr.Addr,
|
|
|
|
LastAttempt: addr.LastAttempt,
|
|
|
|
LastSuccess: addr.LastSuccess,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
sort.Sort(of)
|
|
|
|
return of
|
|
|
|
}
|
|
|
|
|
|
|
|
// crawlPeers will crawl the network looking for new peer addresses. (once)
|
|
|
|
func (r *PEXReactor) crawlPeers() {
|
2018-01-20 18:28:40 -05:00
|
|
|
peerInfos := r.getPeersToCrawl()
|
2018-01-09 20:12:41 -05:00
|
|
|
|
|
|
|
now := time.Now()
|
|
|
|
// Use addresses we know of to reach additional peers
|
2018-01-20 18:28:40 -05:00
|
|
|
for _, pi := range peerInfos {
|
2018-01-09 20:12:41 -05:00
|
|
|
// Do not attempt to connect with peers we recently dialed
|
2018-01-20 18:28:40 -05:00
|
|
|
if now.Sub(pi.LastAttempt) < defaultCrawlPeerInterval {
|
2018-01-09 20:12:41 -05:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
// Otherwise, attempt to connect with the known address
|
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
|
|
|
err := r.Switch.DialPeerWithAddress(pi.Addr, false)
|
2018-01-09 20:12:41 -05:00
|
|
|
if err != nil {
|
2018-01-20 18:28:40 -05:00
|
|
|
r.book.MarkAttempt(pi.Addr)
|
2018-01-09 20:12:41 -05:00
|
|
|
continue
|
|
|
|
}
|
2018-03-24 17:04:59 +01:00
|
|
|
// Ask for more addresses
|
|
|
|
peer := r.Switch.Peers().Get(pi.Addr.ID)
|
2018-06-29 16:03:31 +04:00
|
|
|
if peer != nil {
|
|
|
|
r.RequestAddrs(peer)
|
|
|
|
}
|
2018-01-09 20:12:41 -05:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-01-20 18:28:40 -05:00
|
|
|
// 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() {
|
2018-01-20 18:28:40 -05:00
|
|
|
for _, peer := range r.Switch.Peers().List() {
|
2018-03-24 17:04:32 +01:00
|
|
|
if peer.Status().Duration < defaultSeedDisconnectWaitPeriod {
|
2018-01-09 20:12:41 -05:00
|
|
|
continue
|
|
|
|
}
|
2018-01-20 18:28:40 -05:00
|
|
|
if peer.IsPersistent() {
|
2018-01-09 20:12:41 -05:00
|
|
|
continue
|
|
|
|
}
|
2018-01-20 18:28:40 -05:00
|
|
|
r.Switch.StopPeerGracefully(peer)
|
2018-01-14 13:03:57 -05:00
|
|
|
}
|
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
|
|
|
|
2018-07-09 13:01:23 +04:00
|
|
|
func decodeMsg(bz []byte) (msg PexMessage, err error) {
|
2018-04-09 15:14:33 +03:00
|
|
|
if len(bz) > maxMsgSize {
|
2018-07-09 13:01:23 +04:00
|
|
|
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
2018-04-09 15:14:33 +03:00
|
|
|
}
|
2018-07-09 13:11:41 +04: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)
|
|
|
|
}
|