2018-06-04 21:10:59 +00:00
|
|
|
package p2p
|
|
|
|
|
|
|
|
import (
|
2018-07-17 18:39:04 +00:00
|
|
|
"context"
|
2018-12-30 21:20:43 +00:00
|
|
|
"errors"
|
2018-09-09 22:15:24 +00:00
|
|
|
"fmt"
|
2018-12-17 08:57:22 +00:00
|
|
|
"net"
|
2018-06-12 12:33:00 +00:00
|
|
|
"reflect"
|
2018-07-25 16:11:15 +00:00
|
|
|
"sync"
|
2018-06-12 12:33:00 +00:00
|
|
|
|
2018-12-23 20:34:59 +00:00
|
|
|
"github.com/gogo/protobuf/proto"
|
2018-11-07 01:52:36 +00:00
|
|
|
ds "github.com/ipfs/go-datastore"
|
|
|
|
dsync "github.com/ipfs/go-datastore/sync"
|
2018-07-17 18:39:04 +00:00
|
|
|
libp2p "github.com/libp2p/go-libp2p"
|
|
|
|
host "github.com/libp2p/go-libp2p-host"
|
2018-11-07 01:52:36 +00:00
|
|
|
kaddht "github.com/libp2p/go-libp2p-kad-dht"
|
2018-11-03 16:59:39 +00:00
|
|
|
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
2018-11-07 01:52:36 +00:00
|
|
|
rhost "github.com/libp2p/go-libp2p/p2p/host/routed"
|
2018-11-08 03:22:31 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/event"
|
2018-12-17 18:34:28 +00:00
|
|
|
"github.com/prysmaticlabs/prysm/shared/iputils"
|
2018-11-08 03:22:31 +00:00
|
|
|
"github.com/sirupsen/logrus"
|
2018-11-26 16:18:43 +00:00
|
|
|
"go.opencensus.io/trace"
|
2018-06-04 21:10:59 +00:00
|
|
|
)
|
|
|
|
|
2018-07-28 19:53:02 +00:00
|
|
|
// Sender represents a struct that is able to relay information via p2p.
|
2018-06-27 18:19:36 +00:00
|
|
|
// Server implements this interface.
|
|
|
|
type Sender interface {
|
|
|
|
Send(msg interface{}, peer Peer)
|
|
|
|
}
|
|
|
|
|
2018-06-13 11:44:45 +00:00
|
|
|
// Server is a placeholder for a p2p service. To be designed.
|
2018-06-12 12:33:00 +00:00
|
|
|
type Server struct {
|
2018-11-15 02:25:06 +00:00
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
mutex *sync.Mutex
|
|
|
|
feeds map[reflect.Type]Feed
|
|
|
|
host host.Host
|
2018-11-26 16:18:43 +00:00
|
|
|
dht *kaddht.IpfsDHT
|
2018-11-15 02:25:06 +00:00
|
|
|
gsub *pubsub.PubSub
|
|
|
|
topicMapping map[reflect.Type]string
|
|
|
|
bootstrapNode string
|
2018-11-25 16:55:02 +00:00
|
|
|
relayNodeAddr string
|
|
|
|
}
|
|
|
|
|
|
|
|
// ServerConfig for peer to peer networking.
|
|
|
|
type ServerConfig struct {
|
|
|
|
BootstrapNodeAddr string
|
|
|
|
RelayNodeAddr string
|
2018-11-26 02:54:02 +00:00
|
|
|
Port int
|
2018-06-12 12:33:00 +00:00
|
|
|
}
|
2018-06-04 21:10:59 +00:00
|
|
|
|
2018-06-13 02:06:59 +00:00
|
|
|
// NewServer creates a new p2p server instance.
|
2018-11-25 16:55:02 +00:00
|
|
|
func NewServer(cfg *ServerConfig) (*Server, error) {
|
2018-07-17 18:39:04 +00:00
|
|
|
ctx, cancel := context.WithCancel(context.Background())
|
2018-11-26 02:54:02 +00:00
|
|
|
opts := buildOptions(cfg.Port)
|
2018-11-25 16:55:02 +00:00
|
|
|
if cfg.RelayNodeAddr != "" {
|
|
|
|
opts = append(opts, libp2p.AddrsFactory(relayAddrsOnly(cfg.RelayNodeAddr)))
|
|
|
|
}
|
2018-12-17 08:57:22 +00:00
|
|
|
if !checkAvailablePort(cfg.Port) {
|
|
|
|
cancel()
|
|
|
|
return nil, fmt.Errorf("error listening on p2p, port %d already taken", cfg.Port)
|
|
|
|
}
|
2018-11-07 01:52:36 +00:00
|
|
|
h, err := libp2p.New(ctx, opts...)
|
2018-07-17 18:39:04 +00:00
|
|
|
if err != nil {
|
|
|
|
cancel()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-11-07 01:52:36 +00:00
|
|
|
dht := kaddht.NewDHT(ctx, h, dsync.MutexWrap(ds.NewMapDatastore()))
|
|
|
|
// Wrap host with a routed host so that peers can be looked up in the
|
|
|
|
// distributed hash table by their peer ID.
|
|
|
|
h = rhost.Wrap(h, dht)
|
|
|
|
|
|
|
|
gsub, err := pubsub.NewGossipSub(ctx, h)
|
2018-07-17 18:39:04 +00:00
|
|
|
if err != nil {
|
|
|
|
cancel()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-06-12 12:33:00 +00:00
|
|
|
return &Server{
|
2018-11-15 02:25:06 +00:00
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
|
|
|
feeds: make(map[reflect.Type]Feed),
|
|
|
|
host: h,
|
2018-11-26 16:18:43 +00:00
|
|
|
dht: dht,
|
2018-11-15 02:25:06 +00:00
|
|
|
gsub: gsub,
|
|
|
|
mutex: &sync.Mutex{},
|
|
|
|
topicMapping: make(map[reflect.Type]string),
|
2018-11-25 16:55:02 +00:00
|
|
|
bootstrapNode: cfg.BootstrapNodeAddr,
|
|
|
|
relayNodeAddr: cfg.RelayNodeAddr,
|
2018-06-12 12:33:00 +00:00
|
|
|
}, nil
|
2018-06-04 21:10:59 +00:00
|
|
|
}
|
|
|
|
|
2018-12-17 08:57:22 +00:00
|
|
|
func checkAvailablePort(port int) bool {
|
|
|
|
ip, err := iputils.ExternalIPv4()
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Could not get IPv4 address: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
ln, err := net.Listen("tcp", fmt.Sprintf("%s:%d", ip, port))
|
|
|
|
if err != nil {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2018-12-17 18:34:28 +00:00
|
|
|
if err := ln.Close(); err != nil {
|
|
|
|
log.Errorf("Could not close listener %v", err)
|
|
|
|
}
|
2018-12-17 08:57:22 +00:00
|
|
|
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
2018-06-13 02:06:59 +00:00
|
|
|
// Start the main routine for an p2p server.
|
2018-06-12 04:46:53 +00:00
|
|
|
func (s *Server) Start() {
|
2018-11-26 16:18:43 +00:00
|
|
|
ctx, span := trace.StartSpan(s.ctx, "p2p_server_start")
|
|
|
|
defer span.End()
|
2018-07-28 19:53:02 +00:00
|
|
|
log.Info("Starting service")
|
2018-11-15 02:25:06 +00:00
|
|
|
|
|
|
|
if s.bootstrapNode != "" {
|
2018-11-26 16:18:43 +00:00
|
|
|
if err := startDHTDiscovery(ctx, s.host, s.bootstrapNode); err != nil {
|
2018-11-15 02:25:06 +00:00
|
|
|
log.Errorf("Could not start peer discovery via DHT: %v", err)
|
|
|
|
}
|
2018-11-26 16:18:43 +00:00
|
|
|
if err := s.dht.Bootstrap(ctx); err != nil {
|
|
|
|
log.Errorf("Failed to bootstrap DHT: %v", err)
|
|
|
|
}
|
2018-11-15 02:25:06 +00:00
|
|
|
}
|
|
|
|
|
2018-11-25 16:55:02 +00:00
|
|
|
if s.relayNodeAddr != "" {
|
2018-11-26 16:18:43 +00:00
|
|
|
if err := dialRelayNode(ctx, s.host, s.relayNodeAddr); err != nil {
|
2018-11-25 16:55:02 +00:00
|
|
|
log.Errorf("Could not dial relay node: %v", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-11-26 16:18:43 +00:00
|
|
|
if err := startmDNSDiscovery(ctx, s.host); err != nil {
|
2018-11-15 02:25:06 +00:00
|
|
|
log.Errorf("Could not start peer discovery via mDNS: %v", err)
|
2018-07-17 18:39:04 +00:00
|
|
|
return
|
|
|
|
}
|
2018-11-25 16:55:02 +00:00
|
|
|
|
2018-11-26 16:18:43 +00:00
|
|
|
startPeerWatcher(ctx, s.host)
|
2018-06-04 21:10:59 +00:00
|
|
|
}
|
|
|
|
|
2018-06-13 02:06:59 +00:00
|
|
|
// Stop the main p2p loop.
|
2018-06-04 21:10:59 +00:00
|
|
|
func (s *Server) Stop() error {
|
2018-07-28 19:53:02 +00:00
|
|
|
log.Info("Stopping service")
|
2018-07-17 18:39:04 +00:00
|
|
|
|
|
|
|
s.cancel()
|
2018-06-04 21:10:59 +00:00
|
|
|
return nil
|
|
|
|
}
|
2018-06-13 02:13:24 +00:00
|
|
|
|
2018-12-30 21:20:43 +00:00
|
|
|
// Status returns an error if the p2p service does not have sufficient peers.
|
|
|
|
func (s *Server) Status() error {
|
|
|
|
if peerCount(s.host) < 5 {
|
|
|
|
return errors.New("less than 5 peers")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-08-29 16:32:54 +00:00
|
|
|
// RegisterTopic with a message and the adapter stack for the given topic. The
|
|
|
|
// message type provided will be feed selector for emitting messages received
|
|
|
|
// on a given topic.
|
|
|
|
//
|
|
|
|
// The topics can originate from multiple sources. In other words, messages on
|
|
|
|
// TopicA may come from direct peer communication or a pub/sub channel.
|
2018-09-09 22:15:24 +00:00
|
|
|
func (s *Server) RegisterTopic(topic string, message proto.Message, adapters ...Adapter) {
|
2018-08-29 16:32:54 +00:00
|
|
|
log.WithFields(logrus.Fields{
|
|
|
|
"topic": topic,
|
|
|
|
}).Debug("Subscribing to topic")
|
|
|
|
|
2018-09-09 22:15:24 +00:00
|
|
|
msgType := messageType(message)
|
2018-08-29 16:32:54 +00:00
|
|
|
s.topicMapping[msgType] = topic
|
|
|
|
|
|
|
|
sub, err := s.gsub.Subscribe(topic)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Failed to subscribe to topic: %v", err)
|
|
|
|
return
|
|
|
|
}
|
2018-09-09 22:15:24 +00:00
|
|
|
feed := s.Feed(message)
|
2018-08-29 16:32:54 +00:00
|
|
|
|
|
|
|
// Reverse adapter order
|
|
|
|
for i := len(adapters)/2 - 1; i >= 0; i-- {
|
|
|
|
opp := len(adapters) - 1 - i
|
|
|
|
adapters[i], adapters[opp] = adapters[opp], adapters[i]
|
|
|
|
}
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
defer sub.Cancel()
|
|
|
|
for {
|
|
|
|
msg, err := sub.Next(s.ctx)
|
|
|
|
|
|
|
|
if s.ctx.Err() != nil {
|
|
|
|
log.WithError(s.ctx.Err()).Debug("Context error")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Failed to get next message: %v", err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-11-15 12:54:45 +00:00
|
|
|
d := message
|
|
|
|
if err := proto.Unmarshal(msg.Data, d); err != nil {
|
|
|
|
log.WithError(err).Error("Failed to decode data")
|
|
|
|
continue
|
2018-08-29 16:32:54 +00:00
|
|
|
}
|
|
|
|
|
2018-11-15 12:54:45 +00:00
|
|
|
var h Handler = func(pMsg Message) {
|
|
|
|
s.emit(pMsg, feed)
|
|
|
|
}
|
2018-08-29 16:32:54 +00:00
|
|
|
|
2018-11-15 12:54:45 +00:00
|
|
|
pMsg := Message{Ctx: s.ctx, Data: d}
|
2018-08-29 16:32:54 +00:00
|
|
|
for _, adapter := range adapters {
|
|
|
|
h = adapter(h)
|
|
|
|
}
|
|
|
|
|
2018-09-20 11:46:35 +00:00
|
|
|
h(pMsg)
|
2018-08-29 16:32:54 +00:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
2018-11-15 12:54:45 +00:00
|
|
|
func (s *Server) emit(msg Message, feed Feed) {
|
|
|
|
i := feed.Send(msg)
|
2018-08-29 16:32:54 +00:00
|
|
|
log.WithFields(logrus.Fields{
|
|
|
|
"numSubs": i,
|
2018-11-15 12:54:45 +00:00
|
|
|
"msgType": fmt.Sprintf("%T", msg.Data),
|
|
|
|
"msgName": proto.MessageName(msg.Data),
|
2018-09-09 22:15:24 +00:00
|
|
|
}).Debug("Emit p2p message to feed subscribers")
|
2018-08-29 16:32:54 +00:00
|
|
|
}
|
|
|
|
|
2018-07-26 14:16:31 +00:00
|
|
|
// Subscribe returns a subscription to a feed of msg's Type and adds the channels to the feed.
|
2018-09-09 22:15:24 +00:00
|
|
|
func (s *Server) Subscribe(msg proto.Message, channel chan Message) event.Subscription {
|
2018-07-26 14:16:31 +00:00
|
|
|
return s.Feed(msg).Subscribe(channel)
|
|
|
|
}
|
|
|
|
|
2018-06-13 02:13:24 +00:00
|
|
|
// Send a message to a specific peer.
|
2018-09-09 22:15:24 +00:00
|
|
|
func (s *Server) Send(msg proto.Message, peer Peer) {
|
2018-09-16 18:02:53 +00:00
|
|
|
// TODO(#175)
|
2018-07-20 21:31:26 +00:00
|
|
|
// https://github.com/prysmaticlabs/prysm/issues/175
|
2018-07-17 18:39:04 +00:00
|
|
|
|
2018-09-16 18:02:53 +00:00
|
|
|
// TODO(#175): Remove debug log after send is implemented.
|
2018-07-17 18:39:04 +00:00
|
|
|
_ = peer
|
2018-07-21 17:51:18 +00:00
|
|
|
log.Debug("Broadcasting to everyone rather than sending a single peer")
|
2018-07-17 18:39:04 +00:00
|
|
|
s.Broadcast(msg)
|
2018-06-13 02:13:24 +00:00
|
|
|
}
|
|
|
|
|
2019-01-11 08:29:30 +00:00
|
|
|
// Broadcast publishes a message to all localized peers using gossipsub.
|
|
|
|
// msg must be a proto.Message that can be encoded into a byte array.
|
|
|
|
// It publishes the first 100 chars of msg over the msg's mapped topic.
|
2019-01-11 03:56:17 +00:00
|
|
|
// To map a messageType to a topic, use RegisterTopic.
|
|
|
|
//
|
2019-01-11 08:29:30 +00:00
|
|
|
// It logs an error if msg is not a protobuf message,
|
|
|
|
// if msg cannot be encoded into a byte array,
|
|
|
|
// or if the server is unable to publish the message over gossipsub.
|
2019-01-11 03:56:17 +00:00
|
|
|
//
|
|
|
|
// msg := make(chan p2p.Message, 100) // Choose a reasonable buffer size!
|
|
|
|
// ps.RegisterTopic("message_topic_here", msg)
|
|
|
|
// ps.Broadcast(msg)
|
2018-09-09 22:15:24 +00:00
|
|
|
func (s *Server) Broadcast(msg proto.Message) {
|
|
|
|
topic := s.topicMapping[messageType(msg)]
|
2018-11-21 18:00:36 +00:00
|
|
|
|
|
|
|
// Shorten message if it is too long to avoid
|
|
|
|
// polluting the logs.
|
|
|
|
if len(msg.String()) > 100 {
|
|
|
|
newMessage := msg.String()[:100]
|
|
|
|
|
|
|
|
log.WithFields(logrus.Fields{
|
|
|
|
"topic": topic,
|
|
|
|
}).Debugf("Broadcasting msg %+v --Message too long to be displayed", newMessage)
|
|
|
|
|
|
|
|
} else {
|
|
|
|
log.WithFields(logrus.Fields{
|
|
|
|
"topic": topic,
|
|
|
|
}).Debugf("Broadcasting msg %+v", msg)
|
|
|
|
}
|
2018-07-17 18:39:04 +00:00
|
|
|
|
2018-08-29 16:32:54 +00:00
|
|
|
if topic == "" {
|
2018-07-17 18:39:04 +00:00
|
|
|
log.Warnf("Topic is unknown for message type %T. %v", msg, msg)
|
|
|
|
}
|
|
|
|
|
|
|
|
m, ok := msg.(proto.Message)
|
|
|
|
if !ok {
|
|
|
|
log.Errorf("Message to broadcast (type: %T) is not a protobuf message: %v", msg, msg)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
b, err := proto.Marshal(m)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Failed to marshal data for broadcast: %v", err)
|
|
|
|
return
|
|
|
|
}
|
2018-08-29 16:32:54 +00:00
|
|
|
if err := s.gsub.Publish(topic, b); err != nil {
|
2018-07-29 00:44:24 +00:00
|
|
|
log.Errorf("Failed to publish to gossipsub topic: %v", err)
|
|
|
|
}
|
2018-07-17 18:39:04 +00:00
|
|
|
}
|