mirror of https://github.com/status-im/go-waku.git
feat(rln-relay): ensure execution order for pubsub validators
This commit is contained in:
parent
ab7e45c735
commit
7beaa3f029
|
@ -43,7 +43,7 @@ func execute(options Options) {
|
||||||
}
|
}
|
||||||
|
|
||||||
if options.RLNRelay.Enable {
|
if options.RLNRelay.Enable {
|
||||||
spamHandler := func(message *pb.WakuMessage) error {
|
spamHandler := func(message *pb.WakuMessage, topic string) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -10,7 +10,6 @@ import (
|
||||||
backoffv4 "github.com/cenkalti/backoff/v4"
|
backoffv4 "github.com/cenkalti/backoff/v4"
|
||||||
golog "github.com/ipfs/go-log/v2"
|
golog "github.com/ipfs/go-log/v2"
|
||||||
"github.com/libp2p/go-libp2p"
|
"github.com/libp2p/go-libp2p"
|
||||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/crypto"
|
"github.com/ethereum/go-ethereum/crypto"
|
||||||
|
@ -66,13 +65,13 @@ type IdentityCredential = struct {
|
||||||
IDCommitment byte32 `json:"idCommitment"`
|
IDCommitment byte32 `json:"idCommitment"`
|
||||||
}
|
}
|
||||||
|
|
||||||
type SpamHandler = func(message *pb.WakuMessage) error
|
type SpamHandler = func(message *pb.WakuMessage, topic string) error
|
||||||
|
|
||||||
type RLNRelay interface {
|
type RLNRelay interface {
|
||||||
IdentityCredential() (IdentityCredential, error)
|
IdentityCredential() (IdentityCredential, error)
|
||||||
MembershipIndex() uint
|
MembershipIndex() uint
|
||||||
AppendRLNProof(msg *pb.WakuMessage, senderEpochTime time.Time) error
|
AppendRLNProof(msg *pb.WakuMessage, senderEpochTime time.Time) error
|
||||||
Validator(spamHandler SpamHandler) func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool
|
Validator(spamHandler SpamHandler) func(ctx context.Context, message *pb.WakuMessage, topic string) bool
|
||||||
Start(ctx context.Context) error
|
Start(ctx context.Context) error
|
||||||
Stop() error
|
Stop() error
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,7 +8,6 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
|
|
||||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/rln"
|
"github.com/waku-org/go-waku/waku/v2/protocol/rln"
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/rln/group_manager"
|
"github.com/waku-org/go-waku/waku/v2/protocol/rln/group_manager"
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/rln/group_manager/dynamic"
|
"github.com/waku-org/go-waku/waku/v2/protocol/rln/group_manager/dynamic"
|
||||||
|
@ -29,6 +28,10 @@ func (w *WakuNode) setupRLNRelay() error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if !w.opts.enableRelay {
|
||||||
|
return errors.New("rln requires relay")
|
||||||
|
}
|
||||||
|
|
||||||
var groupManager group_manager.GroupManager
|
var groupManager group_manager.GroupManager
|
||||||
|
|
||||||
rlnInstance, rootTracker, err := rln.GetRLNInstanceAndRootTracker(w.opts.rlnTreePath)
|
rlnInstance, rootTracker, err := rln.GetRLNInstanceAndRootTracker(w.opts.rlnTreePath)
|
||||||
|
@ -85,8 +88,7 @@ func (w *WakuNode) setupRLNRelay() error {
|
||||||
|
|
||||||
w.rlnRelay = rlnRelay
|
w.rlnRelay = rlnRelay
|
||||||
|
|
||||||
// Adding RLN as a default validator
|
w.Relay().RegisterDefaultValidator(w.rlnRelay.Validator(w.opts.rlnSpamHandler))
|
||||||
w.opts.pubsubOpts = append(w.opts.pubsubOpts, pubsub.WithDefaultValidator(rlnRelay.Validator(w.opts.rlnSpamHandler)))
|
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -96,7 +96,7 @@ type WakuNodeParameters struct {
|
||||||
enableRLN bool
|
enableRLN bool
|
||||||
rlnRelayMemIndex *uint
|
rlnRelayMemIndex *uint
|
||||||
rlnRelayDynamic bool
|
rlnRelayDynamic bool
|
||||||
rlnSpamHandler func(message *pb.WakuMessage) error
|
rlnSpamHandler func(message *pb.WakuMessage, topic string) error
|
||||||
rlnETHClientAddress string
|
rlnETHClientAddress string
|
||||||
keystorePath string
|
keystorePath string
|
||||||
keystorePassword string
|
keystorePassword string
|
||||||
|
|
|
@ -20,12 +20,12 @@ type Envelope struct {
|
||||||
// as well as generating a hash based on the bytes that compose the message
|
// as well as generating a hash based on the bytes that compose the message
|
||||||
func NewEnvelope(msg *wpb.WakuMessage, receiverTime int64, pubSubTopic string) *Envelope {
|
func NewEnvelope(msg *wpb.WakuMessage, receiverTime int64, pubSubTopic string) *Envelope {
|
||||||
messageHash := msg.Hash(pubSubTopic)
|
messageHash := msg.Hash(pubSubTopic)
|
||||||
hash := hash.SHA256([]byte(msg.ContentTopic), msg.Payload)
|
digest := hash.SHA256([]byte(msg.ContentTopic), msg.Payload)
|
||||||
return &Envelope{
|
return &Envelope{
|
||||||
msg: msg,
|
msg: msg,
|
||||||
hash: messageHash,
|
hash: messageHash,
|
||||||
index: &pb.Index{
|
index: &pb.Index{
|
||||||
Digest: hash[:],
|
Digest: digest[:],
|
||||||
ReceiverTime: receiverTime,
|
ReceiverTime: receiverTime,
|
||||||
SenderTime: msg.Timestamp,
|
SenderTime: msg.Timestamp,
|
||||||
PubsubTopic: pubSubTopic,
|
PubsubTopic: pubSubTopic,
|
||||||
|
|
|
@ -10,14 +10,14 @@ import (
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/crypto"
|
"github.com/ethereum/go-ethereum/crypto"
|
||||||
"github.com/ethereum/go-ethereum/crypto/secp256k1"
|
"github.com/ethereum/go-ethereum/crypto/secp256k1"
|
||||||
|
|
||||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||||
"github.com/libp2p/go-libp2p/core/peer"
|
"github.com/libp2p/go-libp2p/core/peer"
|
||||||
|
proto "google.golang.org/protobuf/proto"
|
||||||
|
|
||||||
"github.com/waku-org/go-waku/waku/v2/hash"
|
"github.com/waku-org/go-waku/waku/v2/hash"
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
||||||
"github.com/waku-org/go-waku/waku/v2/timesource"
|
"github.com/waku-org/go-waku/waku/v2/timesource"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
proto "google.golang.org/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func msgHash(pubSubTopic string, msg *pb.WakuMessage) []byte {
|
func msgHash(pubSubTopic string, msg *pb.WakuMessage) []byte {
|
||||||
|
@ -38,6 +38,68 @@ func msgHash(pubSubTopic string, msg *pb.WakuMessage) []byte {
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type validatorFn = func(ctx context.Context, msg *pb.WakuMessage, topic string) bool
|
||||||
|
|
||||||
|
func (w *WakuRelay) RegisterDefaultValidator(fn validatorFn) {
|
||||||
|
w.topicValidatorMutex.Lock()
|
||||||
|
defer w.topicValidatorMutex.Unlock()
|
||||||
|
w.defaultTopicValidators = append(w.defaultTopicValidators, fn)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *WakuRelay) RegisterTopicValidator(topic string, fn validatorFn) {
|
||||||
|
w.topicValidatorMutex.Lock()
|
||||||
|
defer w.topicValidatorMutex.Unlock()
|
||||||
|
|
||||||
|
w.topicValidators[topic] = append(w.topicValidators[topic], fn)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *WakuRelay) RemoveTopicValidator(topic string) {
|
||||||
|
w.topicValidatorMutex.Lock()
|
||||||
|
defer w.topicValidatorMutex.Unlock()
|
||||||
|
|
||||||
|
delete(w.topicValidators, topic)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *WakuRelay) topicValidator(topic string) func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
||||||
|
return func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
||||||
|
msg := new(pb.WakuMessage)
|
||||||
|
err := proto.Unmarshal(message.Data, msg)
|
||||||
|
if err != nil {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
w.topicValidatorMutex.RLock()
|
||||||
|
validators, exists := w.topicValidators[topic]
|
||||||
|
validators = append(validators, w.defaultTopicValidators...)
|
||||||
|
w.topicValidatorMutex.RUnlock()
|
||||||
|
|
||||||
|
if exists {
|
||||||
|
for _, v := range validators {
|
||||||
|
if !v(ctx, msg, topic) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// AddSignedTopicValidator registers a gossipsub validator for a topic which will check that messages Meta field contains a valid ECDSA signature for the specified pubsub topic. This is used as a DoS prevention mechanism
|
||||||
|
func (w *WakuRelay) AddSignedTopicValidator(topic string, publicKey *ecdsa.PublicKey) error {
|
||||||
|
w.log.Info("adding validator to signed topic", zap.String("topic", topic), zap.String("publicKey", hex.EncodeToString(elliptic.Marshal(publicKey.Curve, publicKey.X, publicKey.Y))))
|
||||||
|
|
||||||
|
fn := signedTopicBuilder(w.timesource, publicKey)
|
||||||
|
|
||||||
|
w.RegisterTopicValidator(topic, fn)
|
||||||
|
|
||||||
|
if !w.IsSubscribed(topic) {
|
||||||
|
w.log.Warn("relay is not subscribed to signed topic", zap.String("topic", topic))
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
const messageWindowDuration = time.Minute * 5
|
const messageWindowDuration = time.Minute * 5
|
||||||
|
|
||||||
func withinTimeWindow(t timesource.Timesource, msg *pb.WakuMessage) bool {
|
func withinTimeWindow(t timesource.Timesource, msg *pb.WakuMessage) bool {
|
||||||
|
@ -51,17 +113,9 @@ func withinTimeWindow(t timesource.Timesource, msg *pb.WakuMessage) bool {
|
||||||
return now.Sub(msgTime).Abs() <= messageWindowDuration
|
return now.Sub(msgTime).Abs() <= messageWindowDuration
|
||||||
}
|
}
|
||||||
|
|
||||||
type validatorFn = func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool
|
func signedTopicBuilder(t timesource.Timesource, publicKey *ecdsa.PublicKey) validatorFn {
|
||||||
|
|
||||||
func validatorFnBuilder(t timesource.Timesource, topic string, publicKey *ecdsa.PublicKey) (validatorFn, error) {
|
|
||||||
publicKeyBytes := crypto.FromECDSAPub(publicKey)
|
publicKeyBytes := crypto.FromECDSAPub(publicKey)
|
||||||
return func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
return func(ctx context.Context, msg *pb.WakuMessage, topic string) bool {
|
||||||
msg := new(pb.WakuMessage)
|
|
||||||
err := proto.Unmarshal(message.Data, msg)
|
|
||||||
if err != nil {
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
|
|
||||||
if !withinTimeWindow(t, msg) {
|
if !withinTimeWindow(t, msg) {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
@ -70,28 +124,7 @@ func validatorFnBuilder(t timesource.Timesource, topic string, publicKey *ecdsa.
|
||||||
signature := msg.Meta
|
signature := msg.Meta
|
||||||
|
|
||||||
return secp256k1.VerifySignature(publicKeyBytes, msgHash, signature)
|
return secp256k1.VerifySignature(publicKeyBytes, msgHash, signature)
|
||||||
}, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// AddSignedTopicValidator registers a gossipsub validator for a topic which will check that messages Meta field contains a valid ECDSA signature for the specified pubsub topic. This is used as a DoS prevention mechanism
|
|
||||||
func (w *WakuRelay) AddSignedTopicValidator(topic string, publicKey *ecdsa.PublicKey) error {
|
|
||||||
w.log.Info("adding validator to signed topic", zap.String("topic", topic), zap.String("publicKey", hex.EncodeToString(elliptic.Marshal(publicKey.Curve, publicKey.X, publicKey.Y))))
|
|
||||||
|
|
||||||
fn, err := validatorFnBuilder(w.timesource, topic, publicKey)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
err = w.pubsub.RegisterTopicValidator(topic, fn)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
if !w.IsSubscribed(topic) {
|
|
||||||
w.log.Warn("relay is not subscribed to signed topic", zap.String("topic", topic))
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// SignMessage adds an ECDSA signature to a WakuMessage as an opt-in mechanism for DoS prevention
|
// SignMessage adds an ECDSA signature to a WakuMessage as an opt-in mechanism for DoS prevention
|
||||||
|
|
|
@ -7,11 +7,8 @@ import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/crypto"
|
"github.com/ethereum/go-ethereum/crypto"
|
||||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
||||||
pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb"
|
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
||||||
"google.golang.org/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type FakeTimesource struct {
|
type FakeTimesource struct {
|
||||||
|
@ -59,39 +56,23 @@ func TestMsgHash(t *testing.T) {
|
||||||
// expectedSignature, _ := hex.DecodeString("127FA211B2514F0E974A055392946DC1A14052182A6ABEFB8A6CD7C51DA1BF2E40595D28EF1A9488797C297EED3AAC45430005FB3A7F037BDD9FC4BD99F59E63")
|
// expectedSignature, _ := hex.DecodeString("127FA211B2514F0E974A055392946DC1A14052182A6ABEFB8A6CD7C51DA1BF2E40595D28EF1A9488797C297EED3AAC45430005FB3A7F037BDD9FC4BD99F59E63")
|
||||||
// require.True(t, bytes.Equal(expectedSignature, msg.Meta))
|
// require.True(t, bytes.Equal(expectedSignature, msg.Meta))
|
||||||
|
|
||||||
msgData, _ := proto.Marshal(msg)
|
|
||||||
|
|
||||||
//expectedMessageHash, _ := hex.DecodeString("662F8C20A335F170BD60ABC1F02AD66F0C6A6EE285DA2A53C95259E7937C0AE9")
|
//expectedMessageHash, _ := hex.DecodeString("662F8C20A335F170BD60ABC1F02AD66F0C6A6EE285DA2A53C95259E7937C0AE9")
|
||||||
//messageHash := MsgHash(pubsubTopic, msg)
|
//messageHash := MsgHash(pubsubTopic, msg)
|
||||||
//require.True(t, bytes.Equal(expectedMessageHash, messageHash))
|
//require.True(t, bytes.Equal(expectedMessageHash, messageHash))
|
||||||
|
|
||||||
myValidator, err := validatorFnBuilder(NewFakeTimesource(timestamp), protectedPubSubTopic, &prvKey.PublicKey)
|
myValidator := signedTopicBuilder(NewFakeTimesource(timestamp), &prvKey.PublicKey)
|
||||||
require.NoError(t, err)
|
result := myValidator(context.Background(), msg, protectedPubSubTopic)
|
||||||
result := myValidator(context.Background(), "", &pubsub.Message{
|
|
||||||
Message: &pubsub_pb.Message{
|
|
||||||
Data: msgData,
|
|
||||||
},
|
|
||||||
})
|
|
||||||
require.True(t, result)
|
require.True(t, result)
|
||||||
|
|
||||||
// Exceed 5m window in both directions
|
// Exceed 5m window in both directions
|
||||||
now5m1sInPast := timestamp.Add(-5 * time.Minute).Add(-1 * time.Second)
|
now5m1sInPast := timestamp.Add(-5 * time.Minute).Add(-1 * time.Second)
|
||||||
myValidator, err = validatorFnBuilder(NewFakeTimesource(now5m1sInPast), protectedPubSubTopic, &prvKey.PublicKey)
|
myValidator = signedTopicBuilder(NewFakeTimesource(now5m1sInPast), &prvKey.PublicKey)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
result = myValidator(context.Background(), "", &pubsub.Message{
|
result = myValidator(context.Background(), msg, protectedPubSubTopic)
|
||||||
Message: &pubsub_pb.Message{
|
|
||||||
Data: msgData,
|
|
||||||
},
|
|
||||||
})
|
|
||||||
require.False(t, result)
|
require.False(t, result)
|
||||||
|
|
||||||
now5m1sInFuture := timestamp.Add(5 * time.Minute).Add(1 * time.Second)
|
now5m1sInFuture := timestamp.Add(5 * time.Minute).Add(1 * time.Second)
|
||||||
myValidator, err = validatorFnBuilder(NewFakeTimesource(now5m1sInFuture), protectedPubSubTopic, &prvKey.PublicKey)
|
myValidator = signedTopicBuilder(NewFakeTimesource(now5m1sInFuture), &prvKey.PublicKey)
|
||||||
require.NoError(t, err)
|
result = myValidator(context.Background(), msg, protectedPubSubTopic)
|
||||||
result = myValidator(context.Background(), "", &pubsub.Message{
|
|
||||||
Message: &pubsub_pb.Message{
|
|
||||||
Data: msgData,
|
|
||||||
},
|
|
||||||
})
|
|
||||||
require.False(t, result)
|
require.False(t, result)
|
||||||
}
|
}
|
||||||
|
|
|
@ -49,6 +49,10 @@ type WakuRelay struct {
|
||||||
|
|
||||||
minPeersToPublish int
|
minPeersToPublish int
|
||||||
|
|
||||||
|
topicValidatorMutex sync.RWMutex
|
||||||
|
topicValidators map[string][]validatorFn
|
||||||
|
defaultTopicValidators []validatorFn
|
||||||
|
|
||||||
// TODO: convert to concurrent maps
|
// TODO: convert to concurrent maps
|
||||||
topicsMutex sync.Mutex
|
topicsMutex sync.Mutex
|
||||||
wakuRelayTopics map[string]*pubsub.Topic
|
wakuRelayTopics map[string]*pubsub.Topic
|
||||||
|
@ -83,6 +87,7 @@ func NewWakuRelay(bcaster Broadcaster, minPeersToPublish int, timesource timesou
|
||||||
w.timesource = timesource
|
w.timesource = timesource
|
||||||
w.wakuRelayTopics = make(map[string]*pubsub.Topic)
|
w.wakuRelayTopics = make(map[string]*pubsub.Topic)
|
||||||
w.relaySubs = make(map[string]*pubsub.Subscription)
|
w.relaySubs = make(map[string]*pubsub.Subscription)
|
||||||
|
w.topicValidators = make(map[string][]validatorFn)
|
||||||
w.bcaster = bcaster
|
w.bcaster = bcaster
|
||||||
w.minPeersToPublish = minPeersToPublish
|
w.minPeersToPublish = minPeersToPublish
|
||||||
w.CommonService = waku_proto.NewCommonService()
|
w.CommonService = waku_proto.NewCommonService()
|
||||||
|
@ -177,12 +182,6 @@ func NewWakuRelay(bcaster Broadcaster, minPeersToPublish int, timesource timesou
|
||||||
pubsub.WithSeenMessagesTTL(2 * time.Minute),
|
pubsub.WithSeenMessagesTTL(2 * time.Minute),
|
||||||
pubsub.WithPeerScore(w.peerScoreParams, w.peerScoreThresholds),
|
pubsub.WithPeerScore(w.peerScoreParams, w.peerScoreThresholds),
|
||||||
pubsub.WithPeerScoreInspect(w.peerScoreInspector, 6*time.Second),
|
pubsub.WithPeerScoreInspect(w.peerScoreInspector, 6*time.Second),
|
||||||
// TODO: to improve - setup default validator only if no default validator has been set.
|
|
||||||
pubsub.WithDefaultValidator(func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
|
||||||
msg := new(pb.WakuMessage)
|
|
||||||
err := proto.Unmarshal(message.Data, msg)
|
|
||||||
return err == nil
|
|
||||||
}),
|
|
||||||
}, opts...)
|
}, opts...)
|
||||||
|
|
||||||
return w
|
return w
|
||||||
|
@ -270,6 +269,11 @@ func (w *WakuRelay) upsertTopic(topic string) (*pubsub.Topic, error) {
|
||||||
|
|
||||||
pubSubTopic, ok := w.wakuRelayTopics[topic]
|
pubSubTopic, ok := w.wakuRelayTopics[topic]
|
||||||
if !ok { // Joins topic if node hasn't joined yet
|
if !ok { // Joins topic if node hasn't joined yet
|
||||||
|
err := w.pubsub.RegisterTopicValidator(topic, w.topicValidator(topic))
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
newTopic, err := w.pubsub.Join(string(topic))
|
newTopic, err := w.pubsub.Join(string(topic))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -419,6 +423,8 @@ func (w *WakuRelay) Unsubscribe(ctx context.Context, topic string) error {
|
||||||
}
|
}
|
||||||
delete(w.wakuRelayTopics, topic)
|
delete(w.wakuRelayTopics, topic)
|
||||||
|
|
||||||
|
w.RemoveTopicValidator(topic)
|
||||||
|
|
||||||
err = w.emitters.EvtRelayUnsubscribed.Emit(EvtRelayUnsubscribed{topic})
|
err = w.emitters.EvtRelayUnsubscribed.Emit(EvtRelayUnsubscribed{topic})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
|
@ -79,10 +79,6 @@ func TestGossipsubScore(t *testing.T) {
|
||||||
relay := make([]*WakuRelay, 5)
|
relay := make([]*WakuRelay, 5)
|
||||||
for i := 0; i < 5; i++ {
|
for i := 0; i < 5; i++ {
|
||||||
hosts[i], relay[i] = createRelayNode(t)
|
hosts[i], relay[i] = createRelayNode(t)
|
||||||
if i == 0 {
|
|
||||||
// This is a hack to remove the default validator from the list of default options
|
|
||||||
relay[i].opts = relay[i].opts[:len(relay[i].opts)-1]
|
|
||||||
}
|
|
||||||
err := relay[i].Start(context.Background())
|
err := relay[i].Start(context.Background())
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
}
|
}
|
||||||
|
@ -119,6 +115,11 @@ func TestGossipsubScore(t *testing.T) {
|
||||||
// We obtain the go-libp2p topic directly because we normally can't publish anything other than WakuMessages
|
// We obtain the go-libp2p topic directly because we normally can't publish anything other than WakuMessages
|
||||||
pubsubTopic, err := relay[0].upsertTopic(testTopic)
|
pubsubTopic, err := relay[0].upsertTopic(testTopic)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
// Removing validator from relayer0 to allow it to send invalid messages
|
||||||
|
err = relay[0].pubsub.UnregisterTopicValidator(testTopic)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
for i := 0; i < 50; i++ {
|
for i := 0; i < 50; i++ {
|
||||||
buf := make([]byte, 1000)
|
buf := make([]byte, 1000)
|
||||||
_, err := rand.Read(buf)
|
_, err := rand.Read(buf)
|
||||||
|
|
|
@ -26,7 +26,7 @@ const acceptableRootWindowSize = 5
|
||||||
|
|
||||||
type RegistrationHandler = func(tx *types.Transaction)
|
type RegistrationHandler = func(tx *types.Transaction)
|
||||||
|
|
||||||
type SpamHandler = func(message *pb.WakuMessage) error
|
type SpamHandler = func(msg *pb.WakuMessage, topic string) error
|
||||||
|
|
||||||
func toRLNSignal(wakuMessage *pb.WakuMessage) []byte {
|
func toRLNSignal(wakuMessage *pb.WakuMessage) []byte {
|
||||||
if wakuMessage == nil {
|
if wakuMessage == nil {
|
||||||
|
|
|
@ -2,14 +2,11 @@ package rln
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/hex"
|
|
||||||
"errors"
|
"errors"
|
||||||
"math"
|
"math"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/ethereum/go-ethereum/log"
|
"github.com/ethereum/go-ethereum/log"
|
||||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
||||||
"github.com/libp2p/go-libp2p/core/peer"
|
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/waku-org/go-waku/logging"
|
"github.com/waku-org/go-waku/logging"
|
||||||
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
||||||
|
@ -17,7 +14,6 @@ import (
|
||||||
"github.com/waku-org/go-waku/waku/v2/timesource"
|
"github.com/waku-org/go-waku/waku/v2/timesource"
|
||||||
"github.com/waku-org/go-zerokit-rln/rln"
|
"github.com/waku-org/go-zerokit-rln/rln"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
proto "google.golang.org/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type WakuRLNRelay struct {
|
type WakuRLNRelay struct {
|
||||||
|
@ -218,52 +214,49 @@ func (rlnRelay *WakuRLNRelay) AppendRLNProof(msg *pb.WakuMessage, senderEpochTim
|
||||||
// Validator returns a validator for the waku messages.
|
// Validator returns a validator for the waku messages.
|
||||||
// The message validation logic is according to https://rfc.vac.dev/spec/17/
|
// The message validation logic is according to https://rfc.vac.dev/spec/17/
|
||||||
func (rlnRelay *WakuRLNRelay) Validator(
|
func (rlnRelay *WakuRLNRelay) Validator(
|
||||||
spamHandler SpamHandler) func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
spamHandler SpamHandler) func(ctx context.Context, msg *pb.WakuMessage, topic string) bool {
|
||||||
return func(ctx context.Context, peerID peer.ID, message *pubsub.Message) bool {
|
return func(ctx context.Context, msg *pb.WakuMessage, topic string) bool {
|
||||||
rlnRelay.log.Debug("rln-relay topic validator called")
|
|
||||||
|
hash := msg.Hash(topic)
|
||||||
|
|
||||||
|
log := rlnRelay.log.With(
|
||||||
|
logging.HexBytes("hash", hash),
|
||||||
|
zap.String("pubsubTopic", topic),
|
||||||
|
zap.String("contentTopic", msg.ContentTopic),
|
||||||
|
)
|
||||||
|
|
||||||
|
log.Debug("rln-relay topic validator called")
|
||||||
|
|
||||||
rlnRelay.metrics.RecordMessage()
|
rlnRelay.metrics.RecordMessage()
|
||||||
|
|
||||||
wakuMessage := &pb.WakuMessage{}
|
|
||||||
if err := proto.Unmarshal(message.Data, wakuMessage); err != nil {
|
|
||||||
rlnRelay.log.Debug("could not unmarshal message")
|
|
||||||
return true
|
|
||||||
}
|
|
||||||
|
|
||||||
// validate the message
|
// validate the message
|
||||||
validationRes, err := rlnRelay.ValidateMessage(wakuMessage, nil)
|
validationRes, err := rlnRelay.ValidateMessage(msg, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
rlnRelay.log.Debug("validating message", zap.Error(err))
|
log.Debug("validating message", zap.Error(err))
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
switch validationRes {
|
switch validationRes {
|
||||||
case validMessage:
|
case validMessage:
|
||||||
rlnRelay.log.Debug("message verified",
|
log.Debug("message verified")
|
||||||
zap.String("id", hex.EncodeToString([]byte(message.ID))),
|
|
||||||
)
|
|
||||||
return true
|
return true
|
||||||
case invalidMessage:
|
case invalidMessage:
|
||||||
rlnRelay.log.Debug("message could not be verified",
|
log.Debug("message could not be verified")
|
||||||
zap.String("id", hex.EncodeToString([]byte(message.ID))),
|
|
||||||
)
|
|
||||||
return false
|
return false
|
||||||
case spamMessage:
|
case spamMessage:
|
||||||
rlnRelay.log.Debug("spam message found",
|
log.Debug("spam message found")
|
||||||
zap.String("id", hex.EncodeToString([]byte(message.ID))),
|
|
||||||
)
|
|
||||||
|
|
||||||
rlnRelay.metrics.RecordSpam(wakuMessage.ContentTopic)
|
rlnRelay.metrics.RecordSpam(msg.ContentTopic)
|
||||||
|
|
||||||
if spamHandler != nil {
|
if spamHandler != nil {
|
||||||
if err := spamHandler(wakuMessage); err != nil {
|
if err := spamHandler(msg, topic); err != nil {
|
||||||
rlnRelay.log.Error("executing spam handler", zap.Error(err))
|
log.Error("executing spam handler", zap.Error(err))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return false
|
return false
|
||||||
default:
|
default:
|
||||||
rlnRelay.log.Debug("unhandled validation result", zap.Int("validationResult", int(validationRes)))
|
log.Debug("unhandled validation result", zap.Int("validationResult", int(validationRes)))
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue