Skip to content

Commit

Permalink
GossipSub v1.2: IDONTWANT control message and priority queue. (#553)
Browse files Browse the repository at this point in the history
## 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 <[email protected]>

* 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 6fabcdd.

* Increase GossipSubMaxIDontWantMessages to 1000

* fixup! Handle IDONTWANT control messages

* Skip TestGossipsubConnTagMessageDeliveries

* Skip FuzzAppendOrMergeRPC

* Revert "Skip FuzzAppendOrMergeRPC"

This reverts commit f141e13.

* 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 <[email protected]>
  • Loading branch information
ppopth and Stebalien authored Aug 16, 2024
1 parent 19ffbb3 commit b421b3a
Show file tree
Hide file tree
Showing 17 changed files with 2,032 additions and 247 deletions.
41 changes: 20 additions & 21 deletions comm.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Expand All @@ -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())

Expand All @@ -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
}
}
Expand All @@ -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,
},
},
}
Expand Down
12 changes: 7 additions & 5 deletions floodsub.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand All @@ -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)
}
}

Expand Down
Loading

0 comments on commit b421b3a

Please sign in to comment.