Skip to content

Commit

Permalink
test: use the regular libp2p host
Browse files Browse the repository at this point in the history
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).
  • Loading branch information
Stebalien committed Jul 11, 2024
1 parent 8e498e9 commit 890a45a
Show file tree
Hide file tree
Showing 15 changed files with 139 additions and 156 deletions.
6 changes: 3 additions & 3 deletions blacklist_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ func TestBlacklist(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)
connect(t, hosts[0], hosts[1])

Expand Down Expand Up @@ -66,7 +66,7 @@ func TestBlacklist2(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)
connect(t, hosts[0], hosts[1])

Expand Down Expand Up @@ -99,7 +99,7 @@ func TestBlacklist3(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)

psubs[1].BlacklistPeer(hosts[0].ID())
Expand Down
4 changes: 2 additions & 2 deletions discovery_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ func TestSimpleDiscovery(t *testing.T) {
server := newDiscoveryServer()
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(1 * time.Minute)}

hosts := getNetHosts(t, ctx, numHosts)
hosts := getDefaultHosts(t, numHosts)
psubs := make([]*PubSub, numHosts)
topicHandlers := make([]*Topic, numHosts)

Expand Down Expand Up @@ -234,7 +234,7 @@ func TestGossipSubDiscoveryAfterBootstrap(t *testing.T) {
discOpts := []discovery.Option{discovery.Limit(numHosts), discovery.TTL(ttl)}

// Put the pubsub clients into two partitions
hosts := getNetHosts(t, ctx, numHosts)
hosts := getDefaultHosts(t, numHosts)
psubs := make([]*PubSub, numHosts)
topicHandlers := make([]*Topic, numHosts)

Expand Down
73 changes: 27 additions & 46 deletions floodsub_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,6 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"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"
)

