2018-08-28 03:01:08 +00:00
|
|
|
package pubsub
|
2018-02-16 20:01:15 +00:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2018-10-26 12:49:34 +00:00
|
|
|
"fmt"
|
2018-02-20 12:00:33 +00:00
|
|
|
"math/rand"
|
2020-03-02 13:55:25 +00:00
|
|
|
"sort"
|
2018-02-19 12:50:14 +00:00
|
|
|
"time"
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2018-08-28 03:01:08 +00:00
|
|
|
pb "github.com/libp2p/go-libp2p-pubsub/pb"
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2019-05-26 16:19:03 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/host"
|
2019-11-22 19:53:00 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/network"
|
2019-05-26 16:19:03 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/peer"
|
2019-11-22 19:53:00 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/peerstore"
|
2019-05-26 16:19:03 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/protocol"
|
2020-01-23 18:27:10 +00:00
|
|
|
"github.com/libp2p/go-libp2p-core/record"
|
2018-02-16 20:01:15 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
2019-11-21 22:09:37 +00:00
|
|
|
GossipSubID_v10 = protocol.ID("/meshsub/1.0.0")
|
2019-11-21 21:13:20 +00:00
|
|
|
GossipSubID_v11 = protocol.ID("/meshsub/1.1.0")
|
2018-03-10 08:23:55 +00:00
|
|
|
)
|
2018-02-19 14:13:18 +00:00
|
|
|
|
2018-03-10 08:23:55 +00:00
|
|
|
var (
|
2018-02-19 14:13:18 +00:00
|
|
|
// overlay parameters
|
2020-03-02 13:55:25 +00:00
|
|
|
GossipSubD = 6
|
|
|
|
GossipSubDlo = 5
|
|
|
|
GossipSubDhi = 12
|
2020-03-03 15:49:37 +00:00
|
|
|
GossipSubDscore = 4
|
2018-02-20 14:10:53 +00:00
|
|
|
|
|
|
|
// gossip parameters
|
|
|
|
GossipSubHistoryLength = 5
|
|
|
|
GossipSubHistoryGossip = 3
|
2018-03-10 08:23:55 +00:00
|
|
|
|
2020-03-05 12:52:40 +00:00
|
|
|
GossipSubDlazy = 6
|
|
|
|
GossipSubGossipFactor = 0.25
|
|
|
|
|
2020-03-05 22:49:40 +00:00
|
|
|
GossipSubGossipRetransmission = 3
|
|
|
|
|
2018-03-10 08:23:55 +00:00
|
|
|
// heartbeat interval
|
2018-08-29 08:15:41 +00:00
|
|
|
GossipSubHeartbeatInitialDelay = 100 * time.Millisecond
|
|
|
|
GossipSubHeartbeatInterval = 1 * time.Second
|
2018-03-10 09:21:50 +00:00
|
|
|
|
|
|
|
// fanout ttl
|
|
|
|
GossipSubFanoutTTL = 60 * time.Second
|
2019-11-21 22:09:37 +00:00
|
|
|
|
|
|
|
// number of peers to include in prune Peer eXchange
|
2020-03-27 09:52:36 +00:00
|
|
|
GossipSubPrunePeers = 16
|
2019-11-22 18:46:13 +00:00
|
|
|
|
|
|
|
// backoff time for pruned peers
|
|
|
|
GossipSubPruneBackoff = time.Minute
|
2019-11-22 19:53:00 +00:00
|
|
|
|
|
|
|
// number of active connection attempts for peers obtained through px
|
2020-03-04 12:53:28 +00:00
|
|
|
GossipSubConnectors = 8
|
2019-11-25 11:58:29 +00:00
|
|
|
|
2019-12-06 15:22:49 +00:00
|
|
|
// maximum number of pending connections for peers attempted through px
|
2020-03-04 12:53:28 +00:00
|
|
|
GossipSubMaxPendingConnections = 128
|
2019-12-06 15:22:49 +00:00
|
|
|
|
2019-11-25 11:58:29 +00:00
|
|
|
// timeout for connection attempts
|
|
|
|
GossipSubConnectionTimeout = 30 * time.Second
|
2020-04-10 10:54:14 +00:00
|
|
|
|
|
|
|
// Number of heartbeat ticks for attempting to reconnect direct peers that are not
|
|
|
|
// currently connected
|
|
|
|
GossipSubDirectConnectTicks uint64 = 300
|
2020-04-10 11:21:10 +00:00
|
|
|
|
2020-04-22 14:48:34 +00:00
|
|
|
// Initial delay before opening connections to direct peers
|
|
|
|
GossipSubDirectConnectInitialDelay = time.Second
|
|
|
|
|
2020-04-10 11:21:10 +00:00
|
|
|
// Number of heartbeat ticks for attempting to improve the mesh with opportunistic
|
|
|
|
// grafting
|
|
|
|
GossipSubOpportunisticGraftTicks uint64 = 60
|
2020-04-10 19:37:07 +00:00
|
|
|
|
|
|
|
// Number of peers to opportunistically graft
|
|
|
|
GossipSubOpportunisticGraftPeers = 2
|
2020-04-20 07:26:32 +00:00
|
|
|
|
|
|
|
// If a GRAFT comes before GossipSubGraftFloodThreshold has ellapsed since the last PRUNE,
|
|
|
|
// then there is no PRUNE response emitted. This protects against GRAFT floods and should be
|
|
|
|
// less than GossipSubPruneBackoff.
|
|
|
|
GossipSubGraftFloodThreshold = 10 * time.Second
|
2020-04-20 09:15:59 +00:00
|
|
|
|
2020-04-20 13:03:44 +00:00
|
|
|
// backoff penalty for GRAFT floods
|
|
|
|
GossipSubPruneBackoffPenalty = time.Hour
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
// Maximum number of messages to include in an IHAVE message. Also controls the maximum
|
|
|
|
// number of IHAVE ids we will accept and request with IWANT from a peer within a heartbeat,
|
|
|
|
// to protect from IHAVE floods. You should adjust this value from the default if your
|
|
|
|
// system is pushing more than 5000 messages in GossipSubHistoryGossip heartbeats; with the
|
|
|
|
// defaults this is 1666 messages/s.
|
|
|
|
GossipSubMaxIHaveLength = 5000
|
2020-04-20 12:13:15 +00:00
|
|
|
|
|
|
|
// Maximum number of IHAVE messages to accept from a peer within a heartbeat.
|
|
|
|
GossipSubMaxIHaveMessages = 10
|
2018-02-16 20:01:15 +00:00
|
|
|
)
|
|
|
|
|
2019-01-04 11:09:21 +00:00
|
|
|
// NewGossipSub returns a new PubSub object using GossipSubRouter as the router.
|
2018-02-16 20:01:15 +00:00
|
|
|
func NewGossipSub(ctx context.Context, h host.Host, opts ...Option) (*PubSub, error) {
|
2018-02-19 14:13:18 +00:00
|
|
|
rt := &GossipSubRouter{
|
2020-04-20 12:02:58 +00:00
|
|
|
peers: make(map[peer.ID]protocol.ID),
|
|
|
|
mesh: make(map[string]map[peer.ID]struct{}),
|
|
|
|
fanout: make(map[string]map[peer.ID]struct{}),
|
|
|
|
lastpub: make(map[string]int64),
|
|
|
|
gossip: make(map[peer.ID][]*pb.ControlIHave),
|
|
|
|
control: make(map[peer.ID]*pb.ControlMessage),
|
|
|
|
backoff: make(map[string]map[peer.ID]time.Time),
|
|
|
|
peerhave: make(map[peer.ID]int),
|
|
|
|
iasked: make(map[peer.ID]int),
|
|
|
|
connect: make(chan connectInfo, GossipSubMaxPendingConnections),
|
|
|
|
mcache: NewMessageCache(GossipSubHistoryGossip, GossipSubHistoryLength),
|
2020-04-23 09:53:09 +00:00
|
|
|
|
2020-04-22 17:46:35 +00:00
|
|
|
// these must be pulled in to resolve races in tests... sigh.
|
2020-04-23 09:53:09 +00:00
|
|
|
D: GossipSubD,
|
|
|
|
Dlo: GossipSubDlo,
|
|
|
|
Dhi: GossipSubDhi,
|
|
|
|
Dscore: GossipSubDscore,
|
|
|
|
Dlazy: GossipSubDlazy,
|
|
|
|
|
2020-04-22 17:46:35 +00:00
|
|
|
directConnectTicks: GossipSubDirectConnectTicks,
|
|
|
|
opportunisticGraftTicks: GossipSubOpportunisticGraftTicks,
|
2018-02-19 14:13:18 +00:00
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
return NewPubSub(ctx, h, rt, opts...)
|
|
|
|
}
|
|
|
|
|
2020-03-01 12:29:24 +00:00
|
|
|
// WithPeerScore is a gossipsub router option that enables peer scoring.
|
2020-03-27 09:38:48 +00:00
|
|
|
func WithPeerScore(params *PeerScoreParams, thresholds *PeerScoreThresholds) Option {
|
2020-03-01 12:29:24 +00:00
|
|
|
return func(ps *PubSub) error {
|
|
|
|
gs, ok := ps.rt.(*GossipSubRouter)
|
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("pubsub router is not gossipsub")
|
|
|
|
}
|
|
|
|
|
2020-03-16 19:32:32 +00:00
|
|
|
// sanity check: validate the score parameters
|
2020-03-09 12:08:42 +00:00
|
|
|
err := params.validate()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2020-03-16 19:32:32 +00:00
|
|
|
// sanity check: validate the threshold values
|
2020-03-27 09:38:48 +00:00
|
|
|
err = thresholds.validate()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2020-03-16 19:32:32 +00:00
|
|
|
}
|
|
|
|
|
2020-03-07 15:59:22 +00:00
|
|
|
gs.score = newPeerScore(params)
|
2020-03-27 09:38:48 +00:00
|
|
|
gs.gossipThreshold = thresholds.GossipThreshold
|
|
|
|
gs.publishThreshold = thresholds.PublishThreshold
|
|
|
|
gs.graylistThreshold = thresholds.GraylistThreshold
|
|
|
|
gs.acceptPXThreshold = thresholds.AcceptPXThreshold
|
2020-04-10 11:21:10 +00:00
|
|
|
gs.opportunisticGraftThreshold = thresholds.OpportunisticGraftThreshold
|
2020-03-01 12:29:24 +00:00
|
|
|
|
|
|
|
// hook the tracer
|
|
|
|
if ps.tracer != nil {
|
|
|
|
ps.tracer.score = gs.score
|
|
|
|
} else {
|
|
|
|
ps.tracer = &pubsubTracer{score: gs.score, pid: ps.host.ID(), msgID: ps.msgID}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-03 13:31:16 +00:00
|
|
|
// WithFloodPublish is a gossipsub router option that enables flood publishing.
|
|
|
|
// When this is enabled, published messages are forwarded to all peers with score >=
|
|
|
|
// to publishThreshold
|
|
|
|
func WithFloodPublish(floodPublish bool) Option {
|
|
|
|
return func(ps *PubSub) error {
|
|
|
|
gs, ok := ps.rt.(*GossipSubRouter)
|
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("pubsub router is not gossipsub")
|
|
|
|
}
|
|
|
|
|
|
|
|
gs.floodPublish = floodPublish
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-27 09:24:51 +00:00
|
|
|
// WithPeerExchange is a gossipsub router option that enables Peer eXchange on PRUNE.
|
|
|
|
// This should generally be enabled in bootstrappers and well connected/trusted nodes
|
|
|
|
// used for bootstrapping.
|
|
|
|
func WithPeerExchange(doPX bool) Option {
|
|
|
|
return func(ps *PubSub) error {
|
|
|
|
gs, ok := ps.rt.(*GossipSubRouter)
|
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("pubsub router is not gossipsub")
|
|
|
|
}
|
|
|
|
|
|
|
|
gs.doPX = doPX
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-31 10:23:04 +00:00
|
|
|
// WithDirectPeers is a gossipsub router option that specifies peers with direct
|
|
|
|
// peering agreements. These peers are connected outside of the mesh, with all (valid)
|
|
|
|
// message unconditionally forwarded to them. The router will maintain open connections
|
|
|
|
// to these peers. Note that the peering agreement should be reciprocal with direct peers
|
|
|
|
// symmetrically configured at both ends.
|
|
|
|
func WithDirectPeers(pis []peer.AddrInfo) Option {
|
|
|
|
return func(ps *PubSub) error {
|
|
|
|
gs, ok := ps.rt.(*GossipSubRouter)
|
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("pubsub router is not gossipsub")
|
|
|
|
}
|
|
|
|
|
|
|
|
direct := make(map[peer.ID]struct{})
|
|
|
|
for _, pi := range pis {
|
|
|
|
direct[pi.ID] = struct{}{}
|
|
|
|
ps.host.Peerstore().AddAddrs(pi.ID, pi.Addrs, peerstore.PermanentAddrTTL)
|
|
|
|
}
|
|
|
|
|
|
|
|
gs.direct = direct
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-03-10 09:21:50 +00:00
|
|
|
// GossipSubRouter is a router that implements the gossipsub protocol.
|
|
|
|
// For each topic we have joined, we maintain an overlay through which
|
|
|
|
// messages flow; this is the mesh map.
|
|
|
|
// For each topic we publish to without joining, we maintain a list of peers
|
|
|
|
// to use for injecting our messages in the overlay with stable routes; this
|
|
|
|
// is the fanout map. Fanout peer lists are expired if we don't publish any
|
|
|
|
// messages to their topic for GossipSubFanoutTTL.
|
2018-02-16 20:01:15 +00:00
|
|
|
type GossipSubRouter struct {
|
2020-04-20 12:02:58 +00:00
|
|
|
p *PubSub
|
|
|
|
peers map[peer.ID]protocol.ID // peer protocols
|
|
|
|
direct map[peer.ID]struct{} // direct peers
|
|
|
|
mesh map[string]map[peer.ID]struct{} // topic meshes
|
|
|
|
fanout map[string]map[peer.ID]struct{} // topic fanout
|
|
|
|
lastpub map[string]int64 // last publish time for fanout topics
|
|
|
|
gossip map[peer.ID][]*pb.ControlIHave // pending gossip
|
|
|
|
control map[peer.ID]*pb.ControlMessage // pending control messages
|
|
|
|
peerhave map[peer.ID]int // number of IHAVEs received from peer in the last heartbeat
|
|
|
|
iasked map[peer.ID]int // number of messages we have asked from peer in the last heartbeat
|
|
|
|
backoff map[string]map[peer.ID]time.Time // prune backoff
|
|
|
|
connect chan connectInfo // px connection requests
|
|
|
|
mcache *MessageCache
|
|
|
|
tracer *pubsubTracer
|
|
|
|
score *peerScore
|
2020-03-02 13:05:24 +00:00
|
|
|
|
2020-03-27 09:24:51 +00:00
|
|
|
// whether PX is enabled; this should be enabled in bootstrappers and other well connected/trusted
|
|
|
|
// nodes.
|
|
|
|
doPX bool
|
|
|
|
|
|
|
|
// threshold for accepting PX from a peer; this should be positive and limited to scores
|
|
|
|
// attainable by bootstrappers and trusted nodes
|
|
|
|
acceptPXThreshold float64
|
|
|
|
|
2020-03-02 13:05:24 +00:00
|
|
|
// threshold for peer score to emit/accept gossip
|
|
|
|
// If the peer score is below this threshold, we won't emit or accept gossip from the peer.
|
|
|
|
// When there is no score, this value is 0.
|
|
|
|
gossipThreshold float64
|
2020-03-03 13:31:16 +00:00
|
|
|
|
|
|
|
// flood publish score threshold; we only publish to peers with score >= to the threshold
|
|
|
|
// when using flood publishing or the peer is a fanout or floodsub peer.
|
|
|
|
publishThreshold float64
|
|
|
|
|
2020-03-04 12:51:10 +00:00
|
|
|
// threshold for peer score before we graylist the peer and silently ignore its RPCs
|
|
|
|
graylistThreshold float64
|
|
|
|
|
2020-04-10 17:23:31 +00:00
|
|
|
// threshold for median peer score before triggering opportunistic grafting
|
2020-04-10 11:21:10 +00:00
|
|
|
opportunisticGraftThreshold float64
|
|
|
|
|
2020-03-03 13:31:16 +00:00
|
|
|
// whether to use flood publishing
|
|
|
|
floodPublish bool
|
2020-03-23 07:49:11 +00:00
|
|
|
|
|
|
|
// number of heartbeats since the beginning of time; this allows us to amortize some resource
|
|
|
|
// clean up -- eg backoff clean up.
|
|
|
|
heartbeatTicks uint64
|
2020-04-22 17:46:35 +00:00
|
|
|
|
2020-04-23 09:53:09 +00:00
|
|
|
// overly parameter "constants"
|
|
|
|
// these are pulled from their global value or else the race detector is angry on travis
|
|
|
|
// it also allows us to change them per peer in tests, which is a plus
|
|
|
|
D, Dlo, Dhi, Dscore, Dlazy int
|
|
|
|
|
2020-04-22 17:46:35 +00:00
|
|
|
// tick "constants" for triggering direct connect and opportunistic grafting
|
|
|
|
// these are pulled from their global value or else the race detector is angry on travis
|
|
|
|
directConnectTicks, opportunisticGraftTicks uint64
|
2018-02-16 20:01:15 +00:00
|
|
|
}
|
|
|
|
|
2019-11-22 19:53:00 +00:00
|
|
|
type connectInfo struct {
|
|
|
|
p peer.ID
|
2020-01-22 15:23:47 +00:00
|
|
|
spr *record.Envelope
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) Protocols() []protocol.ID {
|
2019-11-21 22:09:37 +00:00
|
|
|
return []protocol.ID{GossipSubID_v11, GossipSubID_v10, FloodSubID}
|
2018-02-16 20:01:15 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) Attach(p *PubSub) {
|
|
|
|
gs.p = p
|
2019-11-04 18:47:19 +00:00
|
|
|
gs.tracer = p.tracer
|
2020-03-07 15:59:22 +00:00
|
|
|
|
2020-03-16 19:27:52 +00:00
|
|
|
// start the scoring
|
|
|
|
gs.score.Start(gs)
|
2020-03-07 15:59:22 +00:00
|
|
|
|
2019-12-16 02:38:28 +00:00
|
|
|
// start using the same msg ID function as PubSub for caching messages.
|
2019-12-16 11:46:30 +00:00
|
|
|
gs.mcache.SetMsgIdFn(p.msgID)
|
2020-03-07 15:59:22 +00:00
|
|
|
|
|
|
|
// start the heartbeat
|
2018-02-19 14:13:18 +00:00
|
|
|
go gs.heartbeatTimer()
|
2020-03-07 15:59:22 +00:00
|
|
|
|
|
|
|
// start the PX connectors
|
2019-11-22 19:53:00 +00:00
|
|
|
for i := 0; i < GossipSubConnectors; i++ {
|
|
|
|
go gs.connector()
|
|
|
|
}
|
2020-03-31 10:23:04 +00:00
|
|
|
|
|
|
|
// connect to direct peers
|
|
|
|
if len(gs.direct) > 0 {
|
|
|
|
go func() {
|
2020-04-22 14:48:34 +00:00
|
|
|
if GossipSubDirectConnectInitialDelay > 0 {
|
|
|
|
time.Sleep(GossipSubDirectConnectInitialDelay)
|
|
|
|
}
|
2020-03-31 10:23:04 +00:00
|
|
|
for p := range gs.direct {
|
|
|
|
gs.connect <- connectInfo{p: p}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) AddPeer(p peer.ID, proto protocol.ID) {
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("PEERUP: Add new peer %s using %s", p, proto)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.AddPeer(p, proto)
|
2018-02-19 14:13:18 +00:00
|
|
|
gs.peers[p] = proto
|
2020-03-31 10:23:04 +00:00
|
|
|
|
|
|
|
// tag peer if it is a direct peer
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
if direct {
|
|
|
|
gs.p.host.ConnManager().TagPeer(p, "pubsub:direct", 1000)
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) RemovePeer(p peer.ID) {
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("PEERDOWN: Remove disconnected peer %s", p)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.RemovePeer(p)
|
2018-02-19 14:13:18 +00:00
|
|
|
delete(gs.peers, p)
|
|
|
|
for _, peers := range gs.mesh {
|
|
|
|
delete(peers, p)
|
|
|
|
}
|
|
|
|
for _, peers := range gs.fanout {
|
|
|
|
delete(peers, p)
|
|
|
|
}
|
2018-02-20 16:23:28 +00:00
|
|
|
delete(gs.gossip, p)
|
|
|
|
delete(gs.control, p)
|
2018-02-19 14:13:18 +00:00
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2019-10-31 19:56:09 +00:00
|
|
|
func (gs *GossipSubRouter) EnoughPeers(topic string, suggested int) bool {
|
|
|
|
// check all peers in the topic
|
|
|
|
tmap, ok := gs.p.topics[topic]
|
|
|
|
if !ok {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
fsPeers, gsPeers := 0, 0
|
|
|
|
// floodsub peers
|
|
|
|
for p := range tmap {
|
|
|
|
if gs.peers[p] == FloodSubID {
|
|
|
|
fsPeers++
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// gossipsub peers
|
|
|
|
gsPeers = len(gs.mesh[topic])
|
|
|
|
|
|
|
|
if suggested == 0 {
|
2020-04-23 09:53:09 +00:00
|
|
|
suggested = gs.Dlo
|
2019-10-31 19:56:09 +00:00
|
|
|
}
|
|
|
|
|
2020-04-23 09:53:09 +00:00
|
|
|
if fsPeers+gsPeers >= suggested || gsPeers >= gs.Dhi {
|
2019-10-31 19:56:09 +00:00
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2020-03-04 12:51:10 +00:00
|
|
|
func (gs *GossipSubRouter) AcceptFrom(p peer.ID) bool {
|
2020-03-31 10:23:04 +00:00
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return direct || gs.score.Score(p) >= gs.graylistThreshold
|
2020-03-04 12:51:10 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) HandleRPC(rpc *RPC) {
|
2018-02-19 16:45:10 +00:00
|
|
|
ctl := rpc.GetControl()
|
|
|
|
if ctl == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
iwant := gs.handleIHave(rpc.from, ctl)
|
|
|
|
ihave := gs.handleIWant(rpc.from, ctl)
|
2018-02-19 16:45:10 +00:00
|
|
|
prune := gs.handleGraft(rpc.from, ctl)
|
|
|
|
gs.handlePrune(rpc.from, ctl)
|
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
if len(iwant) == 0 && len(ihave) == 0 && len(prune) == 0 {
|
2018-02-19 16:45:10 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
out := rpcWithControl(ihave, nil, iwant, nil, prune)
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(rpc.from, out)
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
func (gs *GossipSubRouter) handleIHave(p peer.ID, ctl *pb.ControlMessage) []*pb.ControlIWant {
|
2020-03-02 13:05:24 +00:00
|
|
|
// we ignore IHAVE gossip from any peer whose score is below the gossip threshold
|
|
|
|
score := gs.score.Score(p)
|
|
|
|
if score < gs.gossipThreshold {
|
|
|
|
log.Debugf("IHAVE: ignoring peer %s with score below threshold [score = %f]", p, score)
|
|
|
|
return nil
|
|
|
|
}
|
2018-02-19 16:45:10 +00:00
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
// IHAVE flood protection
|
2020-04-20 13:43:33 +00:00
|
|
|
gs.peerhave[p]++
|
2020-04-20 12:13:15 +00:00
|
|
|
if gs.peerhave[p] > GossipSubMaxIHaveMessages {
|
2020-04-20 13:43:33 +00:00
|
|
|
log.Debugf("IHAVE: peer %s has advertised too many times (%d) within this heartbeat interval; ignoring", p, gs.peerhave[p])
|
2020-04-20 12:02:58 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
if gs.iasked[p] >= GossipSubMaxIHaveLength {
|
2020-04-20 13:43:33 +00:00
|
|
|
log.Debugf("IHAVE: peer %s has already advertised too many messages (%d); ignoring", p, gs.iasked[p])
|
2020-04-20 09:15:59 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-03-02 13:05:24 +00:00
|
|
|
iwant := make(map[string]struct{})
|
2018-02-19 16:45:10 +00:00
|
|
|
for _, ihave := range ctl.GetIhave() {
|
|
|
|
topic := ihave.GetTopicID()
|
|
|
|
_, ok := gs.mesh[topic]
|
|
|
|
if !ok {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, mid := range ihave.GetMessageIDs() {
|
|
|
|
if gs.p.seenMessage(mid) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
iwant[mid] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(iwant) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
iask := len(iwant)
|
|
|
|
if iask+gs.iasked[p] > GossipSubMaxIHaveLength {
|
|
|
|
iask = GossipSubMaxIHaveLength - gs.iasked[p]
|
|
|
|
}
|
|
|
|
|
2020-04-20 13:43:33 +00:00
|
|
|
log.Debugf("IHAVE: Asking for %d out of %d messages from %s", iask, len(iwant), p)
|
2018-08-09 09:16:19 +00:00
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
iwantlst := make([]string, 0, len(iwant))
|
|
|
|
for mid := range iwant {
|
|
|
|
iwantlst = append(iwantlst, mid)
|
|
|
|
}
|
|
|
|
|
2020-03-22 18:52:19 +00:00
|
|
|
// ask in random order
|
|
|
|
shuffleStrings(iwantlst)
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
// truncate to the messages we are actually asking for and update the iasked counter
|
|
|
|
iwantlst = iwantlst[:iask]
|
|
|
|
gs.iasked[p] += iask
|
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
return []*pb.ControlIWant{&pb.ControlIWant{MessageIDs: iwantlst}}
|
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
func (gs *GossipSubRouter) handleIWant(p peer.ID, ctl *pb.ControlMessage) []*pb.Message {
|
2020-03-02 13:05:24 +00:00
|
|
|
// we don't respond to IWANT requests from any peer whose score is below the gossip threshold
|
|
|
|
score := gs.score.Score(p)
|
|
|
|
if score < gs.gossipThreshold {
|
2020-04-01 22:20:53 +00:00
|
|
|
log.Debugf("IWANT: ignoring peer %s with score below threshold [score = %f]", p, score)
|
2020-03-02 13:05:24 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
ihave := make(map[string]*pb.Message)
|
|
|
|
for _, iwant := range ctl.GetIwant() {
|
|
|
|
for _, mid := range iwant.GetMessageIDs() {
|
2020-03-05 22:49:40 +00:00
|
|
|
msg, count, ok := gs.mcache.GetForPeer(mid, p)
|
|
|
|
if !ok {
|
|
|
|
continue
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
2020-03-05 22:49:40 +00:00
|
|
|
|
|
|
|
if count > GossipSubGossipRetransmission {
|
|
|
|
log.Debugf("IWANT: Peer %s has asked for message %s too many times; ignoring request", p, mid)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
ihave[mid] = msg
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(ihave) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("IWANT: Sending %d messages to %s", len(ihave), p)
|
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
msgs := make([]*pb.Message, 0, len(ihave))
|
|
|
|
for _, msg := range ihave {
|
|
|
|
msgs = append(msgs, msg)
|
|
|
|
}
|
|
|
|
|
|
|
|
return msgs
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) handleGraft(p peer.ID, ctl *pb.ControlMessage) []*pb.ControlPrune {
|
|
|
|
var prune []string
|
2020-03-02 13:05:24 +00:00
|
|
|
|
2020-03-27 09:24:51 +00:00
|
|
|
doPX := gs.doPX
|
2020-03-02 13:05:24 +00:00
|
|
|
score := gs.score.Score(p)
|
2020-03-23 07:34:28 +00:00
|
|
|
now := time.Now()
|
2020-03-02 13:05:24 +00:00
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
for _, graft := range ctl.GetGraft() {
|
|
|
|
topic := graft.GetTopicID()
|
|
|
|
peers, ok := gs.mesh[topic]
|
|
|
|
if !ok {
|
2020-03-02 13:05:24 +00:00
|
|
|
// don't do PX when there is an unknown topic to avoid leaking our peers
|
|
|
|
doPX = false
|
2020-04-01 22:20:53 +00:00
|
|
|
// spam hardening: ignore GRAFTs for unknown topics
|
2020-03-02 13:05:24 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-03-31 10:23:04 +00:00
|
|
|
// we don't GRAFT to/from direct peers; complain loudly if this happens
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
if direct {
|
|
|
|
log.Warningf("GRAFT: ignoring request from direct peer %s", p)
|
|
|
|
// this is possibly a bug from non-reciprocal configuration; send a PRUNE
|
|
|
|
prune = append(prune, topic)
|
|
|
|
// but don't PX
|
|
|
|
doPX = false
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-04-20 07:26:32 +00:00
|
|
|
// make sure we are not backing off that peer
|
|
|
|
expire, backoff := gs.backoff[topic][p]
|
|
|
|
if backoff && now.Before(expire) {
|
|
|
|
log.Debugf("GRAFT: ignoring backed off peer %s", p)
|
|
|
|
// check the flood cutoff -- is the GRAFT coming too fast?
|
|
|
|
floodCutoff := expire.Add(GossipSubGraftFloodThreshold - GossipSubPruneBackoff)
|
|
|
|
if now.Before(floodCutoff) {
|
|
|
|
// no prune, and no PX either
|
|
|
|
doPX = false
|
2020-04-20 13:03:44 +00:00
|
|
|
// and a penalty so that we don't GRAFT on this peer ourselves for a while
|
|
|
|
gs.addBackoffPenalty(p, topic)
|
2020-04-20 07:26:32 +00:00
|
|
|
} else {
|
|
|
|
prune = append(prune, topic)
|
2020-04-20 13:03:44 +00:00
|
|
|
// refresh the backoff
|
|
|
|
gs.addBackoff(p, topic)
|
2020-04-20 07:26:32 +00:00
|
|
|
}
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-03-02 13:05:24 +00:00
|
|
|
// check the score
|
|
|
|
if score < 0 {
|
|
|
|
// we don't GRAFT peers with negative score
|
|
|
|
log.Debugf("GRAFT: ignoring peer %s with negative score [score = %f, topic = %s]", p, score, topic)
|
|
|
|
// we do send them PRUNE however, because it's a matter of protocol correctness
|
2018-02-19 16:45:10 +00:00
|
|
|
prune = append(prune, topic)
|
2020-03-02 13:05:24 +00:00
|
|
|
// but we won't PX to them
|
|
|
|
doPX = false
|
2020-03-23 06:29:46 +00:00
|
|
|
// add/refresh backoff so that we don't reGRAFT too early even if the score decays back up
|
|
|
|
gs.addBackoff(p, topic)
|
2020-03-02 13:05:24 +00:00
|
|
|
continue
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
2020-03-02 13:05:24 +00:00
|
|
|
|
2020-04-01 22:20:53 +00:00
|
|
|
log.Debugf("GRAFT: add mesh link from %s in %s", p, topic)
|
2020-03-02 13:05:24 +00:00
|
|
|
gs.tracer.Graft(p, topic)
|
|
|
|
peers[p] = struct{}{}
|
|
|
|
gs.tagPeer(p, topic)
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if len(prune) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
cprune := make([]*pb.ControlPrune, 0, len(prune))
|
|
|
|
for _, topic := range prune {
|
2020-03-02 13:05:24 +00:00
|
|
|
cprune = append(cprune, gs.makePrune(p, topic, doPX))
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return cprune
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) handlePrune(p peer.ID, ctl *pb.ControlMessage) {
|
2020-03-02 13:05:24 +00:00
|
|
|
score := gs.score.Score(p)
|
|
|
|
|
2018-02-19 16:45:10 +00:00
|
|
|
for _, prune := range ctl.GetPrune() {
|
|
|
|
topic := prune.GetTopicID()
|
|
|
|
peers, ok := gs.mesh[topic]
|
2020-03-02 13:05:24 +00:00
|
|
|
if !ok {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debugf("PRUNE: Remove mesh link to %s in %s", p, topic)
|
|
|
|
gs.tracer.Prune(p, topic)
|
|
|
|
delete(peers, p)
|
|
|
|
gs.untagPeer(p, topic)
|
|
|
|
gs.addBackoff(p, topic)
|
|
|
|
|
|
|
|
px := prune.GetPeers()
|
|
|
|
if len(px) > 0 {
|
2020-03-27 09:24:51 +00:00
|
|
|
// we ignore PX from peers with insufficient score
|
|
|
|
if score < gs.acceptPXThreshold {
|
|
|
|
log.Debugf("PRUNE: ignoring PX from peer %s with insufficient score [score = %f, topic = %s]", p, score, topic)
|
2020-03-02 13:05:24 +00:00
|
|
|
continue
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
2020-03-02 13:05:24 +00:00
|
|
|
|
|
|
|
gs.pxConnect(px)
|
2018-02-19 16:45:10 +00:00
|
|
|
}
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
}
|
|
|
|
|
2019-11-22 18:46:13 +00:00
|
|
|
func (gs *GossipSubRouter) addBackoff(p peer.ID, topic string) {
|
2020-04-20 13:03:44 +00:00
|
|
|
gs.doAddBackoff(p, topic, GossipSubPruneBackoff)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) addBackoffPenalty(p peer.ID, topic string) {
|
|
|
|
gs.doAddBackoff(p, topic, GossipSubPruneBackoffPenalty)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) doAddBackoff(p peer.ID, topic string, interval time.Duration) {
|
2019-11-22 18:46:13 +00:00
|
|
|
backoff, ok := gs.backoff[topic]
|
|
|
|
if !ok {
|
|
|
|
backoff = make(map[peer.ID]time.Time)
|
|
|
|
gs.backoff[topic] = backoff
|
|
|
|
}
|
2020-04-20 13:03:44 +00:00
|
|
|
expire := time.Now().Add(interval)
|
|
|
|
if backoff[p].Before(expire) {
|
|
|
|
backoff[p] = expire
|
|
|
|
}
|
2019-11-22 18:46:13 +00:00
|
|
|
}
|
|
|
|
|
2019-11-22 19:53:00 +00:00
|
|
|
func (gs *GossipSubRouter) pxConnect(peers []*pb.PeerInfo) {
|
2019-12-05 16:53:41 +00:00
|
|
|
if len(peers) > GossipSubPrunePeers {
|
2019-12-05 19:01:37 +00:00
|
|
|
shufflePeerInfo(peers)
|
2019-12-05 16:53:41 +00:00
|
|
|
peers = peers[:GossipSubPrunePeers]
|
|
|
|
}
|
|
|
|
|
2019-11-22 19:53:00 +00:00
|
|
|
toconnect := make([]connectInfo, 0, len(peers))
|
2019-12-05 16:53:41 +00:00
|
|
|
|
2019-11-22 19:53:00 +00:00
|
|
|
for _, pi := range peers {
|
|
|
|
p := peer.ID(pi.PeerID)
|
|
|
|
|
|
|
|
_, connected := gs.peers[p]
|
|
|
|
if connected {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-01-27 18:52:54 +00:00
|
|
|
var spr *record.Envelope
|
2020-01-22 15:23:47 +00:00
|
|
|
if pi.SignedPeerRecord != nil {
|
2019-11-22 19:53:00 +00:00
|
|
|
// the peer sent us a signed record; ensure that it is valid
|
2020-01-22 15:23:47 +00:00
|
|
|
envelope, r, err := record.ConsumeEnvelope(pi.SignedPeerRecord, peer.PeerRecordEnvelopeDomain)
|
2019-11-22 19:53:00 +00:00
|
|
|
if err != nil {
|
2020-01-22 15:23:47 +00:00
|
|
|
log.Warningf("error unmarshalling peer record obtained through px: %s", err)
|
2019-11-22 19:53:00 +00:00
|
|
|
continue
|
|
|
|
}
|
2020-01-22 15:23:47 +00:00
|
|
|
rec, ok := r.(*peer.PeerRecord)
|
|
|
|
if !ok {
|
2020-01-22 15:23:47 +00:00
|
|
|
log.Warningf("bogus peer record obtained through px: envelope payload is not PeerRecord")
|
2020-01-27 18:52:36 +00:00
|
|
|
continue
|
2020-01-22 15:23:47 +00:00
|
|
|
}
|
|
|
|
if rec.PeerID != p {
|
2020-01-22 15:23:47 +00:00
|
|
|
log.Warningf("bogus peer record obtained through px: peer ID %s doesn't match expected peer %s", rec.PeerID, p)
|
2019-11-22 19:53:00 +00:00
|
|
|
continue
|
|
|
|
}
|
2020-01-27 18:52:54 +00:00
|
|
|
spr = envelope
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
|
|
|
|
2020-01-27 18:52:54 +00:00
|
|
|
toconnect = append(toconnect, connectInfo{p, spr})
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if len(toconnect) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2019-12-06 15:22:49 +00:00
|
|
|
for _, ci := range toconnect {
|
|
|
|
select {
|
|
|
|
case gs.connect <- ci:
|
|
|
|
default:
|
|
|
|
log.Debugf("ignoring peer connection attempt; too many pending connections")
|
|
|
|
break
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
2019-12-06 15:22:49 +00:00
|
|
|
}
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) connector() {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case ci := <-gs.connect:
|
|
|
|
if gs.p.host.Network().Connectedness(ci.p) == network.Connected {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debugf("connecting to %s", ci.p)
|
2020-01-22 15:23:47 +00:00
|
|
|
cab, ok := peerstore.GetCertifiedAddrBook(gs.p.host.Peerstore())
|
|
|
|
if ok && ci.spr != nil {
|
2020-03-24 13:29:00 +00:00
|
|
|
_, err := cab.ConsumePeerRecord(ci.spr, peerstore.TempAddrTTL)
|
2020-01-22 15:23:47 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Debugf("error processing peer record: %s", err)
|
|
|
|
}
|
2019-11-22 19:53:00 +00:00
|
|
|
}
|
|
|
|
|
2019-11-25 11:58:29 +00:00
|
|
|
ctx, cancel := context.WithTimeout(gs.p.ctx, GossipSubConnectionTimeout)
|
2019-11-22 19:53:00 +00:00
|
|
|
err := gs.p.host.Connect(ctx, peer.AddrInfo{ID: ci.p})
|
|
|
|
cancel()
|
|
|
|
if err != nil {
|
|
|
|
log.Debugf("error connecting to %s: %s", ci.p, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
case <-gs.p.ctx.Done():
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-04 12:39:44 +00:00
|
|
|
func (gs *GossipSubRouter) Publish(msg *Message) {
|
2020-03-03 13:44:44 +00:00
|
|
|
gs.mcache.Put(msg.Message)
|
2020-03-04 12:39:44 +00:00
|
|
|
from := msg.ReceivedFrom
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
tosend := make(map[peer.ID]struct{})
|
|
|
|
for _, topic := range msg.GetTopicIDs() {
|
|
|
|
// any peers in the topic?
|
|
|
|
tmap, ok := gs.p.topics[topic]
|
|
|
|
if !ok {
|
|
|
|
continue
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2020-03-27 18:56:03 +00:00
|
|
|
if gs.floodPublish && from == gs.p.host.ID() {
|
2020-03-03 13:31:16 +00:00
|
|
|
for p := range tmap {
|
2020-03-31 10:23:04 +00:00
|
|
|
_, direct := gs.direct[p]
|
|
|
|
if direct || gs.score.Score(p) >= gs.publishThreshold {
|
2020-03-03 13:31:16 +00:00
|
|
|
tosend[p] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-03-31 10:23:04 +00:00
|
|
|
// direct peers
|
|
|
|
for p := range gs.direct {
|
|
|
|
_, inTopic := tmap[p]
|
|
|
|
if inTopic {
|
|
|
|
tosend[p] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
// floodsub peers
|
|
|
|
for p := range tmap {
|
2020-03-03 13:31:16 +00:00
|
|
|
if gs.peers[p] == FloodSubID && gs.score.Score(p) >= gs.publishThreshold {
|
2018-02-19 14:13:18 +00:00
|
|
|
tosend[p] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
// gossipsub peers
|
|
|
|
gmap, ok := gs.mesh[topic]
|
2018-02-19 17:47:54 +00:00
|
|
|
if !ok {
|
|
|
|
// we are not in the mesh for topic, use fanout peers
|
2018-02-19 14:13:18 +00:00
|
|
|
gmap, ok = gs.fanout[topic]
|
2019-05-15 10:19:35 +00:00
|
|
|
if !ok || len(gmap) == 0 {
|
2020-03-03 13:31:16 +00:00
|
|
|
// we don't have any, pick some with score above the publish threshold
|
2020-04-23 09:53:09 +00:00
|
|
|
peers := gs.getPeers(topic, gs.D, func(p peer.ID) bool {
|
2020-03-03 13:31:16 +00:00
|
|
|
return gs.score.Score(p) >= gs.publishThreshold
|
|
|
|
})
|
2018-02-19 14:13:18 +00:00
|
|
|
|
|
|
|
if len(peers) > 0 {
|
2018-02-21 10:47:45 +00:00
|
|
|
gmap = peerListToMap(peers)
|
2018-02-19 14:13:18 +00:00
|
|
|
gs.fanout[topic] = gmap
|
|
|
|
}
|
|
|
|
}
|
2018-03-10 09:21:50 +00:00
|
|
|
gs.lastpub[topic] = time.Now().UnixNano()
|
2018-02-19 14:13:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for p := range gmap {
|
|
|
|
tosend[p] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-03 13:44:44 +00:00
|
|
|
out := rpcWithMessages(msg.Message)
|
2018-02-19 14:13:18 +00:00
|
|
|
for pid := range tosend {
|
|
|
|
if pid == from || pid == peer.ID(msg.GetFrom()) {
|
|
|
|
continue
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(pid, out)
|
2018-02-19 18:16:58 +00:00
|
|
|
}
|
|
|
|
}
|
2018-02-16 20:01:15 +00:00
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
func (gs *GossipSubRouter) Join(topic string) {
|
|
|
|
gmap, ok := gs.mesh[topic]
|
|
|
|
if ok {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("JOIN %s", topic)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.Join(topic)
|
2018-08-09 09:16:19 +00:00
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
gmap, ok = gs.fanout[topic]
|
|
|
|
if ok {
|
2020-03-03 13:31:16 +00:00
|
|
|
// these peers have a score above the publish threshold, which may be negative
|
|
|
|
// so drop the ones with a negative score
|
|
|
|
for p := range gmap {
|
|
|
|
if gs.score.Score(p) < 0 {
|
|
|
|
delete(gmap, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-23 09:53:09 +00:00
|
|
|
if len(gmap) < gs.D {
|
2019-08-13 09:01:55 +00:00
|
|
|
// we need more peers; eager, as this would get fixed in the next heartbeat
|
2020-04-23 09:53:09 +00:00
|
|
|
more := gs.getPeers(topic, gs.D-len(gmap), func(p peer.ID) bool {
|
2020-03-31 10:23:04 +00:00
|
|
|
// filter our current peers, direct peers, and peers with negative scores
|
|
|
|
_, inMesh := gmap[p]
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return !inMesh && !direct && gs.score.Score(p) >= 0
|
2019-08-13 09:01:55 +00:00
|
|
|
})
|
|
|
|
for _, p := range more {
|
|
|
|
gmap[p] = struct{}{}
|
|
|
|
}
|
|
|
|
}
|
2018-02-19 18:16:58 +00:00
|
|
|
gs.mesh[topic] = gmap
|
|
|
|
delete(gs.fanout, topic)
|
2018-03-10 09:21:50 +00:00
|
|
|
delete(gs.lastpub, topic)
|
2018-02-19 18:16:58 +00:00
|
|
|
} else {
|
2020-04-23 09:53:09 +00:00
|
|
|
peers := gs.getPeers(topic, gs.D, func(p peer.ID) bool {
|
2020-03-31 10:23:04 +00:00
|
|
|
// filter direct peers and peers with negative score
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return !direct && gs.score.Score(p) >= 0
|
2020-03-02 13:13:15 +00:00
|
|
|
})
|
2018-02-21 10:47:45 +00:00
|
|
|
gmap = peerListToMap(peers)
|
2018-02-19 18:16:58 +00:00
|
|
|
gs.mesh[topic] = gmap
|
|
|
|
}
|
|
|
|
|
|
|
|
for p := range gmap {
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("JOIN: Add mesh link to %s in %s", p, topic)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.Graft(p, topic)
|
2018-02-19 18:16:58 +00:00
|
|
|
gs.sendGraft(p, topic)
|
2018-10-26 12:49:34 +00:00
|
|
|
gs.tagPeer(p, topic)
|
2018-02-19 18:16:58 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) Leave(topic string) {
|
|
|
|
gmap, ok := gs.mesh[topic]
|
|
|
|
if !ok {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("LEAVE %s", topic)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.Leave(topic)
|
2018-08-09 09:16:19 +00:00
|
|
|
|
2018-02-20 12:12:52 +00:00
|
|
|
delete(gs.mesh, topic)
|
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
for p := range gmap {
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("LEAVE: Remove mesh link to %s in %s", p, topic)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.Prune(p, topic)
|
2018-02-19 18:16:58 +00:00
|
|
|
gs.sendPrune(p, topic)
|
2018-10-26 12:49:34 +00:00
|
|
|
gs.untagPeer(p, topic)
|
2018-02-19 18:16:58 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) sendGraft(p peer.ID, topic string) {
|
|
|
|
graft := []*pb.ControlGraft{&pb.ControlGraft{TopicID: &topic}}
|
|
|
|
out := rpcWithControl(nil, nil, nil, graft, nil)
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(p, out)
|
2018-02-19 18:16:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) sendPrune(p peer.ID, topic string) {
|
2020-03-02 13:05:24 +00:00
|
|
|
prune := []*pb.ControlPrune{gs.makePrune(p, topic, true)}
|
2018-02-19 18:16:58 +00:00
|
|
|
out := rpcWithControl(nil, nil, nil, nil, prune)
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(p, out)
|
2018-02-19 18:16:58 +00:00
|
|
|
}
|
|
|
|
|
2018-02-20 10:08:18 +00:00
|
|
|
func (gs *GossipSubRouter) sendRPC(p peer.ID, out *RPC) {
|
2018-08-28 18:07:09 +00:00
|
|
|
// do we own the RPC?
|
|
|
|
own := false
|
|
|
|
|
2018-09-10 15:17:23 +00:00
|
|
|
// piggyback control message retries
|
2018-02-20 10:08:18 +00:00
|
|
|
ctl, ok := gs.control[p]
|
|
|
|
if ok {
|
2018-08-28 18:07:09 +00:00
|
|
|
out = copyRPC(out)
|
|
|
|
own = true
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.piggybackControl(p, out, ctl)
|
|
|
|
delete(gs.control, p)
|
|
|
|
}
|
2018-02-19 18:16:58 +00:00
|
|
|
|
2018-02-20 10:08:18 +00:00
|
|
|
// piggyback gossip
|
|
|
|
ihave, ok := gs.gossip[p]
|
|
|
|
if ok {
|
2018-08-28 18:07:09 +00:00
|
|
|
if !own {
|
|
|
|
out = copyRPC(out)
|
|
|
|
own = true
|
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.piggybackGossip(p, out, ihave)
|
|
|
|
delete(gs.gossip, p)
|
|
|
|
}
|
2018-02-19 18:16:58 +00:00
|
|
|
|
|
|
|
mch, ok := gs.p.peers[p]
|
|
|
|
if !ok {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
select {
|
|
|
|
case mch <- out:
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.SendRPC(out, p)
|
2018-02-19 18:16:58 +00:00
|
|
|
default:
|
|
|
|
log.Infof("dropping message to peer %s: queue full", p)
|
2019-11-04 17:22:14 +00:00
|
|
|
gs.tracer.DropRPC(out, p)
|
2018-02-20 10:08:18 +00:00
|
|
|
// push control messages that need to be retried
|
|
|
|
ctl := out.GetControl()
|
|
|
|
if ctl != nil {
|
|
|
|
gs.pushControl(p, ctl)
|
2018-02-19 17:47:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) heartbeatTimer() {
|
2018-08-29 08:15:41 +00:00
|
|
|
time.Sleep(GossipSubHeartbeatInitialDelay)
|
|
|
|
select {
|
|
|
|
case gs.p.eval <- gs.heartbeat:
|
|
|
|
case <-gs.p.ctx.Done():
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-03-10 08:23:55 +00:00
|
|
|
ticker := time.NewTicker(GossipSubHeartbeatInterval)
|
2018-02-19 12:50:14 +00:00
|
|
|
defer ticker.Stop()
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ticker.C:
|
|
|
|
select {
|
2018-02-19 14:13:18 +00:00
|
|
|
case gs.p.eval <- gs.heartbeat:
|
|
|
|
case <-gs.p.ctx.Done():
|
2018-02-19 12:50:14 +00:00
|
|
|
return
|
|
|
|
}
|
2018-02-19 14:13:18 +00:00
|
|
|
case <-gs.p.ctx.Done():
|
2018-02-19 12:50:14 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func (gs *GossipSubRouter) heartbeat() {
|
2018-08-09 09:16:19 +00:00
|
|
|
defer log.EventBegin(gs.p.ctx, "heartbeat").Done()
|
|
|
|
|
2020-03-23 07:49:11 +00:00
|
|
|
gs.heartbeatTicks++
|
|
|
|
|
2018-02-19 19:24:17 +00:00
|
|
|
tograft := make(map[peer.ID][]string)
|
|
|
|
toprune := make(map[peer.ID][]string)
|
2020-03-02 13:55:25 +00:00
|
|
|
noPX := make(map[peer.ID]bool)
|
2018-02-19 19:24:17 +00:00
|
|
|
|
2019-11-22 18:46:13 +00:00
|
|
|
// clean up expired backoffs
|
|
|
|
gs.clearBackoff()
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
// clean up iasked counters
|
2020-04-20 12:02:58 +00:00
|
|
|
gs.clearIHaveCounters()
|
2020-04-20 09:15:59 +00:00
|
|
|
|
2020-03-31 10:23:04 +00:00
|
|
|
// ensure direct peers are connected
|
|
|
|
gs.directConnect()
|
|
|
|
|
2018-02-21 14:34:25 +00:00
|
|
|
// maintain the mesh for topics we have joined
|
2018-02-19 19:24:17 +00:00
|
|
|
for topic, peers := range gs.mesh {
|
2020-03-02 13:55:25 +00:00
|
|
|
prunePeer := func(p peer.ID) {
|
|
|
|
gs.tracer.Prune(p, topic)
|
|
|
|
delete(peers, p)
|
|
|
|
gs.untagPeer(p, topic)
|
2020-03-03 16:00:59 +00:00
|
|
|
gs.addBackoff(p, topic)
|
2020-03-02 13:55:25 +00:00
|
|
|
topics := toprune[p]
|
|
|
|
toprune[p] = append(topics, topic)
|
|
|
|
}
|
|
|
|
|
2020-04-10 11:21:10 +00:00
|
|
|
graftPeer := func(p peer.ID) {
|
|
|
|
log.Debugf("HEARTBEAT: Add mesh link to %s in %s", p, topic)
|
|
|
|
gs.tracer.Graft(p, topic)
|
|
|
|
peers[p] = struct{}{}
|
|
|
|
gs.tagPeer(p, topic)
|
|
|
|
topics := tograft[p]
|
|
|
|
tograft[p] = append(topics, topic)
|
|
|
|
}
|
|
|
|
|
2020-03-02 13:55:25 +00:00
|
|
|
// compute mesh peer scores
|
|
|
|
scores := make(map[peer.ID]float64)
|
|
|
|
for p := range peers {
|
|
|
|
scores[p] = gs.score.Score(p)
|
|
|
|
}
|
|
|
|
|
|
|
|
// drop all peers with negative score, without PX
|
|
|
|
for p := range peers {
|
|
|
|
if scores[p] < 0 {
|
|
|
|
log.Debugf("HEARTBEAT: Prune peer %s with negative score [score = %f, topic = %s]", p, scores[p], topic)
|
|
|
|
prunePeer(p)
|
|
|
|
noPX[p] = true
|
|
|
|
}
|
|
|
|
}
|
2018-02-19 19:24:17 +00:00
|
|
|
|
2018-02-20 08:22:53 +00:00
|
|
|
// do we have enough peers?
|
2020-04-23 09:53:09 +00:00
|
|
|
if l := len(peers); l < gs.Dlo {
|
2019-11-22 18:46:13 +00:00
|
|
|
backoff := gs.backoff[topic]
|
2020-04-23 09:53:09 +00:00
|
|
|
ineed := gs.D - l
|
2018-02-21 10:47:45 +00:00
|
|
|
plst := gs.getPeers(topic, ineed, func(p peer.ID) bool {
|
2020-03-31 10:23:04 +00:00
|
|
|
// filter our current and direct peers, peers we are backing off, and peers with negative score
|
2019-11-22 18:46:13 +00:00
|
|
|
_, inMesh := peers[p]
|
|
|
|
_, doBackoff := backoff[p]
|
2020-03-31 10:23:04 +00:00
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return !inMesh && !doBackoff && !direct && gs.score.Score(p) >= 0
|
2018-02-19 19:24:17 +00:00
|
|
|
})
|
|
|
|
|
2018-02-21 10:47:45 +00:00
|
|
|
for _, p := range plst {
|
2020-04-10 11:21:10 +00:00
|
|
|
graftPeer(p)
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-20 12:00:33 +00:00
|
|
|
// do we have too many peers?
|
2020-04-23 09:53:09 +00:00
|
|
|
if len(peers) > gs.Dhi {
|
2018-02-19 19:24:17 +00:00
|
|
|
plst := peerMapToList(peers)
|
2020-03-02 13:55:25 +00:00
|
|
|
|
|
|
|
// sort by score (but shuffle first for the case we don't use the score)
|
2018-02-19 19:24:17 +00:00
|
|
|
shufflePeers(plst)
|
2020-03-02 13:55:25 +00:00
|
|
|
sort.Slice(plst, func(i, j int) bool {
|
|
|
|
return scores[plst[i]] > scores[plst[j]]
|
|
|
|
})
|
2018-02-20 08:22:53 +00:00
|
|
|
|
2020-03-02 13:55:25 +00:00
|
|
|
// We keep the first D_score peers by score and the remaining up to D_lo randomly
|
2020-04-23 09:53:09 +00:00
|
|
|
shufflePeers(plst[gs.Dscore:])
|
|
|
|
for _, p := range plst[gs.D:] {
|
2018-08-09 09:16:19 +00:00
|
|
|
log.Debugf("HEARTBEAT: Remove mesh link to %s in %s", p, topic)
|
2020-03-02 13:55:25 +00:00
|
|
|
prunePeer(p)
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-10 11:21:10 +00:00
|
|
|
// should we try to improve the mesh with opportunistic grafting?
|
2020-04-22 17:46:35 +00:00
|
|
|
if gs.heartbeatTicks%gs.opportunisticGraftTicks == 0 && len(peers) > 1 {
|
2020-04-10 11:21:10 +00:00
|
|
|
// Opportunistic grafting works as follows: we check the median score of peers in the
|
2020-04-10 19:37:07 +00:00
|
|
|
// mesh; if this score is below the opportunisticGraftThreshold, we select a few peers at
|
2020-04-10 11:21:10 +00:00
|
|
|
// random with score over the median.
|
|
|
|
// The intention is to (slowly) improve an underperforming mesh by introducing good
|
|
|
|
// scoring peers that may have been gossiping at us. This allows us to get out of sticky
|
|
|
|
// situations where we are stuck with poor peers and also recover from churn of good peers.
|
|
|
|
|
|
|
|
// first cache scores for new peers that may have been added since the initial score computation
|
|
|
|
for p := range peers {
|
|
|
|
_, haveScore := scores[p]
|
|
|
|
if haveScore {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
scores[p] = gs.score.Score(p)
|
|
|
|
}
|
|
|
|
|
|
|
|
// now compute the median peer score in the mesh
|
|
|
|
plst := peerMapToList(peers)
|
|
|
|
sort.Slice(plst, func(i, j int) bool {
|
|
|
|
return scores[plst[i]] < scores[plst[j]]
|
|
|
|
})
|
|
|
|
medianIndex := len(peers) / 2
|
|
|
|
medianScore := scores[plst[medianIndex]]
|
|
|
|
|
2020-04-10 17:23:45 +00:00
|
|
|
// if the median score is below the threshold, select a better peer (if any) and GRAFT
|
2020-04-10 11:21:10 +00:00
|
|
|
if medianScore < gs.opportunisticGraftThreshold {
|
|
|
|
backoff := gs.backoff[topic]
|
2020-04-10 19:37:07 +00:00
|
|
|
plst = gs.getPeers(topic, GossipSubOpportunisticGraftPeers, func(p peer.ID) bool {
|
2020-04-10 11:21:10 +00:00
|
|
|
_, inMesh := peers[p]
|
|
|
|
_, doBackoff := backoff[p]
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return !inMesh && !doBackoff && !direct && gs.score.Score(p) > medianScore
|
|
|
|
})
|
|
|
|
|
2020-04-22 10:37:27 +00:00
|
|
|
for _, p := range plst {
|
|
|
|
log.Debugf("HEARTBEAT: Opportunistically graft peer %s on topic %s", p, topic)
|
|
|
|
graftPeer(p)
|
2020-04-10 11:21:10 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-06 09:50:17 +00:00
|
|
|
// 2nd arg are mesh peers excluded from gossip. We already push
|
|
|
|
// messages to them, so its redundant to gossip IHAVEs.
|
2018-02-21 14:34:25 +00:00
|
|
|
gs.emitGossip(topic, peers)
|
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
|
2018-03-10 09:21:50 +00:00
|
|
|
// expire fanout for topics we haven't published to in a while
|
|
|
|
now := time.Now().UnixNano()
|
|
|
|
for topic, lastpub := range gs.lastpub {
|
|
|
|
if lastpub+int64(GossipSubFanoutTTL) < now {
|
|
|
|
delete(gs.fanout, topic)
|
|
|
|
delete(gs.lastpub, topic)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-21 14:34:25 +00:00
|
|
|
// maintain our fanout for topics we are publishing but we have not joined
|
|
|
|
for topic, peers := range gs.fanout {
|
2020-03-03 13:31:16 +00:00
|
|
|
// check whether our peers are still in the topic and have a score above the publish threshold
|
2018-02-21 14:34:25 +00:00
|
|
|
for p := range peers {
|
|
|
|
_, ok := gs.p.topics[topic][p]
|
2020-03-16 19:24:17 +00:00
|
|
|
if !ok || gs.score.Score(p) < gs.publishThreshold {
|
2018-02-21 14:34:25 +00:00
|
|
|
delete(peers, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// do we need more peers?
|
2020-04-23 09:53:09 +00:00
|
|
|
if len(peers) < gs.D {
|
|
|
|
ineed := gs.D - len(peers)
|
2018-02-21 14:34:25 +00:00
|
|
|
plst := gs.getPeers(topic, ineed, func(p peer.ID) bool {
|
2020-03-31 10:23:04 +00:00
|
|
|
// filter our current and direct peers and peers with score above the publish threshold
|
|
|
|
_, inFanout := peers[p]
|
|
|
|
_, direct := gs.direct[p]
|
|
|
|
return !inFanout && !direct && gs.score.Score(p) >= gs.publishThreshold
|
2018-02-21 14:34:25 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
for _, p := range plst {
|
|
|
|
peers[p] = struct{}{}
|
2018-02-20 13:10:16 +00:00
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
}
|
2018-02-21 14:34:25 +00:00
|
|
|
|
2019-10-06 09:50:17 +00:00
|
|
|
// 2nd arg are fanout peers excluded from gossip. We already push
|
|
|
|
// messages to them, so its redundant to gossip IHAVEs.
|
2018-02-21 14:34:25 +00:00
|
|
|
gs.emitGossip(topic, peers)
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
|
2018-02-21 14:34:25 +00:00
|
|
|
// send coalesced GRAFT/PRUNE messages (will piggyback gossip)
|
2020-03-02 13:55:25 +00:00
|
|
|
gs.sendGraftPrune(tograft, toprune, noPX)
|
2018-02-22 08:18:48 +00:00
|
|
|
|
2020-04-03 19:46:14 +00:00
|
|
|
// flush all pending gossip that wasn't piggybacked above
|
|
|
|
gs.flush()
|
|
|
|
|
2018-02-22 08:18:48 +00:00
|
|
|
// advance the message history window
|
|
|
|
gs.mcache.Shift()
|
|
|
|
}
|
|
|
|
|
2020-04-20 12:02:58 +00:00
|
|
|
func (gs *GossipSubRouter) clearIHaveCounters() {
|
|
|
|
if len(gs.peerhave) > 0 {
|
|
|
|
// throw away the old map and make a new one
|
|
|
|
gs.peerhave = make(map[peer.ID]int)
|
|
|
|
}
|
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
if len(gs.iasked) > 0 {
|
|
|
|
// throw away the old map and make a new one
|
|
|
|
gs.iasked = make(map[peer.ID]int)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-11-22 18:46:13 +00:00
|
|
|
func (gs *GossipSubRouter) clearBackoff() {
|
2020-03-23 07:49:11 +00:00
|
|
|
// we only clear once every 15 ticks to avoid iterating over the map(s) too much
|
|
|
|
if gs.heartbeatTicks%15 != 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2019-11-22 18:46:13 +00:00
|
|
|
now := time.Now()
|
|
|
|
for topic, backoff := range gs.backoff {
|
|
|
|
for p, expire := range backoff {
|
|
|
|
if expire.Before(now) {
|
|
|
|
delete(backoff, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if len(backoff) == 0 {
|
|
|
|
delete(gs.backoff, topic)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-31 10:23:04 +00:00
|
|
|
func (gs *GossipSubRouter) directConnect() {
|
2020-04-10 10:54:14 +00:00
|
|
|
// we donly do this every some ticks to allow pending connections to complete and account
|
2020-03-31 10:23:04 +00:00
|
|
|
// for restarts/downtime
|
2020-04-22 17:46:35 +00:00
|
|
|
if gs.heartbeatTicks%gs.directConnectTicks != 0 {
|
2020-03-31 10:23:04 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
var toconnect []peer.ID
|
|
|
|
for p := range gs.direct {
|
|
|
|
_, connected := gs.peers[p]
|
|
|
|
if !connected {
|
|
|
|
toconnect = append(toconnect, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(toconnect) > 0 {
|
|
|
|
go func() {
|
|
|
|
for _, p := range toconnect {
|
|
|
|
gs.connect <- connectInfo{p: p}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-02 13:55:25 +00:00
|
|
|
func (gs *GossipSubRouter) sendGraftPrune(tograft, toprune map[peer.ID][]string, noPX map[peer.ID]bool) {
|
2018-02-19 19:24:17 +00:00
|
|
|
for p, topics := range tograft {
|
|
|
|
graft := make([]*pb.ControlGraft, 0, len(topics))
|
|
|
|
for _, topic := range topics {
|
|
|
|
graft = append(graft, &pb.ControlGraft{TopicID: &topic})
|
|
|
|
}
|
|
|
|
|
|
|
|
var prune []*pb.ControlPrune
|
|
|
|
pruning, ok := toprune[p]
|
|
|
|
if ok {
|
|
|
|
delete(toprune, p)
|
|
|
|
prune = make([]*pb.ControlPrune, 0, len(pruning))
|
|
|
|
for _, topic := range pruning {
|
2020-03-27 09:24:51 +00:00
|
|
|
prune = append(prune, gs.makePrune(p, topic, gs.doPX && !noPX[p]))
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
out := rpcWithControl(nil, nil, nil, graft, prune)
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(p, out)
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for p, topics := range toprune {
|
|
|
|
prune := make([]*pb.ControlPrune, 0, len(topics))
|
|
|
|
for _, topic := range topics {
|
2020-03-27 09:24:51 +00:00
|
|
|
prune = append(prune, gs.makePrune(p, topic, gs.doPX && !noPX[p]))
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
out := rpcWithControl(nil, nil, nil, nil, prune)
|
2018-02-20 10:08:18 +00:00
|
|
|
gs.sendRPC(p, out)
|
2018-02-19 19:24:17 +00:00
|
|
|
}
|
2018-02-20 08:43:27 +00:00
|
|
|
|
2018-02-21 14:34:25 +00:00
|
|
|
}
|
2018-02-20 08:43:27 +00:00
|
|
|
|
2019-10-06 09:50:17 +00:00
|
|
|
// emitGossip emits IHAVE gossip advertising items in the message cache window
|
|
|
|
// of this topic.
|
|
|
|
func (gs *GossipSubRouter) emitGossip(topic string, exclude map[peer.ID]struct{}) {
|
2018-02-21 14:34:25 +00:00
|
|
|
mids := gs.mcache.GetGossipIDs(topic)
|
|
|
|
if len(mids) == 0 {
|
|
|
|
return
|
|
|
|
}
|
2018-02-20 08:43:27 +00:00
|
|
|
|
2020-04-20 09:15:59 +00:00
|
|
|
// shuffle to emit in random order
|
|
|
|
shuffleStrings(mids)
|
|
|
|
|
|
|
|
// if we are emitting more than GossipSubMaxIHaveLength mids, truncate the list
|
|
|
|
if len(mids) > GossipSubMaxIHaveLength {
|
2020-04-20 13:50:02 +00:00
|
|
|
// we do the truncation (with shuffling) per peer below
|
|
|
|
log.Debugf("too many messages for gossip; will truncate IHAVE list (%d messages)", len(mids))
|
2020-04-20 09:15:59 +00:00
|
|
|
}
|
|
|
|
|
2020-03-05 12:52:40 +00:00
|
|
|
// Send gossip to GossipFactor peers above threshold, with a minimum of D_lazy.
|
|
|
|
// First we collect the peers above gossipThreshold that are not in the exclude set
|
|
|
|
// and then randomly select from that set.
|
2020-03-31 10:23:04 +00:00
|
|
|
// We also exclude direct peers, as there is no reason to emit gossip to them.
|
2020-03-05 12:52:40 +00:00
|
|
|
peers := make([]peer.ID, 0, len(gs.p.topics[topic]))
|
|
|
|
for p := range gs.p.topics[topic] {
|
|
|
|
_, inExclude := exclude[p]
|
2020-03-31 10:23:04 +00:00
|
|
|
_, direct := gs.direct[p]
|
|
|
|
if !inExclude && !direct && (gs.peers[p] == GossipSubID_v10 || gs.peers[p] == GossipSubID_v11) && gs.score.Score(p) >= gs.gossipThreshold {
|
2020-03-05 12:52:40 +00:00
|
|
|
peers = append(peers, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-23 09:53:09 +00:00
|
|
|
target := gs.Dlazy
|
2020-03-05 12:52:40 +00:00
|
|
|
factor := int(GossipSubGossipFactor * float64(len(peers)))
|
|
|
|
if factor > target {
|
|
|
|
target = factor
|
|
|
|
}
|
|
|
|
|
|
|
|
if target > len(peers) {
|
|
|
|
target = len(peers)
|
|
|
|
} else {
|
|
|
|
shufflePeers(peers)
|
|
|
|
}
|
|
|
|
peers = peers[:target]
|
2019-10-06 09:52:11 +00:00
|
|
|
|
|
|
|
// Emit the IHAVE gossip to the selected peers.
|
2020-03-05 12:52:40 +00:00
|
|
|
for _, p := range peers {
|
2020-04-20 13:50:02 +00:00
|
|
|
peerMids := mids
|
|
|
|
if len(mids) > GossipSubMaxIHaveLength {
|
|
|
|
// we do this per peer so that we emit a different set for each peer.
|
|
|
|
// we have enough redundancy in the system that this will significantly increase the message
|
|
|
|
// coverage when we do truncate.
|
|
|
|
peerMids = make([]string, GossipSubMaxIHaveLength)
|
|
|
|
shuffleStrings(mids)
|
|
|
|
copy(peerMids, mids)
|
|
|
|
}
|
|
|
|
gs.enqueueGossip(p, &pb.ControlIHave{TopicID: &topic, MessageIDs: peerMids})
|
2018-02-20 08:43:27 +00:00
|
|
|
}
|
2018-02-19 14:13:18 +00:00
|
|
|
}
|
2018-02-19 12:50:14 +00:00
|
|
|
|
2018-02-20 10:08:18 +00:00
|
|
|
func (gs *GossipSubRouter) flush() {
|
2020-04-03 19:46:14 +00:00
|
|
|
// send gossip first, which will also piggyback pending control
|
2018-02-20 10:50:24 +00:00
|
|
|
for p, ihave := range gs.gossip {
|
|
|
|
delete(gs.gossip, p)
|
|
|
|
out := rpcWithControl(nil, ihave, nil, nil, nil)
|
|
|
|
gs.sendRPC(p, out)
|
|
|
|
}
|
|
|
|
|
2020-04-03 19:46:14 +00:00
|
|
|
// send the remaining control messages that wasn't merged with gossip
|
2018-02-20 10:50:24 +00:00
|
|
|
for p, ctl := range gs.control {
|
|
|
|
delete(gs.control, p)
|
|
|
|
out := rpcWithControl(nil, nil, nil, ctl.Graft, ctl.Prune)
|
|
|
|
gs.sendRPC(p, out)
|
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
}
|
|
|
|
|
2019-10-12 12:50:20 +00:00
|
|
|
func (gs *GossipSubRouter) enqueueGossip(p peer.ID, ihave *pb.ControlIHave) {
|
2018-02-20 10:08:18 +00:00
|
|
|
gossip := gs.gossip[p]
|
|
|
|
gossip = append(gossip, ihave)
|
|
|
|
gs.gossip[p] = gossip
|
|
|
|
}
|
|
|
|
|
2018-02-20 10:36:49 +00:00
|
|
|
func (gs *GossipSubRouter) piggybackGossip(p peer.ID, out *RPC, ihave []*pb.ControlIHave) {
|
|
|
|
ctl := out.GetControl()
|
|
|
|
if ctl == nil {
|
|
|
|
ctl = &pb.ControlMessage{}
|
|
|
|
out.Control = ctl
|
|
|
|
}
|
|
|
|
|
|
|
|
ctl.Ihave = ihave
|
2018-02-20 10:08:18 +00:00
|
|
|
}
|
|
|
|
|
2018-02-20 10:36:49 +00:00
|
|
|
func (gs *GossipSubRouter) pushControl(p peer.ID, ctl *pb.ControlMessage) {
|
|
|
|
// remove IHAVE/IWANT from control message, gossip is not retried
|
|
|
|
ctl.Ihave = nil
|
|
|
|
ctl.Iwant = nil
|
2018-02-20 10:50:24 +00:00
|
|
|
if ctl.Graft != nil || ctl.Prune != nil {
|
|
|
|
gs.control[p] = ctl
|
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) piggybackControl(p peer.ID, out *RPC, ctl *pb.ControlMessage) {
|
2018-02-20 10:36:49 +00:00
|
|
|
// check control message for staleness first
|
|
|
|
var tograft []*pb.ControlGraft
|
|
|
|
var toprune []*pb.ControlPrune
|
|
|
|
|
|
|
|
for _, graft := range ctl.GetGraft() {
|
|
|
|
topic := graft.GetTopicID()
|
|
|
|
peers, ok := gs.mesh[topic]
|
|
|
|
if !ok {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
_, ok = peers[p]
|
|
|
|
if ok {
|
|
|
|
tograft = append(tograft, graft)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, prune := range ctl.GetPrune() {
|
|
|
|
topic := prune.GetTopicID()
|
|
|
|
peers, ok := gs.mesh[topic]
|
|
|
|
if !ok {
|
|
|
|
toprune = append(toprune, prune)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
_, ok = peers[p]
|
|
|
|
if !ok {
|
|
|
|
toprune = append(toprune, prune)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-22 09:01:14 +00:00
|
|
|
if len(tograft) == 0 && len(toprune) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-02-20 10:36:49 +00:00
|
|
|
xctl := out.Control
|
|
|
|
if xctl == nil {
|
|
|
|
xctl = &pb.ControlMessage{}
|
|
|
|
out.Control = xctl
|
|
|
|
}
|
|
|
|
|
2018-02-22 09:01:14 +00:00
|
|
|
if len(tograft) > 0 {
|
|
|
|
xctl.Graft = append(xctl.Graft, tograft...)
|
|
|
|
}
|
|
|
|
if len(toprune) > 0 {
|
|
|
|
xctl.Prune = append(xctl.Prune, toprune...)
|
|
|
|
}
|
2018-02-20 10:08:18 +00:00
|
|
|
}
|
|
|
|
|
2020-03-02 13:05:24 +00:00
|
|
|
func (gs *GossipSubRouter) makePrune(p peer.ID, topic string, doPX bool) *pb.ControlPrune {
|
2019-11-21 22:09:37 +00:00
|
|
|
if gs.peers[p] == GossipSubID_v10 {
|
|
|
|
// GossipSub v1.0 -- no peer exchange, the peer won't be able to parse it anyway
|
|
|
|
return &pb.ControlPrune{TopicID: &topic}
|
|
|
|
}
|
|
|
|
|
2020-03-02 13:05:24 +00:00
|
|
|
var px []*pb.PeerInfo
|
|
|
|
if doPX {
|
|
|
|
// select peers for Peer eXchange
|
|
|
|
peers := gs.getPeers(topic, GossipSubPrunePeers, func(xp peer.ID) bool {
|
2020-03-04 12:53:28 +00:00
|
|
|
return p != xp && gs.score.Score(xp) >= 0
|
2020-03-02 13:05:24 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
cab, ok := peerstore.GetCertifiedAddrBook(gs.p.host.Peerstore())
|
|
|
|
px = make([]*pb.PeerInfo, 0, len(peers))
|
|
|
|
for _, p := range peers {
|
|
|
|
// see if we have a signed peer record to send back; if we don't, just send
|
|
|
|
// the peer ID and let the pruned peer find them in the DHT -- we can't trust
|
|
|
|
// unsigned address records through px anyway.
|
|
|
|
var recordBytes []byte
|
|
|
|
if ok {
|
|
|
|
spr := cab.GetPeerRecord(p)
|
|
|
|
var err error
|
|
|
|
if spr != nil {
|
|
|
|
recordBytes, err = spr.Marshal()
|
|
|
|
if err != nil {
|
|
|
|
log.Warningf("error marshaling signed peer record for %s: %s", p, err)
|
|
|
|
}
|
2020-01-22 15:23:47 +00:00
|
|
|
}
|
2019-11-21 22:09:37 +00:00
|
|
|
}
|
2020-03-02 13:05:24 +00:00
|
|
|
px = append(px, &pb.PeerInfo{PeerID: []byte(p), SignedPeerRecord: recordBytes})
|
2019-11-21 22:09:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return &pb.ControlPrune{TopicID: &topic, Peers: px}
|
|
|
|
}
|
|
|
|
|
2018-02-21 10:47:45 +00:00
|
|
|
func (gs *GossipSubRouter) getPeers(topic string, count int, filter func(peer.ID) bool) []peer.ID {
|
2018-02-20 10:08:18 +00:00
|
|
|
tmap, ok := gs.p.topics[topic]
|
|
|
|
if !ok {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
peers := make([]peer.ID, 0, len(tmap))
|
|
|
|
for p := range tmap {
|
2019-11-21 22:09:37 +00:00
|
|
|
if (gs.peers[p] == GossipSubID_v10 || gs.peers[p] == GossipSubID_v11) && filter(p) {
|
2018-02-20 10:08:18 +00:00
|
|
|
peers = append(peers, p)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
shufflePeers(peers)
|
2018-02-21 10:47:45 +00:00
|
|
|
|
|
|
|
if count > 0 && len(peers) > count {
|
|
|
|
peers = peers[:count]
|
|
|
|
}
|
|
|
|
|
2018-02-20 10:08:18 +00:00
|
|
|
return peers
|
|
|
|
}
|
|
|
|
|
2018-10-26 12:49:34 +00:00
|
|
|
func (gs *GossipSubRouter) tagPeer(p peer.ID, topic string) {
|
|
|
|
tag := topicTag(topic)
|
2020-03-27 15:56:10 +00:00
|
|
|
gs.p.host.ConnManager().TagPeer(p, tag, 20)
|
2018-10-26 12:49:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (gs *GossipSubRouter) untagPeer(p peer.ID, topic string) {
|
|
|
|
tag := topicTag(topic)
|
|
|
|
gs.p.host.ConnManager().UntagPeer(p, tag)
|
|
|
|
}
|
|
|
|
|
|
|
|
func topicTag(topic string) string {
|
|
|
|
return fmt.Sprintf("pubsub:%s", topic)
|
|
|
|
}
|
|
|
|
|
2018-02-19 18:16:58 +00:00
|
|
|
func peerListToMap(peers []peer.ID) map[peer.ID]struct{} {
|
|
|
|
pmap := make(map[peer.ID]struct{})
|
|
|
|
for _, p := range peers {
|
|
|
|
pmap[p] = struct{}{}
|
|
|
|
}
|
|
|
|
return pmap
|
|
|
|
}
|
|
|
|
|
2018-02-19 19:24:17 +00:00
|
|
|
func peerMapToList(peers map[peer.ID]struct{}) []peer.ID {
|
|
|
|
plst := make([]peer.ID, 0, len(peers))
|
|
|
|
for p := range peers {
|
|
|
|
plst = append(plst, p)
|
|
|
|
}
|
|
|
|
return plst
|
|
|
|
}
|
|
|
|
|
2018-02-19 14:13:18 +00:00
|
|
|
func shufflePeers(peers []peer.ID) {
|
2018-02-20 12:00:33 +00:00
|
|
|
for i := range peers {
|
|
|
|
j := rand.Intn(i + 1)
|
|
|
|
peers[i], peers[j] = peers[j], peers[i]
|
|
|
|
}
|
2018-02-19 12:50:14 +00:00
|
|
|
}
|
2019-12-05 19:01:37 +00:00
|
|
|
|
|
|
|
func shufflePeerInfo(peers []*pb.PeerInfo) {
|
|
|
|
for i := range peers {
|
|
|
|
j := rand.Intn(i + 1)
|
|
|
|
peers[i], peers[j] = peers[j], peers[i]
|
|
|
|
}
|
|
|
|
}
|
2020-03-22 18:52:19 +00:00
|
|
|
|
|
|
|
func shuffleStrings(lst []string) {
|
|
|
|
for i := range lst {
|
|
|
|
j := rand.Intn(i + 1)
|
|
|
|
lst[i], lst[j] = lst[j], lst[i]
|
|
|
|
}
|
|
|
|
}
|