diff --git a/gossipsub.go b/gossipsub.go index a519712..4ea25b0 100644 --- a/gossipsub.go +++ b/gossipsub.go @@ -75,13 +75,9 @@ var ( GossipSubOpportunisticGraftPeers = 2 // If a GRAFT comes before GossipSubGraftFloodThreshold has ellapsed since the last PRUNE, - // then there is no PRUNE response emitted. This protects against GRAFT floods and should be - // less than GossipSubPruneBackoff. + // then there is an extra score penalty applied to the peer through P7. GossipSubGraftFloodThreshold = 10 * time.Second - // backoff penalty for GRAFT floods - GossipSubPruneBackoffPenalty = time.Hour - // Maximum number of messages to include in an IHAVE message. Also controls the maximum // number of IHAVE ids we will accept and request with IWANT from a peer within a heartbeat, // to protect from IHAVE floods. You should adjust this value from the default if your @@ -538,18 +534,19 @@ func (gs *GossipSubRouter) handleGraft(p peer.ID, ctl *pb.ControlMessage) []*pb. expire, backoff := gs.backoff[topic][p] if backoff && now.Before(expire) { log.Debugf("GRAFT: ignoring backed off peer %s", p) + // add behavioural penalty + gs.score.AddPenalty(p, 1) + // no PX + doPX = false // check the flood cutoff -- is the GRAFT coming too fast? floodCutoff := expire.Add(GossipSubGraftFloodThreshold - GossipSubPruneBackoff) if now.Before(floodCutoff) { - // no prune, and no PX either - doPX = false - // and a penalty so that we don't GRAFT on this peer ourselves for a while - gs.addBackoffPenalty(p, topic) - } else { - prune = append(prune, topic) - // refresh the backoff - gs.addBackoff(p, topic) + // extra penalty + gs.score.AddPenalty(p, 1) } + // refresh the backoff + gs.addBackoff(p, topic) + prune = append(prune, topic) continue } @@ -598,7 +595,13 @@ func (gs *GossipSubRouter) handlePrune(p peer.ID, ctl *pb.ControlMessage) { gs.tracer.Prune(p, topic) delete(peers, p) gs.untagPeer(p, topic) - gs.addBackoff(p, topic) + // is there a backoff specified by the peer? if so obey it. + backoff := prune.GetBackoff() + if backoff > 0 { + gs.doAddBackoff(p, topic, time.Duration(backoff)*time.Second) + } else { + gs.addBackoff(p, topic) + } px := prune.GetPeers() if len(px) > 0 { @@ -617,10 +620,6 @@ func (gs *GossipSubRouter) addBackoff(p peer.ID, topic string) { gs.doAddBackoff(p, topic, GossipSubPruneBackoff) } -func (gs *GossipSubRouter) addBackoffPenalty(p peer.ID, topic string) { - gs.doAddBackoff(p, topic, GossipSubPruneBackoffPenalty) -} - func (gs *GossipSubRouter) doAddBackoff(p peer.ID, topic string, interval time.Duration) { backoff, ok := gs.backoff[topic] if !ok { @@ -1503,6 +1502,7 @@ func (gs *GossipSubRouter) makePrune(p peer.ID, topic string, doPX bool) *pb.Con return &pb.ControlPrune{TopicID: &topic} } + backoff := uint64(GossipSubPruneBackoff / time.Second) var px []*pb.PeerInfo if doPX { // select peers for Peer eXchange @@ -1531,7 +1531,7 @@ func (gs *GossipSubRouter) makePrune(p peer.ID, topic string, doPX bool) *pb.Con } } - return &pb.ControlPrune{TopicID: &topic, Peers: px} + return &pb.ControlPrune{TopicID: &topic, Peers: px, Backoff: &backoff} } func (gs *GossipSubRouter) getPeers(topic string, count int, filter func(peer.ID) bool) []peer.ID { diff --git a/gossipsub_spam_test.go b/gossipsub_spam_test.go index 11dd7ef..a3c801f 100644 --- a/gossipsub_spam_test.go +++ b/gossipsub_spam_test.go @@ -312,17 +312,15 @@ func TestGossipsubAttackGRAFTNonExistentTopic(t *testing.T) { } // Test that when Gossipsub receives GRAFT for a peer that has been PRUNED, -// it ignores the request if the GRAFTs are coming too fast +// it penalizes through P7 and eventually graylists and ignores the requests if the +// GRAFTs are coming too fast func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { originalGossipSubPruneBackoff := GossipSubPruneBackoff GossipSubPruneBackoff = 200 * time.Millisecond originalGossipSubGraftFloodThreshold := GossipSubGraftFloodThreshold GossipSubGraftFloodThreshold = 100 * time.Millisecond - originalGossipSubPruneBackoffPenalty := GossipSubPruneBackoffPenalty - GossipSubPruneBackoffPenalty = 500 * time.Millisecond defer func() { GossipSubPruneBackoff = originalGossipSubPruneBackoff - GossipSubPruneBackoffPenalty = originalGossipSubPruneBackoffPenalty GossipSubGraftFloodThreshold = originalGossipSubGraftFloodThreshold }() @@ -335,7 +333,20 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { attacker := hosts[1] // Set up gossipsub on the legit host - ps, err := NewGossipSub(ctx, legit) + ps, err := NewGossipSub(ctx, legit, + WithPeerScore( + &PeerScoreParams{ + AppSpecificScore: func(peer.ID) float64 { return 0 }, + BehaviourPenaltyWeight: -100, + BehaviourPenaltyDecay: ScoreParameterDecay(time.Minute), + DecayInterval: DefaultDecayInterval, + DecayToZero: DefaultDecayToZero, + }, + &PeerScoreThresholds{ + GossipThreshold: -100, + PublishThreshold: -500, + GraylistThreshold: -1000, + })) if err != nil { t.Fatal(err) } @@ -401,7 +412,7 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { } // wait for the GossipSubGraftFloodThreshold to pass before attempting another graft - time.Sleep(GossipSubGraftFloodThreshold) + time.Sleep(GossipSubGraftFloodThreshold + time.Millisecond) // Send a GRAFT to attempt to rejoin the mesh writeMsg(&pb.RPC{ @@ -410,15 +421,18 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { time.Sleep(20 * time.Millisecond) - // It's been less than the flood threshold time since the last - // PRUNE, so we shouldn't get any prunes back + // We should have been peanalized by the peer for sending before the backoff has expired + // but should still receive a PRUNE because we haven't dropped below GraylistThreshold + // yet. pc = getPruneCount() if pc != 1 { t.Fatalf("Expected %d PRUNE messages but got %d", 1, pc) } - // Wait until after the prune backoff penalty period - time.Sleep(GossipSubPruneBackoffPenalty + time.Second) + score1 := ps.rt.(*GossipSubRouter).score.Score(attacker.ID()) + if score1 >= 0 { + t.Fatalf("Expected negative score, but got %f", score1) + } // Send a GRAFT again to attempt to rejoin the mesh writeMsg(&pb.RPC{ @@ -427,14 +441,55 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { time.Sleep(20 * time.Millisecond) - // The prune backoff period has passed so the GRAFT should - // be accepted and this node should not receive a PRUNE + // we are before the flood threshold so we should be penalized twice, but still get + // a PRUNE because we are before the flood threshold pc = getPruneCount() - if pc != 1 { - t.Fatalf("Expected %d PRUNE messages but got %d", 1, pc) + if pc != 2 { + t.Fatalf("Expected %d PRUNE messages but got %d", 2, pc) } - // make sure we are in the mesh of the legit host now + score2 := ps.rt.(*GossipSubRouter).score.Score(attacker.ID()) + if score2 >= score1 { + t.Fatalf("Expected score below %f, but got %f", score1, score2) + } + + // Send another GRAFT; this should get us a PRUNE, but penalize us below the graylist threshold + writeMsg(&pb.RPC{ + Control: &pb.ControlMessage{Graft: graft}, + }) + + time.Sleep(20 * time.Millisecond) + + pc = getPruneCount() + if pc != 3 { + t.Fatalf("Expected %d PRUNE messages but got %d", 3, pc) + } + + score3 := ps.rt.(*GossipSubRouter).score.Score(attacker.ID()) + if score3 >= score2 { + t.Fatalf("Expected score below %f, but got %f", score2, score3) + } + if score3 >= -1000 { + t.Fatalf("Expected score below %f, but got %f", -1000.0, score3) + } + + // Wait for the PRUNE backoff to expire and try again; this time we should fail + // because we are below the graylist threshold, so our RPC should be ignored and + // we should get no PRUNE back + time.Sleep(GossipSubPruneBackoff + time.Millisecond) + + writeMsg(&pb.RPC{ + Control: &pb.ControlMessage{Graft: graft}, + }) + + time.Sleep(20 * time.Millisecond) + + pc = getPruneCount() + if pc != 3 { + t.Fatalf("Expected %d PRUNE messages but got %d", 3, pc) + } + + // make sure we are _not_ in the mesh res := make(chan bool) ps.eval <- func() { mesh := ps.rt.(*GossipSubRouter).mesh[mytopic] @@ -443,8 +498,8 @@ func TestGossipsubAttackGRAFTDuringBackoff(t *testing.T) { } inMesh := <-res - if !inMesh { - t.Fatal("Expected to be in the mesh of the legitimate host") + if inMesh { + t.Fatal("Expected to not be in the mesh of the legitimate host") } }() }