mirror of
https://github.com/logos-messaging/go-libp2p-pubsub.git
synced 2026-01-04 05:43:06 +00:00
allow gossipsub to graylist peers based on score
This commit is contained in:
parent
87bbe69a8b
commit
2ddc4b999f
@ -67,6 +67,10 @@ func (fs *FloodSubRouter) EnoughPeers(topic string, suggested int) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (fs *FloodSubRouter) AcceptFrom(peer.ID) bool {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
func (fs *FloodSubRouter) HandleRPC(rpc *RPC) {}
|
func (fs *FloodSubRouter) HandleRPC(rpc *RPC) {}
|
||||||
|
|
||||||
func (fs *FloodSubRouter) Publish(msg *Message) {
|
func (fs *FloodSubRouter) Publish(msg *Message) {
|
||||||
|
|||||||
20
gossipsub.go
20
gossipsub.go
@ -74,10 +74,18 @@ func NewGossipSub(ctx context.Context, h host.Host, opts ...Option) (*PubSub, er
|
|||||||
}
|
}
|
||||||
|
|
||||||
// WithPeerScore is a gossipsub router option that enables peer scoring.
|
// WithPeerScore is a gossipsub router option that enables peer scoring.
|
||||||
|
//
|
||||||
// gossipThreshold is the score threshold below which gossip propagation is supressed.
|
// gossipThreshold is the score threshold below which gossip propagation is supressed.
|
||||||
|
//
|
||||||
// publishThreshold is the score threshold below which we shouldn't publish when using flood
|
// publishThreshold is the score threshold below which we shouldn't publish when using flood
|
||||||
// publishing (also applies to fanout and floodsub peers).
|
// publishing (also applies to fanout and floodsub peers).
|
||||||
func WithPeerScore(params *PeerScoreParams, gossipThreshold float64, publishThreshold float64) Option {
|
//
|
||||||
|
// graylistThreshold is the score threshold below which message processing is supressed altogether,
|
||||||
|
// implementing an effective graylist according to peer score.
|
||||||
|
//
|
||||||
|
// These thresholds should generally be negative, allowing some information to disseminate from low
|
||||||
|
// scoring peers.
|
||||||
|
func WithPeerScore(params *PeerScoreParams, gossipThreshold, publishThreshold, graylistThreshold float64) Option {
|
||||||
return func(ps *PubSub) error {
|
return func(ps *PubSub) error {
|
||||||
gs, ok := ps.rt.(*GossipSubRouter)
|
gs, ok := ps.rt.(*GossipSubRouter)
|
||||||
if !ok {
|
if !ok {
|
||||||
@ -87,6 +95,7 @@ func WithPeerScore(params *PeerScoreParams, gossipThreshold float64, publishThre
|
|||||||
gs.score = newPeerScore(gs, params)
|
gs.score = newPeerScore(gs, params)
|
||||||
gs.gossipThreshold = gossipThreshold
|
gs.gossipThreshold = gossipThreshold
|
||||||
gs.publishThreshold = publishThreshold
|
gs.publishThreshold = publishThreshold
|
||||||
|
gs.graylistThreshold = graylistThreshold
|
||||||
|
|
||||||
// hook the tracer
|
// hook the tracer
|
||||||
if ps.tracer != nil {
|
if ps.tracer != nil {
|
||||||
@ -145,6 +154,9 @@ type GossipSubRouter struct {
|
|||||||
// when using flood publishing or the peer is a fanout or floodsub peer.
|
// when using flood publishing or the peer is a fanout or floodsub peer.
|
||||||
publishThreshold float64
|
publishThreshold float64
|
||||||
|
|
||||||
|
// threshold for peer score before we graylist the peer and silently ignore its RPCs
|
||||||
|
graylistThreshold float64
|
||||||
|
|
||||||
// whether to use flood publishing
|
// whether to use flood publishing
|
||||||
floodPublish bool
|
floodPublish bool
|
||||||
}
|
}
|
||||||
@ -218,6 +230,10 @@ func (gs *GossipSubRouter) EnoughPeers(topic string, suggested int) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (gs *GossipSubRouter) AcceptFrom(p peer.ID) bool {
|
||||||
|
return gs.score.Score(p) >= gs.graylistThreshold
|
||||||
|
}
|
||||||
|
|
||||||
func (gs *GossipSubRouter) HandleRPC(rpc *RPC) {
|
func (gs *GossipSubRouter) HandleRPC(rpc *RPC) {
|
||||||
ctl := rpc.GetControl()
|
ctl := rpc.GetControl()
|
||||||
if ctl == nil {
|
if ctl == nil {
|
||||||
|
|||||||
11
pubsub.go
11
pubsub.go
@ -151,6 +151,11 @@ type PubSubRouter interface {
|
|||||||
// EnoughPeers returns whether the router needs more peers before it's ready to publish new records.
|
// EnoughPeers returns whether the router needs more peers before it's ready to publish new records.
|
||||||
// Suggested (if greater than 0) is a suggested number of peers that the router should need.
|
// Suggested (if greater than 0) is a suggested number of peers that the router should need.
|
||||||
EnoughPeers(topic string, suggested int) bool
|
EnoughPeers(topic string, suggested int) bool
|
||||||
|
// AcceptFrom is invoked on any incoming message before pushing it to the validation pipeline
|
||||||
|
// or processing control information.
|
||||||
|
// Allows routers with internal scoring to vet peers before commiting any processing resources
|
||||||
|
// to the message and implement an affective graylist.
|
||||||
|
AcceptFrom(peer.ID) bool
|
||||||
// HandleRPC is invoked to process control messages in the RPC envelope.
|
// HandleRPC is invoked to process control messages in the RPC envelope.
|
||||||
// It is invoked after subscriptions and payload messages have been processed.
|
// It is invoked after subscriptions and payload messages have been processed.
|
||||||
HandleRPC(*RPC)
|
HandleRPC(*RPC)
|
||||||
@ -779,6 +784,12 @@ func (p *PubSub) handleIncomingRPC(rpc *RPC) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ask the router to vet the peer before commiting any processing resources
|
||||||
|
if !p.rt.AcceptFrom(rpc.from) {
|
||||||
|
log.Warningf("received message from router graylisted peer %s. Dropping RPC", rpc.from)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
for _, pmsg := range rpc.GetPublish() {
|
for _, pmsg := range rpc.GetPublish() {
|
||||||
if !p.subscribedToMsg(pmsg) {
|
if !p.subscribedToMsg(pmsg) {
|
||||||
log.Warning("received message we didn't subscribe to. Dropping.")
|
log.Warning("received message we didn't subscribe to. Dropping.")
|
||||||
|
|||||||
@ -86,6 +86,10 @@ func (rs *RandomSubRouter) EnoughPeers(topic string, suggested int) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (rs *RandomSubRouter) AcceptFrom(peer.ID) bool {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
func (rs *RandomSubRouter) HandleRPC(rpc *RPC) {}
|
func (rs *RandomSubRouter) HandleRPC(rpc *RPC) {}
|
||||||
|
|
||||||
func (rs *RandomSubRouter) Publish(msg *Message) {
|
func (rs *RandomSubRouter) Publish(msg *Message) {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user