better backoff handling

- Specify backoff in PRUNE message
- Obey peer backoff if specified
- Remove BackoffPenalty, handle lack of respect for backoff through P7
This commit is contained in:
vyzo 2020-05-06 20:26:20 +03:00
parent dd069798bb
commit 5363d59203
2 changed files with 91 additions and 36 deletions

View File

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

View File

@ -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")
}
}()
}