Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
66d895b
update protobufs for partial messages
MarcoPolo Jul 11, 2025
a294843
add structured rpc logging
MarcoPolo Aug 29, 2025
7c88c9e
implement Partial Messages
MarcoPolo Aug 29, 2025
2b8043f
add partial messages to gossipsub router
MarcoPolo Aug 29, 2025
6f10ff5
partialmessages: add basic bitmap package
MarcoPolo Oct 9, 2025
81b0766
partialmessages: add explicit MergePartsMetadata function
MarcoPolo Oct 9, 2025
2acf705
Add documentation for the RequestPartialMessages topic option
MarcoPolo Oct 3, 2025
f1c889a
don't send IDONTWANT to partial message peers
MarcoPolo Oct 13, 2025
2318ac2
add todo
MarcoPolo Oct 13, 2025
50afc31
ensure that the Hello Packet is the first rpc sent
sukunrt Oct 13, 2025
408cb91
set write deadline for outgoing messages
sukunrt Oct 13, 2025
154a2ab
refactor score methods to accept topic string
MarcoPolo Oct 13, 2025
69b08b2
Add ability to update peer scores if using partial messages
MarcoPolo Oct 13, 2025
b0ca1be
nit: rename method from old form
MarcoPolo Oct 13, 2025
451d92c
limit the number of peer initiated group states we track
MarcoPolo Oct 13, 2025
8b37311
support fanout topics for partial messages
MarcoPolo Oct 14, 2025
b773ff0
pb: add `supportsPartial` field in SubOpts
MarcoPolo Oct 23, 2025
adec0e9
Add support for `supportsPartial`
MarcoPolo Oct 23, 2025
c7aab40
add test for SupportsPartial subscribe option
MarcoPolo Oct 23, 2025
2c63415
partialmessages: Add pairwise interaction test
MarcoPolo Nov 13, 2025
a6c2238
partialmessages: add per peer bounds on peer initiated groups
MarcoPolo Oct 24, 2025
d704f03
partialmessages: rename field to reflect use
MarcoPolo Nov 13, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 36 additions & 6 deletions comm.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,16 @@ func (p *PubSub) getHelloPacket() *RPC {
}

for t := range subscriptions {
var requestPartial, supportsPartialMessages bool
if ts, ok := p.myTopics[t]; ok {
requestPartial = ts.requestPartialMessages
supportsPartialMessages = ts.supportsPartialMessages
}
as := &pb.RPC_SubOpts{
Topicid: proto.String(t),
Subscribe: proto.Bool(true),
Topicid: proto.String(t),
Subscribe: proto.Bool(true),
RequestsPartial: &requestPartial,
SupportsSendingPartial: &supportsPartialMessages,
}
rpc.Subscriptions = append(rpc.Subscriptions, as)
}
Expand Down Expand Up @@ -123,7 +130,7 @@ func (p *PubSub) notifyPeerDead(pid peer.ID) {
}

func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing *rpcQueue) {
s, err := p.host.NewStream(p.ctx, pid, p.rt.Protocols()...)
s, err := p.host.NewStream(ctx, pid, p.rt.Protocols()...)
if err != nil {
p.logger.Debug("error opening new stream to peer", "err", err, "peer", pid)

Expand All @@ -135,11 +142,14 @@ func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing *rpcQu
return
}

go p.handleSendingMessages(ctx, s, outgoing)
firstMessage := make(chan *RPC, 1)
sCtx, cancel := context.WithCancel(ctx)
go p.handleSendingMessages(sCtx, s, outgoing, firstMessage)
go p.handlePeerDead(s)
select {
case p.newPeerStream <- s:
case p.newPeerStream <- peerOutgoingStream{Stream: s, FirstMessage: firstMessage, Cancel: cancel}:
case <-ctx.Done():
cancel()
}
}

Expand All @@ -164,7 +174,7 @@ func (p *PubSub) handlePeerDead(s network.Stream) {
p.notifyPeerDead(pid)
}

