tendermint/p2p/pex/pex_reactor.go

700 lines
20 KiB
Go
Raw Normal View History

2018-01-20 18:12:04 -08:00
package pex
2015-10-25 18:21:51 -07:00
import (
"fmt"
"reflect"
2018-01-09 17:12:41 -08:00
"sort"
"sync"
2015-10-25 18:21:51 -07:00
"time"
2018-03-25 21:40:02 -07:00
"github.com/tendermint/go-amino"
2017-05-02 00:53:32 -07:00
cmn "github.com/tendermint/tmlibs/common"
2018-01-20 16:12:04 -08:00
2018-01-20 18:12:04 -08:00
"github.com/tendermint/tendermint/p2p"
2018-01-20 21:33:53 -08:00
"github.com/tendermint/tendermint/p2p/conn"
2015-10-25 18:21:51 -07:00
)
2018-01-20 18:12:04 -08:00
type Peer = p2p.Peer
2015-10-25 18:21:51 -07:00
const (
2017-01-16 11:57:07 -08:00
// PexChannel is a channel for PEX messages
2017-01-12 05:56:40 -08:00
PexChannel = byte(0x00)
2017-01-16 11:57:07 -08: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 12:39:09 -07:00
// small request results in up to maxMsgSize response
maxMsgSize = maxAddressSize * maxGetSelection
// ensure we have enough peers
defaultEnsurePeersPeriod = 30 * time.Second
2018-04-28 12:52:05 -07:00
defaultMinNumOutboundPeers = p2p.DefaultMinNumOutboundPeers
2018-01-13 22:40:29 -08:00
// Seed/Crawler constants
// 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 01:20:51 -08:00
2018-03-11 03:13:34 -07:00
maxAttemptsToDial = 16 // ~ 35h in total (last attempt - 18h)
2018-03-24 09:49:42 -07: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 03:03:29 -08:00
// PEXReactor handles PEX (peer exchange) and ensures that an
// adequate number of peers are connected to the switch.
2017-01-16 11:57:07 -08:00
//
// It uses `AddrBook` (address book) to store `NetAddress`es of the peers.
2017-01-17 10:30:03 -08:00
//
// ## Preventing abuse
//
2018-01-14 00:22:01 -08: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 18:12:04 -08:00
p2p.BaseReactor
2015-10-25 18:21:51 -07:00
2018-01-20 18:12:04 -08:00
book AddrBook
config *PEXReactorConfig
2018-03-20 13:41:08 -07:00
ensurePeersPeriod time.Duration // TODO: should go in the config
2017-01-12 05:56:40 -08:00
2018-01-14 00:22:01 -08:00
// maps to prevent abuse
2018-01-14 10:03:57 -08:00
requestsSent *cmn.CMap // ID->struct{}: unanswered send requests
lastReceivedRequests *cmn.CMap // ID->time.Time: last time peer requested from us
2018-03-11 03:00:49 -07:00
attemptsToDial sync.Map // address (string) -> {number of attempts (int), last time dialed (time.Time)}
2015-10-25 18:21:51 -07:00
}
2018-03-20 13:41:08 -07:00
func (pexR *PEXReactor) minReceiveRequestInterval() time.Duration {
// NOTE: must be less than ensurePeersPeriod, otherwise we'll request
// peers too quickly from others and they'll think we're bad!
return pexR.ensurePeersPeriod / 3
}
// PEXReactorConfig holds reactor specific configuration data.
type PEXReactorConfig struct {
// Seed/Crawler mode
SeedMode bool
// Seeds is a list of addresses reactor may use
// if it can't connect to peers in the addrbook.
Seeds []string
// PrivatePeerIDs is a list of peer IDs, which must not be gossiped to other
// peers.
PrivatePeerIDs []string
}
2018-03-11 03:00:49 -07:00
type _attemptsToDial struct {
number int
lastDialed time.Time
}
2017-01-16 11:57:07 -08:00
// NewPEXReactor creates new PEX reactor.
2018-01-20 18:12:04 -08:00
func NewPEXReactor(b AddrBook, config *PEXReactorConfig) *PEXReactor {
2017-01-11 03:03:29 -08:00
r := &PEXReactor{
2018-01-14 00:22:01 -08:00
book: b,
config: config,
ensurePeersPeriod: defaultEnsurePeersPeriod,
requestsSent: cmn.NewCMap(),
lastReceivedRequests: cmn.NewCMap(),
2015-10-25 18:21:51 -07:00
}
2018-01-20 18:12:04 -08:00
r.BaseReactor = *p2p.NewBaseReactor("PEXReactor", r)
2017-01-11 03:03:29 -08:00
return r
2015-10-25 18:21:51 -07:00
}
2017-01-16 11:57:07 -08:00
// OnStart implements BaseService
2017-01-11 03:03:29 -08:00
func (r *PEXReactor) OnStart() error {
2017-09-21 09:38:48 -07:00
if err := r.BaseReactor.OnStart(); err != nil {
return err
}
err := r.book.Start()
if err != nil && err != cmn.ErrAlreadyStarted {
2017-09-21 09:38:48 -07:00
return err
}
2018-01-14 00:56:15 -08:00
2018-01-14 10:03:57 -08:00
// return err if user provided a bad seed address
2018-04-28 12:19:33 -07:00
// or a host name that we cant resolve
2018-01-14 00:56:15 -08:00
if err := r.checkSeeds(); err != nil {
return err
}
2018-01-13 22:40:29 -08: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 11:57:07 -08:00
// OnStop implements BaseService
2017-01-11 03:03:29 -08:00
func (r *PEXReactor) OnStop() {
r.BaseReactor.OnStop()
r.book.Stop()
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
// GetChannels implements Reactor
2018-01-20 21:33:53 -08: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 03:03:29 -08:00
// AddPeer implements Reactor by adding peer to the address book (if inbound)
// or by requesting more addresses (if outbound).
2017-09-12 17:49:22 -07:00
func (r *PEXReactor) AddPeer(p Peer) {
if p.IsOutbound() {
2018-01-14 00:22:01 -08: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 03:03:29 -08:00
if r.book.NeedMoreAddrs() {
r.RequestAddrs(p)
2015-10-25 18:21:51 -07:00
}
2018-01-13 13:06:31 -08:00
} else {
2018-04-28 10:08:44 -07:00
// inbound peer is its own source
2018-01-13 13:14:28 -08:00
addr := p.NodeInfo().NetAddress()
2018-04-28 10:08:44 -07:00
src := addr
// ignore private addrs
if isAddrPrivate(addr, r.config.PrivatePeerIDs) {
return
}
// 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 10:08:44 -07: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:
// non-routable, self, full book, etc.
r.Logger.Debug("Failed to add new address", "err", err)
}
2015-10-25 18:21:51 -07:00
}
}
2017-04-20 02:04:40 -07:00
// RemovePeer implements Reactor.
2017-09-12 17:49:22 -07:00
func (r *PEXReactor) RemovePeer(p Peer, reason interface{}) {
2018-01-14 00:22:01 -08:00
id := string(p.ID())
r.requestsSent.Delete(id)
r.lastReceivedRequests.Delete(id)
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
// Receive implements Reactor by handling incoming PEX messages.
2017-09-12 17:49:22 -07:00
func (r *PEXReactor) Receive(chID byte, src Peer, msgBytes []byte) {
2018-03-25 21:40:02 -07:00
msg, err := DecodeMessage(msgBytes)
2015-10-25 18:21:51 -07:00
if err != nil {
2018-03-04 01:42:45 -08: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 11:37:57 -08: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:
// Check we're not receiving too many requests
2018-01-14 00:22:01 -08:00
if err := r.receiveRequest(src); err != nil {
r.Switch.StopPeerForError(src, err)
return
}
// Seeds disconnect after sending a batch of addrs
2018-03-20 13:41:08 -07: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
if r.config.SeedMode {
2018-03-24 09:49:42 -07:00
r.SendAddrs(src, r.book.GetSelectionWithBias(biasToSelectNewPeers))
r.Switch.StopPeerGracefully(src)
} else {
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 00:22:01 -08:00
r.Switch.StopPeerForError(src, err)
return
2015-10-25 18:21:51 -07:00
}
default:
2017-05-02 00:53:32 -07:00
r.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
2015-10-25 18:21:51 -07:00
}
}
2018-03-20 13:41:08 -07:00
// enforces a minimum amount of time between requests
2018-01-14 00:22:01 -08: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 10:03:57 -08:00
lastReceived = time.Now()
2018-01-14 00:22:01 -08:00
r.lastReceivedRequests.Set(id, lastReceived)
return nil
}
now := time.Now()
2018-03-20 13:41:08 -07:00
minInterval := r.minReceiveRequestInterval()
if now.Sub(lastReceived) < minInterval {
return fmt.Errorf("Peer (%v) send next PEX request too soon. lastReceived: %v, now: %v, minInterval: %v. Disconnecting",
src.ID(),
lastReceived,
now,
minInterval,
)
2018-01-14 00:22:01 -08:00
}
r.lastReceivedRequests.Set(id, now)
return nil
}
// RequestAddrs asks peer for more addresses if we do not already
2018-01-14 00:22:01 -08:00
// have a request out for this peer.
func (r *PEXReactor) RequestAddrs(p Peer) {
2018-01-14 00:22:01 -08:00
id := string(p.ID())
if r.requestsSent.Has(id) {
return
}
r.requestsSent.Set(id, struct{}{})
2018-03-25 21:40:02 -07:00
p.Send(PexChannel, cdc.MustMarshalBinary(&pexRequestMessage{}))
2015-10-25 18:21:51 -07:00
}
// ReceiveAddrs adds the given addrs to the addrbook if theres an open
2018-01-14 00:22:01 -08: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-20 21:33:53 -08:00
func (r *PEXReactor) ReceiveAddrs(addrs []*p2p.NetAddress, src Peer) error {
2018-01-14 00:22:01 -08:00
2018-04-28 10:08:44 -07:00
id := string(src.ID())
2018-01-14 00:22:01 -08:00
if !r.requestsSent.Has(id) {
2018-03-25 21:40:02 -07:00
return cmn.NewError("Received unsolicited pexAddrsMessage")
2018-01-14 00:22:01 -08:00
}
r.requestsSent.Delete(id)
srcAddr := src.NodeInfo().NetAddress()
for _, netAddr := range addrs {
2018-04-28 12:01:33 -07:00
// NOTE: GetSelection methods should never return nil addrs
2018-04-28 10:08:44 -07:00
if netAddr == nil {
return cmn.NewError("received nil addr")
}
// ignore private peers
// TODO: give private peers to AddrBook so it can enforce this on AddAddress.
// We'd then have to check for ErrPrivatePeer on AddAddress here, which is
// an error we just ignore (maybe peer is probing us for our private peers :P)
if isAddrPrivate(netAddr, r.config.PrivatePeerIDs) {
continue
}
err := r.book.AddAddress(netAddr, srcAddr)
r.logErrAddrBook(err)
2018-01-14 00:22:01 -08:00
}
return nil
}
2017-01-11 03:03:29 -08:00
// SendAddrs sends addrs to the peer.
2018-01-20 21:33:53 -08:00
func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*p2p.NetAddress) {
2018-03-25 21:40:02 -07:00
p.Send(PexChannel, cdc.MustMarshalBinary(&pexAddrsMessage{Addrs: netAddrs}))
2015-10-25 18:21:51 -07:00
}
2017-01-11 12:17:15 -08: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 03:03:29 -08:00
func (r *PEXReactor) ensurePeersRoutine() {
var (
seed = cmn.NewRand()
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 00:56:15 -08: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 03:03:29 -08:00
r.ensurePeers()
case <-r.Quit():
2017-01-12 05:56:40 -08:00
ticker.Stop()
return
2015-10-25 18:21:51 -07:00
}
}
}
2017-01-11 12:17:15 -08:00
// ensurePeers ensures that sufficient peers are connected. (once)
2017-04-20 01:49:54 -07: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 03:03:29 -08:00
func (r *PEXReactor) ensurePeers() {
var (
out, in, dial = r.Switch.NumPeers()
numToDial = defaultMinNumOutboundPeers - (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-15 20:30:38 -08: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 16:00:44 -08: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-20 21:33:53 -08:00
toDial := make(map[p2p.ID]*p2p.NetAddress)
2017-11-15 20:30:38 -08:00
// Try maxAttempts times to pick numToDial addresses to dial
maxAttempts := numToDial * 3
2017-11-15 20:30:38 -08: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 13:14:28 -08:00
if _, selected := toDial[try.ID]; selected {
2017-11-20 11:30:05 -08:00
continue
}
if dialling := r.Switch.IsDialing(try.ID); dialling {
2017-11-20 11:30:05 -08:00
continue
}
if connected := r.Switch.Peers().Has(try.ID); connected {
2015-10-25 18:21:51 -07:00
continue
}
2018-03-20 13:41:08 -07: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 12:01:33 -07:00
// before dialling again, or have dialed too many times already
2017-11-15 20:30:38 -08:00
r.Logger.Info("Will dial address", "addr", try)
2018-01-13 13:14:28 -08:00
toDial[try.ID] = try
2015-10-25 18:21:51 -07:00
}
// Dial picked addresses
for _, addr := range toDial {
2018-03-08 03:55:23 -08: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 03:03:29 -08:00
if r.book.NeedMoreAddrs() {
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
}
}
2018-01-14 00:56:15 -08:00
// If we are not connected to nor dialing anybody, fallback to dialing a seed.
if out+in+dial+len(toDial) == 0 {
2018-01-14 00:56:15 -08:00
r.Logger.Info("No addresses to dial nor connected peers. Falling back to seeds")
r.dialSeeds()
2018-01-14 00:56:15 -08:00
}
}
2018-03-20 13:41:08 -07: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 03:00:49 -07:00
}
2018-03-20 13:41:08 -07:00
atd := _attempts.(_attemptsToDial)
return atd.number, atd.lastDialed
}
func (r *PEXReactor) dialPeer(addr *p2p.NetAddress) {
attempts, lastDialed := r.dialAttemptsInfo(addr)
2018-03-11 01:20:51 -08:00
if attempts > maxAttemptsToDial {
r.Logger.Error("Reached max attempts to dial", "addr", addr, "attempts", attempts)
r.book.MarkBad(addr)
return
}
2018-03-08 03:55:23 -08:00
// exponential backoff if it's not our first attempt to dial given address
2018-03-11 03:00:49 -07:00
if attempts > 0 {
jitterSeconds := time.Duration(cmn.RandFloat64() * float64(time.Second)) // 1s == (1e9 ns)
2018-03-08 03:55:23 -08:00
backoffDuration := jitterSeconds + ((1 << uint(attempts)) * time.Second)
2018-03-11 03:13:34 -07:00
sinceLastDialed := time.Since(lastDialed)
2018-03-11 03:00:49 -07: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 03:55:23 -08:00
}
err := r.Switch.DialPeerWithAddress(addr, false)
if err != nil {
2018-03-11 01:20:51 -08:00
r.Logger.Error("Dialing failed", "addr", addr, "err", err, "attempts", attempts)
2018-03-08 03:55:23 -08:00
// TODO: detect more "bad peer" scenarios
if _, ok := err.(p2p.ErrSwitchAuthenticationFailure); ok {
r.book.MarkBad(addr)
r.attemptsToDial.Delete(addr.DialString())
2018-03-08 03:55:23 -08:00
} else {
r.book.MarkAttempt(addr)
// 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 03:55:23 -08:00
}
} else {
// cleanup any history
r.attemptsToDial.Delete(addr.DialString())
}
}
2018-01-14 10:03:57 -08:00
// check seed addresses are well formed
2018-01-14 00:56:15 -08:00
func (r *PEXReactor) checkSeeds() error {
lSeeds := len(r.config.Seeds)
2018-01-14 10:03:57 -08:00
if lSeeds == 0 {
return nil
}
2018-01-20 21:33:53 -08:00
_, errs := p2p.NewNetAddressStrings(r.config.Seeds)
2018-01-14 10:03:57 -08:00
for _, err := range errs {
if err != nil {
return err
2018-01-14 00:56:15 -08:00
}
}
2018-01-14 10:03:57 -08:00
return nil
2018-01-14 00:56:15 -08:00
}
// randomly dial seeds until we connect to one or exhaust them
func (r *PEXReactor) dialSeeds() {
lSeeds := len(r.config.Seeds)
if lSeeds == 0 {
return
}
2018-01-20 21:33:53 -08:00
seedAddrs, _ := p2p.NewNetAddressStrings(r.config.Seeds)
perm := cmn.RandPerm(lSeeds)
2018-01-20 18:12:04 -08:00
// perm := r.Switch.rng.Perm(lSeeds)
for _, i := range perm {
// dial a random seed
seedAddr := 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 03:54:40 -08:00
err := r.Switch.DialPeerWithAddress(seedAddr, false)
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 03:54:40 -08:00
r.Switch.Logger.Error("Error dialing seed", "err", err, "seed", seedAddr)
}
r.Switch.Logger.Error("Couldn't connect to any seeds")
}
2018-03-09 04:23:52 -08: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 03:00:49 -07:00
lAttempts, attempted := r.attemptsToDial.Load(addr.DialString())
2018-03-09 04:23:52 -08:00
if attempted {
2018-03-11 03:00:49 -07:00
return lAttempts.(_attemptsToDial).number
2018-03-09 04:23:52 -08:00
}
return 0
2018-03-09 04:23:52 -08:00
}
//----------------------------------------------------------
2018-01-09 17:12:41 -08: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-13 22:40:29 -08:00
func (r *PEXReactor) crawlPeersRoutine() {
2018-01-09 17:12:41 -08:00
// Do an initial crawl
r.crawlPeers()
// Fire periodically
ticker := time.NewTicker(defaultCrawlPeersPeriod)
2018-01-09 17:12:41 -08:00
for {
select {
case <-ticker.C:
r.attemptDisconnects()
r.crawlPeers()
case <-r.Quit():
2018-01-09 17:12:41 -08:00
return
}
}
}
// 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
}
// crawlPeerInfo handles temporary data needed for the
2018-01-09 17:12:41 -08:00
// network crawling performed during seed/crawler mode.
type crawlPeerInfo struct {
// The listening address of a potential peer we learned about
2018-01-20 21:33:53 -08:00
Addr *p2p.NetAddress
2018-01-09 17:12:41 -08:00
// The last time we attempt to reach this address
LastAttempt time.Time
// The last time we successfully reached this address
LastSuccess time.Time
}
// oldestFirst implements sort.Interface for []crawlPeerInfo
2018-01-09 17:12:41 -08:00
// based on the LastAttempt field.
type oldestFirst []crawlPeerInfo
2018-01-09 17:12:41 -08: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) }
// getPeersToCrawl returns addresses of potential peers that we wish to validate.
2018-01-09 17:12:41 -08:00
// NOTE: The status information is ordered as described above.
func (r *PEXReactor) getPeersToCrawl() []crawlPeerInfo {
2018-01-09 17:12:41 -08:00
var of oldestFirst
// TODO: be more selective
2018-01-09 17:12:41 -08:00
addrs := r.book.ListOfKnownAddresses()
for _, addr := range addrs {
if len(addr.ID()) == 0 {
continue // dont use peers without id
2018-01-09 17:12:41 -08:00
}
of = append(of, crawlPeerInfo{
2018-01-09 17:12:41 -08: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() {
peerInfos := r.getPeersToCrawl()
2018-01-09 17:12:41 -08:00
now := time.Now()
// Use addresses we know of to reach additional peers
for _, pi := range peerInfos {
2018-01-09 17:12:41 -08:00
// Do not attempt to connect with peers we recently dialed
if now.Sub(pi.LastAttempt) < defaultCrawlPeerInterval {
2018-01-09 17:12:41 -08: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 03:54:40 -08:00
err := r.Switch.DialPeerWithAddress(pi.Addr, false)
2018-01-09 17:12:41 -08:00
if err != nil {
r.book.MarkAttempt(pi.Addr)
2018-01-09 17:12:41 -08:00
continue
}
// Ask for more addresses
peer := r.Switch.Peers().Get(pi.Addr.ID)
r.RequestAddrs(peer)
2018-01-09 17:12:41 -08:00
}
}
// attemptDisconnects checks if we've been with each peer long enough to disconnect
2018-01-09 17:12:41 -08:00
func (r *PEXReactor) attemptDisconnects() {
for _, peer := range r.Switch.Peers().List() {
if peer.Status().Duration < defaultSeedDisconnectWaitPeriod {
2018-01-09 17:12:41 -08:00
continue
}
if peer.IsPersistent() {
2018-01-09 17:12:41 -08:00
continue
}
r.Switch.StopPeerGracefully(peer)
2018-01-14 10:03:57 -08:00
}
2015-10-25 18:21:51 -07:00
}
2018-04-28 10:08:44 -07:00
// isAddrPrivate returns true if addr.ID is a private ID.
func isAddrPrivate(addr *p2p.NetAddress, privatePeerIDs []string) bool {
for _, id := range privatePeerIDs {
if string(addr.ID) == id {
return true
}
}
return false
}
2015-10-25 18:21:51 -07:00
//-----------------------------------------------------------------------------
// Messages
2017-01-16 11:57:07 -08: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-25 21:40:02 -07: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
2017-01-16 11:57:07 -08:00
// DecodeMessage implements interface registered above.
2018-03-25 21:40:02 -07:00
func DecodeMessage(bz []byte) (msg PexMessage, err error) {
2018-04-09 05:14:33 -07:00
if len(bz) > maxMsgSize {
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)",
len(bz), maxMsgSize)
}
2018-03-25 21:40:02 -07:00
err = cdc.UnmarshalBinary(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-20 21:33:53 -08:00
Addrs []*p2p.NetAddress
2015-10-25 18:21:51 -07:00
}
func (m *pexAddrsMessage) String() string {
return fmt.Sprintf("[pexAddrs %v]", m.Addrs)
}