mirror of https://github.com/status-im/go-waku.git
refactor: filter
Moves the filter logic from wakunode2 to waku_filter
This commit is contained in:
parent
99248e9931
commit
793e7f572f
|
@ -17,7 +17,7 @@ type RelayOptions struct {
|
|||
|
||||
type FilterOptions struct {
|
||||
Enable bool `long:"filter" description:"Enable filter protocol"`
|
||||
DisableFullNode bool `long:"no-subscribers" description:"Don't accept filter subscribers"`
|
||||
DisableFullNode bool `long:"light-client" description:"Don't accept filter subscribers"`
|
||||
Nodes []string `long:"filter-node" description:"Multiaddr of a peer that supports filter protocol. Option may be repeated"`
|
||||
}
|
||||
|
||||
|
|
|
@ -2,7 +2,6 @@ package node
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
|
@ -22,10 +21,8 @@ import (
|
|||
rendezvous "github.com/status-im/go-waku-rendezvous"
|
||||
v2 "github.com/status-im/go-waku/waku/v2"
|
||||
"github.com/status-im/go-waku/waku/v2/metrics"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/filter"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/lightpush"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/relay"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/store"
|
||||
"github.com/status-im/go-waku/waku/v2/utils"
|
||||
|
@ -48,8 +45,6 @@ type WakuNode struct {
|
|||
|
||||
bcaster v2.Broadcaster
|
||||
|
||||
filters filter.Filters
|
||||
|
||||
connectionNotif ConnectionNotifier
|
||||
protocolEventSub event.Subscription
|
||||
identificationEventSub event.Subscription
|
||||
|
@ -139,11 +134,7 @@ func (w *WakuNode) Start() error {
|
|||
}
|
||||
|
||||
if w.opts.enableFilter {
|
||||
w.filters = make(filter.Filters)
|
||||
err := w.mountFilter()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
w.filter = filter.NewWakuFilter(w.ctx, w.host, w.opts.isFilterFullNode)
|
||||
}
|
||||
|
||||
if w.opts.enableRendezvous {
|
||||
|
@ -201,10 +192,6 @@ func (w *WakuNode) Stop() {
|
|||
|
||||
if w.filter != nil {
|
||||
w.filter.Stop()
|
||||
for _, filter := range w.filters {
|
||||
close(filter.Chan)
|
||||
}
|
||||
w.filters = nil
|
||||
}
|
||||
|
||||
w.relay.Stop()
|
||||
|
@ -266,18 +253,6 @@ func (w *WakuNode) mountRelay(opts ...pubsub.Option) error {
|
|||
return err
|
||||
}
|
||||
|
||||
func (w *WakuNode) mountFilter() error {
|
||||
filterHandler := func(requestId string, msg pb.MessagePush) {
|
||||
for _, message := range msg.Messages {
|
||||
w.filters.Notify(message, requestId) // Trigger filter handlers on a light node
|
||||
}
|
||||
}
|
||||
|
||||
w.filter = filter.NewWakuFilter(w.ctx, w.host, w.opts.isFilterFullNode, filterHandler)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *WakuNode) mountRendezvous() error {
|
||||
w.rendezvous = rendezvous.NewRendezvousService(w.host, w.opts.rendevousStorage)
|
||||
|
||||
|
@ -344,113 +319,6 @@ func (w *WakuNode) AddPeer(address ma.Multiaddr, protocolID p2pproto.ID) (*peer.
|
|||
return &info.ID, w.addPeer(info, protocolID)
|
||||
}
|
||||
|
||||
// Wrapper around WakuFilter.Subscribe
|
||||
// that adds a Filter object to node.filters
|
||||
func (w *WakuNode) SubscribeFilter(ctx context.Context, f filter.ContentFilter) (filterID string, ch chan *protocol.Envelope, err error) {
|
||||
if w.filter == nil {
|
||||
err = errors.New("WakuFilter is not set")
|
||||
return
|
||||
}
|
||||
|
||||
// TODO: should be possible to pass the peerID as option or autoselect peer.
|
||||
// TODO: check if there's an existing pubsub topic that uses the same peer. If so, reuse filter, and return same channel and filterID
|
||||
|
||||
// Registers for messages that match a specific filter. Triggers the handler whenever a message is received.
|
||||
// ContentFilterChan takes MessagePush structs
|
||||
subs, err := w.filter.Subscribe(ctx, f)
|
||||
if err != nil || subs.RequestID == "" {
|
||||
// Failed to subscribe
|
||||
log.Error("remote subscription to filter failed", err)
|
||||
return
|
||||
}
|
||||
|
||||
ch = make(chan *protocol.Envelope, 1024) // To avoid blocking
|
||||
|
||||
// Register handler for filter, whether remote subscription succeeded or not
|
||||
w.filters[subs.RequestID] = filter.Filter{
|
||||
PeerID: subs.Peer,
|
||||
Topic: f.Topic,
|
||||
ContentFilters: f.ContentTopics,
|
||||
Chan: ch,
|
||||
}
|
||||
|
||||
return subs.RequestID, ch, nil
|
||||
}
|
||||
|
||||
// UnsubscribeFilterByID removes a subscription to a filter node completely
|
||||
// using the filterID returned when the subscription was created
|
||||
func (w *WakuNode) UnsubscribeFilterByID(ctx context.Context, filterID string) error {
|
||||
|
||||
var f filter.Filter
|
||||
var ok bool
|
||||
if f, ok = w.filters[filterID]; !ok {
|
||||
return errors.New("filter not found")
|
||||
}
|
||||
|
||||
cf := filter.ContentFilter{
|
||||
Topic: f.Topic,
|
||||
ContentTopics: f.ContentFilters,
|
||||
}
|
||||
|
||||
err := w.filter.Unsubscribe(ctx, cf, f.PeerID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
close(f.Chan)
|
||||
delete(w.filters, filterID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Unsubscribe filter removes content topics from a filter subscription. If all
|
||||
// the contentTopics are removed the subscription is dropped completely
|
||||
func (w *WakuNode) UnsubscribeFilter(ctx context.Context, cf filter.ContentFilter) error {
|
||||
// Remove local filter
|
||||
var idsToRemove []string
|
||||
for id, f := range w.filters {
|
||||
if f.Topic != cf.Topic {
|
||||
continue
|
||||
}
|
||||
|
||||
// Send message to full node in order to unsubscribe
|
||||
err := w.filter.Unsubscribe(ctx, cf, f.PeerID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Iterate filter entries to remove matching content topics
|
||||
// make sure we delete the content filter
|
||||
// if no more topics are left
|
||||
for _, cfToDelete := range cf.ContentTopics {
|
||||
for i, cf := range f.ContentFilters {
|
||||
if cf == cfToDelete {
|
||||
l := len(f.ContentFilters) - 1
|
||||
f.ContentFilters[l], f.ContentFilters[i] = f.ContentFilters[i], f.ContentFilters[l]
|
||||
f.ContentFilters = f.ContentFilters[:l]
|
||||
break
|
||||
}
|
||||
|
||||
}
|
||||
if len(f.ContentFilters) == 0 {
|
||||
idsToRemove = append(idsToRemove, id)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for _, rId := range idsToRemove {
|
||||
for id := range w.filters {
|
||||
if id == rId {
|
||||
close(w.filters[id].Chan)
|
||||
delete(w.filters, id)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *WakuNode) DialPeerWithMultiAddress(ctx context.Context, address ma.Multiaddr) error {
|
||||
info, err := peer.AddrInfoFromP2pAddr(address)
|
||||
if err != nil {
|
||||
|
|
|
@ -47,7 +47,7 @@ type (
|
|||
ContentTopics []string
|
||||
}
|
||||
|
||||
// @TODO MAYBE MORE INFO?
|
||||
// TODO: MAYBE MORE INFO?
|
||||
Filters map[string]Filter
|
||||
|
||||
Subscriber struct {
|
||||
|
@ -61,14 +61,14 @@ type (
|
|||
Peer peer.ID
|
||||
}
|
||||
|
||||
MessagePushHandler func(requestId string, msg pb.MessagePush)
|
||||
|
||||
WakuFilter struct {
|
||||
ctx context.Context
|
||||
h host.Host
|
||||
isFullNode bool
|
||||
pushHandler MessagePushHandler
|
||||
MsgC chan *protocol.Envelope
|
||||
ctx context.Context
|
||||
h host.Host
|
||||
isFullNode bool
|
||||
MsgC chan *protocol.Envelope
|
||||
|
||||
filtersMutex sync.RWMutex
|
||||
filters Filters
|
||||
|
||||
subscriberMutex sync.Mutex
|
||||
subscribers []Subscriber
|
||||
|
@ -145,7 +145,11 @@ func (wf *WakuFilter) onRequest(s network.Stream) {
|
|||
if filterRPCRequest.Push != nil && len(filterRPCRequest.Push.Messages) > 0 {
|
||||
// We're on a light node.
|
||||
// This is a message push coming from a full node.
|
||||
wf.pushHandler(filterRPCRequest.RequestId, *filterRPCRequest.Push)
|
||||
for _, message := range filterRPCRequest.Push.Messages {
|
||||
wf.filtersMutex.RLock()
|
||||
wf.filters.Notify(message, filterRPCRequest.RequestId) // Trigger filter handlers on a light node
|
||||
wf.filtersMutex.RUnlock()
|
||||
}
|
||||
|
||||
log.Info("filter light node, received a message push. ", len(filterRPCRequest.Push.Messages), " messages")
|
||||
stats.Record(wf.ctx, metrics.Messages.M(int64(len(filterRPCRequest.Push.Messages))))
|
||||
|
@ -212,7 +216,7 @@ func (wf *WakuFilter) onRequest(s network.Stream) {
|
|||
}
|
||||
}
|
||||
|
||||
func NewWakuFilter(ctx context.Context, host host.Host, isFullNode bool, handler MessagePushHandler) *WakuFilter {
|
||||
func NewWakuFilter(ctx context.Context, host host.Host, isFullNode bool) *WakuFilter {
|
||||
ctx, err := tag.New(ctx, tag.Insert(metrics.KeyType, "filter"))
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
|
@ -222,8 +226,8 @@ func NewWakuFilter(ctx context.Context, host host.Host, isFullNode bool, handler
|
|||
wf.ctx = ctx
|
||||
wf.MsgC = make(chan *protocol.Envelope)
|
||||
wf.h = host
|
||||
wf.pushHandler = handler
|
||||
wf.isFullNode = isFullNode
|
||||
wf.filters = make(Filters)
|
||||
|
||||
wf.h.SetStreamHandlerMatch(FilterID_v20beta1, protocol.PrefixTextMatch(string(FilterID_v20beta1)), wf.onRequest)
|
||||
go wf.FilterListener()
|
||||
|
@ -294,7 +298,7 @@ func (wf *WakuFilter) FilterListener() {
|
|||
// Having a FilterRequest struct,
|
||||
// select a peer with filter support, dial it,
|
||||
// and submit FilterRequest wrapped in FilterRPC
|
||||
func (wf *WakuFilter) Subscribe(ctx context.Context, filter ContentFilter, opts ...FilterSubscribeOption) (subscription *FilterSubscription, err error) {
|
||||
func (wf *WakuFilter) requestSubscription(ctx context.Context, filter ContentFilter, opts ...FilterSubscribeOption) (subscription *FilterSubscription, err error) {
|
||||
params := new(FilterSubscribeParameters)
|
||||
params.host = wf.h
|
||||
|
||||
|
@ -346,7 +350,7 @@ func (wf *WakuFilter) Subscribe(ctx context.Context, filter ContentFilter, opts
|
|||
return
|
||||
}
|
||||
|
||||
func (wf *WakuFilter) Unsubscribe(ctx context.Context, filter ContentFilter, peer peer.ID) error {
|
||||
func (wf *WakuFilter) Unsubscribe(ctx context.Context, contentFilter ContentFilter, peer peer.ID) error {
|
||||
conn, err := wf.h.NewStream(ctx, peer, FilterID_v20beta1)
|
||||
|
||||
if err != nil {
|
||||
|
@ -359,13 +363,13 @@ func (wf *WakuFilter) Unsubscribe(ctx context.Context, filter ContentFilter, pee
|
|||
id := protocol.GenerateRequestId()
|
||||
|
||||
var contentFilters []*pb.FilterRequest_ContentFilter
|
||||
for _, ct := range filter.ContentTopics {
|
||||
for _, ct := range contentFilter.ContentTopics {
|
||||
contentFilters = append(contentFilters, &pb.FilterRequest_ContentFilter{ContentTopic: ct})
|
||||
}
|
||||
|
||||
request := pb.FilterRequest{
|
||||
Subscribe: false,
|
||||
Topic: filter.Topic,
|
||||
Topic: contentFilter.Topic,
|
||||
ContentFilters: contentFilters,
|
||||
}
|
||||
|
||||
|
@ -381,4 +385,119 @@ func (wf *WakuFilter) Unsubscribe(ctx context.Context, filter ContentFilter, pee
|
|||
|
||||
func (wf *WakuFilter) Stop() {
|
||||
wf.h.RemoveStreamHandler(FilterID_v20beta1)
|
||||
wf.filtersMutex.Lock()
|
||||
defer wf.filtersMutex.Unlock()
|
||||
for _, filter := range wf.filters {
|
||||
close(filter.Chan)
|
||||
}
|
||||
}
|
||||
|
||||
func (wf *WakuFilter) Subscribe(ctx context.Context, f ContentFilter, opts ...FilterSubscribeOption) (filterID string, theFilter Filter, err error) {
|
||||
// TODO: check if there's an existing pubsub topic that uses the same peer. If so, reuse filter, and return same channel and filterID
|
||||
|
||||
// Registers for messages that match a specific filter. Triggers the handler whenever a message is received.
|
||||
// ContentFilterChan takes MessagePush structs
|
||||
remoteSubs, err := wf.requestSubscription(ctx, f, opts...)
|
||||
if err != nil || remoteSubs.RequestID == "" {
|
||||
// Failed to subscribe
|
||||
log.Error("remote subscription to filter failed", err)
|
||||
return
|
||||
}
|
||||
|
||||
// Register handler for filter, whether remote subscription succeeded or not
|
||||
wf.filtersMutex.Lock()
|
||||
defer wf.filtersMutex.Unlock()
|
||||
|
||||
filterID = remoteSubs.RequestID
|
||||
theFilter = Filter{
|
||||
PeerID: remoteSubs.Peer,
|
||||
Topic: f.Topic,
|
||||
ContentFilters: f.ContentTopics,
|
||||
Chan: make(chan *protocol.Envelope, 1024), // To avoid blocking
|
||||
}
|
||||
|
||||
wf.filters[filterID] = theFilter
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// UnsubscribeFilterByID removes a subscription to a filter node completely
|
||||
// using the filterID returned when the subscription was created
|
||||
func (wf *WakuFilter) UnsubscribeFilterByID(ctx context.Context, filterID string) error {
|
||||
|
||||
var f Filter
|
||||
var ok bool
|
||||
|
||||
wf.filtersMutex.Lock()
|
||||
defer wf.filtersMutex.Unlock()
|
||||
|
||||
if f, ok = wf.filters[filterID]; !ok {
|
||||
return errors.New("filter not found")
|
||||
}
|
||||
|
||||
cf := ContentFilter{
|
||||
Topic: f.Topic,
|
||||
ContentTopics: f.ContentFilters,
|
||||
}
|
||||
|
||||
err := wf.Unsubscribe(ctx, cf, f.PeerID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
close(f.Chan)
|
||||
delete(wf.filters, filterID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Unsubscribe filter removes content topics from a filter subscription. If all
|
||||
// the contentTopics are removed the subscription is dropped completely
|
||||
func (wf *WakuFilter) UnsubscribeFilter(ctx context.Context, cf ContentFilter) error {
|
||||
wf.filtersMutex.Lock()
|
||||
defer wf.filtersMutex.Unlock()
|
||||
|
||||
// Remove local filter
|
||||
var idsToRemove []string
|
||||
for id, f := range wf.filters {
|
||||
if f.Topic != cf.Topic {
|
||||
continue
|
||||
}
|
||||
|
||||
// Send message to full node in order to unsubscribe
|
||||
err := wf.Unsubscribe(ctx, cf, f.PeerID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Iterate filter entries to remove matching content topics
|
||||
// make sure we delete the content filter
|
||||
// if no more topics are left
|
||||
for _, cfToDelete := range cf.ContentTopics {
|
||||
for i, cf := range f.ContentFilters {
|
||||
if cf == cfToDelete {
|
||||
l := len(f.ContentFilters) - 1
|
||||
f.ContentFilters[l], f.ContentFilters[i] = f.ContentFilters[i], f.ContentFilters[l]
|
||||
f.ContentFilters = f.ContentFilters[:l]
|
||||
break
|
||||
}
|
||||
|
||||
}
|
||||
if len(f.ContentFilters) == 0 {
|
||||
idsToRemove = append(idsToRemove, id)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for _, rId := range idsToRemove {
|
||||
for id := range wf.filters {
|
||||
if id == rId {
|
||||
close(wf.filters[id].Chan)
|
||||
delete(wf.filters, id)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -11,8 +11,6 @@ import (
|
|||
"github.com/libp2p/go-libp2p-core/peerstore"
|
||||
"github.com/status-im/go-waku/tests"
|
||||
v2 "github.com/status-im/go-waku/waku/v2"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/relay"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
@ -33,20 +31,14 @@ func makeWakuRelay(t *testing.T, topic relay.Topic, broadcaster v2.Broadcaster)
|
|||
return relay, sub, host
|
||||
}
|
||||
|
||||
func makeWakuFilter(t *testing.T, filters Filters) (*WakuFilter, host.Host) {
|
||||
func makeWakuFilter(t *testing.T) (*WakuFilter, host.Host) {
|
||||
port, err := tests.FindFreePort(t, "", 5)
|
||||
require.NoError(t, err)
|
||||
|
||||
host, err := tests.MakeHost(context.Background(), port, rand.Reader)
|
||||
require.NoError(t, err)
|
||||
|
||||
filterHandler := func(requestId string, msg pb.MessagePush) {
|
||||
for _, message := range msg.Messages {
|
||||
filters.Notify(message, requestId)
|
||||
}
|
||||
}
|
||||
|
||||
filter := NewWakuFilter(context.Background(), host, false, filterHandler)
|
||||
filter := NewWakuFilter(context.Background(), host, false)
|
||||
|
||||
return filter, host
|
||||
}
|
||||
|
@ -65,24 +57,18 @@ func TestWakuFilter(t *testing.T) {
|
|||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) // Test can't exceed 10 seconds
|
||||
defer cancel()
|
||||
|
||||
var filters = make(Filters)
|
||||
var testTopic relay.Topic = "/waku/2/go/filter/test"
|
||||
testContentTopic := "TopicA"
|
||||
|
||||
node1, host1 := makeWakuFilter(t, filters)
|
||||
node1, host1 := makeWakuFilter(t)
|
||||
defer node1.Stop()
|
||||
|
||||
broadcaster := v2.NewBroadcaster(10)
|
||||
node2, sub2, host2 := makeWakuRelay(t, testTopic, broadcaster)
|
||||
defer node2.Stop()
|
||||
defer sub2.Unsubscribe()
|
||||
filterHandler := func(requestId string, msg pb.MessagePush) {
|
||||
for _, message := range msg.Messages {
|
||||
filters.Notify(message, requestId)
|
||||
}
|
||||
}
|
||||
|
||||
node2Filter := NewWakuFilter(ctx, host2, true, filterHandler)
|
||||
node2Filter := NewWakuFilter(ctx, host2, true)
|
||||
broadcaster.Register(node2Filter.MsgC)
|
||||
|
||||
host1.Peerstore().AddAddr(host2.ID(), tests.GetHostAddress(host2), peerstore.PermanentAddrTTL)
|
||||
|
@ -93,25 +79,19 @@ func TestWakuFilter(t *testing.T) {
|
|||
Topic: string(testTopic),
|
||||
ContentTopics: []string{testContentTopic},
|
||||
}
|
||||
sub, err := node1.Subscribe(ctx, *contentFilter, WithPeer(node2Filter.h.ID()))
|
||||
|
||||
_, f, err := node1.Subscribe(ctx, *contentFilter, WithPeer(node2Filter.h.ID()))
|
||||
require.NoError(t, err)
|
||||
|
||||
// Sleep to make sure the filter is subscribed
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
ch := make(chan *protocol.Envelope, 1024)
|
||||
filters[sub.RequestID] = Filter{
|
||||
PeerID: sub.Peer,
|
||||
Topic: contentFilter.Topic,
|
||||
ContentFilters: contentFilter.ContentTopics,
|
||||
Chan: ch,
|
||||
}
|
||||
var wg sync.WaitGroup
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
env := <-ch
|
||||
env := <-f.Chan
|
||||
require.Equal(t, contentFilter.ContentTopics[0], env.Message().GetContentTopic())
|
||||
}()
|
||||
|
||||
|
@ -123,7 +103,7 @@ func TestWakuFilter(t *testing.T) {
|
|||
wg.Add(1)
|
||||
go func() {
|
||||
select {
|
||||
case <-ch:
|
||||
case <-f.Chan:
|
||||
require.Fail(t, "should not receive another message")
|
||||
case <-time.After(1 * time.Second):
|
||||
defer wg.Done()
|
||||
|
@ -140,7 +120,7 @@ func TestWakuFilter(t *testing.T) {
|
|||
wg.Add(1)
|
||||
go func() {
|
||||
select {
|
||||
case <-ch:
|
||||
case <-f.Chan:
|
||||
require.Fail(t, "should not receive another message")
|
||||
case <-time.After(1 * time.Second):
|
||||
defer wg.Done()
|
||||
|
|
Loading…
Reference in New Issue