tendermint/p2p/pex_reactor.go

686 lines
19 KiB
Go
Raw Normal View History

2015-10-25 18:21:51 -07:00
package p2p
import (
"bytes"
"fmt"
"math/rand"
"reflect"
2018-01-09 17:12:41 -08:00
"sort"
2015-10-25 18:21:51 -07:00
"time"
2018-01-14 00:22:01 -08:00
"github.com/pkg/errors"
2017-04-07 03:57:03 -07:00
wire "github.com/tendermint/go-wire"
2017-05-02 00:53:32 -07:00
cmn "github.com/tendermint/tmlibs/common"
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
2017-01-12 05:56:40 -08:00
// period to ensure peers connected
2017-01-11 12:17:15 -08:00
defaultEnsurePeersPeriod = 30 * time.Second
2015-10-25 18:21:51 -07:00
minNumOutboundPeers = 10
2015-11-10 12:29:43 -08:00
maxPexMessageSize = 1048576 // 1MB
2018-01-13 22:40:29 -08:00
// Seed/Crawler constants
defaultSeedDisconnectWaitPeriod = 2 * time.Minute
defaultCrawlPeerInterval = 2 * time.Minute
defaultCrawlPeersPeriod = 30 * time.Second
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 {
BaseReactor
2017-01-11 12:17:15 -08:00
book *AddrBook
config *PEXReactorConfig
2017-01-11 12:17:15 -08:00
ensurePeersPeriod time.Duration
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
2015-10-25 18:21:51 -07:00
}
// PEXReactorConfig holds reactor specific configuration data.
type PEXReactorConfig struct {
// Seeds is a list of addresses reactor may use if it can't connect to peers
// in the addrbook.
Seeds []string
}
2017-01-16 11:57:07 -08:00
// NewPEXReactor creates new PEX reactor.
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
}
2017-05-02 00:53:32 -07:00
r.BaseReactor = *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-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
func (r *PEXReactor) GetChannels() []*ChannelDescriptor {
2015-10-25 18:21:51 -07:00
return []*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.RequestPEX(p)
2015-10-25 18:21:51 -07:00
}
2018-01-13 13:06:31 -08:00
} else {
2018-01-14 00:22:01 -08:00
// For inbound peers, the peer is its own source,
// and its NodeInfo has already been validated.
// Let the ensurePeersRoutine handle asking for more
// peers when we need - we don't trust inbound peers as much.
2018-01-13 13:14:28 -08:00
addr := p.NodeInfo().NetAddress()
2017-01-11 07:23:31 -08:00
r.book.AddAddress(addr, addr)
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) {
2015-10-25 18:21:51 -07:00
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
r.Logger.Error("Error decoding message", "err", 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:
2018-01-14 00:22:01 -08:00
// We received a request for peers from src.
if err := r.receiveRequest(src); err != nil {
r.Switch.StopPeerForError(src, err)
return
}
2017-01-11 03:03:29 -08:00
r.SendAddrs(src, r.book.GetSelection())
2015-10-25 18:21:51 -07:00
case *pexAddrsMessage:
// We received some peer addresses from src.
2018-01-14 00:22:01 -08:00
if err := r.ReceivePEX(msg.Addrs, src); err != nil {
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-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()
if now.Sub(lastReceived) < r.ensurePeersPeriod/3 {
return fmt.Errorf("Peer (%v) is sending too many PEX requests. Disconnecting", src.ID())
}
r.lastReceivedRequests.Set(id, now)
return nil
}
// RequestPEX asks peer for more addresses if we do not already
// have a request out for this peer.
2017-09-12 17:49:22 -07:00
func (r *PEXReactor) RequestPEX(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{}{})
2017-01-11 03:03:29 -08:00
p.Send(PexChannel, struct{ PexMessage }{&pexRequestMessage{}})
2015-10-25 18:21:51 -07:00
}
2018-01-14 00:22:01 -08:00
// ReceivePEX adds the given addrs to the addrbook if theres an open
// request for this peer and deletes the open request.
// If there's no open request for the src peer, it returns an error.
func (r *PEXReactor) ReceivePEX(addrs []*NetAddress, src Peer) error {
id := string(src.ID())
if !r.requestsSent.Has(id) {
return errors.New("Received unsolicited pexAddrsMessage")
}
r.requestsSent.Delete(id)
srcAddr := src.NodeInfo().NetAddress()
for _, netAddr := range addrs {
if netAddr != nil {
r.book.AddAddress(netAddr, srcAddr)
}
}
return nil
}
2017-01-11 03:03:29 -08:00
// SendAddrs sends addrs to the peer.
func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*NetAddress) {
p.Send(PexChannel, struct{ PexMessage }{&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() {
2015-10-25 18:21:51 -07:00
// Randomize when routine starts
2017-01-11 12:17:15 -08:00
ensurePeersPeriodMs := r.ensurePeersPeriod.Nanoseconds() / 1e6
time.Sleep(time.Duration(rand.Int63n(ensurePeersPeriodMs)) * time.Millisecond)
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
//
// Old bucket / New bucket are arbitrary categories to denote whether an
// address is vetted or not, and this needs to be determined over time via a
// 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.
//
// TODO Basically, we need to work harder on our good-peer/bad-peer marking.
// What we're currently doing in terms of marking good/bad peers is just a
// placeholder. It should not be the case that an address becomes old/vetted
// upon a single successful connection.
2017-01-11 03:03:29 -08:00
func (r *PEXReactor) ensurePeers() {
numOutPeers, numInPeers, numDialing := r.Switch.NumPeers()
2015-10-25 18:21:51 -07:00
numToDial := minNumOutboundPeers - (numOutPeers + numDialing)
2017-05-02 00:53:32 -07:00
r.Logger.Info("Ensure peers", "numOutPeers", numOutPeers, "numDialing", numDialing, "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 ?
2017-11-15 20:30:38 -08:00
newBias := cmn.MinInt(numOutPeers, 8)*10 + 10
2015-10-25 18:21:51 -07:00
2018-01-13 13:14:28 -08:00
toDial := make(map[ID]*NetAddress)
2017-11-15 20:30:38 -08:00
// Try maxAttempts times to pick numToDial addresses to dial
maxAttempts := numToDial * 3
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
}
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 _, item := range toDial {
2015-10-25 18:21:51 -07:00
go func(picked *NetAddress) {
2017-01-11 03:03:29 -08:00
_, err := r.Switch.DialPeerWithAddress(picked, false)
2015-10-25 18:21:51 -07:00
if err != nil {
2017-01-11 03:03:29 -08:00
r.book.MarkAttempt(picked)
2015-10-25 18:21:51 -07:00
}
}(item)
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[rand.Int()%peersCount] // nolint: gas
r.Logger.Info("We need more addresses. Sending pexRequest to random peer", "peer", peer)
2017-01-11 03:03:29 -08:00
r.RequestPEX(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 numOutPeers+numInPeers+numDialing+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.dialSeed()
}
}
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
}
_, errs := NewNetAddressStrings(r.config.Seeds)
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
}
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(defaultSeedModePeriod)
for {
select {
case <-ticker.C:
r.attemptDisconnects()
r.crawlPeers()
case <-r.Quit:
return
}
}
}
// crawlStatus handles temporary data needed for the
// network crawling performed during seed/crawler mode.
type crawlStatus struct {
// The remote address of a potential peer we learned about
Addr *NetAddress
// Not empty if we are connected to the address
PeerID string
// 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 []crawlStatus
// based on the LastAttempt field.
type oldestFirst []crawlStatus
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) }
// getCrawlStatus returns addresses of potential peers that we wish to validate.
// NOTE: The status information is ordered as described above.
func (r *PEXReactor) getCrawlStatus() []crawlStatus {
var of oldestFirst
addrs := r.book.ListOfKnownAddresses()
// Go through all the addresses in the AddressBook
for _, addr := range addrs {
var peerID string
// Check if a peer is already connected from this addr
if p := r.Switch.peers.GetByRemoteAddr(addr.Addr); p != nil {
peerID = p.Key()
}
of = append(of, crawlStatus{
Addr: addr.Addr,
PeerID: peerID,
LastAttempt: addr.LastAttempt,
LastSuccess: addr.LastSuccess,
})
}
sort.Sort(of)
return of
}
// crawlPeers will crawl the network looking for new peer addresses. (once)
//
// TODO Basically, we need to work harder on our good-peer/bad-peer marking.
// What we're currently doing in terms of marking good/bad peers is just a
// placeholder. It should not be the case that an address becomes old/vetted
// upon a single successful connection.
func (r *PEXReactor) crawlPeers() {
crawlerStatus := r.getCrawlStatus()
now := time.Now()
// Use addresses we know of to reach additional peers
for _, cs := range crawlerStatus {
// Do not dial peers that are already connected
if cs.PeerID != "" {
continue
}
// Do not attempt to connect with peers we recently dialed
if now.Sub(cs.LastAttempt) < defaultCrawlPeerInterval {
continue
}
// Otherwise, attempt to connect with the known address
p, err := r.Switch.DialPeerWithAddress(cs.Addr, false)
if err != nil {
r.book.MarkAttempt(cs.Addr)
continue
}
// Enter the peer ID into our crawl status information
cs.PeerID = p.Key()
r.book.MarkGood(cs.Addr)
}
// Crawl the connected peers asking for more addresses
for _, cs := range crawlerStatus {
if cs.PeerID == "" {
continue
}
// We will wait a minimum period of time before crawling peers again
if now.Sub(cs.LastAttempt) >= defaultCrawlPeerInterval {
p := r.Switch.Peers().Get(cs.PeerID)
if p != nil {
r.RequestPEX(p)
r.book.MarkAttempt(cs.Addr)
}
}
}
}
// attemptDisconnects checks the crawlStatus info for Peers to disconnect from. (once)
func (r *PEXReactor) attemptDisconnects() {
crawlerStatus := r.getCrawlStatus()
now := time.Now()
// Go through each peer we have connected with
// looking for opportunities to disconnect
for _, cs := range crawlerStatus {
if cs.PeerID == "" {
continue
}
// Remain connected to each peer for a minimum period of time
if now.Sub(cs.LastSuccess) < defaultSeedDisconnectWaitPeriod {
continue
}
// Fetch the Peer using the saved ID
p := r.Switch.Peers().Get(cs.PeerID)
if p == nil {
continue
}
// Do not disconnect from persistent peers.
// Specifically, we need to remain connected to other seeds
if p.IsPersistent() {
continue
}
// Otherwise, disconnect from the peer
r.Switch.StopPeerGracefully(p)
}
}
// crawlStatus handles temporary data needed for the
// network crawling performed during seed/crawler mode.
type crawlStatus struct {
// The remote address of a potential peer we learned about
Addr *NetAddress
// Not empty if we are connected to the address
PeerID string
// The last time we attempt to reach this address
LastAttempt time.Time
// The last time we successfully reached this address
LastSuccess time.Time
}
// oldestAttempt implements sort.Interface for []crawlStatus
// based on the LastAttempt field.
type oldestAttempt []crawlStatus
func (oa oldestAttempt) Len() int { return len(oa) }
func (oa oldestAttempt) Swap(i, j int) { oa[i], oa[j] = oa[j], oa[i] }
func (oa oldestAttempt) Less(i, j int) bool { return oa[i].LastAttempt.Before(oa[j].LastAttempt) }
// getCrawlStatus returns addresses of potential peers that we wish to validate.
// NOTE: The status information is ordered as described above.
func (r *PEXReactor) getCrawlStatus() []crawlStatus {
var oa oldestAttempt
addrs := r.book.ListOfKnownAddresses()
// Go through all the addresses in the AddressBook
for _, addr := range addrs {
p := r.Switch.peers.GetByRemoteAddr(addr.Addr)
oa = append(oa, crawlStatus{
Addr: addr.Addr,
PeerID: p.Key(),
LastAttempt: addr.LastAttempt,
LastSuccess: addr.LastSuccess,
})
}
sort.Sort(oa)
return oa
}
// crawlPeers will crawl the network looking for new peer addresses. (once)
//
// TODO Basically, we need to work harder on our good-peer/bad-peer marking.
// What we're currently doing in terms of marking good/bad peers is just a
// placeholder. It should not be the case that an address becomes old/vetted
// upon a single successful connection.
func (r *PEXReactor) crawlPeers() {
crawlerStatus := r.getCrawlStatus()
now := time.Now()
// Use addresses we know of to reach additional peers
for _, cs := range crawlerStatus {
// Do not dial peers that are already connected
if cs.PeerID != "" {
continue
}
// Do not attempt to connect with peers we recently dialed
if now.Sub(cs.LastAttempt) < defaultCrawlPeerInterval {
continue
}
// Otherwise, attempt to connect with the known address
p, err := r.Switch.DialPeerWithAddress(cs.Addr, false)
if err != nil {
r.book.MarkAttempt(cs.Addr)
continue
}
// Enter the peer ID into our crawl status information
cs.PeerID = p.Key()
r.book.MarkGood(cs.Addr)
}
// Crawl the connected peers asking for more addresses
for _, cs := range crawlerStatus {
if cs.PeerID == "" {
continue
}
// We will wait a minimum period of time before crawling peers again
if now.Sub(cs.LastAttempt) >= defaultCrawlPeerInterval {
p := r.Switch.peers.Get(cs.PeerID)
if p != nil {
r.RequestPEX(p)
}
}
}
}
// attemptDisconnects checks the crawlStatus info for Peers to disconnect from. (once)
func (r *PEXReactor) attemptDisconnects() {
crawlerStatus := r.getCrawlStatus()
now := time.Now()
// Go through each peer we have connected with
// looking for opportunities to disconnect
for _, cs := range crawlerStatus {
if cs.PeerID == "" {
continue
}
// Remain connected to each peer for a minimum period of time
if now.Sub(cs.LastSuccess) < defaultSeedDisconnectWaitPeriod {
continue
}
// Fetch the Peer using the saved ID
p := r.Switch.peers.Get(cs.PeerID)
if p == nil {
continue
}
// Do not disconnect from persistent peers.
// Specifically, we need to remain connected to other seeds
if p.IsPersistent() {
continue
}
// Otherwise, disconnect from the peer
r.Switch.StopPeerGracefully(p)
}
}
2018-01-14 10:03:57 -08:00
// randomly dial seeds until we connect to one or exhaust them
func (r *PEXReactor) dialSeed() {
2018-01-14 00:56:15 -08:00
lSeeds := len(r.config.Seeds)
2018-01-14 10:03:57 -08:00
if lSeeds == 0 {
return
}
seedAddrs, _ := NewNetAddressStrings(r.config.Seeds)
perm := r.Switch.rng.Perm(lSeeds)
for _, i := range perm {
// dial a random seed
seedAddr := seedAddrs[i]
peer, err := r.Switch.DialPeerWithAddress(seedAddr, false)
if err != nil {
r.Switch.Logger.Error("Error dialing seed", "err", err, "seed", seedAddr)
} else {
r.Switch.Logger.Info("Connected to seed", "peer", peer)
return
2018-01-14 00:56:15 -08:00
}
}
2018-01-14 10:03:57 -08:00
r.Switch.Logger.Error("Couldn't connect to any seeds")
2015-10-25 18:21:51 -07:00
}
//-----------------------------------------------------------------------------
// Messages
const (
msgTypeRequest = byte(0x01)
msgTypeAddrs = byte(0x02)
)
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{}
var _ = wire.RegisterInterface(
struct{ PexMessage }{},
wire.ConcreteType{&pexRequestMessage{}, msgTypeRequest},
wire.ConcreteType{&pexAddrsMessage{}, msgTypeAddrs},
)
2017-01-16 11:57:07 -08:00
// DecodeMessage implements interface registered above.
2015-10-25 18:21:51 -07:00
func DecodeMessage(bz []byte) (msgType byte, msg PexMessage, err error) {
msgType = bz[0]
2015-11-10 12:29:43 -08:00
n := new(int)
2015-10-25 18:21:51 -07:00
r := bytes.NewReader(bz)
2015-11-10 12:29:43 -08:00
msg = wire.ReadBinary(struct{ PexMessage }{}, r, maxPexMessageSize, n, &err).(struct{ PexMessage }).PexMessage
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 {
Addrs []*NetAddress
}
func (m *pexAddrsMessage) String() string {
return fmt.Sprintf("[pexAddrs %v]", m.Addrs)
}