GossipSub v1.2: IDONTWANT control message and priority queue. (#553)

## GossipSub v1.2 implementation

Specification: libp2p/specs#548

### Work Summary
Sending IDONTWANT

Implement a smart queue
Add priorities to the smart queue

    Put IDONTWANT packets into the smart priority queue as soon as the node gets the packets

Handling IDONTWANT

Use a map to remember the message ids whose IDONTWANT packets have been received
Implement max_idontwant_messages (ignore the IDONWANT packets if the max is reached)
Clear the message IDs from the cache after 3 heartbeats

    Hash the message IDs before putting them into the cache.

More requested features

    Add a feature test to not send IDONTWANT if the other side doesnt support it

### Commit Summary

* Replace sending channel with the smart rpcQueue

Since we want to implement a priority queue later, we need to replace
the normal sending channels with the new smart structures first.

* Implement UrgentPush in the smart rpcQueue

UrgentPush allows you to push an rpc packet to the front of the queue so
that it will be popped out fast.

* Add IDONTWANT to rpc.proto and trace.proto

* Send IDONTWANT right before validation step

Most importantly, this commit adds a new method called PreValidation to
the interface PubSubRouter, which will be called right before validating
the gossipsub message.

In GossipSubRouter, PreValidation will send the IDONTWANT controll
messages to all the mesh peers of the topics of the received messages.

* Test GossipSub IDONWANT sending

* Send IDONWANT only for large messages

* Handle IDONTWANT control messages

When receiving IDONTWANTs, the host should remember the message ids
contained in IDONTWANTs using a hash map.

When receiving messages with those ids, it shouldn't forward them to the
peers who already sent the IDONTWANTs.

When the maximum number of IDONTWANTs is reached for any particular
peer, the host should ignore any excessive IDONTWANTs from that peer.

* Clear expired message IDs from the IDONTWANT cache

If the messages IDs received from IDONTWANTs are older than 3
heartbeats, they should be removed from the IDONTWANT cache.

* Keep the hashes of IDONTWANT message ids instead

Rather than keeping the raw message ids, keep their hashes instead to
save memory and protect again memory DoS attacks.

* Increase GossipSubMaxIHaveMessages to 1000

* fixup! Clear expired message IDs from the IDONTWANT cache

* Not send IDONTWANT if the receiver doesn't support

* fixup! Replace sending channel with the smart rpcQueue

* Not use pointers in rpcQueue

* Simply rcpQueue by using only one mutex

* Check ctx error in rpc sending worker

Co-authored-by: Steven Allen <steven@stebalien.com>

* fixup! Simply rcpQueue by using only one mutex

* fixup! Keep the hashes of IDONTWANT message ids instead

* Use AfterFunc instead implementing our own

* Fix misc lint errors

* fixup! Fix misc lint errors

* Revert "Increase GossipSubMaxIHaveMessages to 1000"

This reverts commit 6fabcdd068a5f5238c5280a3460af9c3998418ec.

* Increase GossipSubMaxIDontWantMessages to 1000

* fixup! Handle IDONTWANT control messages

* Skip TestGossipsubConnTagMessageDeliveries

* Skip FuzzAppendOrMergeRPC

* Revert "Skip FuzzAppendOrMergeRPC"

This reverts commit f141e13234de0960d139339acb636a1afea9e219.

* fixup! Send IDONWANT only for large messages

* fixup! fixup! Keep the hashes of IDONTWANT message ids instead

* fixup! Implement UrgentPush in the smart rpcQueue

* fixup! Use AfterFunc instead implementing our own

---------

Co-authored-by: Steven Allen <steven@stebalien.com>
This commit is contained in:
Pop Chunhapanya 2024-08-16 22:16:35 +07:00 committed by GitHub
parent 19ffbb3a48
commit b421b3ab05
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 2033 additions and 248 deletions

41
comm.go
View File

@ -114,7 +114,7 @@ func (p *PubSub) notifyPeerDead(pid peer.ID) {
}
}
func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing <-chan *RPC) {
func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing *rpcQueue) {
s, err := p.host.NewStream(p.ctx, pid, p.rt.Protocols()...)
if err != nil {
log.Debug("opening new stream to peer: ", err, pid)
@ -135,7 +135,7 @@ func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing <-chan
}
}
func (p *PubSub) handleNewPeerWithBackoff(ctx context.Context, pid peer.ID, backoff time.Duration, outgoing <-chan *RPC) {
func (p *PubSub) handleNewPeerWithBackoff(ctx context.Context, pid peer.ID, backoff time.Duration, outgoing *rpcQueue) {
select {
case <-time.After(backoff):
p.handleNewPeer(ctx, pid, outgoing)
@ -156,7 +156,7 @@ func (p *PubSub) handlePeerDead(s network.Stream) {
p.notifyPeerDead(pid)
}
func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, outgoing <-chan *RPC) {
func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, outgoing *rpcQueue) {
writeRpc := func(rpc *RPC) error {
size := uint64(rpc.Size())
@ -174,20 +174,17 @@ func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, ou
}
defer s.Close()
for {
select {
case rpc, ok := <-outgoing:
if !ok {
return
}
for ctx.Err() == nil {
rpc, err := outgoing.Pop(ctx)
if err != nil {
log.Debugf("popping message from the queue to send to %s: %s", s.Conn().RemotePeer(), err)
return
}
err := writeRpc(rpc)
if err != nil {
s.Reset()
log.Debugf("writing message to %s: %s", s.Conn().RemotePeer(), err)
return
}
case <-ctx.Done():
err = writeRpc(rpc)
if err != nil {
s.Reset()
log.Debugf("writing message to %s: %s", s.Conn().RemotePeer(), err)
return
}
}
@ -209,15 +206,17 @@ func rpcWithControl(msgs []*pb.Message,
ihave []*pb.ControlIHave,
iwant []*pb.ControlIWant,
graft []*pb.ControlGraft,
prune []*pb.ControlPrune) *RPC {
prune []*pb.ControlPrune,
idontwant []*pb.ControlIDontWant) *RPC {
return &RPC{
RPC: pb.RPC{
Publish: msgs,
Control: &pb.ControlMessage{
Ihave: ihave,
Iwant: iwant,
Graft: graft,
Prune: prune,
Ihave: ihave,
Iwant: iwant,
Graft: graft,
Prune: prune,
Idontwant: idontwant,
},
},
}

View File

@ -71,6 +71,8 @@ func (fs *FloodSubRouter) AcceptFrom(peer.ID) AcceptStatus {
return AcceptAll
}
func (fs *FloodSubRouter) PreValidation([]*Message) {}
func (fs *FloodSubRouter) HandleRPC(rpc *RPC) {}
func (fs *FloodSubRouter) Publish(msg *Message) {
@ -83,19 +85,19 @@ func (fs *FloodSubRouter) Publish(msg *Message) {
continue
}
mch, ok := fs.p.peers[pid]
q, ok := fs.p.peers[pid]
if !ok {
continue
}
select {
case mch <- out:
fs.tracer.SendRPC(out, pid)
default:
err := q.Push(out, false)
if err != nil {
log.Infof("dropping message to peer %s: queue full", pid)
fs.tracer.DropRPC(out, pid)
// Drop it. The peer is too slow.
continue
}
fs.tracer.SendRPC(out, pid)
}
}

View File

@ -2,6 +2,7 @@ package pubsub
import (
"context"
"crypto/sha256"
"fmt"
"io"
"math/rand"
@ -22,13 +23,19 @@ import (
const (
// GossipSubID_v10 is the protocol ID for version 1.0.0 of the GossipSub protocol.
// It is advertised along with GossipSubID_v11 for backwards compatibility.
// It is advertised along with GossipSubID_v11 and GossipSubID_v12 for backwards compatibility.
GossipSubID_v10 = protocol.ID("/meshsub/1.0.0")
// GossipSubID_v11 is the protocol ID for version 1.1.0 of the GossipSub protocol.
// It is advertised along with GossipSubID_v12 for backwards compatibility.
// See the spec for details about how v1.1.0 compares to v1.0.0:
// https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.1.md
GossipSubID_v11 = protocol.ID("/meshsub/1.1.0")
// GossipSubID_v12 is the protocol ID for version 1.2.0 of the GossipSub protocol.
// See the spec for details about how v1.2.0 compares to v1.1.0:
// https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.2.md
GossipSubID_v12 = protocol.ID("/meshsub/1.2.0")
)
// Defines the default gossipsub parameters.
@ -59,9 +66,17 @@ var (
GossipSubGraftFloodThreshold = 10 * time.Second
GossipSubMaxIHaveLength = 5000
GossipSubMaxIHaveMessages = 10
GossipSubMaxIDontWantMessages = 1000
GossipSubIWantFollowupTime = 3 * time.Second
GossipSubIDontWantMessageThreshold = 1024 // 1KB
GossipSubIDontWantMessageTTL = 3 // 3 heartbeats
)
type checksum struct {
payload [32]byte
length uint8
}
// GossipSubParams defines all the gossipsub specific parameters.
type GossipSubParams struct {
// overlay parameters.
@ -201,10 +216,21 @@ type GossipSubParams struct {
// MaxIHaveMessages is the maximum number of IHAVE messages to accept from a peer within a heartbeat.
MaxIHaveMessages int
// MaxIDontWantMessages is the maximum number of IDONTWANT messages to accept from a peer within a heartbeat.
MaxIDontWantMessages int
// Time to wait for a message requested through IWANT following an IHAVE advertisement.
// If the message is not received within this window, a broken promise is declared and
// the router may apply bahavioural penalties.
IWantFollowupTime time.Duration
// IDONTWANT is only sent for messages larger than the threshold. This should be greater than
// D_high * the size of the message id. Otherwise, the attacker can do the amplication attack by sending
// small messages while the receiver replies back with larger IDONTWANT messages.
IDontWantMessageThreshold int
// IDONTWANT is cleared when it's older than the TTL.
IDontWantMessageTTL int
}
// NewGossipSub returns a new PubSub object using the default GossipSubRouter as the router.
@ -223,23 +249,25 @@ func NewGossipSubWithRouter(ctx context.Context, h host.Host, rt PubSubRouter, o
func DefaultGossipSubRouter(h host.Host) *GossipSubRouter {
params := DefaultGossipSubParams()
return &GossipSubRouter{
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),
outbound: make(map[peer.ID]bool),
connect: make(chan connectInfo, params.MaxPendingConnections),
cab: pstoremem.NewAddrBook(),
mcache: NewMessageCache(params.HistoryGossip, params.HistoryLength),
protos: GossipSubDefaultProtocols,
feature: GossipSubDefaultFeatures,
tagTracer: newTagTracer(h.ConnManager()),
params: params,
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),
peerdontwant: make(map[peer.ID]int),
unwanted: make(map[peer.ID]map[checksum]int),
iasked: make(map[peer.ID]int),
outbound: make(map[peer.ID]bool),
connect: make(chan connectInfo, params.MaxPendingConnections),
cab: pstoremem.NewAddrBook(),
mcache: NewMessageCache(params.HistoryGossip, params.HistoryLength),
protos: GossipSubDefaultProtocols,
feature: GossipSubDefaultFeatures,
tagTracer: newTagTracer(h.ConnManager()),
params: params,
}
}
@ -273,7 +301,10 @@ func DefaultGossipSubParams() GossipSubParams {
GraftFloodThreshold: GossipSubGraftFloodThreshold,
MaxIHaveLength: GossipSubMaxIHaveLength,
MaxIHaveMessages: GossipSubMaxIHaveMessages,
MaxIDontWantMessages: GossipSubMaxIDontWantMessages,
IWantFollowupTime: GossipSubIWantFollowupTime,
IDontWantMessageThreshold: GossipSubIDontWantMessageThreshold,
IDontWantMessageTTL: GossipSubIDontWantMessageTTL,
SlowHeartbeatWarning: 0.1,
}
}
@ -422,20 +453,22 @@ func WithGossipSubParams(cfg GossipSubParams) Option {
// is the fanout map. Fanout peer lists are expired if we don't publish any
// messages to their topic for GossipSubFanoutTTL.
type GossipSubRouter struct {
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
outbound map[peer.ID]bool // connection direction cache, marks peers with outbound connections
backoff map[string]map[peer.ID]time.Time // prune backoff
connect chan connectInfo // px connection requests
cab peerstore.AddrBook
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
peerdontwant map[peer.ID]int // number of IDONTWANTs received from peer in the last heartbeat
unwanted map[peer.ID]map[checksum]int // TTL of the message ids peers don't want
iasked map[peer.ID]int // number of messages we have asked from peer in the last heartbeat
outbound map[peer.ID]bool // connection direction cache, marks peers with outbound connections
backoff map[string]map[peer.ID]time.Time // prune backoff
connect chan connectInfo // px connection requests
cab peerstore.AddrBook
protos []protocol.ID
feature GossipSubFeatureTest
@ -663,6 +696,36 @@ func (gs *GossipSubRouter) AcceptFrom(p peer.ID) AcceptStatus {
return gs.gate.AcceptFrom(p)
}
// PreValidation sends the IDONTWANT control messages to all the mesh
// peers. They need to be sent right before the validation because they
// should be seen by the peers as soon as possible.
func (gs *GossipSubRouter) PreValidation(msgs []*Message) {
tmids := make(map[string][]string)
for _, msg := range msgs {
if len(msg.GetData()) < gs.params.IDontWantMessageThreshold {
continue
}
topic := msg.GetTopic()
tmids[topic] = append(tmids[topic], gs.p.idGen.ID(msg))
}
for topic, mids := range tmids {
if len(mids) == 0 {
continue
}
// shuffle the messages got from the RPC envelope
shuffleStrings(mids)
// send IDONTWANT to all the mesh peers
for p := range gs.mesh[topic] {
// send to only peers that support IDONTWANT
if gs.feature(GossipSubFeatureIdontwant, gs.peers[p]) {
idontwant := []*pb.ControlIDontWant{{MessageIDs: mids}}
out := rpcWithControl(nil, nil, nil, nil, nil, idontwant)
gs.sendRPC(p, out, true)
}
}
}
}
func (gs *GossipSubRouter) HandleRPC(rpc *RPC) {
ctl := rpc.GetControl()
if ctl == nil {
@ -673,13 +736,14 @@ func (gs *GossipSubRouter) HandleRPC(rpc *RPC) {
ihave := gs.handleIWant(rpc.from, ctl)
prune := gs.handleGraft(rpc.from, ctl)
gs.handlePrune(rpc.from, ctl)
gs.handleIDontWant(rpc.from, ctl)
if len(iwant) == 0 && len(ihave) == 0 && len(prune) == 0 {
return
}
out := rpcWithControl(ihave, nil, iwant, nil, prune)
gs.sendRPC(rpc.from, out)
out := rpcWithControl(ihave, nil, iwant, nil, prune, nil)
gs.sendRPC(rpc.from, out, false)
}
func (gs *GossipSubRouter) handleIHave(p peer.ID, ctl *pb.ControlMessage) []*pb.ControlIWant {
@ -931,6 +995,26 @@ func (gs *GossipSubRouter) handlePrune(p peer.ID, ctl *pb.ControlMessage) {
}
}
func (gs *GossipSubRouter) handleIDontWant(p peer.ID, ctl *pb.ControlMessage) {
if gs.unwanted[p] == nil {
gs.unwanted[p] = make(map[checksum]int)
}
// IDONTWANT flood protection
if gs.peerdontwant[p] >= gs.params.MaxIDontWantMessages {
log.Debugf("IDONWANT: peer %s has advertised too many times (%d) within this heartbeat interval; ignoring", p, gs.peerdontwant[p])
return
}
gs.peerdontwant[p]++
// Remember all the unwanted message ids
for _, idontwant := range ctl.GetIdontwant() {
for _, mid := range idontwant.GetMessageIDs() {
gs.unwanted[p][computeChecksum(mid)] = gs.params.IDontWantMessageTTL
}
}
}
func (gs *GossipSubRouter) addBackoff(p peer.ID, topic string, isUnsubscribe bool) {
backoff := gs.params.PruneBackoff
if isUnsubscribe {
@ -1091,6 +1175,12 @@ func (gs *GossipSubRouter) Publish(msg *Message) {
}
for p := range gmap {
mid := gs.p.idGen.ID(msg)
// Check if it has already received an IDONTWANT for the message.
// If so, don't send it to the peer
if _, ok := gs.unwanted[p][computeChecksum(mid)]; ok {
continue
}
tosend[p] = struct{}{}
}
}
@ -1101,7 +1191,7 @@ func (gs *GossipSubRouter) Publish(msg *Message) {
continue
}
gs.sendRPC(pid, out)
gs.sendRPC(pid, out, false)
}
}
@ -1186,17 +1276,17 @@ func (gs *GossipSubRouter) Leave(topic string) {
func (gs *GossipSubRouter) sendGraft(p peer.ID, topic string) {
graft := []*pb.ControlGraft{{TopicID: &topic}}
out := rpcWithControl(nil, nil, nil, graft, nil)
gs.sendRPC(p, out)
out := rpcWithControl(nil, nil, nil, graft, nil, nil)
gs.sendRPC(p, out, false)
}
func (gs *GossipSubRouter) sendPrune(p peer.ID, topic string, isUnsubscribe bool) {
prune := []*pb.ControlPrune{gs.makePrune(p, topic, gs.doPX, isUnsubscribe)}
out := rpcWithControl(nil, nil, nil, nil, prune)
gs.sendRPC(p, out)
out := rpcWithControl(nil, nil, nil, nil, prune, nil)
gs.sendRPC(p, out, false)
}
func (gs *GossipSubRouter) sendRPC(p peer.ID, out *RPC) {
func (gs *GossipSubRouter) sendRPC(p peer.ID, out *RPC, urgent bool) {
// do we own the RPC?
own := false
@ -1220,14 +1310,14 @@ func (gs *GossipSubRouter) sendRPC(p peer.ID, out *RPC) {
delete(gs.gossip, p)
}
mch, ok := gs.p.peers[p]
q, ok := gs.p.peers[p]
if !ok {
return
}
// If we're below the max message size, go ahead and send
if out.Size() < gs.p.maxMessageSize {
gs.doSendRPC(out, p, mch)
gs.doSendRPC(out, p, q, urgent)
return
}
@ -1239,7 +1329,7 @@ func (gs *GossipSubRouter) sendRPC(p peer.ID, out *RPC) {
gs.doDropRPC(out, p, fmt.Sprintf("Dropping oversized RPC. Size: %d, limit: %d. (Over by %d bytes)", rpc.Size(), gs.p.maxMessageSize, rpc.Size()-gs.p.maxMessageSize))
continue
}
gs.doSendRPC(rpc, p, mch)
gs.doSendRPC(rpc, p, q, urgent)
}
}
@ -1253,13 +1343,18 @@ func (gs *GossipSubRouter) doDropRPC(rpc *RPC, p peer.ID, reason string) {
}
}
func (gs *GossipSubRouter) doSendRPC(rpc *RPC, p peer.ID, mch chan *RPC) {
select {
case mch <- rpc:
gs.tracer.SendRPC(rpc, p)
default:
gs.doDropRPC(rpc, p, "queue full")
func (gs *GossipSubRouter) doSendRPC(rpc *RPC, p peer.ID, q *rpcQueue, urgent bool) {
var err error
if urgent {
err = q.UrgentPush(rpc, false)
} else {
err = q.Push(rpc, false)
}
if err != nil {
gs.doDropRPC(rpc, p, "queue full")
return
}
gs.tracer.SendRPC(rpc, p)
}
// appendOrMergeRPC appends the given RPCs to the slice, merging them if possible.
@ -1441,6 +1536,9 @@ func (gs *GossipSubRouter) heartbeat() {
// clean up iasked counters
gs.clearIHaveCounters()
// clean up IDONTWANT counters
gs.clearIDontWantCounters()
// apply IWANT request penalties
gs.applyIwantPenalties()
@ -1693,6 +1791,23 @@ func (gs *GossipSubRouter) clearIHaveCounters() {
}
}
func (gs *GossipSubRouter) clearIDontWantCounters() {
if len(gs.peerdontwant) > 0 {
// throw away the old map and make a new one
gs.peerdontwant = make(map[peer.ID]int)
}
// decrement TTLs of all the IDONTWANTs and delete it from the cache when it reaches zero
for _, mids := range gs.unwanted {
for mid := range mids {
mids[mid]--
if mids[mid] == 0 {
delete(mids, mid)
}
}
}
}
func (gs *GossipSubRouter) applyIwantPenalties() {
for p, count := range gs.gossipTracer.GetBrokenPromises() {
log.Infof("peer %s didn't follow up in %d IWANT requests; adding penalty", p, count)
@ -1767,8 +1882,8 @@ func (gs *GossipSubRouter) sendGraftPrune(tograft, toprune map[peer.ID][]string,
}
}
out := rpcWithControl(nil, nil, nil, graft, prune)
gs.sendRPC(p, out)
out := rpcWithControl(nil, nil, nil, graft, prune, nil)
gs.sendRPC(p, out, false)
}
for p, topics := range toprune {
@ -1777,8 +1892,8 @@ func (gs *GossipSubRouter) sendGraftPrune(tograft, toprune map[peer.ID][]string,
prune = append(prune, gs.makePrune(p, topic, gs.doPX && !noPX[p], false))
}
out := rpcWithControl(nil, nil, nil, nil, prune)
gs.sendRPC(p, out)
out := rpcWithControl(nil, nil, nil, nil, prune, nil)
gs.sendRPC(p, out, false)
}
}
@ -1844,15 +1959,15 @@ func (gs *GossipSubRouter) flush() {
// send gossip first, which will also piggyback pending control
for p, ihave := range gs.gossip {
delete(gs.gossip, p)
out := rpcWithControl(nil, ihave, nil, nil, nil)
gs.sendRPC(p, out)
out := rpcWithControl(nil, ihave, nil, nil, nil, nil)
gs.sendRPC(p, out, false)
}
// send the remaining control messages that wasn't merged with gossip
for p, ctl := range gs.control {
delete(gs.control, p)
out := rpcWithControl(nil, nil, nil, ctl.Graft, ctl.Prune)
gs.sendRPC(p, out)
out := rpcWithControl(nil, nil, nil, ctl.Graft, ctl.Prune, nil)
gs.sendRPC(p, out, false)
}
}
@ -1873,9 +1988,10 @@ func (gs *GossipSubRouter) piggybackGossip(p peer.ID, out *RPC, ihave []*pb.Cont
}
func (gs *GossipSubRouter) pushControl(p peer.ID, ctl *pb.ControlMessage) {
// remove IHAVE/IWANT from control message, gossip is not retried
// remove IHAVE/IWANT/IDONTWANT from control message, gossip is not retried
ctl.Ihave = nil
ctl.Iwant = nil
ctl.Idontwant = nil
if ctl.Graft != nil || ctl.Prune != nil {
gs.control[p] = ctl
}
@ -2037,3 +2153,13 @@ func shuffleStrings(lst []string) {
lst[i], lst[j] = lst[j], lst[i]
}
}
func computeChecksum(mid string) checksum {
var cs checksum
if len(mid) > 32 || len(mid) == 0 {
cs.payload = sha256.Sum256([]byte(mid))
} else {
cs.length = uint8(copy(cs.payload[:], mid))
}
return cs
}

View File

@ -15,6 +15,7 @@ import (
)
func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
t.Skip("flaky test disabled")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

View File

@ -18,18 +18,22 @@ const (
GossipSubFeatureMesh = iota
// Protocol supports Peer eXchange on prune -- gossipsub-v1.1 compatible
GossipSubFeaturePX
// Protocol supports IDONTWANT -- gossipsub-v1.2 compatible
GossipSubFeatureIdontwant
)
// GossipSubDefaultProtocols is the default gossipsub router protocol list
var GossipSubDefaultProtocols = []protocol.ID{GossipSubID_v11, GossipSubID_v10, FloodSubID}
var GossipSubDefaultProtocols = []protocol.ID{GossipSubID_v12, GossipSubID_v11, GossipSubID_v10, FloodSubID}
// GossipSubDefaultFeatures is the feature test function for the default gossipsub protocols
func GossipSubDefaultFeatures(feat GossipSubFeature, proto protocol.ID) bool {
switch feat {
case GossipSubFeatureMesh:
return proto == GossipSubID_v11 || proto == GossipSubID_v10
return proto == GossipSubID_v12 || proto == GossipSubID_v11 || proto == GossipSubID_v10
case GossipSubFeaturePX:
return proto == GossipSubID_v11
return proto == GossipSubID_v12 || proto == GossipSubID_v11
case GossipSubFeatureIdontwant:
return proto == GossipSubID_v12
default:
return false
}

View File

@ -21,6 +21,9 @@ func TestDefaultGossipSubFeatures(t *testing.T) {
if !GossipSubDefaultFeatures(GossipSubFeatureMesh, GossipSubID_v11) {
t.Fatal("gossipsub-v1.1 should support Mesh")
}
if !GossipSubDefaultFeatures(GossipSubFeatureMesh, GossipSubID_v12) {
t.Fatal("gossipsub-v1.2 should support Mesh")
}
if GossipSubDefaultFeatures(GossipSubFeaturePX, FloodSubID) {
t.Fatal("floodsub should not support PX")
@ -28,9 +31,25 @@ func TestDefaultGossipSubFeatures(t *testing.T) {
if GossipSubDefaultFeatures(GossipSubFeaturePX, GossipSubID_v10) {
t.Fatal("gossipsub-v1.0 should not support PX")
}
if !GossipSubDefaultFeatures(GossipSubFeatureMesh, GossipSubID_v11) {
if !GossipSubDefaultFeatures(GossipSubFeaturePX, GossipSubID_v11) {
t.Fatal("gossipsub-v1.1 should support PX")
}
if !GossipSubDefaultFeatures(GossipSubFeaturePX, GossipSubID_v12) {
t.Fatal("gossipsub-v1.2 should support PX")
}
if GossipSubDefaultFeatures(GossipSubFeatureIdontwant, FloodSubID) {
t.Fatal("floodsub should not support IDONTWANT")
}
if GossipSubDefaultFeatures(GossipSubFeatureIdontwant, GossipSubID_v10) {
t.Fatal("gossipsub-v1.0 should not support IDONTWANT")
}
if GossipSubDefaultFeatures(GossipSubFeatureIdontwant, GossipSubID_v11) {
t.Fatal("gossipsub-v1.1 should not support IDONTWANT")
}
if !GossipSubDefaultFeatures(GossipSubFeatureIdontwant, GossipSubID_v12) {
t.Fatal("gossipsub-v1.2 should support IDONTWANT")
}
}
func TestGossipSubCustomProtocols(t *testing.T) {

View File

@ -3,6 +3,7 @@ package pubsub
import (
"context"
"crypto/rand"
"encoding/base64"
"strconv"
"sync"
"testing"
@ -121,7 +122,7 @@ func TestGossipsubAttackSpamIWANT(t *testing.T) {
// being spammy)
iwantlst := []string{DefaultMsgIdFn(msg)}
iwant := []*pb.ControlIWant{{MessageIDs: iwantlst}}
orpc := rpcWithControl(nil, nil, iwant, nil, nil)
orpc := rpcWithControl(nil, nil, iwant, nil, nil, nil)
writeMsg(&orpc.RPC)
}
})
@ -208,7 +209,7 @@ func TestGossipsubAttackSpamIHAVE(t *testing.T) {
for i := 0; i < 3*GossipSubMaxIHaveLength; i++ {
ihavelst := []string{"someid" + strconv.Itoa(i)}
ihave := []*pb.ControlIHave{{TopicID: sub.Topicid, MessageIDs: ihavelst}}
orpc := rpcWithControl(nil, ihave, nil, nil, nil)
orpc := rpcWithControl(nil, ihave, nil, nil, nil, nil)
writeMsg(&orpc.RPC)
}
@ -238,7 +239,7 @@ func TestGossipsubAttackSpamIHAVE(t *testing.T) {
for i := 0; i < 3*GossipSubMaxIHaveLength; i++ {
ihavelst := []string{"someid" + strconv.Itoa(i+100)}
ihave := []*pb.ControlIHave{{TopicID: sub.Topicid, MessageIDs: ihavelst}}
orpc := rpcWithControl(nil, ihave, nil, nil, nil)
orpc := rpcWithControl(nil, ihave, nil, nil, nil, nil)
writeMsg(&orpc.RPC)
}
@ -765,11 +766,139 @@ func TestGossipsubAttackInvalidMessageSpam(t *testing.T) {
<-ctx.Done()
}
// Test that when Gossipsub receives too many IDONTWANT messages from a peer
func TestGossipsubAttackSpamIDONTWANT(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
msgID := func(pmsg *pb.Message) string {
// silly content-based test message-ID: just use the data as whole
return base64.URLEncoding.EncodeToString(pmsg.Data)
}
psubs := make([]*PubSub, 2)
psubs[0] = getGossipsub(ctx, hosts[0], WithMessageIdFn(msgID))
psubs[1] = getGossipsub(ctx, hosts[1], WithMessageIdFn(msgID))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking the result
msgWaitMax := time.Second + GossipSubHeartbeatInterval
msgTimer := time.NewTimer(msgWaitMax)
// Checks we received some messages
var expMid string
var actMids []string
checkMsgs := func() {
if len(actMids) == 0 {
t.Fatalf("Expected some messages when the maximum number of IDONTWANTs is reached")
}
if actMids[0] != expMid {
t.Fatalf("The expected message is incorrect")
}
if len(actMids) > 1 {
t.Fatalf("The spam prevention should be reset after the heartbeat")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// Each time the host receives a message
for _, msg := range irpc.GetPublish() {
actMids = append(actMids, msgID(msg))
}
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and grafting to the middle peer
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe + graft
time.Sleep(100 * time.Millisecond)
// Generate a message and send IDONTWANT to the middle peer
data := make([]byte, 16)
var mid string
for i := 0; i < 1+GossipSubMaxIDontWantMessages; i++ {
rand.Read(data)
mid = msgID(&pb.Message{Data: data})
writeMsg(&pb.RPC{
Control: &pb.ControlMessage{Idontwant: []*pb.ControlIDontWant{{MessageIDs: []string{mid}}}},
})
}
// The host should receives this message id because the maximum was reached
expMid = mid
// Wait for a short interval to make sure the middle peer
// received and processed the IDONTWANTs
time.Sleep(100 * time.Millisecond)
// Publish the message from the first peer
if err := psubs[0].Publish(topic, data); err != nil {
t.Error(err)
return // cannot call t.Fatal in a non-test goroutine
}
// Wait for the next heartbeat so that the prevention will be reset
select {
case <-ctx.Done():
return
case <-time.After(GossipSubHeartbeatInterval):
}
// Test IDONTWANT again to see that it now works again
rand.Read(data)
mid = msgID(&pb.Message{Data: data})
writeMsg(&pb.RPC{
Control: &pb.ControlMessage{Idontwant: []*pb.ControlIDontWant{{MessageIDs: []string{mid}}}},
})
time.Sleep(100 * time.Millisecond)
if err := psubs[0].Publish(topic, data); err != nil {
t.Error(err)
return // cannot call t.Fatal in a non-test goroutine
}
}()
}
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
type mockGSOnRead func(writeMsg func(*pb.RPC), irpc *pb.RPC)
func newMockGS(ctx context.Context, t *testing.T, attacker host.Host, onReadMsg mockGSOnRead) {
newMockGSWithVersion(ctx, t, attacker, protocol.ID("/meshsub/1.2.0"), onReadMsg)
}
func newMockGSWithVersion(ctx context.Context, t *testing.T, attacker host.Host, gossipSubID protocol.ID, onReadMsg mockGSOnRead) {
// Listen on the gossipsub protocol
const gossipSubID = protocol.ID("/meshsub/1.0.0")
const maxMessageSize = 1024 * 1024
attacker.SetStreamHandler(gossipSubID, func(stream network.Stream) {
// When an incoming stream is opened, set up an outgoing stream

View File

@ -4,9 +4,11 @@ import (
"bytes"
"context"
crand "crypto/rand"
"encoding/base64"
"fmt"
"io"
mrand "math/rand"
"sort"
"sync"
"sync/atomic"
"testing"
@ -18,6 +20,7 @@ import (
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p/core/peerstore"
"github.com/libp2p/go-libp2p/core/protocol"
"github.com/libp2p/go-libp2p/core/record"
//lint:ignore SA1019 "github.com/libp2p/go-msgio/protoio" is deprecated
@ -2326,7 +2329,7 @@ func (iwe *iwantEverything) handleStream(s network.Stream) {
}
}
out := rpcWithControl(nil, nil, iwants, nil, prunes)
out := rpcWithControl(nil, nil, iwants, nil, prunes, nil)
err = w.WriteMsg(out)
if err != nil {
panic(err)
@ -2590,3 +2593,585 @@ func TestGossipsubManagesAnAddressBook(t *testing.T) {
t.Fatalf("expected no addrs, got %d addrs", len(addrs))
}
}
func TestGossipsubIdontwantSend(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
msgID := func(pmsg *pb.Message) string {
// silly content-based test message-ID: just use the data as whole
return base64.URLEncoding.EncodeToString(pmsg.Data)
}
validated := false
validate := func(context.Context, peer.ID, *Message) bool {
time.Sleep(100 * time.Millisecond)
validated = true
return true
}
params := DefaultGossipSubParams()
params.IDontWantMessageThreshold = 16
psubs := make([]*PubSub, 2)
psubs[0] = getGossipsub(ctx, hosts[0],
WithGossipSubParams(params),
WithMessageIdFn(msgID))
psubs[1] = getGossipsub(ctx, hosts[1],
WithGossipSubParams(params),
WithMessageIdFn(msgID),
WithDefaultValidator(validate))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
var expMids []string
var actMids []string
// Used to publish a message with random data
publishMsg := func() {
data := make([]byte, 16)
crand.Read(data)
m := &pb.Message{Data: data}
expMids = append(expMids, msgID(m))
if err := psubs[0].Publish(topic, data); err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking we got the right messages
msgWaitMax := time.Second
msgTimer := time.NewTimer(msgWaitMax)
// Checks we received the right IDONTWANT messages
checkMsgs := func() {
sort.Strings(actMids)
sort.Strings(expMids)
if len(actMids) != len(expMids) {
t.Fatalf("Expected %d IDONTWANT messages, got %d", len(expMids), len(actMids))
}
for i, expMid := range expMids {
actMid := actMids[i]
if actMid != expMid {
t.Fatalf("Expected the id of %s in the %d'th IDONTWANT messages, got %s", expMid, i+1, actMid)
}
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and grafting to the middle peer
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe + graft
time.Sleep(100 * time.Millisecond)
// Publish messages from the first peer
for i := 0; i < 10; i++ {
publishMsg()
}
}()
}
}
// Each time the middle peer sends an IDONTWANT message
for _, idonthave := range irpc.GetControl().GetIdontwant() {
// If true, it means that, when we get IDONTWANT, the middle peer has done validation
// already, which should not be the case
if validated {
t.Fatalf("IDONTWANT should be sent before doing validation")
}
for _, mid := range idonthave.GetMessageIDs() {
// Add the message to the list and reset the timer
actMids = append(actMids, mid)
msgTimer.Reset(msgWaitMax)
}
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
func TestGossipsubIdontwantReceive(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
msgID := func(pmsg *pb.Message) string {
// silly content-based test message-ID: just use the data as whole
return base64.URLEncoding.EncodeToString(pmsg.Data)
}
psubs := make([]*PubSub, 2)
psubs[0] = getGossipsub(ctx, hosts[0], WithMessageIdFn(msgID))
psubs[1] = getGossipsub(ctx, hosts[1], WithMessageIdFn(msgID))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking the result
msgWaitMax := time.Second
msgTimer := time.NewTimer(msgWaitMax)
// Checks we received no messages
received := false
checkMsgs := func() {
if received {
t.Fatalf("Expected no messages received after IDONWANT")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// Check if it receives any message
if len(irpc.GetPublish()) > 0 {
received = true
}
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and grafting to the middle peer
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe + graft
time.Sleep(100 * time.Millisecond)
// Generate a message and send IDONTWANT to the middle peer
data := make([]byte, 16)
crand.Read(data)
mid := msgID(&pb.Message{Data: data})
writeMsg(&pb.RPC{
Control: &pb.ControlMessage{Idontwant: []*pb.ControlIDontWant{{MessageIDs: []string{mid}}}},
})
// Wait for a short interval to make sure the middle peer
// received and processed the IDONTWANTs
time.Sleep(100 * time.Millisecond)
// Publish the message from the first peer
if err := psubs[0].Publish(topic, data); err != nil {
t.Error(err)
return // cannot call t.Fatal in a non-test goroutine
}
}()
}
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
// Test that non-mesh peers will not get IDONTWANT
func TestGossipsubIdontwantNonMesh(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
params := DefaultGossipSubParams()
params.IDontWantMessageThreshold = 16
psubs := getGossipsubs(ctx, hosts[:2], WithGossipSubParams(params))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Used to publish a message with random data
publishMsg := func() {
data := make([]byte, 16)
crand.Read(data)
if err := psubs[0].Publish(topic, data); err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking we got the right messages
msgWaitMax := time.Second
msgTimer := time.NewTimer(msgWaitMax)
received := false
// Checks if we received any IDONTWANT
checkMsgs := func() {
if received {
t.Fatalf("No IDONTWANT is expected")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and pruning to the middle peer to make sure
// that it's not in the mesh
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Prune: []*pb.ControlPrune{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe
time.Sleep(100 * time.Millisecond)
// Publish messages from the first peer
for i := 0; i < 10; i++ {
publishMsg()
}
}()
}
}
// Each time the middle peer sends an IDONTWANT message
for range irpc.GetControl().GetIdontwant() {
received = true
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
// Test that peers with incompatible versions will not get IDONTWANT
func TestGossipsubIdontwantIncompat(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
params := DefaultGossipSubParams()
params.IDontWantMessageThreshold = 16
psubs := getGossipsubs(ctx, hosts[:2], WithGossipSubParams(params))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Used to publish a message with random data
publishMsg := func() {
data := make([]byte, 16)
crand.Read(data)
if err := psubs[0].Publish(topic, data); err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking we got the right messages
msgWaitMax := time.Second
msgTimer := time.NewTimer(msgWaitMax)
received := false
// Checks if we received any IDONTWANT
checkMsgs := func() {
if received {
t.Fatalf("No IDONTWANT is expected")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
// Use the old GossipSub version
newMockGSWithVersion(ctx, t, hosts[2], protocol.ID("/meshsub/1.1.0"), func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and grafting to the middle peer
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe + graft
time.Sleep(100 * time.Millisecond)
// Publish messages from the first peer
for i := 0; i < 10; i++ {
publishMsg()
}
}()
}
}
// Each time the middle peer sends an IDONTWANT message
for range irpc.GetControl().GetIdontwant() {
received = true
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
// Test that IDONTWANT will not be sent for small messages
func TestGossipsubIdontwantSmallMessage(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
params := DefaultGossipSubParams()
params.IDontWantMessageThreshold = 16
psubs := getGossipsubs(ctx, hosts[:2], WithGossipSubParams(params))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Used to publish a message with random data
publishMsg := func() {
data := make([]byte, 8)
crand.Read(data)
if err := psubs[0].Publish(topic, data); err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking we got the right messages
msgWaitMax := time.Second
msgTimer := time.NewTimer(msgWaitMax)
received := false
// Checks if we received any IDONTWANT
checkMsgs := func() {
if received {
t.Fatalf("No IDONTWANT is expected")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and pruning to the middle peer to make sure
// that it's not in the mesh
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe
time.Sleep(100 * time.Millisecond)
// Publish messages from the first peer
for i := 0; i < 10; i++ {
publishMsg()
}
}()
}
}
// Each time the middle peer sends an IDONTWANT message
for range irpc.GetControl().GetIdontwant() {
received = true
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}
// Test that IDONTWANT will cleared when it's old enough
func TestGossipsubIdontwantClear(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getDefaultHosts(t, 3)
msgID := func(pmsg *pb.Message) string {
// silly content-based test message-ID: just use the data as whole
return base64.URLEncoding.EncodeToString(pmsg.Data)
}
psubs := make([]*PubSub, 2)
psubs[0] = getGossipsub(ctx, hosts[0], WithMessageIdFn(msgID))
psubs[1] = getGossipsub(ctx, hosts[1], WithMessageIdFn(msgID))
topic := "foobar"
for _, ps := range psubs {
_, err := ps.Subscribe(topic)
if err != nil {
t.Fatal(err)
}
}
// Wait a bit after the last message before checking the result
msgWaitMax := 5 * time.Second
msgTimer := time.NewTimer(msgWaitMax)
// Checks we received some message after the IDONTWANT is cleared
received := false
checkMsgs := func() {
if !received {
t.Fatalf("Expected some message after the IDONTWANT is cleared")
}
}
// Wait for the timer to expire
go func() {
select {
case <-msgTimer.C:
checkMsgs()
cancel()
return
case <-ctx.Done():
checkMsgs()
}
}()
newMockGS(ctx, t, hosts[2], func(writeMsg func(*pb.RPC), irpc *pb.RPC) {
// Check if it receives any message
if len(irpc.GetPublish()) > 0 {
received = true
}
// When the middle peer connects it will send us its subscriptions
for _, sub := range irpc.GetSubscriptions() {
if sub.GetSubscribe() {
// Reply by subcribing to the topic and grafting to the middle peer
writeMsg(&pb.RPC{
Subscriptions: []*pb.RPC_SubOpts{{Subscribe: sub.Subscribe, Topicid: sub.Topicid}},
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
})
go func() {
// Wait for a short interval to make sure the middle peer
// received and processed the subscribe + graft
time.Sleep(100 * time.Millisecond)
// Generate a message and send IDONTWANT to the middle peer
data := make([]byte, 16)
crand.Read(data)
mid := msgID(&pb.Message{Data: data})
writeMsg(&pb.RPC{
Control: &pb.ControlMessage{Idontwant: []*pb.ControlIDontWant{{MessageIDs: []string{mid}}}},
})
// Wait for a short interval to make sure the middle peer
// received and processed the IDONTWANTs
time.Sleep(100 * time.Millisecond)
// Wait for 4 heartbeats to make sure the IDONTWANT is cleared
time.Sleep(4 * time.Second)
// Publish the message from the first peer
if err := psubs[0].Publish(topic, data); err != nil {
t.Error(err)
return // cannot call t.Fatal in a non-test goroutine
}
}()
}
}
})
connect(t, hosts[0], hosts[1])
connect(t, hosts[1], hosts[2])
<-ctx.Done()
}

View File

@ -228,13 +228,14 @@ func (m *Message) GetKey() []byte {
}
type ControlMessage struct {
Ihave []*ControlIHave `protobuf:"bytes,1,rep,name=ihave" json:"ihave,omitempty"`
Iwant []*ControlIWant `protobuf:"bytes,2,rep,name=iwant" json:"iwant,omitempty"`
Graft []*ControlGraft `protobuf:"bytes,3,rep,name=graft" json:"graft,omitempty"`
Prune []*ControlPrune `protobuf:"bytes,4,rep,name=prune" json:"prune,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Ihave []*ControlIHave `protobuf:"bytes,1,rep,name=ihave" json:"ihave,omitempty"`
Iwant []*ControlIWant `protobuf:"bytes,2,rep,name=iwant" json:"iwant,omitempty"`
Graft []*ControlGraft `protobuf:"bytes,3,rep,name=graft" json:"graft,omitempty"`
Prune []*ControlPrune `protobuf:"bytes,4,rep,name=prune" json:"prune,omitempty"`
Idontwant []*ControlIDontWant `protobuf:"bytes,5,rep,name=idontwant" json:"idontwant,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ControlMessage) Reset() { *m = ControlMessage{} }
@ -298,6 +299,13 @@ func (m *ControlMessage) GetPrune() []*ControlPrune {
return nil
}
func (m *ControlMessage) GetIdontwant() []*ControlIDontWant {
if m != nil {
return m.Idontwant
}
return nil
}
type ControlIHave struct {
TopicID *string `protobuf:"bytes,1,opt,name=topicID" json:"topicID,omitempty"`
// implementors from other languages should use bytes here - go protobuf emits invalid utf8 strings
@ -512,6 +520,54 @@ func (m *ControlPrune) GetBackoff() uint64 {
return 0
}
type ControlIDontWant struct {
// implementors from other languages should use bytes here - go protobuf emits invalid utf8 strings
MessageIDs []string `protobuf:"bytes,1,rep,name=messageIDs" json:"messageIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ControlIDontWant) Reset() { *m = ControlIDontWant{} }
func (m *ControlIDontWant) String() string { return proto.CompactTextString(m) }
func (*ControlIDontWant) ProtoMessage() {}
func (*ControlIDontWant) Descriptor() ([]byte, []int) {
return fileDescriptor_77a6da22d6a3feb1, []int{7}
}
func (m *ControlIDontWant) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *ControlIDontWant) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_ControlIDontWant.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *ControlIDontWant) XXX_Merge(src proto.Message) {
xxx_messageInfo_ControlIDontWant.Merge(m, src)
}
func (m *ControlIDontWant) XXX_Size() int {
return m.Size()
}
func (m *ControlIDontWant) XXX_DiscardUnknown() {
xxx_messageInfo_ControlIDontWant.DiscardUnknown(m)
}
var xxx_messageInfo_ControlIDontWant proto.InternalMessageInfo
func (m *ControlIDontWant) GetMessageIDs() []string {
if m != nil {
return m.MessageIDs
}
return nil
}
type PeerInfo struct {
PeerID []byte `protobuf:"bytes,1,opt,name=peerID" json:"peerID,omitempty"`
SignedPeerRecord []byte `protobuf:"bytes,2,opt,name=signedPeerRecord" json:"signedPeerRecord,omitempty"`
@ -524,7 +580,7 @@ func (m *PeerInfo) Reset() { *m = PeerInfo{} }
func (m *PeerInfo) String() string { return proto.CompactTextString(m) }
func (*PeerInfo) ProtoMessage() {}
func (*PeerInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_77a6da22d6a3feb1, []int{7}
return fileDescriptor_77a6da22d6a3feb1, []int{8}
}
func (m *PeerInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -576,43 +632,46 @@ func init() {
proto.RegisterType((*ControlIWant)(nil), "pubsub.pb.ControlIWant")
proto.RegisterType((*ControlGraft)(nil), "pubsub.pb.ControlGraft")
proto.RegisterType((*ControlPrune)(nil), "pubsub.pb.ControlPrune")
proto.RegisterType((*ControlIDontWant)(nil), "pubsub.pb.ControlIDontWant")
proto.RegisterType((*PeerInfo)(nil), "pubsub.pb.PeerInfo")
}
func init() { proto.RegisterFile("rpc.proto", fileDescriptor_77a6da22d6a3feb1) }
var fileDescriptor_77a6da22d6a3feb1 = []byte{
// 480 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x92, 0xc1, 0x8e, 0xd3, 0x3c,
0x10, 0xc7, 0xe5, 0x6d, 0xbb, 0xd9, 0xcc, 0xe6, 0xfb, 0xb4, 0x32, 0x68, 0x31, 0x08, 0x55, 0x55,
0x4e, 0x01, 0x41, 0x0e, 0xcb, 0x95, 0x0b, 0xb4, 0x12, 0x9b, 0x03, 0x50, 0x99, 0x03, 0x67, 0x27,
0x75, 0xba, 0xd1, 0x6e, 0x63, 0x63, 0x3b, 0x8b, 0x78, 0x08, 0xde, 0x8b, 0x03, 0x07, 0x1e, 0x01,
0xf5, 0xc6, 0x5b, 0x20, 0x3b, 0x4e, 0x9a, 0xa5, 0x94, 0x9b, 0xe7, 0xef, 0xdf, 0xcc, 0xfc, 0x3d,
0x1e, 0x08, 0x95, 0x2c, 0x52, 0xa9, 0x84, 0x11, 0x38, 0x94, 0x4d, 0xae, 0x9b, 0x3c, 0x95, 0x79,
0xfc, 0x0b, 0xc1, 0x88, 0x2e, 0xe7, 0xf8, 0x25, 0xfc, 0xa7, 0x9b, 0x5c, 0x17, 0xaa, 0x92, 0xa6,
0x12, 0xb5, 0x26, 0x68, 0x36, 0x4a, 0x4e, 0x2f, 0xce, 0xd3, 0x1e, 0x4d, 0xe9, 0x72, 0x9e, 0x7e,
0x68, 0xf2, 0xf7, 0xd2, 0x68, 0x7a, 0x17, 0xc6, 0xcf, 0x20, 0x90, 0x4d, 0x7e, 0x53, 0xe9, 0x2b,
0x72, 0xe4, 0xf2, 0xf0, 0x20, 0xef, 0x2d, 0xd7, 0x9a, 0xad, 0x39, 0xed, 0x10, 0xfc, 0x02, 0x82,
0x42, 0xd4, 0x46, 0x89, 0x1b, 0x32, 0x9a, 0xa1, 0xe4, 0xf4, 0xe2, 0xe1, 0x80, 0x9e, 0xb7, 0x37,
0x7d, 0x92, 0x27, 0x1f, 0xbd, 0x82, 0xc0, 0x37, 0xc7, 0x8f, 0x21, 0xf4, 0xed, 0x73, 0x4e, 0xd0,
0x0c, 0x25, 0x27, 0x74, 0x27, 0x60, 0x02, 0x81, 0x11, 0xb2, 0x2a, 0xaa, 0x15, 0x39, 0x9a, 0xa1,
0x24, 0xa4, 0x5d, 0x18, 0x7f, 0x45, 0x10, 0xf8, 0xba, 0x18, 0xc3, 0xb8, 0x54, 0x62, 0xe3, 0xd2,
0x23, 0xea, 0xce, 0x56, 0x5b, 0x31, 0xc3, 0x5c, 0x5a, 0x44, 0xdd, 0x19, 0xdf, 0x87, 0x89, 0xe6,
0x9f, 0x6a, 0xe1, 0x9c, 0x46, 0xb4, 0x0d, 0xac, 0xea, 0x8a, 0x92, 0xb1, 0xeb, 0xd0, 0x06, 0xce,
0x57, 0xb5, 0xae, 0x99, 0x69, 0x14, 0x27, 0x13, 0xc7, 0xef, 0x04, 0x7c, 0x06, 0xa3, 0x6b, 0xfe,
0x85, 0x1c, 0x3b, 0xdd, 0x1e, 0xe3, 0xef, 0x08, 0xfe, 0xbf, 0xfb, 0x5c, 0xfc, 0x1c, 0x26, 0xd5,
0x15, 0xbb, 0xe5, 0x7e, 0xfc, 0x0f, 0xf6, 0x07, 0x93, 0x5d, 0xb2, 0x5b, 0x4e, 0x5b, 0xca, 0xe1,
0x9f, 0x59, 0x6d, 0xfc, 0xd4, 0xff, 0x86, 0x7f, 0x64, 0xb5, 0xa1, 0x2d, 0x65, 0xf1, 0xb5, 0x62,
0xa5, 0x21, 0xa3, 0x43, 0xf8, 0x1b, 0x7b, 0x4d, 0x5b, 0xca, 0xe2, 0x52, 0x35, 0x35, 0x27, 0xe3,
0x43, 0xf8, 0xd2, 0x5e, 0xd3, 0x96, 0x8a, 0x2f, 0x21, 0x1a, 0x7a, 0xec, 0x3f, 0x22, 0x5b, 0xb8,
0x29, 0x77, 0x1f, 0x91, 0x2d, 0xf0, 0x14, 0x60, 0xd3, 0x3e, 0x38, 0x5b, 0x68, 0xe7, 0x3d, 0xa4,
0x03, 0x25, 0x4e, 0x77, 0x95, 0xac, 0xfd, 0x3f, 0x78, 0xb4, 0xc7, 0x27, 0x3d, 0xef, 0xfc, 0x1f,
0xee, 0x1c, 0x6f, 0x7a, 0xd2, 0x59, 0xff, 0x87, 0xc7, 0x27, 0x30, 0x91, 0x9c, 0x2b, 0xed, 0x47,
0x7b, 0x6f, 0xf0, 0xf8, 0x25, 0xe7, 0x2a, 0xab, 0x4b, 0x41, 0x5b, 0xc2, 0x16, 0xc9, 0x59, 0x71,
0x2d, 0xca, 0xd2, 0x6d, 0xc9, 0x98, 0x76, 0x61, 0xfc, 0x0e, 0x4e, 0x3a, 0x18, 0x9f, 0xc3, 0xb1,
0xc5, 0x7d, 0xa7, 0x88, 0xfa, 0x08, 0x3f, 0x85, 0x33, 0xbb, 0x24, 0x7c, 0x65, 0x49, 0xca, 0x0b,
0xa1, 0x56, 0x7e, 0x03, 0xf7, 0xf4, 0xd7, 0xd1, 0xb7, 0xed, 0x14, 0xfd, 0xd8, 0x4e, 0xd1, 0xcf,
0xed, 0x14, 0xfd, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xb2, 0xf8, 0xc4, 0x6e, 0xd2, 0x03, 0x00, 0x00,
// 511 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x92, 0xcd, 0x6e, 0x13, 0x31,
0x10, 0xc7, 0xe5, 0x7c, 0x34, 0xdd, 0xe9, 0x82, 0x22, 0x83, 0x8a, 0xf9, 0x50, 0x14, 0xed, 0x29,
0x20, 0xd8, 0x43, 0x38, 0x21, 0x71, 0x81, 0x44, 0xa2, 0x39, 0x00, 0x91, 0x39, 0x70, 0xde, 0xdd,
0x38, 0xe9, 0xaa, 0x8d, 0x6d, 0x6c, 0x6f, 0x11, 0x4f, 0xc0, 0x89, 0xf7, 0xe2, 0xc8, 0x23, 0xa0,
0xdc, 0x78, 0x0b, 0xe4, 0x59, 0xe7, 0xa3, 0x4d, 0x03, 0x37, 0xcf, 0xf8, 0x37, 0xfe, 0xff, 0x67,
0xc6, 0x10, 0x19, 0x5d, 0xa4, 0xda, 0x28, 0xa7, 0x68, 0xa4, 0xab, 0xdc, 0x56, 0x79, 0xaa, 0xf3,
0xe4, 0x0f, 0x81, 0x26, 0x9f, 0x8e, 0xe8, 0x6b, 0xb8, 0x63, 0xab, 0xdc, 0x16, 0xa6, 0xd4, 0xae,
0x54, 0xd2, 0x32, 0xd2, 0x6f, 0x0e, 0x4e, 0x86, 0xa7, 0xe9, 0x06, 0x4d, 0xf9, 0x74, 0x94, 0x7e,
0xaa, 0xf2, 0x8f, 0xda, 0x59, 0x7e, 0x1d, 0xa6, 0xcf, 0xa1, 0xa3, 0xab, 0xfc, 0xb2, 0xb4, 0xe7,
0xac, 0x81, 0x75, 0x74, 0xa7, 0xee, 0xbd, 0xb0, 0x36, 0x5b, 0x08, 0xbe, 0x46, 0xe8, 0x4b, 0xe8,
0x14, 0x4a, 0x3a, 0xa3, 0x2e, 0x59, 0xb3, 0x4f, 0x06, 0x27, 0xc3, 0x87, 0x3b, 0xf4, 0xa8, 0xbe,
0xd9, 0x14, 0x05, 0xf2, 0xd1, 0x1b, 0xe8, 0x04, 0x71, 0xfa, 0x04, 0xa2, 0x20, 0x9f, 0x0b, 0x46,
0xfa, 0x64, 0x70, 0xcc, 0xb7, 0x09, 0xca, 0xa0, 0xe3, 0x94, 0x2e, 0x8b, 0x72, 0xc6, 0x1a, 0x7d,
0x32, 0x88, 0xf8, 0x3a, 0x4c, 0x7e, 0x10, 0xe8, 0x84, 0x77, 0x29, 0x85, 0xd6, 0xdc, 0xa8, 0x25,
0x96, 0xc7, 0x1c, 0xcf, 0x3e, 0x37, 0xcb, 0x5c, 0x86, 0x65, 0x31, 0xc7, 0x33, 0xbd, 0x0f, 0x6d,
0x2b, 0xbe, 0x48, 0x85, 0x4e, 0x63, 0x5e, 0x07, 0x3e, 0x8b, 0x8f, 0xb2, 0x16, 0x2a, 0xd4, 0x01,
0xfa, 0x2a, 0x17, 0x32, 0x73, 0x95, 0x11, 0xac, 0x8d, 0xfc, 0x36, 0x41, 0xbb, 0xd0, 0xbc, 0x10,
0xdf, 0xd8, 0x11, 0xe6, 0xfd, 0x31, 0xf9, 0xde, 0x80, 0xbb, 0xd7, 0xdb, 0xa5, 0x2f, 0xa0, 0x5d,
0x9e, 0x67, 0x57, 0x22, 0x8c, 0xff, 0xc1, 0xfe, 0x60, 0x26, 0x67, 0xd9, 0x95, 0xe0, 0x35, 0x85,
0xf8, 0xd7, 0x4c, 0xba, 0x30, 0xf5, 0xdb, 0xf0, 0xcf, 0x99, 0x74, 0xbc, 0xa6, 0x3c, 0xbe, 0x30,
0xd9, 0xdc, 0xb1, 0xe6, 0x21, 0xfc, 0x9d, 0xbf, 0xe6, 0x35, 0xe5, 0x71, 0x6d, 0x2a, 0x29, 0x58,
0xeb, 0x10, 0x3e, 0xf5, 0xd7, 0xbc, 0xa6, 0xe8, 0x2b, 0x88, 0xca, 0x99, 0x92, 0x0e, 0x0d, 0xb5,
0xb1, 0xe4, 0xf1, 0x2d, 0x86, 0xc6, 0x4a, 0x3a, 0x34, 0xb5, 0xa5, 0x93, 0x33, 0x88, 0x77, 0xdb,
0xdb, 0xec, 0x70, 0x32, 0xc6, 0x05, 0xad, 0x77, 0x38, 0x19, 0xd3, 0x1e, 0xc0, 0xb2, 0x9e, 0xd5,
0x64, 0x6c, 0xb1, 0xed, 0x88, 0xef, 0x64, 0x92, 0x74, 0xfb, 0x92, 0x17, 0xb9, 0xc1, 0x93, 0x3d,
0x7e, 0xb0, 0xe1, 0xb1, 0xf5, 0xc3, 0xca, 0xc9, 0x72, 0x43, 0x62, 0xd7, 0xff, 0xf0, 0xf8, 0x14,
0xda, 0x5a, 0x08, 0x63, 0xc3, 0x56, 0xee, 0xed, 0x0c, 0x61, 0x2a, 0x84, 0x99, 0xc8, 0xb9, 0xe2,
0x35, 0xe1, 0x1f, 0xc9, 0xb3, 0xe2, 0x42, 0xcd, 0xe7, 0xf8, 0xc1, 0x5a, 0x7c, 0x1d, 0x26, 0x43,
0xe8, 0xde, 0x9c, 0xd8, 0x7f, 0x9b, 0xf9, 0x00, 0xc7, 0x6b, 0x01, 0x7a, 0x0a, 0x47, 0x5e, 0x22,
0xb8, 0x8b, 0x79, 0x88, 0xe8, 0x33, 0xe8, 0xfa, 0x3f, 0x29, 0x66, 0x9e, 0xe4, 0xa2, 0x50, 0x66,
0x16, 0x3e, 0xfc, 0x5e, 0xfe, 0x6d, 0xfc, 0x73, 0xd5, 0x23, 0xbf, 0x56, 0x3d, 0xf2, 0x7b, 0xd5,
0x23, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xba, 0x73, 0x8e, 0xbf, 0x41, 0x04, 0x00, 0x00,
}
func (m *RPC) Marshal() (dAtA []byte, err error) {
@ -819,6 +878,20 @@ func (m *ControlMessage) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i -= len(m.XXX_unrecognized)
copy(dAtA[i:], m.XXX_unrecognized)
}
if len(m.Idontwant) > 0 {
for iNdEx := len(m.Idontwant) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Idontwant[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintRpc(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x2a
}
}
if len(m.Prune) > 0 {
for iNdEx := len(m.Prune) - 1; iNdEx >= 0; iNdEx-- {
{
@ -1044,6 +1117,42 @@ func (m *ControlPrune) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *ControlIDontWant) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *ControlIDontWant) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *ControlIDontWant) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if m.XXX_unrecognized != nil {
i -= len(m.XXX_unrecognized)
copy(dAtA[i:], m.XXX_unrecognized)
}
if len(m.MessageIDs) > 0 {
for iNdEx := len(m.MessageIDs) - 1; iNdEx >= 0; iNdEx-- {
i -= len(m.MessageIDs[iNdEx])
copy(dAtA[i:], m.MessageIDs[iNdEx])
i = encodeVarintRpc(dAtA, i, uint64(len(m.MessageIDs[iNdEx])))
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
}
func (m *PeerInfo) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -1209,6 +1318,12 @@ func (m *ControlMessage) Size() (n int) {
n += 1 + l + sovRpc(uint64(l))
}
}
if len(m.Idontwant) > 0 {
for _, e := range m.Idontwant {
l = e.Size()
n += 1 + l + sovRpc(uint64(l))
}
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
@ -1296,6 +1411,24 @@ func (m *ControlPrune) Size() (n int) {
return n
}
func (m *ControlIDontWant) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.MessageIDs) > 0 {
for _, s := range m.MessageIDs {
l = len(s)
n += 1 + l + sovRpc(uint64(l))
}
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func (m *PeerInfo) Size() (n int) {
if m == nil {
return 0
@ -2001,6 +2134,40 @@ func (m *ControlMessage) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
case 5:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Idontwant", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowRpc
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthRpc
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthRpc
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Idontwant = append(m.Idontwant, &ControlIDontWant{})
if err := m.Idontwant[len(m.Idontwant)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipRpc(dAtA[iNdEx:])
@ -2444,6 +2611,89 @@ func (m *ControlPrune) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *ControlIDontWant) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowRpc
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: ControlIDontWant: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: ControlIDontWant: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field MessageIDs", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowRpc
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthRpc
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthRpc
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.MessageIDs = append(m.MessageIDs, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipRpc(dAtA[iNdEx:])
if err != nil {
return err
}
if (skippy < 0) || (iNdEx+skippy) < 0 {
return ErrInvalidLengthRpc
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *PeerInfo) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0

View File

@ -28,6 +28,7 @@ message ControlMessage {
repeated ControlIWant iwant = 2;
repeated ControlGraft graft = 3;
repeated ControlPrune prune = 4;
repeated ControlIDontWant idontwant = 5;
}
message ControlIHave {
@ -51,7 +52,12 @@ message ControlPrune {
optional uint64 backoff = 3;
}
message ControlIDontWant {
// implementors from other languages should use bytes here - go protobuf emits invalid utf8 strings
repeated string messageIDs = 1;
}
message PeerInfo {
optional bytes peerID = 1;
optional bytes signedPeerRecord = 2;
}
}

View File

@ -1159,13 +1159,14 @@ func (m *TraceEvent_SubMeta) GetTopic() string {
}
type TraceEvent_ControlMeta struct {
Ihave []*TraceEvent_ControlIHaveMeta `protobuf:"bytes,1,rep,name=ihave" json:"ihave,omitempty"`
Iwant []*TraceEvent_ControlIWantMeta `protobuf:"bytes,2,rep,name=iwant" json:"iwant,omitempty"`
Graft []*TraceEvent_ControlGraftMeta `protobuf:"bytes,3,rep,name=graft" json:"graft,omitempty"`
Prune []*TraceEvent_ControlPruneMeta `protobuf:"bytes,4,rep,name=prune" json:"prune,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Ihave []*TraceEvent_ControlIHaveMeta `protobuf:"bytes,1,rep,name=ihave" json:"ihave,omitempty"`
Iwant []*TraceEvent_ControlIWantMeta `protobuf:"bytes,2,rep,name=iwant" json:"iwant,omitempty"`
Graft []*TraceEvent_ControlGraftMeta `protobuf:"bytes,3,rep,name=graft" json:"graft,omitempty"`
Prune []*TraceEvent_ControlPruneMeta `protobuf:"bytes,4,rep,name=prune" json:"prune,omitempty"`
Idontwant []*TraceEvent_ControlIDontWantMeta `protobuf:"bytes,5,rep,name=idontwant" json:"idontwant,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *TraceEvent_ControlMeta) Reset() { *m = TraceEvent_ControlMeta{} }
@ -1229,6 +1230,13 @@ func (m *TraceEvent_ControlMeta) GetPrune() []*TraceEvent_ControlPruneMeta {
return nil
}
func (m *TraceEvent_ControlMeta) GetIdontwant() []*TraceEvent_ControlIDontWantMeta {
if m != nil {
return m.Idontwant
}
return nil
}
type TraceEvent_ControlIHaveMeta struct {
Topic *string `protobuf:"bytes,1,opt,name=topic" json:"topic,omitempty"`
MessageIDs [][]byte `protobuf:"bytes,2,rep,name=messageIDs" json:"messageIDs,omitempty"`
@ -1433,6 +1441,53 @@ func (m *TraceEvent_ControlPruneMeta) GetPeers() [][]byte {
return nil
}
type TraceEvent_ControlIDontWantMeta struct {
MessageIDs [][]byte `protobuf:"bytes,1,rep,name=messageIDs" json:"messageIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *TraceEvent_ControlIDontWantMeta) Reset() { *m = TraceEvent_ControlIDontWantMeta{} }
func (m *TraceEvent_ControlIDontWantMeta) String() string { return proto.CompactTextString(m) }
func (*TraceEvent_ControlIDontWantMeta) ProtoMessage() {}
func (*TraceEvent_ControlIDontWantMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_0571941a1d628a80, []int{0, 21}
}
func (m *TraceEvent_ControlIDontWantMeta) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *TraceEvent_ControlIDontWantMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_TraceEvent_ControlIDontWantMeta.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *TraceEvent_ControlIDontWantMeta) XXX_Merge(src proto.Message) {
xxx_messageInfo_TraceEvent_ControlIDontWantMeta.Merge(m, src)
}
func (m *TraceEvent_ControlIDontWantMeta) XXX_Size() int {
return m.Size()
}
func (m *TraceEvent_ControlIDontWantMeta) XXX_DiscardUnknown() {
xxx_messageInfo_TraceEvent_ControlIDontWantMeta.DiscardUnknown(m)
}
var xxx_messageInfo_TraceEvent_ControlIDontWantMeta proto.InternalMessageInfo
func (m *TraceEvent_ControlIDontWantMeta) GetMessageIDs() [][]byte {
if m != nil {
return m.MessageIDs
}
return nil
}
type TraceEventBatch struct {
Batch []*TraceEvent `protobuf:"bytes,1,rep,name=batch" json:"batch,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -1504,76 +1559,79 @@ func init() {
proto.RegisterType((*TraceEvent_ControlIWantMeta)(nil), "pubsub.pb.TraceEvent.ControlIWantMeta")
proto.RegisterType((*TraceEvent_ControlGraftMeta)(nil), "pubsub.pb.TraceEvent.ControlGraftMeta")
proto.RegisterType((*TraceEvent_ControlPruneMeta)(nil), "pubsub.pb.TraceEvent.ControlPruneMeta")
proto.RegisterType((*TraceEvent_ControlIDontWantMeta)(nil), "pubsub.pb.TraceEvent.ControlIDontWantMeta")
proto.RegisterType((*TraceEventBatch)(nil), "pubsub.pb.TraceEventBatch")
}
func init() { proto.RegisterFile("trace.proto", fileDescriptor_0571941a1d628a80) }
var fileDescriptor_0571941a1d628a80 = []byte{
// 999 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0x51, 0x6f, 0xda, 0x56,
0x14, 0xc7, 0xe7, 0x00, 0x01, 0x0e, 0x84, 0x78, 0x77, 0x6d, 0x65, 0xb1, 0x36, 0x62, 0x59, 0x55,
0x21, 0x4d, 0x42, 0x6a, 0xa4, 0xa9, 0x0f, 0x6b, 0xab, 0x11, 0xec, 0x26, 0x44, 0x24, 0xb1, 0x0e,
0x24, 0x7b, 0xcc, 0x0c, 0xdc, 0x35, 0x8e, 0xc0, 0xb6, 0xec, 0x0b, 0x53, 0x9f, 0xf6, 0xb4, 0xef,
0xd6, 0xb7, 0xed, 0x23, 0x54, 0xf9, 0x24, 0xd3, 0xbd, 0xd7, 0x36, 0x36, 0xd8, 0xb4, 0x8b, 0xfa,
0xe6, 0x73, 0xf3, 0xff, 0x9d, 0x7b, 0xce, 0xbd, 0xe7, 0x7f, 0x03, 0xd4, 0x98, 0x6f, 0x4d, 0x68,
0xc7, 0xf3, 0x5d, 0xe6, 0x92, 0xaa, 0xb7, 0x18, 0x07, 0x8b, 0x71, 0xc7, 0x1b, 0x1f, 0x7e, 0x7a,
0x02, 0x30, 0xe2, 0x7f, 0x32, 0x96, 0xd4, 0x61, 0xa4, 0x03, 0x45, 0xf6, 0xc1, 0xa3, 0x9a, 0xd2,
0x52, 0xda, 0x8d, 0xa3, 0x66, 0x27, 0x16, 0x76, 0x56, 0xa2, 0xce, 0xe8, 0x83, 0x47, 0x51, 0xe8,
0xc8, 0x13, 0xd8, 0xf5, 0x28, 0xf5, 0xfb, 0xba, 0xb6, 0xd3, 0x52, 0xda, 0x75, 0x0c, 0x23, 0xf2,
0x14, 0xaa, 0xcc, 0x9e, 0xd3, 0x80, 0x59, 0x73, 0x4f, 0x2b, 0xb4, 0x94, 0x76, 0x01, 0x57, 0x0b,
0x64, 0x00, 0x0d, 0x6f, 0x31, 0x9e, 0xd9, 0xc1, 0xed, 0x39, 0x0d, 0x02, 0xeb, 0x3d, 0xd5, 0x8a,
0x2d, 0xa5, 0x5d, 0x3b, 0x7a, 0x9e, 0xbd, 0x9f, 0x99, 0xd2, 0xe2, 0x1a, 0x4b, 0xfa, 0xb0, 0xe7,
0xd3, 0x3b, 0x3a, 0x61, 0x51, 0xb2, 0x92, 0x48, 0xf6, 0x63, 0x76, 0x32, 0x4c, 0x4a, 0x31, 0x4d,
0x12, 0x04, 0x75, 0xba, 0xf0, 0x66, 0xf6, 0xc4, 0x62, 0x34, 0xca, 0xb6, 0x2b, 0xb2, 0xbd, 0xc8,
0xce, 0xa6, 0xaf, 0xa9, 0x71, 0x83, 0xe7, 0xcd, 0x4e, 0xe9, 0xcc, 0x5e, 0x52, 0x3f, 0xca, 0x58,
0xde, 0xd6, 0xac, 0x9e, 0xd2, 0xe2, 0x1a, 0x4b, 0x5e, 0x41, 0xd9, 0x9a, 0x4e, 0x4d, 0x4a, 0x7d,
0xad, 0x22, 0xd2, 0x3c, 0xcb, 0x4e, 0xd3, 0x95, 0x22, 0x8c, 0xd4, 0xe4, 0x57, 0x00, 0x9f, 0xce,
0xdd, 0x25, 0x15, 0x6c, 0x55, 0xb0, 0xad, 0xbc, 0x23, 0x8a, 0x74, 0x98, 0x60, 0xf8, 0xd6, 0x3e,
0x9d, 0x2c, 0xd1, 0xec, 0x69, 0xb0, 0x6d, 0x6b, 0x94, 0x22, 0x8c, 0xd4, 0x1c, 0x0c, 0xa8, 0x33,
0xe5, 0x60, 0x6d, 0x1b, 0x38, 0x94, 0x22, 0x8c, 0xd4, 0x1c, 0x9c, 0xfa, 0xae, 0xc7, 0xc1, 0xfa,
0x36, 0x50, 0x97, 0x22, 0x8c, 0xd4, 0x7c, 0x8c, 0xef, 0x5c, 0xdb, 0xd1, 0xf6, 0x04, 0x95, 0x33,
0xc6, 0x67, 0xae, 0xed, 0xa0, 0xd0, 0x91, 0x97, 0x50, 0x9a, 0x51, 0x6b, 0x49, 0xb5, 0x86, 0x00,
0xbe, 0xcf, 0x06, 0x06, 0x5c, 0x82, 0x52, 0xc9, 0x91, 0xf7, 0xbe, 0xf5, 0x07, 0xd3, 0xf6, 0xb7,
0x21, 0x27, 0x5c, 0x82, 0x52, 0xc9, 0x11, 0xcf, 0x5f, 0x38, 0x54, 0x53, 0xb7, 0x21, 0x26, 0x97,
0xa0, 0x54, 0x36, 0x75, 0x68, 0xa4, 0xa7, 0x9f, 0x3b, 0x6b, 0x2e, 0x3f, 0xfb, 0xba, 0xb0, 0x69,
0x1d, 0x57, 0x0b, 0xe4, 0x11, 0x94, 0x98, 0xeb, 0xd9, 0x13, 0x61, 0xc7, 0x2a, 0xca, 0xa0, 0xf9,
0x17, 0xec, 0xa5, 0xc6, 0xfe, 0x33, 0x49, 0x0e, 0xa1, 0xee, 0xd3, 0x09, 0xb5, 0x97, 0x74, 0xfa,
0xce, 0x77, 0xe7, 0xa1, 0xb5, 0x53, 0x6b, 0xdc, 0xf8, 0x3e, 0xb5, 0x02, 0xd7, 0x11, 0xee, 0xae,
0x62, 0x18, 0xad, 0x0a, 0x28, 0x26, 0x0b, 0xb8, 0x03, 0x75, 0xdd, 0x29, 0x5f, 0xa1, 0x86, 0x78,
0xaf, 0x42, 0x72, 0xaf, 0x5b, 0x68, 0xa4, 0x3d, 0xf4, 0x90, 0x23, 0xdb, 0xd8, 0xbf, 0xb0, 0xb9,
0x7f, 0xf3, 0x15, 0x94, 0x43, 0x9b, 0x25, 0xde, 0x41, 0x25, 0xf5, 0x0e, 0x3e, 0xe2, 0x57, 0xee,
0x32, 0x37, 0x4a, 0x2e, 0x82, 0xe6, 0x73, 0x80, 0x95, 0xc7, 0xf2, 0xd8, 0xe6, 0xef, 0x50, 0x0e,
0xad, 0xb4, 0x51, 0x8d, 0x92, 0x71, 0x1a, 0x2f, 0xa1, 0x38, 0xa7, 0xcc, 0x12, 0x3b, 0xe5, 0x7b,
0xd3, 0xec, 0x9d, 0x53, 0x66, 0xa1, 0x90, 0x36, 0x47, 0x50, 0x0e, 0x3d, 0xc7, 0x8b, 0xe0, 0xae,
0x1b, 0xb9, 0x51, 0x11, 0x32, 0x7a, 0x60, 0xd6, 0xd0, 0x90, 0x5f, 0x33, 0xeb, 0x53, 0x28, 0x72,
0xc3, 0xae, 0xae, 0x4b, 0x49, 0x5e, 0xfa, 0x33, 0x28, 0x09, 0x77, 0xe6, 0x18, 0xe0, 0x67, 0x28,
0x09, 0x27, 0x6e, 0xbb, 0xa7, 0x6c, 0x4c, 0xb8, 0xf1, 0x7f, 0x62, 0x1f, 0x15, 0x28, 0x87, 0xc5,
0x93, 0x37, 0x50, 0x09, 0x47, 0x2d, 0xd0, 0x94, 0x56, 0xa1, 0x5d, 0x3b, 0xfa, 0x21, 0xbb, 0xdb,
0x70, 0x58, 0x45, 0xc7, 0x31, 0x42, 0xba, 0x50, 0x0f, 0x16, 0xe3, 0x60, 0xe2, 0xdb, 0x1e, 0xb3,
0x5d, 0x47, 0xdb, 0x11, 0x29, 0xf2, 0xde, 0xcf, 0xc5, 0x58, 0xe0, 0x29, 0x84, 0xfc, 0x02, 0xe5,
0x89, 0xeb, 0x30, 0xdf, 0x9d, 0x89, 0x21, 0xce, 0x2d, 0xa0, 0x27, 0x45, 0x22, 0x43, 0x44, 0x34,
0xbb, 0x50, 0x4b, 0x14, 0xf6, 0xa0, 0xc7, 0xe7, 0x0d, 0x94, 0xc3, 0xc2, 0x38, 0x1e, 0x96, 0x36,
0x96, 0x3f, 0x31, 0x2a, 0xb8, 0x5a, 0xc8, 0xc1, 0xff, 0xde, 0x81, 0x5a, 0xa2, 0x34, 0xf2, 0x1a,
0x4a, 0xf6, 0x2d, 0x7f, 0xaa, 0xe5, 0x69, 0xbe, 0xd8, 0xda, 0x4c, 0xff, 0xd4, 0x5a, 0xca, 0x23,
0x95, 0x90, 0xa0, 0xff, 0xb4, 0x1c, 0x16, 0x1e, 0xe4, 0x67, 0xe8, 0xdf, 0x2c, 0x87, 0x85, 0x34,
0x87, 0x38, 0x2d, 0xdf, 0xfc, 0xc2, 0x17, 0xd0, 0x62, 0xe0, 0x24, 0x2d, 0x9f, 0xff, 0xd7, 0xd1,
0xf3, 0x5f, 0xfc, 0x02, 0x5a, 0xcc, 0x9d, 0xa4, 0xe5, 0x7f, 0x82, 0x53, 0x50, 0xd7, 0x9b, 0xca,
0xf6, 0x02, 0x39, 0x00, 0x88, 0xef, 0x24, 0x10, 0x8d, 0xd6, 0x31, 0xb1, 0xd2, 0x3c, 0x5a, 0x65,
0x8a, 0x1a, 0x5c, 0x63, 0x94, 0x0d, 0xa6, 0x1d, 0x33, 0x71, 0x5b, 0x39, 0x4e, 0x7c, 0x1b, 0x2b,
0xe3, 0x16, 0x72, 0xea, 0xe4, 0x6f, 0x23, 0xa5, 0x7e, 0x54, 0xa2, 0x0c, 0x0e, 0xff, 0x51, 0xa0,
0xc8, 0x7f, 0x60, 0x92, 0xef, 0x60, 0xdf, 0xbc, 0x3a, 0x1e, 0xf4, 0x87, 0xa7, 0x37, 0xe7, 0xc6,
0x70, 0xd8, 0x3d, 0x31, 0xd4, 0x6f, 0x08, 0x81, 0x06, 0x1a, 0x67, 0x46, 0x6f, 0x14, 0xaf, 0x29,
0xe4, 0x31, 0x7c, 0xab, 0x5f, 0x99, 0x83, 0x7e, 0xaf, 0x3b, 0x32, 0xe2, 0xe5, 0x1d, 0xce, 0xeb,
0xc6, 0xa0, 0x7f, 0x6d, 0x60, 0xbc, 0x58, 0x20, 0x75, 0xa8, 0x74, 0x75, 0xfd, 0xc6, 0x34, 0x0c,
0x54, 0x8b, 0x64, 0x1f, 0x6a, 0x68, 0x9c, 0x5f, 0x5e, 0x1b, 0x72, 0xa1, 0xc4, 0xff, 0x8c, 0x46,
0xef, 0xfa, 0x06, 0xcd, 0x9e, 0xba, 0xcb, 0xa3, 0xa1, 0x71, 0xa1, 0x8b, 0xa8, 0xcc, 0x23, 0x1d,
0x2f, 0x4d, 0x11, 0x55, 0x48, 0x05, 0x8a, 0x67, 0x97, 0xfd, 0x0b, 0xb5, 0x4a, 0xaa, 0x50, 0x1a,
0x18, 0xdd, 0x6b, 0x43, 0x05, 0xfe, 0x79, 0x82, 0xdd, 0x77, 0x23, 0xb5, 0xc6, 0x3f, 0x4d, 0xbc,
0xba, 0x30, 0xd4, 0xfa, 0xe1, 0x5b, 0xd8, 0x5f, 0xdd, 0xef, 0xb1, 0xc5, 0x26, 0xb7, 0xe4, 0x27,
0x28, 0x8d, 0xf9, 0x47, 0x38, 0xc4, 0x8f, 0x33, 0x47, 0x01, 0xa5, 0xe6, 0xb8, 0xfe, 0xf1, 0xfe,
0x40, 0xf9, 0xf7, 0xfe, 0x40, 0xf9, 0x74, 0x7f, 0xa0, 0xfc, 0x17, 0x00, 0x00, 0xff, 0xff, 0xdb,
0x3a, 0x1c, 0xe4, 0xc9, 0x0b, 0x00, 0x00,
// 1027 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xdf, 0x6e, 0xe2, 0x46,
0x14, 0xc6, 0xeb, 0x80, 0x03, 0x1c, 0x08, 0x71, 0xa7, 0xd9, 0xd6, 0x72, 0x77, 0x23, 0x9a, 0xae,
0x56, 0xa8, 0x95, 0x90, 0x36, 0x52, 0xbb, 0x17, 0xdd, 0x5d, 0x95, 0x60, 0x6f, 0x42, 0x44, 0x12,
0x6b, 0x20, 0xe9, 0x65, 0x6a, 0x60, 0xba, 0x71, 0x04, 0xb6, 0x65, 0x0f, 0x54, 0x7b, 0xd5, 0xd7,
0xdb, 0xbb, 0xed, 0x23, 0x54, 0x79, 0x92, 0x6a, 0x66, 0xfc, 0x07, 0x83, 0xed, 0xec, 0x46, 0xb9,
0xf3, 0x19, 0xbe, 0xdf, 0x99, 0x33, 0x67, 0xce, 0x37, 0x02, 0xea, 0xd4, 0xb7, 0x26, 0xa4, 0xe3,
0xf9, 0x2e, 0x75, 0x51, 0xcd, 0x5b, 0x8c, 0x83, 0xc5, 0xb8, 0xe3, 0x8d, 0x0f, 0xee, 0xbe, 0x03,
0x18, 0xb1, 0x9f, 0x8c, 0x25, 0x71, 0x28, 0xea, 0x40, 0x99, 0x7e, 0xf0, 0x88, 0x2a, 0xb5, 0xa4,
0x76, 0xf3, 0x50, 0xeb, 0xc4, 0xc2, 0x4e, 0x22, 0xea, 0x8c, 0x3e, 0x78, 0x04, 0x73, 0x1d, 0xfa,
0x16, 0xb6, 0x3d, 0x42, 0xfc, 0xbe, 0xae, 0x6e, 0xb5, 0xa4, 0x76, 0x03, 0x87, 0x11, 0x7a, 0x0a,
0x35, 0x6a, 0xcf, 0x49, 0x40, 0xad, 0xb9, 0xa7, 0x96, 0x5a, 0x52, 0xbb, 0x84, 0x93, 0x05, 0x34,
0x80, 0xa6, 0xb7, 0x18, 0xcf, 0xec, 0xe0, 0xe6, 0x8c, 0x04, 0x81, 0xf5, 0x9e, 0xa8, 0xe5, 0x96,
0xd4, 0xae, 0x1f, 0x3e, 0xcf, 0xde, 0xcf, 0x4c, 0x69, 0xf1, 0x1a, 0x8b, 0xfa, 0xb0, 0xe3, 0x93,
0x5b, 0x32, 0xa1, 0x51, 0x32, 0x99, 0x27, 0xfb, 0x31, 0x3b, 0x19, 0x5e, 0x95, 0xe2, 0x34, 0x89,
0x30, 0x28, 0xd3, 0x85, 0x37, 0xb3, 0x27, 0x16, 0x25, 0x51, 0xb6, 0x6d, 0x9e, 0xed, 0x45, 0x76,
0x36, 0x7d, 0x4d, 0x8d, 0x37, 0x78, 0x76, 0xd8, 0x29, 0x99, 0xd9, 0x4b, 0xe2, 0x47, 0x19, 0x2b,
0x45, 0x87, 0xd5, 0x53, 0x5a, 0xbc, 0xc6, 0xa2, 0x57, 0x50, 0xb1, 0xa6, 0x53, 0x93, 0x10, 0x5f,
0xad, 0xf2, 0x34, 0xcf, 0xb2, 0xd3, 0x74, 0x85, 0x08, 0x47, 0x6a, 0xf4, 0x3b, 0x80, 0x4f, 0xe6,
0xee, 0x92, 0x70, 0xb6, 0xc6, 0xd9, 0x56, 0x5e, 0x8b, 0x22, 0x1d, 0x5e, 0x61, 0xd8, 0xd6, 0x3e,
0x99, 0x2c, 0xb1, 0xd9, 0x53, 0xa1, 0x68, 0x6b, 0x2c, 0x44, 0x38, 0x52, 0x33, 0x30, 0x20, 0xce,
0x94, 0x81, 0xf5, 0x22, 0x70, 0x28, 0x44, 0x38, 0x52, 0x33, 0x70, 0xea, 0xbb, 0x1e, 0x03, 0x1b,
0x45, 0xa0, 0x2e, 0x44, 0x38, 0x52, 0xb3, 0x31, 0xbe, 0x75, 0x6d, 0x47, 0xdd, 0xe1, 0x54, 0xce,
0x18, 0x9f, 0xba, 0xb6, 0x83, 0xb9, 0x0e, 0xbd, 0x04, 0x79, 0x46, 0xac, 0x25, 0x51, 0x9b, 0x1c,
0xf8, 0x3e, 0x1b, 0x18, 0x30, 0x09, 0x16, 0x4a, 0x86, 0xbc, 0xf7, 0xad, 0xbf, 0xa8, 0xba, 0x5b,
0x84, 0x1c, 0x33, 0x09, 0x16, 0x4a, 0x86, 0x78, 0xfe, 0xc2, 0x21, 0xaa, 0x52, 0x84, 0x98, 0x4c,
0x82, 0x85, 0x52, 0xd3, 0xa1, 0x99, 0x9e, 0x7e, 0xe6, 0xac, 0xb9, 0xf8, 0xec, 0xeb, 0xdc, 0xa6,
0x0d, 0x9c, 0x2c, 0xa0, 0x3d, 0x90, 0xa9, 0xeb, 0xd9, 0x13, 0x6e, 0xc7, 0x1a, 0x16, 0x81, 0xf6,
0x0f, 0xec, 0xa4, 0xc6, 0xfe, 0x9e, 0x24, 0x07, 0xd0, 0xf0, 0xc9, 0x84, 0xd8, 0x4b, 0x32, 0x7d,
0xe7, 0xbb, 0xf3, 0xd0, 0xda, 0xa9, 0x35, 0x66, 0x7c, 0x9f, 0x58, 0x81, 0xeb, 0x70, 0x77, 0xd7,
0x70, 0x18, 0x25, 0x05, 0x94, 0x57, 0x0b, 0xb8, 0x05, 0x65, 0xdd, 0x29, 0x8f, 0x50, 0x43, 0xbc,
0x57, 0x69, 0x75, 0xaf, 0x1b, 0x68, 0xa6, 0x3d, 0xf4, 0x90, 0x96, 0x6d, 0xec, 0x5f, 0xda, 0xdc,
0x5f, 0x7b, 0x05, 0x95, 0xd0, 0x66, 0x2b, 0xef, 0xa0, 0x94, 0x7a, 0x07, 0xf7, 0xd8, 0x95, 0xbb,
0xd4, 0x8d, 0x92, 0xf3, 0x40, 0x7b, 0x0e, 0x90, 0x78, 0x2c, 0x8f, 0xd5, 0xfe, 0x84, 0x4a, 0x68,
0xa5, 0x8d, 0x6a, 0xa4, 0x8c, 0x6e, 0xbc, 0x84, 0xf2, 0x9c, 0x50, 0x8b, 0xef, 0x94, 0xef, 0x4d,
0xb3, 0x77, 0x46, 0xa8, 0x85, 0xb9, 0x54, 0x1b, 0x41, 0x25, 0xf4, 0x1c, 0x2b, 0x82, 0xb9, 0x6e,
0xe4, 0x46, 0x45, 0x88, 0xe8, 0x81, 0x59, 0x43, 0x43, 0x3e, 0x66, 0xd6, 0xa7, 0x50, 0x66, 0x86,
0x4d, 0xae, 0x4b, 0x5a, 0xbd, 0xf4, 0x67, 0x20, 0x73, 0x77, 0xe6, 0x18, 0xe0, 0x17, 0x90, 0xb9,
0x13, 0x8b, 0xee, 0x29, 0x1b, 0xe3, 0x6e, 0xfc, 0x42, 0xec, 0xa3, 0x04, 0x95, 0xb0, 0x78, 0xf4,
0x06, 0xaa, 0xe1, 0xa8, 0x05, 0xaa, 0xd4, 0x2a, 0xb5, 0xeb, 0x87, 0x3f, 0x64, 0x9f, 0x36, 0x1c,
0x56, 0x7e, 0xe2, 0x18, 0x41, 0x5d, 0x68, 0x04, 0x8b, 0x71, 0x30, 0xf1, 0x6d, 0x8f, 0xda, 0xae,
0xa3, 0x6e, 0xf1, 0x14, 0x79, 0xef, 0xe7, 0x62, 0xcc, 0xf1, 0x14, 0x82, 0x7e, 0x83, 0xca, 0xc4,
0x75, 0xa8, 0xef, 0xce, 0xf8, 0x10, 0xe7, 0x16, 0xd0, 0x13, 0x22, 0x9e, 0x21, 0x22, 0xb4, 0x2e,
0xd4, 0x57, 0x0a, 0x7b, 0xd0, 0xe3, 0xf3, 0x06, 0x2a, 0x61, 0x61, 0x0c, 0x0f, 0x4b, 0x1b, 0x8b,
0xbf, 0x18, 0x55, 0x9c, 0x2c, 0xe4, 0xe0, 0x9f, 0xb6, 0xa0, 0xbe, 0x52, 0x1a, 0x7a, 0x0d, 0xb2,
0x7d, 0xc3, 0x9e, 0x6a, 0xd1, 0xcd, 0x17, 0x85, 0x87, 0xe9, 0x9f, 0x58, 0x4b, 0xd1, 0x52, 0x01,
0x71, 0xfa, 0x6f, 0xcb, 0xa1, 0x61, 0x23, 0xef, 0xa1, 0xff, 0xb0, 0x1c, 0x1a, 0xd2, 0x0c, 0x62,
0xb4, 0x78, 0xf3, 0x4b, 0x9f, 0x41, 0xf3, 0x81, 0x13, 0xb4, 0x78, 0xfe, 0x5f, 0x47, 0xcf, 0x7f,
0xf9, 0x33, 0x68, 0x3e, 0x77, 0x82, 0xe6, 0x10, 0x3a, 0x81, 0x9a, 0x3d, 0x75, 0x1d, 0xca, 0xab,
0x97, 0x79, 0x86, 0x9f, 0x8a, 0xab, 0xd7, 0x5d, 0x87, 0xc6, 0x27, 0x48, 0x60, 0xed, 0x04, 0x94,
0xf5, 0xf6, 0x64, 0xbb, 0x0a, 0xed, 0x03, 0xc4, 0xb7, 0x1b, 0xf0, 0x96, 0x35, 0xf0, 0xca, 0x8a,
0x76, 0x98, 0x64, 0x8a, 0x36, 0x5a, 0x63, 0xa4, 0x0d, 0xa6, 0x1d, 0x33, 0x71, 0x83, 0x72, 0x3c,
0xfd, 0x36, 0x56, 0xc6, 0xcd, 0xc8, 0xa9, 0x93, 0xbd, 0xb2, 0x84, 0xf8, 0x51, 0x89, 0x22, 0xd0,
0x7e, 0x85, 0xbd, 0xac, 0x56, 0xdc, 0x57, 0xe1, 0xc1, 0x27, 0x09, 0xca, 0xec, 0x2f, 0x2e, 0xfa,
0x06, 0x76, 0xcd, 0xcb, 0xa3, 0x41, 0x7f, 0x78, 0x72, 0x7d, 0x66, 0x0c, 0x87, 0xdd, 0x63, 0x43,
0xf9, 0x0a, 0x21, 0x68, 0x62, 0xe3, 0xd4, 0xe8, 0x8d, 0xe2, 0x35, 0x09, 0x3d, 0x81, 0xaf, 0xf5,
0x4b, 0x73, 0xd0, 0xef, 0x75, 0x47, 0x46, 0xbc, 0xbc, 0xc5, 0x78, 0xdd, 0x18, 0xf4, 0xaf, 0x0c,
0x1c, 0x2f, 0x96, 0x50, 0x03, 0xaa, 0x5d, 0x5d, 0xbf, 0x36, 0x0d, 0x03, 0x2b, 0x65, 0xb4, 0x0b,
0x75, 0x6c, 0x9c, 0x5d, 0x5c, 0x19, 0x62, 0x41, 0x66, 0x3f, 0x63, 0xa3, 0x77, 0x75, 0x8d, 0xcd,
0x9e, 0xb2, 0xcd, 0xa2, 0xa1, 0x71, 0xae, 0xf3, 0xa8, 0xc2, 0x22, 0x1d, 0x5f, 0x98, 0x3c, 0xaa,
0xa2, 0x2a, 0x94, 0x4f, 0x2f, 0xfa, 0xe7, 0x4a, 0x0d, 0xd5, 0x40, 0x1e, 0x18, 0xdd, 0x2b, 0x43,
0x01, 0xf6, 0x79, 0x8c, 0xbb, 0xef, 0x46, 0x4a, 0x9d, 0x7d, 0x9a, 0xf8, 0xf2, 0xdc, 0x50, 0x1a,
0x07, 0x6f, 0x61, 0x37, 0x99, 0x8f, 0x23, 0x8b, 0x4e, 0x6e, 0xd0, 0xcf, 0x20, 0x8f, 0xd9, 0x47,
0x68, 0xa3, 0x27, 0x99, 0xa3, 0x84, 0x85, 0xe6, 0xa8, 0xf1, 0xf1, 0x6e, 0x5f, 0xfa, 0xf7, 0x6e,
0x5f, 0xfa, 0xef, 0x6e, 0x5f, 0xfa, 0x3f, 0x00, 0x00, 0xff, 0xff, 0x17, 0x7f, 0xbd, 0x0d, 0x4b,
0x0c, 0x00, 0x00,
}
func (m *TraceEvent) Marshal() (dAtA []byte, err error) {
@ -2509,6 +2567,20 @@ func (m *TraceEvent_ControlMeta) MarshalToSizedBuffer(dAtA []byte) (int, error)
i -= len(m.XXX_unrecognized)
copy(dAtA[i:], m.XXX_unrecognized)
}
if len(m.Idontwant) > 0 {
for iNdEx := len(m.Idontwant) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Idontwant[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintTrace(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x2a
}
}
if len(m.Prune) > 0 {
for iNdEx := len(m.Prune) - 1; iNdEx >= 0; iNdEx-- {
{
@ -2724,6 +2796,42 @@ func (m *TraceEvent_ControlPruneMeta) MarshalToSizedBuffer(dAtA []byte) (int, er
return len(dAtA) - i, nil
}
func (m *TraceEvent_ControlIDontWantMeta) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *TraceEvent_ControlIDontWantMeta) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *TraceEvent_ControlIDontWantMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if m.XXX_unrecognized != nil {
i -= len(m.XXX_unrecognized)
copy(dAtA[i:], m.XXX_unrecognized)
}
if len(m.MessageIDs) > 0 {
for iNdEx := len(m.MessageIDs) - 1; iNdEx >= 0; iNdEx-- {
i -= len(m.MessageIDs[iNdEx])
copy(dAtA[i:], m.MessageIDs[iNdEx])
i = encodeVarintTrace(dAtA, i, uint64(len(m.MessageIDs[iNdEx])))
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
}
func (m *TraceEventBatch) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -3211,6 +3319,12 @@ func (m *TraceEvent_ControlMeta) Size() (n int) {
n += 1 + l + sovTrace(uint64(l))
}
}
if len(m.Idontwant) > 0 {
for _, e := range m.Idontwant {
l = e.Size()
n += 1 + l + sovTrace(uint64(l))
}
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
@ -3295,6 +3409,24 @@ func (m *TraceEvent_ControlPruneMeta) Size() (n int) {
return n
}
func (m *TraceEvent_ControlIDontWantMeta) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.MessageIDs) > 0 {
for _, b := range m.MessageIDs {
l = len(b)
n += 1 + l + sovTrace(uint64(l))
}
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func (m *TraceEventBatch) Size() (n int) {
if m == nil {
return 0
@ -6032,6 +6164,40 @@ func (m *TraceEvent_ControlMeta) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
case 5:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Idontwant", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTrace
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthTrace
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthTrace
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Idontwant = append(m.Idontwant, &TraceEvent_ControlIDontWantMeta{})
if err := m.Idontwant[len(m.Idontwant)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipTrace(dAtA[iNdEx:])
@ -6453,6 +6619,89 @@ func (m *TraceEvent_ControlPruneMeta) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *TraceEvent_ControlIDontWantMeta) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTrace
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: ControlIDontWantMeta: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: ControlIDontWantMeta: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field MessageIDs", wireType)
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTrace
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthTrace
}
postIndex := iNdEx + byteLen
if postIndex < 0 {
return ErrInvalidLengthTrace
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.MessageIDs = append(m.MessageIDs, make([]byte, postIndex-iNdEx))
copy(m.MessageIDs[len(m.MessageIDs)-1], dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipTrace(dAtA[iNdEx:])
if err != nil {
return err
}
if (skippy < 0) || (iNdEx+skippy) < 0 {
return ErrInvalidLengthTrace
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *TraceEventBatch) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0

View File

@ -124,6 +124,7 @@ message TraceEvent {
repeated ControlIWantMeta iwant = 2;
repeated ControlGraftMeta graft = 3;
repeated ControlPruneMeta prune = 4;
repeated ControlIDontWantMeta idontwant = 5;
}
message ControlIHaveMeta {
@ -143,6 +144,10 @@ message TraceEvent {
optional string topic = 1;
repeated bytes peers = 2;
}
message ControlIDontWantMeta {
repeated bytes messageIDs = 1;
}
}
message TraceEventBatch {

View File

@ -147,7 +147,7 @@ type PubSub struct {
blacklist Blacklist
blacklistPeer chan peer.ID
peers map[peer.ID]chan *RPC
peers map[peer.ID]*rpcQueue
inboundStreamsMx sync.Mutex
inboundStreams map[peer.ID]network.Stream
@ -196,11 +196,14 @@ type PubSubRouter interface {
// EnoughPeers returns whether the router needs more peers before it's ready to publish new records.
// Suggested (if greater than 0) is a suggested number of peers that the router should need.
EnoughPeers(topic string, suggested int) bool
// AcceptFrom is invoked on any incoming message before pushing it to the validation pipeline
// AcceptFrom is invoked on any RPC envelope before pushing it to the validation pipeline
// or processing control information.
// Allows routers with internal scoring to vet peers before committing any processing resources
// to the message and implement an effective graylist and react to validation queue overload.
AcceptFrom(peer.ID) AcceptStatus
// PreValidation is invoked on messages in the RPC envelope right before pushing it to
// the validation pipeline
PreValidation([]*Message)
// HandleRPC is invoked to process control messages in the RPC envelope.
// It is invoked after subscriptions and payload messages have been processed.
HandleRPC(*RPC)
@ -285,7 +288,7 @@ func NewPubSub(ctx context.Context, h host.Host, rt PubSubRouter, opts ...Option
mySubs: make(map[string]map[*Subscription]struct{}),
myRelays: make(map[string]int),
topics: make(map[string]map[peer.ID]struct{}),
peers: make(map[peer.ID]chan *RPC),
peers: make(map[peer.ID]*rpcQueue),
inboundStreams: make(map[peer.ID]network.Stream),
blacklist: NewMapBlacklist(),
blacklistPeer: make(chan peer.ID),
@ -559,8 +562,8 @@ func WithAppSpecificRpcInspector(inspector func(peer.ID, *RPC) error) Option {
func (p *PubSub) processLoop(ctx context.Context) {
defer func() {
// Clean up go routines.
for _, ch := range p.peers {
close(ch)
for _, queue := range p.peers {
queue.Close()
}
p.peers = nil
p.topics = nil
@ -575,7 +578,7 @@ func (p *PubSub) processLoop(ctx context.Context) {
case s := <-p.newPeerStream:
pid := s.Conn().RemotePeer()
ch, ok := p.peers[pid]
q, ok := p.peers[pid]
if !ok {
log.Warn("new stream for unknown peer: ", pid)
s.Reset()
@ -584,7 +587,7 @@ func (p *PubSub) processLoop(ctx context.Context) {
if p.blacklist.Contains(pid) {
log.Warn("closing stream for blacklisted peer: ", pid)
close(ch)
q.Close()
delete(p.peers, pid)
s.Reset()
continue
@ -652,9 +655,9 @@ func (p *PubSub) processLoop(ctx context.Context) {
log.Infof("Blacklisting peer %s", pid)
p.blacklist.Add(pid)
ch, ok := p.peers[pid]
q, ok := p.peers[pid]
if ok {
close(ch)
q.Close()
delete(p.peers, pid)
for t, tmap := range p.topics {
if _, ok := tmap[pid]; ok {
@ -701,10 +704,10 @@ func (p *PubSub) handlePendingPeers() {
continue
}
messages := make(chan *RPC, p.peerOutboundQueueSize)
messages <- p.getHelloPacket()
go p.handleNewPeer(p.ctx, pid, messages)
p.peers[pid] = messages
rpcQueue := newRpcQueue(p.peerOutboundQueueSize)
rpcQueue.Push(p.getHelloPacket(), true)
go p.handleNewPeer(p.ctx, pid, rpcQueue)
p.peers[pid] = rpcQueue
}
}
@ -721,12 +724,12 @@ func (p *PubSub) handleDeadPeers() {
p.peerDeadPrioLk.Unlock()
for pid := range deadPeers {
ch, ok := p.peers[pid]
q, ok := p.peers[pid]
if !ok {
continue
}
close(ch)
q.Close()
delete(p.peers, pid)
for t, tmap := range p.topics {
@ -748,10 +751,10 @@ func (p *PubSub) handleDeadPeers() {
// still connected, must be a duplicate connection being closed.
// we respawn the writer as we need to ensure there is a stream active
log.Debugf("peer declared dead but still connected; respawning writer: %s", pid)
messages := make(chan *RPC, p.peerOutboundQueueSize)
messages <- p.getHelloPacket()
p.peers[pid] = messages
go p.handleNewPeerWithBackoff(p.ctx, pid, backoffDelay, messages)
rpcQueue := newRpcQueue(p.peerOutboundQueueSize)
rpcQueue.Push(p.getHelloPacket(), true)
p.peers[pid] = rpcQueue
go p.handleNewPeerWithBackoff(p.ctx, pid, backoffDelay, rpcQueue)
}
}
}
@ -915,14 +918,14 @@ func (p *PubSub) announce(topic string, sub bool) {
out := rpcWithSubs(subopt)
for pid, peer := range p.peers {
select {
case peer <- out:
p.tracer.SendRPC(out, pid)
default:
err := peer.Push(out, false)
if err != nil {
log.Infof("Can't send announce message to peer %s: queue full; scheduling retry", pid)
p.tracer.DropRPC(out, pid)
go p.announceRetry(pid, topic, sub)
continue
}
p.tracer.SendRPC(out, pid)
}
}
@ -958,14 +961,14 @@ func (p *PubSub) doAnnounceRetry(pid peer.ID, topic string, sub bool) {
}
out := rpcWithSubs(subopt)
select {
case peer <- out:
p.tracer.SendRPC(out, pid)
default:
err := peer.Push(out, false)
if err != nil {
log.Infof("Can't send announce message to peer %s: queue full; scheduling retry", pid)
p.tracer.DropRPC(out, pid)
go p.announceRetry(pid, topic, sub)
return
}
p.tracer.SendRPC(out, pid)
}
// notifySubs sends a given message to all corresponding subscribers.
@ -1091,13 +1094,21 @@ func (p *PubSub) handleIncomingRPC(rpc *RPC) {
p.tracer.ThrottlePeer(rpc.from)
case AcceptAll:
var toPush []*Message
for _, pmsg := range rpc.GetPublish() {
if !(p.subscribedToMsg(pmsg) || p.canRelayMsg(pmsg)) {
log.Debug("received message in topic we didn't subscribe to; ignoring message")
continue
}
p.pushMsg(&Message{pmsg, "", rpc.from, nil, false})
msg := &Message{pmsg, "", rpc.from, nil, false}
if p.shouldPush(msg) {
toPush = append(toPush, msg)
}
}
p.rt.PreValidation(toPush)
for _, msg := range toPush {
p.pushMsg(msg)
}
}
@ -1114,27 +1125,28 @@ func DefaultPeerFilter(pid peer.ID, topic string) bool {
return true
}
// pushMsg pushes a message performing validation as necessary
func (p *PubSub) pushMsg(msg *Message) {
// shouldPush filters a message before validating and pushing it
// It returns true if the message can be further validated and pushed
func (p *PubSub) shouldPush(msg *Message) bool {
src := msg.ReceivedFrom
// reject messages from blacklisted peers
if p.blacklist.Contains(src) {
log.Debugf("dropping message from blacklisted peer %s", src)
p.tracer.RejectMessage(msg, RejectBlacklstedPeer)
return
return false
}
// even if they are forwarded by good peers
if p.blacklist.Contains(msg.GetFrom()) {
log.Debugf("dropping message from blacklisted source %s", src)
p.tracer.RejectMessage(msg, RejectBlacklistedSource)
return
return false
}
err := p.checkSigningPolicy(msg)
if err != nil {
log.Debugf("dropping message from %s: %s", src, err)
return
return false
}
// reject messages claiming to be from ourselves but not locally published
@ -1142,16 +1154,24 @@ func (p *PubSub) pushMsg(msg *Message) {
if peer.ID(msg.GetFrom()) == self && src != self {
log.Debugf("dropping message claiming to be from self but forwarded from %s", src)
p.tracer.RejectMessage(msg, RejectSelfOrigin)
return
return false
}
// have we already seen and validated this message?
id := p.idGen.ID(msg)
if p.seenMessage(id) {
p.tracer.DuplicateMessage(msg)
return
return false
}
return true
}
// pushMsg pushes a message performing validation as necessary
func (p *PubSub) pushMsg(msg *Message) {
src := msg.ReceivedFrom
id := p.idGen.ID(msg)
if !p.val.Push(src, msg) {
return
}

View File

@ -94,6 +94,8 @@ func (rs *RandomSubRouter) AcceptFrom(peer.ID) AcceptStatus {
return AcceptAll
}
func (rs *RandomSubRouter) PreValidation([]*Message) {}
func (rs *RandomSubRouter) HandleRPC(rpc *RPC) {}
func (rs *RandomSubRouter) Publish(msg *Message) {
@ -144,18 +146,18 @@ func (rs *RandomSubRouter) Publish(msg *Message) {
out := rpcWithMessages(msg.Message)
for p := range tosend {
mch, ok := rs.p.peers[p]
q, ok := rs.p.peers[p]
if !ok {
continue
}
select {
case mch <- out:
rs.tracer.SendRPC(out, p)
default:
err := q.Push(out, false)
if err != nil {
log.Infof("dropping message to peer %s: queue full", p)
rs.tracer.DropRPC(out, p)
continue
}
rs.tracer.SendRPC(out, p)
}
}

147
rpc_queue.go Normal file
View File

@ -0,0 +1,147 @@
package pubsub
import (
"context"
"errors"
"sync"
)
var (
ErrQueueCancelled = errors.New("rpc queue operation cancelled")
ErrQueueClosed = errors.New("rpc queue closed")
ErrQueueFull = errors.New("rpc queue full")
ErrQueuePushOnClosed = errors.New("push on closed rpc queue")
)
type priorityQueue struct {
normal []*RPC
priority []*RPC
}
func (q *priorityQueue) Len() int {
return len(q.normal) + len(q.priority)
}
func (q *priorityQueue) NormalPush(rpc *RPC) {
q.normal = append(q.normal, rpc)
}
func (q *priorityQueue) PriorityPush(rpc *RPC) {
q.priority = append(q.priority, rpc)
}
func (q *priorityQueue) Pop() *RPC {
var rpc *RPC
if len(q.priority) > 0 {
rpc = q.priority[0]
q.priority[0] = nil
q.priority = q.priority[1:]
} else if len(q.normal) > 0 {
rpc = q.normal[0]
q.normal[0] = nil
q.normal = q.normal[1:]
}
return rpc
}
type rpcQueue struct {
dataAvailable sync.Cond
spaceAvailable sync.Cond
// Mutex used to access queue
queueMu sync.Mutex
queue priorityQueue
closed bool
maxSize int
}
func newRpcQueue(maxSize int) *rpcQueue {
q := &rpcQueue{maxSize: maxSize}
q.dataAvailable.L = &q.queueMu
q.spaceAvailable.L = &q.queueMu
return q
}
func (q *rpcQueue) Push(rpc *RPC, block bool) error {
return q.push(rpc, false, block)
}
func (q *rpcQueue) UrgentPush(rpc *RPC, block bool) error {
return q.push(rpc, true, block)
}
func (q *rpcQueue) push(rpc *RPC, urgent bool, block bool) error {
q.queueMu.Lock()
defer q.queueMu.Unlock()
if q.closed {
panic(ErrQueuePushOnClosed)
}
for q.queue.Len() == q.maxSize {
if block {
q.spaceAvailable.Wait()
// It can receive a signal because the queue is closed.
if q.closed {
panic(ErrQueuePushOnClosed)
}
} else {
return ErrQueueFull
}
}
if urgent {
q.queue.PriorityPush(rpc)
} else {
q.queue.NormalPush(rpc)
}
q.dataAvailable.Signal()
return nil
}
// Note that, when the queue is empty and there are two blocked Pop calls, it
// doesn't mean that the first Pop will get the item from the next Push. The
// second Pop will probably get it instead.
func (q *rpcQueue) Pop(ctx context.Context) (*RPC, error) {
q.queueMu.Lock()
defer q.queueMu.Unlock()
if q.closed {
return nil, ErrQueueClosed
}
unregisterAfterFunc := context.AfterFunc(ctx, func() {
// Wake up all the waiting routines. The only routine that correponds
// to this Pop call will return from the function. Note that this can
// be expensive, if there are too many waiting routines.
q.dataAvailable.Broadcast()
})
defer unregisterAfterFunc()
for q.queue.Len() == 0 {
select {
case <-ctx.Done():
return nil, ErrQueueCancelled
default:
}
q.dataAvailable.Wait()
// It can receive a signal because the queue is closed.
if q.closed {
return nil, ErrQueueClosed
}
}
rpc := q.queue.Pop()
q.spaceAvailable.Signal()
return rpc, nil
}
func (q *rpcQueue) Close() {
q.queueMu.Lock()
defer q.queueMu.Unlock()
q.closed = true
q.dataAvailable.Broadcast()
q.spaceAvailable.Broadcast()
}

229
rpc_queue_test.go Normal file
View File

@ -0,0 +1,229 @@
package pubsub
import (
"context"
"testing"
"time"
)
func TestNewRpcQueue(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
if q.maxSize != maxSize {
t.Fatalf("rpc queue has wrong max size, expected %d but got %d", maxSize, q.maxSize)
}
if q.dataAvailable.L != &q.queueMu {
t.Fatalf("the dataAvailable field of rpc queue has an incorrect mutex")
}
if q.spaceAvailable.L != &q.queueMu {
t.Fatalf("the spaceAvailable field of rpc queue has an incorrect mutex")
}
}
func TestRpcQueueUrgentPush(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
rpc1 := &RPC{}
rpc2 := &RPC{}
rpc3 := &RPC{}
rpc4 := &RPC{}
q.Push(rpc1, true)
q.UrgentPush(rpc2, true)
q.Push(rpc3, true)
q.UrgentPush(rpc4, true)
pop1, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
pop2, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
pop3, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
pop4, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
if pop1 != rpc2 {
t.Fatalf("get wrong item from rpc queue Pop")
}
if pop2 != rpc4 {
t.Fatalf("get wrong item from rpc queue Pop")
}
if pop3 != rpc1 {
t.Fatalf("get wrong item from rpc queue Pop")
}
if pop4 != rpc3 {
t.Fatalf("get wrong item from rpc queue Pop")
}
}
func TestRpcQueuePushThenPop(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
rpc1 := &RPC{}
rpc2 := &RPC{}
q.Push(rpc1, true)
q.Push(rpc2, true)
pop1, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
pop2, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
if pop1 != rpc1 {
t.Fatalf("get wrong item from rpc queue Pop")
}
if pop2 != rpc2 {
t.Fatalf("get wrong item from rpc queue Pop")
}
}
func TestRpcQueuePopThenPush(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
rpc1 := &RPC{}
rpc2 := &RPC{}
go func() {
// Wait to make sure the main goroutine is blocked.
time.Sleep(1 * time.Millisecond)
q.Push(rpc1, true)
q.Push(rpc2, true)
}()
pop1, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
pop2, err := q.Pop(context.Background())
if err != nil {
t.Fatal(err)
}
if pop1 != rpc1 {
t.Fatalf("get wrong item from rpc queue Pop")
}
if pop2 != rpc2 {
t.Fatalf("get wrong item from rpc queue Pop")
}
}
func TestRpcQueueBlockPushWhenFull(t *testing.T) {
maxSize := 1
q := newRpcQueue(maxSize)
finished := make(chan struct{})
q.Push(&RPC{}, true)
go func() {
q.Push(&RPC{}, true)
finished <- struct{}{}
}()
// Wait to make sure the goroutine is blocked.
time.Sleep(1 * time.Millisecond)
select {
case <-finished:
t.Fatalf("blocking rpc queue Push is not blocked when it is full")
default:
}
}
func TestRpcQueueNonblockPushWhenFull(t *testing.T) {
maxSize := 1
q := newRpcQueue(maxSize)
q.Push(&RPC{}, true)
err := q.Push(&RPC{}, false)
if err != ErrQueueFull {
t.Fatalf("non-blocking rpc queue Push returns wrong error when it is full")
}
}
func TestRpcQueuePushAfterClose(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
q.Close()
defer func() {
if r := recover(); r == nil {
t.Fatalf("rpc queue Push does not panick after closed")
}
}()
q.Push(&RPC{}, true)
}
func TestRpcQueuePopAfterClose(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
q.Close()
_, err := q.Pop(context.Background())
if err != ErrQueueClosed {
t.Fatalf("rpc queue Pop returns wrong error after closed")
}
}
func TestRpcQueueCloseWhilePush(t *testing.T) {
maxSize := 1
q := newRpcQueue(maxSize)
q.Push(&RPC{}, true)
defer func() {
if r := recover(); r == nil {
t.Fatalf("rpc queue Push does not panick when it's closed on the fly")
}
}()
go func() {
// Wait to make sure the main goroutine is blocked.
time.Sleep(1 * time.Millisecond)
q.Close()
}()
q.Push(&RPC{}, true)
}
func TestRpcQueueCloseWhilePop(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
go func() {
// Wait to make sure the main goroutine is blocked.
time.Sleep(1 * time.Millisecond)
q.Close()
}()
_, err := q.Pop(context.Background())
if err != ErrQueueClosed {
t.Fatalf("rpc queue Pop returns wrong error when it's closed on the fly")
}
}
func TestRpcQueuePushWhenFullThenPop(t *testing.T) {
maxSize := 1
q := newRpcQueue(maxSize)
q.Push(&RPC{}, true)
go func() {
// Wait to make sure the main goroutine is blocked.
time.Sleep(1 * time.Millisecond)
q.Pop(context.Background())
}()
q.Push(&RPC{}, true)
}
func TestRpcQueueCancelPop(t *testing.T) {
maxSize := 32
q := newRpcQueue(maxSize)
ctx, cancel := context.WithCancel(context.Background())
go func() {
// Wait to make sure the main goroutine is blocked.
time.Sleep(1 * time.Millisecond)
cancel()
}()
_, err := q.Pop(ctx)
if err != ErrQueueCancelled {
t.Fatalf("rpc queue Pop returns wrong error when it's cancelled")
}
}

View File

@ -402,11 +402,23 @@ func (t *pubsubTracer) traceRPCMeta(rpc *RPC) *pb.TraceEvent_RPCMeta {
})
}
var idontwant []*pb.TraceEvent_ControlIDontWantMeta
for _, ctl := range rpc.Control.Idontwant {
var mids [][]byte
for _, mid := range ctl.MessageIDs {
mids = append(mids, []byte(mid))
}
idontwant = append(idontwant, &pb.TraceEvent_ControlIDontWantMeta{
MessageIDs: mids,
})
}
rpcMeta.Control = &pb.TraceEvent_ControlMeta{
Ihave: ihave,
Iwant: iwant,
Graft: graft,
Prune: prune,
Ihave: ihave,
Iwant: iwant,
Graft: graft,
Prune: prune,
Idontwant: idontwant,
}
}