From a5addee5a075e38cb20db8ccd095eeffd9ecbc18 Mon Sep 17 00:00:00 2001 From: Dirk McCormick Date: Mon, 13 Apr 2020 18:53:52 -0400 Subject: [PATCH] test: add IWANT spam test --- gossipsub_test.go | 142 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 142 insertions(+) diff --git a/gossipsub_test.go b/gossipsub_test.go index 98767e7..a217d9e 100644 --- a/gossipsub_test.go +++ b/gossipsub_test.go @@ -8,8 +8,12 @@ import ( "testing" "time" + ggio "github.com/gogo/protobuf/io" "github.com/libp2p/go-libp2p-core/host" + "github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p-core/peerstore" + "github.com/libp2p/go-libp2p-core/protocol" + pb "github.com/libp2p/go-libp2p-pubsub/pb" ) func getGossipsub(ctx context.Context, h host.Host, opts ...Option) *PubSub { @@ -982,3 +986,141 @@ func TestGossipsubStarTopology(t *testing.T) { } } } + +// Test that when Gossipsub receives too many IWANT messages from a peer +// for the same message ID, it cuts off the peer +func TestGossipsubSpamIWANT(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Create legitimate and attacker hosts + hosts := getNetHosts(t, ctx, 2) + legit := hosts[0] + attacker := hosts[1] + + // Set up gossipsub on the legit host + ps, err := NewGossipSub(ctx, legit) + if err != nil { + t.Fatal(err) + } + + // Subscribe to mytopic on the legit host + mytopic := "mytopic" + _, err = ps.Subscribe(mytopic) + if err != nil { + t.Fatal(err) + } + + // Used to publish a message with random data + publishMsg := func() { + data := make([]byte, 16) + rand.Read(data) + + if err = ps.Publish(mytopic, data); err != nil { + t.Fatal(err) + } + } + + // Listen on the gossipsub protocol + const gossipSubID = protocol.ID("/meshsub/1.0.0") + const maxMessageSize = 1024 * 1024 + attacker.SetStreamHandler(gossipSubID, func(stream network.Stream) { + // When an incoming stream is openeed, set up an outgoing stream + p := stream.Conn().RemotePeer() + ostream, err := attacker.NewStream(ctx, p, gossipSubID) + if err != nil { + t.Fatal(err) + } + + r := ggio.NewDelimitedReader(stream, maxMessageSize) + w := ggio.NewDelimitedWriter(ostream) + + var irpc pb.RPC + + writeMsg := func(rpc *pb.RPC) { + if err = w.WriteMsg(rpc); err != nil { + t.Fatalf("error writing RPC: %s", err) + } + } + + // Wait for 200ms after the last message before checking we got the + // right number of messages + msgWaitMax := 200 * time.Millisecond + msgCount := 0 + msgTimer := time.NewTimer(msgWaitMax) + + // Checks we received the right number of messages + checkMsgCount := func() { + // After the original message from the legit host, we keep sending + // IWANT until it stops replying. So the number of messages is + // + GossipSubGossipRetransmission + exp := 1 + GossipSubGossipRetransmission + if msgCount != exp { + t.Fatalf("Expected %d messages, got %d", exp, msgCount) + } + } + + // Wait for the timer to expire + go func() { + select { + case <-msgTimer.C: + checkMsgCount() + cancel() + return + case <-ctx.Done(): + checkMsgCount() + } + }() + + // Keep reading messages and responding + for { + irpc.Reset() + + r.ReadMsg(&irpc) + + // When the legit host connects it will send us its subscriptions + for _, sub := range irpc.GetSubscriptions() { + if sub.GetSubscribe() { + // Reply by subcribing to the topic and grafting to the peer + var subs []*pb.RPC_SubOpts + var graft []*pb.ControlGraft + subs = append(subs, &pb.RPC_SubOpts{Subscribe: sub.Subscribe, Topicid: sub.Topicid}) + graft = append(graft, &pb.ControlGraft{TopicID: sub.Topicid}) + + writeMsg(&pb.RPC{ + Subscriptions: subs, + Control: &pb.ControlMessage{Graft: graft}, + }) + + go func() { + // Wait for a short interval to make sure the legit host + // received and processed the subscribe + graft + time.Sleep(100 * time.Millisecond) + + // Publish a message from the legit host + publishMsg() + }() + } + } + + // Each time the legit host sends a message + for _, msg := range irpc.GetPublish() { + // Increment the number of messages and reset the timer + msgCount++ + msgTimer.Reset(msgWaitMax) + + // Send an IWANT with the message ID, causing the legit host + // to send another message (until it cuts off the attacker for + // being spammy) + iwantlst := []string{DefaultMsgIdFn(msg)} + iwant := []*pb.ControlIWant{&pb.ControlIWant{MessageIDs: iwantlst}} + orpc := rpcWithControl(nil, nil, iwant, nil, nil) + writeMsg(&orpc.RPC) + } + } + }) + + connect(t, hosts[0], hosts[1]) + + <-ctx.Done() +}