mirror of
https://github.com/logos-messaging/go-libp2p-pubsub.git
synced 2026-01-04 05:43:06 +00:00
test: use the regular libp2p host (#565)
This removes dependencies on swarm/testing and the blank host. 1. swarm/testing really shouldn't be used at all except for internal libp2p stuff. 2. The blank host should only be used in _very_ special cases (autonat, mostly).
This commit is contained in:
parent
8e498e9e96
commit
1f5b81fb61
@ -38,7 +38,7 @@ func TestBlacklist(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
|
|
||||||
@ -66,7 +66,7 @@ func TestBlacklist2(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
|
|
||||||
@ -99,7 +99,7 @@ func TestBlacklist3(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
psubs[1].BlacklistPeer(hosts[0].ID())
|
psubs[1].BlacklistPeer(hosts[0].ID())
|
||||||
|
|||||||
@ -134,7 +134,7 @@ func TestSimpleDiscovery(t *testing.T) {
|
|||||||
server := newDiscoveryServer()
|
server := newDiscoveryServer()
|
||||||
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(1 * time.Minute)}
|
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(1 * time.Minute)}
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
psubs := make([]*PubSub, numHosts)
|
psubs := make([]*PubSub, numHosts)
|
||||||
topicHandlers := make([]*Topic, numHosts)
|
topicHandlers := make([]*Topic, numHosts)
|
||||||
|
|
||||||
@ -234,7 +234,7 @@ func TestGossipSubDiscoveryAfterBootstrap(t *testing.T) {
|
|||||||
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(ttl)}
|
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(ttl)}
|
||||||
|
|
||||||
// Put the pubsub clients into two partitions
|
// Put the pubsub clients into two partitions
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
psubs := make([]*PubSub, numHosts)
|
psubs := make([]*PubSub, numHosts)
|
||||||
topicHandlers := make([]*Topic, numHosts)
|
topicHandlers := make([]*Topic, numHosts)
|
||||||
|
|
||||||
|
|||||||
@ -20,9 +20,6 @@ import (
|
|||||||
"github.com/libp2p/go-libp2p/core/peer"
|
"github.com/libp2p/go-libp2p/core/peer"
|
||||||
"github.com/libp2p/go-libp2p/core/protocol"
|
"github.com/libp2p/go-libp2p/core/protocol"
|
||||||
|
|
||||||
bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
|
|
||||||
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
|
|
||||||
|
|
||||||
"github.com/libp2p/go-msgio/protoio"
|
"github.com/libp2p/go-msgio/protoio"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -42,19 +39,6 @@ func checkMessageRouting(t *testing.T, topic string, pubs []*PubSub, subs []*Sub
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func getNetHosts(t *testing.T, ctx context.Context, n int) []host.Host {
|
|
||||||
var out []host.Host
|
|
||||||
|
|
||||||
for i := 0; i < n; i++ {
|
|
||||||
netw := swarmt.GenSwarm(t)
|
|
||||||
h := bhost.NewBlankHost(netw)
|
|
||||||
t.Cleanup(func() { h.Close() })
|
|
||||||
out = append(out, h)
|
|
||||||
}
|
|
||||||
|
|
||||||
return out
|
|
||||||
}
|
|
||||||
|
|
||||||
func connect(t *testing.T, a, b host.Host) {
|
func connect(t *testing.T, a, b host.Host) {
|
||||||
pinfo := a.Peerstore().PeerInfo(a.ID())
|
pinfo := a.Peerstore().PeerInfo(a.ID())
|
||||||
err := b.Connect(context.Background(), pinfo)
|
err := b.Connect(context.Background(), pinfo)
|
||||||
@ -151,7 +135,7 @@ func assertNeverReceives(t *testing.T, ch *Subscription, timeout time.Duration)
|
|||||||
func TestBasicFloodsub(t *testing.T) {
|
func TestBasicFloodsub(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -193,7 +177,7 @@ func TestMultihops(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 6)
|
hosts := getDefaultHosts(t, 6)
|
||||||
|
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -235,7 +219,7 @@ func TestReconnects(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
|
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -309,7 +293,7 @@ func TestNoConnection(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
|
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -334,7 +318,7 @@ func TestSelfReceive(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
host := getNetHosts(t, ctx, 1)[0]
|
host := getDefaultHosts(t, 1)[0]
|
||||||
|
|
||||||
psub, err := NewFloodSub(ctx, host)
|
psub, err := NewFloodSub(ctx, host)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -368,7 +352,7 @@ func TestOneToOne(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -401,7 +385,7 @@ func TestTreeTopology(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -464,7 +448,7 @@ func TestFloodSubPluggableProtocol(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
|
|
||||||
psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub", "/lsr/floodsub")
|
psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub", "/lsr/floodsub")
|
||||||
psubB := mustCreatePubSub(ctx, t, hosts[1], "/esh/floodsub")
|
psubB := mustCreatePubSub(ctx, t, hosts[1], "/esh/floodsub")
|
||||||
@ -496,7 +480,7 @@ func TestFloodSubPluggableProtocol(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
|
|
||||||
psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub")
|
psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub")
|
||||||
psubB := mustCreatePubSub(ctx, t, hosts[1], "/lsr/floodsub")
|
psubB := mustCreatePubSub(ctx, t, hosts[1], "/lsr/floodsub")
|
||||||
@ -551,7 +535,7 @@ func TestSubReporting(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
host := getNetHosts(t, ctx, 1)[0]
|
host := getDefaultHosts(t, 1)[0]
|
||||||
psub, err := NewFloodSub(ctx, host)
|
psub, err := NewFloodSub(ctx, host)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
@ -593,7 +577,7 @@ func TestPeerTopicReporting(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 4)
|
hosts := getDefaultHosts(t, 4)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -650,7 +634,7 @@ func TestSubscribeMultipleTimes(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -695,7 +679,7 @@ func TestPeerDisconnect(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -743,7 +727,7 @@ func TestWithNoSigning(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts, WithNoAuthor(), WithMessageIdFn(func(pmsg *pb.Message) string {
|
psubs := getPubsubs(ctx, hosts, WithNoAuthor(), WithMessageIdFn(func(pmsg *pb.Message) string {
|
||||||
// silly content-based test message-ID: just use the data as whole
|
// silly content-based test message-ID: just use the data as whole
|
||||||
return base64.URLEncoding.EncodeToString(pmsg.Data)
|
return base64.URLEncoding.EncodeToString(pmsg.Data)
|
||||||
@ -788,7 +772,7 @@ func TestWithSigning(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts, WithStrictSignatureVerification(true))
|
psubs := getPubsubs(ctx, hosts, WithStrictSignatureVerification(true))
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -830,7 +814,7 @@ func TestImproperlySignedMessageRejected(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
adversary := hosts[0]
|
adversary := hosts[0]
|
||||||
honestPeer := hosts[1]
|
honestPeer := hosts[1]
|
||||||
|
|
||||||
@ -948,7 +932,7 @@ func TestMessageSender(t *testing.T) {
|
|||||||
|
|
||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
var msgs []*Subscription
|
var msgs []*Subscription
|
||||||
@ -1002,7 +986,7 @@ func TestConfigurableMaxMessageSize(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
|
|
||||||
// use a 4mb limit; default is 1mb; we'll test with a 2mb payload.
|
// use a 4mb limit; default is 1mb; we'll test with a 2mb payload.
|
||||||
psubs := getPubsubs(ctx, hosts, WithMaxMessageSize(1<<22))
|
psubs := getPubsubs(ctx, hosts, WithMaxMessageSize(1<<22))
|
||||||
@ -1045,7 +1029,7 @@ func TestAnnounceRetry(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
ps := getPubsub(ctx, hosts[0])
|
ps := getPubsub(ctx, hosts[0])
|
||||||
watcher := &announceWatcher{}
|
watcher := &announceWatcher{}
|
||||||
hosts[1].SetStreamHandler(FloodSubID, watcher.handleStream)
|
hosts[1].SetStreamHandler(FloodSubID, watcher.handleStream)
|
||||||
@ -1117,7 +1101,7 @@ func TestPubsubWithAssortedOptions(t *testing.T) {
|
|||||||
return string(hash[:])
|
return string(hash[:])
|
||||||
}
|
}
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts,
|
psubs := getPubsubs(ctx, hosts,
|
||||||
WithMessageIdFn(hashMsgID),
|
WithMessageIdFn(hashMsgID),
|
||||||
WithPeerOutboundQueueSize(10),
|
WithPeerOutboundQueueSize(10),
|
||||||
@ -1152,8 +1136,7 @@ func TestWithInvalidMessageAuthor(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
h := getDefaultHosts(t, 1)[0]
|
||||||
defer h.Close()
|
|
||||||
_, err := NewFloodSub(ctx, h, WithMessageAuthor("bogotr0n"))
|
_, err := NewFloodSub(ctx, h, WithMessageAuthor("bogotr0n"))
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatal("expected error")
|
t.Fatal("expected error")
|
||||||
@ -1168,10 +1151,9 @@ func TestPreconnectedNodes(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create hosts
|
// Create hosts
|
||||||
h1 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
hosts := getDefaultHosts(t, 2)
|
||||||
h2 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
h1 := hosts[0]
|
||||||
defer h1.Close()
|
h2 := hosts[1]
|
||||||
defer h2.Close()
|
|
||||||
|
|
||||||
opts := []Option{WithDiscovery(&dummyDiscovery{})}
|
opts := []Option{WithDiscovery(&dummyDiscovery{})}
|
||||||
// Setup first PubSub
|
// Setup first PubSub
|
||||||
@ -1229,10 +1211,9 @@ func TestDedupInboundStreams(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h1 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
hosts := getDefaultHosts(t, 2)
|
||||||
h2 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
h1 := hosts[0]
|
||||||
defer h1.Close()
|
h2 := hosts[1]
|
||||||
defer h2.Close()
|
|
||||||
|
|
||||||
_, err := NewFloodSub(ctx, h1)
|
_, err := NewFloodSub(ctx, h1)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1247,18 +1228,30 @@ func TestDedupInboundStreams(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
_, err = s1.Read(nil) // force protocol negotiation to complete
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
|
||||||
s2, err := h2.NewStream(ctx, h1.ID(), FloodSubID)
|
s2, err := h2.NewStream(ctx, h1.ID(), FloodSubID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
_, err = s2.Read(nil) // force protocol negotiation to complete
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
|
||||||
s3, err := h2.NewStream(ctx, h1.ID(), FloodSubID)
|
s3, err := h2.NewStream(ctx, h1.ID(), FloodSubID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
_, err = s3.Read(nil) // force protocol negotiation to complete
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
|
||||||
// check that s1 and s2 have been reset
|
// check that s1 and s2 have been reset
|
||||||
|
|||||||
@ -7,10 +7,10 @@ import (
|
|||||||
|
|
||||||
"github.com/benbjohnson/clock"
|
"github.com/benbjohnson/clock"
|
||||||
"github.com/libp2p/go-libp2p/core/host"
|
"github.com/libp2p/go-libp2p/core/host"
|
||||||
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
|
|
||||||
|
|
||||||
|
"github.com/libp2p/go-libp2p"
|
||||||
|
"github.com/libp2p/go-libp2p/core/network"
|
||||||
"github.com/libp2p/go-libp2p/core/peer"
|
"github.com/libp2p/go-libp2p/core/peer"
|
||||||
bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
|
|
||||||
"github.com/libp2p/go-libp2p/p2p/net/connmgr"
|
"github.com/libp2p/go-libp2p/p2p/net/connmgr"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -70,9 +70,14 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
netw := swarmt.GenSwarm(t)
|
h, err := libp2p.New(
|
||||||
defer netw.Close()
|
libp2p.ResourceManager(&network.NullResourceManager{}),
|
||||||
h := bhost.NewBlankHost(netw, bhost.WithConnectionManager(connmgrs[i]))
|
libp2p.ConnectionManager(connmgrs[i]),
|
||||||
|
)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
t.Cleanup(func() { h.Close() })
|
||||||
honestHosts[i] = h
|
honestHosts[i] = h
|
||||||
honestPeers[h.ID()] = struct{}{}
|
honestPeers[h.ID()] = struct{}{}
|
||||||
}
|
}
|
||||||
@ -83,7 +88,7 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
|
|||||||
WithFloodPublish(true))
|
WithFloodPublish(true))
|
||||||
|
|
||||||
// sybil squatters to be connected later
|
// sybil squatters to be connected later
|
||||||
sybilHosts := getNetHosts(t, ctx, nSquatter)
|
sybilHosts := getDefaultHosts(t, nSquatter)
|
||||||
for _, h := range sybilHosts {
|
for _, h := range sybilHosts {
|
||||||
squatter := &sybilSquatter{h: h}
|
squatter := &sybilSquatter{h: h}
|
||||||
h.SetStreamHandler(GossipSubID_v10, squatter.handleStream)
|
h.SetStreamHandler(GossipSubID_v10, squatter.handleStream)
|
||||||
|
|||||||
@ -42,7 +42,7 @@ func TestGossipSubCustomProtocols(t *testing.T) {
|
|||||||
|
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
|
|
||||||
gsubs := getGossipsubs(ctx, hosts[:2], WithGossipSubProtocols(protos, features))
|
gsubs := getGossipsubs(ctx, hosts[:2], WithGossipSubProtocols(protos, features))
|
||||||
fsub := getPubsub(ctx, hosts[2])
|
fsub := getPubsub(ctx, hosts[2])
|
||||||
|
|||||||
@ -17,7 +17,7 @@ func TestGossipSubMatchingFn(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 4)
|
h := getDefaultHosts(t, 4)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA100, GossipSubID_v11}, GossipSubDefaultFeatures)),
|
getGossipsub(ctx, h[0], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA100, GossipSubID_v11}, GossipSubDefaultFeatures)),
|
||||||
getGossipsub(ctx, h[1], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA101Beta}, GossipSubDefaultFeatures)),
|
getGossipsub(ctx, h[1], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA101Beta}, GossipSubDefaultFeatures)),
|
||||||
|
|||||||
@ -25,7 +25,7 @@ func TestGossipsubAttackSpamIWANT(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create legitimate and attacker hosts
|
// Create legitimate and attacker hosts
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
legit := hosts[0]
|
legit := hosts[0]
|
||||||
attacker := hosts[1]
|
attacker := hosts[1]
|
||||||
|
|
||||||
@ -142,7 +142,7 @@ func TestGossipsubAttackSpamIHAVE(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create legitimate and attacker hosts
|
// Create legitimate and attacker hosts
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
legit := hosts[0]
|
legit := hosts[0]
|
||||||
attacker := hosts[1]
|
attacker := hosts[1]
|
||||||
|
|
||||||
@ -195,6 +195,7 @@ func TestGossipsubAttackSpamIHAVE(t *testing.T) {
|
|||||||
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
|
Control: &pb.ControlMessage{Graft: []*pb.ControlGraft{{TopicID: sub.Topicid}}},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
sub := sub
|
||||||
go func() {
|
go func() {
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
@ -292,7 +293,7 @@ func TestGossipsubAttackGRAFTNonExistentTopic(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create legitimate and attacker hosts
|
// Create legitimate and attacker hosts
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
legit := hosts[0]
|
legit := hosts[0]
|
||||||
attacker := hosts[1]
|
attacker := hosts[1]
|
||||||
|
|
||||||
@ -376,7 +377,7 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create legitimate and attacker hosts
|
// Create legitimate and attacker hosts
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
legit := hosts[0]
|
legit := hosts[0]
|
||||||
attacker := hosts[1]
|
attacker := hosts[1]
|
||||||
|
|
||||||
@ -430,6 +431,7 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) {
|
|||||||
Control: &pb.ControlMessage{Graft: graft},
|
Control: &pb.ControlMessage{Graft: graft},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
sub := sub
|
||||||
go func() {
|
go func() {
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
@ -617,7 +619,7 @@ func TestGossipsubAttackInvalidMessageSpam(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// Create legitimate and attacker hosts
|
// Create legitimate and attacker hosts
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
legit := hosts[0]
|
legit := hosts[0]
|
||||||
attacker := hosts[1]
|
attacker := hosts[1]
|
||||||
|
|
||||||
|
|||||||
@ -13,16 +13,12 @@ import (
|
|||||||
|
|
||||||
pb "github.com/libp2p/go-libp2p-pubsub/pb"
|
pb "github.com/libp2p/go-libp2p-pubsub/pb"
|
||||||
|
|
||||||
"github.com/libp2p/go-libp2p"
|
|
||||||
"github.com/libp2p/go-libp2p/core/host"
|
"github.com/libp2p/go-libp2p/core/host"
|
||||||
"github.com/libp2p/go-libp2p/core/network"
|
"github.com/libp2p/go-libp2p/core/network"
|
||||||
"github.com/libp2p/go-libp2p/core/peer"
|
"github.com/libp2p/go-libp2p/core/peer"
|
||||||
"github.com/libp2p/go-libp2p/core/peerstore"
|
"github.com/libp2p/go-libp2p/core/peerstore"
|
||||||
"github.com/libp2p/go-libp2p/core/record"
|
"github.com/libp2p/go-libp2p/core/record"
|
||||||
|
|
||||||
bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
|
|
||||||
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
|
|
||||||
|
|
||||||
"github.com/libp2p/go-msgio/protoio"
|
"github.com/libp2p/go-msgio/protoio"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -45,7 +41,7 @@ func getGossipsubs(ctx context.Context, hs []host.Host, opts ...Option) []*PubSu
|
|||||||
func TestSparseGossipsub(t *testing.T) {
|
func TestSparseGossipsub(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -86,7 +82,7 @@ func TestSparseGossipsub(t *testing.T) {
|
|||||||
func TestDenseGossipsub(t *testing.T) {
|
func TestDenseGossipsub(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -127,7 +123,7 @@ func TestDenseGossipsub(t *testing.T) {
|
|||||||
func TestGossipsubFanout(t *testing.T) {
|
func TestGossipsubFanout(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -196,7 +192,7 @@ func TestGossipsubFanout(t *testing.T) {
|
|||||||
func TestGossipsubFanoutMaintenance(t *testing.T) {
|
func TestGossipsubFanoutMaintenance(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -281,7 +277,7 @@ func TestGossipsubFanoutExpiry(t *testing.T) {
|
|||||||
|
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -340,7 +336,7 @@ func TestGossipsubFanoutExpiry(t *testing.T) {
|
|||||||
func TestGossipsubGossip(t *testing.T) {
|
func TestGossipsubGossip(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -388,7 +384,7 @@ func TestGossipsubGossipPiggyback(t *testing.T) {
|
|||||||
t.Skip("test no longer relevant; gossip propagation has become eager")
|
t.Skip("test no longer relevant; gossip propagation has become eager")
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -457,7 +453,7 @@ func TestGossipsubGossipPropagation(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
hosts1 := hosts[:GossipSubD+1]
|
hosts1 := hosts[:GossipSubD+1]
|
||||||
@ -537,7 +533,7 @@ func TestGossipsubGossipPropagation(t *testing.T) {
|
|||||||
func TestGossipsubPrune(t *testing.T) {
|
func TestGossipsubPrune(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -586,7 +582,7 @@ func TestGossipsubPrune(t *testing.T) {
|
|||||||
func TestGossipsubPruneBackoffTime(t *testing.T) {
|
func TestGossipsubPruneBackoffTime(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
|
|
||||||
// App specific score that we'll change later.
|
// App specific score that we'll change later.
|
||||||
currentScoreForHost0 := int32(0)
|
currentScoreForHost0 := int32(0)
|
||||||
@ -684,7 +680,7 @@ func TestGossipsubPruneBackoffTime(t *testing.T) {
|
|||||||
func TestGossipsubGraft(t *testing.T) {
|
func TestGossipsubGraft(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -729,7 +725,7 @@ func TestGossipsubGraft(t *testing.T) {
|
|||||||
func TestGossipsubRemovePeer(t *testing.T) {
|
func TestGossipsubRemovePeer(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -779,7 +775,7 @@ func TestGossipsubGraftPruneRetry(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
denseConnect(t, hosts)
|
denseConnect(t, hosts)
|
||||||
|
|
||||||
@ -829,7 +825,7 @@ func TestGossipsubControlPiggyback(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
denseConnect(t, hosts)
|
denseConnect(t, hosts)
|
||||||
|
|
||||||
@ -910,7 +906,7 @@ func TestGossipsubControlPiggyback(t *testing.T) {
|
|||||||
func TestMixedGossipsub(t *testing.T) {
|
func TestMixedGossipsub(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
hosts := getNetHosts(t, ctx, 30)
|
hosts := getDefaultHosts(t, 30)
|
||||||
|
|
||||||
gsubs := getGossipsubs(ctx, hosts[:20])
|
gsubs := getGossipsubs(ctx, hosts[:20])
|
||||||
fsubs := getPubsubs(ctx, hosts[20:])
|
fsubs := getPubsubs(ctx, hosts[20:])
|
||||||
@ -954,7 +950,7 @@ func TestGossipsubMultihops(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 6)
|
hosts := getDefaultHosts(t, 6)
|
||||||
|
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
@ -997,7 +993,7 @@ func TestGossipsubTreeTopology(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -1061,7 +1057,7 @@ func TestGossipsubStarTopology(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts, WithPeerExchange(true), WithFloodPublish(true))
|
psubs := getGossipsubs(ctx, hosts, WithPeerExchange(true), WithFloodPublish(true))
|
||||||
|
|
||||||
// configure the center of the star with a very low D
|
// configure the center of the star with a very low D
|
||||||
@ -1223,7 +1219,7 @@ func TestGossipsubDirectPeers(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 3)
|
h := getDefaultHosts(t, 3)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0], WithDirectConnectTicks(2)),
|
getGossipsub(ctx, h[0], WithDirectConnectTicks(2)),
|
||||||
getGossipsub(ctx, h[1], WithDirectPeers([]peer.AddrInfo{{ID: h[2].ID(), Addrs: h[2].Addrs()}}), WithDirectConnectTicks(2)),
|
getGossipsub(ctx, h[1], WithDirectPeers([]peer.AddrInfo{{ID: h[2].ID(), Addrs: h[2].Addrs()}}), WithDirectConnectTicks(2)),
|
||||||
@ -1287,7 +1283,7 @@ func TestGossipSubPeerFilter(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 3)
|
h := getDefaultHosts(t, 3)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0], WithPeerFilter(func(pid peer.ID, topic string) bool {
|
getGossipsub(ctx, h[0], WithPeerFilter(func(pid peer.ID, topic string) bool {
|
||||||
return pid == h[1].ID()
|
return pid == h[1].ID()
|
||||||
@ -1329,7 +1325,7 @@ func TestGossipsubDirectPeersFanout(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 3)
|
h := getDefaultHosts(t, 3)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0]),
|
getGossipsub(ctx, h[0]),
|
||||||
getGossipsub(ctx, h[1], WithDirectPeers([]peer.AddrInfo{{ID: h[2].ID(), Addrs: h[2].Addrs()}})),
|
getGossipsub(ctx, h[1], WithDirectPeers([]peer.AddrInfo{{ID: h[2].ID(), Addrs: h[2].Addrs()}})),
|
||||||
@ -1416,7 +1412,7 @@ func TestGossipsubFloodPublish(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts, WithFloodPublish(true))
|
psubs := getGossipsubs(ctx, hosts, WithFloodPublish(true))
|
||||||
|
|
||||||
// build the star
|
// build the star
|
||||||
@ -1451,7 +1447,7 @@ func TestGossipsubEnoughPeers(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
|
|
||||||
for _, ps := range psubs {
|
for _, ps := range psubs {
|
||||||
@ -1500,7 +1496,7 @@ func TestGossipsubCustomParams(t *testing.T) {
|
|||||||
|
|
||||||
wantedMaxPendingConns := 23
|
wantedMaxPendingConns := 23
|
||||||
params.MaxPendingConnections = wantedMaxPendingConns
|
params.MaxPendingConnections = wantedMaxPendingConns
|
||||||
hosts := getNetHosts(t, ctx, 1)
|
hosts := getDefaultHosts(t, 1)
|
||||||
psubs := getGossipsubs(ctx, hosts,
|
psubs := getGossipsubs(ctx, hosts,
|
||||||
WithGossipSubParams(params))
|
WithGossipSubParams(params))
|
||||||
|
|
||||||
@ -1529,7 +1525,7 @@ func TestGossipsubNegativeScore(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts,
|
psubs := getGossipsubs(ctx, hosts,
|
||||||
WithPeerScore(
|
WithPeerScore(
|
||||||
&PeerScoreParams{
|
&PeerScoreParams{
|
||||||
@ -1613,7 +1609,7 @@ func TestGossipsubScoreValidatorEx(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
psubs := getGossipsubs(ctx, hosts,
|
psubs := getGossipsubs(ctx, hosts,
|
||||||
WithPeerScore(
|
WithPeerScore(
|
||||||
&PeerScoreParams{
|
&PeerScoreParams{
|
||||||
@ -1701,8 +1697,7 @@ func TestGossipsubPiggybackControl(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
h := getDefaultHosts(t, 1)[0]
|
||||||
defer h.Close()
|
|
||||||
ps := getGossipsub(ctx, h)
|
ps := getGossipsub(ctx, h)
|
||||||
|
|
||||||
blah := peer.ID("bogotr0n")
|
blah := peer.ID("bogotr0n")
|
||||||
@ -1750,7 +1745,7 @@ func TestGossipsubMultipleGraftTopics(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getGossipsubs(ctx, hosts)
|
psubs := getGossipsubs(ctx, hosts)
|
||||||
sparseConnect(t, hosts)
|
sparseConnect(t, hosts)
|
||||||
|
|
||||||
@ -1818,7 +1813,7 @@ func TestGossipsubOpportunisticGrafting(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 50)
|
hosts := getDefaultHosts(t, 50)
|
||||||
// pubsubs for the first 10 hosts
|
// pubsubs for the first 10 hosts
|
||||||
psubs := getGossipsubs(ctx, hosts[:10],
|
psubs := getGossipsubs(ctx, hosts[:10],
|
||||||
WithFloodPublish(true),
|
WithFloodPublish(true),
|
||||||
@ -1919,7 +1914,7 @@ func TestGossipSubLeaveTopic(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 2)
|
h := getDefaultHosts(t, 2)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0]),
|
getGossipsub(ctx, h[0]),
|
||||||
getGossipsub(ctx, h[1]),
|
getGossipsub(ctx, h[1]),
|
||||||
@ -1990,7 +1985,7 @@ func TestGossipSubJoinTopic(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h := getNetHosts(t, ctx, 3)
|
h := getDefaultHosts(t, 3)
|
||||||
psubs := []*PubSub{
|
psubs := []*PubSub{
|
||||||
getGossipsub(ctx, h[0]),
|
getGossipsub(ctx, h[0]),
|
||||||
getGossipsub(ctx, h[1]),
|
getGossipsub(ctx, h[1]),
|
||||||
@ -2072,7 +2067,7 @@ func TestGossipsubPeerScoreInspect(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
|
|
||||||
inspector := &mockPeerScoreInspector{}
|
inspector := &mockPeerScoreInspector{}
|
||||||
psub1 := getGossipsub(ctx, hosts[0],
|
psub1 := getGossipsub(ctx, hosts[0],
|
||||||
@ -2132,7 +2127,7 @@ func TestGossipsubPeerScoreResetTopicParams(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 1)
|
hosts := getDefaultHosts(t, 1)
|
||||||
|
|
||||||
ps := getGossipsub(ctx, hosts[0],
|
ps := getGossipsub(ctx, hosts[0],
|
||||||
WithPeerScore(
|
WithPeerScore(
|
||||||
@ -2199,7 +2194,7 @@ func TestGossipsubRPCFragmentation(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
ps := getGossipsub(ctx, hosts[0])
|
ps := getGossipsub(ctx, hosts[0])
|
||||||
|
|
||||||
// make a fake peer that requests everything through IWANT gossip
|
// make a fake peer that requests everything through IWANT gossip
|
||||||
@ -2553,21 +2548,6 @@ func FuzzAppendOrMergeRPC(f *testing.F) {
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func getDefaultHosts(t *testing.T, n int) []host.Host {
|
|
||||||
var out []host.Host
|
|
||||||
|
|
||||||
for i := 0; i < n; i++ {
|
|
||||||
h, err := libp2p.New(libp2p.ResourceManager(&network.NullResourceManager{}))
|
|
||||||
if err != nil {
|
|
||||||
t.Fatal(err)
|
|
||||||
}
|
|
||||||
t.Cleanup(func() { h.Close() })
|
|
||||||
out = append(out, h)
|
|
||||||
}
|
|
||||||
|
|
||||||
return out
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestGossipsubManagesAnAddressBook(t *testing.T) {
|
func TestGossipsubManagesAnAddressBook(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|||||||
@ -4,13 +4,32 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/libp2p/go-libp2p"
|
||||||
|
"github.com/libp2p/go-libp2p/core/host"
|
||||||
|
"github.com/libp2p/go-libp2p/core/network"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
func getDefaultHosts(t *testing.T, n int) []host.Host {
|
||||||
|
var out []host.Host
|
||||||
|
|
||||||
|
for i := 0; i < n; i++ {
|
||||||
|
h, err := libp2p.New(libp2p.ResourceManager(&network.NullResourceManager{}))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
t.Cleanup(func() { h.Close() })
|
||||||
|
out = append(out, h)
|
||||||
|
}
|
||||||
|
|
||||||
|
return out
|
||||||
|
}
|
||||||
|
|
||||||
// See https://github.com/libp2p/go-libp2p-pubsub/issues/426
|
// See https://github.com/libp2p/go-libp2p-pubsub/issues/426
|
||||||
func TestPubSubRemovesBlacklistedPeer(t *testing.T) {
|
func TestPubSubRemovesBlacklistedPeer(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
|
|
||||||
bl := NewMapBlacklist()
|
bl := NewMapBlacklist()
|
||||||
|
|
||||||
|
|||||||
@ -40,7 +40,7 @@ func TestRandomsubSmall(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getRandomsubs(ctx, hosts, 10)
|
psubs := getRandomsubs(ctx, hosts, 10)
|
||||||
|
|
||||||
connectAll(t, hosts)
|
connectAll(t, hosts)
|
||||||
@ -77,7 +77,7 @@ func TestRandomsubBig(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 50)
|
hosts := getDefaultHosts(t, 50)
|
||||||
psubs := getRandomsubs(ctx, hosts, 50)
|
psubs := getRandomsubs(ctx, hosts, 50)
|
||||||
|
|
||||||
connectSome(t, hosts, 12)
|
connectSome(t, hosts, 12)
|
||||||
@ -114,7 +114,7 @@ func TestRandomsubMixed(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 40)
|
hosts := getDefaultHosts(t, 40)
|
||||||
fsubs := getPubsubs(ctx, hosts[:10])
|
fsubs := getPubsubs(ctx, hosts[:10])
|
||||||
rsubs := getRandomsubs(ctx, hosts[10:], 30)
|
rsubs := getRandomsubs(ctx, hosts[10:], 30)
|
||||||
psubs := append(fsubs, rsubs...)
|
psubs := append(fsubs, rsubs...)
|
||||||
@ -153,7 +153,7 @@ func TestRandomsubEnoughPeers(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 40)
|
hosts := getDefaultHosts(t, 40)
|
||||||
fsubs := getPubsubs(ctx, hosts[:10])
|
fsubs := getPubsubs(ctx, hosts[:10])
|
||||||
rsubs := getRandomsubs(ctx, hosts[10:], 30)
|
rsubs := getRandomsubs(ctx, hosts[10:], 30)
|
||||||
psubs := append(fsubs, rsubs...)
|
psubs := append(fsubs, rsubs...)
|
||||||
|
|||||||
@ -150,7 +150,7 @@ func TestSubscriptionFilterRPC(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
ps1 := getPubsub(ctx, hosts[0], WithSubscriptionFilter(NewAllowlistSubscriptionFilter("test1", "test2")))
|
ps1 := getPubsub(ctx, hosts[0], WithSubscriptionFilter(NewAllowlistSubscriptionFilter("test1", "test2")))
|
||||||
ps2 := getPubsub(ctx, hosts[1], WithSubscriptionFilter(NewAllowlistSubscriptionFilter("test2", "test3")))
|
ps2 := getPubsub(ctx, hosts[1], WithSubscriptionFilter(NewAllowlistSubscriptionFilter("test2", "test3")))
|
||||||
|
|
||||||
|
|||||||
@ -99,7 +99,7 @@ func testTopicCloseWithOpenResource(t *testing.T, openResource func(topic *Topic
|
|||||||
|
|
||||||
const numHosts = 1
|
const numHosts = 1
|
||||||
topicID := "foobar"
|
topicID := "foobar"
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
ps := getPubsub(ctx, hosts[0])
|
ps := getPubsub(ctx, hosts[0])
|
||||||
|
|
||||||
// Try create and cancel topic
|
// Try create and cancel topic
|
||||||
@ -139,7 +139,7 @@ func TestTopicReuse(t *testing.T) {
|
|||||||
|
|
||||||
const numHosts = 2
|
const numHosts = 2
|
||||||
topicID := "foobar"
|
topicID := "foobar"
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
|
|
||||||
sender := getPubsub(ctx, hosts[0], WithDiscovery(&dummyDiscovery{}))
|
sender := getPubsub(ctx, hosts[0], WithDiscovery(&dummyDiscovery{}))
|
||||||
receiver := getPubsub(ctx, hosts[1])
|
receiver := getPubsub(ctx, hosts[1])
|
||||||
@ -233,7 +233,7 @@ func TestTopicEventHandlerCancel(t *testing.T) {
|
|||||||
|
|
||||||
const numHosts = 5
|
const numHosts = 5
|
||||||
topicID := "foobar"
|
topicID := "foobar"
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
ps := getPubsub(ctx, hosts[0])
|
ps := getPubsub(ctx, hosts[0])
|
||||||
|
|
||||||
// Try create and cancel topic
|
// Try create and cancel topic
|
||||||
@ -265,7 +265,7 @@ func TestSubscriptionJoinNotification(t *testing.T) {
|
|||||||
|
|
||||||
const numLateSubscribers = 10
|
const numLateSubscribers = 10
|
||||||
const numHosts = 20
|
const numHosts = 20
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), "foobar")
|
topics := getTopics(getPubsubs(ctx, hosts), "foobar")
|
||||||
evts := getTopicEvts(topics)
|
evts := getTopicEvts(topics)
|
||||||
|
|
||||||
@ -331,7 +331,7 @@ func TestSubscriptionLeaveNotification(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
const numHosts = 20
|
const numHosts = 20
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
topics := getTopics(psubs, "foobar")
|
topics := getTopics(psubs, "foobar")
|
||||||
evts := getTopicEvts(topics)
|
evts := getTopicEvts(topics)
|
||||||
@ -416,7 +416,7 @@ func TestSubscriptionManyNotifications(t *testing.T) {
|
|||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
|
|
||||||
const numHosts = 33
|
const numHosts = 33
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
evts := getTopicEvts(topics)
|
evts := getTopicEvts(topics)
|
||||||
|
|
||||||
@ -521,7 +521,7 @@ func TestSubscriptionNotificationSubUnSub(t *testing.T) {
|
|||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
|
|
||||||
const numHosts = 35
|
const numHosts = 35
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
|
|
||||||
for i := 1; i < numHosts; i++ {
|
for i := 1; i < numHosts; i++ {
|
||||||
@ -539,7 +539,7 @@ func TestTopicRelay(t *testing.T) {
|
|||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
const numHosts = 5
|
const numHosts = 5
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
|
|
||||||
// [0.Rel] - [1.Rel] - [2.Sub]
|
// [0.Rel] - [1.Rel] - [2.Sub]
|
||||||
@ -603,7 +603,7 @@ func TestTopicRelayReuse(t *testing.T) {
|
|||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
const numHosts = 1
|
const numHosts = 1
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
pubsubs := getPubsubs(ctx, hosts)
|
pubsubs := getPubsubs(ctx, hosts)
|
||||||
topics := getTopics(pubsubs, topic)
|
topics := getTopics(pubsubs, topic)
|
||||||
|
|
||||||
@ -670,7 +670,7 @@ func TestTopicRelayOnClosedTopic(t *testing.T) {
|
|||||||
const topic = "foobar"
|
const topic = "foobar"
|
||||||
const numHosts = 1
|
const numHosts = 1
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
|
|
||||||
err := topics[0].Close()
|
err := topics[0].Close()
|
||||||
@ -690,7 +690,7 @@ func TestProducePanic(t *testing.T) {
|
|||||||
|
|
||||||
const numHosts = 5
|
const numHosts = 5
|
||||||
topicID := "foobar"
|
topicID := "foobar"
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
ps := getPubsub(ctx, hosts[0])
|
ps := getPubsub(ctx, hosts[0])
|
||||||
|
|
||||||
// Create topic
|
// Create topic
|
||||||
@ -792,7 +792,7 @@ func TestMinTopicSizeNoDiscovery(t *testing.T) {
|
|||||||
|
|
||||||
const numHosts = 3
|
const numHosts = 3
|
||||||
topicID := "foobar"
|
topicID := "foobar"
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
|
|
||||||
sender := getPubsub(ctx, hosts[0])
|
sender := getPubsub(ctx, hosts[0])
|
||||||
receiver1 := getPubsub(ctx, hosts[1])
|
receiver1 := getPubsub(ctx, hosts[1])
|
||||||
@ -872,7 +872,7 @@ func TestWithTopicMsgIdFunction(t *testing.T) {
|
|||||||
const topicA, topicB = "foobarA", "foobarB"
|
const topicA, topicB = "foobarA", "foobarB"
|
||||||
const numHosts = 2
|
const numHosts = 2
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
pubsubs := getPubsubs(ctx, hosts, WithMessageIdFn(func(pmsg *pb.Message) string {
|
pubsubs := getPubsubs(ctx, hosts, WithMessageIdFn(func(pmsg *pb.Message) string {
|
||||||
hash := sha256.Sum256(pmsg.Data)
|
hash := sha256.Sum256(pmsg.Data)
|
||||||
return string(hash[:])
|
return string(hash[:])
|
||||||
@ -932,7 +932,7 @@ func TestTopicPublishWithKeyInvalidParameters(t *testing.T) {
|
|||||||
const numHosts = 5
|
const numHosts = 5
|
||||||
|
|
||||||
virtualPeer := tnet.RandPeerNetParamsOrFatal(t)
|
virtualPeer := tnet.RandPeerNetParamsOrFatal(t)
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
|
|
||||||
t.Run("nil sign private key should error", func(t *testing.T) {
|
t.Run("nil sign private key should error", func(t *testing.T) {
|
||||||
@ -959,7 +959,7 @@ func TestTopicRelayPublishWithKey(t *testing.T) {
|
|||||||
const numHosts = 5
|
const numHosts = 5
|
||||||
|
|
||||||
virtualPeer := tnet.RandPeerNetParamsOrFatal(t)
|
virtualPeer := tnet.RandPeerNetParamsOrFatal(t)
|
||||||
hosts := getNetHosts(t, ctx, numHosts)
|
hosts := getDefaultHosts(t, numHosts)
|
||||||
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
topics := getTopics(getPubsubs(ctx, hosts), topic)
|
||||||
|
|
||||||
// [0.Rel] - [1.Rel] - [2.Sub]
|
// [0.Rel] - [1.Rel] - [2.Sub]
|
||||||
@ -1026,7 +1026,7 @@ func TestWithLocalPublication(t *testing.T) {
|
|||||||
|
|
||||||
const topic = "test"
|
const topic = "test"
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
pubsubs := getPubsubs(ctx, hosts)
|
pubsubs := getPubsubs(ctx, hosts)
|
||||||
topics := getTopics(pubsubs, topic)
|
topics := getTopics(pubsubs, topic)
|
||||||
connectAll(t, hosts)
|
connectAll(t, hosts)
|
||||||
|
|||||||
@ -17,9 +17,6 @@ import (
|
|||||||
"github.com/libp2p/go-libp2p/core/peer"
|
"github.com/libp2p/go-libp2p/core/peer"
|
||||||
"github.com/libp2p/go-libp2p/core/peerstore"
|
"github.com/libp2p/go-libp2p/core/peerstore"
|
||||||
|
|
||||||
bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
|
|
||||||
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
|
|
||||||
|
|
||||||
"github.com/libp2p/go-msgio/protoio"
|
"github.com/libp2p/go-msgio/protoio"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -27,7 +24,7 @@ func testWithTracer(t *testing.T, tracer EventTracer) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getGossipsubs(ctx, hosts,
|
psubs := getGossipsubs(ctx, hosts,
|
||||||
WithEventTracer(tracer),
|
WithEventTracer(tracer),
|
||||||
// to bootstrap from star topology
|
// to bootstrap from star topology
|
||||||
@ -302,10 +299,9 @@ func TestRemoteTracer(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
h1 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
hosts := getDefaultHosts(t, 2)
|
||||||
h2 := bhost.NewBlankHost(swarmt.GenSwarm(t))
|
h1 := hosts[0]
|
||||||
defer h1.Close()
|
h2 := hosts[1]
|
||||||
defer h2.Close()
|
|
||||||
|
|
||||||
mrt := &mockRemoteTracer{}
|
mrt := &mockRemoteTracer{}
|
||||||
h1.SetStreamHandler(RemoteTracerProtoID, mrt.handleStream)
|
h1.SetStreamHandler(RemoteTracerProtoID, mrt.handleStream)
|
||||||
|
|||||||
@ -38,7 +38,7 @@ func testBasicSeqnoValidator(t *testing.T, ttl time.Duration) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getPubsubsWithOptionC(ctx, hosts,
|
psubs := getPubsubsWithOptionC(ctx, hosts,
|
||||||
func(i int) Option {
|
func(i int) Option {
|
||||||
return WithDefaultValidator(NewBasicSeqnoValidator(newMockPeerMetadataStore()))
|
return WithDefaultValidator(NewBasicSeqnoValidator(newMockPeerMetadataStore()))
|
||||||
@ -86,7 +86,7 @@ func TestBasicSeqnoValidatorReplay(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 20)
|
hosts := getDefaultHosts(t, 20)
|
||||||
psubs := getPubsubsWithOptionC(ctx, hosts[:19],
|
psubs := getPubsubsWithOptionC(ctx, hosts[:19],
|
||||||
func(i int) Option {
|
func(i int) Option {
|
||||||
return WithDefaultValidator(NewBasicSeqnoValidator(newMockPeerMetadataStore()))
|
return WithDefaultValidator(NewBasicSeqnoValidator(newMockPeerMetadataStore()))
|
||||||
|
|||||||
@ -15,7 +15,7 @@ func TestRegisterUnregisterValidator(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 1)
|
hosts := getDefaultHosts(t, 1)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
err := psubs[0].RegisterTopicValidator("foo", func(context.Context, peer.ID, *Message) bool {
|
err := psubs[0].RegisterTopicValidator("foo", func(context.Context, peer.ID, *Message) bool {
|
||||||
@ -40,7 +40,7 @@ func TestRegisterValidatorEx(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 3)
|
hosts := getDefaultHosts(t, 3)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
err := psubs[0].RegisterTopicValidator("test",
|
err := psubs[0].RegisterTopicValidator("test",
|
||||||
@ -69,7 +69,7 @@ func TestValidate(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -123,7 +123,7 @@ func TestValidate2(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 1)
|
hosts := getDefaultHosts(t, 1)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
topic := "foobar"
|
topic := "foobar"
|
||||||
@ -201,7 +201,7 @@ func TestValidateOverload(t *testing.T) {
|
|||||||
|
|
||||||
for tci, tc := range tcs {
|
for tci, tc := range tcs {
|
||||||
t.Run(fmt.Sprintf("%d", tci), func(t *testing.T) {
|
t.Run(fmt.Sprintf("%d", tci), func(t *testing.T) {
|
||||||
hosts := getNetHosts(t, ctx, 2)
|
hosts := getDefaultHosts(t, 2)
|
||||||
psubs := getPubsubs(ctx, hosts)
|
psubs := getPubsubs(ctx, hosts)
|
||||||
|
|
||||||
connect(t, hosts[0], hosts[1])
|
connect(t, hosts[0], hosts[1])
|
||||||
@ -273,7 +273,7 @@ func TestValidateAssortedOptions(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
hosts := getNetHosts(t, ctx, 10)
|
hosts := getDefaultHosts(t, 10)
|
||||||
psubs := getPubsubs(ctx, hosts,
|
psubs := getPubsubs(ctx, hosts,
|
||||||
WithValidateQueueSize(10),
|
WithValidateQueueSize(10),
|
||||||
WithValidateThrottle(10),
|
WithValidateThrottle(10),
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user