Expand All @@ -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) {
pinfo := a.Peerstore().PeerInfo(a.ID())
err := b.Connect(context.Background(), pinfo)
Expand Down Expand Up @@ -151,7 +135,7 @@ func assertNeverReceives(t *testing.T, ch *Subscription, timeout time.Duration)
func TestBasicFloodsub(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getNetHosts(t, ctx, 20)
hosts := getDefaultHosts(t, 20)

psubs := getPubsubs(ctx, hosts)

Expand Down Expand Up @@ -193,7 +177,7 @@ func TestMultihops(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 6)
hosts := getDefaultHosts(t, 6)

psubs := getPubsubs(ctx, hosts)

Expand Down Expand Up @@ -235,7 +219,7 @@ func TestReconnects(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 3)
hosts := getDefaultHosts(t, 3)

psubs := getPubsubs(ctx, hosts)

Expand Down Expand Up @@ -309,7 +293,7 @@ func TestNoConnection(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 10)
hosts := getDefaultHosts(t, 10)

psubs := getPubsubs(ctx, hosts)

Expand All @@ -334,7 +318,7 @@ func TestSelfReceive(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

host := getNetHosts(t, ctx, 1)[0]
host := getDefaultHosts(t, 1)[0]

psub, err := NewFloodSub(ctx, host)
if err != nil {
Expand Down Expand Up @@ -368,7 +352,7 @@ func TestOneToOne(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -401,7 +385,7 @@ func TestTreeTopology(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 10)
hosts := getDefaultHosts(t, 10)
psubs := getPubsubs(ctx, hosts)

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -464,7 +448,7 @@ func TestFloodSubPluggableProtocol(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 3)
hosts := getDefaultHosts(t, 3)

psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub", "/lsr/floodsub")
psubB := mustCreatePubSub(ctx, t, hosts[1], "/esh/floodsub")
Expand Down Expand Up @@ -496,7 +480,7 @@ func TestFloodSubPluggableProtocol(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)

psubA := mustCreatePubSub(ctx, t, hosts[0], "/esh/floodsub")
psubB := mustCreatePubSub(ctx, t, hosts[1], "/lsr/floodsub")
Expand Down Expand Up @@ -551,7 +535,7 @@ func TestSubReporting(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

host := getNetHosts(t, ctx, 1)[0]
host := getDefaultHosts(t, 1)[0]
psub, err := NewFloodSub(ctx, host)
if err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -593,7 +577,7 @@ func TestPeerTopicReporting(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 4)
hosts := getDefaultHosts(t, 4)
psubs := getPubsubs(ctx, hosts)

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -650,7 +634,7 @@ func TestSubscribeMultipleTimes(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -695,7 +679,7 @@ func TestPeerDisconnect(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts)

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -743,7 +727,7 @@ func TestWithNoSigning(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts, WithNoAuthor(), WithMessageIdFn(func(pmsg *pb.Message) string {
// silly content-based test message-ID: just use the data as whole
return base64.URLEncoding.EncodeToString(pmsg.Data)
Expand Down Expand Up @@ -788,7 +772,7 @@ func TestWithSigning(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts, WithStrictSignatureVerification(true))

connect(t, hosts[0], hosts[1])
Expand Down Expand Up @@ -830,7 +814,7 @@ func TestImproperlySignedMessageRejected(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
adversary := hosts[0]
honestPeer := hosts[1]

Expand Down Expand Up @@ -948,7 +932,7 @@ func TestMessageSender(t *testing.T) {

const topic = "foobar"

hosts := getNetHosts(t, ctx, 3)
hosts := getDefaultHosts(t, 3)
psubs := getPubsubs(ctx, hosts)

var msgs []*Subscription
Expand Down Expand Up @@ -1002,7 +986,7 @@ func TestConfigurableMaxMessageSize(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
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.
psubs := getPubsubs(ctx, hosts, WithMaxMessageSize(1<<22))
Expand Down Expand Up @@ -1045,7 +1029,7 @@ func TestAnnounceRetry(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
ps := getPubsub(ctx, hosts[0])
watcher := &announceWatcher{}
hosts[1].SetStreamHandler(FloodSubID, watcher.handleStream)
Expand Down Expand Up @@ -1117,7 +1101,7 @@ func TestPubsubWithAssortedOptions(t *testing.T) {
return string(hash[:])
}

hosts := getNetHosts(t, ctx, 2)
hosts := getDefaultHosts(t, 2)
psubs := getPubsubs(ctx, hosts,
WithMessageIdFn(hashMsgID),
WithPeerOutboundQueueSize(10),
Expand Down Expand Up @@ -1152,8 +1136,7 @@ func TestWithInvalidMessageAuthor(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

h := bhost.NewBlankHost(swarmt.GenSwarm(t))
defer h.Close()
h := getDefaultHosts(t, 1)[0]
_, err := NewFloodSub(ctx, h, WithMessageAuthor("bogotr0n"))
if err == nil {
t.Fatal("expected error")
Expand All @@ -1168,10 +1151,9 @@ func TestPreconnectedNodes(t *testing.T) {
defer cancel()

// Create hosts
h1 := bhost.NewBlankHost(swarmt.GenSwarm(t))
h2 := bhost.NewBlankHost(swarmt.GenSwarm(t))
defer h1.Close()
defer h2.Close()
hosts := getDefaultHosts(t, 2)
h1 := hosts[0]
h2 := hosts[1]

opts := []Option{WithDiscovery(&dummyDiscovery{})}
// Setup first PubSub
Expand Down Expand Up @@ -1229,10 +1211,9 @@ func TestDedupInboundStreams(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

h1 := bhost.NewBlankHost(swarmt.GenSwarm(t))
h2 := bhost.NewBlankHost(swarmt.GenSwarm(t))
defer h1.Close()
defer h2.Close()
hosts := getDefaultHosts(t, 2)
h1 := hosts[0]
h2 := hosts[1]

_, err := NewFloodSub(ctx, h1)
if err != nil {
Expand Down
17 changes: 11 additions & 6 deletions gossipsub_connmgr_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,10 +7,10 @@ import (

"github.com/benbjohnson/clock"
"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"
bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
"github.com/libp2p/go-libp2p/p2p/net/connmgr"
)

Expand Down Expand Up @@ -70,9 +70,14 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
t.Fatal(err)
}

netw := swarmt.GenSwarm(t)
defer netw.Close()
h := bhost.NewBlankHost(netw, bhost.WithConnectionManager(connmgrs[i]))
h, err := libp2p.New(
libp2p.ResourceManager(&network.NullResourceManager{}),
libp2p.ConnectionManager(connmgrs[i]),
)
if err != nil {
t.Fatal(err)
}
t.Cleanup(func() { h.Close() })
honestHosts[i] = h
honestPeers[h.ID()] = struct{}{}
}
Expand All @@ -83,7 +88,7 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
WithFloodPublish(true))

// sybil squatters to be connected later
sybilHosts := getNetHosts(t, ctx, nSquatter)
sybilHosts := getDefaultHosts(t, nSquatter)
for _, h := range sybilHosts {
squatter := &sybilSquatter{h: h}
h.SetStreamHandler(GossipSubID_v10, squatter.handleStream)
Expand Down
2 changes: 1 addition & 1 deletion gossipsub_feat_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestGossipSubCustomProtocols(t *testing.T) {

ctx, cancel := context.WithCancel(context.Background())
defer cancel()
hosts := getNetHosts(t, ctx, 3)
hosts := getDefaultHosts(t, 3)

gsubs := getGossipsubs(ctx, hosts[:2], WithGossipSubProtocols(protos, features))
fsub := getPubsub(ctx, hosts[2])
Expand Down
2 changes: 1 addition & 1 deletion gossipsub_matchfn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ func TestGossipSubMatchingFn(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

h := getNetHosts(t, ctx, 4)
h := getDefaultHosts(t, 4)
psubs := []*PubSub{
getGossipsub(ctx, h[0], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA100, GossipSubID_v11}, GossipSubDefaultFeatures)),
getGossipsub(ctx, h[1], WithProtocolMatchFn(protocolNameMatch), WithGossipSubProtocols([]protocol.ID{customsubA101Beta}, GossipSubDefaultFeatures)),
Expand Down
Loading

0 comments on commit 890a45a

Please sign in to comment.