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:
parent
dd069798bb
commit
5363d59203
34
gossipsub.go
34
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)
|
||||
// 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)
|
||||
// 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 {
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue