Skip to content

Commit 829f902

Browse files
authored
Default validator support (#525)
* add default validator support * add an implementation for basic seqno as nonce validation * missing return * the nonce belongs to the origin peer * add note about rust predicament * add seqno validator tests * minor test tweak, ensure at least 1ms before replay
1 parent 56c0e6c commit 829f902

File tree

4 files changed

+456
-22
lines changed

4 files changed

+456
-22
lines changed

floodsub_test.go

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,6 +115,18 @@ func getPubsubs(ctx context.Context, hs []host.Host, opts ...Option) []*PubSub {
115115
return psubs
116116
}
117117

118+
func getPubsubsWithOptionC(ctx context.Context, hs []host.Host, cons ...func(int) Option) []*PubSub {
119+
var psubs []*PubSub
120+
for _, h := range hs {
121+
var opts []Option
122+
for i, c := range cons {
123+
opts = append(opts, c(i))
124+
}
125+
psubs = append(psubs, getPubsub(ctx, h, opts...))
126+
}
127+
return psubs
128+
}
129+
118130
func assertReceive(t *testing.T, ch *Subscription, exp []byte) {
119131
select {
120132
case msg := <-ch.ch:
@@ -175,7 +187,6 @@ func TestBasicFloodsub(t *testing.T) {
175187
}
176188
}
177189
}
178-
179190
}
180191

