mirror of
https://github.com/logos-messaging/go-libp2p-pubsub.git
synced 2026-01-07 07:13:13 +00:00
refactor: create MeshPeer function in Pubsub
This commit is contained in:
parent
ab817a0cb1
commit
3408a4ce1a
@ -2133,7 +2133,7 @@ func (gs *GossipSubRouter) getPeers(topic string, count int, filter func(peer.ID
|
|||||||
return peers
|
return peers
|
||||||
}
|
}
|
||||||
|
|
||||||
func (gs *GossipSubRouter) MeshPeers(topic string) []peer.ID {
|
func (gs *GossipSubRouter) meshPeers(topic string) []peer.ID {
|
||||||
peers, ok := gs.mesh[topic]
|
peers, ok := gs.mesh[topic]
|
||||||
if !ok {
|
if !ok {
|
||||||
return nil
|
return nil
|
||||||
|
|||||||
29
pubsub.go
29
pubsub.go
@ -90,6 +90,9 @@ type PubSub struct {
|
|||||||
// get chan of peers we are connected to
|
// get chan of peers we are connected to
|
||||||
getPeers chan *listPeerReq
|
getPeers chan *listPeerReq
|
||||||
|
|
||||||
|
// get chan to obtain list of full mesh peers (only applies when ussing gossipsub)
|
||||||
|
getMeshPeers chan *listPeerReq
|
||||||
|
|
||||||
// send subscription here to cancel it
|
// send subscription here to cancel it
|
||||||
cancelCh chan *Subscription
|
cancelCh chan *Subscription
|
||||||
|
|
||||||
@ -274,6 +277,7 @@ func NewPubSub(ctx context.Context, h host.Host, rt PubSubRouter, opts ...Option
|
|||||||
deadPeerBackoff: newBackoff(ctx, 1000, BackoffCleanupInterval, MaxBackoffAttempts),
|
deadPeerBackoff: newBackoff(ctx, 1000, BackoffCleanupInterval, MaxBackoffAttempts),
|
||||||
cancelCh: make(chan *Subscription),
|
cancelCh: make(chan *Subscription),
|
||||||
getPeers: make(chan *listPeerReq),
|
getPeers: make(chan *listPeerReq),
|
||||||
|
getMeshPeers: make(chan *listPeerReq),
|
||||||
addSub: make(chan *addSubReq),
|
addSub: make(chan *addSubReq),
|
||||||
addRelay: make(chan *addRelayReq),
|
addRelay: make(chan *addRelayReq),
|
||||||
rmRelay: make(chan string),
|
rmRelay: make(chan string),
|
||||||
@ -619,6 +623,13 @@ func (p *PubSub) processLoop(ctx context.Context) {
|
|||||||
p.handleAddRelay(relay)
|
p.handleAddRelay(relay)
|
||||||
case topic := <-p.rmRelay:
|
case topic := <-p.rmRelay:
|
||||||
p.handleRemoveRelay(topic)
|
p.handleRemoveRelay(topic)
|
||||||
|
case meshpreq := <-p.getMeshPeers:
|
||||||
|
var peers []peer.ID
|
||||||
|
rt, ok := p.rt.(*GossipSubRouter)
|
||||||
|
if ok {
|
||||||
|
peers = rt.meshPeers(meshpreq.topic)
|
||||||
|
}
|
||||||
|
meshpreq.resp <- peers
|
||||||
case preq := <-p.getPeers:
|
case preq := <-p.getPeers:
|
||||||
tmap, ok := p.topics[preq.topic]
|
tmap, ok := p.topics[preq.topic]
|
||||||
if preq.topic != "" && !ok {
|
if preq.topic != "" && !ok {
|
||||||
@ -1384,6 +1395,20 @@ func (p *PubSub) ListPeers(topic string) []peer.ID {
|
|||||||
return <-out
|
return <-out
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MeshPeers returns a list of full mesh peers for a given topic
|
||||||
|
func (p *PubSub) MeshPeers(topic string) []peer.ID {
|
||||||
|
out := make(chan []peer.ID)
|
||||||
|
select {
|
||||||
|
case p.getMeshPeers <- &listPeerReq{
|
||||||
|
resp: out,
|
||||||
|
topic: topic,
|
||||||
|
}:
|
||||||
|
case <-p.ctx.Done():
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return <-out
|
||||||
|
}
|
||||||
|
|
||||||
// BlacklistPeer blacklists a peer; all messages from this peer will be unconditionally dropped.
|
// BlacklistPeer blacklists a peer; all messages from this peer will be unconditionally dropped.
|
||||||
func (p *PubSub) BlacklistPeer(pid peer.ID) {
|
func (p *PubSub) BlacklistPeer(pid peer.ID) {
|
||||||
select {
|
select {
|
||||||
@ -1440,7 +1465,3 @@ type addRelayReq struct {
|
|||||||
topic string
|
topic string
|
||||||
resp chan RelayCancelFunc
|
resp chan RelayCancelFunc
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) Router() PubSubRouter {
|
|
||||||
return p.rt
|
|
||||||
}
|
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user