tendermint/pex_reactor.go

353 lines
10 KiB
Go
Raw Normal View History

2015-10-25 18:21:51 -07:00
package p2p
import (
"bytes"
"fmt"
"math/rand"
"reflect"
"time"
2017-04-20 01:49:54 -07:00
cmn "github.com/tendermint/go-common"
2017-04-07 03:57:03 -07:00
wire "github.com/tendermint/go-wire"
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
2017-01-12 05:56:40 -08:00
// maximum messages one peer can send to us during `msgCountByPeerFlushInterval`
defaultMaxMsgCountByPeer = 1000
msgCountByPeerFlushInterval = 1 * time.Hour
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
//
// For now, it just limits the number of messages from one peer to
// `defaultMaxMsgCountByPeer` messages per `msgCountByPeerFlushInterval` (1000
// msg/hour).
//
// NOTE [2017-01-17]:
// Limiting is fine for now. Maybe down the road we want to keep track of the
// quality of peer messages so if peerA keeps telling us about peers we can't
// connect to then maybe we should care less about peerA. But I don't think
// that kind of complexity is priority right now.
2015-10-25 18:21:51 -07:00
type PEXReactor struct {
BaseReactor
2017-01-11 12:17:15 -08:00
sw *Switch
book *AddrBook
ensurePeersPeriod time.Duration
2017-01-12 05:56:40 -08:00
// tracks message count by peer, so we can prevent abuse
msgCountByPeer map[string]uint16
maxMsgCountByPeer uint16
2015-10-25 18:21:51 -07:00
}
2017-01-16 11:57:07 -08:00
// NewPEXReactor creates new PEX reactor.
2017-01-11 03:03:29 -08:00
func NewPEXReactor(b *AddrBook) *PEXReactor {
r := &PEXReactor{
2017-01-11 12:17:15 -08:00
book: b,
ensurePeersPeriod: defaultEnsurePeersPeriod,
2017-01-12 05:56:40 -08:00
msgCountByPeer: make(map[string]uint16),
maxMsgCountByPeer: defaultMaxMsgCountByPeer,
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
r.BaseReactor = *NewBaseReactor(log, "PEXReactor", r)
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 {
r.BaseReactor.OnStart()
r.book.Start()
go r.ensurePeersRoutine()
2017-01-12 05:56:40 -08:00
go r.flushMsgCountByPeer()
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{
&ChannelDescriptor{
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).
func (r *PEXReactor) AddPeer(p *Peer) {
if p.IsOutbound() { // For outbound peers, the address is already in the books
if r.book.NeedMoreAddrs() {
r.RequestPEX(p)
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
} else { // For inbound connections, the peer is its own source
2017-04-14 12:59:22 -07:00
addr, err := NewNetAddressString(p.ListenAddr)
if err != nil {
// this should never happen
log.Error("Error in AddPeer: invalid peer address", "addr", p.ListenAddr, "error", err)
return
}
2017-01-11 07:23:31 -08:00
r.book.AddAddress(addr, addr)
2015-10-25 18:21:51 -07:00
}
}
2017-01-16 11:57:07 -08:00
// RemovePeer implements Reactor by removing peer from the address book.
//
// The peer will be proposed to us by other peers (PexAddrsMessage) or himself
// and we will add him again upon successful connection. Note that other peers
// will remove him too. The peer will need to send first requests to others by
// himself (he will have an addrbook or the seeds).
2017-01-11 03:03:29 -08:00
func (r *PEXReactor) RemovePeer(p *Peer, reason interface{}) {
2017-04-14 12:59:22 -07:00
addr, err := NewNetAddressString(p.ListenAddr)
if err != nil {
// this should never happen
log.Error("Error in AddPeer: invalid peer address", "addr", p.ListenAddr, "error", err)
return
}
r.book.RemoveAddress(addr)
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
// Receive implements Reactor by handling incoming PEX messages.
func (r *PEXReactor) Receive(chID byte, src *Peer, msgBytes []byte) {
2017-01-12 05:56:40 -08:00
srcAddr := src.Connection().RemoteAddress
srcAddrStr := srcAddr.String()
r.msgCountByPeer[srcAddrStr]++
if r.ReachedMaxMsgCountForPeer(srcAddrStr) {
log.Warn("Maximum number of messages reached for peer", "peer", srcAddrStr)
// TODO remove src from peers?
return
}
2015-10-25 18:21:51 -07:00
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "error", err)
return
}
log.Notice("Received message", "msg", msg)
switch msg := msg.(type) {
case *pexRequestMessage:
// src requested some peers.
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.
// (We don't want to get spammed with bad peers)
for _, addr := range msg.Addrs {
prevent nil addr Error: ``` Error: runtime error: invalid memoryaddress or nil pointer dereference\nStack: goroutine 549 [running]:\nruntime/debug.Stack(0x0, 0x0, 0x0)\n\t/usr/local/go/src/runtime/debug/stack.go:24 +0x80\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection)._recover(0xc821723b00)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:173 +0x53\npanic(0xbe1500, 0xc820012080)\n\t/usr/local/go/src/runtime/panic.go:443 +0x4e9\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*NetAddress).Valid(0x0, 0x0)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/netaddress.go:125 +0x1c\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*NetAddress).Routable(0x0, 0xc8217bb740)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/netaddress.go:117 +0x25\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*AddrBook).addAddress(0xc820108380, 0x0, 0xc821739590)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/addrbook.go:524 +0x45\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*AddrBook).AddAddress(0xc820108380, 0x0, 0xc821739590)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/addrbook.go:160 +0x286\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*PEXReactor).Receive(0xc82000be60, 0xc820149f00, 0xc8218163f0, 0xc82184e000, 0x5b, 0x1000)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/pex_reactor.go:109 +0x457\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.newPeer.func1(0xc82011d500, 0xc82184e000, 0x5b, 0x1000)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/peer.go:58 +0x202\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection).recvRoutine(0xc821723b00)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:439 +0x1177\ncreated by github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection).OnStart\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:138 +0x1a1\n ```
2017-01-16 00:39:57 -08:00
if addr != nil {
2017-01-11 03:03:29 -08:00
r.book.AddAddress(addr, srcAddr)
prevent nil addr Error: ``` Error: runtime error: invalid memoryaddress or nil pointer dereference\nStack: goroutine 549 [running]:\nruntime/debug.Stack(0x0, 0x0, 0x0)\n\t/usr/local/go/src/runtime/debug/stack.go:24 +0x80\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection)._recover(0xc821723b00)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:173 +0x53\npanic(0xbe1500, 0xc820012080)\n\t/usr/local/go/src/runtime/panic.go:443 +0x4e9\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*NetAddress).Valid(0x0, 0x0)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/netaddress.go:125 +0x1c\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*NetAddress).Routable(0x0, 0xc8217bb740)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/netaddress.go:117 +0x25\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*AddrBook).addAddress(0xc820108380, 0x0, 0xc821739590)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/addrbook.go:524 +0x45\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*AddrBook).AddAddress(0xc820108380, 0x0, 0xc821739590)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/addrbook.go:160 +0x286\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*PEXReactor).Receive(0xc82000be60, 0xc820149f00, 0xc8218163f0, 0xc82184e000, 0x5b, 0x1000)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/pex_reactor.go:109 +0x457\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.newPeer.func1(0xc82011d500, 0xc82184e000, 0x5b, 0x1000)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/peer.go:58 +0x202\ngithub.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection).recvRoutine(0xc821723b00)\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:439 +0x1177\ncreated by github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p.(*MConnection).OnStart\n\t/go/src/github.com/tendermint/tendermint/vendor/github.com/tendermint/go-p2p/connection.go:138 +0x1a1\n ```
2017-01-16 00:39:57 -08:00
}
2015-10-25 18:21:51 -07:00
}
default:
2017-01-16 11:57:07 -08:00
log.Warn(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
2015-10-25 18:21:51 -07:00
}
}
2017-01-11 03:03:29 -08:00
// RequestPEX asks peer for more addresses.
func (r *PEXReactor) RequestPEX(p *Peer) {
p.Send(PexChannel, struct{ PexMessage }{&pexRequestMessage{}})
2015-10-25 18:21:51 -07:00
}
2017-01-11 03:03:29 -08:00
// SendAddrs sends addrs to the peer.
func (r *PEXReactor) SendAddrs(p *Peer, addrs []*NetAddress) {
p.Send(PexChannel, struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}})
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
}
2017-01-12 05:56:40 -08:00
// SetMaxMsgCountByPeer sets maximum messages one peer can send to us during 'msgCountByPeerFlushInterval'.
func (r *PEXReactor) SetMaxMsgCountByPeer(v uint16) {
r.maxMsgCountByPeer = v
}
// ReachedMaxMsgCountForPeer returns true if we received too many
// messages from peer with address `addr`.
func (r *PEXReactor) ReachedMaxMsgCountForPeer(addr string) bool {
return r.msgCountByPeer[addr] >= r.maxMsgCountByPeer
}
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
// fire once immediately.
2017-01-11 03:03:29 -08:00
r.ensurePeers()
2015-10-25 18:21:51 -07:00
// fire periodically
ticker := time.NewTicker(r.ensurePeersPeriod)
2017-01-12 05:56:40 -08:00
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, _, numDialing := r.Switch.NumPeers()
2015-10-25 18:21:51 -07:00
numToDial := minNumOutboundPeers - (numOutPeers + numDialing)
log.Info("Ensure peers", "numOutPeers", numOutPeers, "numDialing", numDialing, "numToDial", numToDial)
if numToDial <= 0 {
return
}
toDial := make(map[string]*NetAddress)
2015-10-25 18:21:51 -07:00
// Try to pick numToDial addresses to dial.
for i := 0; i < numToDial; i++ {
2017-04-20 01:49:54 -07:00
// The purpose of newBias is to first prioritize old (more vetted) peers
// when we have few connections, but to allow for new (less vetted) peers
// if we already have many connections. This algorithm isn't perfect, but
// it somewhat ensures that we prioritize connecting to more-vetted
// peers.
newBias := cmn.MinInt(numOutPeers, 8)*10 + 10
2015-10-25 18:21:51 -07:00
var picked *NetAddress
// Try to fetch a new peer 3 times.
// This caps the maximum number of tries to 3 * numToDial.
for j := 0; j < 3; j++ {
2017-04-20 01:49:54 -07:00
try := r.book.PickAddress(newBias)
2015-10-25 18:21:51 -07:00
if try == nil {
break
}
_, alreadySelected := toDial[try.IP.String()]
2017-01-11 03:03:29 -08:00
alreadyDialing := r.Switch.IsDialing(try)
2017-04-20 01:49:54 -07:00
alreadyConnected := r.Switch.Peers().Has(try.IP.String())
if alreadySelected || alreadyDialing || alreadyConnected {
// log.Info("Cannot dial address", "addr", try,
// "alreadySelected", alreadySelected,
2017-04-20 01:49:54 -07:00
// "alreadyDialing", alreadyDialing,
// "alreadyConnected", alreadyConnected)
2015-10-25 18:21:51 -07:00
continue
} else {
log.Info("Will dial address", "addr", try)
picked = try
break
}
}
if picked == nil {
continue
}
toDial[picked.IP.String()] = picked
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() {
if peers := r.Switch.Peers().List(); len(peers) > 0 {
2015-10-25 18:21:51 -07:00
i := rand.Int() % len(peers)
peer := peers[i]
log.Info("No addresses to dial. 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
}
}
}
2017-01-12 05:56:40 -08:00
func (r *PEXReactor) flushMsgCountByPeer() {
ticker := time.NewTicker(msgCountByPeerFlushInterval)
for {
select {
case <-ticker.C:
r.msgCountByPeer = make(map[string]uint16)
case <-r.Quit:
ticker.Stop()
return
}
}
}
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)
}