181192
func TestMultihops(t *testing.T) {

validation.go

Lines changed: 65 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,10 @@ type validation struct {
7070
// mx protects the validator map
7171
mx sync.Mutex
7272
// topicVals tracks per topic validators
73-
topicVals map[string]*topicVal
73+
topicVals map[string]*validatorImpl
74+
75+
// defaultVals tracks default validators applicable to all topics
76+
defaultVals []*validatorImpl
7477

7578
// validateQ is the front-end to the validation pipeline
7679
validateQ chan *validateReq
@@ -84,13 +87,13 @@ type validation struct {
8487

8588
// validation requests
8689
type validateReq struct {
87-
vals []*topicVal
90+
vals []*validatorImpl
8891
src peer.ID
8992
msg *Message
9093
}
9194

9295
// representation of topic validators
93-
type topicVal struct {
96+
type validatorImpl struct {
9497
topic string
9598
validate ValidatorEx
9699
validateTimeout time.Duration
@@ -117,7 +120,7 @@ type rmValReq struct {
117120
// newValidation creates a new validation pipeline
118121
func newValidation() *validation {
119122
return &validation{
120-
topicVals: make(map[string]*topicVal),
123+
topicVals: make(map[string]*validatorImpl),
121124
validateQ: make(chan *validateReq, defaultValidateQueueSize),
122125
validateThrottle: make(chan struct{}, defaultValidateThrottle),
123126
validateWorkers: runtime.NumCPU(),
@@ -136,17 +139,28 @@ func (v *validation) Start(p *PubSub) {
136139

137140
// AddValidator adds a new validator
138141
func (v *validation) AddValidator(req *addValReq) {
142+
val, err := v.makeValidator(req)
143+
if err != nil {
144+
req.resp <- err
145+
return
146+
}
147+
139148
v.mx.Lock()
140149
defer v.mx.Unlock()
141150

142-
topic := req.topic
151+
topic := val.topic
143152

144153
_, ok := v.topicVals[topic]
145154
if ok {
146155
req.resp <- fmt.Errorf("duplicate validator for topic %s", topic)
147156
return
148157
}
149158

159+
v.topicVals[topic] = val
160+
req.resp <- nil
161+
}
162+
163+
func (v *validation) makeValidator(req *addValReq) (*validatorImpl, error) {
150164
makeValidatorEx := func(v Validator) ValidatorEx {
151165
return func(ctx context.Context, p peer.ID, msg *Message) ValidationResult {
152166
if v(ctx, p, msg) {
@@ -170,12 +184,15 @@ func (v *validation) AddValidator(req *addValReq) {
170184
validator = v
171185

172186
default:
173-
req.resp <- fmt.Errorf("unknown validator type for topic %s; must be an instance of Validator or ValidatorEx", topic)
174-
return
187+
topic := req.topic
188+
if req.topic == "" {
189+
topic = "(default)"
190+
}
191+
return nil, fmt.Errorf("unknown validator type for topic %s; must be an instance of Validator or ValidatorEx", topic)
175192
}
176193

177-
val := &topicVal{
178-
topic: topic,
194+
val := &validatorImpl{
195+
topic: req.topic,
179196
validate: validator,
180197
validateTimeout: 0,
181198
validateThrottle: make(chan struct{}, defaultValidateConcurrency),
@@ -190,8 +207,7 @@ func (v *validation) AddValidator(req *addValReq) {
190207
val.validateThrottle = make(chan struct{}, req.throttle)
191208
}
192209

193-
v.topicVals[topic] = val
194-
req.resp <- nil
210+
return val, nil
195211
}
196212

197213
// RemoveValidator removes an existing validator
@@ -244,18 +260,21 @@ func (v *validation) Push(src peer.ID, msg *Message) bool {
244260
}
245261

246262
// getValidators returns all validators that apply to a given message
247-
func (v *validation) getValidators(msg *Message) []*topicVal {
263+
func (v *validation) getValidators(msg *Message) []*validatorImpl {
248264
v.mx.Lock()
249265
defer v.mx.Unlock()
250266

267+
var vals []*validatorImpl
268+
vals = append(vals, v.defaultVals...)
269+
251270
topic := msg.GetTopic()
252271

253272
val, ok := v.topicVals[topic]
254273
if !ok {
255-
return nil
274+
return vals
256275
}
257276

258-
return []*topicVal{val}
277+
return append(vals, val)
259278
}
260279

261280
// validateWorker is an active goroutine performing inline validation
@@ -271,7 +290,7 @@ func (v *validation) validateWorker() {
271290
}
272291

273292
// validate performs validation and only sends the message if all validators succeed
274-
func (v *validation) validate(vals []*topicVal, src peer.ID, msg *Message, synchronous bool) error {
293+
func (v *validation) validate(vals []*validatorImpl, src peer.ID, msg *Message, synchronous bool) error {
275294
// If signature verification is enabled, but signing is disabled,
276295
// the Signature is required to be nil upon receiving the message in PubSub.pushMsg.
277296
if msg.Signature != nil {
@@ -292,7 +311,7 @@ func (v *validation) validate(vals []*topicVal, src peer.ID, msg *Message, synch
292311
v.tracer.ValidateMessage(msg)
293312
}
294313

295-
var inline, async []*topicVal
314+
var inline, async []*validatorImpl
296315
for _, val := range vals {
297316
if val.validateInline || synchronous {
298317
inline = append(inline, val)
@@ -360,7 +379,7 @@ func (v *validation) validateSignature(msg *Message) bool {
360379
return true
361380
}
362381

363-
func (v *validation) doValidateTopic(vals []*topicVal, src peer.ID, msg *Message, r ValidationResult) {
382+
func (v *validation) doValidateTopic(vals []*validatorImpl, src peer.ID, msg *Message, r ValidationResult) {
364383
result := v.validateTopic(vals, src, msg)
365384

366385
if result == ValidationAccept && r != ValidationAccept {
@@ -388,7 +407,7 @@ func (v *validation) doValidateTopic(vals []*topicVal, src peer.ID, msg *Message
388407
}
389408
}
390409

391-
func (v *validation) validateTopic(vals []*topicVal, src peer.ID, msg *Message) ValidationResult {
410+
func (v *validation) validateTopic(vals []*validatorImpl, src peer.ID, msg *Message) ValidationResult {
392411
if len(vals) == 1 {
393412
return v.validateSingleTopic(vals[0], src, msg)
394413
}
@@ -404,7 +423,7 @@ func (v *validation) validateTopic(vals []*topicVal, src peer.ID, msg *Message)
404423

405424
select {
406425
case val.validateThrottle <- struct{}{}:
407-
go func(val *topicVal) {
426+
go func(val *validatorImpl) {
408427
rch <- val.validateMsg(ctx, src, msg)
409428
<-val.validateThrottle
410429
}(val)
@@ -438,7 +457,7 @@ loop:
438457
}
439458

440459
// fast path for single topic validation that avoids the extra goroutine
441-
func (v *validation) validateSingleTopic(val *topicVal, src peer.ID, msg *Message) ValidationResult {
460+
func (v *validation) validateSingleTopic(val *validatorImpl, src peer.ID, msg *Message) ValidationResult {
442461
select {
443462
case val.validateThrottle <- struct{}{}:
444463
res := val.validateMsg(v.p.ctx, src, msg)
@@ -451,7 +470,7 @@ func (v *validation) validateSingleTopic(val *topicVal, src peer.ID, msg *Messag
451470
}
452471
}
453472

454-
func (val *topicVal) validateMsg(ctx context.Context, src peer.ID, msg *Message) ValidationResult {
473+
func (val *validatorImpl) validateMsg(ctx context.Context, src peer.ID, msg *Message) ValidationResult {
455474
start := time.Now()
456475
defer func() {
457476
log.Debugf("validation done; took %s", time.Since(start))
@@ -479,6 +498,31 @@ func (val *topicVal) validateMsg(ctx context.Context, src peer.ID, msg *Message)
479498
}
480499

481500
// / Options
501+
// WithDefaultValidator adds a validator that applies to all topics by default; it can be used
502+
// more than once and add multiple validators. Having a defult validator does not inhibit registering
503+
// a per topic validator.
504+
func WithDefaultValidator(val interface{}, opts ...ValidatorOpt) Option {
505+
return func(ps *PubSub) error {
506+
addVal := &addValReq{
507+
validate: val,
508+
}
509+
510+
for _, opt := range opts {
511+
err := opt(addVal)
512+
if err != nil {
513+
return err
514+
}
515+
}
516+
517+
val, err := ps.val.makeValidator(addVal)
518+
if err != nil {
519+
return err
520+
}
521+
522+
ps.val.defaultVals = append(ps.val.defaultVals, val)
523+
return nil
524+
}
525+
}
482526

483527
// WithValidateQueueSize sets the buffer of validate queue. Defaults to 32.
484528
// When queue is full, validation is throttled and new messages are dropped.

validation_builtin.go

Lines changed: 101 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,101 @@
1+
package pubsub
2+
3+
import (
4+
"context"
5+
"encoding/binary"
6+
"sync"
7+
8+
"github.com/libp2p/go-libp2p/core/peer"
9+
)
10+
11+
// PeerMetadataStore is an interface for storing and retrieving per peer metadata
12+
type PeerMetadataStore interface {
13+
// Get retrieves the metadata associated with a peer;
14+
// It should return nil if there is no metadata associated with the peer and not an error.
15+
Get(context.Context, peer.ID) ([]byte, error)
16+
// Put sets the metadata associated with a peer.
17+
Put(context.Context, peer.ID, []byte) error
18+
}
19+
20+
// BasicSeqnoValidator is a basic validator, usable as a default validator, that ignores replayed
21+
// messages outside the seen cache window. The validator uses the message seqno as a peer-specific
22+
// nonce to decide whether the message should be propagated, comparing to the maximal nonce store
23+
// in the peer metadata store. This is useful to ensure that there can be no infinitely propagating
24+
// messages in the network regardless of the seen cache span and network diameter.
25+
// It requires that pubsub is instantiated with a strict message signing policy and that seqnos
26+
// are not disabled, ie it doesn't support anonymous mode.
27+
//
28+
// Warning: See https://github.com/libp2p/rust-libp2p/issues/3453
29+
// TL;DR: rust is currently violating the spec by issuing a random seqno, which creates an
30+
// interoperability hazard. We expect this issue to be addressed in the not so distant future,
31+
// but keep this in mind if you are in a mixed environment with (older) rust nodes.
32+
type BasicSeqnoValidator struct {
33+
mx sync.RWMutex
34+
meta PeerMetadataStore
35+
}
36+
37+
// NewBasicSeqnoValidator constructs a BasicSeqnoValidator using the givven PeerMetadataStore.
38+
func NewBasicSeqnoValidator(meta PeerMetadataStore) ValidatorEx {
39+
val := &BasicSeqnoValidator{
40+
meta: meta,
41+
}
42+
return val.validate
43+
}
44+
45+
func (v *BasicSeqnoValidator) validate(ctx context.Context, _ peer.ID, m *Message) ValidationResult {
46+
p := m.GetFrom()
47+
48+
v.mx.RLock()
49+
nonceBytes, err := v.meta.Get(ctx, p)
50+
v.mx.RUnlock()
51+
52+
if err != nil {
53+
log.Warn("error retrieving peer nonce: %s", err)
54+
return ValidationIgnore
55+
}
56+
57+
var nonce uint64
58+
if len(nonceBytes) > 0 {
59+
nonce = binary.BigEndian.Uint64(nonceBytes)
60+
}
61+
62+
var seqno uint64
63+
seqnoBytes := m.GetSeqno()
64+
if len(seqnoBytes) > 0 {
65+
seqno = binary.BigEndian.Uint64(seqnoBytes)
66+
}
67+
68+
// compare against the largest seen nonce
69+
if seqno <= nonce {
70+
return ValidationIgnore
71+
}
72+
73+
// get the nonce and compare again with an exclusive lock before commiting (cf concurrent validation)
74+
v.mx.Lock()
75+
defer v.mx.Unlock()
76+
77+
nonceBytes, err = v.meta.Get(ctx, p)
78+
if err != nil {
79+
log.Warn("error retrieving peer nonce: %s", err)
80+
return ValidationIgnore
81+
}
82+
83+
if len(nonceBytes) > 0 {
84+
nonce = binary.BigEndian.Uint64(nonceBytes)
85+
}
86+
87+
if seqno <= nonce {
88+
return ValidationIgnore
89+
}
90+
91+
// update the nonce
92+
nonceBytes = make([]byte, 8)
93+
binary.BigEndian.PutUint64(nonceBytes, seqno)
94+
95+
err = v.meta.Put(ctx, p, nonceBytes)
96+
if err != nil {
97+
log.Warn("error storing peer nonce: %s", err)
98+
}
99+
100+
return ValidationAccept
101+
}

0 commit comments

Comments
 (0)