mirror of
https://github.com/logos-messaging/go-libp2p-pubsub.git
synced 2026-01-03 21:33:07 +00:00
export rejection named string constants
This commit is contained in:
parent
6c1addf493
commit
05c505ef60
@ -133,9 +133,9 @@ func (gt *gossipTracer) RejectMessage(msg *Message, reason string) {
|
|||||||
// We do take exception and apply promise penalty regardless in the following cases, where
|
// We do take exception and apply promise penalty regardless in the following cases, where
|
||||||
// the peer delivered an obviously invalid message.
|
// the peer delivered an obviously invalid message.
|
||||||
switch reason {
|
switch reason {
|
||||||
case rejectMissingSignature:
|
case RejectMissingSignature:
|
||||||
return
|
return
|
||||||
case rejectInvalidSignature:
|
case RejectInvalidSignature:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -411,13 +411,13 @@ func (pg *peerGater) RejectMessage(msg *Message, reason string) {
|
|||||||
defer pg.Unlock()
|
defer pg.Unlock()
|
||||||
|
|
||||||
switch reason {
|
switch reason {
|
||||||
case rejectValidationQueueFull:
|
case RejectValidationQueueFull:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectValidationThrottled:
|
case RejectValidationThrottled:
|
||||||
pg.lastThrottle = time.Now()
|
pg.lastThrottle = time.Now()
|
||||||
pg.throttle++
|
pg.throttle++
|
||||||
|
|
||||||
case rejectValidationIgnored:
|
case RejectValidationIgnored:
|
||||||
st := pg.getPeerStats(msg.ReceivedFrom)
|
st := pg.getPeerStats(msg.ReceivedFrom)
|
||||||
st.ignore++
|
st.ignore++
|
||||||
|
|
||||||
|
|||||||
@ -46,21 +46,21 @@ func TestPeerGater(t *testing.T) {
|
|||||||
t.Fatal("expected AcceptAll")
|
t.Fatal("expected AcceptAll")
|
||||||
}
|
}
|
||||||
|
|
||||||
pg.RejectMessage(msg, rejectValidationQueueFull)
|
pg.RejectMessage(msg, RejectValidationQueueFull)
|
||||||
status = pg.AcceptFrom(peerA)
|
status = pg.AcceptFrom(peerA)
|
||||||
if status != AcceptAll {
|
if status != AcceptAll {
|
||||||
t.Fatal("expected AcceptAll")
|
t.Fatal("expected AcceptAll")
|
||||||
}
|
}
|
||||||
|
|
||||||
pg.RejectMessage(msg, rejectValidationThrottled)
|
pg.RejectMessage(msg, RejectValidationThrottled)
|
||||||
status = pg.AcceptFrom(peerA)
|
status = pg.AcceptFrom(peerA)
|
||||||
if status != AcceptAll {
|
if status != AcceptAll {
|
||||||
t.Fatal("expected AcceptAll")
|
t.Fatal("expected AcceptAll")
|
||||||
}
|
}
|
||||||
|
|
||||||
for i := 0; i < 100; i++ {
|
for i := 0; i < 100; i++ {
|
||||||
pg.RejectMessage(msg, rejectValidationIgnored)
|
pg.RejectMessage(msg, RejectValidationIgnored)
|
||||||
pg.RejectMessage(msg, rejectValidationFailed)
|
pg.RejectMessage(msg, RejectValidationFailed)
|
||||||
}
|
}
|
||||||
|
|
||||||
accepted := false
|
accepted := false
|
||||||
|
|||||||
12
pubsub.go
12
pubsub.go
@ -987,14 +987,14 @@ func (p *PubSub) pushMsg(msg *Message) {
|
|||||||
// reject messages from blacklisted peers
|
// reject messages from blacklisted peers
|
||||||
if p.blacklist.Contains(src) {
|
if p.blacklist.Contains(src) {
|
||||||
log.Debugf("dropping message from blacklisted peer %s", src)
|
log.Debugf("dropping message from blacklisted peer %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectBlacklstedPeer)
|
p.tracer.RejectMessage(msg, RejectBlacklstedPeer)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// even if they are forwarded by good peers
|
// even if they are forwarded by good peers
|
||||||
if p.blacklist.Contains(msg.GetFrom()) {
|
if p.blacklist.Contains(msg.GetFrom()) {
|
||||||
log.Debugf("dropping message from blacklisted source %s", src)
|
log.Debugf("dropping message from blacklisted source %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectBlacklistedSource)
|
p.tracer.RejectMessage(msg, RejectBlacklistedSource)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1003,7 +1003,7 @@ func (p *PubSub) pushMsg(msg *Message) {
|
|||||||
if p.signPolicy.mustSign() {
|
if p.signPolicy.mustSign() {
|
||||||
if msg.Signature == nil {
|
if msg.Signature == nil {
|
||||||
log.Debugf("dropping unsigned message from %s", src)
|
log.Debugf("dropping unsigned message from %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectMissingSignature)
|
p.tracer.RejectMessage(msg, RejectMissingSignature)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
// Actual signature verification happens in the validation pipeline,
|
// Actual signature verification happens in the validation pipeline,
|
||||||
@ -1012,7 +1012,7 @@ func (p *PubSub) pushMsg(msg *Message) {
|
|||||||
} else {
|
} else {
|
||||||
if msg.Signature != nil {
|
if msg.Signature != nil {
|
||||||
log.Debugf("dropping message with unexpected signature from %s", src)
|
log.Debugf("dropping message with unexpected signature from %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectUnexpectedSignature)
|
p.tracer.RejectMessage(msg, RejectUnexpectedSignature)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
// If we are expecting signed messages, and not authoring messages,
|
// If we are expecting signed messages, and not authoring messages,
|
||||||
@ -1022,7 +1022,7 @@ func (p *PubSub) pushMsg(msg *Message) {
|
|||||||
if p.signID == "" {
|
if p.signID == "" {
|
||||||
if msg.Seqno != nil || msg.From != nil || msg.Key != nil {
|
if msg.Seqno != nil || msg.From != nil || msg.Key != nil {
|
||||||
log.Debugf("dropping message with unexpected auth info from %s", src)
|
log.Debugf("dropping message with unexpected auth info from %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectUnexpectedAuthInfo)
|
p.tracer.RejectMessage(msg, RejectUnexpectedAuthInfo)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1033,7 +1033,7 @@ func (p *PubSub) pushMsg(msg *Message) {
|
|||||||
self := p.host.ID()
|
self := p.host.ID()
|
||||||
if peer.ID(msg.GetFrom()) == self && src != self {
|
if peer.ID(msg.GetFrom()) == self && src != self {
|
||||||
log.Debugf("dropping message claiming to be from self but forwarded from %s", src)
|
log.Debugf("dropping message claiming to be from self but forwarded from %s", src)
|
||||||
p.tracer.RejectMessage(msg, rejectSelfOrigin)
|
p.tracer.RejectMessage(msg, RejectSelfOrigin)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
20
score.go
20
score.go
@ -722,25 +722,25 @@ func (ps *peerScore) RejectMessage(msg *Message, reason string) {
|
|||||||
|
|
||||||
switch reason {
|
switch reason {
|
||||||
// we don't track those messages, but we penalize the peer as they are clearly invalid
|
// we don't track those messages, but we penalize the peer as they are clearly invalid
|
||||||
case rejectMissingSignature:
|
case RejectMissingSignature:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectInvalidSignature:
|
case RejectInvalidSignature:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectUnexpectedSignature:
|
case RejectUnexpectedSignature:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectUnexpectedAuthInfo:
|
case RejectUnexpectedAuthInfo:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectSelfOrigin:
|
case RejectSelfOrigin:
|
||||||
ps.markInvalidMessageDelivery(msg.ReceivedFrom, msg)
|
ps.markInvalidMessageDelivery(msg.ReceivedFrom, msg)
|
||||||
return
|
return
|
||||||
|
|
||||||
// we ignore those messages, so do nothing.
|
// we ignore those messages, so do nothing.
|
||||||
case rejectBlacklstedPeer:
|
case RejectBlacklstedPeer:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectBlacklistedSource:
|
case RejectBlacklistedSource:
|
||||||
return
|
return
|
||||||
|
|
||||||
case rejectValidationQueueFull:
|
case RejectValidationQueueFull:
|
||||||
// the message was rejected before it entered the validation pipeline;
|
// the message was rejected before it entered the validation pipeline;
|
||||||
// we don't know if this message has a valid signature, and thus we also don't know if
|
// we don't know if this message has a valid signature, and thus we also don't know if
|
||||||
// it has a valid message ID; all we can do is ignore it.
|
// it has a valid message ID; all we can do is ignore it.
|
||||||
@ -756,14 +756,14 @@ func (ps *peerScore) RejectMessage(msg *Message, reason string) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
switch reason {
|
switch reason {
|
||||||
case rejectValidationThrottled:
|
case RejectValidationThrottled:
|
||||||
// if we reject with "validation throttled" we don't penalize the peer(s) that forward it
|
// if we reject with "validation throttled" we don't penalize the peer(s) that forward it
|
||||||
// because we don't know if it was valid.
|
// because we don't know if it was valid.
|
||||||
drec.status = deliveryThrottled
|
drec.status = deliveryThrottled
|
||||||
// release the delivery time tracking map to free some memory early
|
// release the delivery time tracking map to free some memory early
|
||||||
drec.peers = nil
|
drec.peers = nil
|
||||||
return
|
return
|
||||||
case rejectValidationIgnored:
|
case RejectValidationIgnored:
|
||||||
// we were explicitly instructed by the validator to ignore the message but not penalize
|
// we were explicitly instructed by the validator to ignore the message but not penalize
|
||||||
// the peer
|
// the peer
|
||||||
drec.status = deliveryIgnored
|
drec.status = deliveryIgnored
|
||||||
|
|||||||
@ -475,7 +475,7 @@ func TestScoreInvalidMessageDeliveries(t *testing.T) {
|
|||||||
pbMsg := makeTestMessage(i)
|
pbMsg := makeTestMessage(i)
|
||||||
pbMsg.Topic = &mytopic
|
pbMsg.Topic = &mytopic
|
||||||
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
||||||
ps.RejectMessage(&msg, rejectInvalidSignature)
|
ps.RejectMessage(&msg, RejectInvalidSignature)
|
||||||
}
|
}
|
||||||
|
|
||||||
ps.refreshScores()
|
ps.refreshScores()
|
||||||
@ -512,7 +512,7 @@ func TestScoreInvalidMessageDeliveriesDecay(t *testing.T) {
|
|||||||
pbMsg := makeTestMessage(i)
|
pbMsg := makeTestMessage(i)
|
||||||
pbMsg.Topic = &mytopic
|
pbMsg.Topic = &mytopic
|
||||||
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
||||||
ps.RejectMessage(&msg, rejectInvalidSignature)
|
ps.RejectMessage(&msg, RejectInvalidSignature)
|
||||||
}
|
}
|
||||||
|
|
||||||
ps.refreshScores()
|
ps.refreshScores()
|
||||||
@ -561,9 +561,9 @@ func TestScoreRejectMessageDeliveries(t *testing.T) {
|
|||||||
msg2 := Message{ReceivedFrom: peerB, Message: pbMsg}
|
msg2 := Message{ReceivedFrom: peerB, Message: pbMsg}
|
||||||
|
|
||||||
// these should have no effect in the score
|
// these should have no effect in the score
|
||||||
ps.RejectMessage(&msg, rejectBlacklstedPeer)
|
ps.RejectMessage(&msg, RejectBlacklstedPeer)
|
||||||
ps.RejectMessage(&msg, rejectBlacklistedSource)
|
ps.RejectMessage(&msg, RejectBlacklistedSource)
|
||||||
ps.RejectMessage(&msg, rejectValidationQueueFull)
|
ps.RejectMessage(&msg, RejectValidationQueueFull)
|
||||||
|
|
||||||
aScore := ps.Score(peerA)
|
aScore := ps.Score(peerA)
|
||||||
expected := 0.0
|
expected := 0.0
|
||||||
@ -576,7 +576,7 @@ func TestScoreRejectMessageDeliveries(t *testing.T) {
|
|||||||
|
|
||||||
// this should have no effect in the score, and subsequent duplicate messages should have no
|
// this should have no effect in the score, and subsequent duplicate messages should have no
|
||||||
// effect either
|
// effect either
|
||||||
ps.RejectMessage(&msg, rejectValidationThrottled)
|
ps.RejectMessage(&msg, RejectValidationThrottled)
|
||||||
ps.DuplicateMessage(&msg2)
|
ps.DuplicateMessage(&msg2)
|
||||||
|
|
||||||
aScore = ps.Score(peerA)
|
aScore = ps.Score(peerA)
|
||||||
@ -601,7 +601,7 @@ func TestScoreRejectMessageDeliveries(t *testing.T) {
|
|||||||
|
|
||||||
// this should have no effect in the score, and subsequent duplicate messages should have no
|
// this should have no effect in the score, and subsequent duplicate messages should have no
|
||||||
// effect either
|
// effect either
|
||||||
ps.RejectMessage(&msg, rejectValidationIgnored)
|
ps.RejectMessage(&msg, RejectValidationIgnored)
|
||||||
ps.DuplicateMessage(&msg2)
|
ps.DuplicateMessage(&msg2)
|
||||||
|
|
||||||
aScore = ps.Score(peerA)
|
aScore = ps.Score(peerA)
|
||||||
@ -625,7 +625,7 @@ func TestScoreRejectMessageDeliveries(t *testing.T) {
|
|||||||
ps.ValidateMessage(&msg)
|
ps.ValidateMessage(&msg)
|
||||||
|
|
||||||
// and reject the message to make sure duplicates are also penalized
|
// and reject the message to make sure duplicates are also penalized
|
||||||
ps.RejectMessage(&msg, rejectValidationFailed)
|
ps.RejectMessage(&msg, RejectValidationFailed)
|
||||||
ps.DuplicateMessage(&msg2)
|
ps.DuplicateMessage(&msg2)
|
||||||
|
|
||||||
aScore = ps.Score(peerA)
|
aScore = ps.Score(peerA)
|
||||||
@ -650,7 +650,7 @@ func TestScoreRejectMessageDeliveries(t *testing.T) {
|
|||||||
|
|
||||||
// and reject the message after a duplciate has arrived
|
// and reject the message after a duplciate has arrived
|
||||||
ps.DuplicateMessage(&msg2)
|
ps.DuplicateMessage(&msg2)
|
||||||
ps.RejectMessage(&msg, rejectValidationFailed)
|
ps.RejectMessage(&msg, RejectValidationFailed)
|
||||||
|
|
||||||
aScore = ps.Score(peerA)
|
aScore = ps.Score(peerA)
|
||||||
expected = -4.0
|
expected = -4.0
|
||||||
@ -1032,7 +1032,7 @@ func TestScoreResetTopicParams(t *testing.T) {
|
|||||||
pbMsg.Topic = &mytopic
|
pbMsg.Topic = &mytopic
|
||||||
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
msg := Message{ReceivedFrom: peerA, Message: pbMsg}
|
||||||
ps.ValidateMessage(&msg)
|
ps.ValidateMessage(&msg)
|
||||||
ps.RejectMessage(&msg, rejectValidationFailed)
|
ps.RejectMessage(&msg, RejectValidationFailed)
|
||||||
}
|
}
|
||||||
|
|
||||||
// check the topic score
|
// check the topic score
|
||||||
|
|||||||
@ -242,11 +242,11 @@ func (t *tagTracer) RejectMessage(msg *Message, reason string) {
|
|||||||
// the validation pipeline. Other rejection reasons (missing signature, etc) skip the validation
|
// the validation pipeline. Other rejection reasons (missing signature, etc) skip the validation
|
||||||
// queue, so we don't want to remove the state in case the message is still validating.
|
// queue, so we don't want to remove the state in case the message is still validating.
|
||||||
switch reason {
|
switch reason {
|
||||||
case rejectValidationThrottled:
|
case RejectValidationThrottled:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectValidationIgnored:
|
case RejectValidationIgnored:
|
||||||
fallthrough
|
fallthrough
|
||||||
case rejectValidationFailed:
|
case RejectValidationFailed:
|
||||||
delete(t.nearFirst, t.msgID(msg.Message))
|
delete(t.nearFirst, t.msgID(msg.Message))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
22
tracer.go
22
tracer.go
@ -25,17 +25,17 @@ var MinTraceBatchSize = 16
|
|||||||
|
|
||||||
// rejection reasons
|
// rejection reasons
|
||||||
const (
|
const (
|
||||||
rejectBlacklstedPeer = "blacklisted peer"
|
RejectBlacklstedPeer = "blacklisted peer"
|
||||||
rejectBlacklistedSource = "blacklisted source"
|
RejectBlacklistedSource = "blacklisted source"
|
||||||
rejectMissingSignature = "missing signature"
|
RejectMissingSignature = "missing signature"
|
||||||
rejectUnexpectedSignature = "unexpected signature"
|
RejectUnexpectedSignature = "unexpected signature"
|
||||||
rejectUnexpectedAuthInfo = "unexpected auth info"
|
RejectUnexpectedAuthInfo = "unexpected auth info"
|
||||||
rejectInvalidSignature = "invalid signature"
|
RejectInvalidSignature = "invalid signature"
|
||||||
rejectValidationQueueFull = "validation queue full"
|
RejectValidationQueueFull = "validation queue full"
|
||||||
rejectValidationThrottled = "validation throttled"
|
RejectValidationThrottled = "validation throttled"
|
||||||
rejectValidationFailed = "validation failed"
|
RejectValidationFailed = "validation failed"
|
||||||
rejectValidationIgnored = "validation ignored"
|
RejectValidationIgnored = "validation ignored"
|
||||||
rejectSelfOrigin = "self originated message"
|
RejectSelfOrigin = "self originated message"
|
||||||
)
|
)
|
||||||
|
|
||||||
type basicTracer struct {
|
type basicTracer struct {
|
||||||
|
|||||||
@ -201,7 +201,7 @@ func (v *validation) Push(src peer.ID, msg *Message) bool {
|
|||||||
case v.validateQ <- &validateReq{vals, src, msg}:
|
case v.validateQ <- &validateReq{vals, src, msg}:
|
||||||
default:
|
default:
|
||||||
log.Debugf("message validation throttled: queue full; dropping message from %s", src)
|
log.Debugf("message validation throttled: queue full; dropping message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationQueueFull)
|
v.tracer.RejectMessage(msg, RejectValidationQueueFull)
|
||||||
}
|
}
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
@ -242,7 +242,7 @@ func (v *validation) validate(vals []*topicVal, src peer.ID, msg *Message) {
|
|||||||
if msg.Signature != nil {
|
if msg.Signature != nil {
|
||||||
if !v.validateSignature(msg) {
|
if !v.validateSignature(msg) {
|
||||||
log.Debugf("message signature validation failed; dropping message from %s", src)
|
log.Debugf("message signature validation failed; dropping message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectInvalidSignature)
|
v.tracer.RejectMessage(msg, RejectInvalidSignature)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -282,7 +282,7 @@ loop:
|
|||||||
|
|
||||||
if result == ValidationReject {
|
if result == ValidationReject {
|
||||||
log.Debugf("message validation failed; dropping message from %s", src)
|
log.Debugf("message validation failed; dropping message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationFailed)
|
v.tracer.RejectMessage(msg, RejectValidationFailed)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -296,13 +296,13 @@ loop:
|
|||||||
}()
|
}()
|
||||||
default:
|
default:
|
||||||
log.Debugf("message validation throttled; dropping message from %s", src)
|
log.Debugf("message validation throttled; dropping message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationThrottled)
|
v.tracer.RejectMessage(msg, RejectValidationThrottled)
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
if result == ValidationIgnore {
|
if result == ValidationIgnore {
|
||||||
v.tracer.RejectMessage(msg, rejectValidationIgnored)
|
v.tracer.RejectMessage(msg, RejectValidationIgnored)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -332,15 +332,15 @@ func (v *validation) doValidateTopic(vals []*topicVal, src peer.ID, msg *Message
|
|||||||
v.p.sendMsg <- msg
|
v.p.sendMsg <- msg
|
||||||
case ValidationReject:
|
case ValidationReject:
|
||||||
log.Debugf("message validation failed; dropping message from %s", src)
|
log.Debugf("message validation failed; dropping message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationFailed)
|
v.tracer.RejectMessage(msg, RejectValidationFailed)
|
||||||
return
|
return
|
||||||
case ValidationIgnore:
|
case ValidationIgnore:
|
||||||
log.Debugf("message validation punted; ignoring message from %s", src)
|
log.Debugf("message validation punted; ignoring message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationIgnored)
|
v.tracer.RejectMessage(msg, RejectValidationIgnored)
|
||||||
return
|
return
|
||||||
case validationThrottled:
|
case validationThrottled:
|
||||||
log.Debugf("message validation throttled; ignoring message from %s", src)
|
log.Debugf("message validation throttled; ignoring message from %s", src)
|
||||||
v.tracer.RejectMessage(msg, rejectValidationThrottled)
|
v.tracer.RejectMessage(msg, RejectValidationThrottled)
|
||||||
|
|
||||||
default:
|
default:
|
||||||
// BUG: this would be an internal programming error, so a panic seems appropiate.
|
// BUG: this would be an internal programming error, so a panic seems appropiate.
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user