mirror of
https://github.com/logos-messaging/go-libp2p-pubsub.git
synced 2026-01-04 05:43:06 +00:00
rename to floodsub
This commit is contained in:
parent
b69b777643
commit
e07f002705
@ -1,4 +1,4 @@
|
|||||||
package dumbsub
|
package floodsub
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bufio"
|
"bufio"
|
||||||
@ -7,16 +7,22 @@ import (
|
|||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
peer "github.com/ipfs/go-libp2p-peer"
|
logging "gx/ipfs/QmSpJByNKFX1sCsHBEp3R73FL4NF6FnQTEGyNAXHm2GS52/go-log"
|
||||||
logging "github.com/ipfs/go-log"
|
peer "gx/ipfs/QmWtbQU15LaB5B1JC2F7TV9P4K88vD3PpA4AJrwfCjhML8/go-libp2p-peer"
|
||||||
host "github.com/libp2p/go-libp2p/p2p/host"
|
host "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/host"
|
||||||
inet "github.com/libp2p/go-libp2p/p2p/net"
|
inet "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/net"
|
||||||
protocol "github.com/libp2p/go-libp2p/p2p/protocol"
|
protocol "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/protocol"
|
||||||
)
|
)
|
||||||
|
|
||||||
const ID = protocol.ID("/dumbsub/1.0.0")
|
const ID = protocol.ID("/floodsub/1.0.0")
|
||||||
|
|
||||||
var log = logging.Logger("dumbsub")
|
const (
|
||||||
|
AddSubMessageType = "sub"
|
||||||
|
UnsubMessageType = "unsub"
|
||||||
|
PubMessageType = "pub"
|
||||||
|
)
|
||||||
|
|
||||||
|
var log = logging.Logger("floodsub")
|
||||||
|
|
||||||
type PubSub struct {
|
type PubSub struct {
|
||||||
host host.Host
|
host host.Host
|
||||||
@ -84,7 +90,7 @@ type RPC struct {
|
|||||||
from peer.ID
|
from peer.ID
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewDumbSub(h host.Host) *PubSub {
|
func NewFloodSub(h host.Host) *PubSub {
|
||||||
ps := &PubSub{
|
ps := &PubSub{
|
||||||
host: h,
|
host: h,
|
||||||
incoming: make(chan *RPC, 32),
|
incoming: make(chan *RPC, 32),
|
||||||
@ -112,7 +118,7 @@ func (p *PubSub) getHelloPacket() *RPC {
|
|||||||
for t, _ := range p.myTopics {
|
for t, _ := range p.myTopics {
|
||||||
rpc.Topics = append(rpc.Topics, t)
|
rpc.Topics = append(rpc.Topics, t)
|
||||||
}
|
}
|
||||||
rpc.Type = "add"
|
rpc.Type = AddSubMessageType
|
||||||
return &rpc
|
return &rpc
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -177,17 +183,7 @@ func (p *PubSub) processLoop() {
|
|||||||
case pid := <-p.peerDead:
|
case pid := <-p.peerDead:
|
||||||
delete(p.peers, pid)
|
delete(p.peers, pid)
|
||||||
case sub := <-p.addSub:
|
case sub := <-p.addSub:
|
||||||
_, ok := p.myTopics[sub.topic]
|
p.handleSubscriptionChange(sub)
|
||||||
if ok {
|
|
||||||
// we don't allow multiple subs per topic at this point
|
|
||||||
sub.resp <- nil
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
resp := make(chan *Message, 16)
|
|
||||||
p.myTopics[sub.topic] = resp
|
|
||||||
sub.resp <- resp
|
|
||||||
|
|
||||||
case rpc := <-p.incoming:
|
case rpc := <-p.incoming:
|
||||||
err := p.handleIncomingRPC(rpc)
|
err := p.handleIncomingRPC(rpc)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -195,11 +191,11 @@ func (p *PubSub) processLoop() {
|
|||||||
}
|
}
|
||||||
case rpc := <-p.outgoing:
|
case rpc := <-p.outgoing:
|
||||||
switch rpc.Type {
|
switch rpc.Type {
|
||||||
case "add", "del":
|
case AddSubMessageType, UnsubMessageType:
|
||||||
for _, mch := range p.peers {
|
for _, mch := range p.peers {
|
||||||
mch <- rpc
|
mch <- rpc
|
||||||
}
|
}
|
||||||
case "pub":
|
case PubMessageType:
|
||||||
//fmt.Println("publishing outgoing message")
|
//fmt.Println("publishing outgoing message")
|
||||||
err := p.recvMessage(rpc)
|
err := p.recvMessage(rpc)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -214,6 +210,37 @@ func (p *PubSub) processLoop() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
func (p *PubSub) handleSubscriptionChange(sub *addSub) {
|
||||||
|
ch, ok := p.myTopics[sub.topic]
|
||||||
|
out := &RPC{
|
||||||
|
Topics: []string{sub.topic},
|
||||||
|
}
|
||||||
|
|
||||||
|
if sub.cancel {
|
||||||
|
if !ok {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
close(ch)
|
||||||
|
delete(p.myTopics, sub.topic)
|
||||||
|
out.Type = UnsubMessageType
|
||||||
|
} else {
|
||||||
|
if ok {
|
||||||
|
// we don't allow multiple subs per topic at this point
|
||||||
|
sub.resp <- nil
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
resp := make(chan *Message, 16)
|
||||||
|
p.myTopics[sub.topic] = resp
|
||||||
|
sub.resp <- resp
|
||||||
|
out.Type = AddSubMessageType
|
||||||
|
}
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
p.outgoing <- out
|
||||||
|
}()
|
||||||
|
}
|
||||||
|
|
||||||
func (p *PubSub) recvMessage(rpc *RPC) error {
|
func (p *PubSub) recvMessage(rpc *RPC) error {
|
||||||
subch, ok := p.myTopics[rpc.Msg.Topic]
|
subch, ok := p.myTopics[rpc.Msg.Topic]
|
||||||
@ -226,7 +253,7 @@ func (p *PubSub) recvMessage(rpc *RPC) error {
|
|||||||
|
|
||||||
func (p *PubSub) handleIncomingRPC(rpc *RPC) error {
|
func (p *PubSub) handleIncomingRPC(rpc *RPC) error {
|
||||||
switch rpc.Type {
|
switch rpc.Type {
|
||||||
case "add":
|
case AddSubMessageType:
|
||||||
for _, t := range rpc.Topics {
|
for _, t := range rpc.Topics {
|
||||||
tmap, ok := p.topics[t]
|
tmap, ok := p.topics[t]
|
||||||
if !ok {
|
if !ok {
|
||||||
@ -236,7 +263,7 @@ func (p *PubSub) handleIncomingRPC(rpc *RPC) error {
|
|||||||
|
|
||||||
tmap[rpc.from] = struct{}{}
|
tmap[rpc.from] = struct{}{}
|
||||||
}
|
}
|
||||||
case "del":
|
case UnsubMessageType:
|
||||||
for _, t := range rpc.Topics {
|
for _, t := range rpc.Topics {
|
||||||
tmap, ok := p.topics[t]
|
tmap, ok := p.topics[t]
|
||||||
if !ok {
|
if !ok {
|
||||||
@ -244,11 +271,11 @@ func (p *PubSub) handleIncomingRPC(rpc *RPC) error {
|
|||||||
}
|
}
|
||||||
delete(tmap, rpc.from)
|
delete(tmap, rpc.from)
|
||||||
}
|
}
|
||||||
case "pub":
|
case PubMessageType:
|
||||||
//fmt.Println("incoming message! ", rpc.from)
|
|
||||||
if rpc.Msg == nil {
|
if rpc.Msg == nil {
|
||||||
return fmt.Errorf("nil pub message")
|
return fmt.Errorf("nil pub message")
|
||||||
}
|
}
|
||||||
|
|
||||||
// Note: Obviously this is an incredibly insecure way of
|
// Note: Obviously this is an incredibly insecure way of
|
||||||
// filtering out "messages we've already seen". But it works for a
|
// filtering out "messages we've already seen". But it works for a
|
||||||
// cool demo, so i'm not gonna waste time thinking about it any more
|
// cool demo, so i'm not gonna waste time thinking about it any more
|
||||||
@ -282,7 +309,7 @@ func (p *PubSub) publishMessage(rpc *RPC) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
for pid, _ := range tmap {
|
for pid, _ := range tmap {
|
||||||
if pid == rpc.from {
|
if pid == rpc.from || pid == rpc.Msg.From {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -298,8 +325,9 @@ func (p *PubSub) publishMessage(rpc *RPC) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
type addSub struct {
|
type addSub struct {
|
||||||
topic string
|
topic string
|
||||||
resp chan chan *Message
|
cancel bool
|
||||||
|
resp chan chan *Message
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) Subscribe(topic string) (<-chan *Message, error) {
|
func (p *PubSub) Subscribe(topic string) (<-chan *Message, error) {
|
||||||
@ -318,7 +346,10 @@ func (p *PubSub) Subscribe(topic string) (<-chan *Message, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) Unsub(topic string) {
|
func (p *PubSub) Unsub(topic string) {
|
||||||
panic("NYI")
|
p.addSub <- &addSub{
|
||||||
|
topic: topic,
|
||||||
|
cancel: true,
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) Publish(topic string, data []byte) error {
|
func (p *PubSub) Publish(topic string, data []byte) error {
|
||||||
@ -329,16 +360,11 @@ func (p *PubSub) Publish(topic string, data []byte) error {
|
|||||||
From: p.host.ID(),
|
From: p.host.ID(),
|
||||||
Timestamp: uint64(time.Now().UnixNano()),
|
Timestamp: uint64(time.Now().UnixNano()),
|
||||||
},
|
},
|
||||||
Type: "pub",
|
Type: PubMessageType,
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) sendHelloPacket(s inet.Stream) error {
|
|
||||||
hello := p.getHelloPacket()
|
|
||||||
return writeRPC(s, hello)
|
|
||||||
}
|
|
||||||
|
|
||||||
func writeRPC(s inet.Stream, rpc *RPC) error {
|
func writeRPC(s inet.Stream, rpc *RPC) error {
|
||||||
data, err := json.Marshal(rpc)
|
data, err := json.Marshal(rpc)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1,4 +1,4 @@
|
|||||||
package dumbsub
|
package floodsub
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
@ -8,8 +8,8 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
host "github.com/libp2p/go-libp2p/p2p/host"
|
host "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/host"
|
||||||
netutil "github.com/libp2p/go-libp2p/p2p/test/util"
|
netutil "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/test/util"
|
||||||
)
|
)
|
||||||
|
|
||||||
func getNetHosts(t *testing.T, n int) []host.Host {
|
func getNetHosts(t *testing.T, n int) []host.Host {
|
||||||
@ -43,12 +43,12 @@ func connectAll(t *testing.T, hosts []host.Host) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestBasicDumbsub(t *testing.T) {
|
func TestBasicFloodsub(t *testing.T) {
|
||||||
hosts := getNetHosts(t, 10)
|
hosts := getNetHosts(t, 20)
|
||||||
|
|
||||||
var psubs []*PubSub
|
var psubs []*PubSub
|
||||||
for _, h := range hosts {
|
for _, h := range hosts {
|
||||||
psubs = append(psubs, NewDumbSub(h))
|
psubs = append(psubs, NewFloodSub(h))
|
||||||
}
|
}
|
||||||
|
|
||||||
var msgs []<-chan *Message
|
var msgs []<-chan *Message
|
||||||
@ -1,11 +1,10 @@
|
|||||||
package dumbsub
|
package floodsub
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
|
||||||
|
|
||||||
ma "github.com/jbenet/go-multiaddr"
|
ma "gx/ipfs/QmYzDkkgAEmrcNzFCiYo6L1dTX4EAG1gZkbtdbd9trL4vd/go-multiaddr"
|
||||||
inet "github.com/libp2p/go-libp2p/p2p/net"
|
inet "gx/ipfs/Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS/go-libp2p/p2p/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
var _ inet.Notifiee = (*PubSub)(nil)
|
var _ inet.Notifiee = (*PubSub)(nil)
|
||||||
@ -19,7 +18,6 @@ func (p *PubSub) ClosedStream(n inet.Network, s inet.Stream) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (p *PubSub) Connected(n inet.Network, c inet.Conn) {
|
func (p *PubSub) Connected(n inet.Network, c inet.Conn) {
|
||||||
fmt.Printf("got connection! %s -> %s\n", c.LocalPeer(), c.RemotePeer())
|
|
||||||
s, err := p.host.NewStream(context.Background(), c.RemotePeer(), ID)
|
s, err := p.host.NewStream(context.Background(), c.RemotePeer(), ID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("opening new stream to peer: ", err)
|
log.Error("opening new stream to peer: ", err)
|
||||||
|
|||||||
@ -9,9 +9,9 @@
|
|||||||
"gxDependencies": [
|
"gxDependencies": [
|
||||||
{
|
{
|
||||||
"author": "whyrusleeping",
|
"author": "whyrusleeping",
|
||||||
"hash": "QmXnaDLonE9YBTVDdWBM6Jb5YxxmW1MHMkXzgsnu1jTEmK",
|
"hash": "Qmf4ETeAWXuThBfWwonVyFqGFSgTWepUDEr1txcctvpTXS",
|
||||||
"name": "go-libp2p",
|
"name": "go-libp2p",
|
||||||
"version": "3.4.3"
|
"version": "3.4.1"
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"gxVersion": "0.9.0",
|
"gxVersion": "0.9.0",
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user