func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, outgoing *rpcQueue) {
func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, outgoing *rpcQueue, firstMessage chan *RPC) {
writeRpc := func(rpc *RPC) error {
size := uint64(rpc.Size())

Expand All @@ -177,6 +187,11 @@ func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, ou
return err
}

if err := s.SetWriteDeadline(time.Now().Add(time.Second * 30)); err != nil {
p.rpcLogger.Debug("failed to set write deadline", "peer", s.Conn().RemotePeer(), "err", err)
return err
}

_, err = s.Write(buf)
if err != nil {
p.rpcLogger.Debug("failed to send message", "peer", s.Conn().RemotePeer(), "rpc", rpc, "err", err)
Expand All @@ -186,6 +201,21 @@ func (p *PubSub) handleSendingMessages(ctx context.Context, s network.Stream, ou
return nil
}

select {
case rpc := <-firstMessage:
if rpc.Size() > 0 {
err := writeRpc(rpc)
if err != nil {
s.Reset()
p.logger.Debug("error writing message to peer", "peer", s.Conn().RemotePeer(), "err", err)
return
}
}
case <-ctx.Done():
s.Reset()
return
}

defer s.Close()
for ctx.Err() == nil {
rpc, err := outgoing.Pop(ctx)
Expand Down
108 changes: 104 additions & 4 deletions extensions.go
Original file line number Diff line number Diff line change
@@ -1,12 +1,17 @@
package pubsub

import (
"errors"
"iter"

"github.com/libp2p/go-libp2p-pubsub/partialmessages"
pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p/core/peer"
)

type PeerExtensions struct {
TestExtension bool
TestExtension bool
PartialMessages bool
}

type TestExtensionConfig struct {
Expand Down Expand Up @@ -37,6 +42,7 @@ func peerExtensionsFromRPC(rpc *RPC) PeerExtensions {
out := PeerExtensions{}
if hasPeerExtensions(rpc) {
out.TestExtension = rpc.Control.Extensions.GetTestExtension()
out.PartialMessages = rpc.Control.Extensions.GetPartialMessages()
}
return out
}
Expand All @@ -46,9 +52,19 @@ func (pe *PeerExtensions) ExtendRPC(rpc *RPC) *RPC {
if rpc.Control == nil {
rpc.Control = &pubsub_pb.ControlMessage{}
}
rpc.Control.Extensions = &pubsub_pb.ControlExtensions{
TestExtension: &pe.TestExtension,
if rpc.Control.Extensions == nil {
rpc.Control.Extensions = &pubsub_pb.ControlExtensions{}
}
rpc.Control.Extensions.TestExtension = &pe.TestExtension
}
if pe.PartialMessages {
if rpc.Control == nil {
rpc.Control = &pubsub_pb.ControlMessage{}
}
if rpc.Control.Extensions == nil {
rpc.Control.Extensions = &pubsub_pb.ControlExtensions{}
}
rpc.Control.Extensions.PartialMessages = &pe.PartialMessages
}
return rpc
}
Expand All @@ -59,8 +75,9 @@ type extensionsState struct {
sentExtensions map[peer.ID]struct{}
reportMisbehavior func(peer.ID)
sendRPC func(p peer.ID, r *RPC, urgent bool)
testExtension *testExtension

testExtension *testExtension
partialMessagesExtension *partialmessages.PartialMessageExtension
}

func newExtensionsState(myExtensions PeerExtensions, reportMisbehavior func(peer.ID), sendRPC func(peer.ID, *RPC, bool)) *extensionsState {
Expand Down Expand Up @@ -132,14 +149,97 @@ func (es *extensionsState) extensionsAddPeer(id peer.ID) {
if es.myExtensions.TestExtension && es.peerExtensions[id].TestExtension {
es.testExtension.AddPeer(id)
}

if es.myExtensions.PartialMessages && es.peerExtensions[id].PartialMessages {
es.partialMessagesExtension.AddPeer(id)
}
}

// extensionsRemovePeer is always called after extensionsAddPeer.
func (es *extensionsState) extensionsRemovePeer(id peer.ID) {
if es.myExtensions.PartialMessages && es.peerExtensions[id].PartialMessages {
es.partialMessagesExtension.RemovePeer(id)
}
}

func (es *extensionsState) extensionsHandleRPC(rpc *RPC) {
if es.myExtensions.TestExtension && es.peerExtensions[rpc.from].TestExtension {
es.testExtension.HandleRPC(rpc.from, rpc.TestExtension)
}

if es.myExtensions.PartialMessages && es.peerExtensions[rpc.from].PartialMessages && rpc.Partial != nil {
es.partialMessagesExtension.HandleRPC(rpc.from, rpc.Partial)
}
}

func (es *extensionsState) Heartbeat() {
if es.myExtensions.PartialMessages {
es.partialMessagesExtension.Heartbeat()
}
}

func WithPartialMessagesExtension(pm *partialmessages.PartialMessageExtension) Option {
return func(ps *PubSub) error {
gs, ok := ps.rt.(*GossipSubRouter)
if !ok {
return errors.New("pubsub router is not gossipsub")
}
err := pm.Init(partialMessageRouter{gs})
if err != nil {
return err
}

gs.extensions.myExtensions.PartialMessages = true
gs.extensions.partialMessagesExtension = pm
return nil
}
}

type partialMessageRouter struct {
gs *GossipSubRouter
}

// PeerRequestsPartial implements partialmessages.Router.
func (r partialMessageRouter) PeerRequestsPartial(peer peer.ID, topic string) bool {
return r.gs.peerRequestsPartial(peer, topic)
}

// MeshPeers implements partialmessages.Router.
func (r partialMessageRouter) MeshPeers(topic string) iter.Seq[peer.ID] {
return func(yield func(peer.ID) bool) {
peerSet, ok := r.gs.mesh[topic]
if !ok {
// Possibly a fanout topic
peerSet, ok = r.gs.fanout[topic]
if !ok {
return
}
}

myTopicState := r.gs.p.myTopics[topic]
iRequestPartial := myTopicState != nil && myTopicState.requestPartialMessages
for peer := range peerSet {
if r.gs.extensions.peerExtensions[peer].PartialMessages {
peerSupportsPartial := r.gs.peerSupportsPartial(peer, topic)
peerRequestsPartial := r.gs.peerRequestsPartial(peer, topic)
if (iRequestPartial && peerSupportsPartial) || peerRequestsPartial {
// Peer supports partial messages
if !yield(peer) {
return
}
}
}
}
}
}

// SendRPC implements partialmessages.Router.
func (r partialMessageRouter) SendRPC(p peer.ID, rpc *pubsub_pb.PartialMessagesExtension, urgent bool) {
r.gs.sendRPC(p, &RPC{
RPC: pubsub_pb.RPC{
Partial: rpc,
},
}, urgent)
}

var _ partialmessages.Router = partialMessageRouter{}
6 changes: 6 additions & 0 deletions floodsub_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,12 @@ func denseConnect(t *testing.T, hosts []host.Host) {
connectSome(t, hosts, 10)
}

func ringConnect(t *testing.T, hosts []host.Host) {
for i := range hosts {
connect(t, hosts[i], hosts[(i+1)%len(hosts)])
}
}

func connectSome(t *testing.T, hosts []host.Host, d int) {
for i, a := range hosts {
for j := 0; j < d; j++ {
Expand Down
22 changes: 22 additions & 0 deletions gossipsub.go
Original file line number Diff line number Diff line change
Expand Up @@ -864,6 +864,12 @@ func (gs *GossipSubRouter) Preprocess(from peer.ID, msgs []*Message) {
// We don't send IDONTWANT to the peer that sent us the messages
continue
}
if gs.peerRequestsPartial(p, topic) {
// Don't send IDONTWANT to peers that are using partial messages
// for this topic
continue
}

// send to only peers that support IDONTWANT
if gs.feature(GossipSubFeatureIdontwant, gs.peers[p]) {
idontwant := []*pb.ControlIDontWant{{MessageIDs: mids}}
Expand Down Expand Up @@ -1375,6 +1381,10 @@ func (gs *GossipSubRouter) rpcs(msg *Message) iter.Seq2[peer.ID, *RPC] {
if pid == from || pid == peer.ID(msg.GetFrom()) {
continue
}
if gs.peerRequestsPartial(pid, topic) {
// The peer requested partial messages. We'll skip sending them full messages
continue
}

if !yield(pid, out) {
return
Expand All @@ -1383,6 +1393,16 @@ func (gs *GossipSubRouter) rpcs(msg *Message) iter.Seq2[peer.ID, *RPC] {
}
}

func (gs *GossipSubRouter) peerSupportsPartial(p peer.ID, topic string) bool {
peerStates, ok := gs.p.topics[topic]
return ok && gs.extensions.myExtensions.PartialMessages && peerStates[p].supportsPartial
}

func (gs *GossipSubRouter) peerRequestsPartial(p peer.ID, topic string) bool {
peerStates, ok := gs.p.topics[topic]
return ok && gs.extensions.myExtensions.PartialMessages && peerStates[p].requestsPartial
}

func (gs *GossipSubRouter) Join(topic string) {
gmap, ok := gs.mesh[topic]
if ok {
Expand Down Expand Up @@ -1833,6 +1853,8 @@ func (gs *GossipSubRouter) heartbeat() {

// advance the message history window
gs.mcache.Shift()

gs.extensions.Heartbeat()
}

func (gs *GossipSubRouter) clearIHaveCounters() {
Expand Down
Loading
Loading