rename to floodsub

This commit is contained in:
Jeromy 2016-09-09 20:13:50 -07:00
parent b69b777643
commit e07f002705
4 changed files with 74 additions and 50 deletions

View File

@ -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 {

View File

@ -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

View File

@ -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)

View File

@ -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",